From c90c84f700bf35cd7f57a43304d95cdf2dac8a77 Mon Sep 17 00:00:00 2001 From: Bukhtawar Khan Date: Sun, 26 May 2019 18:03:49 +0530 Subject: [PATCH 1/9] Auto-release of read-only-allow-delete block when disk utilization falls below the low watermark. Relates to #39334 --- .../modules/cluster/disk_allocator.asciidoc | 6 +- .../allocation/DiskThresholdMonitor.java | 69 +++++++- .../allocation/DiskThresholdSettings.java | 18 +++ .../allocation/DiskThresholdMonitorTests.java | 150 ++++++++++++++---- .../DiskThresholdSettingsTests.java | 3 + .../allocation/decider/MockDiskUsagesIT.java | 95 +++++++++++ 6 files changed, 300 insertions(+), 41 deletions(-) diff --git a/docs/reference/modules/cluster/disk_allocator.asciidoc b/docs/reference/modules/cluster/disk_allocator.asciidoc index 5daa50d9c25b0..c0cc934734d6b 100644 --- a/docs/reference/modules/cluster/disk_allocator.asciidoc +++ b/docs/reference/modules/cluster/disk_allocator.asciidoc @@ -40,8 +40,10 @@ Elasticsearch enforces a read-only index block (`index.blocks.read_only_allow_delete`) on every index that has one or more shards allocated on the node that has at least one disk exceeding the flood stage. This is a last resort to prevent nodes from running out of disk space. -The index block must be released manually once there is enough disk space -available to allow indexing operations to continue. +The index block is automatically released once the disk utilization falls below +the high watermark. +The automatic release can however be disabled in 7.x through a system property +`es.disk.auto_release_flood_stage_block` NOTE: You can not mix the usage of percentage values and byte values within these settings. Either all are set to percentage values, or all are set to byte diff --git a/server/src/main/java/org/elasticsearch/cluster/routing/allocation/DiskThresholdMonitor.java b/server/src/main/java/org/elasticsearch/cluster/routing/allocation/DiskThresholdMonitor.java index 4badab5a0cafa..a1a30361ec063 100644 --- a/server/src/main/java/org/elasticsearch/cluster/routing/allocation/DiskThresholdMonitor.java +++ b/server/src/main/java/org/elasticsearch/cluster/routing/allocation/DiskThresholdMonitor.java @@ -19,9 +19,12 @@ package org.elasticsearch.cluster.routing.allocation; +import java.util.HashMap; import java.util.HashSet; +import java.util.Map; import java.util.Set; import java.util.function.Supplier; +import java.util.stream.Collectors; import com.carrotsearch.hppc.ObjectLookupContainer; import com.carrotsearch.hppc.cursors.ObjectObjectCursor; @@ -34,9 +37,11 @@ import org.elasticsearch.cluster.block.ClusterBlockLevel; import org.elasticsearch.cluster.metadata.IndexMetaData; import org.elasticsearch.cluster.routing.RoutingNode; +import org.elasticsearch.cluster.routing.RoutingNodes; import org.elasticsearch.cluster.routing.ShardRouting; import org.elasticsearch.common.Strings; import org.elasticsearch.common.collect.ImmutableOpenMap; +import org.elasticsearch.common.logging.DeprecationLogger; import org.elasticsearch.common.settings.ClusterSettings; import org.elasticsearch.common.settings.Settings; import org.elasticsearch.common.util.set.Sets; @@ -55,6 +60,7 @@ public class DiskThresholdMonitor { private final Set nodeHasPassedWatermark = Sets.newConcurrentHashSet(); private final Supplier clusterStateSupplier; private long lastRunNS; + private final DeprecationLogger deprecationLogger = new DeprecationLogger(logger); public DiskThresholdMonitor(Settings settings, Supplier clusterStateSupplier, ClusterSettings clusterSettings, Client client) { @@ -109,13 +115,24 @@ public void onNewInfo(ClusterInfo info) { } ClusterState state = clusterStateSupplier.get(); Set indicesToMarkReadOnly = new HashSet<>(); + RoutingNodes routingNodes = state.getRoutingNodes(); + Map indexAutoReleaseEligibility = new HashMap<>(); + // Ensure we release indices on nodes that have a usage response from node stats + markNodesMissingUsageIneligibleForRelease(routingNodes, usages, indexAutoReleaseEligibility); for (ObjectObjectCursor entry : usages) { String node = entry.key; DiskUsage usage = entry.value; warnAboutDiskIfNeeded(usage); + RoutingNode routingNode = state.getRoutingNodes().node(node); + // Only unblock index if all nodes that contain shards of it are below the high disk watermark + if (usage.getFreeBytes() < diskThresholdSettings.getFreeBytesThresholdHigh().getBytes() + || usage.getFreeDiskAsPercentage() < diskThresholdSettings.getFreeDiskThresholdHigh()) { + markEligiblityForAutoRelease(routingNode, indexAutoReleaseEligibility, false); + } else { + markEligiblityForAutoRelease(routingNode, indexAutoReleaseEligibility, true); + } if (usage.getFreeBytes() < diskThresholdSettings.getFreeBytesThresholdFloodStage().getBytes() || usage.getFreeDiskAsPercentage() < diskThresholdSettings.getFreeDiskThresholdFloodStage()) { - RoutingNode routingNode = state.getRoutingNodes().node(node); if (routingNode != null) { // this might happen if we haven't got the full cluster-state yet?! for (ShardRouting routing : routingNode) { indicesToMarkReadOnly.add(routing.index().getName()); @@ -159,17 +176,59 @@ public void onNewInfo(ClusterInfo info) { logger.info("rerouting shards: [{}]", explanation); reroute(); } + + // Get set of indices that are eligible to be automatically unblocked + // Only collect indices that are currently blocked + Set indicesToAutoRelease = indexAutoReleaseEligibility.entrySet().stream() + .filter(Map.Entry::getValue) + .map(Map.Entry::getKey) + .filter(index -> state.getBlocks().hasIndexBlock(index, IndexMetaData.INDEX_READ_ONLY_ALLOW_DELETE_BLOCK)) + .collect(Collectors.toCollection(HashSet::new)); + + if (indicesToAutoRelease.isEmpty() == false) { + if (diskThresholdSettings.isAutoReleaseIndexEnabled()) { + logger.info("Releasing read-only allow delete block on indices: [{}]", indicesToAutoRelease); + updateIndicesReadOnly(indicesToAutoRelease, false); + } else { + deprecationLogger.deprecated("es.disk.auto_release_flood_stage_block will be removed in 8.0.0"); + } + } indicesToMarkReadOnly.removeIf(index -> state.getBlocks().indexBlocked(ClusterBlockLevel.WRITE, index)); if (indicesToMarkReadOnly.isEmpty() == false) { - markIndicesReadOnly(indicesToMarkReadOnly); + updateIndicesReadOnly(indicesToMarkReadOnly, true); + } + } + } + + + private void markNodesMissingUsageIneligibleForRelease(RoutingNodes routingNodes, ImmutableOpenMap usages, + Map indexAutoReleaseEligibility) { + if (routingNodes.size() != usages.size()) { + for (RoutingNode routingNode : routingNodes) { + if (!usages.keys().contains(routingNode.nodeId())) { + markEligiblityForAutoRelease(routingNode, indexAutoReleaseEligibility, false); + } + } + } + + } + + private void markEligiblityForAutoRelease(RoutingNode routingNode, Map indexAutoReleaseEligibility, + boolean eligible) { + if (routingNode != null) { + for (ShardRouting routing : routingNode) { + String indexName = routing.index().getName(); + boolean value = indexAutoReleaseEligibility.getOrDefault(indexName, true); + indexAutoReleaseEligibility.put(indexName, value && eligible); } } } - protected void markIndicesReadOnly(Set indicesToMarkReadOnly) { + protected void updateIndicesReadOnly(Set indicesToUpdate, boolean readOnly) { // set read-only block but don't block on the response - client.admin().indices().prepareUpdateSettings(indicesToMarkReadOnly.toArray(Strings.EMPTY_ARRAY)). - setSettings(Settings.builder().put(IndexMetaData.SETTING_READ_ONLY_ALLOW_DELETE, true).build()).execute(); + String value = readOnly ? Boolean.TRUE.toString() : null; + client.admin().indices().prepareUpdateSettings(indicesToUpdate.toArray(Strings.EMPTY_ARRAY)). + setSettings(Settings.builder().put(IndexMetaData.SETTING_READ_ONLY_ALLOW_DELETE, value).build()).execute(); } protected void reroute() { diff --git a/server/src/main/java/org/elasticsearch/cluster/routing/allocation/DiskThresholdSettings.java b/server/src/main/java/org/elasticsearch/cluster/routing/allocation/DiskThresholdSettings.java index b8d234e9f1086..d71e764f6c022 100644 --- a/server/src/main/java/org/elasticsearch/cluster/routing/allocation/DiskThresholdSettings.java +++ b/server/src/main/java/org/elasticsearch/cluster/routing/allocation/DiskThresholdSettings.java @@ -72,6 +72,20 @@ public class DiskThresholdSettings { private volatile TimeValue rerouteInterval; private volatile Double freeDiskThresholdFloodStage; private volatile ByteSizeValue freeBytesThresholdFloodStage; + private static boolean autoReleaseIndexEnabled; + + static { + final String AUTO_RELEASE_INDEX_ENABLED_KEY = "es.disk.auto_release_flood_stage_block"; + final String property = System.getProperty(AUTO_RELEASE_INDEX_ENABLED_KEY); + if (property == null) { + autoReleaseIndexEnabled = true; + } else if (Boolean.FALSE.toString().equals(property)){ + autoReleaseIndexEnabled = false; + } else { + throw new IllegalArgumentException(AUTO_RELEASE_INDEX_ENABLED_KEY + " may only be unset or set to [false] but was [" + + property + "]"); + } + } public DiskThresholdSettings(Settings settings, ClusterSettings clusterSettings) { final String lowWatermark = CLUSTER_ROUTING_ALLOCATION_LOW_DISK_WATERMARK_SETTING.get(settings); @@ -286,6 +300,10 @@ public ByteSizeValue getFreeBytesThresholdFloodStage() { return freeBytesThresholdFloodStage; } + public boolean isAutoReleaseIndexEnabled() { + return autoReleaseIndexEnabled; + } + public boolean includeRelocations() { return includeRelocations; } diff --git a/server/src/test/java/org/elasticsearch/cluster/routing/allocation/DiskThresholdMonitorTests.java b/server/src/test/java/org/elasticsearch/cluster/routing/allocation/DiskThresholdMonitorTests.java index b245b0d35d6c6..ebb6414184f2c 100644 --- a/server/src/test/java/org/elasticsearch/cluster/routing/allocation/DiskThresholdMonitorTests.java +++ b/server/src/test/java/org/elasticsearch/cluster/routing/allocation/DiskThresholdMonitorTests.java @@ -46,34 +46,12 @@ public class DiskThresholdMonitorTests extends ESAllocationTestCase { public void testMarkFloodStageIndicesReadOnly() { - AllocationService allocation = createAllocationService(Settings.builder() - .put("cluster.routing.allocation.node_concurrent_recoveries", 10).build()); Settings settings = Settings.EMPTY; - MetaData metaData = MetaData.builder() - .put(IndexMetaData.builder("test").settings(settings(Version.CURRENT) - .put("index.routing.allocation.require._id", "node2")).numberOfShards(1).numberOfReplicas(0)) - .put(IndexMetaData.builder("test_1").settings(settings(Version.CURRENT) - .put("index.routing.allocation.require._id", "node1")).numberOfShards(1).numberOfReplicas(0)) - .put(IndexMetaData.builder("test_2").settings(settings(Version.CURRENT) - .put("index.routing.allocation.require._id", "node1")).numberOfShards(1).numberOfReplicas(0)) - .build(); - RoutingTable routingTable = RoutingTable.builder() - .addAsNew(metaData.index("test")) - .addAsNew(metaData.index("test_1")) - .addAsNew(metaData.index("test_2")) - - .build(); - ClusterState clusterState = ClusterState.builder(ClusterName.CLUSTER_NAME_SETTING.getDefault(Settings.EMPTY)) - .metaData(metaData).routingTable(routingTable).build(); - logger.info("adding two nodes and performing rerouting"); - clusterState = ClusterState.builder(clusterState).nodes(DiscoveryNodes.builder().add(newNode("node1")) - .add(newNode("node2"))).build(); - clusterState = allocation.reroute(clusterState, "reroute"); - logger.info("start primary shard"); - clusterState = allocation.applyStartedShards(clusterState, clusterState.getRoutingNodes().shardsWithState(INITIALIZING)); + ClusterState clusterState = bootstrapCluster(); ClusterState finalState = clusterState; AtomicBoolean reroute = new AtomicBoolean(false); - AtomicReference> indices = new AtomicReference<>(); + AtomicReference> indicesToMarkReadOnly = new AtomicReference<>(); + AtomicReference> indicesToRelease = new AtomicReference<>(); DiskThresholdMonitor monitor = new DiskThresholdMonitor(settings, () -> finalState, new ClusterSettings(Settings.EMPTY, ClusterSettings.BUILT_IN_CLUSTER_SETTINGS), null) { @Override @@ -82,8 +60,12 @@ protected void reroute() { } @Override - protected void markIndicesReadOnly(Set indicesToMarkReadOnly) { - assertTrue(indices.compareAndSet(null, indicesToMarkReadOnly)); + protected void updateIndicesReadOnly(Set indicesToUpdate, boolean readOnly) { + if (readOnly) { + assertTrue(indicesToMarkReadOnly.compareAndSet(null, indicesToUpdate)); + } else { + assertTrue(indicesToRelease.compareAndSet(null, indicesToUpdate)); + } } }; ImmutableOpenMap.Builder builder = ImmutableOpenMap.builder(); @@ -91,15 +73,18 @@ protected void markIndicesReadOnly(Set indicesToMarkReadOnly) { builder.put("node2", new DiskUsage("node2","node2", "/foo/bar", 100, 30)); monitor.onNewInfo(new ClusterInfo(builder.build(), null, null, null)); assertFalse(reroute.get()); - assertEquals(new HashSet<>(Arrays.asList("test_1", "test_2")), indices.get()); + assertEquals(new HashSet<>(Arrays.asList("test_1", "test_2")), indicesToMarkReadOnly.get()); + assertNull(indicesToRelease.get()); - indices.set(null); + indicesToMarkReadOnly.set(null); builder = ImmutableOpenMap.builder(); builder.put("node1", new DiskUsage("node1","node1", "/foo/bar", 100, 4)); builder.put("node2", new DiskUsage("node2","node2", "/foo/bar", 100, 5)); monitor.onNewInfo(new ClusterInfo(builder.build(), null, null, null)); assertTrue(reroute.get()); - assertEquals(new HashSet<>(Arrays.asList("test_1", "test_2")), indices.get()); + assertEquals(new HashSet<>(Arrays.asList("test_1", "test_2")), indicesToMarkReadOnly.get()); + assertNull(indicesToRelease.get()); + IndexMetaData indexMetaData = IndexMetaData.builder(clusterState.metaData().index("test_2")).settings(Settings.builder() .put(clusterState.metaData() .index("test_2").getSettings()) @@ -121,18 +106,115 @@ protected void reroute() { } @Override - protected void markIndicesReadOnly(Set indicesToMarkReadOnly) { - assertTrue(indices.compareAndSet(null, indicesToMarkReadOnly)); + protected void updateIndicesReadOnly(Set indicesToUpdate, boolean readOnly) { + if (readOnly) { + assertTrue(indicesToMarkReadOnly.compareAndSet(null, indicesToUpdate)); + } else { + assertTrue(indicesToRelease.compareAndSet(null, indicesToUpdate)); + } } }; - indices.set(null); + indicesToMarkReadOnly.set(null); reroute.set(false); builder = ImmutableOpenMap.builder(); builder.put("node1", new DiskUsage("node1","node1", "/foo/bar", 100, 4)); builder.put("node2", new DiskUsage("node2","node2", "/foo/bar", 100, 5)); monitor.onNewInfo(new ClusterInfo(builder.build(), null, null, null)); assertTrue(reroute.get()); - assertEquals(new HashSet<>(Arrays.asList("test_1")), indices.get()); + assertEquals(new HashSet<>(Arrays.asList("test_1")), indicesToMarkReadOnly.get()); + assertNull(indicesToRelease.get()); + } + + public void testAutoReleaseIndices() { + Settings settings = Settings.EMPTY; + ClusterState clusterState = bootstrapCluster(); + + AtomicReference> indicesToMarkReadOnly = new AtomicReference<>(); + AtomicReference> indicesToRelease = new AtomicReference<>(); + + // Change cluster state so that "test" index is blocked (read only) + IndexMetaData indexMetaData = IndexMetaData.builder(clusterState.metaData().index("test")).settings(Settings.builder() + .put(clusterState.metaData() + .index("test").getSettings()) + .put(IndexMetaData.INDEX_BLOCKS_READ_ONLY_ALLOW_DELETE_SETTING.getKey(), true)).build(); + + final ClusterState anotherFinalClusterState = ClusterState.builder(clusterState).metaData(MetaData.builder(clusterState.metaData()) + .put(clusterState.metaData().index("test_1"), false) + .put(clusterState.metaData().index("test_2"), false) + .put(indexMetaData, true).build()) + .blocks(ClusterBlocks.builder().addBlocks(indexMetaData).build()).build(); + + assertTrue(anotherFinalClusterState.blocks().indexBlocked(ClusterBlockLevel.WRITE, "test")); + + DiskThresholdMonitor monitor = new DiskThresholdMonitor(settings, () -> anotherFinalClusterState, + new ClusterSettings(Settings.EMPTY, ClusterSettings.BUILT_IN_CLUSTER_SETTINGS), null) { + @Override + protected void updateIndicesReadOnly(Set indicesToUpdate, boolean readOnly) { + if (readOnly) { + assertTrue(indicesToMarkReadOnly.compareAndSet(null, indicesToUpdate)); + } else { + assertTrue(indicesToRelease.compareAndSet(null, indicesToUpdate)); + } + } + }; + + // When free disk on node2 goes above threshold (10% high watermark in this test case), index on node2 marked for + // auto release + ImmutableOpenMap.Builder builder = ImmutableOpenMap.builder(); + builder.put("node1", new DiskUsage("node1", "node1", "/foo/bar", 100, 4)); + builder.put("node2", new DiskUsage("node2", "node2", "/foo/bar", 100, 11)); + monitor.onNewInfo(new ClusterInfo(builder.build(), null, null, null)); + assertEquals(new HashSet<>(Arrays.asList("test_1", "test_2")), indicesToMarkReadOnly.get()); + assertEquals(new HashSet<>(Arrays.asList("test")), indicesToRelease.get()); + + // When free disk on node1 and node2 goes below threshold (10% high watermark in this test case). + // index with read only block isn't marked again to be read only and no indices to be auto-released + indicesToMarkReadOnly.set(null); + indicesToRelease.set(null); + builder = ImmutableOpenMap.builder(); + builder.put("node1", new DiskUsage("node1", "node1", "/foo/bar", 100, 4)); + builder.put("node2", new DiskUsage("node2", "node2", "/foo/bar", 100, 4)); + monitor.onNewInfo(new ClusterInfo(builder.build(), null, null, null)); + assertEquals(new HashSet<>(Arrays.asList("test_1", "test_2")), indicesToMarkReadOnly.get()); + assertNull(indicesToRelease.get()); + + // When free disk on node1 and node2 goes above threshold (10% high watermark in this test case). + // index with read only block to be auto-released + indicesToMarkReadOnly.set(null); + indicesToRelease.set(null); + builder = ImmutableOpenMap.builder(); + builder.put("node1", new DiskUsage("node1", "node1", "/foo/bar", 100, 14)); + builder.put("node2", new DiskUsage("node2", "node2", "/foo/bar", 100, 14)); + monitor.onNewInfo(new ClusterInfo(builder.build(), null, null, null)); + assertNull(indicesToMarkReadOnly.get()); + assertEquals(new HashSet<>(Arrays.asList("test")), indicesToRelease.get()); + } + + private ClusterState bootstrapCluster() { + AllocationService allocation = createAllocationService(Settings.builder() + .put("cluster.routing.allocation.node_concurrent_recoveries", 10).build()); + Settings settings = Settings.EMPTY; + MetaData metaData = MetaData.builder() + .put(IndexMetaData.builder("test").settings(settings(Version.CURRENT) + .put("index.routing.allocation.require._id", "node2")).numberOfShards(1).numberOfReplicas(0)) + .put(IndexMetaData.builder("test_1").settings(settings(Version.CURRENT) + .put("index.routing.allocation.require._id", "node1")).numberOfShards(1).numberOfReplicas(0)) + .put(IndexMetaData.builder("test_2").settings(settings(Version.CURRENT) + .put("index.routing.allocation.require._id", "node1")).numberOfShards(1).numberOfReplicas(0)) + .build(); + RoutingTable routingTable = RoutingTable.builder() + .addAsNew(metaData.index("test")) + .addAsNew(metaData.index("test_1")) + .addAsNew(metaData.index("test_2")) + .build(); + ClusterState clusterState = ClusterState.builder(ClusterName.CLUSTER_NAME_SETTING.getDefault(Settings.EMPTY)) + .metaData(metaData).routingTable(routingTable).build(); + logger.info("adding two nodes and performing rerouting"); + clusterState = ClusterState.builder(clusterState).nodes(DiscoveryNodes.builder().add(newNode("node1")) + .add(newNode("node2"))).build(); + clusterState = allocation.reroute(clusterState, "reroute"); + logger.info("start primary shard"); + return allocation.applyStartedShards(clusterState, clusterState.getRoutingNodes().shardsWithState(INITIALIZING)); } } diff --git a/server/src/test/java/org/elasticsearch/cluster/routing/allocation/DiskThresholdSettingsTests.java b/server/src/test/java/org/elasticsearch/cluster/routing/allocation/DiskThresholdSettingsTests.java index d9e157187d581..0ce9a89127901 100644 --- a/server/src/test/java/org/elasticsearch/cluster/routing/allocation/DiskThresholdSettingsTests.java +++ b/server/src/test/java/org/elasticsearch/cluster/routing/allocation/DiskThresholdSettingsTests.java @@ -45,6 +45,9 @@ public void testDefaults() { assertEquals(60L, diskThresholdSettings.getRerouteInterval().seconds()); assertTrue(diskThresholdSettings.isEnabled()); assertTrue(diskThresholdSettings.includeRelocations()); + assertEquals(zeroBytes, diskThresholdSettings.getFreeBytesThresholdFloodStage()); + assertEquals(5.0D, diskThresholdSettings.getFreeDiskThresholdFloodStage(), 0.0D); + assertTrue(diskThresholdSettings.isAutoReleaseIndexEnabled()); } public void testUpdate() { diff --git a/server/src/test/java/org/elasticsearch/cluster/routing/allocation/decider/MockDiskUsagesIT.java b/server/src/test/java/org/elasticsearch/cluster/routing/allocation/decider/MockDiskUsagesIT.java index 8565beb1b89d7..efbd1232013f9 100644 --- a/server/src/test/java/org/elasticsearch/cluster/routing/allocation/decider/MockDiskUsagesIT.java +++ b/server/src/test/java/org/elasticsearch/cluster/routing/allocation/decider/MockDiskUsagesIT.java @@ -24,6 +24,8 @@ import org.elasticsearch.cluster.ClusterState; import org.elasticsearch.cluster.DiskUsage; import org.elasticsearch.cluster.MockInternalClusterInfoService; +import org.elasticsearch.cluster.block.ClusterBlockException; +import org.elasticsearch.cluster.metadata.IndexMetaData; import org.elasticsearch.cluster.routing.RoutingNode; import org.elasticsearch.cluster.routing.allocation.DiskThresholdSettings; import org.elasticsearch.common.settings.Settings; @@ -41,6 +43,8 @@ import static org.hamcrest.Matchers.equalTo; import static org.hamcrest.Matchers.greaterThan; import static org.hamcrest.Matchers.greaterThanOrEqualTo; +import static org.elasticsearch.test.hamcrest.ElasticsearchAssertions.assertBlocked; +import static org.elasticsearch.test.hamcrest.ElasticsearchAssertions.assertSearchHits; @ESIntegTestCase.ClusterScope(scope = ESIntegTestCase.Scope.TEST, numDataNodes = 0) public class MockDiskUsagesIT extends ESIntegTestCase { @@ -131,4 +135,95 @@ public void testRerouteOccursOnDiskPassingHighWatermark() throws Exception { assertThat("node3 has at least 3 shards", nodesToShardCount.get(realNodeNames.get(2)), greaterThanOrEqualTo(3)); }); } + + public void testAutomaticReleaseOfIndexBlock() throws Exception { + List nodes = internalCluster().startNodes(3); + + // Wait for all 3 nodes to be up + assertBusy(() -> { + NodesStatsResponse resp = client().admin().cluster().prepareNodesStats().get(); + assertThat(resp.getNodes().size(), equalTo(3)); + }); + + // Start with all nodes at 50% usage + final MockInternalClusterInfoService cis = (MockInternalClusterInfoService) + internalCluster().getInstance(ClusterInfoService.class, internalCluster().getMasterName()); + cis.setUpdateFrequency(TimeValue.timeValueMillis(100)); + cis.onMaster(); + cis.setN1Usage(nodes.get(0), new DiskUsage(nodes.get(0), "n1", "/dev/null", 100, 50)); + cis.setN2Usage(nodes.get(1), new DiskUsage(nodes.get(1), "n2", "/dev/null", 100, 50)); + cis.setN3Usage(nodes.get(2), new DiskUsage(nodes.get(2), "n3", "/dev/null", 100, 50)); + + final boolean watermarkBytes = randomBoolean(); // we have to consistently use bytes or percentage for the disk watermark settings + client().admin().cluster().prepareUpdateSettings().setTransientSettings(Settings.builder() + .put(DiskThresholdSettings.CLUSTER_ROUTING_ALLOCATION_LOW_DISK_WATERMARK_SETTING.getKey(), watermarkBytes ? "15b" : "85%") + .put(DiskThresholdSettings.CLUSTER_ROUTING_ALLOCATION_HIGH_DISK_WATERMARK_SETTING.getKey(), watermarkBytes ? "10b" : "90%") + .put( + DiskThresholdSettings.CLUSTER_ROUTING_ALLOCATION_DISK_FLOOD_STAGE_WATERMARK_SETTING.getKey(), + watermarkBytes ? "5b" : "95%") + .put(DiskThresholdSettings.CLUSTER_ROUTING_ALLOCATION_REROUTE_INTERVAL_SETTING.getKey(), "150ms")).get(); + // Create an index with 10 shards so we can check allocation for it + prepareCreate("test").setSettings(Settings.builder() + .put("number_of_shards", 6) + .put("number_of_replicas", 0) + .put("index.routing.allocation.exclude._name", "")).get(); + ensureGreen("test"); + + // Block until the "fake" cluster info is retrieved at least once + assertBusy(() -> { + ClusterInfo info = cis.getClusterInfo(); + logger.info("--> got: {} nodes", info.getNodeLeastAvailableDiskUsages().size()); + assertThat(info.getNodeLeastAvailableDiskUsages().size(), greaterThan(0)); + }); + + final List realNodeNames = new ArrayList<>(); + ClusterStateResponse resp = client().admin().cluster().prepareState().get(); + Iterator iter = resp.getState().getRoutingNodes().iterator(); + while (iter.hasNext()) { + RoutingNode node = iter.next(); + realNodeNames.add(node.nodeId()); + logger.info("--> node {} has {} shards", + node.nodeId(), resp.getState().getRoutingNodes().node(node.nodeId()).numberOfOwningShards()); + } + + client().prepareIndex().setIndex("test").setType("doc").setId("1").setSource("foo", "bar").get(); + refresh(); + assertSearchHits(client().prepareSearch().get(), "1"); + + // Block all nodes so that re-balancing does not occur (BalancedShardsAllocator) + cis.setN1Usage(realNodeNames.get(0), new DiskUsage(nodes.get(0), "n1", "_na_", 100, 3)); + cis.setN2Usage(realNodeNames.get(1), new DiskUsage(nodes.get(1), "n2", "_na_", 100, 3)); + cis.setN3Usage(realNodeNames.get(2), new DiskUsage(nodes.get(2), "n3", "_na_", 100, 3)); + + refresh(); + // Wait until index "test" is blocked + assertBusy(() -> { + assertBlocked(client().prepareIndex().setIndex("test").setType("doc").setId("2").setSource("foo", "bar"), + IndexMetaData.INDEX_READ_ONLY_ALLOW_DELETE_BLOCK); + }); + + // Cannot add further documents + assertBlocked(client().prepareIndex().setIndex("test").setType("doc").setId("3").setSource("foo", "bar"), + IndexMetaData.INDEX_READ_ONLY_ALLOW_DELETE_BLOCK); + refresh(); + assertSearchHits(client().prepareSearch().get(), "1", "2"); + + // Update the disk usages so all nodes are back under the high and flood watermarks + cis.setN1Usage(realNodeNames.get(0), new DiskUsage(nodes.get(0), "n1", "_na_", 100, 11)); + cis.setN2Usage(realNodeNames.get(1), new DiskUsage(nodes.get(1), "n2", "_na_", 100, 11)); + cis.setN3Usage(realNodeNames.get(2), new DiskUsage(nodes.get(2), "n3", "_na_", 100, 11)); + + refresh(); + // Attempt to create a new document until DiskUsageMonitor unblocks the index + assertBusy(() -> { + try { + client().prepareIndex().setIndex("test").setType("doc").setId("3").setSource("foo", "bar").get(); + } catch (ClusterBlockException e) { + fail(); + } + }); + refresh(); + assertSearchHits(client().prepareSearch().get(), "1", "2", "3"); + } + } From 92e7d7b0edfb2afd264abfc683b0582911f191c5 Mon Sep 17 00:00:00 2001 From: Bukhtawar Khan Date: Fri, 31 May 2019 21:49:34 +0530 Subject: [PATCH 2/9] Tracking indicesToMarkIneligibleForAutoRelease instead of a Map and addressing other minor comments --- .../allocation/DiskThresholdMonitor.java | 39 ++++++++----------- .../allocation/DiskThresholdSettings.java | 2 +- 2 files changed, 17 insertions(+), 24 deletions(-) diff --git a/server/src/main/java/org/elasticsearch/cluster/routing/allocation/DiskThresholdMonitor.java b/server/src/main/java/org/elasticsearch/cluster/routing/allocation/DiskThresholdMonitor.java index a1a30361ec063..a1b1623ad83c3 100644 --- a/server/src/main/java/org/elasticsearch/cluster/routing/allocation/DiskThresholdMonitor.java +++ b/server/src/main/java/org/elasticsearch/cluster/routing/allocation/DiskThresholdMonitor.java @@ -19,9 +19,8 @@ package org.elasticsearch.cluster.routing.allocation; -import java.util.HashMap; +import java.util.Arrays; import java.util.HashSet; -import java.util.Map; import java.util.Set; import java.util.function.Supplier; import java.util.stream.Collectors; @@ -116,20 +115,18 @@ public void onNewInfo(ClusterInfo info) { ClusterState state = clusterStateSupplier.get(); Set indicesToMarkReadOnly = new HashSet<>(); RoutingNodes routingNodes = state.getRoutingNodes(); - Map indexAutoReleaseEligibility = new HashMap<>(); - // Ensure we release indices on nodes that have a usage response from node stats - markNodesMissingUsageIneligibleForRelease(routingNodes, usages, indexAutoReleaseEligibility); + Set indicesToMarkIneligibleForAutoRelease = new HashSet<>(); + //Ensure we release indices on nodes that have a usage response from node stats + markNodesMissingUsageIneligibleForRelease(routingNodes, usages, indicesToMarkIneligibleForAutoRelease); for (ObjectObjectCursor entry : usages) { String node = entry.key; DiskUsage usage = entry.value; warnAboutDiskIfNeeded(usage); - RoutingNode routingNode = state.getRoutingNodes().node(node); + RoutingNode routingNode = routingNodes.node(node); // Only unblock index if all nodes that contain shards of it are below the high disk watermark if (usage.getFreeBytes() < diskThresholdSettings.getFreeBytesThresholdHigh().getBytes() || usage.getFreeDiskAsPercentage() < diskThresholdSettings.getFreeDiskThresholdHigh()) { - markEligiblityForAutoRelease(routingNode, indexAutoReleaseEligibility, false); - } else { - markEligiblityForAutoRelease(routingNode, indexAutoReleaseEligibility, true); + markIneligiblityForAutoRelease(routingNode, indicesToMarkIneligibleForAutoRelease); } if (usage.getFreeBytes() < diskThresholdSettings.getFreeBytesThresholdFloodStage().getBytes() || usage.getFreeDiskAsPercentage() < diskThresholdSettings.getFreeDiskThresholdFloodStage()) { @@ -179,9 +176,9 @@ public void onNewInfo(ClusterInfo info) { // Get set of indices that are eligible to be automatically unblocked // Only collect indices that are currently blocked - Set indicesToAutoRelease = indexAutoReleaseEligibility.entrySet().stream() - .filter(Map.Entry::getValue) - .map(Map.Entry::getKey) + final String[] indices = state.routingTable().indicesRouting().keys().toArray(String.class); + Set indicesToAutoRelease = Arrays.stream(indices) + .filter(index -> indicesToMarkIneligibleForAutoRelease.contains(index) == false) .filter(index -> state.getBlocks().hasIndexBlock(index, IndexMetaData.INDEX_READ_ONLY_ALLOW_DELETE_BLOCK)) .collect(Collectors.toCollection(HashSet::new)); @@ -190,7 +187,7 @@ public void onNewInfo(ClusterInfo info) { logger.info("Releasing read-only allow delete block on indices: [{}]", indicesToAutoRelease); updateIndicesReadOnly(indicesToAutoRelease, false); } else { - deprecationLogger.deprecated("es.disk.auto_release_flood_stage_block will be removed in 8.0.0"); + deprecationLogger.deprecated("[{}] will be removed in 8.0.0", DiskThresholdSettings.AUTO_RELEASE_INDEX_ENABLED_KEY); } } indicesToMarkReadOnly.removeIf(index -> state.getBlocks().indexBlocked(ClusterBlockLevel.WRITE, index)); @@ -202,24 +199,20 @@ public void onNewInfo(ClusterInfo info) { private void markNodesMissingUsageIneligibleForRelease(RoutingNodes routingNodes, ImmutableOpenMap usages, - Map indexAutoReleaseEligibility) { - if (routingNodes.size() != usages.size()) { - for (RoutingNode routingNode : routingNodes) { - if (!usages.keys().contains(routingNode.nodeId())) { - markEligiblityForAutoRelease(routingNode, indexAutoReleaseEligibility, false); - } + Set indicesToMarkIneligibleForAutoRelease) { + for (RoutingNode routingNode : routingNodes) { + if (usages.containsKey(routingNode.nodeId()) == false) { + markIneligiblityForAutoRelease(routingNode, indicesToMarkIneligibleForAutoRelease); } } } - private void markEligiblityForAutoRelease(RoutingNode routingNode, Map indexAutoReleaseEligibility, - boolean eligible) { + private void markIneligiblityForAutoRelease(RoutingNode routingNode, Set indicesToMarkIneligibleForAutoRelease) { if (routingNode != null) { for (ShardRouting routing : routingNode) { String indexName = routing.index().getName(); - boolean value = indexAutoReleaseEligibility.getOrDefault(indexName, true); - indexAutoReleaseEligibility.put(indexName, value && eligible); + indicesToMarkIneligibleForAutoRelease.add(indexName); } } } diff --git a/server/src/main/java/org/elasticsearch/cluster/routing/allocation/DiskThresholdSettings.java b/server/src/main/java/org/elasticsearch/cluster/routing/allocation/DiskThresholdSettings.java index d71e764f6c022..ebd4b90996d8e 100644 --- a/server/src/main/java/org/elasticsearch/cluster/routing/allocation/DiskThresholdSettings.java +++ b/server/src/main/java/org/elasticsearch/cluster/routing/allocation/DiskThresholdSettings.java @@ -73,9 +73,9 @@ public class DiskThresholdSettings { private volatile Double freeDiskThresholdFloodStage; private volatile ByteSizeValue freeBytesThresholdFloodStage; private static boolean autoReleaseIndexEnabled; + public static final String AUTO_RELEASE_INDEX_ENABLED_KEY = "es.disk.auto_release_flood_stage_block"; static { - final String AUTO_RELEASE_INDEX_ENABLED_KEY = "es.disk.auto_release_flood_stage_block"; final String property = System.getProperty(AUTO_RELEASE_INDEX_ENABLED_KEY); if (property == null) { autoReleaseIndexEnabled = true; From ce28f9e36677486bd58b82b3fd287916645f9e09 Mon Sep 17 00:00:00 2001 From: Bukhtawar Khan Date: Sun, 26 May 2019 18:03:49 +0530 Subject: [PATCH 3/9] Auto-release of read-only-allow-delete block when disk utilization falls below the low watermark. Relates to #39334 MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit Tracking indicesToMarkIneligibleForAutoRelease instead of a Map and addressing other minor comments Unmute FullClusterRestartIT#testClosedIndices Fixed in #39566 Closes #39576 Add debug log for retention leases (#42557) We need more information to understand why CcrRetentionLeaseIT is failing. This commit adds some debug log to retention leases and enables them in CcrRetentionLeaseIT. Improve how internal representation of pipelines are updated (#42257) If a single pipeline is updated then the internal representation of all pipelines was updated. With this change, only the internal representation of the pipelines that have been modified will be updated. Prior to this change the IngestMetadata of the previous and current cluster was used to determine whether the internal representation of pipelines should be updated. If applying the previous cluster state change failed then subsequent cluster state changes that have no changes to IngestMetadata will not attempt to update the internal representation of the pipelines. This commit, changes how the IngestService updates the internal representation by keeping track of the underlying configuration and use that to detect against the new IngestMetadata whether a pipeline configuration has been changed and if so, then the internal pipeline representation will be updated. Fix RareClusterStateIT (#42430) * It looks like we might be cancelling a previous publication instead of the one triggered by the given request with a very low likelihood. * Fixed by adding a wait for no in-progress publications * Also added debug logging that would've identified this problem * Closes #36813 Update script-fields.asciidoc (#42490) Fixed typo in docker.asciidoc (#42455) Remove unused mapStringsOrdered method (#42513) Remove unused mapStringsOrdered method Dry up BlobStoreRepository#basePath Implementations (#42578) * This method is just a getter in every implementation => moved the field and concrete getter to the base class to simplify implementations Add Infrastructure to Run 3rd Party Repository Tests (#42586) * Add Infrastructure to Run 3rd Party Repository Tests * Add infrastructure to run third party repository tests using our standard JUnit infrastructure * This is a prerequisite of #42189 Add test ensure we can execute update requests in mixed cluster Relates #42596 Allocate to data-only nodes in ReopenWhileClosingIT (#42560) If all primary shards are allocated on the master node, then the verifying before close step will never interact with mock transport service. This change prefers to allocate shards on data-only nodes. Closes #39757 Reset mock transport service in CcrRetentionLeaseIT (#42600) testRetentionLeaseIsAddedIfItDisappearsWhileFollowing does not reset the mock transport service after test. Surviving transport interceptors from that test can sneaky remove retention leases and make other tests fail. Closes #39331 Closes #39509 Closes #41428 Closes #41679 Closes #41737 Closes #41756 Fixed ignoring name parameter for percolator queries (#42598) Closes #40405 [Ml Data Frame] Return bad_request on preview when config is invalid (#42447) Mute AsyncTwoPhaseIndexerTests#testStateMachine() (#42609) Relates #42084 [ML DataFrame] Use date histogram fixed_interval syntax and remove test skip Mute NodeTests (#42614) Relates #42577 Fix Incorrect Time Math in MockTransport (#42595) * Fix Incorrect Time Math in MockTransport * The timeunit here must be nanos for the current time (we even convert it accordingly in the logging) * Also, changed the log message when dumping stack traces a little to make it easier to grep for (otherwise it's the same as the message on unregister) Remove PRE_60_NODE_CHECKPOINT (#42527) This commit removes the obsolete `PRE_60_NODE_CHECKPOINT` constant for dealing with 5.x nodes' lack of sequence number support. Backported as #42531 Reset state recovery after successful recovery (#42576) The problem this commit addresses is that state recovery is not reset on a node that then becomes master with a cluster state that has a state not recovered flag in it. The situation that was observed in a failed test run of MinimumMasterNodesIT.testThreeNodesNoMasterBlock (see below) is that we have 3 master nodes (node_t0, node_t1, node_t2), two of them are shut down (node_t2 remains), when the first one comes back (renamed to node_t4) it becomes leader in term 2 and sends state (with state_not_recovered_block) to node_t2, which accepts. node_t2 becomes leader in term 3, and as it was previously leader in term1 and successfully completed state recovery, does never retry state recovery in term 3. Closes #39172 [DOCS] Escape cross-ref link comma for Asciidoctor (#42402) [DOCS] Fix API Quick Reference rollup attribute for Asciidoctor (#42403) [ML] adding delayed_data_check_config to datafeed update docs (#42095) * [ML] adding delayed_data_check_config to datafeed update docs * [DOCS] Edits delayed data configuration details Avoid loading retention leases while writing them (#42620) Resolves #41430. Validate routing commands using updated routing state (#42066) When multiple commands are called in sequence, fetch shards from mutable, up-to-date routing nodes to ensure each command's changes are visible to subsequent commands. This addresses an issue uncovered during work on #41050. remove 6.4.x version constants (#42127) relates refactoring initiative #41164. [ML Data Frame] Set DF task state when stopping (#42516) Set the state to stopped prior to persisting [DOCS] Reorg monitoring configuration for re-use (#42547) Remove suppresions for "unchecked" for hamcrest varargs methods (#41528) In hamcrest 2.1 warnings for unchecked varargs were fixed by hamcrest using @SafeVarargs for those matchers where this warning occurred. This PR is aimed to remove these annotations when Matchers.contains ,Matchers.containsInAnyOrder or Matchers.hasItems was used Remove support for chained multi-fields. (#42333) Follow-up to #41926, where we deprecated support for multi-fields within multi-fields. Addresses #41267. Lazily compute Java 8 home in reindex configuration (#42630) In the reindex from old tests we require Java 8. Today when configuring the reindex from old tests, we eagerly evalulate Java 8 home, which means that we require JAVA8_HOME to be set even if the reindex from old test tasks are not in the task graph. This is an onerous requirement if, for example, all that you want to do is build a distribution. This commit addresses this by making evaluation of Java 8 home lazy, so that it is only done and required if the reindex from old test tasks would be executed. Remove "nodes/0" folder prefix from data path (#42489) With the removal of node.max_local_storage_nodes, there is no need anymore to keep the data in subfolders indexed by a node ordinal. This commit makes it so that ES 8.0 will store data directly in $DATA_DIR instead of $DATA_DIR/nodes/$nodeOrdinal. Upon startup, Elasticsearch will check to see if there is data in the old location, and automatically move it to the new location. This automatic migration only works if $nodeOrdinal is 0, i.e., multiple node instances have not previously run on the same data path, which required for node.max_local_storage_nodes to explicitly be configured. [DOCS] Set explicit anchors for Asciidoctor (#42521) unmute 'Test url escaping with url mustache function' and bump logging (#42400) check position before and after latch (#42623) check position before and after latch [DOCS] Fix X-Pack tag for Asciidoctor (#42443) fix javadoc of SearchRequestBuilder#setTrackTotalHits (#42219) [ML Data Frame] Mute stop start test Relates to https://github.com/elastic/elasticsearch/issues/42650 Add 7.1.2 version constant. (#42643) Relates to #42635 Adjust use of Deprecated Netty API (#42613) * With the recent upgrade to Netty 4.1.36 this method became deprecated and I made the advised change to fix the deprecation Fix a callout in the field alias docs. Add explicit build flag for experimenting with test execution cacheability (#42649) * Add build flag for ignoring random test seed as task input * Fix checkstyle violations Use correct global checkpoint sync interval (#42642) A disruption test case need to use a lower checkpoint sync interval since they verify sequence numbers after the test waiting max 10 seconds for it to stabilize. Closes #42637 Removes types from SearchRequest and QueryShardContext (#42112) [ML-DataFrame] rewrite start and stop to answer with acknowledged (#42589) rewrite start and stop to answer with acknowledged fixes #42450 Added param ignore_throttled=false when indicesOptions.ignoreThrottled() is false (#42393) and fixed test RequestConvertersTests and added ignore_throttled on all request [DOCS] Set explicit anchors for TLS/SSL settings (#42524) Testclusters: convert ccr tests (#42313) un-mute ActivateWatchTests, bump up logging, and remove explicit sleeps (#42396) un-mute Watcher rolling upgrade tests and bump up logging (#42377) Fixes watcher test to remove typed api call Muting WatcherRestIT webhook url escaping test See #41172 [DOCS] Adds more monitoring tagged regions Add warning scores are floats (#42667) Allow aggregations using expressions to use _score (#42652) _score was removed from use in aggregations using expressions unintentionally when script contexts were added. This allows _score to once again be used. Refactor HLRC RequestConverters parameters to be more explicit (#42128) The existing `RequestConverters.Params` is confusing, because it wraps an underlying request object and mutations of the `Params` object actually mutate the `Request` that was used in the construction of the `Params`. This leads to a situation where we create a `RequestConverter.Params` object, mutate it, and then it appears nothing happens to it - it appears to be unused. What happens behind the scenes is that the Request object is mutated when methods on `Params` are invoked. This results in unclear, confusing code where mutating one object changes another with no obvious connection. This commit refactors `RequestConverters.Params` to be a simple helper class to produce a `Map` which must be passed explicitly to a Request object. This makes it apparent that the `Params` are actually used, and that they have an effect on the `request` object explicit and easier to understand. Co-authored-by: Ojas Gulati Propogate version in reindex from remote search (#42412) This is related to #31908. In order to use the external version in a reindex from remote request, the search request must be configured to request the version (as it is not returned by default). This commit modifies the search request to request the version. Additionally, it modifies our current reindex from remote tests to randomly use the external version_type. Fix inverted condition so we never cache rest integ tests Remove unused import Geo: Refactor libs/geo parsers (#42549) Refactors the WKT and GeoJSON parsers from an utility class into an instantiatable objects. This is a preliminary step in preparation for moving out coordinate validators from Geometry constructors. This should allow us to make validators plugable. Detect when security index is closed (#42191) If the security index is closed, it should be treated as unavailable for security purposes. Prior to 8.0 (or in a mixed cluster) a closed security index has no routing data, which would cause a NPE in the cluster change handler, and the index state would not be updated correctly. This commit fixese that problem Fix testTokenExpiry flaky test (#42585) Test was using ClockMock#rewind passing the amount of nanoseconds in order to "strip" nanos from the time value. This was intentional as the expiration time of the UserToken doesn't have nanosecond precision. However, ClockMock#rewind doesn't support nanos either, so when it's called with a TimeValue, it rewinds the clock by the TimeValue's millis instead. This was causing the clock to go enough millis before token expiration time and the test was passing. Once every few hundred times though, the TimeValue by which we attempted to rewind the clock only had nanos and no millis, so rewind moved the clock back just a few millis, but still after expiration time. This change moves the clock explicitly to the same instant as expiration, using clock.setTime and disregarding nanos. Revert "un-mute Watcher rolling upgrade tests and bump up logging (#42377)" This reverts commit 697c793dcbabf1df0351d75a3705047ac4435dca. Log leader and handshake failures by default (#42342) Today the `LeaderChecker` and `HandshakingTransportAddressConnector` do not log anything above `DEBUG` level. However there are some situations where it is appropriate for them to log at a higher level: - if the low-level handshake succeeds but the high-level one fails then this indicates a config error that the user should resolve, and the exception will help them to do so. - if leader checks fail repeatedly then we restart discovery, and the exception will help to determine what went wrong. Resolves #42153 Deprecate CommonTermsQuery and cutoff_frequency (#42619) * Deprecate CommonTermsQuery and cutoff_frequency Since the max_score optimization landed in Elasticsearch 7, the CommonTermsQuery is redundant and slower. Moreover the cutoff_frequency parameter for MatchQuery and MultiMatchQuery is redundant. Relates to #27096 Fix Class Load Order in Netty4Plugin (#42591) * Don't force the logger in the Netty4Plugin class already, at this point log4j might not be fully initialized. * The call was redundant anyway since we do the same thing in the Netty4Transport and Netty4HttpServerTransport classes already and there we do it properly after setting up log4j by initilizing the loggers * Relates #42532 [DOCS] Rewrite 'wildcard' query (#42670) [DOCS] path_hierarchy tokenizer examples (#39630) Closes #17138 Fix error with mapping in docs Fix refresh remote JWKS logic (#42662) This change ensures that: - We only attempt to refresh the remote JWKS when there is a signature related error only ( BadJWSException instead of the geric BadJOSEException ) - We do call OpenIDConnectAuthenticator#getUserClaims upon successful refresh. - We test this in OpenIdConnectAuthenticatorTests. Without this fix, when using the OpenID Connect realm with a remote JWKSet configured in `op.jwks_path`, the refresh would be triggered for most configuration errors ( i.e. wrong value for `op.issuer` ) and the kibana wouldn't get a response and timeout since `getUserClaims` wouldn't be called because `ReloadableJWKSource#reloadAsync` wouldn't call `onResponse` on the future. [ML] [Data Frame] add support for weighted_avg agg (#42646) Remove unused Gradle plugin (#42684) Remove usage of deprecated compare gradle builds plugin (#42687) * Remove usage of deprecated compare gradle builds plugin * Remove system property only used by build comparison Prevent merging nodes' data paths (#42665) Today Elasticsearch does not prevent you from reconfiguring a node's `path.data` to point to data paths that previously belonged to more than one node. There's no good reason to be able to do this, and the consequences can be quietly disastrous. Furthermore, #42489 might result in a user trying to split up a previously-shared collection of data paths by hand and there's definitely scope for mixing the paths up across nodes when doing this. This change adds a check during startup to ensure that each data path belongs to the same node. Clarify the settings around limiting nested mappings. (#42686) * Previously, we mentioned multiple times that each nested object was indexed as its own document. This is repetitive, and is also a bit confusing in the context of `index.mapping.nested_fields.limit`, as that applies to the number of distinct `nested` types in the mappings, not the number of nested objects. We now just describe the issue once at the beginning of the section, to illustrate why `nested` types can be expensive. * Reference the ongoing example to clarify the meaning of the two settings. Addresses #28363. Make hashed token ids url safe (#42651) This commit changes the way token ids are hashed so that the output is url safe without requiring encoding. This follows the pattern that we use for document ids that are autogenerated, see UUIDs and the associated classes for additional details. [DOCS] Disable Metricbeat system module (#42601) Remove SecurityClient from x-pack (#42471) This commit removes the SecurityClient class from x-pack. This client class is a relic of the transport client, which is in the process of being removed. Some tests were changed to use the high level rest client and others use a client directly without the security client wrapping it. Remove Log4j 1.2 API as a dependency (#42702) We had this as a dependency for legacy dependencies that still needed the Log4j 1.2 API. This appears to no longer be necessary, so this commit removes this artifact as a dependency. To remove this dependency, we had to fix a few places where we were accidentally relying on Log4j 1.2 instead of Log4j 2 (easy to do, since both APIs were on the compile-time classpath). Finally, we can remove our custom Netty logger factory. This was needed when we were on Log4j 1.2 and handled logging in our own unique way. When we migrated to Log4j 2 we could have dropped this dependency. However, even then Netty would still pick up Log4j 1.2 since it was on the classpath, thus the advantage to removing this as a dependency now. Remove client jar support from build (#42640) The client jars were a way for modules and plugins to produce an additional jar that contained classes for use by the transport client. This commit removes that configuration as the transport client is being removed. relates #42638 mute failing search template test (#42730) tracking issue #42664. Remove groovy client docs (#42731) The groovy client api was a wrapper around the transport client. However, it has not been published since 2.4, as it had many issues with the java security manager. This commit removes the docs from master for the groovy client. relates #42638 Fix docs typo in the certutil CSR mode (#42593) Changes the mention of `cert` to `csr`. Co-Authored-By: Alex Pang Remove transport client docs (#42483) This commit removes the transport client documentation. remove v6.5.x and v6.6.x version constants (#42130) related to refactoring initiative #41164. Log the status of security on license change (#42488) Whether security is enabled/disabled is dependent on the combination of the node settings and the cluster license. This commit adds a license state listener that logs when the license change causes security to switch state (or to be initialised). This is primarily useful for diagnosing cluster formation issues. Remove leftover transport module docs (#42734) This commit removes docs for alternate transport implementations which were removed years ago. These were missed because they have redirects masking their existsence. Add option to ObjectParser to consume unknown fields (#42491) ObjectParser has two ways of dealing with unknown fields: ignore them entirely, or throw an error. Sometimes it can be useful instead to gather up these unknown fields and record them separately, for example as arbitrary entries in a map. This commit adds the ability to specify an unknown field consumer on an ObjectParser, called with the field name and parsed value of each unknown field encountered during parsing. The public API of ObjectParser is largely unchanged, with a single new constructor method and interface definition. Return NO_INTERVALS rather than null from empty TokenStream (#42750) IntervalBuilder#analyzeText will currently return null if it is passed an empty TokenStream, which can lead to a confusing NullPointerException later on during querying. This commit changes the code to return NO_INTERVALS instead. Fixes #42587 [ML] [Data Frame] nesting group_by fields like other aggs (#42718) [ML Data Frame] Refactor stop logic (#42644) * Revert "invalid test" This reverts commit 9dd8b52c13c716918ff97e6527aaf43aefc4695d. * Testing * mend * Revert "[ML Data Frame] Mute Data Frame tests" This reverts commit 5d837fa312b0e41a77a65462667a2d92d1114567. * Call onStop and onAbort outside atomic update * Don’t update CS * Tidying up * Remove invalid test that asserted logic that has been removed * Add stopped event * Revert "Add stopped event" This reverts commit 02ba992f4818bebd838e1c7678bd2e1cc090bfab. * Adding check for STOPPED in saveState Re-enable token bwc tests (#42726) This commit re-enables token bwc tests that run as part of the rolling upgrade tests. These tests were muted while #42651 was being backported. [ML] Add Kibana application privilege to data frame admin/user roles (#42757) Data frame transforms are restricted by different roles to ML, but share the ML UI. To prevent the ML UI being hidden for users who only have the data frame admin or user role, it is necessary to add the ML Kibana application privilege to the backend data frame roles. [DOCS] Remove unneeded `ifdef::asciidoctor[]` conditionals (#42758) Several `ifdef::asciidoctor` conditionals were added so that AsciiDoc and Asciidoctor doc builds rendered consistently. With https://github.com/elastic/docs/pull/827, Elasticsearch Reference documentation migrated completely to Asciidoctor. We no longer need to support AsciiDoc so we can remove these conditionals. Resolves #41722 Remove CommonTermsQuery and cutoff_frequency param (#42654) Remove `common` query and `cutoff_frequency` parameter of `match` and `multi_match` queries. Both have already been deprecated for the next 7.x version. Closes: #37096 Clarify that inner_hits must be used to access nested fields. (#42724) This PR updates the docs for `docvalue_fields` and `stored_fields` to clarify that nested fields must be accessed through `inner_hits`. It also tweaks the nested fields documentation to make this point more visible. Addresses #23766. Remove locale-dependent string checking We were checking if an exception was caused by a specific reason "Not a directory". Alas, this reason is locale-dependent and can fail on systems that are not set to en_US.UTF-8. This commit addresses this by deriving what the locale-dependent error message would be and using that for comparison with the actual exception thrown. Closes #41689 [DOCS] Remove unneeded options from `[source,sql]` code blocks (#42759) In AsciiDoc, `subs="attributes,callouts,macros"` options were required to render `include-tagged::` in a code block. With elastic/docs#827, Elasticsearch Reference documentation migrated from AsciiDoc to Asciidoctor. In Asciidoctor, the `subs="attributes,callouts,macros"` options are no longer needed to render `include-tagged::` in a code block. This commit removes those unneeded options. Resolves #41589 address SmokeTestWatcherWithSecurityIT#testSearchInputWithInsufficientPrivileges (#42764) This commit adds busy wait and increases the interval for SmokeTestWatcherWithSecurityIT#testSearchInputWithInsufficientPrivileges. Watcher will not allow the same watch to be executed concurrently. If it finds that case, it will update the watch history with a "not_executed_already_queued" status. Given a slow machine, and 1 second interval this is possible. To address this, this commit increases the interval so the watch can fire at most 2 times with a greater interval between the executions and adds a busy wait for the expected state. While this does not gaurntee a fix, it should greatly reduce the chances of this test erroring. Remove XPackClient from x-pack (#42729) This commit removes the XPackClient class from x-pack. This class is a relic of the TransportClient and simply a wrapper around it. Calls are replaced with direct usage of a client. Additionally, the XPackRestHandler class has been removed as it only served to provide the XPackClient to implementing rest handlers. Remove MonitoringClient from x-pack (#42770) This commit removes the monitoring client from x-pack. This class is a relic of the TransportClient and was only used in a test. Use an anonymous inner class instead of lambda for UP-TO-DATE support remove v6.8.x version constant and the backcompat code that uses it (#42146) Remove Support for VERSION_CHECKPOINTS Translogs (#42782) * Closes #42699 Remove some leftover refs to minimum_master_nodes (#42700) Today `InternalTestCluster` has a few vestigial mentions of the `minimum_master_nodes` setting. This commit removes them and simplifies some of the surrounding logic. Create client-only AnalyzeRequest/AnalyzeResponse classes (#42197) This commit clones the existing AnalyzeRequest/AnalyzeResponse classes to the high-level rest client, and adjusts request converters to use these new classes. This is a prerequisite to removing the Streamable interface from the internal server version of these classes. [ML] Better detection of binary input in find_file_structure (#42707) This change helps to prevent the situation where a binary file uploaded to the find_file_structure endpoint is detected as being text in the UTF-16 character set, and then causes a large amount of CPU to be spent analysing the bogus text structure. The approach is to check the distribution of zero bytes between odd and even file positions, on the grounds that UTF-16BE or UTF16-LE would have a very skewed distribution. [Docs] Add example to reimplement stempel analyzer (#42676) Adding an example of how to re-implement the polish stempel analyzer in case a user want to modify or extend it. In order for the analyzer to be able to use polish stopwords, also registering a polish_stop filter for the stempel plugin. Closes #13150 Clarify heap setting in Docker docs (#42754) Add note in the Docker docs that even when container memory is limited, we still require specifying -Xms/-Xmx using one of the supported methods. [ML] Add a limit on line merging in find_file_structure (#42501) When analysing a semi-structured text file the find_file_structure endpoint merges lines to form multi-line messages using the assumption that the first line in each message contains the timestamp. However, if the timestamp is misdetected then this can lead to excessive numbers of lines being merged to form massive messages. This commit adds a line_merge_size_limit setting (default 10000 characters) that halts the analysis if a message bigger than this is created. This prevents significant CPU time being spent subsequently trying to determine the internal structure of the huge bogus messages. [DOCS] Adds redirect for deprecated `common` terms query (#42767) Make Connection Future Err. Handling more Resilient (#42781) * There were a number of possible (runtime-) exceptions that could be raised in the adjusted code and prevent resolving the listener * Relates #42350 Read the default pipeline for bulk upsert through an alias (#41963) This commit allows bulk upserts to correctly read the default pipeline for the concrete index that belongs to an alias. Bulk upserts are modeled differently from normal index requests such that the index request is a request inside of the update request. The update request (outer) contains the index or alias name is not part of the (inner) index request. This commit adds a secondary check against the update request (outer) if the index request (inner) does not find an alias. RollupStart endpoint should return OK if job already started (#41502) If a job is started or indexing, RollupStart should always return a success (200 OK) response since the job is, in fact, started SQL: [Docs] Fix links syntax (#42806) Fix a couple of wrong links because of the order of the anchor and the usage of backquotes. More improvements to cluster coordination docs (#42799) This commit addresses a few more frequently-asked questions: * clarifies that bootstrapping doesn't happen even after a full cluster restart. * removes the example that uses IP addresses, to try and further encourage the use of node names for bootstrapping. * clarifies that auto-bootstrapping might form different clusters on different hosts, and gives a process for starting again if this wasn't what you wanted. * adds the "do not stop half-or-more of the master-eligible nodes" slogan that was notably absent. * reformats one of the console examples to a narrower width Remove "template" field in IndexTemplateMetaData (#42099) Remove "template" field from XContent parsing in IndexTemplateMetaData Fix error with test conventions on tasks that require Docker (#42719) [ML] [Data Frame] adding and modifying auditor messages (#42722) * [ML] [Data Frame] adding and modifying auditor messages * Update DataFrameTransformTask.java Make high level rest client a fat jar (#42771) The original intention of the high level rest client was to provide a single jar. We tried this long ago, but had issues with intellij not correctly resolving internal tests that relied on the HLRC. This commit tweaks our use of the shadow plugin so we now produce a correct fat jar (minus the LLRC and server jars, which we can address later), with the module "client" dependencies included, as well as the correct pom file omitting those dependencies. relates #42638 Add Basic Date Docs to Painless (#42544) [Docs] Add note for date patterns used for index search. (#42810) Add an explanatory NOTE section to draw attention to the difference between small and capital letters used for the index date patterns. e.g.: HH vs hh, MM vs mm. Closes: #22322 [Docs] Fix reference to `boost` and `slop` params (#42803) For `multi_match` query: link `boost` param to the generic reference for query usage and `slop` to the `match_phrase` query where its usage is documented. Fixes: #40091 Remove unnecessary usage of Gradle dependency substitution rules (#42773) Don't require TLS for single node clusters (#42826) This commit removes the TLS cluster join validator. This validator existed to prevent v6.x nodes (which mandated TLS) from joining an existing cluster of v5.x nodes (which did not mandate TLS) unless the 6.x node (and by implication the 5.x nodes) was configured to use TLS. Since 7.x nodes cannot talk to 5.x nodes, this validator is no longer needed. Removing the validator solves a problem where single node clusters that were bound to local interfaces were incorrectly requiring TLS when they recovered cluster state and joined their own cluster. OIDC Guide additions (#42555) - Call out the fact that the SSL Configuration is important and offer a minimal example of configuring a custom CA for trust. - Add information about the `op.issuer` that was missing and add information about the `rp.post_logout_redirect` in the example since `op.endsession_endpoint` was already mentioned there and these two should be together - Explain that `op.jwkset_path` can be a URL. [ML] [Data Frame] Adding supported aggs in docs (#42728) * [ML] [Data Frame] Adding supported aggs in docs * [DOCS] Moves pivot to definitions list [ML][Data Frame] forcing that no ptask => STOPPED state (#42800) * [ML][Data Frame] forcing that no ptask => STOPPED state * Addressing side-effect, early exit for stop when stopped [Docs] Add to preference parameter docs (#42797) Adding notes to the existing docs about how using `preference` might increase request cache utilization but also add warning about the downsides. Closes #24278 [DOCS] Fix broken bucket script agg link Refactor control flow in TransportAnalyzeAction (#42801) The control flow in TransportAnalyzeAction is currently spread across two large methods, and is quite difficult to follow. This commit tidies things up a bit, to make it clearer when we use pre-defined analyzers and when we use custom built ones. [DOCS] Fix typo in bucket script aggregation link Fix testNoMasterActionsWriteMasterBlock (#42798) This commit performs the proper restore of network disruption. Previously disruptionScheme.stopDisrupting() was called that does not ensure that connectivity between cluster nodes is restored. The test was checking that the cluster has green status, but it was not checking that connectivity between nodes is restored. Here we switch to internalCluster().clearDisruptionScheme(true) which performs both checks before returning. Closes #39688 Change shard allocation filter property and api (#42602) The current example is not working and a bit confused. This change tries to match it with the sample of the watcher blog. NullPointerException when creating a watch with Jira action (#41922) (#42081) NullPointerException when secured_url does not use proper scheme in jira action. This commit will handle Expection and display proper message. Eclipse libs projects setup fix (#42852) Fallout from #42773 for eclipse users. Replicate aliases in cross-cluster replication (#41815) This commit adds functionality so that aliases that are manipulated on leader indices are replicated by the shard follow tasks to the follower indices. Note that we ignore write indices. This is due to the fact that follower indices do not receive direct writes so the concept is not useful. Fix version parsing in various tests (#42871) This commit fixes the version parsing in various tests. The issue here is that the parsing was relying on java.version. However, java.version can contain additional characters such as -ea for early access builds. See JEP 233: Name Syntax ------------------------------ -------------- java.version $VNUM(\-$PRE)? java.runtime.version $VSTR java.vm.version $VSTR java.specification.version $VNUM java.vm.specification.version $VNUM Instead, we want java.specification.version. Adjust BWC version on aliases replication This commit adjusts the BWC version on aliases replication after the change has been backported to 7.x (currently versioned as 7.3.0). Enable testing against JDK 13 EA builds (#40829) This commit adds JDK 13 to the CI rotation for testing. For now, we will be testing against JDK 13 EA builds. Avoid clobbering shared testcluster JAR files when installing modules (#42879) Permit API Keys on Basic License (#42787) Kibana alerting is going to be built using API Keys, and should be permitted on a basic license. This commit moves API Keys (but not Tokens) to the Basic license Relates: kibana#36836 Deduplicate alias and concrete fields in query field expansion (#42328) The full-text query parsers accept field pattern that are expanded using the mapping. Alias field are also detected during the expansion but they are not deduplicated with the concrete fields that are found from other patterns (or the same). This change ensures that we deduplicate the target fields of the full-text query parsers in order to avoid adding the same clause multiple times. Boolean queries are already able to deduplicate clauses during rewrite but since we also use DisjunctionMaxQuery it is preferable to detect these duplicates early on. Enable Parallel Deletes in Azure Repository (#42783) * Parallel deletes via private thread pool More logging in testRerouteOccursOnDiskPassingHighWatermark (#42864) This test is failing because recoveries of these empty shards are not completing in a reasonable time, but the reason for this is still obscure. This commit adds yet more logging. Relates #40174, #42424 Removes type from TermVectors APIs (#42198) Use reader attributes to control term dict memory useage (#42838) This change makes use of the reader attributes added in LUCENE-8671 to ensure that `_id` fields are always on-heap for best update performance and term dicts are generally off-heap on Read-Only engines. Closes #38390 Fix Stuck IO Thread Logging Time Precision (#42882) * The precision of the timestamps we get from the cached time thread is only 200ms by default resulting in a number of needless ~200ms slow network thread execution logs * Fixed by making the warn threshold a function of the precision of the cached time thread found in the settings Enable console audit logs for docker (#42671) Enable audit logs in docker by creating console appenders for audit loggers. also rename field @timestamp to timestamp and add field `type` with value audit The docker build contains now two log4j configuration for oss or default versions. The build now allows override the default configuration. Also changed the format of a timestamp from ISO8601 to include time zone as per this discussion https://github.com/elastic/elasticsearch/pull/36833#discussion_r244225243 closes #42666 [ML] Change dots in CSV column names to underscores (#42839) Dots in the column names cause an error in the ingest pipeline, as dots are special characters in ingest pipeline. This PR changes dots into underscores in CSV field names suggested by the ML find_file_structure endpoint _unless_ the field names are specifically overridden. The reason for allowing them in overrides is that fields that are not mentioned in the ingest pipeline can contain dots. But it's more consistent that the default behaviour is to replace them all. Fixes elastic/kibana#26800 Disable building on JDK 13 in CI This commit disables building on JDK 13 in CI. The reason for this is because Gradle is not yet ready to run on JDK 13. We could re-introduce infrastructure to enable Gralde to run on a different JDK than the build JDK, but rather than introducing such complexity we will instead wait for Gradle to be ready to run on JDK 13. Add Ability to List Child Containers to BlobContainer (#42653) * Add Ability to List Child Containers to BlobContainer * This is a prerequisite of #42189 Fix Azure Plugin Compilation Issue Fix Infinite Loops in ExceptionsHelper#unwrap (#42716) * Fix Infinite Loops in ExceptionsHelper#unwrap * Keep track of all seen exceptions and break out on loops * Closes #42340 Add custom metadata to snapshots (#41281) Adds a metadata field to snapshots which can be used to store arbitrary key-value information. This may be useful for attaching a description of why a snapshot was taken, tagging snapshots to make categorization easier, or identifying the source of automatically-created snapshots. Omit JDK sources archive from bundled JDK (#42821) Clean Up Painless Datetime Docs (#42869) This change abstracts the specific types away from the different representations of datetime as a datetime representation in code can be all kinds of different things. This defines the three most common types of datetimes as numeric, string, and complex while outlining the type most typically used for these as long, String, and ZonedDateTime, respectively. Documentation uses the definitions while examples use the types. This makes the documentation easier to consume especially for people from a non-Java background. Optimize Snapshot Finalization (#42723) * Optimize Snapshot Finalization * Delete index-N blobs and segement blobs in one single bulk delete instead of in separate ones to save RPC calls on implementations that have bulk deletes implemented * Don't fail snapshot because deleting old index-N failed, this results in needlessly logging finalization failures and makes analysis of failures harder going forward as well as incorrect index.latest blobs Make sibling pipeline agg ctor's protected (#42808) SiblingPipelineAggregator is a public interfaces, but the ctor was package-private. These should be protected so that plugin authors can extend and implement their own sibling pipeline agg. [DOCS] Adds discovery.type (#42823) Co-Authored-By: David Turner [Docs] Clarify caveats for phonetic filters replace option (#42807) The `replace` option in the phonetic token filter can have suprising side effects, e.g. such as described in #26921. This PR adds a note to be mindful about such scenarios and offers alternatives to using the `replace` option. Closes #26921 Skip installation of pre-bundled integ-test modules (#42900) Mute failing test Remove alpha/beta/rc from version constants (#42778) Prerelease qualifiers were moved outside of Version logic within Elasticsearch for 7.0.0, where they are now just an external modifier on the filename. However, they still existed inside code to support 6.x constants. Now that those constants have been removed in master, the prerelease logic can be removed. Skip shadow jar logic for javadoc and sources jars (#42904) For shadow jars, we place the original jar in a build/libs directory. This is to avoid clobbering the original jar when building the shadow jar. However, we need to skip this logic for javadoc and sources jars otherwise they would never be copied to the build/distributions directory during assembly. Use jar task name constant in BuildPlugin Rather than comparing to a raw string, this commit uses a built-in constant to refer to the jar task name. Relates #42904 Remove the transport client (#42538) This commit removes the transport client and all remaining uses in the code. Correct versions limits for snapshot metadata field (#42911) Now that the snapshot metadata field has been backported, the version restrictions used in tests and for serialization need to corrected. [ML-DataFrame] increase the scheduler interval to 10s (#42845) increases the scheduler interval to fire less frequently, namely changing it from 1s to 10s. The scheduler interval is used for retrying after an error condition. [ML-DataFrame] reduce log spam: do not trigger indexer if state is indexing or stopping (#42849) reduce log spam: do not trigger indexer if state is indexing or stopping [ML] Add earliest and latest timestamps to field stats (#42890) This change adds the earliest and latest timestamps into the field stats for fields of type "date" in the output of the ML find_file_structure endpoint. This will enable the cards for date fields in the file data visualizer in the UI to be made to look more similar to the cards for date fields in the index data visualizer in the UI. [ML] Close sample stream in find_file_structure endpoint (#42896) A static code analysis revealed that we are not closing the input stream in the find_file_structure endpoint. This actually makes no difference in practice, as the particular InputStream implementation in this case is org.elasticsearch.common.bytes.BytesReferenceStreamInput and its close() method is a no-op. However, it is good practice to close the stream anyway. Mute testEnableDisableBehaviour (#42929) [ML] [Data Frame] Adding pending task wait to the hlrc cleanup (#42907) Add a merge policy that prunes ID postings for soft-deleted but retained documents (#40741) * Add a merge policy that prunes soft-deleted postings This change adds a merge policy that drops all postings for documents that are marked as deleted. This is usually unnecessary unless soft-deletes are used with a rentention policy since otherwise a merge would remove deleted documents anyway. Yet, this merge policy prevents extreme cases where a very large number of soft-deleted documents are retained and are impacting search and update perfromance. Note, using this merge policy will remove all search capabilities for soft-deleted documents. * fix checkstyle * fix assertion * fix imports * fix compilation * add predicate to select fields to prune * only purne ID field * beef up test * roll back retention query * foo * remove redundant modifier * fix assumption about empty Terms * remove null check * Add test for the engine to check if we prune the IDs of retained docs away Mute failing testPerformActionAttrsRequestFails (#42933) [ML][Data Frame] pull state and states for indexer from index (#42856) * [ML][Data Frame] pull state and states for indexer from index * Update DataFrameTransformTask.java Revert "Add a merge policy that prunes ID postings for soft-deleted but retained documents (#40741)" This reverts commit 186b52c5738688b72543d9353539468e719fafce github messed up the commit message due to a retry. A followup commit will add this change again with a corrected commit message. Add a merge policy that prunes ID postings for soft-deleted but retained documents (#40741) This change adds a merge policy that drops all _id postings for documents that are marked as soft-deleted but retained across merges. This is usually unnecessary unless soft-deletes are used with a retention policy since otherwise a merge would remove deleted documents anyway. Yet, this merge policy prevents extreme cases where a very large number of soft-deleted documents are retained and are impacting update performance. Note, using this merge policy will remove all lookup by ID capabilities for soft-deleted documents. configure auto expand for dataframe indexes (#42924) creates the dataframe destination index with auto expand for replicas (0-1) Fix NPE when rejecting bulk updates (#42923) Single updates use a different internal code path than updates that are wrapped in a bulk request. While working on a refactoring to bring both closer together I've noticed that bulk updates were failing some of the tests that single updates passed. In particular, bulk updates cause NullPointerExceptions to be thrown and listeners not being properly notified when being rejected from the thread pool. Fix testPendingTasks (#42922) Fixes a race in the test which can be reliably reproduced by adding Thread.sleep(100) to the end of IndicesService.processPendingDeletes Closes #18747 Fix `InternalEngineTests#testPruneAwayDeletedButRetainedIds` The test failed because we had only a single document in the index that got deleted such that some assertions that expected at least one live doc failed. Relates to: #40741 [TEST] Remove unnecessary log line [DOCS] Rewrite terms query (#42889) Reindex max_docs parameter name (#41894) Previously, a reindex request had two different size specifications in the body: * Outer level, determining the maximum documents to process * Inside the source element, determining the scroll/batch size. The outer level size has now been renamed to max_docs to avoid confusion and clarify its semantics, with backwards compatibility and deprecation warnings for using size. Similarly, the size parameter has been renamed to max_docs for update/delete-by-query to keep the 3 interfaces consistent. Finally, all 3 endpoints now support max_docs in both body and URL. Relates #24344 [DOCS] Move 'Scripting' section to top-level navigation. (#42939) shrink may full copy when using multi data paths (#42913) Additional scenario for full segment copy if hard link cannot work across disks. Fix concurrent search and index delete (#42621) Changed order of listener invocation so that we notify before registering search context and notify after unregistering same. This ensures that count up/down like what we do in ShardSearchStats works. Otherwise, we risk notifying onFreeScrollContext before notifying onNewScrollContext (same for onFreeContext/onNewContext, but we currently have no assertions failing in those). Closes #28053 Wire query cache into sorting nested-filter computation (#42906) Don't use Lucene's default query cache when filtering in sort. Closes #42813 Make PR template reference supported architectures (#42919) This commit changes the GitHub PR template to refer to supported "OS and architecture" (rather than use OS) since we only accept PRs for x86_64 (and not Linux ARM, s390, etc) Relax timeout in NodeConnectionsServiceTests (#42934) Today we assert that the connection thread is blocked by the time the test gets to the barrier, but in fact this is not a valid assertion. The following `Thread.sleep()` will cause the test to fail reasonably often. ```diff diff --git a/server/src/test/java/org/elasticsearch/cluster/NodeConnectionsServiceTests.java b/server/src/test/java/org/elasticsearch/cluster/NodeConnectionsServiceTests.java index 193cde3180d..0e57211cec4 100644 --- a/server/src/test/java/org/elasticsearch/cluster/NodeConnectionsServiceTests.java +++ b/server/src/test/java/org/elasticsearch/cluster/NodeConnectionsServiceTests.java @@ -364,6 +364,7 @@ public class NodeConnectionsServiceTests extends ESTestCase { final CheckedRunnable connectionBlock = nodeConnectionBlocks.get(node); if (connectionBlock != null) { try { + Thread.sleep(50); connectionBlock.run(); } catch (Exception e) { throw new AssertionError(e); ``` This change relaxes the test to allow some time for the connection thread to hit the barrier. Fixes #40170 Improve translog corruption detection (#42744) Today we test for translog corruption by incrementing a byte by 1 somewhere in a file, and verify that this leads to a `TranslogCorruptionException`. However, we rely on _all_ corruptions leading to this exception in the `RemoveCorruptedShardDataCommand`: this command fails if a translog file corruption leads to a different kind of exception, and `EOFException` and `NegativeArraySizeException` are both possible. This commit strengthens the translog corruption detection tests by simulating the following: - a bit is flipped - all bits are cleared or set - a random value is written - the file is truncated It also makes sure that we return a `TranslogCorruptionException` in all such cases. Fixes #42661 Fix FsRepositoryTests.testSnapshotAndRestore (#42925) * The commit generation can be 3 or 2 here -> fixed by checking the actual generation on the second commit instead of hard coding 2 * Closes #42905 Only ignore IOException when fsyncing on dirs (#42972) Today in the method IOUtils#fsync we ignore IOExceptions when fsyncing a directory. However, the catch block here is too broad, for example it would be ignoring IOExceptions when we try to open a non-existant file. This commit addresses that by scoping the ignored exceptions only to the invocation of FileChannel#force. Remove Comma in Example (#41873) The comma is there in error as there are no other parameter after 'value' [ML][Data frame] make sure that fields exist when creating progress (#42943) [TEST] Adding a BWC test for ML categorization config (#42981) This test coverage was previously missing. Remove WatcherClient from x-pack (#42815) This commit removes the WatcherClient and WatcherRestHandler from the codebase. The WatcherClient was a convenience wrapper around the transport client, which is being removed so the client no longer serves a purpose. The WatcherRestHandler is no longer needed as its primary purpose was to provide a WatcherClient to the implementing handlers. Remove the CcrClient (#42816) This commit removes the CcrClient class, which is a wrapper around the transport client. The transport client is being removed so the client is no longer needed. Remove the ILMClient (#42817) This commit removes the ILMClient class, which is a wrapper around the transport client. This class is not used in the codebase and the transport client is being removed. [DOCS] Add explicit `articles_case` parameter to Elision Token Filter example (#42987) Update default shard count per index in readme (#42388) The default shard count has been reduced from 5 to 1. This commit updates the readme to reflect that changed default. [ML][Data Frame] allow null values for aggs with sparse data (#42966) * [ML][Data Frame] allow null values for aggs with sparse data * Making classes static, memory allocation optimization Drop dead code for socket permissions for transport (#42990) This code has not been needed since the removal of tribe nodes, it was left behind when those were dropped (note that regular transport permissions are handled through transport profiles, even if they are not explicitly in use). Fix possible NPE in put mapping validators (#43000) When applying put mapping validators, we apply all the validators in the collection. If a failure occurs, we collect that as a top-level exception, and suppress any additional failures into the top-level exception. However, if a request passes the validator after a top-level exception has been collected, we would try to suppress a null exception into the top-level exception. This is a violation of the Throwable#addSuppressed API. This commit addresses this, and adds test to cover the logic of collecting the failures when validating a put mapping request. Fix put mapping request validators random test This commit fixes a test bug in the request validators random test. In particular, an assertion was not properly nested in a guard that would ensure that was at least one failure. Relates #43000 Fix IOUtils#fsync on Windows fsyncing directories (#43008) Fsyncing directories on Windows is not possible. We always suppressed this by allowing that an AccessDeniedException is thrown when attemping to open the directory for reading. Yet, this suppression also allowed other IOExceptions to be suppressed, and that was a bug (e.g., the directory not existing, or a filesystem error and reasons that we might get an access denied there, like genuine permissions issues). This leniency was previously removed yet it exposed that we were suppressing this case on Windows. Rather than relying on exceptions for flow control and continuing to suppress there, we simply return early if attempting to fsync a directory on Windows (we will not put this burden on the caller). Mute testLookupSeqNoByIdInLucene Tracked at #42979 Mute AutodetectMemoryLimitIT#testTooManyPartitions Relates #43013 Fix assertion in ReadOnlyEngine (#43010) We should execute the assertion before throwing an exception; otherwise, it's a noop. Unmuted testRecoverBrokenIndexMetadata These tests should be okay as we flush at the end of peer recovery. Closes #40867 Refactor put mapping request validation for reuse (#43005) This commit refactors put mapping request validation for reuse. The concrete case that we are after here is the ability to apply effectively the same framework to indices aliases requests. This commit refactors the put mapping request validation framework to allow for that. Do not allow modify aliases on followers (#43017) Now that aliases are replicated by a follower from its leader, this commit prevents directly modifying aliases on follower indices. Adjust IndicesAliasesRequest origin BWC version The work to add the origin field to the IndicesAliasesRequest has been backported to 7.x. Since this version is currently 7.3.0, this commit adjusts the version in master accordingly. Add note to CCR docs regarding alias replication This commit adds a note to the docs regarding the automatic replication of aliases by a follower index from its leader index. Add note to CCR docs about mapping/alias updates This commit adds a note to the docs clarifying that it is not possible to manually update the mapping nor the aliases of a follower index. Unmute PermissionsIT test and enable debug logging for it (#42876) This unmutes `testWhenUserLimitedByOnlyAliasOfIndexCanWriteToIndexWhichWasRolledoverByILMPolicy` and enables DEBUG logging. The failure from this test case from a query running rather than ILM itself, so more information is needed. Relates to #41440 Since SQL is GA, remove the sql language plugin from this list (#41533) SQL: cover the Integer type when extracting values from _source (#42859) * Take into consideration a wider range of Numbers when extracting the values from source, more specifically - BigInteger and BigDecimal. Allow routing commands with ?retry_failed=true (#42658) We respect allocation deciders, including the `MaxRetryAllocationDecider`, when executing reroute commands. If you specify `?retry_failed=true` then the retry counter is reset, but today this does not happen until after trying to execute the reroute commands. This means that if an allocation has repeatedly failed, but you want to take control and assign a shard to a particular node to work around the repeated failures, you cannot execute the routing command in the same call to `POST /_cluster/reroute` as the one that resets the failure counter. This commit fixes this by resetting the failure counter first, meaning that you can now explicitly allocate a repeatedly-failed shard like this: ``` POST /_cluster/reroute?retry_failed=true { "commands": [ { "allocate_replica": { "index": "blahblah", "shard": 2, "node": "node-4" } } ] } ``` Fixes #39546 Fix auto fuzziness in query_string query (#42897) Setting `auto` after the fuzzy operator (e.g. `"query": "foo~auto"`) in the `query_string` does not take the length of the term into account when computing the distance and always use a max distance of 1. This change fixes this disrepancy by ensuring that the term is passed when the fuzziness is computed. Don't run build-tools integ tests on FIPS (#42986) These run Gradle and FIPS isn't supported Closes #41721 Fix typo in create-index.asciidoc (#41806) Update regexp-syntax.asciidoc (#43021) Corrects a typo. Update search-settings.asciidoc (#43016) Grammar and spelling fixes [ML] Re-enable integration test (#41712) Move construction of custom analyzers into AnalysisRegistry (#42940) Both TransportAnalyzeAction and CategorizationAnalyzer have logic to build custom analyzers for index-independent analysis. A lot of this code is duplicated, and it requires the AnalysisRegistry to expose a number of internal provider classes, as well as making some assumptions about when analysis components are constructed. This commit moves the build logic directly into AnalysisRegistry, reducing the registry's API surface considerably. Improve documentation for smart_cn analyzer (#42822) Correct the description of generate_word_parts (#43026) Clean up configuration when docker isn't available (#42745) We initially added `requireDocker` for a way for tasks to say that they absolutely must have it, like the build docker image tasks. Projects using the test fixtures plugin are not in this both, as the intent with these is that they will be skipped if docker and docker-compose is not available. Before this change we were lenient, the docker image build would succeed but produce nothing. The implementation was also confusing as it was not immediately obvious this was the case due to all the indirection in the code. The reason we have this leniency is that when we added the docker image build, docker was a fairly new requirement for us, and we didn't have it deployed in CI widely enough nor had CI configured to prefer workers with docker when possible. We are in a much better position now. The other reason was other stack teams running `./gradlew assemble` in their respective CI and the possibility of breaking them if docker is not installed. We have been advocating for building specific distros for some time now and I will also send out an additional notice The PR also removes the use of `requireDocker` from tests that actually use test fixtures and are ok without it, and fixes a bug in test fixtures that would cause incorrect configuration and allow some tasks to run when docker was not available and they shouldn't have. Closes #42680 and #42829 see also #42719 Better Exception in NetworkUtilsTests (#42109) * We are still running into an exception here every so often * Adjusted exception to contain interface name * Relates to #41549 Fix GCS Blob Repository 3rd Party Tests (#43030) * We have to strip the trailing slash from child names here like we do for AWS * closes #43029 [DOCS] Change `// TESTRESPONSE[_cat]` to `// TESTRESPONSE[non_json]` (#43006) [ML] Get resources action should be lenient when sort field is unmapped (#42991) Get resources action sorts on the resource id. When there are no resources at all, then it is possible the index does not contain a mapping for the resource id field. In that case, the search api fails by default. This commit adjusts the search request to ignore unmapped fields. Closes elastic/kibana#37870 Mute AzureDiscoveryClusterFormationTests (#43049) Relates #43048 Fix IpFilteringIntegrationTests (#43019) * Increase timeout to 5s since we saw 500ms+ GC pauses on CI * closes #40689 Increase waiting time when check retention locks (#42994) WriteActionsTests#testBulk and WriteActionsTests#testIndex sometimes fail with a pending retention lock. We might leak retention locks when switching to async recovery. However, it's more likely that ongoing recoveries prevent the retention lock from releasing. This change increases the waiting time when we check for no pending retention lock and also ensures no ongoing recovery in WriteActionsTests. Closes #41054 [ML][Data Frame] Removes slice specification from DBQ. See #42996 (#43036) Rename processor test fix (#43035) If the source field name is a prefix of the target field name, the source field still exists after rename processor has run. Adjusted test case to handle that case. Default distro run creates elastic-admin user (#43004) When using gradle run by itself, this uses the default distro with a basic license and enables security. There is a setup command to create a elastic-admin user but only when the license is a trial license. Now that security is available with the basic license, we should always run this command when using the default distribution. Fixing handling of auto slices in bulk scroll requests (#43050) * Fixing handling of auto slices in bulk scroll requests * adjusting assertions for tests Unmute IndexFollowingIT#testFollowIndex Fixed in #41987 Fix NPE in CcrRetentionLeaseIT (#43059) The retention leases stats is null if the processing shard copy is being closed. In this the case, we should check against null then retry to avoid failing a test. Closes #41237 [ML] Changes slice specification to auto. See #42996 (#43039) [ML] Adding support for geo_shape, geo_centroid, geo_point in datafeeds (#42969) * [ML] Adding support for geo_shape, geo_centroid, geo_point in datafeeds * only supporting doc_values for geo_point fields * moving validation into GeoPointField ctor Upgrade AWS SDK to Latest Version (#42708) * Just staying up to data on the SDK version * Use `AbstractAmazonEC2` to shorten code Better test diag output on OOM (#42989) If linearizability checking fails with OOM (or other exception), we did not get the serialized history written into the log, making it difficult to debug in cases where the problem is hard to reproduce. Fixed to always attempt dumping the serialized history. Related to #42244 Refresh remote JWKs on all errors (#42850) It turns out that key rotation on the OP, can manifest as both a BadJWSException and a BadJOSEException in nimbus-jose-jwt. As such we cannot depend on matching only BadJWSExceptions to determine if we should poll the remote JWKs for an update. This has the side-effect that a remote JWKs source will be polled exactly one additional time too for errors that have to do with configuration, or for errors that might be caused by not synched clocks, forged JWTs, etc. ( These will throw a BadJWTException which extends BadJOSEException also ) Split search in two when made against throttled and non throttled searches (#42510) When a search on some indices takes a long time, it may cause problems to other indices that are being searched as part of the same search request and being written to as well, because their search context needs to stay open for a long time. This is especially a problem when searching against throttled and non-throttled indices as part of the same request. The problem can be generalized though: this may happen whenever read-only indices are searched together with indices that are being written to. Search contexts staying open for a long time is only an issue for indices that are being written to, in practice. This commit splits the search in two sub-searches: one for read-only indices, and one for ordinary indices. This way the two don't interfere with each other. The split is done only when size is greater than 0, no scroll is provided and query_then_fetch is used as search type. Otherwise, the search executes like before. Note that the returned num_reduce_phases reflect the number of reduction phases that were run. If the search is split in two, there are three reductions: one non-final for each search, and a final one that merges the results of the previous two. Closes #40900 [DOCS] Clarify phrase suggester docs smoothing parameter (#42947) Closes #28512 remove path from rest-api-spec (#41452) SQL: Clarify that the connections the jdbc driver creates are not pooled (#42992) Restructure the SQL Language section to have proper sub-sections (#43007) Rest docs page update - have the section be on separate pages - add an Overview page - add other formats examples Increase test logging for testSyncedFlushSkipOutOfSyncReplicas Relates to #43086 Rename TESTRESPONSE[_cat] to TESTRESPONSE[non_json] (#43087) Documents the new deprecations options on the rest-api-spec (#41444) * Documents the new deprecations options on the rest-api-spec Relates #41439 #38613 #35262 * remove reference to path now that #41452 is merged, also fixed missing a comma rendering the example json invalid * removed one more instance of path * make sure json examples are self contained and not excerpts Handle empty terms index in TermsSliceQuery (#43078) field on soft deleted documents. The TermsSliceQuery assumes that every document has has an entry in the postings for that field so it doesn't check if the terms index exists or not. This change fixes this bug by checking if the terms index for the `_id` field is null and ignore the segment entirely if it's the case. This should be harmless since segments without an `_id` terms index should only contain soft deleted documents. Closes #42996 Enable TRACE for testRecoverBrokenIndexMetadata (#43081) Relates to #43034 CCR should not replicate private/internal settings (#43067) With this change, CCR will not replicate internal or private settings to follower indices. Closes #41268 Defer reroute when nodes join (#42855) Today the master eagerly reroutes the cluster as part of processing node joins. However, it is not necessary to do this reroute straight away, and it is sometimes preferable to defer it until later. For instance, when the master wins its election it processes joins and performs a reroute, but it would be better to defer the reroute until after the master has become properly established. This change defers this reroute into a separate task, and batches multiple such tasks together. Add SAS Token Authentication Support to Azure Repo Plugin (#42982) * Added setting for SAS token * Added support for the token in tests * Relates #42117 Do not use runtime Java when starting BWC nodes (#43071) When starting BWC nodes, it could be that runtime Java home is set. Yet, runtime Java home can advance beyond what a BWC node might be compatible with. For example, if runtime Java home is set to JDK 13 and we are starting a 7.1.2 node, we do not have any guarantees that 7.1.2 is compatible with JDK 13 (since we never did any work to make it so). This will continue to be the case as JDK releases advance, but we still need to test against BWC nodes. This commit stops applying runtime Java home when starting a BWC node. Instead, we would use the bundled JDK. Docs enhancement for quote_field_suffix. (#43093) * Docs enhancement for quote_field_suffix. Mentions the use of a fall-back field when specified field is missing. Closes #40778 [ML] Fix race condition when closing time checker (#43098) The tests for the ML TimeoutChecker rely on threads not being interrupted after the TimeoutChecker is closed. This change ensures this by making the close() and setTimeoutExceeded() methods synchronized so that the code inside them cannot execute simultaneously. Fixes #43097 Fix unresponsive network simulation (#42579) Unresponsive network simulation would throw away requests. However, then we no longer have any guarantees that a transport action either succeeds or fails, which could lead to hangs (example: unclosed IndexShard permits). Closes #42244 [ML][Data Frame] adding dest.index and id validations (#43053) * [ML][Data Frame] adding dest.index and id validations * adjusting message format * Adjusting id validity pattern * Update DataFrameStrings.java Remove description from xpack feature sets (#43065) The description field of xpack featuresets is optionally part of the xpack info api, when using the verbose flag. However, this information is unnecessary, as it is better left for documentation (and the existing descriptions describe anything meaningful). This commit removes the description field from feature sets. [ML-DataFrame] stop task at completion of data frame function (#42955) * stop data frame task after it finishes * test auto stop * adapt tests * persist the state correctly and move stop into listener * Calling `onStop` even if persistence fails, changing `stop` to rely on doSaveState scheduleAtFixedRate would hang (#42993) Though not in use in elasticsearch currently, it seems surprising that ThreadPool.scheduler().scheduleAtFixedRate would hang. A recurring scheduled task is never completed (except on failure) and we test for exceptions using RunnableFuture.get(), which hangs for periodic tasks. Fixed by checking that task is done before calling .get(). Remove the MachineLearningClient (#43108) The transport client has been removed and this follow up change removes the MachineLearningClient as we no longer need this wrapper class since we have a user facing MachineLearningClient within the rest high level client. Adds a minimum interval to `auto_date_histogram`. (#42814) Adds a minimum interval to `auto_date_histogram`. We do this by restricting the roundings passed into to the aggregator. Exclude internal SQL methods from Painless API Docs (#43116) Use transport actions instead of guice for xpack usage (#43011) The Xpack usage API internally gathers all xpack feature sets by using guice to bind many implementations. The usage api then iterates over these feature sets to gather usage information for each. However, since each of these implementations are in different plugins, there is no direct, non-guice way for them to register the implementations. This commit removes the dependence on guice for the usage api by having the core xpack jar contain an action instance for each plugin, and the implementation of those actions are then registered within each xpack feature plugin. The usage api then iterates over each of these actions and calls them with the NodeClient. Fix timing issue in CcrRetentionLeaseIT (#43054) In these tests, we sleep for a small multiple of the renew interval, then check that the retention leases are not changed. If a renewal request takes longer than that interval because of GC or slow CI, then the retention leases are not the same as before sleep. With this change, we relax to assert that we eventually stop the renewable process. Closes #39509 Metricbeat monitoring Elasticsearch: Reorder/remove steps (#42917) [ML] Lock down extraction method when possible (#43104) Docs change for exists query. (#43092) Now emphasises the test is for indexed values. Previous documentation only mentioned the state of the input JSON doc (null values) but this is only one of several reasons why an indexed value may not exist. Closes #24256 Ensure Test Cluster in Azure Discovery Tests Closes (#43057) * Use `internalCluster().close()` to force all nodes (and not just the datanodes) to shut down even if one fails to shut down in time * Force closing httpServer to get cleaner logs if nodes still hang on shut down * Relates #43048 Upgrade HPPC to version 0.8.1 (#43025) Mute testDataFrameTransformCrud Relates to #43139 SearchRequest#allowPartialSearchResults does not handle successful retries (#43095) When set to false, allowPartialSearchResults option does not check if the shard failures have been reseted to null. The atomic array, that is used to record shard failures, is filled with a null value if a successful request on a shard happens after a failure on a shard of another replica. In this case the atomic array is not empty but contains only null values so this shouldn't be considered as a failure since all shards are successful (some replicas have failed but the retries on another replica succeeded). This change fixes this bug by checking the content of the atomic array and fails the request only if allowPartialSearchResults is set to false and at least one shard failure is not null. Closes #40743 Fix suggestions for empty indices (#42927) Currently suggesters return null values on empty shards. Usually this gets replaced by results from other non-epmty shards, but if the index is completely epmty (e.g. after creation) the search responses "suggest" is also "null" and we don't render a corresponding output in the REST response. This is an irritating edge case that requires special handling on the user side (see #42473) and should be fixed. This change makes sure every suggester type (completion, terms, phrase) returns at least an empty skeleton suggestion output, even for empty shards. This way, even if we don't find any suggestions anywhere, we still return and output the empty suggestion. Closes #42473 [ML][Data Frame] change failure count reset logic (#43064) [ML][Data Frame] add the src priv check for view_index_metadata (#43118) [ML][Data Frame] cleaning up tests since tasks are cancelled onfinish (#43136) * [ML][Data Frame] cleaning up usage test since tasks are cancelled onfinish * Update DataFrameUsageIT.java * Fixing additional test, waiting for task to complete * removing unused import * unmuting test Move painless context api spec to test local (#43122) The painless context api is internal and currently meant only for use in generating docs. This commit moves the spec file for the api so that it is only used by the test for this api, and not externally by any clients building from the public rest spec. Fix mustache search template test (#43120) This commit fixes a race in the test for the new response format with search templates. The test indexes a document and then executes a search with the expectation of 0 results. In some instances, the index will refresh prior to the search execution and 1 hit will be found causing the test fail. Closes #42664 Remove usage of FileSwitchDirectory (#42937) We are still using `FileSwitchDirectory` in the case a user configures file based pre-load of mmaps. This is trappy for multiple reasons if the both directories used by `FileSwitchDirectory` point to the same filesystem directory. One issue is LUCENE-8835 that cause issues like #37111 - unless LUCENE-8835 isn't fixed we should not use it in elasticsearch. Instead we use a similar trick as we use for HybridFS and subclass mmap directory directly. IndexAnalyzers doesn't need to extend AbstractIndexComponent (#43149) AIC doesn't add anything here, and it removes the need to pass index settings to the constructor. Stop SeedHostsResolver on shutdown (#42844) Fixes an issue where tests would sometimes hang for 5 seconds when restarting a node. The reason is that the SeedHostsResolver is blockingly waiting on a result for the full 5 seconds when the corresponding threadpool is shut down. [ML DataFrame] Reject Data Frame Ids containing upper case characters (#43145) [DOCS] Adds dataframe authorization details (#43009) Fix Operation Timestamps in Tests (#43155) * For the issue in #43086 we were running into inactive shards because the random timestamps previously used would randomly make `org.elasticsearch.index.shard.IndexShard#checkIdle` see an incorrect+huge inactive time * Also fixed one other spot in tests that passed `ms` instead of `ns` for the same timestamp on an index op to correctly use relative `ns` * Closes #43086 [ML][Data Frame] write a warning audit on bulk index failures (#43106) * [ML][Data Frame] write a warning audit on bulk index failures * adding failure message and moving to use volalitile Also mmap terms index (`.tip`) files for hybridfs (#43150) This change adds the terms index (`.tip`) to the list of extensions that are memory-mapped by hybridfs. These files used to be accessed only once to load the terms index on-heap but since #42838 they can now be used to read the binary FST directly so it is benefical to memory-map them instead of accessing them via NIO. Introduce stability description to the REST API specification (#38413) * introduce state to the REST API specification * change state over to stability * CCR is no GA updated to stable * SQL is now GA so marked as stable * Introduce `internal` as state for API's, marks stable in terms of lifetime but unstable in terms of guarantees on its output format since it exposes internal representations * make setting a wrong stability value, or not setting it at all an error that causes the YAML test suite to fail * update spec files to be explicit about their stability state * Document the fact that stability needs to be defined Otherwise the YAML test runner will fail (with a nice exception message) * address check style violations * update rest spec unit tests to include stability * found one more test spec file not declaring stability, made sure stability appears after documentation everywhere * cluster.state is stable, mark response in some way to denote its a key value format that can be changed during minors * mark data frame API's as beta * remove internal and private as states for an API * removed the wrong enum values in the Stability Enum in the previous commit Introduce unit tests for ValuesSourceType (#43174) As the ValuesSourceType evolves, it is important to be confident that new enum constants do not break backwards-compatibility on the stream. Having dedicated unit tests for this class will help be sure of that. Enable krb5kdc-fixture, kerberos tests mount urandom for kdc container (#41710) Infra has fixed #10462 by installing `haveged` on CI workers. This commit enables the disabled fixture and tests, and mounts `/dev/urandom` for the container so there is enough entropy required for kdc. Note: hdfs-repository tests have been disabled, will raise a separate issue for it. Closes #40624 Closes #40678 Convert PluginPropertiesExtension Groovy to Java (#39605) Only load FST off heap if we are actually using mmaps for the term dictionary (#43158) Given the significant performance impact that NIOFS has when term dicts are loaded off-heap this change enforces FstLoadMode#AUTO that loads term dicts off heap only if the underlying index input indicates a memory map. Relates to #43150 Testclusters: graph (#43033) Convert x-pack graph to use testClusters Document wildcard for network interfaces (#28839) With this commit we mention how Elasticsearch behaves when either `0` or `0.0.0.0` is used for `network.host`. Report timing stats as part of the Job stats response (#42709) Remote reindex failure parse fix (#42928) A search request that partially fails with failures without an index (index: null) in the failure would cause a parse error in reindex from remote. This would hide the original exception, making it hard to debug the root cause. This commit fixes this so that we can tolerate null index entries in a search failure. Fix roundUp parsing with composite patterns (#43080) roundUp parsers were losing the composite pattern information when new JavaDateFormatter was created from methods withLocale or withZone. The roundUp parser should be preserved when calling these methods. This is the same approach in withLocale/Zone methods as in https://github.com/elastic/elasticsearch/blob/daa2ec8a605d385a65b9ab3e89d016b3fd0dffe2/server/src/main/java/org/elasticsearch/common/time/JavaDateFormatter.java closes #42835 Converting DependencyLicensesTask and UpdateShasTask to java (#41921) convert EmptyDirTask.groovy to .java (#34672) Add 6.8 to the remote clusters compatibility table (#42389) The table does not include 6.8 as it was written before we knew we were releasing it. This commit adds it. Convert ILM tests to use testclusters (#43076) Also improove the error message when bin scripts are not found [ML] Restrict detection of epoch timestamps in find_file_structure (#43188) Previously 10 digit numbers were considered candidates to be timestamps recorded as seconds since the epoch and 13 digit numbers as timestamps recorded as milliseconds since the epoch. However, this meant that we could detect these formats for numbers that would represent times far in the future. As an example ISBN numbers starting with 9 were detected as milliseconds since the epoch since they had 13 digits. This change tweaks the logic for detecting such timestamps to require that they begin with 1 or 2. This means that numbers that would represent times beyond about 2065 are no longer detected as epoch timestamps. (We can add 3 to the definition as we get closer to the cutoff date.) Refactor Version class to make version bumps easier (#42668) With this change we only have to add one line to add a new version. The intent is to make it less error prone and easier to write a script to automate the process. [DOCS] Update node descriptions for default distribution (#42812) Compile subset of build-tools classes with Java 8 target compatibility (#43177) [DOCS] Fix link to ML node description Disable x-pack:qa:kerberos-tests due to failures (#43210) relates #40678 Add native code info to ML info api (#43172) The machine learning feature of xpack has native binaries with a different commit id than the rest of code. It is currently exposed in the xpack info api. This commit adds that commit information to the ML info api, so that it may be removed from the info api. Enable node roles to be pluggable (#43175) This commit introduces the possibility for a plugin to introduce additional node roles. Allow big integers and decimals to be mapped dynamically. (#42827) This PR proposes to model big integers as longs (and big decimals as doubles) in the context of dynamic mappings. Previously, the dynamic mapping logic did not recognize big integers or decimals, and would an error of the form "No matching token for number_type [BIG_INTEGER]" when a dynamic big integer was encountered. It now accepts these numeric types and interprets them as 'long' and 'double' respectively. This allows `dynamic_templates` to accept and and remap them as another type such as `keyword` or `scaled_float`. Addresses #37846. Adjust BWC version for discovery node roles Now that pluggable node roles are backported to 7.x (versioned at 7.3.0), this commit adjusts the BWC version in master accordingly. SQL: Fix wrong results when sorting on aggregate (#43154) - Previously, when shorting on an aggregate function the bucket processing ended early when the explicit (LIMIT XXX) or the impliciti limit of 512 was reached. As a consequence, only a set of grouping buckets was processed and the results returned didn't reflect the global ordering. - Previously, the priority queue shorting method had an inverse comparison check and the final response from the priority queue was also returned in the inversed order because of the calls to the `pop()` method. Fixes: #42851 Fix JDK-version dependent exception message parsing This commit fixes some JDK-version dependent exception message checking in the discovery node role tests. [ML][Data Frame] have sum map to a double to prevent overflows (#43213) Use SecureString for password length validation (#42884) This replaces the use of char[] in the password length validation code, with the use of SecureString Although the use of char[] is not in itself problematic, using a SecureString encourages callers to think about the lifetime of the password object and to clear it after use. Fix versions for reindex max_docs tests (#42951) Versions were assuming this went into 7.2, but this ended up going into 7.3, updated test versions to reflect that. Related to #41894 [Docs] Name remote cluster "two" (#42976) All the text and the search example refer to the remote cluster as "two" rather than "cluster_two". This commit changes the definition that is stored through the settings API to match the rest of the examples. Remove explicily enabled build fixture task Correctly configure testclsuters for fips (#43186) Configuration was not being triggered for all projects with the way it was previusly set up. With this change we do it for each project directly. Convert dataframes to use testclusters (#43032) SimpleQ.S.B and QueryStringQ.S.B tests should avoid `now` in query (#43199) Currently the randomization of the q.b. in these tests can create query strings that can cause caching to be disabled for this query if we query all fields and there is a date field present. This is pretty much an anomaly that we shouldn't generally test for in the "testToQuery" tests where cache policies are checked. This change makes sure we don't create offending query strings so the cache checks never hit these cases and adds a special test method to check this edge case. Closes #43112 SQL: Increase hard limit for sorting on aggregates (#43220) To be consistent with the `search.max_buckets` default setting, set the hard limit of the PriorityQueue used for in memory sorting, when sorting on an aggregate function, to 10000. Fixes: #43168 Upgrade GCS Repository Dependencies (#43142) * Upgrade to latest GCS SDK and transitive dependencies (I chose the later version here on conflict) * Remove now unnecessary hack for custom endpoints (the linked bugs were both resolved in the SDK) [DOCS] Adds size and from parameters to data frame APIs (#43212) Move reindex request parsing into request (#43107) Currently the fromXContent logic for reindex requests is implemented in the rest action. This is inconsistent with other requests where the logic is implemented in the request. Additionally, it requires access to the rest action in order to parse the request. This commit moves the logic and tests into the ReindexRequest. Account soft deletes in committed segments (#43126) This change fixes the delete count issue in segment stats where we don't account soft-deleted documents from committed segments. Relates #43103 BWC tests - move vector distance functions to 7.3 [ML] Close sample stream in post_data endpoint (#43235) A static code analysis revealed that we are not closing the input stream in the post_data endpoint. This actually makes no difference in practice, as the particular InputStream implementation in this case is org.elasticsearch.common.bytes.BytesReferenceStreamInput and its close() method is a no-op. However, it is good practice to close the stream anyway. Ensure threads running before closing node (#43240) There are a few tests within NodeTests that submit items to the threadpool and then close the node. The tests are designed to check how running tasks are affected during node close. These tests can cause CI failures since the submitted tasks may not be running when the node is closed and then execute after the thread context is closed, which triggers an unexpected exception. This change ensures the threads are running so we avoid the unexpected exception and can test these cases. The test of task submittal while a node is closing is also important so an additional but muted test has been added that tests the case where a task may be getting submitted while the node is closing and ensuring we do not trigger anything unexpected in these cases. Relates #42774 Relates #42577 Report exponential_avg_bucket_processing_time which gives more weight to recent buckets (#43189) TestClusters: convert kerberos-tests (#43232) Looks like cluster formation tasks no longer plays nice wit test.fixtures so we just convert this to use testclusters. [DOCS] Simplifies wording. (#43226) This PR simplifies the wording of the TOC and eventually makes it shorter. [ML] Fix problem with lost shards in distributed failure test (#43153) We were stopping a node in the cluster at a time when the replica shards of the .ml-state index might not have been created. This change moves the wait for green status to a point where the .ml-state index exists. Fixes #40546 Fixes #41742 Forward port of #43111 Make DiscoveryNodeRole into a value object (#43257) Adds `equals()` and `hashcode()` methods to `DiscoveryNodeRole` to compare these objects' values for equality, and adds a field to allow us to distinguish unknown roles from known ones with the same name and abbreviation, for clearer test failures. Relates #43175 Allow cluster access during node restart (#42946) This commit modifies InternalTestCluster to allow using client() and other operations inside a RestartCallback (onStoppedNode typically). Restarting nodes are now removed from the map and thus all methods now return the state as if the restarting node does not exist. This avoids various exceptions stemming from accessing the stopped node(s). [docs] Add missing comma (#43073) Adds a missing comma to a code example Make ILM force merging best effort (#43246) It's possible for force merges kicked off by ILM to silently stop (due to a node relocating for example). In which case, the segment count may not reach what the user configured. In the subsequent `SegmentCountStep` waiting for the expected segment count may wait indefinitely. Because of this, this commit makes force merges "best effort" and then changes the `SegmentCountStep` to simply report (at INFO level) if the merge was not successful. Relates to #42824 Resolves #43245 Remove unneeded backticks (#43256) Relocation targets are assigned shards too (#43276) Adds relocation targets to the output of `IndexShardRoutingTable#assignedShards`. Add Painless Docs for Datetime Inputs (#43128) This changes add documentation for accessing datetimes in Painless scripts from the three most common inputs of params, _source, and doc. [ML] BWC tests for job_stats.timing_stats field (#43267) [ML][Data Frame] only complete task after state persistence (#43230) * [ML][Data Frame] only complete task after state persistence There is a race condition where the task could be completed, but there is still a pending document write. This change moves the task cancellation into the actionlistener of the state persistence. intermediate commit intermediate commit * removing unused import * removing unused const * refreshing internal index after waiting for task to complete * adjusting test data generation Rebuild version map when opening internal engine (#43202) With this change, we will rebuild the live version map and local checkpoint using documents (including soft-deleted) from the safe commit when opening an internal engine. This allows us to safely prune away _id of all soft-deleted documents as the version map is always in-sync with the Lucene index. Relates #40741 Supersedes #42979 [DOCS] Add introduction to Elasticsearch. (#43075) * [DOCS] Add introduction to Elasticsearch. * [DOCS] Incorporated review comments. * [DOCS] Minor edits to add an abbreviated title and cross refs. * [DOCS] Added sizing tips & link to quantatative sizing video. Add elasticsearch distribution plugin (#43247) Several types of distributions are built and tested in elasticsearch, ranging from the current version, to building or downloading snapshot or released versions. Currently tests relying on these have to contain logic deciding where and how to pull down these distributions. This commit adds an distributiond download plugin for each project to manage which versions and variants the project needs. It abstracts away all need for knowing where a particular version comes from, like a local distribution or bwc project, or pulling from the elastic download service. This will be used in a followup PR by the testclusters and vagrant tests. [DOCS] Fix typo: extraneous {es} [DOCS] Sewing SME says it should be "size 70" needle. Testclusters: convert ml qa tests (#43229) * Testclusters: convert ml qa tests This PR converts the ML tests to use testclusters. Remove Azure Container Exists Check (#43288) * This check is redundant, if the container doesn't exist subsequent operations will fail anyway. Since we are not running this exists check during verification I don't think there's much point to having it in snapshot initialization. * This PR is mainly motivated by the fact that this forces more permissions to be available in shared environments [ML] Speed up persistent task rechecks in ML failover tests (#43291) The ML failover tests sometimes need to wait for jobs to be assigned to new nodes following a node failure. They wait 10 seconds for this to happen. However, if the node that failed was the master node and a new master was elected then this 10 seconds might not be long enough as a refresh of the memory stats will delay job assignment. Once the memory refresh completes the persistent task will be assigned when the next cluster state update occurs or after the periodic recheck interval, which defaults to 30 seconds. Rather than increase the length of the wait for assignment to 31 seconds, this change decreases the periodic recheck interval to 1 second. Fixes #43289 TestClusters: Convert the security plugin (#43242) * TestClusters: Convert the security plugin This PR moves security tests to use TestClusters. The TLS test required support in testclusters itself, so the correct wait condition is configgured based on the cluster settings. * PR review Allow election of nodes outside voting config (#43243) Today we suppress election attempts on master-eligible nodes that are not in the voting configuration. In fact this restriction is not necessary: any master-eligible node can safely become master as long as it has a fresh enough cluster state and can gather a quorum of votes. Moreover, this restriction is sometimes undesirable: there may be a reason why we do not want any of the nodes in the voting configuration to become master. The reason for this restriction is as follows. If you want to shut the master down then you might first exclude it from the voting configuration. When this exclusion succeeds you might reasonably expect that a new master has been elected, since the voting config exclusion is almost always a step towards shutting the node down. If we allow nodes outside the voting configuration to be the master then the excluded node will continue to be master, which is confusing. This commit adjusts the logic to allow master-eligible nodes to attempt an election even if they are not in the voting configuration. If such a master is successfully elected then it adds itself to the voting configuration. This commit also adjusts the logic that causes master nodes to abdicate when they are excluded from the voting configuration, to avoid the confusion described above. Relates #37712, #37802. Local node is discovered when cluster fails (#43316) Today the `ClusterFormationFailureHelper` does not include the local node in the list of nodes it claims to have discovered. This means that it sometimes reports that it has not discovered a quorum when in fact it has. This commit adds the local node to the set of discovered nodes. Recursive Delete on BlobContainer (#43281) This is a prerequisite of #42189: * Add directory delete method to blob container specific to each implementation: * Some notes on the implementations: * AWS + GCS: We can simply exploit the fact that both AWS and GCS return blobs lexicographically ordered which allows us to simply delete in the same order that we receive the blobs from the listing request. For AWS this simply required listing without the delimiter setting (so we get a deep listing) and for GCS the same behavior is achieved by not using the directory mode on the listing invocation. The nice thing about this is, that even for very large numbers of blobs the memory requirements are now capped nicely since we go page by page when deleting. * For Azure I extended the parallelization to the listing calls as well and made it work recursively. I verified that this works with thread count `1` since we only block once in the initial thread and then fan out to a "graph" of child listeners that never block. * HDFS and FS are trivial since we have directory delete methods available for them * Enhances third party tests to ensure the new functionality works (I manually ran them for all cloud providers) Create a separate job solely for the purpose of verifying its timing stats in mixed and upgraded cluster. (#43315) Move dense_vector and sparse_vector to module (#43280) Fixes formatting of CCS compatibility table (#43231) Geo: Add coerce support to libs/geo WKT parser (#43273) Adds support for coercing not closed polygons and ignoring Z value to libs/geo WKT parser. Closes #43173 Deprecate native code info in xpack info api (#43297) The xpack info api currently returns native code info within each feature. This commit deprecates retrieving that info, which is now available directly in the ML info api. specifies which index to search in docs for various queries (#43307) the geo-bounding-box and phrase-suggest docs were susceptible to failing due to other indices in the cluster. This change restricts the queries to the index that is set up for the test. relates to #43271. Remove indices exists action (#43164) The indices exists action was kept around only to not break transport client, as RestGetIndicesAction has its own logic that ties into the unified HEAD requests handling. Now that tranport client is removed, we can remove also `TransportIndicesExistsAction` as well as its corresponding request, request builder, response, and action. Properly format reproduction lines for test methods that contain periods (#43255) Add AwaitsFix for #43342 Some Cleanup in BlobStoreRepository (#43323) * Some Cleanup in BlobStoreRepository * Extracted from #42833: * Dry up index and shard path handling * Shorten XContent handling Fix sporadic failures in QueryStringQueryTests#testToQueryFuzzyQueryAutoFuziness (#43322) This commit ensures that the test does not use reserved keyword (OR, AND, NOT) when generating the random query strings. Closes #43318 Fix randomization in testPerformActionAttrsRequestFails (#43304) The randomization in this test would occasionally generate duplicate node attribute keys, causing spurious test failures. This commit adjusts the randomization to not generate duplicate keys and cleans up the data structure used to hold the generated keys. Add kerberos grant_type to get token in exchange for Kerberos ticket (#42847) Kibana wants to create access_token/refresh_token pair using Token management APIs in exchange for kerberos tickets. `client_credentials` grant_type requires every user to have `cluster:admin/xpack/security/token/create` cluster privilege. This commit introduces `_kerberos` grant_type for generating `access_token` and `refresh_token` in exchange for a valid base64 encoded kerberos ticket. In addition, `kibana_user` role now has cluster privilege to create tokens. This allows Kibana to create access_token/refresh_token pair in exchange for kerberos tickets. Note: The lifetime from the kerberos ticket is not used in ES and so even after it expires the access_token/refresh_token pair will be valid. Care must be taken to invalidate such tokens using token management APIs if required. Closes #41943 Remove trace logging from ML datafeeds in tests This was added to investigate a test failure over two years ago, yet left behind. Since the test failure has been addressed since then, this commit removes the trace logging. Remove trace logging for ML native multi-node tests This trace logging looks like it was copy/pasted from another test, where the logging in that test was only added to investigate a test failure. This commit removes the trace logging. Remove token service trace logging in tests This commit removes some trace logging for the token service in the rolling upgrade tests. If there is an active investigation here, it would be best to annotate this line with a comment in the source indicating such. From my digging, it does not appear there is an active investigation that relies on this logging, so we remove it. Enable trace log for testOpenCloseApiWildcards Relates #39578 Refactor aggregation base classes to remove doEquals() and doHashCode() (#43214) A number of the aggregation base classes have an abstract doEquals() and doHashCode() (e.g. InternalAggregation.java, AbstractPipelineAggregationBuilder.java). Theoretically this is so the sub-classes can add to the equals/hashCode and don't need to worry about calling super.equals(). In practice, it's mostly just confusing/inconsistent. And if there are more than two levels, we end up with situations like InternalMappedSignificantTerms which has to call super.doEquals() which defeats the point of having these overridable methods. This PR removes the do versions and just use equals/hashCode ensuring the super when necessary. This PR is part of #41713 refactoring meta * Refactored all subclasses of InternalAggregation to remove doEquals() and doHashCode() * Refactored all subclasses of AbstractPipelineAggregationBuilder to remove doEquals() and doHashCode() * Refactored all subclasses of AbstractAggregationBuilder and CompositeValuesSourceBuilder to remove doEquals() and doHashCode() SearchPhaseContext to not extend ActionListener (#43269) The fact that SearchPhaseContext extends ActionListener makes it hard to reason about when the original listener is notified and to trace those calls. Also, the corresponding onFailure and onResponse were only needed in two places, one each, where they can be replaced by a more intuitive call, like sendSearchResponse for onResponse. Unmute timing_stats BWC test (#43358) Replace Streamable w/ Writeable in SingleShardRequest and subclasses (#43222) This commit replaces usages of Streamable with Writeable for the SingleShardRequest / TransportSingleShardAction classes and subclasses of these classes. Note that where possible response fields were made final and default constructors were removed. Relates to #34389 change version to 7.3, since backport work has been merged to 7.x Rename AutoDetectResultProcessor* to AutodetectResultProcessor* for consistency with other classes where the spelling is "Autodetect" (#43359) Fix NetworkUtilsTests (#43295) * Follow up to #42109: * Adjust test to only check that interface lookup by name works not actually lookup IPs which is brittle since virtual interfaces can be destroyed/created by Docker while the tests are running Co-authored-by: Jason Tedor Get snapshots support for multiple repositories (#42090) This commit adds multiple repositories support to get snapshots request. If some repository throws an exception this method does not fail fast instead, it returns results for all repositories. This PR is opened in favour of #41799, because we decided to change the response format in a non-BwC manner. It makes sense to read a discussion of the aforementioned PR. This is the continuation of work done here #15151. Do not use soft-deletes to resolve indexing strategy (#43336) This PR reverts #35230. Previously, we reply on soft-deletes to fill the mismatch between the version map and the Lucene index. This is no longer needed after #43202 where we rebuild the version map when opening an engine. Moreover, PrunePostingsMergePolicy can prune _id of soft-deleted documents out of order; thus the lookup result including soft-deletes sometimes does not return the latest version (although it's okay as we only use a valid result in an engine). With this change, we use only live documents in Lucene to resolve the indexing strategy. This is perfectly safe since we keep all deleted documents after the local checkpoint in the version map. Closes #42979 Remove native code info from xpack info api (#43125) The native code info is a portion of the xpack info api that emits arbitrary info about feature plugins that are backed by native code. This is currently only used by machine learning. Additionally, it is the only non enabled/available information reported by the info api. This commit moves the native code info reporting to the usage api for machine learning. The commit info reported is only used for debugging purposes, and not captured by the current uses of the info api (monitoring and telemetry) since it requires passing the verbose flag. Long term, this information would be better suited to node info. Until node info is extendable by plugins, this new location will allow cleaning up the xpack info api to be implemented in a generic way for all xpack features. Return 0 for negative "free" and "total" memory reported by the OS (#42725) * Return 0 for negative "free" and "total" memory reported by the OS We've had a situation where the MX bean reported negative values for the free memory of the OS, in those rare cases we want to return a value of 0 rather than blowing up later down the pipeline. In the event that there is a serialization or creation error with regard to memory use, this adds asserts so the failure will occur as soon as possible and give us a better location for investigation. Resolves #42157 * Fix test passing in invalid memory value * Fix another test passing in invalid memory value * Also change mem check in MachineLearning.machineMemoryFromStats * Add background documentation for why we prevent negative return values * Clarify comment a bit more Fix GeoIP custom database directory in docs (#43383) These docs were misleading for package installations of Elasticsearch. Instead, we should refer to $ES_CONFIG/ingest-geoip as the path to place the custom database files. For non-package installations, this is the same as $ES_HOME/config, but for package installations this is not the case as the config directory for package installations is /etc/elasticsearch, and is not relative to $ES_HOME. This commit corrects the docs. [ML][Data Frame] adds new pipeline field to dest config (#43124) * [ML][Data Frame] adds new pipeline field to dest config * Adding pipeline support to _preview * removing unused import * moving towards extracting _source from pipeline simulation * fixing permission requirement, adding _index entry to doc Muting DataFrameTransformIT.testGetStats See #43324 [ML][Data Frame] make response.count be total count of hits (#43241) * [ML][Data Frame] make response.count be total count of hits * addressing line length check * changing response count for filters * adjusting serialization, variable name, and total count logic * making count mandatory for creation Allocate memory lazily in BestBucketsDeferringCollector (#43339) While investigating memory consumption of deeply nested aggregations for #43091 the memory used to keep track of the doc ids and buckets in the BestBucketsDeferringCollector showed up as one of the main contributor. In my tests half of the memory held in the BestBucketsDeferringCollector is associated to segments that don't have matching docs in the selected buckets. This is expected on fields that have a big cardinality since each bucket can appear in very few segments. By allocating the builders lazily this change reduces the memory consumption by a factor 2 (from 1GB to 512MB), hence reducing the impact on gcs for these volatile allocations. This commit also switches the PackedLongValues.Builder with a RoaringDocIdSet in order to handle very sparse buckets more efficiently. I ran all my tests on the `geoname` rally track with the following query: ```` { "size": 0, "aggs": { "country_population": { "terms": { "size": 100, "field": "country_code.raw" }, "aggs": { "admin1_code": { "terms": { "size": 100, "field": "admin1_code.raw" }, "aggs": { "admin2_code": { "terms": { "size": 100, "field": "admin2_code.raw" }, "aggs": { "sum_population": { "sum": { "field": "population" } } } } } } } } } } ```` Narrow period of Shrink action in which ILM prevents stopping (#43254) * Narrow period of Shrink action in which ILM prevents stopping Prior to this change, we would prevent stopping of ILM if the index was anywhere in the shrink action. This commit changes `IndexLifecycleService` to allow stopping when in any of the innocuous steps during shrink. This changes ILM only to prevent stopping if absolutely necessary. Resolves #43253 * Rename variable for ignore actions -> ignore steps * Fix comment * Factor test out to test *all* stoppable steps Prevent NullPointerException in TransportRolloverAction (#43353) It's possible for the passed in `IndexMetaData` to be null (for instance, cluster state passed in does not have the index in its metadata) which in turn can cause a `NullPointerException` when evaluating the conditions for an index. This commit adds null protection and unit tests for this case. Resolves #43296 Reduce the number of docvalues iterator created in the global ordinals fielddata (#43091) Today the fielddata for global ordinals re-creates docvalues readers of each segment when building the iterator of a single segment. This is required because the lookup of global ordinals needs to access the docvalues's TermsEnum of each segment to retrieve the original terms. This also means that we need to create NxN (where N is the number of segment in the index) docvalues iterators each time we want to collect global ordinal values. This wasn't an issue in previous versions since docvalues readers are stateless before 6.0 so they are reused on each segment but now that docvalues are iterators we need to create a new instance each time we want to access the values. In order to avoid creating too many iterators this change splits the global ordinals fielddata in two classes, one that is used to cache a single instance per directory reader and one that is created from the cached instance that can be used by a single consumer. The latter creates the TermsEnum of each segment once and reuse them to create the segment's iterator. This prevents the creation of all TermsEnums each time we want to access the value of a single segment, hence reducing the number of docvalues iterator to create to Nx2 (one iterator and one lookup per segment). Remove stale test logging annotations (#43403) This commit removes some very old test logging annotations that appeared to be added to investigate test failures that are long since closed. If these are needed, they can be added back on a case-by-case basis with a comment associating them to a test failure. Testclusters: conver remaining x-pack (#43335) Convert x-pack tests Advance checkpoints only after persisting ops (#43205) Local and global checkpoints currently do not correctly reflect what's persisted to disk. The issue is that the local checkpoint is adapted as soon as an operation is processed (but not fsynced yet). This leaves room for the history below the global checkpoint to still change in case of a crash. As we rely on global checkpoints for CCR as well as operation-based recoveries, this has the risk of shard copies / follower clusters going out of sync. This commit required changing some core classes in the system: - The LocalCheckpointTracker keeps track now not only of the information whether an operation has been processed, but also whether that operation has been persisted to disk. - TranslogWriter now keeps track of the sequence numbers that have not been fsynced yet. Once they are fsynced, TranslogWriter notifies LocalCheckpointTracker of this. - ReplicationTracker now keeps track of the persisted local and persisted global checkpoints of all shard copies when in primary mode. The computed global checkpoint (which represents the minimum of all persisted local checkpoints of all in-sync shard copies), which was previously stored in the checkpoint entry for the local shard copy, has been moved to an extra field. - The periodic global checkpoint sync now also takes async durability into account, where the local checkpoints on shards only advance when the translog is asynchronously fsynced. This means that the previous condition to detect inactivity (max sequence number is equal to global checkpoint) is not sufficient anymore. - The new index closing API does not work when combined with async durability. The shard verification step is now requires an additional pre-flight step to fsync the translog, so that the main verify shard step has the most up-to-date global checkpoint at disposition. Fix NPE in case of subsequent scrolled requests for a CSV/TSV formatted response (#43365) Remove mentions of "fields with the same name in the same index" (#43077) Together with types removal, any mention of "fields with the same name in the same index" doesn't make sense anymore. Reconnect remote cluster when seeds are changed (#43379) The RemoteClusterService should close the current RemoteClusterConnection and should build it again if the seeds are changed, similarly to what is done when the ping interval or the compression settings are changed. Closes #37799 [Docs] Remove boost parameter from intervals-query example (#43331) The boost factor doesn't seem to be needed and can be removed. Adapt BWC conditions after backport of #43205 [ML] Unmute NetworkDisruptionIT.testJobRelocation (#43268) This reverts commit 27346a076b5f3ca9a81dabbf1796182d3ffc7e00. Reduce shard started failure logging (#43330) If the master is stepping or shutting down, the error-level logging can cause quite a bit of noise. Fix testGlobalCheckpointSync The test needed adaption after #43205, as the ReplicationTracker now distinguishes between the knowledge of the persisted global checkpoint and the computed global checkpoint on the primary Follow-up to #43205 Docs: Add description of the coerce parameter in geo_shape mapper (#43340) Explains the effect of the coerce parameter on the geo_shape field. Relates #35059 Fail connection attempts earlier in tests (#43320) Today the `DisruptibleMockTransport` always allows a connection to a node to be established, and then fails requests sent to that node such as the subsequent handshake. Since #42342, we log handshake failures on an open connection as a warning, and this makes the test logs rather noisy. This change fails the connection attempt first, avoiding these unrealistic warnings. Mute test NetworkDisruptionIT.testJobRelocation Remove Confusing Comment (#43400) [DOCS] Describe setup for monitoring logs (#42655) Mute put-transform docs test Relates to #43271 [DOCS] Add brew install instructions. Closes #42914 (#42915) Better support for unmapped fields in AggregatorTestCase (#43405) AggregatorTestCase will NPE if only a single, null MappedFieldType is provided (which is required to simulate an unmapped field). While it's possible to test unmapped fields by supplying other, non-related field types... that's clunky and unnecessary. AggregatorTestCase just needs to filter out null field types when setting up. Adapt local checkpoint assertion With async durability, it does not hold true anymore after #43205. This is fine. Remove old Painless doc generator (#43404) This removes the previous Painless API Doc Generator prior to contexts existing. It has been replaced with the new doc generator that uses the documentation rest API. Increase timeout for assertSeqNos Helps with tests that do async translog syncing Added parsing of erroneous field value (#42321) [DOCS] Fixed path to install directory. (#43443) Log Blocked IO Thread State (#43424) * Let's log the state of the thread to find out if it's dead-locked or just stuck after being suspended * Relates #43392 Add painless method getByPath, get value from nested collections with dotted path (#43170) Given a nested structure composed of Lists and Maps, getByPath will return the value keyed by path. getByPath is a method on Lists and Maps. The path is string Map keys and integer List indices separated by dot. An optional third argument returns a default value if the path lookup fails due to a missing value. Eg. ['key0': ['a', 'b'], 'key1': ['c', 'd']].getByPath('key1') = ['c', 'd'] ['key0': ['a', 'b'], 'key1': ['c', 'd']].getByPath('key1.0') = 'c' ['key0': ['a', 'b'], 'key1': ['c', 'd']].getByPath('key2', 'x') = 'x' [['key0': 'value0'], ['key1': 'value1']].getByPath('1.key1') = 'value1' Throws IllegalArgumentException if an item cannot be found and a default is not given. Throws NumberFormatException if a path element operating on a List is not an integer. Fixes #42769 [DOCS] Added blurb & link to Elasticsearch Service. (#43452) Fix round up of date range without rounding (#43303) Today when searching for an exclusive range the java date math parser rounds up the value with the granularity of the operation. So when searching for values that are greater than "now-2M" the parser rounds up the operation to "now-1M". This behavior was introduced when we migrated to java date but it looks like a bug since the joda math parser rounds up values but only when a rounding is used. So "now/M" is rounded to "now-1ms" (minus 1ms to get the largest inclusive value) in the joda parser if the result should be exclusive but no rounding is applied if the input is a simple operation like "now-1M". This change restores the joda behavior in order to have a consistent parsing in all versions. Closes #43277 [DOCS] Updated the URL for starting in the cloud. Clarify unsupported secure settings behavior (#43454) This commit tweaks the docs for secure settings to ensure the user is aware adding non secure settings to the keystore will result in elasticsearch not starting. fixes #43328 Co-Authored-By: James Rodewig Reduce TestLogging usage in DisruptionIT tests (#43411) Removes `@TestLogging` annotations in `*DisruptionIT` tests, so that the only tests with annotations are those with open issues. Also adds links to the open issues in the remaining cases. Relates #43403 Make Recovery API support `detailed` params (#29076) Properly forwards the `detailed` parameter to show the recovery stats details. Closes #28910 Change BWC condition after backporting #29076 SecurityIndexSearcherWrapper doesn't always carry over caches and similarity (#43436) If DocumentLevelSecurity is enabled SecurityIndexSearcherWrapper doesn't carry over the cache, cache policy and similarity from the incoming searcher. Remove types exist action (#43344) The types exists action was kept around only to not break transport client, as RestGetMappingAction has its own logic that ties into the unified HEAD requests handling. Now that tranport client is removed, we can remove also TransportTypesExistsAction as well as its corresponding request, request builder, response, and action. Resolve NetworkDisruptionIT (#43441) [ML][Data Frame] Add version and create_time to transform config (#43384) * [ML][Data Frame] Add version and create_time to transform config * s/transform_version/version s/Date/Instant * fixing getter/setter for version Fix DefaultShardOperationFailedException subclass xcontent serialization (#43435) The current toXContent implementation can fail when the superclasses toXContent is called (see #43423). This change makes sure that DefaultShardOperationFailedException#toXContent is final and implementations need to add special fields in #innerToXContent. All implementations should write to self-contained xContent objects. Also adding a test for xContent deserialization to CloseIndexResponseTests. Closes #43423 Example of how to set slow logs dynamically per-index (#42384) * Example of how to set slow logs dynamically per-index * Make _settings API example more explicit Co-Authored-By: James Rodewig * Add TEST directive to fix CI Co-Authored-By: James Rodewig Recursively Delete Unreferenced Index Directories (#42189) * Use ability to list child "folders" in the blob store to implement recursive delete on all stale index folders when cleaning up instead of using the diff between two `RepositoryData` instances to cover aborted deletes * Runs after ever delete operation * Relates #13159 (fixing most of this issues caused by unreferenced indices, leaving some meta files to be cleaned up only) Simplify BlobStoreRepository (Flatten Nested Classes) (#42833) * In the current codebase it is hardly obvious what code operates on a shard and is run by a datanode what code operates on the global metadata and is run on master * Fixed by adjusting the method names accordingly * The nested context classes don't add much if any value, they simply spread out the parameters that go into a shard snapshot create or delete all over the place since their constructors can be inlined in all spots * Fixed by flattening the nested classes into BlobStoreRepository * Also: * Inlined the other single use inner classes Assert ServerSocketChannel is not Blocking (#43479) * Assert ServerSocketChannel is not Blocking * Relates #43387 which appears to run into blocking accept calls [DOCS] Rewrite term-level queries overview (#43337) [DOCS] Rewrite `constant_score` query (#43374) [ML][Data Frame] fixing some data frame hlrc tests (#43446) * [ML][Data Frame] fixing some data frame hlrc tests * adding task|indexer state checks back Remove aliases exist action (#43430) The aliases exists action was kept around only to not break transport client, as RestGetAliasesAction has its own logic that ties into the unified HEAD requests handling. Now that transport client is removed, we can remove also TransportAliasesExistAction as well as its corresponding request, request builder, response, and action. Mute failing test Mute LicenseDocumentationIT#testGetLicense (#43505) Account for node versions during allocation in ILM Shrink (#43300) This commit ensures that ILM's Shrink action will take node versions into account when choosing which node to allocate to when shrinking an index. Prior to this change, ILM could pick a node with a lower version than some shards are already allocated to, which causes the new allocation to fail as shards can't be relocated onto a node with a lower version than they are already on. As part of this, when making the decision about which node to allocate to prior to Shrink, all shards in the index are considered, rather than choosing a random shard to consider. Further, the unit tests for the logic that chooses a node to allocate shards to pre-shrink has been improved to validate the behavior in more realistic and varied initial conditions. Don't Consistency Check Broken Repository in Test (#43499) * Missed this one in #42189 and it randomly runs into a situation where the broken mock repo is broken such that we can't get to a consistent end state via a delete * Closes #43498 UserRoleMapper.UserData non-null groups and metadata (#41774) The `UserRoleMapper.UserData` is constructed by each realm and is used to "match" role mapping expressions that eventually supply the role names of the principal. `null` values as group names or metadata values were not properly supported. Most of the time they were filtered out, but under some circumstances they might cause NPEs (depending of how the OIDC and SAML's underlying libraries parse purposefully crafted assertions - these libraries don't have internal null checks the same way that the unboundid library does). This PR enforces non-null collection values (lists and maps) earliest in the process, on first building the collection, and utilizes java 9 unmodifiable collections API, with the goal of eliminating redundant null filtering and repeated wrapping in `Colllections.unmodifiable*`. Cleanup legacy logic in CombinedDeletionPolicy (#43484) This change removes the support for pre-v6 index commits which do not have sequence numbers. Replace Streamable w/ Writable in AcknowledgedResponse and subclasses (#43414) This commit replaces usages of Streamable with Writeable for the AcknowledgedResponse and its subclasses, plus associated actions. Note that where possible response fields were made final and default constructors were removed. This is a large PR, but the change is mostly mechanical. Relates to #34389 Fix the bundled jdk flag to be passed through windows startup (#43502) This commit fixes a typo in elasticsearch.bat that prevented the windows distribution from knowing whether it is using the bundled jdk. Use transport actions instead of guice for xpack info (#43449) The Xpack info API internally gathers all xpack feature sets by using guice to bind many implementations. The info api then iterates over these feature sets to gather usage information for each. However, since each of these implementations are in different plugins, there is no direct, non-guice way for them to register the implementations. This commit removes the dependence on guice for the info api by having the core xpack jar contain an action instance for each plugin, and the implementation of those actions are then registered within each xpack feature plugin. The info api then iterates over each of these actions and calls them with the NodeClient. Clarify storage location of ML Snapshots (#43437) The existing language was misleading about the model snapshots and where they are located. Saying "to disk" sounds like files external to Elasticsearch IMO. It raises the obvious question, where on disk? which node? Is it in the Elasticsearch snapshot repo? The model snapshots are held in an internal index. Improove error on waiting for cluster to come up (#43416) Always include all messages from exceptions to make the reason for failure more visible. Test clusters: convert x-pack qa tests (#43283) Assert that NOOPs must succeed (#43483) We currently assert that adding deletion tombstones to Lucene must always succeed if it's not a tragic exception, and the same should also hold true for NOOP tombstones. We rely on this assumption, as without this, we have the risk of creating gaps in the history, which will break operation-based recoveries and CCR. Cleanup IndicesService#CacheCleaner Scheduling (#42060) * Follow up to #42016 Add additional logging for #43034 It's unclear why sometimes the shard is not flushed on closing Do not hang on unsupported HTTP methods (#43362) Unsupported HTTP methods are detected during requests dispatching which generates an appropriate error response. Sadly, this error is never sent back to the client because the method of the original request is checked again in DefaultRestChannel which throws again an IllegalArgumentException that is never handled. This pull request changes the DefaultRestChannel so that the latest exception is swallowed, allowing the error message to be sent back to the client. It also eagerly adds the objects to close to the toClose list so that resources are more likely to be released if something goes wrong during the response creation and sending. Move nio ip filter rule to be a channel handler (#43507) Currently nio implements ip filtering at the channel context level. This is kind of a hack as the application logic should be implemented at the handler level. This commit moves the ip filtering into a channel handler. This requires adding an indicator to the channel handler to show when a channel should be closed. Fix testNoMasterActions (#43471) This commit performs the proper restore of network disruption. Previously disruptionScheme.stopDisrupting() was called that does not ensure that connectivity between cluster nodes is restored. The test was checking that the cluster has green status, but it was not checking that connectivity between nodes is restored. Here we switch to internalCluster().clearDisruptionScheme(true) which performs both checks before returning. Similar to #42798 Closes #42051 [DOCS] Rename "job" to "transform" in data frame transform docs (#43534) Add annotations to Painless whitelist (#43239) * Start to parse annotations in the whitelist * Finish parsing annotations * Finish annotation parsing * Add docs * Parser changes. * Add annotation parser. * Add annotation tests for Painless core. * Add example annotation to Painless example extension. * Response to PR comments. Add TimeValue.toHumanReadableString() to allow specifying frac… (#43346) * Enhance TimeValue.toString() to allow specifying fractional values. This enhances the `TimeValue` class to allow specifying the number of truncated fractional decimals when calling `toString()`. The default remains 1, however, more (or less, such as 0) can be specified to change the output. This commit also re-organizes some things in `TimeValue` such as putting all the class variables near the top of the class, and moving the constructors to the first methods in the class, in order to follow the structure of our other code. * Rename `toString(...)` to `toHumanReadableString(...)` Set document on script when using Bytes.WithScript (#43390) Long and Double ValuesSource set the current document on the script before executing, but Bytes was missing this method call. That meant it was possible to generate an OutOfBoundsException when using a "value" script (field + script) on keyword or other bytes fields. This adds in the method call, and a few yaml tests to verify correct behavior. Distribution download plugin cleanup (#43461) This commit addresses some post-review comments. relates #43247 Sync global checkpoint on pending in-sync shards (#43526) At the end of a peer recovery the primary wants to mark the replica as in-sync. For that the persisted local checkpoint of the replica needs to have caught up with the global checkpoint on the primary. If translog durability is set to ASYNC, this means that information about the persisted local checkpoint can lag on the primary and might need to be explicitly fetched through a global checkpoint sync action. Unfortunately, that action will only be triggered after 30 seconds, and, even worse, will only run based on what the in-sync shard copies say (see IndexShard.maybeSyncGlobalCheckpoint). As the replica has not been marked as in-sync yet, it is not taken into consideration, and the primary might have its global checkpoint equal to the max seq no, so it thinks nothing needs to be done. Closes #43486 [DOCS] Adds administering section (#43493) Fix score mode of the MinimumScoreCollector (#43527) This change fixes the score mode of the minimum score collector to be set based on the score mode of the child collector (top docs). Closes #43497 Fix CreateRepository Requeset in HLRC (#43522) * verify = false is the non-default case for this request -> adjusted the code accordingly and expanded the test to cover this case * Closes #43521 Properly serialize remote query in ReindexRequest (#43457) This commit modifies the RemoteInfo to clarify that a search query must always be serialized as JSON. Additionally, it adds an assertion to ensure that this is the case. This fixes #43406. Additionally, this PR implements AbstractXContentTestCase for the reindex request. This is related to #43456. Remove debug log in testOpenCloseApiWildcards Relates #39578 [DOCS] Edited title/subtitle. (#43552) Enable Kerberos tests (#43519) Now that the fix krb5-kdc fixture (entropy problem in docker container) is in and the converting `kerberos-tests` to testclusters is done, enabling the kerberos-tests Closes #40678 Fix for PemTrustConfigTests.testTrustConfigReloadsFileContents failure (#43539) The test `PemTrustConfigTests.testTrustConfigReloadsFileContents` failed intermittently with `ArrayIndexOutOfBoundsException` while parsing the randomly generated bytes array representing DER encoded stream. This seems to be a bug in JDK (once confirmed we can raise the bug in JDK bugs system). The problem arises when the `X509Factory#parseX509orPKCS7()` tries to [create `PKCS7` block](https://github.com/AdoptOpenJDK/openjdk-jdk11/blob/19fb8f93c59dfd791f62d41f332db9e306bc1422/src/java.base/share/classes/sun/security/provider/X509Factory.java#L460) from der encoded stream. While constructing PKCS7 block it tries to create `ContentInfo` type but fails to do so for the stream where the length after the DER SEQUENCE is 0. `DerInputStream#getSequence` [may return empty array of `DerValue`](https://github.com/AdoptOpenJDK/openjdk-jdk11/blob/19fb8f93c59dfd791f62d41f332db9e306bc1422/src/java.base/share/classes/sun/security/util/DerInputStream.java#L409..L412) but [the code in `ContentInfo`](https://github.com/AdoptOpenJDK/openjdk-jdk11/blob/19fb8f93c59dfd791f62d41f332db9e306bc1422/src/java.base/share/classes/sun/security/pkcs/ContentInfo.java#L135) does not check for the empty thereby throwing `ArrayIndexOutOfBoundsException`. Closes #42509 Extract TimingStats-related functionality into TimingStatsReporter (#43371) Get snapshots response example for the breaking change (#43475) for requesting snapshots from multiple repositories. And it has changed the response format in a non-BwC way. There is a mentioning of a response format change in the breaking changes docs, however, there is no example of how new format looks like. Pointed out by @dakrone. This commit adds the missing example. [ML] Machine learning data frame analytics (#43544) This merges the initial work that adds a framework for performing machine learning analytics on data frames. The feature is currently experimental and requires a platinum license. Note that the original commits can be found in the `feature-ml-data-frame-analytics` branch. A new set of APIs is added which allows the creation of data frame analytics jobs. Configuration allows specifying different types of analysis to be performed on a data frame. At first there is support for outlier detection. The APIs are: - PUT _ml/data_frame/analysis/{id} - GET _ml/data_frame/analysis/{id} - GET _ml/data_frame/analysis/{id}/_stats - POST _ml/data_frame/analysis/{id}/_start - POST _ml/data_frame/analysis/{id}/_stop - DELETE _ml/data_frame/analysis/{id} When a data frame analytics job is started a persistent task is created and started. The main steps of the task are: 1. reindex the source index into the dest index 2. analyze the data through the data_frame_analyzer c++ process 3. merge the results of the process back into the destination index In addition, an evaluation API is added which packages commonly used metrics that provide evaluation of various analysis: - POST _ml/data_frame/_evaluate Make role descriptors optional when creating API keys (#43481) This commit changes the `role_descriptors` field from required to optional when creating API key. The default behavior in .NET ES client is to omit properties with `null` value requiring additional workarounds. The behavior for the API does not change. Field names (`id`, `name`) in the invalidate api keys API documentation have been corrected where they were wrong. Closes #42053 Fix DieWithDignity logs assertion (#43543) This test is likely to kill the server in the middle of writing logs. This means that we can end up with logs with partially written json log lines and standard json parsers would fail on this. This fix is to use regular expressions on json logs.(just like the previous approach on plain text logs) closes #43413 Fix testPostOperationGlobalCheckpointSync The conditions in this test do not hold true anymore after #43205. Relates to #43205 [ML] Improve message when native controller cannot connect (#43565) The error message if the native controller failed to run (for example due to running Elasticsearch on an unsupported platform) was not easy to understand. This change removes pointless detail from the message and adds some hints about likely causes. Fixes #42341 Fix DedicatedClusterSnapshotRestoreIT testSnapshotWithStuckNode (#43537) * Fix DedicatedClusterSnapshotRestoreIT testSnapshotWithStuckNode * See comment in the test: The problem is that when the snapshot delete works out partially on master failover and the retry fails on `SnapshotMissingException` no repository cleanup is run => we still failed even with repo cleanup logic in the delete path now * Fixed the test by rerunning a create snapshot and delete loop to clean up the repo before verifying file counts * Closes #39852 [ML][Data Frame] Adjusting error message (#43455) * Adjusting error message * Update TransportPutDataFrameTransformAction.java * Update TransportPutDataFrameTransformAction.java [ML][TEST] Refresh doc index in setup of test running df analytics Fix GET /_snapshot/_all/_all if there are no repos (#43558) When there are no repositories, a request to GET /_snapshot/_all/_all returns a 504 timeout error. This happens because try to create GroupedActionListener with the size of zero, which leads to an exception. This commit short-circuits if there are no repos and adds a test to verify the fix. Closes #43547 Default number of shards is now 1 instead of 5 (#43573) As specified in the [Breaking changes for 7.X](https://www.elastic.co/guide/en/elasticsearch/reference/7.1/breaking-changes-7.0.html#breaking_70_indices_changes), the default number of shards for an index is now `1` instead of `5`. [ML] Construct ML native controller in standard place (#43570) The ML native controller used to be used within the ML feature set which necessitated constructing it outside of the usual createComponents() method. Now that the ML native code info is reported by the ML info action rather than by the feature set it is possible to move construction to the standard place, which avoids surprises for maintainers. Fix search template request (#43509) A seed was hit in (#43157) that caused mutateInstance to generate an identical instance. This change prevents that. [ML][TEST] Also refresh doc index in another df analytics test Fixing backquote in fail_on_unsupported_field (#43572) Fix indices shown in _cat/indices (#43286) After two recent changes (#38824 and #33888), the _cat/indices API no longer report information for active recovering indices and non-replicated closed indices. It also misreport replicated closed indices that are potentially not authorized for the user. This commit changes how the cat action works by first using the Get Settings API in order to resolve authorized indices. It then uses the Cluster State, Cluster Health and Indices Stats APIs to retrieve information about the indices. Closes #39933 Add voting-only master node (#43410) A voting-only master-eligible node is a node that can participate in master elections but will not act as a master in the cluster. In particular, a voting-only node can help elect another master-eligible node as master, and can serve as a tiebreaker in elections. High availability (HA) clusters require at least three master-eligible nodes, so that if one of the three nodes is down, then the remaining two can still elect a master amongst them-selves. This only requires one of the two remaining nodes to have the capability to act as master, but both need to have voting powers. This means that one of the three master-eligible nodes can be made as voting-only. If this voting-only node is a dedicated master, a less powerful machine or a smaller heap-size can be chosen for this node. Alternatively, a voting-only non-dedicated master node can play the role of the third master-eligible node, which allows running an HA cluster with only two dedicated master nodes. Closes #14340 Co-authored-by: David Turner Fix failing LicensingDocumentationIT test (#43533) This PR brings corrections for cluster name after migrating to testclusters. Not sure how this slipped trough the cracks when converting. Closes #43504 [ML] Introduce a setting for the process connect timeout (#43234) This change introduces a new setting, xpack.ml.process_connect_timeout, to enable the timeout for one of the external ML processes to connect to the ES JVM to be increased. The timeout may need to be increased if many processes are being started simultaneously on the same machine. This is unlikely in clusters with many ML nodes, as we balance the processes across the ML nodes, but can happen in clusters with a single ML node and a high value for xpack.ml.node_concurrent_job_allocations. Disable testing conventions on Windows (#43532) Tests are disabled on Windows. Conventions also need to be disabled. Testclusters: convert left-overs from checkPart1 (#43370) * Testclusters: convert left-overs from checkPart1 Add scripting support to AggregatorTestCase (#43494) This refactors AggregatorTestCase to allow testing mock scripts. The main change is to QueryShardContext. This was previously mocked, but to get the ScriptService you have to invoke a final method which can't be mocked. Instead, we just create a mostly-empty QueryShardContext and populate the fields that are needed for testing. It also introduces a few new helper methods that can be overridden to change the default behavior a bit. Most tests should be able to override getMockScriptService() to supply a ScriptService to the context, which is later used by the aggs. More complicated tests can override queryShardContextMock() as before. Adds a test to MaxAggregatorTests to test out the new functionality. [DOCS] Add redirect for painless examples anchor Fix merge issue between #43410 and #43449 PR builds on #43410 had not picked up the change merged in #43449 include 7.2.1 as a version (#43584) Allow extra time for a warning to be logged (#43597) Today we assert that a warning is logged after no more than `discovery.cluster_formation_warning_timeout`, but the deterministic scheduler adds a small amount of extra randomness to the timing of future events, causing the following build to fail: ./gradlew :server:test --tests "org.elasticsearch.cluster.coordination.CoordinatorTests.testLogsWarningPeriodicallyIfClusterNotFormed" -Dtests.seed=DF35C28D4FA9EE2D This commit adds an allowance for this extra time. [DOCS] Rewrite `range` query (#43282) Ensure relocation target still tracked when start handoff (#42201) If the master removes the relocating shard, but recovery isn't aware of it, then we can enter an invalid state where ReplicationTracker does not include the local shard. Adjust bwc assertion after backporting #42201 Remove guice modules from plugins (#43555) Guice modules provide a way to supply injected parameters to other injected constructors. The last remiaing use in plugins is to provide constructor arguments to transport actions. This commit removes the ability for plugins to provide guice modules. Any transport action parameters should be concretely typed and returned from createComponents, which are still injected. While this does not remove guice completely, it removes it from all remaining plugin apis. Remove unused method not caught by merge relates #43555 Reindex remote version lookup test refactor (#43576) Refactor test to reuse code and ease maintenance Reindex remove outer level size (#43373) This commit finalizes the work done to rename size to max_docs in reindex and update/delete by query. size is no longer supported in URL or outer level body for the 3 APIs (though size in update/delete-by-query will and has always been interpreted as scroll_size, it is not to be relied upon). Continuation of #41894 Closes #24344 Upgrade jcodings dependency to 1.0.44 (#43334) Mute DiskDisruptionIT#testGlobalCheckpointIsSafe Relates to #43626 Mute VotingOnlyNodeCoordinatorTests#testDoesNotElectVotingOnlyMasterNode Relates to #43631 [TEST] Wait for replicas before stopping nodes in ML distributed test (#43622) If we stop a node before replicas exist then the test can fail because we lose a whole index if we stop the node with the primary on. [ML][Data Frame] improve pivot nested field validations (#43548) * [ML][Data Frame] improve pivot nested field validations * addressing pr comments [ML][Data Frame] Add support for allow_no_match for endpoints (#43490) * [ML][Data Frame] Add support for allow_no_match parameter in endpoints Adds support for: * Get Transforms * Get Transforms stats * stop transforms Remove unreleased 7.1.2 version constant (#43629) This was breaking BWC tests as the presence of the constant implied 7.1.2 was released Add prefix intervals source (#43635) This commit adds a prefix intervals source, allowing you to search for intervals that contain terms starting with a given prefix. The source can make use of the index_prefixes mapping option. Relates to #43198 [ML][Data Frame] fixing tag end for df doc tests (#43640) [ML] Tag destination index with data frame metadata (#43567) Optimize Selector Wakeups (#43515) * Use atomic boolean to guard wakeups * Don't trigger wakeups from the select loops thread itself for registering and closing channels * Don't needlessly queue writes Co-authored-by: Tim Brooks [Ml Data Frame] Size the GET stats search by number of Ids requested (#43206) Set the size of the search request to the number of ids limited by 10,000 Avoid AssertionError when closing engine (#43638) Lucene throwing an AlreadyClosedException when closing the engine is fine, and should not trigger an AssertionError. Closes #43626 Remove blank file Relates to #43410 Remove non task-aware version of master and node operations (#43563) TransportNodeAction and TransportMasterNodeAction contains two variations of their abstract operation methods: one taking a Task and one that does not. This commit removes the variant that does not take a Task, eliminating the need to override that method and throw an unsupported operation exception in those cases when the task is required. [ML][Data Frame] adds tests for continuous DF (#43601) [DOCS] Change 'X-Pack APIs' section to 'REST APIs' (#43451) [DOCS] Updates data frame APIs to use API template (#43610) Remove compile-time dependency on test fixtures (#43651) Mute failing test Tracked in #43670 Make the ignore_above docs tests more robust. (#43349) It is possible for internal ML indices like `.data-frame-notifications-1` to leak, causing other docs tests to fail when they accidentally search over these indices. This PR updates the ignore_above tests to only search a specific index. Require [articles] setting in elision filter (#43083) We should throw an exception at construction time if a list of articles is not provided, otherwise we can get random NPEs during indexing. Relates to #43002 Use preconfigured filters correctly in Analyze API (#43568) When a named token filter or char filter is passed as part of an Analyze API request with no index, we currently try and build the relevant filter using no index settings. However, this can miss cases where there is a pre-configured filter defined in the analysis registry. One example here is the elision filter, which has a pre-configured version built with the french elision set; when used as part of normal analysis, this preconfigured set is used, but when used as part of the Analyze API we end up with NPEs because it tries to instantiate the filter with no index settings. This commit changes the Analyze API to check for pre-configured filters in the case that the request has no index defined, and is using a name rather than a custom definition for a filter. It also changes the pre-configured `word_delimiter_graph` filter and `edge_ngram` tokenizer to make their settings consistent with the defaults used when creating them with no settings Closes #43002 Closes #43621 Closes #43582 Remove Unused AWS KMS Dependency (#43671) * We don't make use of KMS at the moment, no need to have this dependency here Move query builder caching check to dedicated tests (#43238) Currently `AbstractQueryTestCase#testToQuery` checks the search context cachable flag. This is a bit fragile due to the high randomization of query builders performed by this general test. Also we might only rarely check the "interesting" cases because they rarely get generated when fully randomizing the query builder. This change moved the general checks out ot #testToQuery and instead adds dedicated cache tests for those query builders that exhibit something other than the default behaviour. Closes #43200 [ML][DataFrame] Consider data frame templates internal in REST tests (#43692) The data frame index template pattern was not in the list considered as internal and therefore not needing cleanup after every test. Remove preconfigured `delimited_payload_filter` (#43686) cleanup of pre-version 7 restrictions, but missed removing the preconfigured version due to #43684. Deduplicate two similar TimeUtils classes. * Deduplicate org.elasticsearch.xpack.core.dataframe.utils.TimeUtils and org.elasticsearch.xpack.core.ml.utils.time.TimeUtils into a common class: org.elasticsearch.xpack.core.common.time.TimeUtils. * Add unit tests for parseTimeField and parseTimeFieldToInstant methods Fix UOE on search requests that match a sparse role query (#43668) Search requests executed through the SecurityIndexSearcherWrapper throw an UnsupportedOperationException if they match a sparse role query. When low level cancellation is activated (which is the default since #42857), the context index searcher creates a weight that doesn't handle #scorer. This change fixes this bug and adds a test to ensure that we check this case. Fix propagation of enablePositionIncrements in QueryStringQueryBuilder (#43578) This change fixes the propagation of the enablePositionIncrements option to the underlying QueryBuilder. Closes #43574 Remove deprecated sort options: nested_path and nested_filter (#42809) This commit removes the nested_path and nested_filter options deprecated in 6x. This change also checks that the sort field has a [nested] option if it is under a nested object and throws an exception if it's not the case. Closes #27098 [ML] Don't write timing stats on no-op (#43680) Similar to elastic/ml-cpp#512, if a job opens and closes and does nothing in between we shouldn't write timing stats to the results index. Handle situation where only voting-only nodes are bootstrapped (#43628) Adds support for the situation where only voting-only nodes are bootstrapped. In that case, they will still try to become elected and bring full master nodes into the cluster. Allow reloading of search time analyzers (#43313) Currently changing resources (like dictionaries, synonym files etc...) of search time analyzers is only possible by closing an index, changing the underlying resource (e.g. synonym files) and then re-opening the index for the change to take effect. This PR adds a new API endpoint that allows triggering reloading of certain analysis resources (currently token filters) that will then pick up changes in underlying file resources. To achieve this we introduce a new type of custom analyzer (ReloadableCustomAnalyzer) that uses a ReuseStrategy that allows swapping out analysis components. Custom analyzers that contain filters that are markes as "updateable" will automatically choose this implementation. This PR also adds this capability to `synonym` token filters for use in search time analyzers. Relates to #29051 [DOCS] Adds anchors and attributes to ML APIs Add version and create_time to data frame analytics config (#43683) Remove nodeId from BaseNodeRequest (#43658) TransportNodesAction provides a mechanism to easily broadcast a request to many nodes, and collect the respones into a high level response. Each node has its own request type, with a base class of BaseNodeRequest. This base request requires passing the nodeId to which the request will be sent. However, that nodeId is not used anywhere. It is private to the base class, yet serialized to each node, where the node could just as easily find the nodeId of the node it is on locally. This commit removes passing the nodeId through to the node request creation, and guards its serialization so that we can remove the base request class altogether in the future. Geo: Makes coordinate validator in libs/geo plugable (#43657) Moves coordinate validation from Geometry constructors into parser. Relates #43644 [DOCS] Updates ML APIs to use new API template (#43711) [DOCS] Adds data frame API response codes for allow_no_match (#43666) Do not use MockInternalEngine in GatewayIndexStateIT (#43716) GatewayIndexStateIT#testRecoverBrokenIndexMetadata replies on the flushing on shutdown. This behaviour, however, can be randomly disabled in MockInternalEngine. Closes #43034 [ML] Allowing stopped status in HLRC testStartStop (#43710) [FEATURE][ML] Support multiple source indices for df-analytics (#43702) This commit adds support for multiple source indices. In order to deal with multiple indices having different mappings, it attempts a best-effort approach to merge the mappings assuming there are no conflicts. In case conflicts exists an error will be returned. To allow users creating custom mappings for special use cases, the destination index is now allowed to exist before the analytics job runs. In addition, settings are no longer copied except for the `index.number_of_shards` and `index.number_of_replicas`. [ML] Rename outlier score setting to feature_influence_threshold (#43705) Renames outlier score setting `minimum_score_to_write_feature_influence` to `feature_influence_threshold`. [TEST][ML] Fix yml test failure for data_frame_analytics_crud Oddly, this passed CI on the PR that introduced it... Remove rests of StandardHtmlStripAnalyzer (#43485) StandardHtmlStripAnalyzer has been deprecated in 6.x and cannot be used for new indices from 7.0 on. This change removes it entirely and also removes the from tests and deprecation logging that has still been around during the 7.x versions. Enabled cannot be updated (#43701) Removed the invalid tip that enabled can be updated for existing fields and clarified instead that it cannot. Related to #33566 and #33933 Add support for 'flattened object' fields. (#42541) This commit merges the `object-fields` feature branch. The new 'flattened object' field type allows an entire JSON object to be indexed into a field, and provides limited search functionality over the field's contents. [DOCS] Rewrite boosting query (#43647) Wildcard intervals (#43691) This commit adds a wildcard intervals source, similar to the prefix. It also changes the term parameter in prefix to read prefix, to bring it in to line with the pattern parameter in wildcard. Closes #43198 [ML][Data Frame] removing format support (#43659) Add missing dependencies so we can build in parallel (#43672) Testclusters: Convert additional projects (#43625) * Testclusters: Convert additional projects Found some more that were not using testclusters from elasticsearch-ci/1 * Allow IOException too * Make the client more resilient Refactor IndexSearcherWrapper to disallow the wrapping of IndexSearcher (#43645) This change removes the ability to wrap an IndexSearcher in plugins. The IndexSearcherWrapper is replaced by an IndexReaderWrapper and allows to wrap the DirectoryReader only. This simplifies the creation of the context IndexSearcher that is used on a per request basis. This change also moves the optimization that was implemented in the security index searcher wrapper to the ContextIndexSearcher that now checks the live docs to determine how the search should be executed. If the underlying live docs is a sparse bit set the searcher will compute the intersection betweeen the query and the live docs instead of checking the live docs on every document that match the query. Fix threshold spelling errors (#43326) Substitutes treshold by threshold Trim translog for closed indices (#43156) Today when an index is closed all its shards are forced flushed but the translog files are left around. As explained in #42445 we'd like to trim the translog for closed indices in order to consume less disk space. This commit reuses the existing AsyncTrimTranslogTask task and reenables it for closed indices. At the time the task is executed, we should have the guarantee that nothing holds the translog files that are going to be removed. It also leaves a short period of time (10 min) during which translog files of a recently closed index are still present on disk. This could also help in some cases where the closed index is reopened shortly after being closed (in order to update an index setting for example). Relates to #42445 Convert some action.execute cases to using NodeClient (#43715) There are several cases where execute is called on actions directly, mostly within other action implementations. Now that transport client is gone, these internal actions can be registered just like normal actions, and called with the NodeClient to execute locally on the same node. This commit converts a few of these cases for shard level actions to use this approach. [DOCS] Rewrite `terms_set` query (#43060) Re-enable bwc tests (#43723) relates #43658 [ML] Mark ml-cpp dependency as regularly changing (#43760) Since #41817 was merged the ml-cpp zip file for any given version has been cached indefinitely by Gradle. This is problematic, particularly in the case of the master branch where the version 8.0.0-SNAPSHOT will be in use for more than a year. This change tells Gradle that the ml-cpp zip file is a "changing" dependency, and to check whether it has changed every two hours. Two hours is a compromise between checking on every build and annoying developers with slow internet connections and checking rarely causing bug fixes in the ml-cpp code to take a long time to propagate through to elasticsearch PRs that rely on them. Migrate watcher hlrc response tests to use AbstractResponseTestCase (#43478) Relates to #43472 Add StreamableResponseAction to aid in deprecation of Streamable (#43770) The Action base class currently works for both Streamable and Writeable response types. This commit intorduces StreamableResponseAction, for which only the legacy Action implementions which provide newResponse() will extend. This eliminates the need for overriding newResponse() with an UnsupportedOperationException. relates #34389 [ML] Rename df-analytics `_id_copy` to `ml__id_copy` (#43754) Renames `_id_copy` to `ml__id_copy` as field names starting with underscore are deprecated. The new field name `ml__id_copy` was chosen as an obscure enough field that users won't have in their data. Otherwise, this field is only intented to be used by df-analytics. [ML] Assert that a no-op job creates no results nor state (#43681) If a job is opened and then closed and does nothing in between then it should not persist any results or state documents. This change adapts the no-op job test to assert no results in addition to no state, and to log any documents that cause this assertion to fail. Relates elastic/ml-cpp#512 Relates #43680 Consistent Secure Settings (#40416) Introduces a new `ConsistentSecureSettingsValidatorService` service that exposes a single public method, namely `allSecureSettingsConsistent`. The method returns `true` if the local node's secure settings (inside the keystore) are equal to the master's, and `false` otherwise. Technically, the local node has to have exactly the same secure settings - setting names should not be missing or in surplus - for all `SecureSetting` instances that are flagged with the newly introduced `Property.Consistent`. It is worth highlighting that the `allSecureSettingsConsistent` is not a consensus view across the cluster, but rather the local node's perspective in relation to the master. Rename Action to ActionType (#43778) Action is a class that encapsulates meta information about an action that allows it to be called remotely, specifically the action name and response type. With recent refactoring, the action class can now be constructed as a static constant, instead of needing to create a subclass. This makes the old pattern of creating a singleton INSTANCE both misnamed and lacking a common placement. This commit renames Action to ActionType, thus allowing the old INSTANCE naming pattern to be TYPE on the transport action itself. ActionType also conveys that this class is also not the action itself, although this change does not rename any concrete classes as those will be removed organically as they are converted to TYPE constants. relates #34389 Make peer recovery clean files step async (#43787) Relates #36195 Reenable bwc tests after #40416 (#43794) Nothing to see here. AsyncIOProcessor preserve thread context (#43729) AsyncIOProcessor now preserves thread context, ensuring that deprecation warnings are not duplicated to other concurrent operations on the same shard. Avoid parallel reroutes in DiskThresholdMonitor (#43381) Today the `DiskThresholdMonitor` limits the frequency with which it submits reroute tasks, but it might still submit these tasks faster than the master can process them if, for instance, each reroute takes over 60 seconds. This causes a problem since the reroute task runs with priority `IMMEDIATE` and is always scheduled when there is a node over the high watermark, so this can starve any other pending tasks on the master. This change avoids further updates from the monitor while its last task(s) are still in progress, and it measures the time of each update from the completion time of the reroute task rather than its start time, to allow a larger window for other tasks to run. It also now makes use of the `RoutingService` to submit the reroute task, in order to batch this task with any other pending reroutes. It enhances the `RoutingService` to notify its listeners on completion. Fixes #40174 Relates #42559 [ML] Correct df-analytics version introduced to 7.3.0 (#43784) Make peer recovery send file info step async (#43792) Relates #36195 show a full ingest example in the index page, to let user fast understand ingest node. (#43476) Async IO Processor release before notify (#43682) This commit changes async IO processor to release the promiseSemaphore before notifying consumers. This ensures that a bad consumer that sometimes does blocking (or otherwise slow) operations does not halt the processor. This should slightly increase the concurrency for shard fsync, but primarily improves safety so that one bad piece of code has less effect on overall system performance. relax trigger count for transform stats test (#43753) relax trigger count test as we can not guarantee it due to async behaviour Testclusters: improove timeout handling (#43440) Switch WriteActionsTests.testBulk to use hamcrest (#43725) If an item in the bulk request fails, that could be for a variety of reasons - it may be that the underlying behaviour of security has changed, or it may just be a transient failure during testing. Simply asserting a `true`/`false` value produces failure messages that are difficult to diagnose and debug. Using hamcert (`assertThat`) will make it easier to understand the causes of failures in this test. Add "manage_api_key" cluster privilege (#43728) This adds a new cluster privilege for manage_api_key. Users with this privilege are able to create new API keys (as a child of their own user identity) and may also get and invalidate any/all API keys (including those owned by other users). Update TLS configuration in Docker docs (#43748) Following the removal of the `unzip` package from the Elasticsearch Docker image in #39040, update setup instructions for TLS in Docker. Also avoid cross-platform ownership+permission issues by not relying on local bind mounts for storing generated certs and don't require `curl` locally installed. HLRC changes for kerberos grant type (#43642) The TODO from last PR for kerbero grant type was missed. This commit adds the changes for kerberos grant type in HLRC. Avoid IP addresses for bootstrapping in setup docs (#43802) Removes the suggestion to use IP addresses for `cluster.initial_master_nodes` in the "important settings" discovery docs, leaving only the suggestion to use node names. Relates #41179, #41569 Expose translog stats in ReadOnlyEngine (#43752) [ML][Data Frame] reduce audit frequency, change log msg, and level (#43771) [ML][Data Frame] account for delay in writing stats docs (#43703) Optimize Azure Directory Delete (#43341) * Follow up to #43281: * Optimizing the Azure directory delete operation: * Same as with GCS and S3 we can simply flat list a prefix and then delete as we iterate instead of listing the directories recursively. This should require fewer actual list RPC calls and the logic becomes simpler Update docs for Open/Close API (#43809) Relates #43530 Add RareTerms aggregation (#35718) This adds a `rare_terms` aggregation. It is an aggregation designed to identify the long-tail of keywords, e.g. terms that are "rare" or have low doc counts. This aggregation is designed to be more memory efficient than the alternative, which is setting a terms aggregation to size: LONG_MAX (or worse, ordering a terms agg by count ascending, which has unbounded error). This aggregation works by maintaining a map of terms that have been seen. A counter associated with each value is incremented when we see the term again. If the counter surpasses a predefined threshold, the term is removed from the map and inserted into a cuckoo filter. If a future term is found in the cuckoo filter we assume it was previously removed from the map and is "common". The map keys are the "rare" terms after collection is done. Yet another `the the` cleanup (#43815) Revert "[TEST] Mute RemoteClusterServiceTests.testCollectNodes" This reverts commit d184056bf5cf9968db95c16cb27f6f2ec505bc3e. Update rare_term version skips, fix SetBackedScalingCuckooFilter javadoc Add more logging to investigate #41067 Upgrade to Gradle 5.5 (#43788) Fix logger usage check SmokeTestWatcherWithSecurityIT: Retry if failures searching (#43781) Retry if failures searching .watcher-history Catch exceptions and fail (which allows the assertBusy to retry) if an error happens from the search against .watcher-history. Related: #42409 Related: #39169 [ML][Data Frame] using transform creation version for node assignment (#43764) * [ML][Data Frame] using transform creation version for node assignment * removing unused imports * Addressing PR comment [ML][Data Frame] Add deduced mappings to _preview response payload (#43742) * [ML][Data Frame] Add deduced mappings to _preview response payload * updating preview docs Add timezone documentation for Painless datetimes (#43768) Document API-Key APIs require manage_api_key priv (#43811) Add the "Authorization" section to the API key API docs. These APIs require The new manage_api_key cluster privilege. Relates: #43728 Rename and refactor RoutingService (#43827) The `RoutingService` has a confusing name, since it doesn't really have anything to do with routing. Its responsibility is submitting reroute commands to the master. This commit renames this class to `BatchedRerouteService`, and extracts the `RerouteService` interface to avoid passing `BiConsumer`s everywhere. It also removes that `BatchedRerouteService extends AbstractLifecycleComponent` since this service has no meaningful lifecycle. Finally, it introduces a small wrapper class to allow for lazy initialization to deal with the dependency loop when constructing a `Node`. Update docs to refer to 6.8 instead of 6.7 (#43685) A few places in the documentation had mentioned 6.7 as the version to upgrade from, when doing an upgrade to 7.0. While this is technically possible, this commit will replace all those mentions to 6.8, as this is the latest version with the latest bugfixes, deprecation checks and ugprade assistant features - which should be the one used for upgrades. Co-Authored-By: James Rodewig Use explicit string keys in data_frame test (#43824) Disable BWC tests (#43856) Until #43823 is backported to 7.x Fix credentials encoding for OIDC token request (#43808) As defined in https://tools.ietf.org/html/rfc6749#section-2.3.1 both client id and client secret need to be encoded with the application/x-www-form-urlencoded encoding algorithm when used as credentials for HTTP Basic Authentication in requests to the OP. Resolves #43709 [ML] Get df-analytics action should require monitor privilege (#43831) Fix DieWithDignity test when waiting on jps (#43861) the test often hangs on executing jps command we don't need to wait for this command to finish. closes #43413 Remove sort by primary term when reading soft-deletes (#43845) With Lucene rollback (#33473), we should never have more than one primary term for each sequence number. Therefore we don't have to sort by the primary term when reading soft-deletes. Return reloaded analyzers in _reload_search_ananlyzer response (#43813) Currently the repsonse of the "_reload_search_analyzer" endpoint contains the index names and nodeIds of indices were analyzers reloading was triggered. This change add the names of the search-time analyzers that were reloaded. Closes #43804 [ML][Data Frame] fix progress measurement for continuous transforms (#43838) * [ML][Data Frame] fix progress measurement for continuous transforms * Update DataFrameIndexer.java Clarify voting-only master node docs (#43857) Clarifies the roles of a dedicated voting-only master-eligible node. Co-Authored-By: James Rodewig Co-Authored-By: David Turner [ML-Data Frame] Add data frame transform cluster privileges to HLRC (#43879) Adds the monitor_data_frame_transforms and manage_data_frame_transforms cluster privileges to the high level rest client. The ALL_ARRAY variable is only used in randomized tests at the within the Elasticsearch code, so it's not a major problem that these cluster privileges weren't added from the start. But since ALL_ARRAY is public HLRC users may be using it to find out which cluster privileges exist, so it's best that it contains them all. Reenable bwc tests (#43872) This reenables the BWC tests disabled in #43856 Link rare_terms docs from index page (#43882) Docs for rare_terms were added in #35718, but neglected to link it from the bucket index page Dry up inputstream to bytesreference (#43675) * Dry up Reading InputStream to BytesReference * Dry up spots where we use the same pattern to get from an InputStream to a BytesReferences Convert replication calls from action.execute to nodeclient (#43834) Replication relies on some internal actions to do local replication operations. This commit converts these actions to registered transport actions so NodeClient can be used. [ML][Data Frame] add node attr to GET _stats (#43842) * [ML][Data Frame] add node attr to GET _stats * addressing testing issues with node.attributes Add dims parameter to dense_vector mapping (#43444) Typically, dense vectors of both documents and queries must have the same number of dimensions. Different number of dimensions among documents or query vector indicate an error. This PR enforces that all vectors for the same field have the same number of dimensions. It also enforces that query vectors have the same number of dimensions. [DOCS] Revise GS intro & remove redundant conceptual info. Closes #43846 (#43847) * [DOCS] Revise GS intro and remove redundant conceptual content. Closes #43846. * [DOCS] Incorporated feedback. * [DOCS] Abbreviated titles for parts don't work in AsciiDoctor. Add Datetime Now to Painless Documentation (#43852) This change explains why Painless doesn't natively support datetime now, and gives examples of how to create a version of now through user-defined parameters. Enable caching of rest tests which use integ-test distribution (#43782) Extend timeout for TimeSeriesLifecycleActionsIT> testFullPolicy (#43891) ilm test - allow more time for policy completion (#43844) Always attach system user to internal actions (#43468) All valid licenses permit security, and the only license state where we don't support security is when there is a missing license. However, for safety we should attach the system (or xpack/security) user to internally originated actions even if the license is missing (or, more strictly, doesn't support security). This allows all nodes to communicate and send internal actions (shard state, handshake/pings, etc) even if a license is transitioning between a broken state and a valid state. Relates: #42215 Use separate BitSet cache in Doc Level Security (#43669) Document level security was depending on the shared "BitsetFilterCache" which (by design) never expires its entries. However, when using DLS queries - particularly templated ones - the number (and memory usage) of generated bitsets can be significant. This change introduces a new cache specifically for BitSets used in DLS queries, that has memory usage constraints and access time expiry. The whole cache is automatically cleared if the role cache is cleared. Individual bitsets are cleared when the corresponding lucene index reader is closed. The cache defaults to 50MB, and entries expire if unused for 7 days. Support builtin privileges in get privileges API (#42134) Adds a new "/_security/privilege/_builtin" endpoint so that builtin index and cluster privileges can be retrieved via the Rest API Resolves: #29771 Refresh translog stats after translog trimming in NoOpEngine (#43825) This commit changes NoOpEngine so that it refreshes its translog stats once translog is trimmed. Relates #43156 Fix index_prefix sub field name on nested text fields (#43862) This change fixes the name of the index_prefix sub field when the `index_prefix` option is set on a text field that is nested under an object or a multi-field. We don't use the full path of the parent field to set the index_prefix field name so the field is registered under the wrong name. This doesn't break queries since we always retrieve the prefix field through its parent field but this breaks other APIs like _field_caps which tries to find the parent of the `index_prefix` field in the mapping but fails. Closes #43741 Adapt version check after backport Relates #43862 [ML] Set df-analytics task state to failed when appropriate (#43880) This introduces a `failed` state to which the data frame analytics persistent task is set to when something unexpected fails. It could be the process crashing, the results processor hitting some error, etc. The failure message is then captured and set on the task state. From there, it becomes available via the _stats API as `failure_reason`. The df-analytics stop API now has a `force` boolean parameter. This allows the user to call it for a failed task in order to reset it to `stopped` after we have ensured the failure has been communicated to the user. This commit also adds the analytics version in the persistent task params as this allows us to prevent tasks to run on unsuitable nodes in the future. Watcher: Allow to execute actions for each element in array (#41997) This adds the ability to execute an action for each element that occurs in an array, for example you could sent a dedicated slack action for each search hit returned from a search. There is also a limit for the number of actions executed, which is hardcoded to 100 right now, to prevent having watches run forever. The watch history logs each action result and the total number of actions the were executed. Relates #34546 Add _reload_search_analyzers endpoint to HLRC (#43733) This change adds the new endpoint that allows reloading of search analyzers to the high-level java rest client. Relates to #43313 Remove the client transport profile filter (#43236) Now that the transport client has been removed, the client transport profile filter can be removed from security. This filter prevented node actions from being executed using a transport client. Ensure to access RecoveryState#fileDetails under lock Closes #43840 [DOCS] Rewrite dis max query (#43586) Move CORS Config into :server package (#43779) This commit moves the config that stores Cors options into the server package. Currently both nio and netty modules must have a copy of this config. Moving it into server allows one copy and the tests to be in a common location. Shortcut simple patterns ending in `*` (#43904) When profiling a call to `AllocationService#reroute()` in a large cluster containing allocation filters of the form `node-name-*` I observed a nontrivial amount of time spent in `Regex#simpleMatch` due to these allocation filters. Patterns ending in a wildcard are not uncommon, and this change treats them as a special case in `Regex#simpleMatch` in order to shave a bit of time off this calculation. It also uses `String#regionMatches()` to avoid an allocation in the case that the pattern's only wildcard is at the start. Microbenchmark results before this change: Result "org.elasticsearch.common.regex.RegexStartsWithBenchmark.performSimpleMatch": 1113.839 ±(99.9%) 6.338 ns/op [Average] (min, avg, max) = (1102.388, 1113.839, 1135.783), stdev = 9.486 CI (99.9%): [1107.502, 1120.177] (assumes normal distribution) Microbenchmark results with this change applied: Result "org.elasticsearch.common.regex.RegexStartsWithBenchmark.performSimpleMatch": 433.190 ±(99.9%) 0.644 ns/op [Average] (min, avg, max) = (431.518, 433.190, 435.456), stdev = 0.964 CI (99.9%): [432.546, 433.833] (assumes normal distribution) The microbenchmark in question was: @Fork(3) @Warmup(iterations = 10) @Measurement(iterations = 10) @BenchmarkMode(Mode.AverageTime) @OutputTimeUnit(TimeUnit.NANOSECONDS) @State(Scope.Benchmark) @SuppressWarnings("unused") //invoked by benchmarking framework public class RegexStartsWithBenchmark { private static final String testString = "abcdefghijklmnopqrstuvwxyz"; private static final String[] patterns; static { patterns = new String[testString.length() + 1]; for (int i = 0; i <= testString.length(); i++) { patterns[i] = testString.substring(0, i) + "*"; } } @Benchmark public void performSimpleMatch() { for (int i = 0; i < patterns.length; i++) { Regex.simpleMatch(patterns[i], testString); } } } Mute failing test Tracked in #43924 Actually close IndexAnalyzers contents (#43914) IndexAnalyzers has a close() method that should iterate through all its wrapped analyzers and close each one in turn. However, instead of delegating to the analyzers' close() methods, it instead wraps them in a Closeable interface, which just returns a list of the analyzers. In addition, whitespace normalizers are ignored entirely. Geo: Refactors libs/geo parser to provide serialization logic as well (#43717) Enables libs/geo parser to return a geometry format object that can perform both serialization and deserialization functions. This can be useful for ingest nodes that are trying to modify an existing geometry in the source. Relates to #43554 [ML][Data Frame] Adding bwc tests for pivot transform (#43506) * [ML][Data Frame] Adding bwc tests for pivot transform * adding continuous transforms * adding continuous dataframes to bwc * adding continuous data frame tests * Adding rolling upgrade tests for continuous df * Fixing test * Adjusting indices used in BWC, and handling NPE for seq_no_stats * updating and muting specific bwc test Enable validating user-supplied missing values on unmapped fields (#43718) Provides a hook for aggregations to introspect the `ValuesSourceType` for a user supplied Missing value on an unmapped field, when the type would otherwise be `ANY`. Mapped field behavior is unchanged, and still applies the `ValuesSourceType` of the field. This PR just provides the hook for doing this, no existing aggregations have their behavior changed. Add version 7.4. (#43930) Relates #43927. Adjust version in vectors tests after backport After backport to 7.3 adjust yml tests Relates to #43444 Remove generic on AggregatorFactory (#43664) AggregatorFactory was generic over itself, but it doesn't appear we use this functionality anywhere (e.g. to allow the super class to declare arguments/return types generically for subclasses to override). Most places use a wildcard constraint, and even when a concrete type is specified it wasn't used. But since AggFactories are widely used, this led to the generic touching many pieces of code and making type signatures fairly complex Ensure test cluster classpath inputs have predictable ordering (#43938) --- .ci/java-versions.properties | 1 + .ci/matrix-runtime-javas.yml | 1 + .github/PULL_REQUEST_TEMPLATE.md | 2 +- CONTRIBUTING.md | 3 +- README.textile | 6 +- TESTING.asciidoc | 6 - benchmarks/build.gradle | 2 +- .../routing/allocation/Allocators.java | 5 +- build.gradle | 99 +- buildSrc/build.gradle | 44 +- .../elasticsearch/gradle/BuildPlugin.groovy | 158 +- .../gradle/doc/SnippetsTask.groovy | 6 +- .../gradle/plugin/PluginBuildPlugin.groovy | 36 +- .../plugin/PluginPropertiesExtension.groovy | 94 - .../precommit/DependencyLicensesTask.groovy | 268 -- .../gradle/precommit/PrecommitTasks.groovy | 38 +- .../gradle/precommit/UpdateShasTask.groovy | 66 - .../gradle/test/ClusterFormationTasks.groovy | 8 +- .../gradle/test/MessyTestPlugin.groovy | 63 - .../elasticsearch/gradle/test/NodeInfo.groovy | 5 + .../gradle/test/RestIntegTestTask.groovy | 11 +- .../test/StandaloneRestTestPlugin.groovy | 5 +- .../AbstractLazyPropertyCollection.java | 27 + .../org/elasticsearch/gradle/BwcVersions.java | 7 +- .../gradle/DistributionDownloadPlugin.java | 275 ++ .../gradle/ElasticsearchDistribution.java | 230 ++ .../elasticsearch/gradle/EmptyDirTask.java | 83 + .../gradle/JdkDownloadPlugin.java | 8 +- .../gradle/LazyPropertyList.java | 205 ++ .../elasticsearch/gradle/LazyPropertyMap.java | 167 + .../gradle/PropertyNormalization.java | 13 + ...emPropertyCommandLineArgumentProvider.java | 30 + .../gradle/http/WaitForHttpResource.java | 20 +- .../gradle/info/GlobalBuildInfoPlugin.java | 2 +- .../plugin/PluginPropertiesExtension.java | 141 + .../precommit/DependencyLicensesTask.java | 328 ++ .../gradle/precommit/UpdateShasTask.java | 86 + .../gradle/test/RestTestRunnerTask.java | 37 + .../testclusters/ElasticsearchCluster.java | 93 +- .../testclusters/ElasticsearchNode.java | 500 ++- .../TestClusterCleanupOnShutdown.java | 59 + .../TestClusterConfiguration.java | 37 +- .../TestClustersCleanupExtension.java | 74 + .../testclusters/TestClustersPlugin.java | 149 +- .../testfixtures/TestFixturesPlugin.java | 121 +- .../gradle/tool/ClasspathUtils.java | 23 + ...icsearch.distribution-download.properties} | 2 +- buildSrc/src/main/resources/buildSrc.marker | 0 .../src/main/resources/minimumGradleVersion | 2 +- .../elasticsearch/gradle/JdkJarHellCheck.java | 0 .../gradle/LazyFileOutputStream.java | 0 .../org/elasticsearch/gradle/LoggedExec.java | 9 +- .../org/elasticsearch/gradle/Version.java | 0 .../gradle/VersionProperties.java | 0 .../gradle/DistributionDownloadPluginIT.java | 136 + .../DistributionDownloadPluginTests.java | 260 ++ .../gradle/EmptyDirTaskTests.java | 78 + .../PluginPropertiesExtensionTests.java | 58 + .../DependencyLicensesTaskTests.java | 268 ++ .../gradle/precommit/UpdateShasTaskTests.java | 140 + .../distribution-download/build.gradle | 44 + .../distribution/build.gradle | 35 + .../files/fake_elasticsearch.tar.gz | Bin 0 -> 188 bytes .../distribution/files/fake_elasticsearch.zip | Bin 0 -> 568 bytes .../distribution-download/settings.gradle | 8 + .../subproj/build.gradle | 58 + client/benchmark/README.md | 4 +- client/benchmark/build.gradle | 10 +- .../client/benchmark/BenchmarkMain.java | 4 - .../transport/TransportClientBenchmark.java | 127 - .../noop/action/bulk/NoopBulkAction.java | 4 +- .../noop/action/search/NoopSearchAction.java | 9 +- client/rest-high-level/build.gradle | 60 +- .../client/CcrRequestConverters.java | 3 +- .../client/ClusterRequestConverters.java | 12 +- .../client/DataFrameRequestConverters.java | 42 +- .../IndexLifecycleRequestConverters.java | 27 +- .../elasticsearch/client/IndicesClient.java | 30 +- .../client/IndicesRequestConverters.java | 137 +- .../client/IngestRequestConverters.java | 14 +- .../client/LicenseRequestConverters.java | 15 +- .../client/MLRequestConverters.java | 156 +- .../client/MachineLearningClient.java | 294 ++ .../client/RequestConverters.java | 116 +- .../client/RollupRequestConverters.java | 13 +- .../elasticsearch/client/SecurityClient.java | 30 + .../client/SecurityRequestConverters.java | 38 +- .../client/SnapshotRequestConverters.java | 35 +- .../client/TasksRequestConverters.java | 9 +- .../client/WatcherRequestConverters.java | 10 +- .../org/elasticsearch/client/XPackClient.java | 4 +- .../client/XPackRequestConverters.java | 3 +- .../client/ccr/IndicesFollowStats.java | 80 +- .../client/core/BroadcastResponse.java | 8 +- .../client/core/TermVectorsResponse.java | 30 +- .../DataFrameNamedXContentProvider.java | 29 +- .../GetDataFrameTransformRequest.java | 16 +- .../GetDataFrameTransformStatsRequest.java | 15 +- .../PreviewDataFrameTransformResponse.java | 19 +- .../StartDataFrameTransformResponse.java | 12 +- .../StopDataFrameTransformRequest.java | 14 +- .../StopDataFrameTransformResponse.java | 13 +- .../transforms/DataFrameTransformConfig.java | 94 +- .../transforms/DataFrameTransformState.java | 18 +- .../dataframe/transforms/DestConfig.java | 54 +- .../dataframe/transforms/NodeAttributes.java | 156 + .../dataframe/transforms/SyncConfig.java | 30 + .../dataframe/transforms/TimeSyncConfig.java | 108 + .../pivot/DateHistogramGroupSource.java | 35 +- .../dataframe/transforms/util/TimeUtil.java | 59 + .../client/indices/AnalyzeRequest.java | 343 ++ .../client/indices/AnalyzeResponse.java | 183 ++ .../client/indices/DetailAnalyzeResponse.java | 214 ++ .../indices/ReloadAnalyzersRequest.java | 68 + .../indices/ReloadAnalyzersResponse.java | 108 + .../ml/DeleteDataFrameAnalyticsRequest.java | 64 + .../client/ml/EvaluateDataFrameRequest.java | 136 + .../client/ml/EvaluateDataFrameResponse.java | 117 + .../client/ml/FindFileStructureRequest.java | 18 +- .../ml/GetDataFrameAnalyticsRequest.java | 104 + .../ml/GetDataFrameAnalyticsResponse.java | 74 + .../ml/GetDataFrameAnalyticsStatsRequest.java | 99 + .../GetDataFrameAnalyticsStatsResponse.java | 102 + .../client/ml/NodeAttributes.java | 6 + .../ml/PutDataFrameAnalyticsRequest.java | 76 + .../ml/PutDataFrameAnalyticsResponse.java | 57 + .../ml/StartDataFrameAnalyticsRequest.java | 74 + .../ml/StopDataFrameAnalyticsRequest.java | 100 + .../ml/StopDataFrameAnalyticsResponse.java | 87 + .../ml/dataframe/DataFrameAnalysis.java | 27 + .../dataframe/DataFrameAnalyticsConfig.java | 259 ++ .../ml/dataframe/DataFrameAnalyticsDest.java | 123 + .../dataframe/DataFrameAnalyticsSource.java | 128 + .../ml/dataframe/DataFrameAnalyticsState.java | 34 + .../ml/dataframe/DataFrameAnalyticsStats.java | 145 + ...ataFrameAnalysisNamedXContentProvider.java | 37 + .../client/ml/dataframe/OutlierDetection.java | 175 + .../client/ml/dataframe/QueryConfig.java | 82 + .../ml/dataframe/evaluation/Evaluation.java | 32 + .../evaluation/EvaluationMetric.java | 32 +- .../MlEvaluationNamedXContentProvider.java | 57 + .../AbstractConfusionMatrixMetric.java | 47 + .../softclassification/AucRocMetric.java | 241 ++ .../BinarySoftClassification.java | 129 + .../ConfusionMatrixMetric.java | 206 ++ .../softclassification/PrecisionMetric.java | 123 + .../softclassification/RecallMetric.java | 123 + .../ml/filestructurefinder/FieldStats.java | 30 +- .../client/ml/job/process/TimingStats.java | 159 + .../client/ml/job/stats/JobStats.java | 23 +- .../client/security/CreateTokenRequest.java | 34 +- .../client/security/CreateTokenResponse.java | 19 +- .../security/GetBuiltinPrivilegesRequest.java | 40 + .../GetBuiltinPrivilegesResponse.java | 82 + .../client/security/user/privileges/Role.java | 7 +- .../client/xpack/XPackInfoResponse.java | 10 +- ...icsearch.plugins.spi.NamedXContentProvider | 5 +- ...AbstractHlrcWriteableXContentTestCase.java | 70 + .../client/AbstractResponseTestCase.java | 6 +- .../elasticsearch/client/BulkProcessorIT.java | 2 - .../BulkRequestWithGlobalParametersIT.java | 5 - .../DataFrameRequestConvertersTests.java | 26 +- .../client/DataFrameTransformIT.java | 77 +- .../client/ESRestHighLevelClientTestCase.java | 2 +- .../elasticsearch/client/IndicesClientIT.java | 20 +- .../client/IndicesRequestConvertersTests.java | 30 +- .../client/MLRequestConvertersTests.java | 137 +- .../client/MachineLearningIT.java | 517 ++- .../client/MlTestStateCleaner.java | 13 + .../elasticsearch/client/PingAndInfoIT.java | 13 +- .../client/RequestConvertersTests.java | 65 +- .../client/RestHighLevelClientTests.java | 28 +- .../org/elasticsearch/client/RollupIT.java | 22 +- .../SecurityRequestConvertersTests.java | 4 +- .../org/elasticsearch/client/SnapshotIT.java | 109 +- .../SnapshotRequestConvertersTests.java | 18 +- .../client/XPackInfoResponseTests.java | 19 +- .../client/ccr/CcrStatsResponseTests.java | 3 + .../client/ccr/FollowStatsResponseTests.java | 2 + .../client/core/TermVectorsResponseTests.java | 8 +- .../GetDataFrameTransformResponseTests.java | 7 +- ...PreviewDataFrameTransformRequestTests.java | 8 +- ...reviewDataFrameTransformResponseTests.java | 8 +- .../PutDataFrameTransformRequestTests.java | 8 +- .../DataFrameTransformConfigTests.java | 25 +- .../DataFrameTransformStateTests.java | 11 +- .../dataframe/transforms/DestConfigTests.java | 3 +- .../transforms/NodeAttributesTests.java | 64 + .../transforms/QueryConfigTests.java | 2 +- .../transforms/SourceConfigTests.java | 2 +- .../transforms/TimeSyncConfigTests.java | 41 +- .../DataFrameTransformStateAndStatsTests.java | 2 +- .../hlrc/DataFrameTransformStateTests.java | 36 +- .../transforms/hlrc/TimeSyncConfigTests.java | 59 + .../pivot/AggregationConfigTests.java | 2 +- .../pivot/DateHistogramGroupSourceTests.java | 1 - .../transforms/pivot/PivotConfigTests.java | 2 +- .../hlrc/DateHistogramGroupSourceTests.java | 4 - .../documentation/CRUDDocumentationIT.java | 25 +- .../DataFrameTransformDocumentationIT.java | 41 +- .../IndicesClientDocumentationIT.java | 136 +- .../LicensingDocumentationIT.java | 4 +- .../MlClientDocumentationIT.java | 570 +++- .../QueryDSLDocumentationTests.java | 8 - .../SecurityDocumentationIT.java | 61 +- .../SnapshotClientDocumentationIT.java | 6 +- .../indices/AnalyzeGlobalRequestTests.java | 58 + .../indices/AnalyzeIndexRequestTests.java | 62 + .../client/indices/AnalyzeRequestTests.java | 54 + .../client/indices/AnalyzeResponseTests.java | 174 + .../indices/ReloadAnalyzersResponseTests.java | 111 + .../license/PutLicenseResponseTests.java | 11 +- .../license/StartBasicResponseTests.java | 11 +- .../ml/AucRocMetricAucRocPointTests.java | 39 +- .../client/ml/AucRocMetricResultTests.java | 63 + ...usionMatrixMetricConfusionMatrixTests.java | 47 + .../ml/ConfusionMatrixMetricResultTests.java | 62 + .../DeleteDataFrameAnalyticsRequestTests.java | 39 + .../ml/EvaluateDataFrameResponseTests.java | 76 + .../ml/FindFileStructureRequestTests.java | 4 + .../ml/GetDataFrameAnalyticsRequestTests.java | 39 + ...etDataFrameAnalyticsStatsRequestTests.java | 39 + .../client/ml/PrecisionMetricResultTests.java | 60 + .../ml/PutDataFrameAnalyticsRequestTests.java | 74 + .../client/ml/RecallMetricResultTests.java | 60 + .../StartDataFrameAnalyticsRequestTests.java | 43 + .../StopDataFrameAnalyticsRequestTests.java | 43 + .../StopDataFrameAnalyticsResponseTests.java | 34 +- .../DataFrameAnalyticsConfigTests.java | 96 + .../DataFrameAnalyticsDestTests.java | 50 + .../DataFrameAnalyticsSourceTests.java | 70 + .../DataFrameAnalyticsStatsTests.java | 70 + .../ml/dataframe/OutlierDetectionTests.java | 73 + .../client/ml/dataframe/QueryConfigTests.java | 62 + .../filestructurefinder/FieldStatsTests.java | 10 +- .../ml/job/config/AnalysisConfigTests.java | 2 +- .../client/ml/job/config/JobTests.java | 2 +- .../ml/job/process/TimingStatsTests.java | 97 + .../client/ml/job/stats/JobStatsTests.java | 6 +- .../security/CreateTokenRequestTests.java | 51 +- .../security/CreateTokenResponseTests.java | 5 + .../watcher/ExecuteWatchResponseTests.java | 115 - .../client/watcher/GetWatchResponseTests.java | 144 +- .../hlrc/DeleteWatchResponseTests.java | 28 +- .../hlrc/ExecuteWatchResponseTests.java | 28 +- .../watcher/hlrc/PutWatchResponseTests.java | 32 +- client/rest/build.gradle | 4 +- .../org/elasticsearch/client/Request.java | 4 + .../elasticsearch/client/RequestOptions.java | 9 +- .../client/RestClientBuilderIntegTests.java | 2 +- client/sniffer/build.gradle | 6 +- client/test/build.gradle | 2 +- client/transport/build.gradle | 57 - .../client/PreBuiltTransportClient.java | 144 - .../client/PreBuiltTransportClientTests.java | 66 - distribution/archives/build.gradle | 8 +- distribution/build.gradle | 12 +- distribution/docker/build.gradle | 11 +- .../src/docker/config/log4j2.properties | 121 + .../src/docker/config/log4j2.properties | 0 distribution/src/bin/elasticsearch.bat | 2 +- distribution/tools/keystore-cli/build.gradle | 6 +- .../common/settings/KeyStoreWrapperTests.java | 23 + distribution/tools/plugin-cli/build.gradle | 6 +- docs/README.asciidoc | 6 +- docs/build.gradle | 9 + docs/groovy-api/anatomy.asciidoc | 102 - docs/groovy-api/client.asciidoc | 59 - docs/groovy-api/delete.asciidoc | 16 - docs/groovy-api/get.asciidoc | 19 - docs/groovy-api/index.asciidoc | 48 - docs/groovy-api/index_.asciidoc | 32 - docs/groovy-api/search.asciidoc | 116 - docs/java-api/admin/cluster/health.asciidoc | 76 - docs/java-api/admin/cluster/index.asciidoc | 16 - .../admin/cluster/stored-scripts.asciidoc | 29 - docs/java-api/admin/index.asciidoc | 18 - .../admin/indices/create-index.asciidoc | 28 - .../admin/indices/get-settings.asciidoc | 22 - docs/java-api/admin/indices/index.asciidoc | 21 - .../admin/indices/put-mapping.asciidoc | 30 - docs/java-api/admin/indices/refresh.asciidoc | 19 - .../admin/indices/update-settings.asciidoc | 16 - docs/java-api/aggregations/bucket.asciidoc | 33 - .../bucket/children-aggregation.asciidoc | 35 - .../bucket/datehistogram-aggregation.asciidoc | 73 - .../bucket/daterange-aggregation.asciidoc | 59 - .../bucket/filter-aggregation.asciidoc | 34 - .../bucket/filters-aggregation.asciidoc | 51 - .../bucket/geodistance-aggregation.asciidoc | 58 - .../bucket/geohashgrid-aggregation.asciidoc | 57 - .../bucket/global-aggregation.asciidoc | 35 - .../bucket/histogram-aggregation.asciidoc | 48 - .../bucket/iprange-aggregation.asciidoc | 79 - .../bucket/missing-aggregation.asciidoc | 34 - .../bucket/nested-aggregation.asciidoc | 34 - .../bucket/range-aggregation.asciidoc | 58 - .../reverse-nested-aggregation.asciidoc | 50 - .../significantterms-aggregation.asciidoc | 47 - .../bucket/terms-aggregation.asciidoc | 97 - docs/java-api/aggregations/metrics.asciidoc | 27 - .../metrics/avg-aggregation.asciidoc | 37 - .../metrics/cardinality-aggregation.asciidoc | 38 - .../extendedstats-aggregation.asciidoc | 44 - .../metrics/geobounds-aggregation.asciidoc | 46 - .../metrics/max-aggregation.asciidoc | 37 - .../metrics/min-aggregation.asciidoc | 37 - .../metrics/percentile-aggregation.asciidoc | 68 - .../percentile-rank-aggregation.asciidoc | 55 - .../scripted-metric-aggregation.asciidoc | 100 - .../metrics/stats-aggregation.asciidoc | 41 - .../metrics/sum-aggregation.asciidoc | 37 - .../metrics/tophits-aggregation.asciidoc | 79 - .../metrics/valuecount-aggregation.asciidoc | 37 - docs/java-api/aggs.asciidoc | 63 - docs/java-api/client.asciidoc | 110 - docs/java-api/docs.asciidoc | 36 - docs/java-api/docs/bulk.asciidoc | 190 -- docs/java-api/docs/delete.asciidoc | 42 - docs/java-api/docs/get.asciidoc | 14 - docs/java-api/docs/index_.asciidoc | 167 - docs/java-api/docs/multi-get.asciidoc | 30 - docs/java-api/docs/reindex.asciidoc | 11 - docs/java-api/docs/update-by-query.asciidoc | 166 - docs/java-api/docs/update.asciidoc | 118 - docs/java-api/index.asciidoc | 149 - docs/java-api/query-dsl.asciidoc | 40 - docs/java-api/query-dsl/bool-query.asciidoc | 13 - .../query-dsl/boosting-query.asciidoc | 12 - .../query-dsl/common-terms-query.asciidoc | 11 - .../query-dsl/compound-queries.asciidoc | 45 - .../query-dsl/constant-score-query.asciidoc | 11 - .../java-api/query-dsl/dis-max-query.asciidoc | 13 - docs/java-api/query-dsl/exists-query.asciidoc | 10 - .../query-dsl/full-text-queries.asciidoc | 44 - .../query-dsl/function-score-query.asciidoc | 19 - docs/java-api/query-dsl/fuzzy-query.asciidoc | 11 - .../query-dsl/geo-bounding-box-query.asciidoc | 12 - .../query-dsl/geo-distance-query.asciidoc | 12 - .../query-dsl/geo-polygon-query.asciidoc | 11 - docs/java-api/query-dsl/geo-queries.asciidoc | 34 - .../query-dsl/geo-shape-query.asciidoc | 56 - .../query-dsl/has-child-query.asciidoc | 23 - .../query-dsl/has-parent-query.asciidoc | 23 - docs/java-api/query-dsl/ids-query.asciidoc | 10 - .../query-dsl/joining-queries.asciidoc | 28 - .../query-dsl/match-all-query.asciidoc | 9 - docs/java-api/query-dsl/match-query.asciidoc | 11 - docs/java-api/query-dsl/mlt-query.asciidoc | 13 - .../query-dsl/multi-match-query.asciidoc | 11 - docs/java-api/query-dsl/nested-query.asciidoc | 12 - .../query-dsl/percolate-query.asciidoc | 61 - docs/java-api/query-dsl/prefix-query.asciidoc | 11 - .../query-dsl/query-string-query.asciidoc | 9 - docs/java-api/query-dsl/range-query.asciidoc | 22 - docs/java-api/query-dsl/regexp-query.asciidoc | 11 - docs/java-api/query-dsl/script-query.asciidoc | 29 - .../simple-query-string-query.asciidoc | 9 - .../query-dsl/span-containing-query.asciidoc | 11 - .../query-dsl/span-first-query.asciidoc | 11 - .../query-dsl/span-multi-term-query.asciidoc | 11 - .../query-dsl/span-near-query.asciidoc | 12 - .../query-dsl/span-not-query.asciidoc | 11 - .../java-api/query-dsl/span-or-query.asciidoc | 10 - docs/java-api/query-dsl/span-queries.asciidoc | 65 - .../query-dsl/span-term-query.asciidoc | 11 - .../query-dsl/span-within-query.asciidoc | 11 - .../query-dsl/special-queries.asciidoc | 31 - .../query-dsl/term-level-queries.asciidoc | 77 - docs/java-api/query-dsl/term-query.asciidoc | 11 - docs/java-api/query-dsl/terms-query.asciidoc | 11 - .../query-dsl/wildcard-query.asciidoc | 11 - .../java-api/query-dsl/wrapper-query.asciidoc | 11 - docs/java-api/search.asciidoc | 250 -- .../dataframe/get_data_frame.asciidoc | 1 + .../dataframe/get_data_frame_stats.asciidoc | 16 +- .../dataframe/put_data_frame.asciidoc | 12 +- .../dataframe/stop_data_frame.asciidoc | 1 + .../document/delete-by-query.asciidoc | 4 +- .../high-level/document/reindex.asciidoc | 6 +- .../high-level/document/term-vectors.asciidoc | 7 +- .../document/update-by-query.asciidoc | 4 +- .../high-level/indices/analyze.asciidoc | 10 +- .../indices/reload_analyzers.asciidoc | 50 + .../ml/delete-data-frame-analytics.asciidoc | 28 + .../ml/evaluate-data-frame.asciidoc | 45 + .../get-data-frame-analytics-stats.asciidoc | 34 + .../ml/get-data-frame-analytics.asciidoc | 34 + .../ml/put-data-frame-analytics.asciidoc | 115 + .../ml/start-data-frame-analytics.asciidoc | 28 + .../ml/stop-data-frame-analytics.asciidoc | 29 + .../high-level/query-builders.asciidoc | 3 +- .../security/get-builtin-privileges.asciidoc | 27 + .../high-level/supported-apis.asciidoc | 19 +- .../packages.asciidoc | 6 + .../packages.asciidoc | 48 + docs/painless/painless-guide/index.asciidoc | 4 + .../painless-guide/painless-datetime.asciidoc | 720 ++++ docs/painless/redirects.asciidoc | 9 + docs/plugins/analysis-phonetic.asciidoc | 8 + docs/plugins/analysis-smartcn.asciidoc | 412 ++- docs/plugins/analysis-stempel.asciidoc | 104 +- docs/plugins/api.asciidoc | 3 - docs/plugins/repository-azure.asciidoc | 10 +- docs/reference/administering.asciidoc | 25 + docs/reference/aggregations/bucket.asciidoc | 3 + .../autodatehistogram-aggregation.asciidoc | 33 + .../bucket/rare-terms-aggregation.asciidoc | 357 ++ .../tokenfilters/elision-tokenfilter.asciidoc | 9 +- .../tokenfilters/synonym-tokenfilter.asciidoc | 2 + .../word-delimiter-tokenfilter.asciidoc | 2 +- docs/reference/analysis/tokenizers.asciidoc | 4 + .../pathhierarchy-tokenizer-examples.asciidoc | 191 ++ .../pathhierarchy-tokenizer.asciidoc | 4 + docs/reference/api-conventions.asciidoc | 4 + docs/reference/cat.asciidoc | 6 +- docs/reference/cat/alias.asciidoc | 2 +- docs/reference/cat/allocation.asciidoc | 2 +- docs/reference/cat/count.asciidoc | 4 +- docs/reference/cat/fielddata.asciidoc | 6 +- docs/reference/cat/health.asciidoc | 4 +- docs/reference/cat/indices.asciidoc | 10 +- docs/reference/cat/master.asciidoc | 2 +- docs/reference/cat/nodeattrs.asciidoc | 4 +- docs/reference/cat/nodes.asciidoc | 6 +- docs/reference/cat/pending_tasks.asciidoc | 2 +- docs/reference/cat/plugins.asciidoc | 2 +- docs/reference/cat/recovery.asciidoc | 6 +- docs/reference/cat/repositories.asciidoc | 2 +- docs/reference/cat/segments.asciidoc | 2 +- docs/reference/cat/shards.asciidoc | 10 +- docs/reference/cat/snapshots.asciidoc | 25 +- docs/reference/cat/templates.asciidoc | 2 +- docs/reference/cat/thread_pool.asciidoc | 4 +- .../ccr/apis/follow-request-body.asciidoc | 1 - .../ccr/apis/follow/get-follow-stats.asciidoc | 5 + .../reference/ccr/apis/get-ccr-stats.asciidoc | 2 + docs/reference/ccr/overview.asciidoc | 7 +- docs/reference/cluster.asciidoc | 18 +- docs/reference/cluster/stats.asciidoc | 9 +- docs/reference/commands/certutil.asciidoc | 2 +- docs/reference/commands/shard-tool.asciidoc | 10 +- .../apis/delete-transform.asciidoc | 35 +- .../apis/get-transform-stats.asciidoc | 92 +- .../data-frames/apis/get-transform.asciidoc | 101 +- .../reference/data-frames/apis/index.asciidoc | 8 +- .../data-frames/apis/pivotresource.asciidoc | 26 + .../apis/preview-transform.asciidoc | 50 +- .../data-frames/apis/put-transform.asciidoc | 68 +- .../data-frames/apis/start-transform.asciidoc | 34 +- .../data-frames/apis/stop-transform.asciidoc | 96 +- docs/reference/docs/data-replication.asciidoc | 24 +- docs/reference/docs/delete-by-query.asciidoc | 10 +- docs/reference/docs/reindex.asciidoc | 20 +- docs/reference/docs/termvectors.asciidoc | 3 - docs/reference/docs/update-by-query.asciidoc | 10 +- docs/reference/frozen-indices.asciidoc | 2 +- docs/reference/getting-started.asciidoc | 136 +- .../how-to/recipes/stemming.asciidoc | 3 + .../reference/ilm/policy-definitions.asciidoc | 6 +- .../images/rare_terms/accuracy_0001.png | Bin 0 -> 25315 bytes .../images/rare_terms/accuracy_001.png | Bin 0 -> 27086 bytes .../images/rare_terms/accuracy_01.png | Bin 0 -> 25075 bytes docs/reference/images/rare_terms/memory.png | Bin 0 -> 20465 bytes docs/reference/index-modules.asciidoc | 3 +- docs/reference/index-modules/slowlog.asciidoc | 39 +- docs/reference/index.asciidoc | 8 +- .../indices/apis/reload-analyzers.asciidoc | 105 + docs/reference/indices/create-index.asciidoc | 2 +- docs/reference/indices/open-close.asciidoc | 64 +- docs/reference/indices/recovery.asciidoc | 2 +- docs/reference/indices/shrink-index.asciidoc | 5 +- docs/reference/ingest.asciidoc | 47 +- .../processors/date-index-name.asciidoc | 2 +- .../ingest/processors/geoip.asciidoc | 2 +- docs/reference/ingest/processors/set.asciidoc | 2 +- docs/reference/intro.asciidoc | 270 ++ docs/reference/mapping.asciidoc | 10 +- docs/reference/mapping/params/coerce.asciidoc | 3 +- .../reference/mapping/params/enabled.asciidoc | 4 +- .../mapping/params/ignore-above.asciidoc | 2 +- .../mapping/params/ignore-malformed.asciidoc | 3 +- .../mapping/params/multi-fields.asciidoc | 3 +- .../mapping/params/normalizer.asciidoc | 2 +- docs/reference/mapping/params/norms.asciidoc | 5 +- .../mapping/removal_of_types.asciidoc | 14 +- docs/reference/mapping/types.asciidoc | 9 +- docs/reference/mapping/types/alias.asciidoc | 2 +- .../mapping/types/dense-vector.asciidoc | 16 +- .../mapping/types/flattened.asciidoc | 188 ++ .../mapping/types/geo-shape.asciidoc | 2 + docs/reference/mapping/types/nested.asciidoc | 59 +- .../mapping/types/sparse-vector.asciidoc | 2 + docs/reference/migration/migrate_8_0.asciidoc | 4 + .../migration/migrate_8_0/mappings.asciidoc | 9 + .../migration/migrate_8_0/node.asciidoc | 22 + .../migration/migrate_8_0/reindex.asciidoc | 16 +- .../migration/migrate_8_0/rollup.asciidoc | 20 + .../migration/migrate_8_0/search.asciidoc | 19 +- .../migration/migrate_8_0/security.asciidoc | 8 + .../migration/migrate_8_0/snapshots.asciidoc | 53 + docs/reference/ml/apis/close-job.asciidoc | 37 +- .../ml/apis/datafeedresource.asciidoc | 30 +- .../ml/apis/delete-calendar-event.asciidoc | 29 +- .../ml/apis/delete-calendar-job.asciidoc | 30 +- .../ml/apis/delete-calendar.asciidoc | 29 +- .../ml/apis/delete-datafeed.asciidoc | 43 +- .../ml/apis/delete-expired-data.asciidoc | 24 +- docs/reference/ml/apis/delete-filter.asciidoc | 29 +- .../ml/apis/delete-forecast.asciidoc | 45 +- docs/reference/ml/apis/delete-job.asciidoc | 37 +- .../ml/apis/delete-snapshot.asciidoc | 34 +- .../ml/apis/find-file-structure.asciidoc | 70 +- docs/reference/ml/apis/flush-job.asciidoc | 45 +- docs/reference/ml/apis/forecast.asciidoc | 36 +- docs/reference/ml/apis/get-bucket.asciidoc | 65 +- .../ml/apis/get-calendar-event.asciidoc | 43 +- docs/reference/ml/apis/get-calendar.asciidoc | 42 +- docs/reference/ml/apis/get-category.asciidoc | 52 +- .../ml/apis/get-datafeed-stats.asciidoc | 35 +- docs/reference/ml/apis/get-datafeed.asciidoc | 35 +- docs/reference/ml/apis/get-filter.asciidoc | 42 +- .../reference/ml/apis/get-influencer.asciidoc | 55 +- docs/reference/ml/apis/get-job-stats.asciidoc | 47 +- docs/reference/ml/apis/get-job.asciidoc | 33 +- docs/reference/ml/apis/get-ml-info.asciidoc | 34 +- .../ml/apis/get-overall-buckets.asciidoc | 59 +- docs/reference/ml/apis/get-record.asciidoc | 56 +- docs/reference/ml/apis/get-snapshot.asciidoc | 49 +- docs/reference/ml/apis/jobcounts.asciidoc | 33 +- docs/reference/ml/apis/open-job.asciidoc | 43 +- .../ml/apis/post-calendar-event.asciidoc | 42 +- docs/reference/ml/apis/post-data.asciidoc | 56 +- .../ml/apis/preview-datafeed.asciidoc | 48 +- .../ml/apis/put-calendar-job.asciidoc | 30 +- docs/reference/ml/apis/put-calendar.asciidoc | 37 +- docs/reference/ml/apis/put-datafeed.asciidoc | 89 +- docs/reference/ml/apis/put-filter.asciidoc | 37 +- docs/reference/ml/apis/put-job.asciidoc | 57 +- .../ml/apis/revert-snapshot.asciidoc | 40 +- .../ml/apis/set-upgrade-mode.asciidoc | 31 +- .../ml/apis/snapshotresource.asciidoc | 2 +- .../reference/ml/apis/start-datafeed.asciidoc | 50 +- docs/reference/ml/apis/stop-datafeed.asciidoc | 43 +- .../ml/apis/update-datafeed.asciidoc | 70 +- docs/reference/ml/apis/update-filter.asciidoc | 39 +- docs/reference/ml/apis/update-job.asciidoc | 30 +- .../ml/apis/update-snapshot.asciidoc | 42 +- .../ml/apis/validate-detector.asciidoc | 28 +- docs/reference/ml/apis/validate-job.asciidoc | 28 +- docs/reference/ml/configuring.asciidoc | 2 +- docs/reference/ml/functions/geo.asciidoc | 38 +- docs/reference/modules.asciidoc | 12 - .../cluster/allocation_awareness.asciidoc | 2 +- .../discovery/adding-removing-nodes.asciidoc | 32 +- .../modules/discovery/bootstrapping.asciidoc | 44 +- .../discovery/discovery-settings.asciidoc | 19 +- .../modules/indices/search-settings.asciidoc | 6 +- docs/reference/modules/memcached.asciidoc | 69 - docs/reference/modules/ml-node.asciidoc | 116 - docs/reference/modules/network.asciidoc | 11 +- docs/reference/modules/node.asciidoc | 232 +- .../modules/remote-clusters.asciidoc | 35 +- docs/reference/modules/snapshots.asciidoc | 59 +- docs/reference/modules/thrift.asciidoc | 25 - .../monitoring/configuring-filebeat.asciidoc | 187 ++ .../configuring-metricbeat.asciidoc | 198 +- .../configuring-monitoring.asciidoc | 3 + .../query-dsl/boosting-query.asciidoc | 52 +- .../query-dsl/common-terms-query.asciidoc | 299 -- .../query-dsl/constant-score-query.asciidoc | 28 +- .../query-dsl/dis-max-query.asciidoc | 87 +- .../reference/query-dsl/exists-query.asciidoc | 21 +- .../query-dsl/full-text-queries.asciidoc | 6 - .../query-dsl/geo-bounding-box-query.asciidoc | 18 +- .../query-dsl/intervals-query.asciidoc | 50 +- docs/reference/query-dsl/match-query.asciidoc | 47 +- docs/reference/query-dsl/mlt-query.asciidoc | 5 +- .../query-dsl/multi-match-query.asciidoc | 20 +- .../query-dsl/query_filter_context.asciidoc | 19 +- docs/reference/query-dsl/range-query.asciidoc | 268 +- .../query-dsl/regexp-syntax.asciidoc | 2 +- .../query-dsl/script-score-query.asciidoc | 5 +- .../query-dsl/term-level-queries.asciidoc | 73 +- docs/reference/query-dsl/terms-query.asciidoc | 291 +- .../query-dsl/terms-set-query.asciidoc | 248 +- .../query-dsl/wildcard-query.asciidoc | 86 +- docs/reference/redirects.asciidoc | 20 +- docs/reference/rest-api/defs.asciidoc | 2 + docs/reference/rest-api/index.asciidoc | 15 +- docs/reference/rest-api/info.asciidoc | 29 +- docs/reference/rollup/api-quickref.asciidoc | 2 +- docs/reference/rollup/rollup-api.asciidoc | 2 +- .../{modules => }/scripting.asciidoc | 16 +- .../{modules => }/scripting/engine.asciidoc | 2 +- .../scripting/expression.asciidoc | 2 +- .../{modules => }/scripting/fields.asciidoc | 8 +- .../{modules => }/scripting/painless.asciidoc | 2 +- .../{modules => }/scripting/security.asciidoc | 2 +- .../{modules => }/scripting/using.asciidoc | 13 +- .../search/request/docvalue-fields.asciidoc | 4 + .../search/request/preference.asciidoc | 12 +- .../search/request/script-fields.asciidoc | 2 +- docs/reference/search/request/sort.asciidoc | 16 +- .../search/request/stored-fields.asciidoc | 5 + .../search/suggesters/phrase-suggest.asciidoc | 31 +- .../configuring-tls-docker.asciidoc | 56 +- .../separating-node-client-traffic.asciidoc | 68 - docs/reference/settings/ml-settings.asciidoc | 8 + .../settings/monitoring-settings.asciidoc | 1 + .../settings/notification-settings.asciidoc | 7 +- .../settings/security-settings.asciidoc | 2 + docs/reference/settings/ssl-settings.asciidoc | 4 +- .../discovery-settings.asciidoc | 6 +- docs/reference/setup/install.asciidoc | 8 + docs/reference/setup/install/brew.asciidoc | 69 + docs/reference/setup/install/docker.asciidoc | 8 +- docs/reference/setup/secure-settings.asciidoc | 8 +- docs/reference/sql/endpoints/jdbc.asciidoc | 5 + docs/reference/sql/endpoints/rest.asciidoc | 156 +- .../sql/functions/conditional.asciidoc | 42 +- .../sql/functions/date-time.asciidoc | 80 +- docs/reference/sql/functions/geo.asciidoc | 4 +- .../reference/sql/functions/grouping.asciidoc | 10 +- .../sql/functions/like-rlike.asciidoc | 4 +- .../sql/functions/operators.asciidoc | 36 +- docs/reference/sql/functions/search.asciidoc | 25 +- docs/reference/sql/functions/string.asciidoc | 42 +- docs/reference/sql/functions/system.asciidoc | 4 +- .../sql/functions/type-conversion.asciidoc | 10 +- docs/reference/sql/getting-started.asciidoc | 2 +- .../sql/language/data-types.asciidoc | 4 +- docs/reference/sql/language/indices.asciidoc | 20 +- .../syntax/commands/describe-table.asciidoc | 2 +- .../language/syntax/commands/index.asciidoc | 2 +- .../language/syntax/commands/select.asciidoc | 64 +- .../syntax/commands/show-columns.asciidoc | 2 +- .../syntax/commands/show-functions.asciidoc | 10 +- .../syntax/commands/show-tables.asciidoc | 12 +- .../sql/language/syntax/lexic/index.asciidoc | 29 +- docs/reference/sql/limitations.asciidoc | 4 +- docs/reference/sql/security.asciidoc | 2 +- docs/reference/upgrade.asciidoc | 4 +- .../upgrade/cluster_restart.asciidoc | 2 +- .../upgrade/reindex_upgrade.asciidoc | 6 +- .../upgrade/rolling_upgrade.asciidoc | 4 +- gradle.properties | 2 +- gradle/wrapper/gradle-wrapper.properties | 4 +- gradlew | 2 +- gradlew.bat | 2 +- libs/build.gradle | 10 +- libs/cli/build.gradle | 2 +- libs/core/build.gradle | 18 +- .../elasticsearch/common/unit/TimeValue.java | 128 +- .../core/internal/io/IOUtils.java | 32 +- libs/core/src/test/eclipse-build.gradle | 2 +- .../common/unit/TimeValueTests.java | 20 + .../core/internal/io/IOUtilsTests.java | 44 + libs/dissect/build.gradle | 14 +- libs/dissect/src/test/eclipse-build.gradle | 2 +- libs/geo/build.gradle | 6 +- .../elasticsearch/geo/geometry/Circle.java | 2 - .../geo/geometry/GeometryUtils.java | 70 - .../org/elasticsearch/geo/geometry/Line.java | 4 - .../org/elasticsearch/geo/geometry/Point.java | 2 - .../elasticsearch/geo/geometry/Rectangle.java | 30 - .../geo/utils/GeographyValidator.java | 178 + .../geo/utils/GeometryValidator.java | 34 + .../geo/utils/WellKnownText.java | 92 +- .../geo/geometry/BaseGeometryTestCase.java | 6 +- .../geo/geometry/CircleTests.java | 22 +- .../geo/geometry/GeometryCollectionTests.java | 10 +- .../geo/geometry/GeometryValidatorTests.java | 127 + .../elasticsearch/geo/geometry/LineTests.java | 31 +- .../geo/geometry/LinearRingTests.java | 19 +- .../geo/geometry/MultiLineTests.java | 10 +- .../geo/geometry/MultiPointTests.java | 18 +- .../geo/geometry/MultiPolygonTests.java | 10 +- .../geo/geometry/PointTests.java | 26 +- .../geo/geometry/PolygonTests.java | 40 +- .../geo/geometry/RectangleTests.java | 24 +- libs/grok/build.gradle | 12 +- libs/grok/licenses/jcodings-1.0.12.jar.sha1 | 1 - libs/grok/licenses/jcodings-1.0.44.jar.sha1 | 1 + libs/grok/src/test/eclipse-build.gradle | 2 +- libs/nio/build.gradle | 18 +- .../nio/BytesChannelContext.java | 11 +- .../elasticsearch/nio/BytesWriteHandler.java | 7 +- .../org/elasticsearch/nio/ChannelFactory.java | 2 + .../elasticsearch/nio/DelegatingHandler.java | 68 + ...iteHandler.java => NioChannelHandler.java} | 11 +- .../org/elasticsearch/nio/NioSelector.java | 99 +- .../nio/SocketChannelContext.java | 25 +- .../org/elasticsearch/nio/WriteOperation.java | 2 +- libs/nio/src/test/eclipse-build.gradle | 2 +- .../elasticsearch/nio/EventHandlerTests.java | 6 +- .../elasticsearch/nio/NioSelectorTests.java | 75 +- .../nio/SocketChannelContextTests.java | 33 +- libs/secure-sm/build.gradle | 18 +- libs/secure-sm/src/test/eclipse-build.gradle | 2 +- libs/ssl-config/build.gradle | 8 +- .../common/ssl/PemTrustConfigTests.java | 27 +- libs/x-content/build.gradle | 18 +- .../common/xcontent/ObjectParser.java | 103 +- .../common/xcontent/XContentParser.java | 6 +- .../common/xcontent/XContentSubParser.java | 5 - .../xcontent/json/JsonXContentParser.java | 4 + .../support/AbstractXContentParser.java | 19 +- libs/x-content/src/test/eclipse-build.gradle | 2 +- .../common/xcontent/ObjectParserTests.java | 39 + .../common/xcontent/XContentParserTests.java | 2 +- modules/aggs-matrix-stats/build.gradle | 1 - .../matrix/stats/InternalMatrixStats.java | 10 +- .../stats/MatrixStatsAggregationBuilder.java | 12 +- .../stats/MatrixStatsAggregatorFactory.java | 5 +- .../ArrayValuesSourceAggregationBuilder.java | 41 +- .../ArrayValuesSourceAggregatorFactory.java | 6 +- .../analysis/common/CommonAnalysisPlugin.java | 35 +- .../common/ElisionTokenFilterFactory.java | 3 + .../common/StandardHtmlStripAnalyzer.java | 58 - .../common/SynonymTokenFilterFactory.java | 13 + .../common/EdgeNGramTokenizerTests.java | 98 + .../common/ElisionFilterFactoryTests.java | 43 + .../common/HighlighterWithAnalyzersTests.java | 2 +- ...DelimiterGraphTokenFilterFactoryTests.java | 57 + .../test/analysis-common/20_analyzers.yml | 9 - .../test/analysis-common/40_token_filters.yml | 14 + .../search.query/50_queries_with_synonyms.yml | 116 +- modules/ingest-common/build.gradle | 4 +- .../ingest/common/GrokProcessorGetAction.java | 4 +- .../ingest/common/RenameProcessorTests.java | 6 +- .../test/ingest/200_default_pipeline.yml | 23 + .../ExpressionAggregationScript.java | 27 +- .../expression/ExpressionScriptEngine.java | 9 +- .../ExpressionFieldScriptTests.java | 2 +- .../ExpressionNumberSortScriptTests.java | 9 +- .../ExpressionTermsSetQueryTests.java | 9 +- .../expression/MoreExpressionTests.java | 13 +- .../expression/StoredExpressionTests.java | 2 +- .../test/lang_expression/20_search.yml | 20 +- modules/lang-mustache/build.gradle | 1 - .../mustache/MultiSearchTemplateAction.java | 9 +- .../script/mustache/SearchTemplateAction.java | 9 +- .../MultiSearchTemplateRequestTests.java | 3 - .../script/mustache/SearchTemplateIT.java | 14 +- .../mustache/SearchTemplateRequestTests.java | 4 +- .../mustache/simple-msearch-template.json | 4 +- .../test/lang_mustache/30_search_template.yml | 3 +- modules/lang-painless/build.gradle | 2 +- modules/lang-painless/spi/build.gradle | 2 +- .../elasticsearch/painless/spi/Whitelist.java | 5 +- .../painless/spi/WhitelistClass.java | 26 +- .../painless/spi/WhitelistClassBinding.java | 18 +- .../painless/spi/WhitelistConstructor.java | 16 +- .../painless/spi/WhitelistField.java | 18 +- .../spi/WhitelistInstanceBinding.java | 18 +- .../painless/spi/WhitelistLoader.java | 239 +- .../painless/spi/WhitelistMethod.java | 17 +- .../spi/annotation/DeprecatedAnnotation.java | 19 +- .../DeprecatedAnnotationParser.java | 44 + .../spi/annotation/NoImportAnnotation.java} | 13 +- .../annotation/NoImportAnnotationParser.java | 21 +- .../annotation/WhitelistAnnotationParser.java | 42 + .../painless/ContextDocGenerator.java | 1 + .../painless/PainlessPlugin.java | 19 +- .../action/PainlessContextAction.java | 9 +- .../action/PainlessExecuteAction.java | 48 +- .../painless/api/Augmentation.java | 113 + .../lookup/PainlessLookupBuilder.java | 4 +- .../elasticsearch/painless/spi/java.util.txt | 4 + .../painless/spi/org.elasticsearch.score.txt | 11 +- .../painless/spi/org.elasticsearch.txt | 18 +- .../painless/AnnotationTestObject.java | 102 + .../painless/AugmentationTests.java | 1 - .../elasticsearch/painless/BindingsTests.java | 4 +- .../elasticsearch/painless/DateTimeTests.java | 194 ++ .../painless/GetByPathAugmentationTests.java | 259 ++ .../painless/NeedsScoreTests.java | 2 +- .../painless/PainlessDocGenerator.java | 451 --- .../painless/WhitelistLoaderTests.java | 85 + .../action/PainlessExecuteRequestTests.java | 13 +- .../spi/org.elasticsearch.painless.annotation | 7 + .../spi/org.elasticsearch.painless.test | 2 +- .../api/scripts_painless_context.json | 2 +- .../test/painless/100_terms_agg.yml | 104 + .../test/painless/71_context_api.yml | 4 +- .../index/mapper/MapperExtrasPlugin.java | 2 - ...asticsearch.painless.spi.PainlessExtension | 1 - .../index/query/docvalues_whitelist.txt | 32 - .../mapper/DenseVectorFieldMapperTests.java | 105 - .../TokenCountFieldMapperIntegrationIT.java | 3 +- modules/parent-join/build.gradle | 1 - .../ChildrenAggregationBuilder.java | 13 +- .../ChildrenAggregatorFactory.java | 4 +- .../ParentAggregationBuilder.java | 13 +- .../aggregations/ParentAggregatorFactory.java | 4 +- .../join/query/HasChildQueryBuilderTests.java | 5 +- .../query/HasParentQueryBuilderTests.java | 4 - modules/percolator/build.gradle | 1 - .../percolator/PercolateQueryBuilder.java | 21 +- .../percolator/QueryAnalyzer.java | 9 - .../percolator/CandidateQueryTests.java | 9 +- .../PercolateQueryBuilderTests.java | 38 +- .../percolator/PercolatorQuerySearchIT.java | 14 +- .../percolator/QueryAnalyzerTests.java | 24 +- .../percolator/QueryBuilderStoreTests.java | 4 +- modules/rank-eval/build.gradle | 1 - .../index/rankeval/RankEvalAction.java | 6 +- .../rankeval/RankEvalRequestBuilder.java | 4 +- .../index/rankeval/RatedRequestsTests.java | 2 +- modules/reindex/build.gradle | 9 +- .../AbstractAsyncBulkByScrollAction.java | 14 +- .../AbstractBaseReindexRestHandler.java | 18 +- .../AbstractBulkByQueryRestHandler.java | 10 +- .../BulkByScrollParallelizationHelper.java | 10 +- .../reindex/RestDeleteByQueryAction.java | 1 + .../index/reindex/RestReindexAction.java | 184 +- .../reindex/RestUpdateByQueryAction.java | 1 + .../index/reindex/RethrottleAction.java | 9 +- .../reindex/RethrottleRequestBuilder.java | 4 +- .../reindex/remote/RemoteRequestBuilders.java | 24 +- .../reindex/remote/RemoteResponseParsers.java | 2 +- .../documentation/ReindexDocumentationIT.java | 4 +- .../reindex/AsyncBulkByScrollActionTests.java | 4 +- .../index/reindex/CancelTests.java | 6 +- .../reindex/DeleteByQueryBasicTests.java | 27 +- .../index/reindex/ManyDocumentsIT.java | 39 +- .../index/reindex/ReindexBasicTests.java | 18 +- .../index/reindex/ReindexFailureTests.java | 4 +- ...ReindexFromRemoteBuildRestClientTests.java | 9 +- .../ReindexFromRemoteWhitelistTests.java | 8 +- .../reindex/ReindexRestClientSslTests.java | 5 +- .../ReindexSourceTargetValidationTests.java | 7 +- .../index/reindex/RestReindexActionTests.java | 158 +- .../index/reindex/RethrottleTests.java | 2 - .../index/reindex/RoundTripTests.java | 4 +- .../reindex/UpdateByQueryBasicTests.java | 15 +- .../remote/ReindexFromOldRemoteIT.java | 43 +- .../index/reindex/remote/RemoteInfoTests.java | 16 +- .../remote/RemoteRequestBuildersTests.java | 66 +- .../remote/RemoteResponseParsersTests.java | 52 + .../RemoteScrollableHitSourceTests.java | 49 +- .../test/delete_by_query/10_basic.yml | 93 +- .../test/delete_by_query/20_validation.yml | 10 +- .../test/reindex/20_validation.yml | 75 +- .../rest-api-spec/test/reindex/30_search.yml | 84 +- .../rest-api-spec/test/reindex/90_remote.yml | 8 +- .../test/update_by_query/10_basic.yml | 70 +- .../test/update_by_query/20_validation.yml | 49 +- .../blobstore/url/URLBlobContainer.java | 11 + .../repositories/url/URLRepository.java | 10 +- .../url/URLSnapshotRestoreTests.java | 10 +- .../test/repository_url/10_basic.yml | 25 +- modules/transport-netty4/build.gradle | 5 +- .../netty4/Netty4HttpServerTransport.java | 65 +- .../http/netty4/cors/Netty4CorsConfig.java | 221 -- .../netty4/cors/Netty4CorsConfigBuilder.java | 265 -- .../http/netty4/cors/Netty4CorsHandler.java | 18 +- .../elasticsearch/transport/Netty4Plugin.java | 5 - .../netty4/Netty4InternalESLogger.java | 187 -- .../transport/netty4/Netty4Transport.java | 8 +- .../transport/netty4/Netty4Utils.java | 17 - .../http/netty4/Netty4CorsTests.java | 3 +- .../netty4/Netty4HttpRequestSizeLimitIT.java | 2 +- .../Netty4HttpServerTransportTests.java | 59 +- .../rest/discovery/Zen2RestApiIT.java | 2 +- ...Netty4TransportMultiPortIntegrationIT.java | 24 - plugins/analysis-icu/build.gradle | 1 - .../ICUCollationKeywordFieldMapperIT.java | 24 +- .../SmartChineseStopTokenFilterFactory.java | 61 + .../smartcn/AnalysisSmartChinesePlugin.java | 10 +- .../pl/PolishStopTokenFilterFactory.java | 73 + .../stempel/AnalysisStempelPlugin.java | 4 +- plugins/discovery-azure-classic/build.gradle | 1 + .../licenses/log4j-1.2-api-2.11.1.jar.sha1 | 0 .../licenses/log4j-LICENSE.txt | 0 .../licenses/log4j-NOTICE.txt | 0 .../AzureDiscoveryClusterFormationTests.java | 14 +- plugins/discovery-ec2/build.gradle | 3 +- .../aws-java-sdk-core-1.11.505.jar.sha1 | 1 - .../aws-java-sdk-core-1.11.562.jar.sha1 | 1 + .../aws-java-sdk-ec2-1.11.505.jar.sha1 | 1 - .../aws-java-sdk-ec2-1.11.562.jar.sha1 | 1 + .../licenses/log4j-1.2-api-2.11.1.jar.sha1 | 1 + .../discovery-ec2/licenses/log4j-LICENSE.txt | 202 ++ .../discovery-ec2/licenses/log4j-NOTICE.txt | 5 + .../discovery-ec2/qa/amazon-ec2/build.gradle | 8 +- .../discovery/ec2/AmazonEC2Mock.java | 2904 +---------------- plugins/discovery-gce/build.gradle | 1 + .../licenses/log4j-1.2-api-2.11.1.jar.sha1 | 1 + .../discovery-gce/licenses/log4j-LICENSE.txt | 202 ++ .../discovery-gce/licenses/log4j-NOTICE.txt | 5 + plugins/discovery-gce/qa/gce/build.gradle | 9 +- plugins/examples/build.gradle | 19 + .../customsuggester/CustomSuggester.java | 22 +- .../ExamplePainlessAnnotation.java | 23 +- .../ExampleWhitelistAnnotationParser.java | 62 + .../ExampleWhitelistExtension.java | 16 +- .../ExampleWhitelistedClass.java | 5 + .../painlesswhitelist/example_whitelist.txt | 3 + .../rest-api-spec/api/cat.example.json | 2 +- .../AnnotatedTextFieldMapperTests.java | 2 +- plugins/repository-azure/build.gradle | 23 + .../qa/microsoft-azure-storage/build.gradle | 21 +- .../test/repository_azure/10_repository.yml | 16 +- .../azure/AzureBlobContainer.java | 60 +- .../repositories/azure/AzureBlobStore.java | 24 +- .../repositories/azure/AzureRepository.java | 57 +- .../azure/AzureRepositoryPlugin.java | 15 + .../azure/AzureStorageService.java | 97 +- .../azure/AzureStorageSettings.java | 57 +- .../azure/AzureBlobStoreContainerTests.java | 20 +- .../azure/AzureBlobStoreTests.java | 20 +- .../AzureStorageCleanupThirdPartyTests.java | 75 + .../azure/AzureStorageServiceMock.java | 5 - .../azure/AzureStorageServiceTests.java | 12 +- plugins/repository-gcs/build.gradle | 71 +- .../licenses/api-common-1.7.0.jar.sha1 | 1 - .../licenses/api-common-1.8.1.jar.sha1 | 1 + .../licenses/gax-1.30.0.jar.sha1 | 1 - .../licenses/gax-1.45.0.jar.sha1 | 1 + .../licenses/gax-httpjson-0.47.0.jar.sha1 | 1 - .../licenses/gax-httpjson-0.62.0.jar.sha1 | 1 + .../google-api-client-1.24.1.jar.sha1 | 1 - .../google-api-client-1.28.0.jar.sha1 | 1 + ...services-storage-v1-rev135-1.24.1.jar.sha1 | 1 - ...ces-storage-v1-rev20190426-1.28.0.jar.sha1 | 1 + ...e-auth-library-credentials-0.10.0.jar.sha1 | 1 - ...e-auth-library-credentials-0.16.1.jar.sha1 | 1 + ...e-auth-library-oauth2-http-0.10.0.jar.sha1 | 1 - ...e-auth-library-oauth2-http-0.16.1.jar.sha1 | 1 + .../google-cloud-core-1.59.0.jar.sha1 | 1 - .../google-cloud-core-1.77.0.jar.sha1 | 1 + .../google-cloud-core-http-1.59.0.jar.sha1 | 1 - .../google-cloud-core-http-1.77.0.jar.sha1 | 1 + .../google-cloud-storage-1.59.0.jar.sha1 | 1 - .../google-cloud-storage-1.77.0.jar.sha1 | 1 + .../google-http-client-1.24.1.jar.sha1 | 1 - .../google-http-client-1.30.1.jar.sha1 | 1 + ...ogle-http-client-appengine-1.24.1.jar.sha1 | 1 - ...ogle-http-client-appengine-1.29.2.jar.sha1 | 1 + ...google-http-client-jackson-1.24.1.jar.sha1 | 1 - ...oogle-http-client-jackson2-1.24.1.jar.sha1 | 1 - ...oogle-http-client-jackson2-1.29.2.jar.sha1 | 1 + .../google-oauth-client-1.24.1.jar.sha1 | 1 - .../google-oauth-client-1.28.0.jar.sha1 | 1 + .../licenses/guava-20.0.jar.sha1 | 1 - .../licenses/guava-26.0-jre.jar.sha1 | 1 + .../repository-gcs/licenses/jackson-LICENSE | 8 - .../repository-gcs/licenses/jackson-NOTICE | 20 - .../licenses/jackson-core-asl-1.9.11.jar.sha1 | 1 - .../licenses/log4j-1.2-api-2.11.1.jar.sha1 | 1 + .../repository-gcs/licenses/log4j-LICENSE.txt | 202 ++ .../repository-gcs/licenses/log4j-NOTICE.txt | 5 + .../licenses/opencensus-api-0.15.0.jar.sha1 | 1 - .../licenses/opencensus-api-0.18.0.jar.sha1 | 1 + ...encensus-contrib-http-util-0.15.0.jar.sha1 | 1 - ...encensus-contrib-http-util-0.18.0.jar.sha1 | 1 + ...proto-google-common-protos-1.12.0.jar.sha1 | 1 - ...proto-google-common-protos-1.16.0.jar.sha1 | 1 + .../licenses/protobuf-java-3.6.0.jar.sha1 | 1 - .../licenses/protobuf-java-3.7.1.jar.sha1 | 1 + .../protobuf-java-util-3.6.0.jar.sha1 | 1 - .../protobuf-java-util-3.7.1.jar.sha1 | 1 + .../qa/google-cloud-storage/build.gradle | 9 +- .../test/repository_gcs/10_repository.yml | 16 +- .../gcs/GoogleCloudStorageBlobContainer.java | 11 + .../gcs/GoogleCloudStorageBlobStore.java | 61 +- .../gcs/GoogleCloudStorageRepository.java | 25 +- .../gcs/GoogleCloudStorageService.java | 62 +- .../GoogleCloudStorageThirdPartyTests.java | 64 + .../repositories/gcs/MockStorage.java | 10 + plugins/repository-hdfs/build.gradle | 41 +- .../licenses/log4j-1.2-api-2.11.1.jar.sha1 | 1 + .../licenses/log4j-LICENSE.txt | 202 ++ .../repository-hdfs/licenses/log4j-NOTICE.txt | 5 + .../repositories/hdfs/HdfsBlobContainer.java | 29 +- .../repositories/hdfs/HdfsRepository.java | 8 +- .../hdfs/HdfsRepositoryTests.java | 61 + .../repositories/hdfs/HdfsTests.java | 9 +- .../test/hdfs_repository/30_snapshot_get.yml | 8 +- .../hdfs_repository/30_snapshot_readonly.yml | 2 +- plugins/repository-s3/build.gradle | 62 +- .../aws-java-sdk-core-1.11.505.jar.sha1 | 1 - .../aws-java-sdk-core-1.11.562.jar.sha1 | 1 + .../aws-java-sdk-kms-1.11.505.jar.sha1 | 1 - .../aws-java-sdk-s3-1.11.505.jar.sha1 | 1 - .../aws-java-sdk-s3-1.11.562.jar.sha1 | 1 + .../licenses/jmespath-java-1.11.505.jar.sha1 | 1 - .../licenses/jmespath-java-1.11.562.jar.sha1 | 1 + .../licenses/log4j-1.2-api-2.11.1.jar.sha1 | 1 + .../repository-s3/licenses/log4j-LICENSE.txt | 202 ++ .../repository-s3/licenses/log4j-NOTICE.txt | 5 + .../repositories/s3/S3BlobContainer.java | 101 +- .../repositories/s3/S3Repository.java | 31 +- .../s3/S3RepositoryThirdPartyTests.java | 114 + .../20_repository_permanent_credentials.yml | 17 +- .../30_repository_temporary_credentials.yml | 16 +- .../40_repository_ec2_credentials.yml | 16 +- .../50_repository_ecs_credentials.yml | 16 +- .../smbmmapfs/SmbMmapFsDirectoryFactory.java | 5 +- plugins/transport-nio/build.gradle | 7 +- .../http/nio/HttpReadWriteHandler.java | 13 +- .../http/nio/NioHttpServerTransport.java | 56 +- .../http/nio/cors/NioCorsConfig.java | 221 -- .../http/nio/cors/NioCorsConfigBuilder.java | 265 -- .../http/nio/cors/NioCorsHandler.java | 17 +- .../http/nio/HttpReadWriteHandlerTests.java | 9 +- .../elasticsearch/http/nio/NioHttpClient.java | 9 +- .../http/nio/NioHttpServerTransportTests.java | 58 +- qa/build.gradle | 29 +- qa/ccs-unavailable-clusters/build.gradle | 3 +- qa/die-with-dignity/build.gradle | 10 +- .../elasticsearch/DieWithDignityPlugin.java | 4 + .../qa/die_with_dignity/DieWithDignityIT.java | 123 +- qa/evil-tests/build.gradle | 1 + .../env/NodeEnvironmentEvilTests.java | 9 +- .../upgrades/FullClusterRestartIT.java | 20 +- .../upgrades/QueryBuilderBWCIT.java | 2 +- qa/logging-config/build.gradle | 10 +- .../common/logging/JsonLoggerTests.java | 4 +- .../custom_logging/CustomLoggingConfigIT.java | 2 +- qa/multi-cluster-search/build.gradle | 41 +- .../elasticsearch/upgrades/RecoveryIT.java | 20 +- qa/smoke-test-client/build.gradle | 52 - .../smoketest/ESSmokeClientTestCase.java | 170 - .../smoketest/SmokeTestClientIT.java | 73 - qa/smoke-test-http/build.gradle | 3 +- qa/smoke-test-ingest-disabled/build.gradle | 3 +- .../build.gradle | 1 + qa/smoke-test-multinode/build.gradle | 15 +- qa/smoke-test-plugins/build.gradle | 13 +- qa/unconfigured-node-name/build.gradle | 13 +- qa/vagrant/build.gradle | 2 +- qa/wildfly/build.gradle | 11 +- .../org/elasticsearch/wildfly/WildflyIT.java | 7 +- rest-api-spec/README.markdown | 94 +- .../resources/rest-api-spec/api/bulk.json | 2 +- .../rest-api-spec/api/cat.aliases.json | 2 +- .../rest-api-spec/api/cat.allocation.json | 2 +- .../rest-api-spec/api/cat.count.json | 2 +- .../rest-api-spec/api/cat.fielddata.json | 2 +- .../rest-api-spec/api/cat.health.json | 2 +- .../resources/rest-api-spec/api/cat.help.json | 2 +- .../rest-api-spec/api/cat.indices.json | 2 +- .../rest-api-spec/api/cat.master.json | 2 +- .../rest-api-spec/api/cat.nodeattrs.json | 2 +- .../rest-api-spec/api/cat.nodes.json | 2 +- .../rest-api-spec/api/cat.pending_tasks.json | 2 +- .../rest-api-spec/api/cat.plugins.json | 2 +- .../rest-api-spec/api/cat.recovery.json | 2 +- .../rest-api-spec/api/cat.repositories.json | 2 +- .../rest-api-spec/api/cat.segments.json | 2 +- .../rest-api-spec/api/cat.shards.json | 2 +- .../rest-api-spec/api/cat.snapshots.json | 2 +- .../rest-api-spec/api/cat.tasks.json | 2 +- .../rest-api-spec/api/cat.templates.json | 2 +- .../rest-api-spec/api/cat.thread_pool.json | 2 +- .../rest-api-spec/api/clear_scroll.json | 2 +- .../api/cluster.allocation_explain.json | 2 +- .../api/cluster.get_settings.json | 2 +- .../rest-api-spec/api/cluster.health.json | 2 +- .../api/cluster.pending_tasks.json | 2 +- .../api/cluster.put_settings.json | 2 +- .../api/cluster.remote_info.json | 2 +- .../rest-api-spec/api/cluster.reroute.json | 2 +- .../rest-api-spec/api/cluster.state.json | 7 +- .../rest-api-spec/api/cluster.stats.json | 2 +- .../resources/rest-api-spec/api/count.json | 9 +- .../resources/rest-api-spec/api/create.json | 2 +- .../resources/rest-api-spec/api/delete.json | 2 +- .../rest-api-spec/api/delete_by_query.json | 6 +- .../api/delete_by_query_rethrottle.json | 2 +- .../rest-api-spec/api/delete_script.json | 2 +- .../resources/rest-api-spec/api/exists.json | 2 +- .../rest-api-spec/api/exists_source.json | 2 +- .../resources/rest-api-spec/api/explain.json | 2 +- .../rest-api-spec/api/field_caps.json | 2 +- .../main/resources/rest-api-spec/api/get.json | 2 +- .../rest-api-spec/api/get_script.json | 2 +- .../rest-api-spec/api/get_source.json | 2 +- .../resources/rest-api-spec/api/index.json | 2 +- .../rest-api-spec/api/indices.analyze.json | 2 +- .../api/indices.clear_cache.json | 2 +- .../rest-api-spec/api/indices.close.json | 2 +- .../rest-api-spec/api/indices.create.json | 2 +- .../rest-api-spec/api/indices.delete.json | 2 +- .../api/indices.delete_alias.json | 2 +- .../api/indices.delete_template.json | 2 +- .../rest-api-spec/api/indices.exists.json | 2 +- .../api/indices.exists_alias.json | 2 +- .../api/indices.exists_template.json | 2 +- .../api/indices.exists_type.json | 2 +- .../rest-api-spec/api/indices.flush.json | 2 +- .../api/indices.flush_synced.json | 2 +- .../rest-api-spec/api/indices.forcemerge.json | 2 +- .../rest-api-spec/api/indices.get.json | 2 +- .../rest-api-spec/api/indices.get_alias.json | 2 +- .../api/indices.get_field_mapping.json | 2 +- .../api/indices.get_mapping.json | 2 +- .../api/indices.get_settings.json | 2 +- .../api/indices.get_template.json | 2 +- .../api/indices.get_upgrade.json | 2 +- .../rest-api-spec/api/indices.open.json | 2 +- .../rest-api-spec/api/indices.put_alias.json | 2 +- .../api/indices.put_mapping.json | 2 +- .../api/indices.put_settings.json | 2 +- .../api/indices.put_template.json | 2 +- .../rest-api-spec/api/indices.recovery.json | 2 +- .../rest-api-spec/api/indices.refresh.json | 2 +- .../rest-api-spec/api/indices.rollover.json | 2 +- .../rest-api-spec/api/indices.segments.json | 2 +- .../api/indices.shard_stores.json | 2 +- .../rest-api-spec/api/indices.shrink.json | 2 +- .../rest-api-spec/api/indices.split.json | 2 +- .../rest-api-spec/api/indices.stats.json | 2 +- .../api/indices.update_aliases.json | 2 +- .../rest-api-spec/api/indices.upgrade.json | 2 +- .../api/indices.validate_query.json | 2 +- .../resources/rest-api-spec/api/info.json | 2 +- .../api/ingest.delete_pipeline.json | 2 +- .../api/ingest.get_pipeline.json | 2 +- .../api/ingest.processor_grok.json | 2 +- .../api/ingest.put_pipeline.json | 2 +- .../rest-api-spec/api/ingest.simulate.json | 2 +- .../resources/rest-api-spec/api/mget.json | 2 +- .../resources/rest-api-spec/api/msearch.json | 2 +- .../rest-api-spec/api/msearch_template.json | 2 +- .../rest-api-spec/api/mtermvectors.json | 2 +- .../rest-api-spec/api/nodes.hot_threads.json | 2 +- .../rest-api-spec/api/nodes.info.json | 2 +- .../api/nodes.reload_secure_settings.json | 2 +- .../rest-api-spec/api/nodes.stats.json | 2 +- .../rest-api-spec/api/nodes.usage.json | 2 +- .../resources/rest-api-spec/api/ping.json | 2 +- .../rest-api-spec/api/put_script.json | 2 +- .../rest-api-spec/api/rank_eval.json | 2 +- .../resources/rest-api-spec/api/reindex.json | 6 +- .../rest-api-spec/api/reindex_rethrottle.json | 2 +- .../api/render_search_template.json | 2 +- .../api/scripts_painless_execute.json | 2 +- .../resources/rest-api-spec/api/scroll.json | 2 +- .../resources/rest-api-spec/api/search.json | 2 +- .../rest-api-spec/api/search_shards.json | 2 +- .../rest-api-spec/api/search_template.json | 2 +- .../rest-api-spec/api/snapshot.create.json | 2 +- .../api/snapshot.create_repository.json | 2 +- .../rest-api-spec/api/snapshot.delete.json | 2 +- .../api/snapshot.delete_repository.json | 2 +- .../rest-api-spec/api/snapshot.get.json | 2 +- .../api/snapshot.get_repository.json | 2 +- .../rest-api-spec/api/snapshot.restore.json | 2 +- .../rest-api-spec/api/snapshot.status.json | 2 +- .../api/snapshot.verify_repository.json | 2 +- .../rest-api-spec/api/tasks.cancel.json | 2 +- .../rest-api-spec/api/tasks.get.json | 2 +- .../rest-api-spec/api/tasks.list.json | 2 +- .../rest-api-spec/api/termvectors.json | 2 +- .../resources/rest-api-spec/api/update.json | 2 +- .../rest-api-spec/api/update_by_query.json | 6 +- .../api/update_by_query_rethrottle.json | 2 +- .../test/cat.snapshots/10_basic.yml | 12 +- .../test/count/11_basic_with_types.yml | 66 - .../test/indices.analyze/10_analyze.yml | 1 - .../test/indices.recovery/10_basic.yml | 25 + .../test/indices.stats/20_translog.yml | 44 + .../rest-api-spec/test/mlt/20_docs.yml | 2 - .../rest-api-spec/test/mlt/30_unlike.yml | 2 - .../test/msearch/12_basic_with_types.yml | 97 - .../test/mtermvectors/20_deprecated.yml | 1 - .../200_top_hits_metric.yml | 9 +- .../search.aggregation/280_rare_terms.yml | 316 ++ .../test/search/230_interval_query.yml | 40 + .../test/search/310_match_bool_prefix.yml | 16 - .../test/snapshot.get/10_basic.yml | 93 +- .../test/termvectors/20_issue7121.yml | 1 - server/build.gradle | 22 +- server/licenses/hppc-0.7.1.jar.sha1 | 1 - server/licenses/hppc-0.8.1.jar.sha1 | 1 + .../lucene/queries/BlendedTermQuery.java | 44 +- .../queries/ExtendedCommonTermsQuery.java | 76 - .../apache/lucene/util/CombinedBitSet.java | 117 + .../elasticsearch/ElasticsearchException.java | 2 +- .../org/elasticsearch/ExceptionsHelper.java | 70 +- .../main/java/org/elasticsearch/Version.java | 207 +- .../elasticsearch/action/ActionModule.java | 106 +- .../action/ActionRequestBuilder.java | 4 +- .../action/{Action.java => ActionType.java} | 32 +- .../action/RequestValidators.java | 67 + .../action/RoutingMissingException.java | 5 + .../action/StreamableResponseActionType.java | 51 + .../action/TransportActionNodeProxy.java | 4 +- .../ClusterAllocationExplainAction.java | 6 +- ...ansportClusterAllocationExplainAction.java | 3 +- .../AddVotingConfigExclusionsAction.java | 9 +- .../ClearVotingConfigExclusionsAction.java | 9 +- ...nsportAddVotingConfigExclusionsAction.java | 3 +- ...portClearVotingConfigExclusionsAction.java | 3 +- .../cluster/health/ClusterHealthAction.java | 4 +- .../health/TransportClusterHealthAction.java | 7 - .../hotthreads/NodesHotThreadsAction.java | 4 +- .../TransportNodesHotThreadsAction.java | 10 +- .../cluster/node/info/NodesInfoAction.java | 4 +- .../cluster/node/info/NodesInfoResponse.java | 6 +- .../node/info/TransportNodesInfoAction.java | 10 +- .../NodesReloadSecureSettingsAction.java | 4 +- ...nsportNodesReloadSecureSettingsAction.java | 10 +- .../admin/cluster/node/stats/NodeStats.java | 5 +- .../cluster/node/stats/NodesStatsAction.java | 4 +- .../node/stats/TransportNodesStatsAction.java | 10 +- .../node/tasks/cancel/CancelTasksAction.java | 11 +- .../cluster/node/tasks/get/GetTaskAction.java | 6 +- .../tasks/get/TransportGetTaskAction.java | 2 +- .../node/tasks/list/ListTasksAction.java | 11 +- .../node/tasks/list/ListTasksResponse.java | 5 +- .../cluster/node/usage/NodesUsageAction.java | 4 +- .../node/usage/NodesUsageRequestBuilder.java | 4 +- .../node/usage/TransportNodesUsageAction.java | 10 +- .../cluster/remote/RemoteInfoAction.java | 4 +- .../delete/DeleteRepositoryAction.java | 9 +- .../TransportDeleteRepositoryAction.java | 13 +- .../get/GetRepositoriesAction.java | 9 +- .../get/GetRepositoriesResponse.java | 13 +- .../get/TransportGetRepositoriesAction.java | 12 +- .../repositories/put/PutRepositoryAction.java | 9 +- .../put/TransportPutRepositoryAction.java | 13 +- .../TransportVerifyRepositoryAction.java | 3 +- .../verify/VerifyRepositoryAction.java | 4 +- .../cluster/reroute/ClusterRerouteAction.java | 9 +- .../reroute/ClusterRerouteResponse.java | 17 +- .../TransportClusterRerouteAction.java | 12 +- .../settings/ClusterUpdateSettingsAction.java | 9 +- .../ClusterUpdateSettingsResponse.java | 32 +- .../TransportClusterUpdateSettingsAction.java | 13 +- .../shards/ClusterSearchShardsAction.java | 9 +- .../TransportClusterSearchShardsAction.java | 3 +- .../create/CreateSnapshotAction.java | 4 +- .../create/CreateSnapshotRequest.java | 53 +- .../create/TransportCreateSnapshotAction.java | 5 +- .../delete/DeleteSnapshotAction.java | 9 +- .../delete/TransportDeleteSnapshotAction.java | 13 +- .../snapshots/get/GetSnapshotsAction.java | 9 +- .../snapshots/get/GetSnapshotsRequest.java | 58 +- .../get/GetSnapshotsRequestBuilder.java | 21 +- .../snapshots/get/GetSnapshotsResponse.java | 248 +- .../get/TransportGetSnapshotsAction.java | 172 +- .../restore/RestoreSnapshotAction.java | 4 +- .../TransportRestoreSnapshotAction.java | 3 +- .../status/SnapshotsStatusAction.java | 4 +- .../status/TransportNodesSnapshotsStatus.java | 12 +- .../TransportSnapshotsStatusAction.java | 3 +- .../cluster/state/ClusterStateAction.java | 4 +- .../state/TransportClusterStateAction.java | 3 +- .../cluster/stats/ClusterStatsAction.java | 4 +- .../cluster/stats/ClusterStatsNodes.java | 27 +- .../stats/TransportClusterStatsAction.java | 10 +- .../DeleteStoredScriptAction.java | 9 +- .../storedscripts/GetStoredScriptAction.java | 4 +- .../storedscripts/PutStoredScriptAction.java | 11 +- .../TransportDeleteStoredScriptAction.java | 13 +- .../TransportGetStoredScriptAction.java | 3 +- .../TransportPutStoredScriptAction.java | 13 +- .../tasks/PendingClusterTasksAction.java | 4 +- .../TransportPendingClusterTasksAction.java | 3 +- .../indices/alias/IndicesAliasesAction.java | 9 +- .../indices/alias/IndicesAliasesRequest.java | 22 + .../alias/TransportIndicesAliasesAction.java | 48 +- .../exists/TransportAliasesExistAction.java | 68 - .../alias/get/BaseAliasesRequestBuilder.java | 4 +- .../indices/alias/get/GetAliasesAction.java | 4 +- .../alias/get/TransportGetAliasesAction.java | 3 +- .../admin/indices/analyze/AnalyzeAction.java | 779 ++++- .../admin/indices/analyze/AnalyzeRequest.java | 302 -- .../analyze/AnalyzeRequestBuilder.java | 7 +- .../indices/analyze/AnalyzeResponse.java | 320 -- .../analyze/DetailAnalyzeResponse.java | 400 --- .../analyze/TransportAnalyzeAction.java | 516 +-- .../cache/clear/ClearIndicesCacheAction.java | 4 +- .../admin/indices/close/CloseIndexAction.java | 9 +- .../indices/close/CloseIndexResponse.java | 36 +- .../close/TransportCloseIndexAction.java | 16 +- ...TransportVerifyShardBeforeCloseAction.java | 38 +- .../indices/create/CreateIndexAction.java | 9 +- .../indices/create/CreateIndexResponse.java | 16 +- .../create/TransportCreateIndexAction.java | 13 +- .../indices/delete/DeleteIndexAction.java | 9 +- .../delete/TransportDeleteIndexAction.java | 11 +- .../exists/indices/IndicesExistsRequest.java | 102 - .../indices/IndicesExistsRequestBuilder.java | 52 - .../indices/TransportIndicesExistsAction.java | 82 - .../types/TransportTypesExistsAction.java | 95 - .../exists/types/TypesExistsRequest.java | 108 - .../types/TypesExistsRequestBuilder.java | 67 - .../admin/indices/flush/FlushAction.java | 4 +- .../indices/flush/SyncedFlushAction.java | 4 +- .../indices/forcemerge/ForceMergeAction.java | 4 +- .../admin/indices/get/GetIndexAction.java | 4 +- .../mapping/get/GetFieldMappingsAction.java | 9 +- .../get/GetFieldMappingsIndexRequest.java | 27 +- .../mapping/get/GetMappingsAction.java | 4 +- .../get/TransportGetFieldMappingsAction.java | 12 +- .../TransportGetFieldMappingsIndexAction.java | 2 + .../mapping/put/MappingRequestValidator.java | 40 - .../indices/mapping/put/PutMappingAction.java | 9 +- .../mapping/put/PutMappingRequest.java | 1 - .../put/TransportPutMappingAction.java | 59 +- .../admin/indices/open/OpenIndexAction.java | 9 +- .../admin/indices/open/OpenIndexResponse.java | 9 +- .../open/TransportOpenIndexAction.java | 12 +- .../indices/recovery/RecoveryAction.java | 4 +- .../admin/indices/refresh/RefreshAction.java | 4 +- .../indices/rollover/RolloverAction.java | 9 +- .../indices/rollover/RolloverResponse.java | 45 +- .../rollover/TransportRolloverAction.java | 41 +- .../segments/IndicesSegmentsAction.java | 4 +- .../settings/get/GetSettingsAction.java | 4 +- .../get/TransportGetSettingsAction.java | 4 +- .../put/TransportUpdateSettingsAction.java | 13 +- .../settings/put/UpdateSettingsAction.java | 9 +- .../IndicesShardStoreRequestBuilder.java | 4 +- .../shards/IndicesShardStoresAction.java | 6 +- .../shards/IndicesShardStoresResponse.java | 7 +- .../TransportIndicesShardStoresAction.java | 3 +- .../admin/indices/shrink/ResizeAction.java | 9 +- .../indices/shrink/ResizeRequestBuilder.java | 4 +- .../admin/indices/shrink/ResizeResponse.java | 6 +- .../admin/indices/shrink/ShrinkAction.java | 9 +- .../indices/shrink/TransportResizeAction.java | 20 +- .../indices/stats/IndicesStatsAction.java | 4 +- .../delete/DeleteIndexTemplateAction.java | 9 +- .../TransportDeleteIndexTemplateAction.java | 13 +- .../template/get/GetIndexTemplatesAction.java | 4 +- .../get/TransportGetIndexTemplatesAction.java | 3 +- .../template/put/PutIndexTemplateAction.java | 9 +- .../put/TransportPutIndexTemplateAction.java | 13 +- .../upgrade/get/UpgradeStatusAction.java | 4 +- .../post/TransportUpgradeSettingsAction.java | 13 +- .../indices/upgrade/post/UpgradeAction.java | 4 +- .../upgrade/post/UpgradeSettingsAction.java | 9 +- .../query/TransportValidateQueryAction.java | 2 +- .../validate/query/ValidateQueryAction.java | 4 +- .../elasticsearch/action/bulk/BulkAction.java | 4 +- .../action/bulk/TransportBulkAction.java | 24 +- .../action/bulk/TransportShardBulkAction.java | 11 +- .../action/delete/DeleteAction.java | 4 +- .../action/explain/ExplainAction.java | 9 +- .../action/explain/ExplainRequest.java | 27 +- .../explain/TransportExplainAction.java | 12 +- .../fieldcaps/FieldCapabilitiesAction.java | 4 +- .../FieldCapabilitiesIndexRequest.java | 17 +- .../TransportFieldCapabilitiesAction.java | 15 +- ...TransportFieldCapabilitiesIndexAction.java | 3 + .../elasticsearch/action/get/GetAction.java | 9 +- .../elasticsearch/action/get/GetRequest.java | 31 +- .../action/get/MultiGetAction.java | 4 +- .../action/get/MultiGetShardRequest.java | 35 +- .../action/get/TransportMultiGetAction.java | 9 +- .../get/TransportShardMultiGetAction.java | 2 + .../action/index/IndexAction.java | 4 +- .../action/ingest/DeletePipelineAction.java | 9 +- .../ingest/DeletePipelineTransportAction.java | 15 +- .../action/ingest/GetPipelineAction.java | 4 +- .../ingest/GetPipelineTransportAction.java | 3 +- .../action/ingest/IngestActionForwarder.java | 4 +- .../action/ingest/PutPipelineAction.java | 9 +- .../ingest/PutPipelineTransportAction.java | 12 +- .../action/ingest/SimulatePipelineAction.java | 4 +- .../elasticsearch/action/main/MainAction.java | 4 +- .../search/AbstractSearchAsyncAction.java | 50 +- .../action/search/ClearScrollAction.java | 4 +- .../action/search/ExpandSearchPhase.java | 3 +- .../action/search/FetchSearchPhase.java | 12 +- .../action/search/InitialSearchPhase.java | 4 +- .../action/search/MultiSearchAction.java | 9 +- .../action/search/MultiSearchRequest.java | 9 - .../action/search/SearchAction.java | 9 +- .../action/search/SearchPhaseContext.java | 13 +- .../action/search/SearchPhaseController.java | 1 + .../action/search/SearchRequest.java | 54 +- .../action/search/SearchRequestBuilder.java | 17 +- .../action/search/SearchScrollAction.java | 9 +- .../action/search/TransportSearchAction.java | 178 +- .../DefaultShardOperationFailedException.java | 7 +- .../support/ThreadedActionListener.java | 39 - .../BroadcastOperationRequestBuilder.java | 4 +- .../master/AcknowledgedRequestBuilder.java | 4 +- .../support/master/AcknowledgedResponse.java | 9 +- .../MasterNodeOperationRequestBuilder.java | 4 +- ...MasterNodeReadOperationRequestBuilder.java | 4 +- .../master/ShardsAcknowledgedResponse.java | 21 +- .../master/TransportMasterNodeAction.java | 10 +- .../info/ClusterInfoRequestBuilder.java | 4 +- .../info/TransportClusterInfoAction.java | 4 +- .../action/support/nodes/BaseNodeRequest.java | 20 +- .../support/nodes/BaseNodesRequest.java | 3 +- .../nodes/NodesOperationRequestBuilder.java | 4 +- .../support/nodes/TransportNodesAction.java | 10 +- .../replication/ReplicationOperation.java | 24 +- .../ReplicationRequestBuilder.java | 4 +- .../TransportReplicationAction.java | 13 +- .../InstanceShardOperationRequestBuilder.java | 4 +- .../SingleShardOperationRequestBuilder.java | 4 +- .../single/shard/SingleShardRequest.java | 19 +- .../shard/TransportSingleShardAction.java | 7 +- .../support/tasks/TasksRequestBuilder.java | 4 +- .../termvectors/MultiTermVectorsAction.java | 4 +- .../MultiTermVectorsItemResponse.java | 10 - .../termvectors/MultiTermVectorsRequest.java | 10 +- .../MultiTermVectorsRequestBuilder.java | 9 +- .../termvectors/MultiTermVectorsResponse.java | 29 +- .../MultiTermVectorsShardRequest.java | 25 +- .../action/termvectors/TermVectorsAction.java | 9 +- .../termvectors/TermVectorsRequest.java | 142 +- .../TermVectorsRequestBuilder.java | 12 +- .../termvectors/TermVectorsResponse.java | 22 +- .../TransportMultiTermVectorsAction.java | 18 +- .../TransportShardMultiTermsVectorAction.java | 14 +- .../TransportTermVectorsAction.java | 2 +- .../action/update/UpdateAction.java | 4 +- .../org/elasticsearch/bootstrap/Security.java | 18 +- .../java/org/elasticsearch/client/Client.java | 7 +- .../client/ClusterAdminClient.java | 8 +- .../client/ElasticsearchClient.java | 10 +- .../elasticsearch/client/FilterClient.java | 4 +- .../client/IndicesAdminClient.java | 88 +- .../client/OriginSettingClient.java | 4 +- .../client/ParentTaskAssigningClient.java | 4 +- .../org/elasticsearch/client/Requests.java | 20 +- .../elasticsearch/client/node/NodeClient.java | 30 +- .../client/support/AbstractClient.java | 93 +- .../client/transport/TransportClient.java | 404 --- .../TransportClientNodesService.java | 582 ---- .../transport/TransportProxyClient.java | 59 - .../elasticsearch/cluster/ClusterModule.java | 34 - .../elasticsearch/cluster/ClusterState.java | 19 +- .../cluster/InternalClusterInfoService.java | 10 +- .../cluster/SnapshotsInProgress.java | 30 +- .../action/shard/ShardStateAction.java | 22 +- .../ClusterFormationFailureHelper.java | 15 +- .../coordination/CoordinationState.java | 55 +- .../cluster/coordination/Coordinator.java | 106 +- .../coordination/ElectionStrategy.java | 74 + .../cluster/coordination/JoinHelper.java | 5 +- .../coordination/JoinTaskExecutor.java | 13 +- .../cluster/coordination/LeaderChecker.java | 40 +- .../coordination/PreVoteCollector.java | 27 +- .../cluster/coordination/Reconfigurator.java | 10 +- .../cluster/metadata/AliasMetaData.java | 8 +- .../cluster/metadata/DiffableStringMap.java | 4 + .../cluster/metadata/IndexMetaData.java | 4 +- .../metadata/IndexTemplateMetaData.java | 14 +- .../cluster/metadata/MetaData.java | 83 +- .../metadata/MetaDataIndexStateService.java | 19 +- .../metadata/MetaDataIndexUpgradeService.java | 2 +- .../cluster/node/DiscoveryNode.java | 155 +- .../cluster/node/DiscoveryNodeRole.java | 167 + .../cluster/node/DiscoveryNodes.java | 18 +- .../routing/BatchedRerouteService.java | 135 + .../routing/IndexShardRoutingTable.java | 11 +- .../LazilyInitializedRerouteService.java | 42 + .../cluster/routing/RerouteService.java | 29 + .../cluster/routing/RoutingService.java | 122 - .../routing/allocation/AllocationService.java | 12 +- .../allocation/IndexMetaDataUpdater.java | 6 +- ...AllocateEmptyPrimaryAllocationCommand.java | 13 +- .../AllocateReplicaAllocationCommand.java | 26 +- ...AllocateStalePrimaryAllocationCommand.java | 13 +- .../decider/DiskThresholdDecider.java | 6 +- .../cluster/service/ClusterService.java | 2 +- .../org/elasticsearch/common/TriConsumer.java | 43 +- .../common/blobstore/BlobContainer.java | 16 + .../common/blobstore/BlobPath.java | 16 + .../common/blobstore/fs/FsBlobContainer.java | 21 +- .../common/compress/CompressorFactory.java | 10 +- .../org/elasticsearch/common/geo/GeoJson.java | 43 +- .../common/geo/GeometryFormat.java} | 35 +- .../common/geo/GeometryParser.java | 73 +- .../common/hash/MessageDigests.java | 19 +- .../common/hash/MurmurHash3.java | 38 + .../elasticsearch/common/inject/Binder.java | 2 +- .../assistedinject/FactoryProvider.java | 2 +- .../org/elasticsearch/common/io/Streams.java | 12 + .../common/io/stream/StreamInput.java | 28 + .../common/io/stream/StreamOutput.java | 26 + .../elasticsearch/common/lucene/Lucene.java | 38 - .../common/lucene/MinimumScoreCollector.java | 2 +- .../common/lucene/search/Queries.java | 3 - .../common/lucene/search/XMoreLikeThis.java | 2 +- .../uid/PerThreadIDVersionAndSeqNoLookup.java | 79 +- .../lucene/uid/VersionsAndSeqNoResolver.java | 21 +- .../common/network/NetworkModule.java | 23 +- .../common/network/NetworkUtils.java | 8 +- .../org/elasticsearch/common/regex/Regex.java | 15 +- .../common/settings/ClusterSettings.java | 11 +- .../settings/ConsistentSettingsService.java | 256 ++ .../common/settings/KeyStoreWrapper.java | 61 +- .../common/settings/SecureSetting.java | 19 +- .../common/settings/SecureSettings.java | 2 + .../common/settings/Setting.java | 20 + .../common/settings/Settings.java | 9 +- .../common/settings/SettingsModule.java | 22 +- .../common/time/DateMathParser.java | 18 +- .../common/time/JavaDateFormatter.java | 17 +- .../common/time/JavaDateMathParser.java | 53 +- .../common/util/CuckooFilter.java | 521 +++ .../util/SetBackedScalingCuckooFilter.java | 408 +++ .../util/concurrent/AsyncIOProcessor.java | 56 +- .../common/util/concurrent/EsExecutors.java | 1 + .../discovery/DiscoveryModule.java | 24 +- .../HandshakingTransportAddressConnector.java | 8 + .../discovery/SeedHostsResolver.java | 15 +- .../elasticsearch/env/NodeEnvironment.java | 186 +- .../gateway/GatewayAllocator.java | 16 +- .../elasticsearch/gateway/GatewayService.java | 11 +- .../TransportNodesListGatewayMetaState.java | 15 +- ...ransportNodesListGatewayStartedShards.java | 10 +- .../http/AbstractHttpServerTransport.java | 2 + .../org/elasticsearch/http/CorsHandler.java | 258 ++ .../http/DefaultRestChannel.java | 57 +- .../org/elasticsearch/http/HttpRequest.java | 6 + .../org/elasticsearch/index/IndexModule.java | 49 +- .../org/elasticsearch/index/IndexService.java | 53 +- .../elasticsearch/index/IndexSettings.java | 2 +- .../elasticsearch/index/SearchSlowLog.java | 9 +- .../index/analysis/AnalysisRegistry.java | 164 +- .../index/analysis/AnalyzerComponents.java | 111 + .../analysis/AnalyzerComponentsProvider.java | 13 +- .../index/analysis/CustomAnalyzer.java | 51 +- .../analysis/CustomAnalyzerProvider.java | 72 +- .../index/analysis/IndexAnalyzers.java | 22 +- .../index/analysis/NameOrDefinition.java | 115 + .../index/analysis/NamedAnalyzer.java | 4 +- .../analysis/ReloadableCustomAnalyzer.java | 162 + .../index/engine/CombinedDeletionPolicy.java | 24 +- .../index/engine/CombinedDocValues.java | 90 + .../elasticsearch/index/engine/Engine.java | 12 +- .../index/engine/InternalEngine.java | 273 +- .../index/engine/LuceneChangesSnapshot.java | 69 +- .../index/engine/NoOpEngine.java | 62 +- .../engine/PrunePostingsMergePolicy.java | 224 ++ .../index/engine/ReadOnlyEngine.java | 43 +- .../index/fielddata/IndexFieldData.java | 8 +- .../fielddata/IndexOrdinalsFieldData.java | 7 + .../ordinals/GlobalOrdinalMapping.java | 12 +- .../GlobalOrdinalsIndexFieldData.java | 162 +- .../plain/AbstractIndexOrdinalsFieldData.java | 17 + .../SortedSetDVOrdinalsIndexFieldData.java | 17 + .../fieldvisitor/IdOnlyFieldVisitor.java | 60 + .../index/mapper/BaseGeoShapeFieldMapper.java | 2 +- .../index/mapper/ContentPath.java | 4 + .../index/mapper/DocumentParser.java | 8 +- .../index/mapper/DynamicKeyFieldMapper.java | 54 + .../index/mapper/FieldMapper.java | 28 +- .../index/mapper/FieldTypeLookup.java | 127 +- .../mapper/LegacyGeoShapeFieldMapper.java | 5 +- .../index/mapper/MappedFieldType.java | 3 +- .../index/mapper/MapperService.java | 34 +- .../index/mapper/ParseContext.java | 17 +- .../index/mapper/TextFieldMapper.java | 61 +- .../index/mapper/TypeParsers.java | 18 +- .../index/query/CommonTermsQueryBuilder.java | 406 --- .../index/query/IdsQueryBuilder.java | 75 +- .../index/query/IntervalBuilder.java | 4 +- .../index/query/IntervalsSourceProvider.java | 232 +- .../index/query/MatchQueryBuilder.java | 44 +- .../index/query/MoreLikeThisQueryBuilder.java | 97 +- .../index/query/MultiMatchQueryBuilder.java | 54 +- .../index/query/QueryBuilders.java | 10 - .../index/query/QueryShardContext.java | 57 +- .../ScriptScoreQueryBuilder.java | 8 +- .../reindex/AbstractBulkByScrollRequest.java | 35 +- .../AbstractBulkByScrollRequestBuilder.java | 20 +- ...stractBulkIndexByScrollRequestBuilder.java | 6 +- .../reindex/ClientScrollableHitSource.java | 5 +- .../index/reindex/DeleteByQueryAction.java | 4 +- .../index/reindex/DeleteByQueryRequest.java | 51 - .../reindex/DeleteByQueryRequestBuilder.java | 6 +- .../index/reindex/ReindexAction.java | 4 +- .../index/reindex/ReindexRequest.java | 199 +- .../index/reindex/ReindexRequestBuilder.java | 6 +- .../index/reindex/RemoteInfo.java | 57 + .../index/reindex/UpdateByQueryAction.java | 4 +- .../index/reindex/UpdateByQueryRequest.java | 28 - .../reindex/UpdateByQueryRequestBuilder.java | 6 +- .../index/search/MatchQuery.java | 46 +- .../index/search/MultiMatchQuery.java | 27 +- .../index/search/NestedHelper.java | 2 +- .../index/search/QueryParserHelper.java | 27 +- .../index/search/QueryStringQueryParser.java | 9 +- .../seqno/GlobalCheckpointSyncAction.java | 30 +- .../index/seqno/LocalCheckpointTracker.java | 151 +- .../index/seqno/ReplicationTracker.java | 189 +- .../index/seqno/RetentionLeaseActions.java | 52 +- .../RetentionLeaseBackgroundSyncAction.java | 46 +- .../index/seqno/RetentionLeaseSyncAction.java | 44 +- .../elasticsearch/index/seqno/SeqNoStats.java | 2 +- .../index/seqno/SequenceNumbers.java | 4 - .../index/shard/IndexSearcherWrapper.java | 138 - .../elasticsearch/index/shard/IndexShard.java | 154 +- .../index/shard/PrimaryReplicaSyncer.java | 2 +- .../RemoveCorruptedShardDataCommand.java | 5 +- .../index/similarity/SimilarityService.java | 28 +- .../index/store/FsDirectoryFactory.java | 133 +- .../org/elasticsearch/index/store/Store.java | 147 +- .../index/termvectors/TermVectorsService.java | 15 +- .../index/translog/BaseTranslogReader.java | 2 +- .../index/translog/Translog.java | 22 +- .../index/translog/TranslogHeader.java | 92 +- .../index/translog/TranslogSnapshot.java | 22 +- .../index/translog/TranslogWriter.java | 30 +- .../translog/TruncateTranslogAction.java | 14 +- .../elasticsearch/indices/IndicesModule.java | 2 - .../elasticsearch/indices/IndicesService.java | 9 +- .../cluster/IndicesClusterStateService.java | 110 +- .../recovery/PeerRecoveryTargetService.java | 18 +- .../recovery/RecoverySourceHandler.java | 219 +- .../indices/recovery/RecoveryState.java | 4 +- .../indices/recovery/RecoveryTarget.java | 117 +- .../recovery/RecoveryTargetHandler.java | 6 +- .../recovery/RemoteRecoveryTargetHandler.java | 18 +- .../TransportNodesListShardStoreMetaData.java | 10 +- .../elasticsearch/ingest/IngestService.java | 219 +- .../org/elasticsearch/monitor/os/OsProbe.java | 44 +- .../org/elasticsearch/monitor/os/OsStats.java | 4 + .../node/InternalSettingsPreparer.java | 5 +- .../java/org/elasticsearch/node/Node.java | 51 +- .../CompletionPersistentTaskAction.java | 9 +- .../PersistentTasksNodeService.java | 2 +- .../persistent/PersistentTasksService.java | 4 +- .../RemovePersistentTaskAction.java | 7 +- .../persistent/StartPersistentTaskAction.java | 7 +- .../UpdatePersistentTaskStatusAction.java | 7 +- .../elasticsearch/plugins/ActionPlugin.java | 53 +- .../plugins/DiscoveryPlugin.java | 8 + .../org/elasticsearch/plugins/Plugin.java | 22 +- .../plugins/PluginLoaderIndirection.java | 1 + .../elasticsearch/plugins/PluginsService.java | 21 +- .../repositories/FilterRepository.java | 6 +- .../repositories/Repository.java | 4 +- .../repositories/RepositoryData.java | 4 +- .../blobstore/BlobStoreRepository.java | 908 +++--- .../blobstore/ChecksumBlobStoreFormat.java | 33 +- .../repositories/blobstore/package-info.java | 16 +- .../repositories/fs/FsRepository.java | 10 +- .../elasticsearch/rest/RestController.java | 90 +- .../org/elasticsearch/rest/RestRequest.java | 6 + .../admin/cluster/RestGetSnapshotsAction.java | 4 +- .../admin/indices/RestAnalyzeAction.java | 104 +- .../rest/action/cat/RestIndicesAction.java | 286 +- .../rest/action/cat/RestNodesAction.java | 3 +- .../rest/action/cat/RestSnapshotAction.java | 74 +- .../document/RestMultiTermVectorsAction.java | 4 - .../document/RestTermVectorsAction.java | 3 +- .../rest/action/search/RestCountAction.java | 15 - .../action/search/RestMultiSearchAction.java | 2 +- .../search/DefaultSearchContext.java | 22 +- .../org/elasticsearch/search/SearchHits.java | 2 +- .../elasticsearch/search/SearchModule.java | 49 +- .../elasticsearch/search/SearchService.java | 41 +- .../AbstractAggregationBuilder.java | 31 +- .../aggregations/AggregationBuilder.java | 2 +- .../aggregations/AggregatorFactories.java | 12 +- .../aggregations/AggregatorFactory.java | 14 +- .../aggregations/InternalAggregation.java | 28 +- .../PipelineAggregationBuilder.java | 2 +- .../bucket/BestBucketsDeferringCollector.java | 68 +- .../bucket/BucketsAggregator.java | 4 +- .../InternalSingleBucketAggregation.java | 10 +- .../MergingBucketsDeferringCollector.java | 254 +- .../AdjacencyMatrixAggregationBuilder.java | 11 +- .../AdjacencyMatrixAggregatorFactory.java | 4 +- .../adjacency/InternalAdjacencyMatrix.java | 10 +- .../CompositeAggregationBuilder.java | 15 +- .../CompositeAggregationFactory.java | 4 +- .../CompositeValuesSourceBuilder.java | 11 +- .../DateHistogramValuesSourceBuilder.java | 10 +- .../HistogramValuesSourceBuilder.java | 10 +- .../bucket/composite/InternalComposite.java | 10 +- .../composite/TermsValuesSourceBuilder.java | 10 - .../filter/FilterAggregationBuilder.java | 11 +- .../filter/FilterAggregatorFactory.java | 4 +- .../filter/FiltersAggregationBuilder.java | 11 +- .../filter/FiltersAggregatorFactory.java | 4 +- .../bucket/filter/InternalFilters.java | 10 +- .../geogrid/GeoGridAggregationBuilder.java | 30 +- .../GeoHashGridAggregationBuilder.java | 4 +- .../geogrid/GeoHashGridAggregatorFactory.java | 4 +- .../GeoTileGridAggregationBuilder.java | 4 +- .../geogrid/GeoTileGridAggregatorFactory.java | 4 +- .../bucket/geogrid/InternalGeoGrid.java | 14 +- .../global/GlobalAggregationBuilder.java | 12 +- .../global/GlobalAggregatorFactory.java | 4 +- .../AutoDateHistogramAggregationBuilder.java | 119 +- .../AutoDateHistogramAggregator.java | 2 +- .../AutoDateHistogramAggregatorFactory.java | 13 +- .../DateHistogramAggregationBuilder.java | 13 +- .../DateHistogramAggregatorFactory.java | 4 +- .../HistogramAggregationBuilder.java | 25 +- .../histogram/HistogramAggregatorFactory.java | 4 +- .../histogram/InternalAutoDateHistogram.java | 10 +- .../histogram/InternalDateHistogram.java | 10 +- .../bucket/histogram/InternalHistogram.java | 10 +- .../missing/MissingAggregationBuilder.java | 16 +- .../missing/MissingAggregatorFactory.java | 4 +- .../nested/NestedAggregationBuilder.java | 12 +- .../nested/NestedAggregatorFactory.java | 4 +- .../ReverseNestedAggregationBuilder.java | 14 +- .../ReverseNestedAggregatorFactory.java | 4 +- .../range/AbstractRangeAggregatorFactory.java | 5 +- .../bucket/range/AbstractRangeBuilder.java | 9 +- .../range/BinaryRangeAggregatorFactory.java | 4 +- .../range/DateRangeAggregationBuilder.java | 2 +- .../range/DateRangeAggregatorFactory.java | 4 +- .../range/GeoDistanceAggregationBuilder.java | 13 +- .../GeoDistanceRangeAggregatorFactory.java | 4 +- .../bucket/range/InternalBinaryRange.java | 12 +- .../bucket/range/InternalRange.java | 10 +- .../range/IpRangeAggregationBuilder.java | 15 +- .../bucket/range/RangeAggregationBuilder.java | 2 +- .../bucket/range/RangeAggregatorFactory.java | 4 +- .../DiversifiedAggregationBuilder.java | 17 +- .../sampler/DiversifiedAggregatorFactory.java | 4 +- .../sampler/SamplerAggregationBuilder.java | 11 +- .../sampler/SamplerAggregatorFactory.java | 4 +- .../InternalMappedSignificantTerms.java | 13 +- .../significant/InternalSignificantTerms.java | 10 +- .../significant/SignificantStringTerms.java | 4 + .../SignificantTermsAggregationBuilder.java | 13 +- .../SignificantTermsAggregatorFactory.java | 4 +- .../SignificantTextAggregationBuilder.java | 14 +- .../SignificantTextAggregatorFactory.java | 4 +- .../terms/AbstractRareTermsAggregator.java | 134 + .../bucket/terms/IncludeExclude.java | 8 +- .../bucket/terms/InternalMappedRareTerms.java | 182 ++ .../bucket/terms/InternalMappedTerms.java | 13 +- .../bucket/terms/InternalRareTerms.java | 205 ++ .../bucket/terms/InternalTerms.java | 10 +- .../bucket/terms/LongRareTerms.java | 156 + .../bucket/terms/LongRareTermsAggregator.java | 169 + .../aggregations/bucket/terms/RareTerms.java | 48 + .../terms/RareTermsAggregationBuilder.java | 203 ++ .../terms/RareTermsAggregatorFactory.java | 164 + .../bucket/terms/StringRareTerms.java | 159 + .../terms/StringRareTermsAggregator.java | 175 + .../bucket/terms/StringTermsAggregator.java | 14 +- .../bucket/terms/TermsAggregationBuilder.java | 24 +- .../bucket/terms/TermsAggregatorFactory.java | 15 +- .../bucket/terms/UnmappedRareTerms.java | 119 + .../AbstractInternalHDRPercentiles.java | 14 +- .../AbstractInternalTDigestPercentiles.java | 10 +- .../metrics/AvgAggregationBuilder.java | 12 +- .../metrics/AvgAggregatorFactory.java | 4 +- .../CardinalityAggregationBuilder.java | 11 +- .../metrics/CardinalityAggregatorFactory.java | 4 +- .../ExtendedStatsAggregationBuilder.java | 11 +- .../ExtendedStatsAggregatorFactory.java | 4 +- .../metrics/GeoBoundsAggregationBuilder.java | 11 +- .../metrics/GeoBoundsAggregatorFactory.java | 4 +- .../GeoCentroidAggregationBuilder.java | 12 +- .../metrics/GeoCentroidAggregatorFactory.java | 4 +- .../HDRPercentileRanksAggregatorFactory.java | 4 +- .../HDRPercentilesAggregatorFactory.java | 4 +- .../aggregations/metrics/InternalAvg.java | 9 +- .../metrics/InternalCardinality.java | 11 +- .../metrics/InternalExtendedStats.java | 13 +- .../metrics/InternalGeoBounds.java | 10 +- .../metrics/InternalGeoCentroid.java | 11 +- .../aggregations/metrics/InternalMax.java | 9 +- .../InternalMedianAbsoluteDeviation.java | 9 +- .../aggregations/metrics/InternalMin.java | 9 +- .../InternalNumericMetricsAggregation.java | 15 +- .../metrics/InternalScriptedMetric.java | 10 +- .../aggregations/metrics/InternalStats.java | 10 +- .../aggregations/metrics/InternalSum.java | 10 +- .../aggregations/metrics/InternalTopHits.java | 13 +- .../metrics/InternalValueCount.java | 10 +- .../metrics/InternalWeightedAvg.java | 9 +- .../metrics/MaxAggregationBuilder.java | 12 +- .../metrics/MaxAggregatorFactory.java | 4 +- ...anAbsoluteDeviationAggregationBuilder.java | 13 +- ...ianAbsoluteDeviationAggregatorFactory.java | 5 +- .../metrics/MinAggregationBuilder.java | 12 +- .../metrics/MinAggregatorFactory.java | 4 +- .../PercentileRanksAggregationBuilder.java | 47 +- .../PercentilesAggregationBuilder.java | 40 +- .../ScriptedMetricAggregationBuilder.java | 19 +- .../ScriptedMetricAggregatorFactory.java | 4 +- .../metrics/StatsAggregationBuilder.java | 12 +- .../metrics/StatsAggregatorFactory.java | 4 +- .../metrics/SumAggregationBuilder.java | 12 +- .../metrics/SumAggregatorFactory.java | 4 +- ...igestPercentileRanksAggregatorFactory.java | 4 +- .../TDigestPercentilesAggregatorFactory.java | 4 +- .../metrics/TopHitsAggregationBuilder.java | 37 +- .../metrics/TopHitsAggregatorFactory.java | 4 +- .../metrics/ValueCountAggregationBuilder.java | 12 +- .../metrics/ValueCountAggregatorFactory.java | 4 +- .../WeightedAvgAggregationBuilder.java | 14 +- .../metrics/WeightedAvgAggregatorFactory.java | 4 +- .../AbstractPipelineAggregationBuilder.java | 32 +- .../AvgBucketPipelineAggregationBuilder.java | 10 - ...cketMetricsPipelineAggregationBuilder.java | 18 +- ...ucketScriptPipelineAggregationBuilder.java | 15 +- ...ketSelectorPipelineAggregationBuilder.java | 15 +- .../BucketSortPipelineAggregationBuilder.java | 11 +- ...mulativeSumPipelineAggregationBuilder.java | 11 +- .../DerivativePipelineAggregationBuilder.java | 24 +- ...StatsBucketPipelineAggregationBuilder.java | 11 +- .../pipeline/InternalBucketMetricValue.java | 9 +- .../pipeline/InternalDerivative.java | 9 +- .../pipeline/InternalPercentilesBucket.java | 10 +- .../pipeline/InternalSimpleValue.java | 9 +- .../MaxBucketPipelineAggregationBuilder.java | 10 - .../MinBucketPipelineAggregationBuilder.java | 10 - .../MovFnPipelineAggregationBuilder.java | 11 +- ...tilesBucketPipelineAggregationBuilder.java | 14 +- .../SerialDiffPipelineAggregationBuilder.java | 12 +- .../pipeline/SiblingPipelineAggregator.java | 2 +- ...StatsBucketPipelineAggregationBuilder.java | 10 - .../SumBucketPipelineAggregationBuilder.java | 10 - .../MultiValuesSourceAggregationBuilder.java | 36 +- .../MultiValuesSourceAggregatorFactory.java | 6 +- .../aggregations/support/ValuesSource.java | 15 + .../ValuesSourceAggregationBuilder.java | 48 +- .../ValuesSourceAggregatorFactory.java | 22 +- .../support/ValuesSourceConfig.java | 12 +- .../search/fetch/FetchPhase.java | 38 +- .../highlight/FragmentBuilderHelper.java | 7 +- .../search/internal/ContextIndexSearcher.java | 127 +- .../internal/ShardSearchLocalRequest.java | 31 +- .../search/internal/ShardSearchRequest.java | 2 - .../internal/ShardSearchTransportRequest.java | 5 - .../search/lookup/DocLookup.java | 13 +- .../search/lookup/FieldsLookup.java | 8 +- .../search/lookup/LeafDocLookup.java | 11 +- .../search/lookup/LeafFieldsLookup.java | 10 +- .../search/lookup/SearchLookup.java | 8 +- .../search/slice/SliceBuilder.java | 8 +- .../search/slice/TermsSliceQuery.java | 3 + .../search/sort/FieldSortBuilder.java | 235 +- .../search/sort/GeoDistanceSortBuilder.java | 132 +- .../search/sort/ScriptSortBuilder.java | 121 +- .../search/sort/SortBuilder.java | 15 +- .../search/suggest/Suggester.java | 7 +- .../completion/CompletionSuggester.java | 19 +- .../suggest/phrase/PhraseSuggester.java | 9 + .../phrase/PhraseSuggestionBuilder.java | 7 +- .../search/suggest/term/TermSuggester.java | 12 + .../elasticsearch/snapshots/SnapshotInfo.java | 71 +- .../snapshots/SnapshotShardsService.java | 3 +- .../snapshots/SnapshotsService.java | 11 +- .../elasticsearch/snapshots/package-info.java | 6 +- .../elasticsearch/threadpool/Scheduler.java | 7 +- .../transport/RemoteClusterAware.java | 4 +- .../transport/RemoteClusterAwareClient.java | 4 +- .../transport/RemoteClusterConnection.java | 1 + .../transport/RemoteClusterService.java | 13 +- .../elasticsearch/transport/TcpTransport.java | 56 +- .../transport/TransportService.java | 9 +- .../elasticsearch/bootstrap/security.policy | 4 +- .../CustomUnifiedHighlighterTests.java | 16 - .../lucene/util/CombinedBitSetTests.java | 111 + .../java/org/elasticsearch/BuildTests.java | 10 - .../elasticsearch/ExceptionsHelperTests.java | 39 +- .../java/org/elasticsearch/VersionTests.java | 72 +- .../action/ActionModuleTests.java | 8 +- .../org/elasticsearch/action/ActionTests.java | 2 +- .../action/IndicesRequestIT.java | 7 +- .../action/ListenerActionIT.java | 8 +- .../action/RequestValidatorsTests.java | 94 + .../ShardValidateQueryRequestTests.java | 2 +- ...AddVotingConfigExclusionsRequestTests.java | 58 +- ...tAddVotingConfigExclusionsActionTests.java | 13 +- .../node/tasks/CancellableTasksTests.java | 28 +- .../node/tasks/TaskManagerTestCase.java | 3 +- .../admin/cluster/node/tasks/TasksIT.java | 7 +- .../cluster/node/tasks/TestTaskPlugin.java | 35 +- .../node/tasks/TransportTasksActionTests.java | 11 +- .../ClusterUpdateSettingsResponseTests.java | 10 +- .../ClusterSearchShardsResponseTests.java | 2 +- .../cluster/snapshots/SnapshotBlocksIT.java | 11 +- .../create/CreateSnapshotRequestTests.java | 63 + .../create/CreateSnapshotResponseTests.java | 15 +- .../get/GetSnapshotsResponseTests.java | 101 +- .../state/ClusterStateRequestTests.java | 6 +- .../admin/cluster/stats/ClusterStatsIT.java | 14 +- .../indices/TransportAnalyzeActionTests.java | 240 +- .../ValidateIndicesAliasesRequestIT.java | 133 + .../indices/analyze/AnalyzeRequestTests.java | 41 +- .../indices/analyze/AnalyzeResponseTests.java | 109 +- .../clear/ClearIndicesCacheResponseTests.java | 2 +- .../indices/close/CloseIndexRequestTests.java | 8 +- .../close/CloseIndexResponseTests.java | 76 +- ...portVerifyShardBeforeCloseActionTests.java | 18 +- .../create/CreateIndexResponseTests.java | 9 +- .../admin/indices/create/ShrinkIndexIT.java | 2 - .../admin/indices/exists/IndicesExistsIT.java | 47 - .../indices/flush/FlushResponseTests.java | 2 +- .../forcemerge/ForceMergeResponseTests.java | 2 +- .../put/ValidateMappingRequestPluginIT.java | 9 +- .../indices/open/OpenIndexResponseTests.java | 9 +- .../indices/refresh/RefreshResponseTests.java | 2 +- .../rollover/RolloverResponseTests.java | 10 +- .../TransportRolloverActionTests.java | 126 +- .../settings/get/GetSettingsActionTests.java | 6 +- .../shards/IndicesShardStoreRequestIT.java | 3 - .../indices/shrink/ResizeResponseTests.java | 9 +- .../shrink/TransportResizeActionTests.java | 3 +- .../query/ValidateQueryResponseTests.java | 2 +- .../action/bulk/BulkProcessorIT.java | 44 - .../action/bulk/BulkProcessorRetryIT.java | 1 - .../action/bulk/BulkRejectionIT.java | 17 + ...ActionIndicesThatCannotBeCreatedTests.java | 2 +- .../bulk/TransportBulkActionIngestTests.java | 28 +- .../action/bulk/TransportBulkActionTests.java | 2 +- .../bulk/TransportBulkActionTookTests.java | 10 +- .../action/explain/ExplainRequestTests.java | 5 +- .../action/get/MultiGetShardRequestTests.java | 4 +- .../get/TransportMultiGetActionTests.java | 4 +- .../action/search/ExpandSearchPhaseTests.java | 48 +- .../action/search/FetchSearchPhaseTests.java | 109 +- .../action/search/MockSearchPhaseContext.java | 12 +- .../search/MultiSearchRequestTests.java | 23 +- .../action/search/SearchAsyncActionTests.java | 97 + .../search/SearchRequestBuilderTests.java | 36 +- .../action/search/SearchRequestTests.java | 7 - .../action/search/SearchResponseTests.java | 2 +- .../TransportMultiSearchActionTests.java | 7 +- .../TransportSearchActionSingleNodeTests.java | 61 + .../search/TransportSearchActionTests.java | 75 + .../support/ActiveShardsObserverIT.java | 2 +- .../master/IndexingMasterFailoverIT.java | 2 +- .../ShardsAcknowledgedResponseTests.java | 64 + .../TransportMasterNodeActionTests.java | 12 +- .../TransportMasterNodeActionUtils.java | 8 +- .../nodes/TransportNodesActionTests.java | 11 +- .../ClusterStateCreationUtils.java | 4 +- .../ReplicationOperationTests.java | 6 + ...ReplicationAllPermitsAcquisitionTests.java | 4 +- .../AbstractTermVectorsTestCase.java | 2 +- .../action/termvectors/GetTermVectorsIT.java | 55 +- .../termvectors/GetTermVectorsTests.java | 2 +- .../termvectors/MultiTermVectorsIT.java | 28 +- .../termvectors/TermVectorsUnitTests.java | 68 +- .../TransportMultiTermVectorsActionTests.java | 12 +- .../elasticsearch/aliases/IndexAliasesIT.java | 70 +- .../elasticsearch/blocks/SimpleBlocksIT.java | 24 +- .../RecoveryWithUnsupportedIndicesIT.java | 17 +- .../client/AbstractClientHeadersTestCase.java | 6 +- .../client/OriginSettingClientTests.java | 4 +- .../ParentTaskAssigningClientTests.java | 4 +- .../client/node/NodeClientHeadersTests.java | 10 +- .../cluster/ClusterChangedEventTests.java | 15 +- .../cluster/ClusterModuleTests.java | 27 - .../cluster/ClusterStateDiffIT.java | 14 +- .../elasticsearch/cluster/DiskUsageTests.java | 2 +- .../cluster/FeatureAwareTests.java | 21 +- .../cluster/MinimumMasterNodesIT.java | 4 +- .../elasticsearch/cluster/NoMasterNodeIT.java | 9 +- .../cluster/NodeConnectionsServiceTests.java | 6 +- .../cluster/SimpleClusterStateIT.java | 2 - .../cluster/SimpleDataNodesIT.java | 54 +- .../cluster/SnapshotsInProgressTests.java | 3 +- .../cluster/SpecificMasterNodesIT.java | 4 +- .../action/shard/ShardStateActionTests.java | 6 +- .../ClusterBootstrapServiceTests.java | 33 +- .../ClusterFormationFailureHelperTests.java | 114 +- .../coordination/CoordinationStateTests.java | 202 +- .../coordination/CoordinatorTests.java | 1332 +------- .../coordination/FollowersCheckerTests.java | 6 +- .../cluster/coordination/JoinHelperTests.java | 4 +- .../coordination/JoinTaskExecutorTests.java | 29 +- .../coordination/LeaderCheckerTests.java | 15 +- .../cluster/coordination/NodeJoinTests.java | 14 +- .../coordination/PreVoteCollectorTests.java | 6 +- .../coordination/PublicationTests.java | 8 +- .../coordination/RareClusterStateIT.java | 6 +- .../coordination/ReconfiguratorTests.java | 2 +- .../UnsafeBootstrapAndDetachCommandIT.java | 6 +- .../coordination/VotingConfigurationIT.java | 75 +- .../cluster/coordination/ZenDiscoveryIT.java | 2 - .../metadata/AutoExpandReplicasTests.java | 16 +- .../MetaDataCreateIndexServiceTests.java | 9 +- .../MetaDataDeleteIndexServiceTests.java | 4 +- .../MetaDataIndexStateServiceTests.java | 9 +- .../metadata/TemplateUpgradeServiceTests.java | 6 +- .../cluster/node/DiscoveryNodeRoleIT.java | 99 + .../cluster/node/DiscoveryNodeRoleTests.java | 100 + .../cluster/node/DiscoveryNodesTests.java | 38 +- .../routing/BatchedRerouteServiceTests.java | 146 + .../DelayedAllocationServiceTests.java | 4 +- .../cluster/routing/PrimaryAllocationIT.java | 3 - .../allocation/AllocationCommandsTests.java | 80 +- .../allocation/BalanceConfigurationTests.java | 22 +- .../allocation/FailedNodeRoutingTests.java | 17 +- .../RetryFailedAllocationTests.java | 97 + .../decider/DiskThresholdDeciderTests.java | 7 +- .../DiskThresholdDeciderUnitTests.java | 14 +- .../allocation/decider/MockDiskUsagesIT.java | 6 +- ...storeInProgressAllocationDeciderTests.java | 3 +- .../cluster/service/ClusterServiceIT.java | 2 - .../cluster/shards/ClusterShardLimitIT.java | 2 +- .../common/geo/BaseGeoParsingTestCase.java | 3 +- .../common/geo/GeoDistanceTests.java | 17 - .../common/geo/GeoJsonParserTests.java | 47 +- .../common/geo/GeoJsonSerializationTests.java | 4 +- .../common/geo/GeometryParserTests.java | 42 +- .../common/io/stream/StreamTests.java | 29 + .../common/joda/JodaDateMathParserTests.java | 5 + .../common/lucene/uid/VersionLookupTests.java | 5 + .../common/lucene/uid/VersionsTests.java | 13 + .../common/network/NetworkModuleTests.java | 26 +- .../common/network/NetworkUtilsTests.java | 30 +- .../common/regex/RegexTests.java | 20 + .../common/settings/ConsistentSettingsIT.java | 188 ++ .../ConsistentSettingsServiceTests.java | 159 + .../common/settings/SettingsModuleTests.java | 36 + .../common/time/JavaDateMathParserTests.java | 18 + .../common/util/CuckooFilterTests.java | 135 + .../SetBackedScalingCuckooFilterTests.java | 231 ++ .../concurrent/AsyncIOProcessorTests.java | 134 +- ...ueueResizingEsThreadPoolExecutorTests.java | 7 +- .../discovery/ClusterDisruptionIT.java | 4 +- .../discovery/DiscoveryDisruptionIT.java | 2 - .../discovery/DiscoveryModuleTests.java | 4 +- .../discovery/DiskDisruptionIT.java | 178 + .../discovery/MasterDisruptionIT.java | 17 - .../discovery/SeedHostsResolverTests.java | 64 +- .../discovery/SnapshotDisruptionIT.java | 4 +- .../discovery/StableMasterDisruptionIT.java | 3 +- .../single/SingleNodeDiscoveryIT.java | 5 +- .../elasticsearch/env/NodeEnvironmentIT.java | 117 +- .../env/NodeEnvironmentTests.java | 12 +- .../gateway/AsyncShardFetchTests.java | 5 +- .../gateway/ClusterStateUpdatersTests.java | 5 +- .../gateway/GatewayIndexStateIT.java | 57 +- .../GatewayMetaStatePersistedStateTests.java | 3 +- .../gateway/GatewayMetaStateTests.java | 4 +- .../gateway/RecoverAfterNodesIT.java | 2 +- .../elasticsearch/http/CorsHandlerTests.java | 90 + .../http/DefaultRestChannelTests.java | 80 +- .../elasticsearch/index/IndexModuleTests.java | 20 +- .../index/IndexServiceTests.java | 47 +- .../index/IndexSettingsTests.java | 15 +- .../index/SearchSlowLogTests.java | 5 - .../index/SettingsListenerIT.java | 17 +- .../index/analysis/IndexAnalyzersTests.java | 53 +- .../ReloadableCustomAnalyzerTests.java | 168 + .../engine/CombinedDeletionPolicyTests.java | 48 - .../index/engine/InternalEngineMergeIT.java | 2 - .../index/engine/InternalEngineTests.java | 527 ++- .../engine/LuceneChangesSnapshotTests.java | 17 +- .../index/engine/NoOpEngineTests.java | 64 +- .../engine/PrunePostingsMergePolicyTests.java | 165 + .../index/engine/ReadOnlyEngineTests.java | 66 +- .../fielddata/AbstractFieldDataTestCase.java | 2 +- .../AbstractStringFieldDataTestCase.java | 7 +- .../fielddata/IndexFieldDataServiceTests.java | 2 +- .../index/mapper/BooleanFieldMapperTests.java | 38 +- .../index/mapper/DateFieldTypeTests.java | 4 +- .../index/mapper/DocumentParserTests.java | 60 + .../mapper/ExternalFieldMapperTests.java | 89 - .../ExternalValuesMapperIntegrationIT.java | 10 +- .../mapper/FieldNamesFieldTypeTests.java | 4 +- .../index/mapper/IdFieldTypeTests.java | 7 - .../index/mapper/KeywordFieldMapperTests.java | 79 + .../index/mapper/MapperServiceTests.java | 109 +- .../index/mapper/NestedObjectMapperTests.java | 42 + .../index/mapper/RangeFieldTypeTests.java | 2 +- .../index/mapper/TextFieldMapperTests.java | 89 +- .../index/mapper/TypeParsersTests.java | 50 +- .../query/AbstractQueryBuilderTests.java | 2 +- .../query/CommonTermsQueryBuilderTests.java | 208 -- .../query/CommonTermsQueryParserTests.java | 50 - .../index/query/FullTextQueryTestCase.java | 60 - .../query/GeoShapeQueryBuilderTests.java | 2 +- .../index/query/IdsQueryBuilderTests.java | 75 +- .../index/query/InnerHitBuilderTests.java | 2 +- .../index/query/IntervalBuilderTests.java | 6 + .../query/IntervalQueryBuilderTests.java | 110 +- .../index/query/MatchQueryBuilderTests.java | 49 +- .../query/MoreLikeThisQueryBuilderTests.java | 51 +- .../query/MultiMatchQueryBuilderTests.java | 58 +- .../index/query/QueryShardContextTests.java | 2 +- .../query/QueryStringQueryBuilderTests.java | 155 +- .../index/query/RandomQueryBuilder.java | 3 +- .../index/query/RangeQueryBuilderTests.java | 21 + .../index/query/RangeQueryRewriteTests.java | 12 +- .../index/query/ScriptQueryBuilderTests.java | 11 +- .../query/ScriptScoreQueryBuilderTests.java | 11 +- .../query/SimpleQueryStringBuilderTests.java | 50 +- .../query/SpanMultiTermQueryBuilderTests.java | 24 +- .../index/query/TermsQueryBuilderTests.java | 7 - .../query/TermsSetQueryBuilderTests.java | 36 +- .../index/query/TypeQueryBuilderTests.java | 6 + .../FunctionScoreQueryBuilderTests.java | 36 +- .../AbstractBulkByScrollRequestTestCase.java | 15 +- .../reindex/DeleteByQueryRequestTests.java | 51 +- .../index/reindex/ReindexRequestTests.java | 248 +- .../reindex/UpdateByQueryRequestTests.java | 27 + .../IndexLevelReplicationTests.java | 17 +- .../RecoveryDuringReplicationTests.java | 36 +- .../index/search/MatchPhraseQueryIT.java | 5 +- .../index/search/MultiMatchQueryTests.java | 14 +- .../search/nested/NestedSortingTests.java | 2 + .../GlobalCheckpointSyncActionTests.java | 2 +- .../index/seqno/GlobalCheckpointSyncIT.java | 37 +- .../seqno/LocalCheckpointTrackerTests.java | 148 +- .../index/seqno/ReplicationTrackerTests.java | 4 +- ...tentionLeaseBackgroundSyncActionTests.java | 79 - .../seqno/RetentionLeaseSyncActionTests.java | 74 - ...ests.java => IndexReaderWrapperTests.java} | 91 +- .../index/shard/IndexShardIT.java | 14 +- .../index/shard/IndexShardTests.java | 152 +- .../index/shard/NewPathForShardTests.java | 2 +- .../shard/PrimaryReplicaSyncerTests.java | 4 +- .../RemoveCorruptedShardDataCommandTests.java | 16 +- .../index/store/CorruptedFileIT.java | 8 +- .../index/store/FsDirectoryFactoryTests.java | 92 +- .../elasticsearch/index/store/StoreTests.java | 44 - .../termvectors/TermVectorsServiceTests.java | 22 +- .../index/translog/TestTranslog.java | 69 +- .../translog/TranslogDeletionPolicyTests.java | 2 +- .../index/translog/TranslogHeaderTests.java | 50 +- .../index/translog/TranslogTests.java | 86 +- .../elasticsearch/indexing/IndexActionIT.java | 4 +- .../IndexingMemoryControllerTests.java | 5 +- .../indices/IndicesOptionsIntegrationIT.java | 79 +- .../indices/IndicesRequestCacheIT.java | 2 - .../indices/IndicesServiceTests.java | 6 +- .../indices/analyze/AnalyzeActionIT.java | 42 +- .../indices/cluster/ClusterStateChanges.java | 2 +- .../FakeThreadPoolMasterServiceTests.java | 4 +- ...ClusterStateServiceRandomUpdatesTests.java | 31 +- .../IndicesClusterStateServiceTests.java | 221 ++ .../exists/indices/IndicesExistsIT.java | 91 - .../indices/exists/types/TypesExistsIT.java | 107 - .../elasticsearch/indices/flush/FlushIT.java | 26 +- .../RandomExceptionCircuitBreakerIT.java | 8 +- .../recovery/IndexPrimaryRelocationIT.java | 4 - .../indices/recovery/IndexRecoveryIT.java | 3 +- .../PeerRecoveryTargetServiceTests.java | 10 +- .../recovery/RecoverySourceHandlerTests.java | 11 +- .../indices/recovery/RecoveryTests.java | 12 +- .../InternalOrPrivateSettingsPlugin.java | 11 +- .../indices/state/CloseIndexIT.java | 5 +- .../state/CloseWhileRelocatingShardsIT.java | 3 - .../indices/state/OpenCloseIndexIT.java | 40 +- .../indices/state/ReopenWhileClosingIT.java | 15 +- .../indices/stats/IndexStatsIT.java | 2 +- .../ingest/IngestServiceTests.java | 70 +- .../org/elasticsearch/node/NodeTests.java | 55 +- .../DestructiveOperationsIT.java | 2 +- .../PersistentTasksClusterServiceTests.java | 3 +- .../PersistentTasksCustomMetaDataTests.java | 37 +- .../PersistentTasksExecutorFullRestartIT.java | 2 - .../persistent/TestPersistentTasksPlugin.java | 9 +- .../plugins/PluginsServiceTests.java | 12 +- .../recovery/RecoveryWhileUnderLoadIT.java | 3 - .../elasticsearch/recovery/RelocationIT.java | 9 - .../recovery/TruncatedRecoveryIT.java | 3 +- .../RepositoriesServiceTests.java | 3 +- .../repositories/fs/FsRepositoryTests.java | 2 +- .../rest/RestControllerTests.java | 87 + .../rest/action/RestActionsTests.java | 2 +- .../admin/indices/RestAnalyzeActionTests.java | 62 +- .../indices/RestValidateQueryActionTests.java | 4 +- .../action/cat/RestIndicesActionTests.java | 266 +- .../action/search/RestCountActionTests.java | 62 - .../routing/SimpleRoutingIT.java | 17 +- .../search/AbstractSearchTestCase.java | 2 +- .../search/DefaultSearchContextTests.java | 1 - .../search/SearchModuleTests.java | 53 +- .../search/SearchServiceTests.java | 19 +- .../AggregatorFactoriesBuilderTests.java | 2 +- .../AggregatorFactoriesTests.java | 2 +- .../BasePipelineAggregationTestCase.java | 2 +- .../InternalAggregationsTests.java | 2 +- .../bucket/AutoDateHistogramTests.java | 1 + .../aggregations/bucket/BooleanTermsIT.java | 6 +- .../aggregations/bucket/DateHistogramIT.java | 1 - .../bucket/DiversifiedSamplerIT.java | 4 +- .../aggregations/bucket/DoubleTermsIT.java | 3 +- .../aggregations/bucket/HistogramIT.java | 4 +- .../aggregations/bucket/LongTermsIT.java | 4 +- .../aggregations/bucket/MinDocCountIT.java | 13 +- .../search/aggregations/bucket/NestedIT.java | 4 +- .../aggregations/bucket/RareTermsTests.java | 101 + .../search/aggregations/bucket/SamplerIT.java | 6 +- .../aggregations/bucket/ShardSizeTermsIT.java | 32 +- .../bucket/ShardSizeTestCase.java | 4 +- .../SignificantTermsSignificanceScoreIT.java | 20 +- .../bucket/TermsDocCountErrorIT.java | 94 +- ...djacencyMatrixAggregationBuilderTests.java | 2 +- ...oDateHistogramAggregationBuilderTests.java | 67 + .../bucket/histogram/ExtendedBoundsTests.java | 2 +- .../histogram/HistogramAggregatorTests.java | 82 + .../InternalAutoDateHistogramTests.java | 16 +- .../bucket/range/IpRangeAggregatorTests.java | 41 + .../SignificanceHeuristicTests.java | 4 +- .../terms/RareTermsAggregatorTests.java | 600 ++++ .../bucket/terms/StringTermsIT.java | 57 +- .../bucket/terms/TermsAggregatorTests.java | 113 +- .../metrics/CardinalityAggregatorTests.java | 61 +- .../aggregations/metrics/CardinalityIT.java | 34 +- .../metrics/MaxAggregatorTests.java | 79 +- .../ScriptedMetricAggregatorTests.java | 7 +- .../metrics/ValueCountAggregatorTests.java | 63 +- .../WeightedAvgAggregationBuilderTests.java | 2 +- .../AbstractBucketMetricsTestCase.java | 1 - .../PipelineAggregationHelperTests.java | 3 +- .../aggregations/pipeline/SerialDiffIT.java | 4 +- .../support/ScriptValuesTests.java | 6 +- .../support/ValuesSourceTypeTests.java | 69 + .../basic/SearchWhileCreatingIndexIT.java | 2 - .../basic/SearchWithRandomExceptionsIT.java | 8 +- .../basic/SearchWithRandomIOExceptionsIT.java | 7 +- .../search/collapse/CollapseBuilderTests.java | 2 +- .../search/fetch/FetchSubPhasePluginIT.java | 2 +- .../subphase/FetchSourceSubPhaseTests.java | 4 +- .../highlight/CustomHighlighterSearchIT.java | 9 +- .../highlight/HighlightBuilderTests.java | 6 +- .../highlight/HighlighterSearchIT.java | 59 +- .../search/fields/SearchFieldsIT.java | 6 +- .../search/geo/GeoDistanceIT.java | 1 - .../search/geo/GeoShapeQueryTests.java | 10 +- .../internal/ContextIndexSearcherTests.java | 425 +++ .../ShardSearchTransportRequestTests.java | 1 - .../search/lookup/LeafDocLookupTests.java | 26 +- .../search/lookup/LeafFieldsLookupTests.java | 4 +- .../search/morelikethis/MoreLikeThisIT.java | 64 +- .../search/nested/SimpleNestedIT.java | 72 +- .../search/profile/query/QueryProfilerIT.java | 1 - .../profile/query/QueryProfilerTests.java | 9 +- .../profile/query/RandomQueryGenerator.java | 33 +- .../search/query/MultiMatchQueryIT.java | 73 - .../search/query/QueryPhaseTests.java | 33 + .../search/query/QuerySearchResultTests.java | 2 +- .../search/query/QueryStringIT.java | 1 + .../search/query/SearchQueryIT.java | 116 +- .../search/query/SimpleQueryStringIT.java | 5 +- .../rescore/QueryRescorerBuilderTests.java | 10 +- .../search/scroll/DuelScrollIT.java | 9 +- .../search/scroll/SearchScrollIT.java | 4 +- .../search/slice/SliceBuilderTests.java | 5 - .../search/slice/TermsSliceQueryTests.java | 19 +- .../search/sort/AbstractSortTestCase.java | 7 +- .../search/sort/FieldSortBuilderTests.java | 73 +- .../search/sort/FieldSortIT.java | 15 +- .../search/sort/GeoDistanceIT.java | 26 +- .../sort/GeoDistanceSortBuilderTests.java | 91 +- .../search/sort/NestedSortBuilderTests.java | 2 +- .../search/sort/ScriptSortBuilderTests.java | 36 +- .../search/sort/SortBuilderTests.java | 12 +- .../AbstractSuggestionBuilderTestCase.java | 6 +- .../suggest/CompletionSuggestSearchIT.java | 23 +- .../search/suggest/SuggestBuilderTests.java | 2 +- .../search/suggest/SuggestSearchIT.java | 66 +- .../search/suggest/SuggestTests.java | 2 +- .../AbstractSnapshotIntegTestCase.java | 30 +- .../DedicatedClusterSnapshotRestoreIT.java | 52 +- ...etadataLoadingDuringSnapshotRestoreIT.java | 6 +- .../MinThreadsSnapshotRestoreIT.java | 6 +- .../snapshots/RepositoriesIT.java | 2 + .../SharedClusterSnapshotRestoreIT.java | 280 +- .../snapshots/SnapshotInfoTests.java | 149 + .../snapshots/SnapshotResiliencyTests.java | 200 +- .../snapshots/SnapshotShardsServiceIT.java | 4 +- ...SnapshotsInProgressSerializationTests.java | 3 +- .../mockstore/BlobContainerWrapper.java | 10 + .../snapshots/mockstore/MockRepository.java | 22 + .../SharedSignificantTermsTestMethods.java | 2 +- .../threadpool/SchedulerTests.java | 12 + .../RemoteClusterConnectionTests.java | 2 - .../transport/RemoteClusterServiceTests.java | 108 +- .../TransportServiceHandshakeTests.java | 39 - .../org/elasticsearch/update/UpdateIT.java | 14 +- .../validate/SimpleValidateQueryIT.java | 24 +- .../ConcurrentSeqNoVersioningIT.java | 24 +- .../action/search/simple-msearch1.json | 2 +- .../action/search/simple-msearch2.json | 2 +- .../action/search/simple-msearch3.json | 4 +- .../action/search/simple-msearch4.json | 4 +- .../action/termvectors/multiRequest1.json | 3 +- .../action/termvectors/multiRequest2.json | 6 +- .../action/termvectors/multiRequest3.json | 1 - .../index/query/commonTerms-query1.json | 11 - .../index/query/commonTerms-query2.json | 11 - .../index/query/commonTerms-query3.json | 9 - .../index/translog/translog-v2.binary | Bin 0 -> 43 bytes settings.gradle | 72 +- .../src/main/java/hdfs/MiniHDFS.java | 25 +- test/fixtures/krb5kdc-fixture/build.gradle | 3 - .../krb5kdc-fixture/docker-compose.yml | 4 + test/framework/build.gradle | 16 +- .../cluster/ESAllocationTestCase.java | 8 +- .../AbstractCoordinatorTestCase.java | 1362 ++++++++ .../CoordinationStateTestCluster.java | 264 ++ .../coordination/LinearizabilityChecker.java | 0 .../common/settings/MockSecureSettings.java | 13 + .../gateway/MockGatewayMetaState.java | 2 - .../index/engine/EngineTestCase.java | 22 +- .../index/engine/TranslogHandler.java | 3 +- .../ESIndexLevelReplicationTestCase.java | 23 +- .../index/shard/IndexShardTestCase.java | 47 +- .../index/shard/RestoreOnlyRepository.java | 2 +- .../cluster/FakeThreadPoolMasterService.java | 0 .../indices/recovery/AsyncRecoveryTarget.java | 11 +- .../AbstractThirdPartyRepositoryTestCase.java | 300 ++ .../blobstore/BlobStoreTestUtil.java | 126 + .../ESBlobStoreRepositoryIntegTestCase.java | 2 +- .../search/RandomSearchRequestGenerator.java | 3 - .../aggregations/AggregatorTestCase.java | 82 +- .../AbstractBroadcastResponseTestCase.java | 3 +- .../test/AbstractBuilderTestCase.java | 8 +- .../test/AbstractQueryTestCase.java | 28 +- .../elasticsearch/test/BackgroundIndexer.java | 65 +- .../test/ClusterServiceUtils.java | 6 +- .../elasticsearch/test/ESIntegTestCase.java | 39 +- .../org/elasticsearch/test/ESTestCase.java | 17 + .../test/InternalAggregationTestCase.java | 2 +- .../test/InternalTestCluster.java | 145 +- .../test/MockIndexEventListener.java | 17 +- .../elasticsearch/test/MockLogAppender.java | 2 +- .../test/NodeConfigurationSource.java | 15 - .../elasticsearch/test/client/NoOpClient.java | 4 +- .../disruption/DisruptableMockTransport.java | 14 +- .../test/engine/MockEngineSupport.java | 19 +- .../hamcrest/ElasticsearchAssertions.java | 17 +- .../test/hamcrest/OptionalMatchers.java | 84 + .../junit/listeners/ReproduceInfoPrinter.java | 10 +- .../test/rest/ESRestTestCase.java | 14 +- .../yaml/restspec/ClientYamlSuiteRestApi.java | 13 + .../ClientYamlSuiteRestApiParser.java | 17 + .../test/transport/MockTransportService.java | 23 +- .../threadpool/TestThreadPool.java | 8 +- .../transport/MockTransportClient.java | 69 - .../transport/nio/MockNioTransport.java | 21 +- .../search/MockSearchServiceTests.java | 4 +- .../elasticsearch/test/VersionUtilsTests.java | 49 +- .../DisruptableMockTransportTests.java | 25 + .../test/disruption/NetworkDisruptionIT.java | 90 +- ...entYamlSuiteRestApiParserFailingTests.java | 4 + .../ClientYamlSuiteRestApiParserTests.java | 4 + .../test/test/InternalTestClusterIT.java | 25 + .../test/test/InternalTestClusterTests.java | 77 +- .../transport/nio/TestEventHandlerTests.java | 5 +- test/logger-usage/build.gradle | 2 +- x-pack/build.gradle | 14 - x-pack/docs/build.gradle | 1 - x-pack/docs/en/rest-api/security.asciidoc | 2 + .../security/create-api-keys.asciidoc | 12 +- .../rest-api/security/get-api-keys.asciidoc | 4 + .../security/get-builtin-privileges.asciidoc | 107 + .../en/rest-api/security/get-tokens.asciidoc | 43 +- .../security/invalidate-api-keys.asciidoc | 11 +- .../authentication/oidc-guide.asciidoc | 52 +- .../cross-cluster.asciidoc | 6 +- .../docs/en/security/configuring-es.asciidoc | 3 - .../security/securing-communications.asciidoc | 5 - x-pack/docs/en/watcher/actions.asciidoc | 43 + .../watcher/condition/array-compare.asciidoc | 2 +- .../en/watcher/how-watcher-works.asciidoc | 6 +- x-pack/license-tools/build.gradle | 6 +- x-pack/plugin/build.gradle | 106 +- x-pack/plugin/ccr/build.gradle | 2 +- x-pack/plugin/ccr/qa/build.gradle | 1 + .../downgrade-to-basic-license/build.gradle | 58 +- .../xpack/ccr/FollowIndexIT.java | 2 +- .../plugin/ccr/qa/multi-cluster/build.gradle | 75 +- .../ccr/qa/non-compliant-license/build.gradle | 42 +- x-pack/plugin/ccr/qa/rest/build.gradle | 20 +- x-pack/plugin/ccr/qa/restart/build.gradle | 69 +- x-pack/plugin/ccr/qa/security/build.gradle | 76 +- .../xpack/ccr/ESCCRRestTestCase.java | 5 + .../java/org/elasticsearch/xpack/ccr/Ccr.java | 27 +- .../xpack/ccr/CcrLicenseChecker.java | 6 +- .../xpack/ccr/action/CcrRequests.java | 59 +- .../xpack/ccr/action/ShardChangesAction.java | 79 +- .../xpack/ccr/action/ShardFollowNodeTask.java | 67 +- .../xpack/ccr/action/ShardFollowTask.java | 2 +- .../ccr/action/ShardFollowTasksExecutor.java | 135 +- .../ccr/action/TransportCcrStatsAction.java | 7 +- ...ransportDeleteAutoFollowPatternAction.java | 12 +- .../ccr/action/TransportFollowInfoAction.java | 3 +- .../TransportGetAutoFollowPatternAction.java | 3 +- .../action/TransportPauseFollowAction.java | 12 +- .../TransportPutAutoFollowPatternAction.java | 12 +- .../ccr/action/TransportPutFollowAction.java | 7 +- .../action/TransportResumeFollowAction.java | 11 +- .../ccr/action/TransportUnfollowAction.java | 16 +- .../bulk/BulkShardOperationsAction.java | 4 +- .../TransportBulkShardOperationsAction.java | 7 +- .../ClearCcrRestoreSessionAction.java | 9 +- .../DeleteInternalCcrRepositoryAction.java | 9 +- .../GetCcrRestoreFileChunkAction.java | 9 +- .../PutCcrRestoreSessionAction.java | 9 +- .../PutCcrRestoreSessionRequest.java | 16 +- .../PutInternalCcrRepositoryAction.java | 9 +- .../ccr/index/engine/FollowingEngine.java | 2 +- .../xpack/ccr/repository/CcrRepository.java | 17 +- .../elasticsearch/xpack/CcrIntegTestCase.java | 10 - .../elasticsearch/xpack/ccr/AutoFollowIT.java | 32 +- .../xpack/ccr/CCRFeatureSetUsageTests.java | 11 +- ....java => CCRInfoTransportActionTests.java} | 39 +- .../elasticsearch/xpack/ccr/CcrAliasesIT.java | 412 +++ .../xpack/ccr/CcrRetentionLeaseIT.java | 224 +- .../xpack/ccr/FollowerFailOverIT.java | 13 +- .../xpack/ccr/IndexFollowingIT.java | 112 +- .../xpack/ccr/LocalIndexFollowingIT.java | 3 +- .../action/AutoFollowCoordinatorTests.java | 11 +- .../ccr/action/ShardChangesActionTests.java | 20 +- .../ccr/action/ShardChangesRequestTests.java | 9 +- .../ccr/action/ShardChangesResponseTests.java | 2 + .../ShardFollowNodeTaskRandomTests.java | 48 +- .../ShardFollowNodeTaskStatusTests.java | 2 + .../ccr/action/ShardFollowNodeTaskTests.java | 154 +- .../ShardFollowTaskReplicationTests.java | 53 +- .../xpack/ccr/action/StatsResponsesTests.java | 1 + .../TransportResumeFollowActionTests.java | 11 - .../action/bulk/BulkShardOperationsTests.java | 2 +- .../index/engine/FollowingEngineTests.java | 36 +- .../ccr/FollowStatsMonitoringDocTests.java | 4 + .../collector/ccr/StatsCollectorTests.java | 8 +- x-pack/plugin/core/build.gradle | 9 +- .../licenses/log4j-1.2-api-2.11.1.jar.sha1 | 1 + x-pack/plugin/core/licenses/log4j-LICENSE.txt | 202 ++ x-pack/plugin/core/licenses/log4j-NOTICE.txt | 5 + .../core/src/main/config/log4j2.properties | 3 +- .../index/engine/FrozenEngine.java | 12 +- .../license/DeleteLicenseAction.java | 9 +- .../license/GetBasicStatusAction.java | 4 +- .../license/GetBasicStatusRequestBuilder.java | 4 +- .../license/GetLicenseAction.java | 4 +- .../license/GetTrialStatusAction.java | 4 +- .../license/GetTrialStatusRequestBuilder.java | 4 +- .../elasticsearch/license/LicenseUtils.java | 12 +- .../license/LicensingClient.java | 8 +- .../license/PostStartBasicAction.java | 9 +- .../license/PostStartBasicRequestBuilder.java | 4 +- .../license/PostStartBasicResponse.java | 42 +- .../license/PostStartTrialAction.java | 4 +- .../license/PostStartTrialRequestBuilder.java | 4 +- .../license/PutLicenseAction.java | 9 +- .../license/RestDeleteLicenseAction.java | 12 +- .../license/RestGetBasicStatus.java | 12 +- .../license/RestGetLicenseAction.java | 14 +- .../license/RestGetTrialStatus.java | 12 +- .../license/RestPostStartBasicLicense.java | 12 +- .../license/RestPostStartTrialLicense.java | 14 +- .../license/RestPutLicenseAction.java | 15 +- .../license/TransportDeleteLicenseAction.java | 15 +- .../TransportGetBasicStatusAction.java | 3 +- .../license/TransportGetLicenseAction.java | 3 +- .../TransportGetTrialStatusAction.java | 3 +- .../TransportPostStartBasicAction.java | 13 +- .../TransportPostStartTrialAction.java | 3 +- .../license/TransportPutLicenseAction.java | 15 +- .../license/XPackLicenseState.java | 31 +- .../protocol/xpack/XPackInfoResponse.java | 52 +- .../xpack/license/PutLicenseResponse.java | 42 +- .../snapshots/SourceOnlySnapshot.java | 4 +- .../SourceOnlySnapshotRepository.java | 5 +- .../CCRInfoTransportAction.java} | 63 +- .../xpack/ccr/CCRUsageTransportAction.java | 75 + .../xpack/core/ClientHelper.java | 6 +- .../xpack/core/EmptyXPackFeatureSet.java | 43 - .../elasticsearch/xpack/core/XPackClient.java | 128 - .../xpack/core/XPackClientPlugin.java | 68 +- .../xpack/core/XPackFeatureSet.java | 8 - .../elasticsearch/xpack/core/XPackField.java | 6 + .../elasticsearch/xpack/core/XPackPlugin.java | 45 +- .../xpack/core/XPackSettings.java | 9 + .../AbstractTransportGetResourcesAction.java | 11 +- .../core/action/ReloadAnalyzerAction.java | 23 + .../core/action/ReloadAnalyzersRequest.java | 43 + .../core/action/ReloadAnalyzersResponse.java | 132 + .../action/TransportFreezeIndexAction.java | 31 +- .../TransportReloadAnalyzersAction.java | 160 + .../core/action/TransportXPackInfoAction.java | 34 +- .../action/TransportXPackUsageAction.java | 34 +- .../xpack/core/action/XPackInfoAction.java | 4 +- .../core/action/XPackInfoFeatureAction.java | 58 + .../core/action/XPackInfoFeatureResponse.java | 42 + .../XPackInfoFeatureTransportAction.java | 38 + .../xpack/core/action/XPackUsageAction.java | 4 +- .../core/action/XPackUsageFeatureAction.java | 58 + .../action/XPackUsageFeatureResponse.java | 42 + .../XPackUsageFeatureTransportAction.java | 41 + .../xpack/core/ccr/AutoFollowMetadata.java | 2 +- .../xpack/core/ccr/AutoFollowStats.java | 15 +- .../core/ccr/ShardFollowNodeTaskStatus.java | 31 +- .../xpack/core/ccr/action/CcrStatsAction.java | 9 +- .../action/DeleteAutoFollowPatternAction.java | 9 +- .../core/ccr/action/FollowInfoAction.java | 9 +- .../core/ccr/action/FollowStatsAction.java | 9 +- .../core/ccr/action/ForgetFollowerAction.java | 4 +- .../action/GetAutoFollowPatternAction.java | 9 +- .../core/ccr/action/PauseFollowAction.java | 9 +- .../action/PutAutoFollowPatternAction.java | 9 +- .../core/ccr/action/PutFollowAction.java | 9 +- .../core/ccr/action/ResumeFollowAction.java | 9 +- .../xpack/core/ccr/action/UnfollowAction.java | 9 +- .../xpack/core/ccr/client/CcrClient.java | 147 - .../{ml/utils => common}/time/TimeUtils.java | 17 +- .../xpack/core/dataframe/DataFrameField.java | 5 + .../core/dataframe/DataFrameMessages.java | 12 +- .../DataFrameNamedXContentProvider.java | 26 + .../DeleteDataFrameTransformAction.java | 15 +- .../action/GetDataFrameTransformsAction.java | 13 +- .../GetDataFrameTransformsStatsAction.java | 76 +- .../PreviewDataFrameTransformAction.java | 89 +- .../action/PutDataFrameTransformAction.java | 34 +- .../action/StartDataFrameTransformAction.java | 29 +- .../StartDataFrameTransformTaskAction.java | 9 +- .../action/StopDataFrameTransformAction.java | 55 +- .../notifications/DataFrameAuditMessage.java | 2 +- .../transforms/DataFrameTransform.java | 30 +- .../transforms/DataFrameTransformConfig.java | 155 +- .../transforms/DataFrameTransformState.java | 46 +- .../core/dataframe/transforms/DestConfig.java | 30 +- .../dataframe/transforms/NodeAttributes.java | 171 + .../core/dataframe/transforms/SyncConfig.java | 25 + .../dataframe/transforms/TimeSyncConfig.java | 148 + .../pivot/DateHistogramGroupSource.java | 43 +- .../pivot/HistogramGroupSource.java | 13 + .../transforms/pivot/PivotConfig.java | 6 +- .../transforms/pivot/SingleGroupSource.java | 6 + .../transforms/pivot/TermsGroupSource.java | 13 + .../dataframe/utils/DataFrameStrings.java | 46 + .../xpack/core/dataframe/utils/TimeUtils.java | 129 - .../deprecation/DeprecationInfoAction.java | 4 +- .../NodesDeprecationCheckAction.java | 10 +- .../flattened/FlattenedFeatureSetUsage.java | 24 + .../core/graph/action/GraphExploreAction.java | 4 +- .../core/indexing/AsyncTwoPhaseIndexer.java | 36 +- .../xpack/core/indexing/IndexerJobStats.java | 29 +- .../IndexLifecycleMetadata.java | 2 +- .../core/indexlifecycle/SegmentCountStep.java | 36 +- .../SetSingleNodeAllocateStep.java | 47 +- .../action/DeleteLifecycleAction.java | 12 +- .../action/ExplainLifecycleAction.java | 4 +- .../action/GetLifecycleAction.java | 4 +- .../action/GetStatusAction.java | 4 +- .../action/MoveToStepAction.java | 12 +- .../action/PutLifecycleAction.java | 12 +- .../RemoveIndexLifecyclePolicyAction.java | 4 +- .../indexlifecycle/action/RetryAction.java | 12 +- .../indexlifecycle/action/StartILMAction.java | 9 +- .../indexlifecycle/action/StopILMAction.java | 9 +- .../core/indexlifecycle/client/ILMClient.java | 165 - .../xpack/core/ml/MachineLearningField.java | 2 +- .../elasticsearch/xpack/core/ml/MlTasks.java | 31 + .../xpack/core/ml/action/CloseJobAction.java | 9 +- .../core/ml/action/DeleteCalendarAction.java | 9 +- .../ml/action/DeleteCalendarEventAction.java | 9 +- .../DeleteDataFrameAnalyticsAction.java | 95 + .../core/ml/action/DeleteDatafeedAction.java | 9 +- .../ml/action/DeleteExpiredDataAction.java | 4 +- .../core/ml/action/DeleteFilterAction.java | 9 +- .../core/ml/action/DeleteForecastAction.java | 9 +- .../xpack/core/ml/action/DeleteJobAction.java | 9 +- .../ml/action/DeleteModelSnapshotAction.java | 9 +- .../ml/action/EvaluateDataFrameAction.java | 215 ++ .../ml/action/FinalizeJobExecutionAction.java | 9 +- .../ml/action/FindFileStructureAction.java | 30 +- .../xpack/core/ml/action/FlushJobAction.java | 9 +- .../core/ml/action/ForecastJobAction.java | 9 +- .../core/ml/action/GetBucketsAction.java | 4 +- .../ml/action/GetCalendarEventsAction.java | 4 +- .../core/ml/action/GetCalendarsAction.java | 4 +- .../core/ml/action/GetCategoriesAction.java | 4 +- .../action/GetDataFrameAnalyticsAction.java | 80 + .../GetDataFrameAnalyticsStatsAction.java | 325 ++ .../core/ml/action/GetDatafeedsAction.java | 4 +- .../ml/action/GetDatafeedsStatsAction.java | 4 +- .../core/ml/action/GetFiltersAction.java | 5 +- .../core/ml/action/GetInfluencersAction.java | 5 +- .../xpack/core/ml/action/GetJobsAction.java | 4 +- .../core/ml/action/GetJobsStatsAction.java | 76 +- .../ml/action/GetModelSnapshotsAction.java | 4 +- .../ml/action/GetOverallBucketsAction.java | 4 +- .../core/ml/action/GetRecordsAction.java | 4 +- .../core/ml/action/IsolateDatafeedAction.java | 9 +- .../core/ml/action/KillProcessAction.java | 9 +- .../xpack/core/ml/action/MlInfoAction.java | 4 +- .../xpack/core/ml/action/OpenJobAction.java | 17 +- .../core/ml/action/PersistJobAction.java | 9 +- .../ml/action/PostCalendarEventsAction.java | 4 +- .../xpack/core/ml/action/PostDataAction.java | 9 +- .../core/ml/action/PreviewDatafeedAction.java | 4 +- .../core/ml/action/PutCalendarAction.java | 4 +- .../action/PutDataFrameAnalyticsAction.java | 153 + .../core/ml/action/PutDatafeedAction.java | 4 +- .../xpack/core/ml/action/PutFilterAction.java | 4 +- .../xpack/core/ml/action/PutJobAction.java | 4 +- .../ml/action/RevertModelSnapshotAction.java | 4 +- .../core/ml/action/SetUpgradeModeAction.java | 9 +- .../action/StartDataFrameAnalyticsAction.java | 245 ++ .../core/ml/action/StartDatafeedAction.java | 21 +- .../action/StopDataFrameAnalyticsAction.java | 234 ++ .../core/ml/action/StopDatafeedAction.java | 9 +- .../ml/action/UpdateCalendarJobAction.java | 4 +- .../core/ml/action/UpdateDatafeedAction.java | 4 +- .../core/ml/action/UpdateFilterAction.java | 4 +- .../xpack/core/ml/action/UpdateJobAction.java | 4 +- .../ml/action/UpdateModelSnapshotAction.java | 4 +- .../core/ml/action/UpdateProcessAction.java | 9 +- .../ml/action/ValidateDetectorAction.java | 9 +- .../ml/action/ValidateJobConfigAction.java | 9 +- .../xpack/core/ml/annotations/Annotation.java | 2 +- .../core/ml/calendars/ScheduledEvent.java | 2 +- .../core/ml/client/MachineLearningClient.java | 385 --- .../core/ml/datafeed/DatafeedConfig.java | 28 +- .../core/ml/datafeed/DatafeedUpdate.java | 15 +- .../ml/datafeed/DelayedDataCheckConfig.java | 2 +- .../dataframe/DataFrameAnalyticsConfig.java | 385 +++ .../ml/dataframe/DataFrameAnalyticsDest.java | 106 + .../dataframe/DataFrameAnalyticsSource.java | 149 + .../ml/dataframe/DataFrameAnalyticsState.java | 44 + .../DataFrameAnalyticsTaskState.java | 121 + .../dataframe/analyses/DataFrameAnalysis.java | 16 + ...ataFrameAnalysisNamedXContentProvider.java | 37 + .../dataframe/analyses/OutlierDetection.java | 167 + .../ml/dataframe/evaluation/Evaluation.java | 37 + .../evaluation/EvaluationMetricResult.java | 20 + .../MlEvaluationNamedXContentProvider.java | 69 + .../AbstractConfusionMatrixMetric.java | 102 + .../evaluation/softclassification/AucRoc.java | 350 ++ .../BinarySoftClassification.java | 212 ++ .../softclassification/ConfusionMatrix.java | 163 + .../softclassification/Precision.java | 91 + .../evaluation/softclassification/Recall.java | 91 + .../ScoreByThresholdResult.java | 63 + .../SoftClassificationMetric.java | 60 + .../ml/filestructurefinder/FieldStats.java | 57 +- .../core/ml/job/config/AnalysisConfig.java | 2 +- .../config/CategorizationAnalyzerConfig.java | 92 +- .../core/ml/job/config/DetectionRule.java | 3 - .../xpack/core/ml/job/config/Detector.java | 6 +- .../xpack/core/ml/job/config/Job.java | 2 +- .../xpack/core/ml/job/config/JobUpdate.java | 10 +- .../xpack/core/ml/job/config/MlFilter.java | 13 +- .../xpack/core/ml/job/messages/Messages.java | 3 + .../persistence/ElasticsearchMappings.java | 86 +- .../output/FlushAcknowledgement.java | 2 +- .../process/autodetect/state/DataCounts.java | 2 +- .../autodetect/state/ModelSizeStats.java | 2 +- .../autodetect/state/ModelSnapshot.java | 2 +- .../process/autodetect/state/TimingStats.java | 222 ++ .../core/ml/job/results/AnomalyRecord.java | 11 +- .../xpack/core/ml/job/results/Bucket.java | 11 +- .../core/ml/job/results/BucketInfluencer.java | 2 +- .../ml/job/results/CategoryDefinition.java | 9 +- .../xpack/core/ml/job/results/Forecast.java | 2 +- .../xpack/core/ml/job/results/Influencer.java | 2 +- .../xpack/core/ml/job/results/ModelPlot.java | 2 +- .../ml/job/results/ReservedFieldNames.java | 27 + .../core/ml/notifications/AuditMessage.java | 2 +- .../core/ml/process/writer/RecordWriter.java | 2 +- .../xpack/core/ml/utils/ExceptionsHelper.java | 13 + .../ml/{datafeed => utils}/QueryProvider.java | 23 +- .../action/MonitoringBulkAction.java | 4 +- .../monitoring/client/MonitoringClient.java | 60 - .../xpack/core/rest/XPackRestHandler.java | 30 - .../rest/action/RestFreezeIndexAction.java | 11 +- .../action/RestReloadAnalyzersAction.java | 40 + .../core/rest/action/RestXPackInfoAction.java | 15 +- .../rest/action/RestXPackUsageAction.java | 14 +- .../rollup/action/DeleteRollupJobAction.java | 9 +- .../rollup/action/GetRollupCapsAction.java | 5 +- .../action/GetRollupIndexCapsAction.java | 5 +- .../rollup/action/GetRollupJobsAction.java | 9 +- .../rollup/action/PutRollupJobAction.java | 9 +- .../rollup/action/RollupSearchAction.java | 9 +- .../rollup/action/StartRollupJobAction.java | 9 +- .../rollup/action/StopRollupJobAction.java | 22 +- .../security/action/CreateApiKeyAction.java | 11 +- .../security/action/CreateApiKeyRequest.java | 12 +- .../action/CreateApiKeyRequestBuilder.java | 2 +- .../core/security/action/GetApiKeyAction.java | 11 +- .../action/InvalidateApiKeyAction.java | 11 +- .../oidc/OpenIdConnectAuthenticateAction.java | 11 +- .../oidc/OpenIdConnectLogoutAction.java | 9 +- ...nIdConnectPrepareAuthenticationAction.java | 9 +- .../privilege/DeletePrivilegesAction.java | 6 +- .../DeletePrivilegesRequestBuilder.java | 4 +- .../privilege/GetBuiltinPrivilegesAction.java | 26 + .../GetBuiltinPrivilegesRequest.java | 23 + .../GetBuiltinPrivilegesResponse.java | 61 + .../action/privilege/GetPrivilegesAction.java | 6 +- .../privilege/GetPrivilegesRequest.java | 1 - .../GetPrivilegesRequestBuilder.java | 4 +- .../privilege/GetPrivilegesResponse.java | 12 +- .../action/privilege/PutPrivilegesAction.java | 6 +- .../PutPrivilegesRequestBuilder.java | 4 +- .../action/realm/ClearRealmCacheAction.java | 4 +- .../action/realm/ClearRealmCacheRequest.java | 3 +- .../action/role/ClearRolesCacheAction.java | 4 +- .../action/role/ClearRolesCacheRequest.java | 3 +- .../action/role/DeleteRoleAction.java | 6 +- .../security/action/role/GetRolesAction.java | 6 +- .../security/action/role/PutRoleAction.java | 6 +- .../rolemapping/DeleteRoleMappingAction.java | 6 +- .../DeleteRoleMappingRequestBuilder.java | 5 +- .../rolemapping/GetRoleMappingsAction.java | 6 +- .../GetRoleMappingsRequestBuilder.java | 4 +- .../rolemapping/PutRoleMappingAction.java | 6 +- .../PutRoleMappingRequestBuilder.java | 4 +- .../action/saml/SamlAuthenticateAction.java | 6 +- .../saml/SamlInvalidateSessionAction.java | 6 +- .../action/saml/SamlLogoutAction.java | 6 +- .../saml/SamlPrepareAuthenticationAction.java | 6 +- .../action/token/CreateTokenAction.java | 6 +- .../action/token/CreateTokenRequest.java | 132 +- .../token/CreateTokenRequestBuilder.java | 4 +- .../action/token/CreateTokenResponse.java | 28 +- .../action/token/InvalidateTokenAction.java | 6 +- .../action/token/RefreshTokenAction.java | 6 +- .../action/user/AuthenticateAction.java | 4 +- .../action/user/ChangePasswordAction.java | 4 +- .../user/ChangePasswordRequestBuilder.java | 2 +- .../action/user/DeleteUserAction.java | 6 +- .../action/user/GetUserPrivilegesAction.java | 6 +- .../security/action/user/GetUsersAction.java | 6 +- .../action/user/HasPrivilegesAction.java | 4 +- .../security/action/user/PutUserAction.java | 6 +- .../action/user/PutUserRequestBuilder.java | 17 +- .../action/user/SetEnabledAction.java | 4 +- .../core/security/authc/support/Hasher.java | 4 +- .../DocumentSubsetBitsetCache.java | 206 ++ .../accesscontrol/DocumentSubsetReader.java | 44 +- .../SecurityIndexReaderWrapper.java | 110 + .../SecurityIndexSearcherWrapper.java | 217 -- .../authz/privilege/ApplicationPrivilege.java | 2 +- .../authz/privilege/ClusterPrivilege.java | 7 + .../authz/privilege/IndexPrivilege.java | 15 +- .../authz/store/ReservedRolesStore.java | 16 +- .../core/security/client/SecurityClient.java | 392 --- .../security/support/CacheIteratorHelper.java | 59 + .../core/security/support/MetadataUtils.java | 3 +- .../core/security/support/Validation.java | 9 +- .../core/security/user/APMSystemUser.java | 2 - .../security/user/RemoteMonitoringUser.java | 3 - .../xpack/core/security/user/User.java | 7 +- .../xpack/core/ssl/CertParsingUtils.java | 22 +- .../xpack/core/ssl/TrustConfig.java | 14 +- .../ssl/action/GetCertificateInfoAction.java | 4 +- .../core/template/IndexTemplateRegistry.java | 3 +- .../xpack/core/template/TemplateUtils.java | 11 +- .../upgrade/actions/IndexUpgradeAction.java | 4 +- .../actions/IndexUpgradeInfoAction.java | 4 +- .../core/vectors/VectorsFeatureSetUsage.java | 24 + .../VotingOnlyNodeFeatureSetUsage.java | 22 + .../core/watcher/actions/ActionWrapper.java | 113 +- .../watcher/client/WatchSourceBuilder.java | 17 +- .../core/watcher/client/WatcherClient.java | 326 -- .../WatcherIndexTemplateRegistryField.java | 3 +- .../xcontent/WatcherXContentParser.java | 5 - .../transport/actions/ack/AckWatchAction.java | 4 +- .../actions/activate/ActivateWatchAction.java | 4 +- .../actions/delete/DeleteWatchAction.java | 4 +- .../actions/execute/ExecuteWatchAction.java | 4 +- .../transport/actions/get/GetWatchAction.java | 4 +- .../transport/actions/put/PutWatchAction.java | 4 +- .../actions/service/WatcherServiceAction.java | 9 +- .../actions/stats/WatcherStatsAction.java | 6 +- .../actions/stats/WatcherStatsRequest.java | 3 +- .../xpack/core/watcher/watch/WatchField.java | 1 + .../src/main/resources/monitoring-es.json | 3 + .../src/main/resources/watch-history.json | 7 + .../index/engine/FrozenEngineTests.java | 12 +- .../index/engine/FrozenIndexTests.java | 133 +- .../license/XPackLicenseStateTests.java | 16 + .../snapshots/SourceOnlySnapshotIT.java | 7 +- .../SourceOnlySnapshotShardTests.java | 4 +- .../core/LocalStateCompositeXPackPlugin.java | 51 +- .../action/ReloadAnalyzersResponseTests.java | 56 + .../action/ReloadSynonymAnalyzerTests.java | 107 + .../action/TransportXPackInfoActionTests.java | 52 +- .../AbstractAuditMessageTests.java | 2 +- .../core/common}/time/TimeUtilsTests.java | 49 +- .../AbstractSerializingDataFrameTestCase.java | 10 +- ...tractWireSerializingDataFrameTestCase.java | 10 +- ...ataFrameTransformsActionResponseTests.java | 6 +- ...ameTransformsStatsActionResponseTests.java | 2 +- ...wDataFrameTransformActionRequestTests.java | 12 +- ...ataFrameTransformsActionResponseTests.java | 27 +- ...pDataFrameTransformActionRequestTests.java | 11 +- .../AbstractSerializingDataFrameTestCase.java | 6 +- .../DataFrameTransformConfigTests.java | 86 +- .../DataFrameTransformStateTests.java | 4 +- .../transforms/DataFrameTransformTests.java | 52 + .../dataframe/transforms/DestConfigTests.java | 3 +- .../MockDeprecatedAggregationBuilder.java | 14 +- .../transforms/NodeAttributeTests.java | 52 + .../transforms/TimeSyncConfigTests.java | 38 + .../pivot/DateHistogramGroupSourceTests.java | 19 +- .../utils/DataFrameStringsTests.java | 30 + .../indexing/AsyncTwoPhaseIndexerTests.java | 24 +- .../indexlifecycle/SegmentCountStepTests.java | 6 +- .../SetSingleNodeAllocateStepTests.java | 319 +- .../WaitForFollowShardTasksStepTests.java | 1 + .../action/DeleteLifecycleResponseTests.java | 9 +- .../action/MoveToStepResponseTests.java | 9 +- .../action/PutLifecycleResponseTests.java | 9 +- .../action/RetryResponseTests.java | 9 +- .../xpack/core/ml/MlTasksTests.java | 7 + .../EvaluateDataFrameActionRequestTests.java | 58 + .../FindFileStructureActionRequestTests.java | 16 + ...DataFrameAnalyticsActionResponseTests.java | 55 + .../GetDataFrameAnalyticsRequestTests.java | 27 + ...rameAnalyticsStatsActionResponseTests.java | 38 + ...etDataFrameAnalyticsStatsRequestTests.java | 26 + .../GetDatafeedStatsActionResponseTests.java | 6 +- .../GetDatafeedsActionResponseTests.java | 2 +- .../GetJobStatsActionResponseTests.java | 40 +- ...tDataFrameAnalyticsActionRequestTests.java | 67 + ...DataFrameAnalyticsActionResponseTests.java | 48 + .../action/PutDatafeedActionRequestTests.java | 4 +- .../PutDatafeedActionResponseTests.java | 2 +- ...taFrameAnalyticsActionTaskParamsTests.java | 37 + .../StartDataFrameAnalyticsRequestTests.java | 28 + ...DataFrameAnalyticsActionResponseTests.java | 23 + .../StopDataFrameAnalyticsRequestTests.java | 43 + .../UpdateDatafeedActionRequestTests.java | 4 +- .../core/ml/datafeed/AggProviderTests.java | 6 +- .../core/ml/datafeed/DatafeedConfigTests.java | 11 +- .../core/ml/datafeed/DatafeedUpdateTests.java | 9 +- .../DataFrameAnalyticsConfigTests.java | 312 ++ .../DataFrameAnalyticsDestTests.java | 55 + .../DataFrameAnalyticsSourceTests.java | 64 + .../DataFrameAnalyticsStateTests.java | 41 + .../DataFrameAnalyticsTaskStateTests.java | 35 + .../analyses/OutlierDetectionTests.java | 59 + .../softclassification/AucRocTests.java | 127 + .../BinarySoftClassificationTests.java | 79 + .../ConfusionMatrixTests.java | 79 + .../softclassification/PrecisionTests.java | 93 + .../softclassification/RecallTests.java | 93 + .../filestructurefinder/FieldStatsTests.java | 10 +- .../integration/MlRestTestStateCleaner.java | 17 + .../ElasticsearchMappingsTests.java | 2 + .../autodetect/state/TimingStatsTests.java | 149 + .../QueryProviderTests.java | 12 +- .../utils/XContentObjectTransformerTests.java | 2 +- .../rest/action/ReloadSynonymAnalyzerIT.java | 125 + .../CreateApiKeyRequestBuilderTests.java | 18 + .../action/CreateApiKeyRequestTests.java | 24 +- .../GetBuiltinPrivilegesResponseTests.java | 32 + .../privilege/GetPrivilegesRequestTests.java | 3 +- .../privilege/GetPrivilegesResponseTests.java | 22 +- .../action/token/CreateTokenRequestTests.java | 66 +- .../token/CreateTokenResponseTests.java | 4 +- .../DocumentSubsetBitsetCacheTests.java | 247 ++ .../DocumentSubsetReaderTests.java | 50 +- ...tyIndexReaderWrapperIntegrationTests.java} | 58 +- .../SecurityIndexReaderWrapperUnitTests.java | 225 ++ ...SecurityIndexSearcherWrapperUnitTests.java | 539 --- .../authz/store/ReservedRolesStoreTests.java | 27 + .../SecurityQueryTemplateEvaluatorTests.java | 2 +- .../security/support/ValidationTests.java | 7 +- .../xpack/core/security/user/UserTests.java | 4 +- .../xpack/core/ssl/CertParsingUtilsTests.java | 6 +- .../ssl/SSLConfigurationReloaderTests.java | 19 +- .../xpack/core/ssl/SSLConfigurationTests.java | 19 +- x-pack/plugin/data-frame/build.gradle | 2 +- .../qa/multi-node-tests/build.gradle | 26 +- .../integration/DataFrameIntegTestCase.java | 73 +- .../integration/DataFrameTransformIT.java | 114 +- .../qa/single-node-tests/build.gradle | 21 +- .../integration/DataFrameAuditorIT.java | 8 +- .../DataFrameConfigurationIndexIT.java | 2 - .../DataFrameGetAndGetStatsIT.java | 123 +- .../integration/DataFrameMetaDataIT.java | 2 - .../integration/DataFramePivotRestIT.java | 353 +- .../integration/DataFrameRestTestCase.java | 130 +- .../DataFrameTaskFailedStateIT.java | 2 - .../DataFrameTransformProgressIT.java | 32 +- .../integration/DataFrameUsageIT.java | 82 +- .../xpack/dataframe/DataFrame.java | 29 +- ...java => DataFrameInfoTransportAction.java} | 111 +- .../DataFrameUsageTransportAction.java | 130 + ...ansportDeleteDataFrameTransformAction.java | 25 +- ...TransportGetDataFrameTransformsAction.java | 2 +- ...portGetDataFrameTransformsStatsAction.java | 91 +- ...nsportPreviewDataFrameTransformAction.java | 96 +- .../TransportPutDataFrameTransformAction.java | 67 +- ...ransportStartDataFrameTransformAction.java | 33 +- ...TransportStopDataFrameTransformAction.java | 47 +- .../DataFrameTransformsCheckpointService.java | 27 +- .../DataFrameTransformsConfigManager.java | 16 +- .../dataframe/persistence/DataframeIndex.java | 84 +- .../RestGetDataFrameTransformsAction.java | 3 + ...RestGetDataFrameTransformsStatsAction.java | 3 + .../RestStopDataFrameTransformAction.java | 8 +- .../transforms/DataFrameIndexer.java | 150 +- ...FrameTransformPersistentTasksExecutor.java | 71 +- .../transforms/DataFrameTransformTask.java | 291 +- .../transforms/TransformProgressGatherer.java | 26 +- .../pivot/AggregationResultUtils.java | 97 +- .../transforms/pivot/Aggregations.java | 3 +- .../dataframe/transforms/pivot/Pivot.java | 87 +- .../transforms/pivot/SchemaUtil.java | 3 +- ...=> DataFrameInfoTransportActionTests.java} | 45 +- .../dataframe/action/DataFrameNodesTests.java | 4 +- ...meTransformCheckpointServiceNodeTests.java | 6 +- ...DataFrameTransformsConfigManagerTests.java | 38 +- .../persistence/DataframeIndexTests.java | 75 + .../transforms/DataFrameIndexerTests.java | 14 +- ...TransformPersistentTasksExecutorTests.java | 83 + .../pivot/AggregationResultUtilsTests.java | 38 +- .../transforms/pivot/AggregationsTests.java | 8 +- .../transforms/pivot/PivotTests.java | 18 +- x-pack/plugin/deprecation/build.gradle | 2 +- .../TransportDeprecationInfoAction.java | 3 +- .../TransportNodeDeprecationCheckAction.java | 7 +- .../deprecation/MlDeprecationChecksTests.java | 2 +- x-pack/plugin/graph/build.gradle | 1 - x-pack/plugin/graph/qa/build.gradle | 28 +- .../graph/qa/with-security/build.gradle | 33 +- .../org/elasticsearch/xpack/graph/Graph.java | 24 +- ...Set.java => GraphInfoTransportAction.java} | 33 +- .../graph/GraphUsageTransportAction.java | 48 + .../action/TransportGraphExploreAction.java | 4 +- .../graph/rest/action/RestGraphAction.java | 18 +- ...ava => GraphInfoTransportActionTests.java} | 29 +- x-pack/plugin/ilm/build.gradle | 1 - .../plugin/ilm/qa/multi-cluster/build.gradle | 55 +- .../indexlifecycle/CCRIndexLifecycleIT.java | 88 +- x-pack/plugin/ilm/qa/multi-node/build.gradle | 16 +- .../ChangePolicyforIndexIT.java | 15 +- .../TimeSeriesLifecycleActionsIT.java | 30 +- x-pack/plugin/ilm/qa/rest/build.gradle | 32 +- .../plugin/ilm/qa/with-security/build.gradle | 35 +- .../xpack/security/PermissionsIT.java | 3 +- .../xpack/indexlifecycle/IndexLifecycle.java | 23 +- .../IndexLifecycleFeatureSet.java | 97 - .../IndexLifecycleInfoTransportAction.java | 46 + .../indexlifecycle/IndexLifecycleService.java | 28 +- .../IndexLifecycleUsageTransportAction.java | 81 + .../LifecyclePolicySecurityClient.java | 6 +- .../TransportDeleteLifecycleAction.java | 12 +- .../action/TransportGetLifecycleAction.java | 3 +- .../action/TransportGetStatusAction.java | 3 +- .../action/TransportMoveToStepAction.java | 13 +- .../action/TransportPutLifecycleAction.java | 12 +- ...sportRemoveIndexLifecyclePolicyAction.java | 3 +- .../action/TransportRetryAction.java | 13 +- .../action/TransportStartILMAction.java | 13 +- .../action/TransportStopILMAction.java | 13 +- ...dexLifecycleInfoTransportActionTests.java} | 38 +- .../IndexLifecycleInitialisationTests.java | 6 +- .../IndexLifecycleMetadataTests.java | 4 +- .../IndexLifecycleServiceTests.java | 67 +- x-pack/plugin/logstash/build.gradle | 1 - .../xpack/logstash/Logstash.java | 26 +- ....java => LogstashInfoTransportAction.java} | 34 +- .../LogstashUsageTransportAction.java | 49 + ... => LogstashInfoTransportActionTests.java} | 37 +- x-pack/plugin/mapper-flattened/build.gradle | 27 + .../FlattenedInfoTransportAction.java | 47 + .../flattened/FlattenedMapperPlugin.java | 50 + .../FlattenedUsageTransportAction.java | 49 + .../mapper/FlatObjectFieldMapper.java | 616 ++++ .../mapper/FlatObjectFieldParser.java | 167 + .../KeyedFlatObjectAtomicFieldData.java | 251 ++ .../mapper/FlatObjectFieldLookupTests.java | 188 ++ .../mapper/FlatObjectFieldMapperTests.java | 457 +++ .../mapper/FlatObjectFieldParserTests.java | 318 ++ .../mapper/FlatObjectIndexFieldDataTests.java | 106 + .../mapper/FlatObjectSearchTests.java | 512 +++ .../KeyedFlatObjectAtomicFieldDataTests.java | 204 ++ .../mapper/KeyedFlatObjectFieldTypeTests.java | 155 + .../mapper/RootFlatObjectFieldTypeTests.java | 116 + x-pack/plugin/ml/build.gradle | 12 +- .../ml/qa/basic-multi-node/build.gradle | 8 +- x-pack/plugin/ml/qa/disabled/build.gradle | 7 +- .../ml/qa/ml-with-security/build.gradle | 90 +- .../plugin/ml/qa/ml-with-security/roles.yml | 4 +- .../smoketest/MlWithSecurityUserRoleIT.java | 33 +- .../qa/native-multi-node-tests/build.gradle | 46 +- .../integration/AutodetectMemoryLimitIT.java | 1 + .../ml/integration/DatafeedJobsRestIT.java | 65 + .../ml/integration/DeleteExpiredDataIT.java | 1 - ...NativeDataFrameAnalyticsIntegTestCase.java | 118 + .../ml/integration/MlNativeIntegTestCase.java | 15 +- .../xpack/ml/integration/PersistJobIT.java | 12 +- .../integration/RunDataFrameAnalyticsIT.java | 386 +++ .../ml/qa/no-bootstrap-tests/build.gradle | 4 +- .../ml/qa/single-node-tests/build.gradle | 6 +- .../xpack/ml/MachineLearning.java | 140 +- .../xpack/ml/MachineLearningFeatureSet.java | 303 -- .../MachineLearningInfoTransportAction.java | 46 + .../MachineLearningUsageTransportAction.java | 209 ++ .../ml/MlConfigMigrationEligibilityCheck.java | 10 - .../xpack/ml/MlLifeCycleService.java | 48 +- .../action/TransportDeleteCalendarAction.java | 3 +- ...ansportDeleteDataFrameAnalyticsAction.java | 114 + .../action/TransportDeleteDatafeedAction.java | 13 +- .../action/TransportDeleteForecastAction.java | 5 +- .../ml/action/TransportDeleteJobAction.java | 19 +- .../TransportEvaluateDataFrameAction.java | 61 + .../TransportFinalizeJobExecutionAction.java | 12 +- .../TransportFindFileStructureAction.java | 10 +- .../TransportGetDataFrameAnalyticsAction.java | 82 + ...sportGetDataFrameAnalyticsStatsAction.java | 196 ++ .../action/TransportGetDatafeedsAction.java | 3 +- .../TransportGetDatafeedsStatsAction.java | 3 +- .../ml/action/TransportGetJobsAction.java | 3 +- .../action/TransportGetJobsStatsAction.java | 24 +- .../ml/action/TransportMlInfoAction.java | 14 +- .../ml/action/TransportOpenJobAction.java | 296 +- .../ml/action/TransportPostDataAction.java | 5 +- .../TransportPutDataFrameAnalyticsAction.java | 166 + .../ml/action/TransportPutDatafeedAction.java | 2 +- .../ml/action/TransportPutJobAction.java | 2 +- .../TransportRevertModelSnapshotAction.java | 3 +- .../action/TransportSetUpgradeModeAction.java | 14 +- ...ransportStartDataFrameAnalyticsAction.java | 490 +++ .../action/TransportStartDatafeedAction.java | 12 +- ...TransportStopDataFrameAnalyticsAction.java | 270 ++ .../action/TransportUpdateDatafeedAction.java | 3 +- .../ml/action/TransportUpdateJobAction.java | 4 +- .../xpack/ml/datafeed/DatafeedJob.java | 8 +- .../AggregationToJsonProcessor.java | 13 +- .../extractor/fields/ExtractedField.java | 144 +- .../extractor/fields/ExtractedFields.java | 20 + .../ml/dataframe/DataFrameAnalyticsIndex.java | 197 ++ .../dataframe/DataFrameAnalyticsManager.java | 219 ++ .../xpack/ml/dataframe/MappingsMerger.java | 100 + .../ml/dataframe/SourceDestValidator.java | 68 + .../extractor/DataFrameDataExtractor.java | 276 ++ .../DataFrameDataExtractorContext.java | 35 + .../DataFrameDataExtractorFactory.java | 170 + .../extractor/ExtractedFieldsDetector.java | 169 + .../DataFrameAnalyticsConfigProvider.java | 122 + .../dataframe/process/AnalyticsBuilder.java | 74 + .../AnalyticsControlMessageWriter.java | 38 + .../dataframe/process/AnalyticsProcess.java | 34 + .../process/AnalyticsProcessConfig.java | 76 + .../process/AnalyticsProcessFactory.java | 24 + .../process/AnalyticsProcessManager.java | 301 ++ .../ml/dataframe/process/AnalyticsResult.java | 77 + .../process/AnalyticsResultProcessor.java | 87 + .../process/DataFrameRowsJoiner.java | 186 ++ .../process/NativeAnalyticsProcess.java | 52 + .../NativeAnalyticsProcessFactory.java | 94 + .../dataframe/process/results/RowResults.java | 73 + .../DelimitedFileStructureFinder.java | 7 +- .../DelimitedFileStructureFinderFactory.java | 3 +- .../FieldStatsCalculator.java | 88 +- .../FileStructureFinderFactory.java | 4 +- .../FileStructureFinderManager.java | 55 +- .../FileStructureUtils.java | 9 +- .../GrokPatternCreator.java | 97 +- .../NdJsonFileStructureFinder.java | 3 +- .../NdJsonFileStructureFinderFactory.java | 3 +- .../TextLogFileStructureFinder.java | 26 +- .../TextLogFileStructureFinderFactory.java | 4 +- .../filestructurefinder/TimeoutChecker.java | 14 +- .../TimestampFormatFinder.java | 6 +- .../XmlFileStructureFinder.java | 3 +- .../XmlFileStructureFinderFactory.java | 2 +- .../xpack/ml/job/JobManager.java | 2 +- .../xpack/ml/job/JobNodeSelector.java | 329 ++ .../CategorizationAnalyzer.java | 188 +- .../persistence/JobDataCountsPersister.java | 2 +- .../ml/job/persistence/JobDataDeleter.java | 3 +- .../job/persistence/JobResultsPersister.java | 12 + .../job/persistence/JobResultsProvider.java | 28 + .../job/persistence/TimingStatsReporter.java | 87 + .../autodetect/AutodetectCommunicator.java | 39 +- .../autodetect/AutodetectProcessManager.java | 34 +- .../autodetect/NativeAutodetectProcess.java | 27 +- .../NativeAutodetectProcessFactory.java | 27 +- ...or.java => AutodetectResultProcessor.java} | 128 +- .../autodetect/params/AutodetectParams.java | 22 +- .../autodetect/params/FlushJobParams.java | 2 +- .../process/autodetect/params/TimeRange.java | 2 +- .../writer/AbstractDataToProcessWriter.java | 4 +- .../normalizer/NativeNormalizerProcess.java | 6 +- .../NativeNormalizerProcessFactory.java | 18 +- .../process/normalizer/NormalizerResult.java | 9 +- .../retention/ExpiredForecastsRemover.java | 3 +- .../job/retention/ExpiredResultsRemover.java | 3 +- .../ml/process/AbstractNativeProcess.java | 23 +- .../xpack/ml/process/DummyController.java | 26 + .../xpack/ml/process/MlController.java | 36 + .../xpack/ml/process/MlControllerHolder.java | 28 + .../xpack/ml/process/MlMemoryTracker.java | 191 +- .../xpack/ml/process/NativeController.java | 13 +- .../ml/process/NativeControllerHolder.java | 60 - .../ProcessResultsParser.java} | 34 +- .../ml/rest/RestFindFileStructureAction.java | 2 + .../RestDeleteDataFrameAnalyticsAction.java | 39 + .../RestEvaluateDataFrameAction.java | 36 + .../RestGetDataFrameAnalyticsAction.java | 51 + .../RestGetDataFrameAnalyticsStatsAction.java | 52 + .../RestPutDataFrameAnalyticsAction.java | 43 + .../RestStartDataFrameAnalyticsAction.java | 50 + .../RestStopDataFrameAnalyticsAction.java | 49 + .../plugin-security-test.policy | 5 + .../MachineLearningLicensingTests.java | 76 +- ...hineLearningInfoTransportActionTests.java} | 108 +- .../xpack/ml/MachineLearningTests.java | 4 +- ...lConfigMigrationEligibilityCheckTests.java | 83 +- .../xpack/ml/MlConfigMigratorTests.java | 2 +- .../xpack/ml/MlMetadataTests.java | 4 +- .../xpack/ml/MlSingleNodeTestCase.java | 2 +- ...nsportFinalizeJobExecutionActionTests.java | 2 +- .../TransportGetJobsStatsActionTests.java | 25 +- .../action/TransportOpenJobActionTests.java | 399 +-- .../TransportStartDatafeedActionTests.java | 2 +- ...portStopDataFrameAnalyticsActionTests.java | 85 + .../datafeed/DatafeedJobValidatorTests.java | 2 +- .../DelayedDataDetectorFactoryTests.java | 2 +- .../extractor/DataExtractorFactoryTests.java | 2 +- .../AggregationDataExtractorFactoryTests.java | 2 +- .../aggregation/AggregationTestUtils.java | 11 + .../AggregationToJsonProcessorTests.java | 15 + .../ChunkedDataExtractorFactoryTests.java | 2 +- .../extractor/fields/ExtractedFieldTests.java | 28 + .../DataFrameAnalyticsIndexTests.java | 175 + .../ml/dataframe/MappingsMergerTests.java | 137 + .../dataframe/SourceDestValidatorTests.java | 180 + .../DataFrameDataExtractorTests.java | 392 +++ .../ExtractedFieldsDetectorTests.java | 319 ++ .../AnalyticsControlMessageWriterTests.java | 50 + .../AnalyticsResultProcessorTests.java | 86 + .../process/AnalyticsResultTests.java | 39 + .../process/DataFrameRowsJoinerTests.java | 280 ++ .../process/results/RowResultsTests.java | 42 + .../DelimitedFileStructureFinderTests.java | 59 +- .../FieldStatsCalculatorTests.java | 102 +- .../FileStructureFinderManagerTests.java | 55 +- .../FileStructureUtilsTests.java | 6 +- .../GrokPatternCreatorTests.java | 20 +- .../NdJsonFileStructureFinderTests.java | 2 +- .../TextLogFileStructureFinderTests.java | 44 +- .../TimestampFormatFinderTests.java | 13 +- .../XmlFileStructureFinderTests.java | 2 +- .../ml/integration/AnnotationIndexIT.java | 3 +- .../AutodetectResultProcessorIT.java | 139 +- .../integration/BasicDistributedJobsIT.java | 17 +- .../ml/integration/MlConfigMigratorIT.java | 3 +- .../integration/MlDistributedFailureIT.java | 9 +- .../ml/integration/NetworkDisruptionIT.java | 6 - .../xpack/ml/job/JobManagerTests.java | 2 +- .../xpack/ml/job/JobNodeSelectorTests.java | 576 ++++ .../CategorizationAnalyzerTests.java | 55 +- .../BatchedDocumentsIteratorTests.java | 1 - .../persistence/JobResultsPersisterTests.java | 32 + .../persistence/JobResultsProviderTests.java | 55 + .../ml/job/persistence/MockClientBuilder.java | 24 +- .../job/persistence/StateStreamerTests.java | 1 - .../persistence/TimingStatsReporterTests.java | 124 + .../AutodetectCommunicatorTests.java | 34 +- .../AutodetectProcessManagerTests.java | 4 - .../NativeAutodetectProcessFactoryTests.java | 61 + .../NativeAutodetectProcessTests.java | 24 +- ...va => AutodetectResultProcessorTests.java} | 348 +- .../output/AutodetectResultsParserTests.java | 422 --- .../params/AutodetectParamsTests.java | 31 + .../AbstractDataToProcessWriterTests.java | 2 +- .../writer/CsvDataToProcessWriterTests.java | 2 +- .../writer/JsonDataToProcessWriterTests.java | 2 +- .../xpack/ml/job/results/BucketTests.java | 2 +- .../ml/process/MlMemoryTrackerTests.java | 63 +- .../ml/process/ProcessResultsParserTests.java | 113 + .../xpack/ml/support/BaseMlIntegTestCase.java | 43 +- x-pack/plugin/monitoring/build.gradle | 11 +- .../xpack/monitoring/Monitoring.java | 36 +- .../monitoring/MonitoringFeatureSet.java | 88 - .../MonitoringInfoTransportAction.java | 45 + .../monitoring/MonitoringUsageServices.java | 23 + .../MonitoringUsageTransportAction.java | 72 + .../collector/ccr/StatsCollector.java | 12 +- .../collector/ml/JobStatsCollector.java | 10 +- .../exporter/ClusterAlertsUtil.java | 13 +- .../exporter/local/LocalExporter.java | 31 +- .../rest/action/RestMonitoringBulkAction.java | 53 +- .../monitoring/LocalStateMonitoring.java | 32 + ...> MonitoringInfoTransportActionTests.java} | 27 +- .../ClusterStatsMonitoringDocTests.java | 13 +- .../IndexRecoveryMonitoringDocTests.java | 5 +- .../collector/ml/JobStatsCollectorTests.java | 9 +- .../ml/JobStatsMonitoringDocTests.java | 21 +- .../node/NodeStatsMonitoringDocTests.java | 2 +- .../exporter/http/HttpExporterSslIT.java | 3 +- .../local/LocalExporterIntegTests.java | 10 +- .../action/RestMonitoringBulkActionTests.java | 51 +- .../test/MonitoringIntegTestCase.java | 15 +- x-pack/plugin/rollup/build.gradle | 5 +- .../elasticsearch/xpack/rollup/Rollup.java | 26 +- ...et.java => RollupInfoTransportAction.java} | 33 +- .../rollup/RollupUsageTransportAction.java | 50 + .../action/TransportPutRollupJobAction.java | 13 +- .../action/TransportRollupSearchAction.java | 4 +- .../xpack/rollup/job/IndexerUtils.java | 5 +- .../xpack/rollup/job/RollupJobTask.java | 24 +- ...va => RollupInfoTransportActionTests.java} | 30 +- .../rollup/RollupRequestTranslationTests.java | 2 +- .../action/PutJobStateMachineTests.java | 5 +- .../rollup/action/SearchActionTests.java | 2 +- .../job/RollupIndexerIndexingTests.java | 2 +- .../xpack/rollup/job/RollupJobTaskTests.java | 9 +- x-pack/plugin/security/build.gradle | 3 +- x-pack/plugin/security/cli/build.gradle | 5 +- .../qa/basic-enable-security/build.gradle | 94 +- .../security/qa/security-basic/build.gradle | 22 +- .../security/SecurityWithBasicLicenseIT.java | 14 +- .../plugin/security/qa/tls-basic/build.gradle | 27 +- .../xpack/security/Security.java | 110 +- .../security/SecurityInfoTransportAction.java | 48 + .../xpack/security/SecurityUsageServices.java | 30 + ...java => SecurityUsageTransportAction.java} | 85 +- .../security/action/SecurityActionMapper.java | 5 +- .../TransportGetBuiltinPrivilegesAction.java | 39 + .../TransportGetPrivilegesAction.java | 6 +- .../realm/TransportClearRealmCacheAction.java | 7 +- .../role/TransportClearRolesCacheAction.java | 7 +- .../token/TransportCreateTokenAction.java | 76 +- .../token/TransportRefreshTokenAction.java | 2 +- .../audit/logfile/LoggingAuditTrail.java | 2 + .../xpack/security/authc/ApiKeyService.java | 46 +- .../xpack/security/authc/TokenService.java | 2 +- .../authc/esnative/NativeUsersStore.java | 12 +- .../esnative/tool/SetupPasswordTool.java | 2 +- .../security/authc/file/tool/UsersTool.java | 30 +- .../authc/kerberos/KerberosRealm.java | 13 +- .../ldap/ActiveDirectoryGroupsResolver.java | 7 +- .../xpack/security/authc/ldap/LdapRealm.java | 3 +- .../authc/ldap/SearchGroupsResolver.java | 6 +- .../ldap/UserAttributeGroupsResolver.java | 11 +- .../ldap/support/LdapMetaDataResolver.java | 16 +- .../oidc/OpenIdConnectAuthenticator.java | 10 +- .../authc/oidc/OpenIdConnectRealm.java | 41 +- .../xpack/security/authc/pki/PkiRealm.java | 7 +- .../security/authc/saml/SamlAttributes.java | 25 +- .../xpack/security/authc/saml/SamlRealm.java | 18 +- .../authc/support/UserRoleMapper.java | 12 +- .../mapper/NativeRoleMappingStore.java | 12 +- .../authz/IndicesAndAliasesResolver.java | 7 - .../authz/store/CompositeRolesStore.java | 56 +- .../authz/store/NativePrivilegeStore.java | 10 +- .../authz/store/NativeRolesStore.java | 10 +- .../action/apikey/RestCreateApiKeyAction.java | 17 +- .../action/oauth2/RestGetTokenAction.java | 42 +- .../privilege/RestDeletePrivilegesAction.java | 32 +- .../RestGetBuiltinPrivilegesAction.java | 58 + .../privilege/RestGetPrivilegesAction.java | 54 +- .../privilege/RestPutPrivilegesAction.java | 7 +- .../realm/RestClearRealmCacheAction.java | 8 +- .../role/RestClearRolesCacheAction.java | 8 +- .../action/role/RestDeleteRoleAction.java | 23 +- .../rest/action/role/RestGetRolesAction.java | 6 +- .../rest/action/role/RestPutRoleAction.java | 9 +- .../RestDeleteRoleMappingAction.java | 21 +- .../RestGetRoleMappingsAction.java | 35 +- .../rolemapping/RestPutRoleMappingAction.java | 9 +- .../saml/RestSamlAuthenticateAction.java | 6 +- .../action/user/RestChangePasswordAction.java | 23 +- .../action/user/RestDeleteUserAction.java | 25 +- .../user/RestGetUserPrivilegesAction.java | 3 +- .../rest/action/user/RestGetUsersAction.java | 4 +- .../action/user/RestHasPrivilegesAction.java | 5 +- .../rest/action/user/RestPutUserAction.java | 9 +- .../action/user/RestSetEnabledAction.java | 18 +- .../support/SecurityIndexManager.java | 77 +- .../support/SecurityStatusChangeListener.java | 45 + .../SecurityServerTransportInterceptor.java | 106 +- .../transport/ServerTransportFilter.java | 185 +- .../security/transport/nio/NioIPFilter.java | 39 +- .../transport/nio/SSLChannelContext.java | 21 +- .../nio/SecurityNioHttpServerTransport.java | 16 +- .../transport/nio/SecurityNioTransport.java | 25 +- .../integration/ClearRealmsCacheTests.java | 6 +- .../integration/ClearRolesCacheTests.java | 79 +- .../DocumentLevelSecurityTests.java | 39 +- .../integration/FieldLevelSecurityTests.java | 64 +- .../MultipleIndicesPermissionsTests.java | 16 +- .../SecurityCachePermissionTests.java | 5 +- .../integration/SecurityClearScrollTests.java | 6 +- .../elasticsearch/license/LicensingTests.java | 4 +- .../test/NativeRealmIntegTestCase.java | 11 +- .../test/SecurityIntegTestCase.java | 37 +- .../test/SecuritySettingsSource.java | 96 +- .../test/SecuritySingleNodeTestCase.java | 6 - .../PutPrivilegesRequestBuilderTests.java | 8 +- .../xpack/security/LocalStateSecurity.java | 54 + .../security/PkiRealmBootstrapCheckTests.java | 2 +- ... => SecurityInfoTransportActionTests.java} | 43 +- .../xpack/security/SecurityTests.java | 79 - .../action/SecurityActionMapperTests.java | 9 +- ...ansportOpenIdConnectLogoutActionTests.java | 2 +- .../PutRoleMappingRequestTests.java | 9 +- ...sportSamlInvalidateSessionActionTests.java | 4 +- .../TransportCreateTokenActionTests.java | 68 +- .../audit/logfile/LoggingAuditTrailTests.java | 6 +- .../security/authc/ApiKeyIntegTests.java | 82 +- .../authc/AuthenticationServiceTests.java | 16 +- .../xpack/security/authc/RunAsIntegTests.java | 10 +- .../security/authc/TokenAuthIntegTests.java | 525 ++- .../security/authc/TokenServiceTests.java | 21 +- .../authc/esnative/NativeRealmIntegTests.java | 287 +- .../authc/esnative/NativeRealmTests.java | 6 +- .../authc/esnative/NativeUsersStoreTests.java | 4 +- .../esnative/ReservedRealmIntegTests.java | 42 +- .../esnative/tool/SetupPasswordToolTests.java | 14 +- .../SessionFactoryLoadBalancingTests.java | 2 - .../oidc/OpenIdConnectAuthenticatorTests.java | 35 + .../authc/pki/PkiAuthenticationTests.java | 3 +- .../authc/saml/SamlMetadataCommandTests.java | 2 +- .../security/authc/saml/SamlRealmTests.java | 3 +- .../mapper/ExpressionRoleMappingTests.java | 19 +- .../mapper/NativeRoleMappingStoreTests.java | 23 +- .../authz/AuthorizationServiceTests.java | 14 +- .../authz/IndicesAndAliasesResolverTests.java | 24 - .../security/authz/ReadActionsTests.java | 34 +- .../security/authz/SecurityScrollTests.java | 9 +- .../authz/SnapshotUserRoleIntegTests.java | 49 +- .../security/authz/WriteActionsTests.java | 100 +- .../authz/store/CompositeRolesStoreTests.java | 53 +- .../store/NativePrivilegeStoreTests.java | 6 +- .../ingest/SetSecurityUserProcessorTests.java | 9 +- .../apikey/RestCreateApiKeyActionTests.java | 7 +- .../apikey/RestGetApiKeyActionTests.java | 7 +- .../RestInvalidateApiKeyActionTests.java | 7 +- .../oauth2/RestGetTokenActionTests.java | 42 +- .../SecurityIndexManagerIntegTests.java | 22 +- .../support/SecurityIndexManagerTests.java | 62 +- .../SecurityStatusChangeListenerTests.java | 115 + ...stractSimpleSecurityTransportTestCase.java | 7 +- ...curityServerTransportInterceptorTests.java | 43 +- ...ServerTransportFilterIntegrationTests.java | 208 -- .../transport/ServerTransportFilterTests.java | 43 +- .../filter/IpFilteringIntegrationTests.java | 4 +- .../transport/nio/NioIPFilterTests.java | 36 +- .../transport/nio/SSLChannelContextTests.java | 6 +- .../transport/ssl/EllipticCurveSSLTests.java | 2 +- .../transport/ssl/SslIntegrationTests.java | 2 +- .../xpack/ssl/SSLClientAuthTests.java | 3 +- .../xpack/ssl/SSLTrustRestrictionsTests.java | 3 - x-pack/plugin/sql/build.gradle | 5 +- x-pack/plugin/sql/jdbc/build.gradle | 6 +- .../xpack/sql/jdbc/TypeConverter.java | 5 +- x-pack/plugin/sql/qa/build.gradle | 20 +- x-pack/plugin/sql/qa/multi-node/build.gradle | 4 +- x-pack/plugin/sql/qa/security/build.gradle | 33 +- .../sql/qa/security/with-ssl/build.gradle | 74 +- .../sql/qa/security/without-ssl/build.gradle | 14 +- x-pack/plugin/sql/qa/single-node/build.gradle | 7 +- .../sql/qa/single_node/JdbcCsvSpecIT.java | 2 +- .../xpack/sql/qa/ErrorsTestCase.java | 1 + .../sql/qa/cli/CliIntegrationTestCase.java | 8 +- .../xpack/sql/qa/cli/ErrorsTestCase.java | 9 +- .../xpack/sql/qa/cli/FetchSizeTestCase.java | 28 + .../xpack/sql/qa/jdbc/DebugCsvSpec.java | 64 - .../xpack/sql/qa/jdbc/DebugSqlSpec.java | 5 +- .../xpack/sql/qa/jdbc/ErrorsTestCase.java | 10 + .../xpack/sql/qa/jdbc/JdbcAssert.java | 5 +- .../xpack/sql/qa/jdbc/SqlSpecTestCase.java | 7 + .../xpack/sql/qa/rest/RestSqlTestCase.java | 104 +- .../src/main/resources/agg-ordering.sql-spec | 6 +- .../qa/src/main/resources/docs/docs.csv-spec | 9 +- .../qa/src/main/resources/fulltext.csv-spec | 8 +- .../single-node-only/command-sys-geo.csv-spec | 12 +- .../single-node-only/command-sys.csv-spec | 162 +- x-pack/plugin/sql/sql-action/build.gradle | 6 +- .../sql/action/SqlClearCursorAction.java | 4 +- .../xpack/sql/action/SqlQueryAction.java | 4 +- .../xpack/sql/action/SqlTranslateAction.java | 4 +- .../sql/action/SqlQueryRequestTests.java | 4 +- .../sql/action/SqlTranslateRequestTests.java | 4 +- x-pack/plugin/sql/sql-cli/build.gradle | 8 +- x-pack/plugin/sql/sql-client/build.gradle | 2 +- x-pack/plugin/sql/sql-proto/build.gradle | 6 +- .../xpack/sql/SqlFeatureSet.java | 87 - .../xpack/sql/SqlInfoTransportAction.java | 46 + .../xpack/sql/SqlUsageTransportAction.java | 75 + .../search/CompositeAggregationCursor.java | 33 +- .../execution/search/CompositeAggsRowSet.java | 20 +- .../xpack/sql/execution/search/Querier.java | 155 +- .../search/SchemaCompositeAggsRowSet.java | 2 +- .../sql/execution/search/SourceGenerator.java | 10 +- .../search/extractor/FieldHitExtractor.java | 6 +- .../function/scalar/geo/GeoShape.java | 16 +- .../elasticsearch/xpack/sql/package-info.java | 12 +- .../xpack/sql/plugin/SqlPlugin.java | 22 +- .../xpack/sql/plugin/SqlStatsAction.java | 4 +- .../xpack/sql/plugin/SqlStatsRequest.java | 3 +- .../xpack/sql/plugin/TextFormat.java | 2 +- .../sql/plugin/TransportSqlStatsAction.java | 7 +- .../xpack/sql/querydsl/query/MatchQuery.java | 1 - .../sql/querydsl/query/MultiMatchQuery.java | 1 - ....java => SqlInfoTransportActionTests.java} | 25 +- .../CompositeAggregationCursorTests.java | 5 +- .../sql/execution/search/QuerierTests.java | 109 + .../extractor/FieldHitExtractorTests.java | 26 +- .../extractor/TestMultiValueAggregation.java | 10 - .../extractor/TestSingleValueAggregation.java | 9 - .../scalar/geo/StWkttosqlProcessorTests.java | 9 + .../xpack/sql/plugin/SqlPluginTests.java | 3 +- .../api/ccr.delete_auto_follow_pattern.json | 2 +- .../rest-api-spec/api/ccr.follow.json | 2 +- .../rest-api-spec/api/ccr.follow_info.json | 2 +- .../rest-api-spec/api/ccr.follow_stats.json | 2 +- .../api/ccr.forget_follower.json | 2 +- .../api/ccr.get_auto_follow_pattern.json | 2 +- .../rest-api-spec/api/ccr.pause_follow.json | 2 +- .../api/ccr.put_auto_follow_pattern.json | 2 +- .../rest-api-spec/api/ccr.resume_follow.json | 2 +- .../rest-api-spec/api/ccr.stats.json | 2 +- .../rest-api-spec/api/ccr.unfollow.json | 2 +- ...ata_frame.delete_data_frame_transform.json | 2 +- .../data_frame.get_data_frame_transform.json | 7 +- ..._frame.get_data_frame_transform_stats.json | 7 +- ...ta_frame.preview_data_frame_transform.json | 2 +- .../data_frame.put_data_frame_transform.json | 2 +- ...data_frame.start_data_frame_transform.json | 2 +- .../data_frame.stop_data_frame_transform.json | 7 +- .../rest-api-spec/api/graph.explore.json | 2 +- .../api/ilm.delete_lifecycle.json | 2 +- .../api/ilm.explain_lifecycle.json | 2 +- .../rest-api-spec/api/ilm.get_lifecycle.json | 2 +- .../rest-api-spec/api/ilm.get_status.json | 2 +- .../rest-api-spec/api/ilm.move_to_step.json | 2 +- .../rest-api-spec/api/ilm.put_lifecycle.json | 2 +- .../rest-api-spec/api/ilm.remove_policy.json | 2 +- .../rest-api-spec/api/ilm.retry.json | 2 +- .../rest-api-spec/api/ilm.start.json | 2 +- .../resources/rest-api-spec/api/ilm.stop.json | 2 +- .../rest-api-spec/api/indices.freeze.json | 2 +- .../api/indices.reload_search_analyzers.json | 33 + .../rest-api-spec/api/indices.unfreeze.json | 2 +- .../rest-api-spec/api/license.delete.json | 2 +- .../rest-api-spec/api/license.get.json | 2 +- .../api/license.get_basic_status.json | 2 +- .../api/license.get_trial_status.json | 2 +- .../rest-api-spec/api/license.post.json | 2 +- .../api/license.post_start_basic.json | 2 +- .../api/license.post_start_trial.json | 2 +- .../api/migration.deprecations.json | 2 +- .../rest-api-spec/api/ml.close_job.json | 2 +- .../rest-api-spec/api/ml.delete_calendar.json | 2 +- .../api/ml.delete_calendar_event.json | 2 +- .../api/ml.delete_calendar_job.json | 2 +- .../api/ml.delete_data_frame_analytics.json | 18 + .../rest-api-spec/api/ml.delete_datafeed.json | 2 +- .../api/ml.delete_expired_data.json | 2 +- .../rest-api-spec/api/ml.delete_filter.json | 2 +- .../rest-api-spec/api/ml.delete_forecast.json | 2 +- .../rest-api-spec/api/ml.delete_job.json | 2 +- .../api/ml.delete_model_snapshot.json | 2 +- .../api/ml.evaluate_data_frame.json | 15 + .../api/ml.find_file_structure.json | 7 +- .../rest-api-spec/api/ml.flush_job.json | 6 +- .../rest-api-spec/api/ml.forecast.json | 2 +- .../rest-api-spec/api/ml.get_buckets.json | 2 +- .../api/ml.get_calendar_events.json | 2 +- .../rest-api-spec/api/ml.get_calendars.json | 2 +- .../rest-api-spec/api/ml.get_categories.json | 2 +- .../api/ml.get_data_frame_analytics.json | 38 + .../ml.get_data_frame_analytics_stats.json | 38 + .../api/ml.get_datafeed_stats.json | 2 +- .../rest-api-spec/api/ml.get_datafeeds.json | 2 +- .../rest-api-spec/api/ml.get_filters.json | 2 +- .../rest-api-spec/api/ml.get_influencers.json | 2 +- .../rest-api-spec/api/ml.get_job_stats.json | 2 +- .../rest-api-spec/api/ml.get_jobs.json | 2 +- .../api/ml.get_model_snapshots.json | 2 +- .../api/ml.get_overall_buckets.json | 2 +- .../rest-api-spec/api/ml.get_records.json | 2 +- .../resources/rest-api-spec/api/ml.info.json | 2 +- .../rest-api-spec/api/ml.open_job.json | 2 +- .../api/ml.post_calendar_events.json | 2 +- .../rest-api-spec/api/ml.post_data.json | 2 +- .../api/ml.preview_datafeed.json | 2 +- .../rest-api-spec/api/ml.put_calendar.json | 2 +- .../api/ml.put_calendar_job.json | 2 +- .../api/ml.put_data_frame_analytics.json | 21 + .../rest-api-spec/api/ml.put_datafeed.json | 2 +- .../rest-api-spec/api/ml.put_filter.json | 2 +- .../rest-api-spec/api/ml.put_job.json | 2 +- .../api/ml.revert_model_snapshot.json | 2 +- .../api/ml.set_upgrade_mode.json | 2 +- .../api/ml.start_data_frame_analytics.json | 27 + .../rest-api-spec/api/ml.start_datafeed.json | 2 +- .../api/ml.stop_data_frame_analytics.json | 37 + .../rest-api-spec/api/ml.stop_datafeed.json | 6 +- .../rest-api-spec/api/ml.update_datafeed.json | 2 +- .../rest-api-spec/api/ml.update_filter.json | 2 +- .../rest-api-spec/api/ml.update_job.json | 2 +- .../api/ml.update_model_snapshot.json | 2 +- .../rest-api-spec/api/ml.validate.json | 2 +- .../api/ml.validate_detector.json | 2 +- .../rest-api-spec/api/monitoring.bulk.json | 2 +- .../rest-api-spec/api/rollup.delete_job.json | 2 +- .../rest-api-spec/api/rollup.get_jobs.json | 2 +- .../api/rollup.get_rollup_caps.json | 2 +- .../api/rollup.get_rollup_index_caps.json | 2 +- .../rest-api-spec/api/rollup.put_job.json | 2 +- .../api/rollup.rollup_search.json | 2 +- .../rest-api-spec/api/rollup.start_job.json | 2 +- .../rest-api-spec/api/rollup.stop_job.json | 2 +- .../api/security.authenticate.json | 2 +- .../api/security.change_password.json | 2 +- .../api/security.clear_cached_realms.json | 2 +- .../api/security.clear_cached_roles.json | 2 +- .../api/security.create_api_key.json | 2 +- .../api/security.delete_privileges.json | 2 +- .../api/security.delete_role.json | 2 +- .../api/security.delete_role_mapping.json | 2 +- .../api/security.delete_user.json | 2 +- .../api/security.disable_user.json | 2 +- .../api/security.enable_user.json | 2 +- .../api/security.get_api_key.json | 2 +- .../api/security.get_builtin_privileges.json | 15 + .../api/security.get_privileges.json | 4 +- .../rest-api-spec/api/security.get_role.json | 2 +- .../api/security.get_role_mapping.json | 2 +- .../rest-api-spec/api/security.get_token.json | 2 +- .../rest-api-spec/api/security.get_user.json | 2 +- .../api/security.get_user_privileges.json | 2 +- .../api/security.has_privileges.json | 2 +- .../api/security.invalidate_api_key.json | 2 +- .../api/security.invalidate_token.json | 2 +- .../api/security.put_privileges.json | 2 +- .../rest-api-spec/api/security.put_role.json | 2 +- .../api/security.put_role_mapping.json | 2 +- .../rest-api-spec/api/security.put_user.json | 2 +- .../rest-api-spec/api/sql.clear_cursor.json | 2 +- .../rest-api-spec/api/sql.query.json | 2 +- .../rest-api-spec/api/sql.translate.json | 2 +- .../rest-api-spec/api/ssl.certificates.json | 2 +- .../rest-api-spec/api/watcher.ack_watch.json | 2 +- .../api/watcher.activate_watch.json | 2 +- .../api/watcher.deactivate_watch.json | 2 +- .../api/watcher.delete_watch.json | 3 +- .../api/watcher.execute_watch.json | 2 +- .../rest-api-spec/api/watcher.get_watch.json | 2 +- .../rest-api-spec/api/watcher.put_watch.json | 2 +- .../rest-api-spec/api/watcher.start.json | 2 +- .../rest-api-spec/api/watcher.stats.json | 2 +- .../rest-api-spec/api/watcher.stop.json | 2 +- .../rest-api-spec/api/xpack.info.json | 2 +- .../rest-api-spec/api/xpack.usage.json | 2 +- .../rest-api-spec/test/api_key/10_basic.yml | 6 +- .../test/data_frame/preview_transforms.yml | 135 +- .../test/data_frame/transforms_crud.yml | 165 +- .../test/data_frame/transforms_start_stop.yml | 149 +- .../test/data_frame/transforms_stats.yml | 67 +- .../rest-api-spec/test/flattened/10_basic.yml | 111 + .../test/indices.freeze/10_basic.yml | 19 + .../test/indices.freeze/20_stats.yml | 62 + .../test/ml/data_frame_analytics_crud.yml | 926 ++++++ .../test/ml/evaluate_data_frame.yml | 520 +++ .../rest-api-spec/test/ml/filter_crud.yml | 23 +- .../test/ml/find_file_structure.yml | 7 +- .../rest-api-spec/test/ml/jobs_get_stats.yml | 100 +- .../test/ml/start_data_frame_analytics.yml | 74 + .../test/ml/stop_data_frame_analytics.yml | 70 + .../test/privileges/11_builtin.yml | 19 + .../rest-api-spec/test/rollup/start_job.yml | 3 +- .../test/security/authz/14_cat_indices.yml | 125 + .../test/vectors/10_dense_vector_basic.yml | 5 +- .../vectors/20_dense_vector_special_cases.yml | 82 +- .../test/vectors/30_sparse_vector_basic.yml | 4 +- .../40_sparse_vector_special_cases.yml | 4 +- .../test/voting_only_node/10_basic.yml | 16 + .../test/watcher/execute_watch/80_foreach.yml | 111 + .../rest-api-spec/test/xpack/15_basic.yml | 8 - x-pack/plugin/vectors/build.gradle | 22 + .../elasticsearch/xpack/vectors/Vectors.java | 55 + .../vectors/VectorsInfoTransportAction.java | 46 + .../vectors/VectorsUsageTransportAction.java | 48 + .../mapper/DenseVectorFieldMapper.java | 99 +- .../mapper/SparseVectorFieldMapper.java | 29 +- .../vectors}/mapper/VectorEncoderDecoder.java | 31 +- .../query/DocValuesWhitelistExtension.java | 30 + .../vectors}/query/ScoreScriptUtils.java | 39 +- .../query/VectorDVAtomicFieldData.java | 22 +- .../query/VectorDVIndexFieldData.java | 22 +- .../vectors}/query/VectorScriptDocValues.java | 22 +- ...asticsearch.painless.spi.PainlessExtension | 1 + .../xpack/vectors/query/whitelist.txt | 18 + .../VectorsInfoTransportActionTests.java | 80 + .../mapper/DenseVectorFieldMapperTests.java | 125 + .../mapper/DenseVectorFieldTypeTests.java | 19 + .../mapper/SparseVectorFieldMapperTests.java | 31 +- .../mapper/SparseVectorFieldTypeTests.java | 19 + .../mapper/VectorEncoderDecoderTests.java | 41 +- .../vectors}/query/ScoreScriptUtilsTests.java | 45 +- x-pack/plugin/voting-only-node/build.gradle | 22 + .../VotingOnlyNodeFeatureSet.java | 102 + .../coordination/VotingOnlyNodePlugin.java | 230 ++ .../LocalStateVotingOnlyNodePlugin.java | 20 + .../VotingOnlyNodeCoordinationStateTests.java | 30 + .../VotingOnlyNodeCoordinatorTests.java | 72 + .../VotingOnlyNodePluginTests.java | 105 + x-pack/plugin/watcher/build.gradle | 6 +- .../xpack/watcher/ClockHolder.java | 23 + .../elasticsearch/xpack/watcher/Watcher.java | 29 +- .../xpack/watcher/WatcherFeatureSet.java | 92 - .../watcher/WatcherInfoTransportAction.java | 46 + .../watcher/WatcherUsageTransportAction.java | 82 + .../xpack/watcher/common/http/Scheme.java | 2 + .../notification/NotificationService.java | 17 +- .../notification/jira/JiraAccount.java | 5 +- .../watcher/rest/WatcherRestHandler.java | 31 - .../rest/action/RestAckWatchAction.java | 19 +- .../rest/action/RestActivateWatchAction.java | 55 +- .../rest/action/RestDeleteWatchAction.java | 17 +- .../rest/action/RestExecuteWatchAction.java | 25 +- .../rest/action/RestGetWatchAction.java | 13 +- .../rest/action/RestPutWatchAction.java | 15 +- .../rest/action/RestWatchServiceAction.java | 23 +- .../rest/action/RestWatcherStatsAction.java | 19 +- .../search/WatcherSearchTemplateService.java | 3 - .../actions/ack/TransportAckWatchAction.java | 5 +- .../TransportActivateWatchAction.java | 5 +- .../execute/TransportExecuteWatchAction.java | 5 +- .../actions/get/TransportGetWatchAction.java | 5 +- .../actions/put/TransportPutWatchAction.java | 5 +- .../TransportWatcherServiceAction.java | 13 +- .../stats/TransportWatcherStatsAction.java | 7 +- .../watcher/WatcherConcreteIndexTests.java | 3 +- .../watcher/WatcherIndexingListenerTests.java | 9 +- ...a => WatcherInfoTransportActionTests.java} | 29 +- .../watcher/WatcherLifeCycleServiceTests.java | 12 +- .../xpack/watcher/WatcherPluginTests.java | 3 +- .../xpack/watcher/WatcherServiceTests.java | 5 +- .../watcher/WatcherXpackUsageStatsTests.java | 52 - .../actions/ActionErrorIntegrationTests.java | 6 +- .../watcher/actions/ActionWrapperTests.java | 147 +- .../actions/TimeThrottleIntegrationTests.java | 12 +- .../actions/email/EmailAttachmentTests.java | 7 +- .../throttler/ActionThrottleTests.java | 51 +- .../webhook/WebhookHttpsIntegrationTests.java | 8 +- .../webhook/WebhookIntegrationTests.java | 10 +- .../watcher/common/http/HttpClientTests.java | 3 +- .../ExecuteWatchQueuedStatsTests.java | 16 +- .../execution/ExecutionServiceTests.java | 27 +- .../history/HistoryActionConditionTests.java | 5 +- .../HistoryTemplateEmailMappingsTests.java | 6 +- .../HistoryTemplateHttpMappingsTests.java | 10 +- ...storyTemplateIndexActionMappingsTests.java | 3 +- ...storyTemplateSearchInputMappingsTests.java | 3 +- .../HistoryTemplateTimeMappingsTests.java | 3 +- ...HistoryTemplateTransformMappingsTests.java | 10 +- .../input/chain/ChainIntegrationTests.java | 3 +- .../NotificationServiceTests.java | 7 + .../email/EmailSecretsIntegrationTests.java | 11 +- .../notification/jira/JiraAccountTests.java | 10 + .../action/RestExecuteWatchActionTests.java | 27 +- .../WatcherIndexTemplateRegistryTests.java | 4 - .../AbstractWatcherIntegrationTestCase.java | 26 +- .../xpack/watcher/test/WatcherTestUtils.java | 4 +- .../WatcherExecutorServiceBenchmark.java | 10 +- .../bench/WatcherScheduleEngineBenchmark.java | 10 +- .../test/integration/BasicWatcherTests.java | 49 +- .../test/integration/BootStrapTests.java | 26 +- .../ExecutionVarsIntegrationTests.java | 17 +- .../integration/HistoryIntegrationTests.java | 21 +- .../HttpSecretsIntegrationTests.java | 18 +- .../integration/RejectedExecutionTests.java | 5 +- .../test/integration/SearchInputTests.java | 2 +- .../test/integration/SingleNodeTests.java | 3 +- .../test/integration/WatchAckTests.java | 31 +- .../test/integration/WatchMetadataTests.java | 8 +- .../transform/TransformIntegrationTests.java | 18 +- .../action/activate/ActivateWatchTests.java | 47 +- .../action/delete/DeleteWatchTests.java | 16 +- .../action/execute/ExecuteWatchTests.java | 17 +- .../transport/action/get/GetWatchTests.java | 12 +- .../ack/TransportAckWatchActionTests.java | 4 +- .../put/TransportPutWatchActionTests.java | 3 +- .../TransportWatcherStatsActionTests.java | 4 +- .../watch/WatchStatusIntegrationTests.java | 8 +- .../xpack/watcher/watch/WatchTests.java | 8 +- .../build.gradle | 41 +- .../xpack/restart/FullClusterRestartIT.java | 1 - .../MlMigrationFullClusterRestartIT.java | 53 - x-pack/qa/kerberos-tests/build.gradle | 52 +- .../kerberos/KerberosAuthenticationIT.java | 45 +- ...SpnegoHttpClientConfigCallbackHandler.java | 76 + .../build.gradle | 67 +- x-pack/qa/multi-node/build.gradle | 23 +- x-pack/qa/oidc-op-tests/build.gradle | 42 +- .../authc/oidc/OpenIdConnectAuthIT.java | 29 +- x-pack/qa/openldap-tests/build.gradle | 1 - ...OpenLdapUserSearchSessionFactoryTests.java | 2 - .../authc/ldap/SearchGroupsResolverTests.java | 1 - .../reindex-tests-with-security/build.gradle | 42 +- x-pack/qa/rolling-upgrade/build.gradle | 8 +- .../upgrades/DataFrameSurvivesUpgradeIT.java | 279 ++ .../test/mixed_cluster/30_ml_jobs_crud.yml | 51 + .../mixed_cluster/80_data_frame_jobs_crud.yml | 292 ++ .../test/old_cluster/30_ml_jobs_crud.yml | 117 + .../old_cluster/80_data_frame_jobs_crud.yml | 218 ++ .../test/upgraded_cluster/30_ml_jobs_crud.yml | 56 + .../80_data_frame_jobs_crud.yml | 259 ++ x-pack/qa/saml-idp-tests/build.gradle | 82 +- .../build.gradle | 36 +- .../build.gradle | 23 +- .../build.gradle | 14 +- .../MonitoringWithWatcherRestIT.java | 2 - x-pack/qa/smoke-test-plugins-ssl/build.gradle | 35 +- .../SmokeTestMonitoringWithSecurityIT.java | 2 +- x-pack/qa/smoke-test-plugins/build.gradle | 28 +- .../build.gradle | 17 +- .../build.gradle | 32 +- .../SmokeTestWatcherWithSecurityIT.java | 13 +- x-pack/qa/smoke-test-watcher/build.gradle | 8 +- .../SmokeTestWatcherTestSuiteIT.java | 4 +- .../test/mustache/50_webhook_url_escaping.yml | 5 +- .../test/rest/XPackRestTestConstants.java | 8 +- .../ldap/AbstractAdLdapRealmTestCase.java | 6 +- .../ActiveDirectoryGroupsResolverTests.java | 1 - .../ActiveDirectorySessionFactoryTests.java | 9 - .../UserAttributeGroupsResolverTests.java | 3 - x-pack/qa/third-party/jira/build.gradle | 28 +- x-pack/qa/third-party/pagerduty/build.gradle | 21 +- x-pack/qa/third-party/slack/build.gradle | 21 +- x-pack/test/feature-aware/build.gradle | 6 +- 3726 files changed, 94944 insertions(+), 49742 deletions(-) delete mode 100644 buildSrc/src/main/groovy/org/elasticsearch/gradle/plugin/PluginPropertiesExtension.groovy delete mode 100644 buildSrc/src/main/groovy/org/elasticsearch/gradle/precommit/DependencyLicensesTask.groovy delete mode 100644 buildSrc/src/main/groovy/org/elasticsearch/gradle/precommit/UpdateShasTask.groovy delete mode 100644 buildSrc/src/main/groovy/org/elasticsearch/gradle/test/MessyTestPlugin.groovy create mode 100644 buildSrc/src/main/java/org/elasticsearch/gradle/AbstractLazyPropertyCollection.java create mode 100644 buildSrc/src/main/java/org/elasticsearch/gradle/DistributionDownloadPlugin.java create mode 100644 buildSrc/src/main/java/org/elasticsearch/gradle/ElasticsearchDistribution.java create mode 100644 buildSrc/src/main/java/org/elasticsearch/gradle/EmptyDirTask.java create mode 100644 buildSrc/src/main/java/org/elasticsearch/gradle/LazyPropertyList.java create mode 100644 buildSrc/src/main/java/org/elasticsearch/gradle/LazyPropertyMap.java create mode 100644 buildSrc/src/main/java/org/elasticsearch/gradle/PropertyNormalization.java create mode 100644 buildSrc/src/main/java/org/elasticsearch/gradle/SystemPropertyCommandLineArgumentProvider.java create mode 100644 buildSrc/src/main/java/org/elasticsearch/gradle/plugin/PluginPropertiesExtension.java create mode 100644 buildSrc/src/main/java/org/elasticsearch/gradle/precommit/DependencyLicensesTask.java create mode 100644 buildSrc/src/main/java/org/elasticsearch/gradle/precommit/UpdateShasTask.java create mode 100644 buildSrc/src/main/java/org/elasticsearch/gradle/test/RestTestRunnerTask.java create mode 100644 buildSrc/src/main/java/org/elasticsearch/gradle/testclusters/TestClusterCleanupOnShutdown.java create mode 100644 buildSrc/src/main/java/org/elasticsearch/gradle/testclusters/TestClustersCleanupExtension.java create mode 100644 buildSrc/src/main/java/org/elasticsearch/gradle/tool/ClasspathUtils.java rename buildSrc/src/main/resources/META-INF/gradle-plugins/{elasticsearch.messy-test.properties => elasticsearch.distribution-download.properties} (91%) create mode 100644 buildSrc/src/main/resources/buildSrc.marker rename buildSrc/src/{main => minimumRuntime}/java/org/elasticsearch/gradle/JdkJarHellCheck.java (100%) rename buildSrc/src/{main => minimumRuntime}/java/org/elasticsearch/gradle/LazyFileOutputStream.java (100%) rename buildSrc/src/{main => minimumRuntime}/java/org/elasticsearch/gradle/LoggedExec.java (94%) rename buildSrc/src/{main => minimumRuntime}/java/org/elasticsearch/gradle/Version.java (100%) rename buildSrc/src/{main => minimumRuntime}/java/org/elasticsearch/gradle/VersionProperties.java (100%) create mode 100644 buildSrc/src/test/java/org/elasticsearch/gradle/DistributionDownloadPluginIT.java create mode 100644 buildSrc/src/test/java/org/elasticsearch/gradle/DistributionDownloadPluginTests.java create mode 100644 buildSrc/src/test/java/org/elasticsearch/gradle/EmptyDirTaskTests.java create mode 100644 buildSrc/src/test/java/org/elasticsearch/gradle/plugin/PluginPropertiesExtensionTests.java create mode 100644 buildSrc/src/test/java/org/elasticsearch/gradle/precommit/DependencyLicensesTaskTests.java create mode 100644 buildSrc/src/test/java/org/elasticsearch/gradle/precommit/UpdateShasTaskTests.java create mode 100644 buildSrc/src/testKit/distribution-download/build.gradle create mode 100644 buildSrc/src/testKit/distribution-download/distribution/build.gradle create mode 100644 buildSrc/src/testKit/distribution-download/distribution/files/fake_elasticsearch.tar.gz create mode 100644 buildSrc/src/testKit/distribution-download/distribution/files/fake_elasticsearch.zip create mode 100644 buildSrc/src/testKit/distribution-download/settings.gradle create mode 100644 buildSrc/src/testKit/distribution-download/subproj/build.gradle delete mode 100644 client/benchmark/src/main/java/org/elasticsearch/client/benchmark/transport/TransportClientBenchmark.java rename modules/mapper-extras/src/main/java/org/elasticsearch/index/query/DocValuesWhitelistExtension.java => client/rest-high-level/src/main/java/org/elasticsearch/client/dataframe/DataFrameNamedXContentProvider.java (52%) create mode 100644 client/rest-high-level/src/main/java/org/elasticsearch/client/dataframe/transforms/NodeAttributes.java create mode 100644 client/rest-high-level/src/main/java/org/elasticsearch/client/dataframe/transforms/SyncConfig.java create mode 100644 client/rest-high-level/src/main/java/org/elasticsearch/client/dataframe/transforms/TimeSyncConfig.java create mode 100644 client/rest-high-level/src/main/java/org/elasticsearch/client/dataframe/transforms/util/TimeUtil.java create mode 100644 client/rest-high-level/src/main/java/org/elasticsearch/client/indices/AnalyzeRequest.java create mode 100644 client/rest-high-level/src/main/java/org/elasticsearch/client/indices/AnalyzeResponse.java create mode 100644 client/rest-high-level/src/main/java/org/elasticsearch/client/indices/DetailAnalyzeResponse.java create mode 100644 client/rest-high-level/src/main/java/org/elasticsearch/client/indices/ReloadAnalyzersRequest.java create mode 100644 client/rest-high-level/src/main/java/org/elasticsearch/client/indices/ReloadAnalyzersResponse.java create mode 100644 client/rest-high-level/src/main/java/org/elasticsearch/client/ml/DeleteDataFrameAnalyticsRequest.java create mode 100644 client/rest-high-level/src/main/java/org/elasticsearch/client/ml/EvaluateDataFrameRequest.java create mode 100644 client/rest-high-level/src/main/java/org/elasticsearch/client/ml/EvaluateDataFrameResponse.java create mode 100644 client/rest-high-level/src/main/java/org/elasticsearch/client/ml/GetDataFrameAnalyticsRequest.java create mode 100644 client/rest-high-level/src/main/java/org/elasticsearch/client/ml/GetDataFrameAnalyticsResponse.java create mode 100644 client/rest-high-level/src/main/java/org/elasticsearch/client/ml/GetDataFrameAnalyticsStatsRequest.java create mode 100644 client/rest-high-level/src/main/java/org/elasticsearch/client/ml/GetDataFrameAnalyticsStatsResponse.java create mode 100644 client/rest-high-level/src/main/java/org/elasticsearch/client/ml/PutDataFrameAnalyticsRequest.java create mode 100644 client/rest-high-level/src/main/java/org/elasticsearch/client/ml/PutDataFrameAnalyticsResponse.java create mode 100644 client/rest-high-level/src/main/java/org/elasticsearch/client/ml/StartDataFrameAnalyticsRequest.java create mode 100644 client/rest-high-level/src/main/java/org/elasticsearch/client/ml/StopDataFrameAnalyticsRequest.java create mode 100644 client/rest-high-level/src/main/java/org/elasticsearch/client/ml/StopDataFrameAnalyticsResponse.java create mode 100644 client/rest-high-level/src/main/java/org/elasticsearch/client/ml/dataframe/DataFrameAnalysis.java create mode 100644 client/rest-high-level/src/main/java/org/elasticsearch/client/ml/dataframe/DataFrameAnalyticsConfig.java create mode 100644 client/rest-high-level/src/main/java/org/elasticsearch/client/ml/dataframe/DataFrameAnalyticsDest.java create mode 100644 client/rest-high-level/src/main/java/org/elasticsearch/client/ml/dataframe/DataFrameAnalyticsSource.java create mode 100644 client/rest-high-level/src/main/java/org/elasticsearch/client/ml/dataframe/DataFrameAnalyticsState.java create mode 100644 client/rest-high-level/src/main/java/org/elasticsearch/client/ml/dataframe/DataFrameAnalyticsStats.java create mode 100644 client/rest-high-level/src/main/java/org/elasticsearch/client/ml/dataframe/MlDataFrameAnalysisNamedXContentProvider.java create mode 100644 client/rest-high-level/src/main/java/org/elasticsearch/client/ml/dataframe/OutlierDetection.java create mode 100644 client/rest-high-level/src/main/java/org/elasticsearch/client/ml/dataframe/QueryConfig.java create mode 100644 client/rest-high-level/src/main/java/org/elasticsearch/client/ml/dataframe/evaluation/Evaluation.java rename server/src/main/java/org/elasticsearch/action/admin/indices/alias/exists/AliasesExistAction.java => client/rest-high-level/src/main/java/org/elasticsearch/client/ml/dataframe/evaluation/EvaluationMetric.java (57%) create mode 100644 client/rest-high-level/src/main/java/org/elasticsearch/client/ml/dataframe/evaluation/MlEvaluationNamedXContentProvider.java create mode 100644 client/rest-high-level/src/main/java/org/elasticsearch/client/ml/dataframe/evaluation/softclassification/AbstractConfusionMatrixMetric.java create mode 100644 client/rest-high-level/src/main/java/org/elasticsearch/client/ml/dataframe/evaluation/softclassification/AucRocMetric.java create mode 100644 client/rest-high-level/src/main/java/org/elasticsearch/client/ml/dataframe/evaluation/softclassification/BinarySoftClassification.java create mode 100644 client/rest-high-level/src/main/java/org/elasticsearch/client/ml/dataframe/evaluation/softclassification/ConfusionMatrixMetric.java create mode 100644 client/rest-high-level/src/main/java/org/elasticsearch/client/ml/dataframe/evaluation/softclassification/PrecisionMetric.java create mode 100644 client/rest-high-level/src/main/java/org/elasticsearch/client/ml/dataframe/evaluation/softclassification/RecallMetric.java create mode 100644 client/rest-high-level/src/main/java/org/elasticsearch/client/ml/job/process/TimingStats.java create mode 100644 client/rest-high-level/src/main/java/org/elasticsearch/client/security/GetBuiltinPrivilegesRequest.java create mode 100644 client/rest-high-level/src/main/java/org/elasticsearch/client/security/GetBuiltinPrivilegesResponse.java create mode 100644 client/rest-high-level/src/test/java/org/elasticsearch/client/AbstractHlrcWriteableXContentTestCase.java create mode 100644 client/rest-high-level/src/test/java/org/elasticsearch/client/dataframe/transforms/NodeAttributesTests.java rename server/src/main/java/org/elasticsearch/action/admin/indices/exists/types/TypesExistsResponse.java => client/rest-high-level/src/test/java/org/elasticsearch/client/dataframe/transforms/TimeSyncConfigTests.java (52%) create mode 100644 client/rest-high-level/src/test/java/org/elasticsearch/client/dataframe/transforms/hlrc/TimeSyncConfigTests.java create mode 100644 client/rest-high-level/src/test/java/org/elasticsearch/client/indices/AnalyzeGlobalRequestTests.java create mode 100644 client/rest-high-level/src/test/java/org/elasticsearch/client/indices/AnalyzeIndexRequestTests.java create mode 100644 client/rest-high-level/src/test/java/org/elasticsearch/client/indices/AnalyzeRequestTests.java create mode 100644 client/rest-high-level/src/test/java/org/elasticsearch/client/indices/AnalyzeResponseTests.java create mode 100644 client/rest-high-level/src/test/java/org/elasticsearch/client/indices/ReloadAnalyzersResponseTests.java rename server/src/main/java/org/elasticsearch/action/admin/indices/alias/exists/AliasesExistResponse.java => client/rest-high-level/src/test/java/org/elasticsearch/client/ml/AucRocMetricAucRocPointTests.java (51%) create mode 100644 client/rest-high-level/src/test/java/org/elasticsearch/client/ml/AucRocMetricResultTests.java create mode 100644 client/rest-high-level/src/test/java/org/elasticsearch/client/ml/ConfusionMatrixMetricConfusionMatrixTests.java create mode 100644 client/rest-high-level/src/test/java/org/elasticsearch/client/ml/ConfusionMatrixMetricResultTests.java create mode 100644 client/rest-high-level/src/test/java/org/elasticsearch/client/ml/DeleteDataFrameAnalyticsRequestTests.java create mode 100644 client/rest-high-level/src/test/java/org/elasticsearch/client/ml/EvaluateDataFrameResponseTests.java create mode 100644 client/rest-high-level/src/test/java/org/elasticsearch/client/ml/GetDataFrameAnalyticsRequestTests.java create mode 100644 client/rest-high-level/src/test/java/org/elasticsearch/client/ml/GetDataFrameAnalyticsStatsRequestTests.java create mode 100644 client/rest-high-level/src/test/java/org/elasticsearch/client/ml/PrecisionMetricResultTests.java create mode 100644 client/rest-high-level/src/test/java/org/elasticsearch/client/ml/PutDataFrameAnalyticsRequestTests.java create mode 100644 client/rest-high-level/src/test/java/org/elasticsearch/client/ml/RecallMetricResultTests.java create mode 100644 client/rest-high-level/src/test/java/org/elasticsearch/client/ml/StartDataFrameAnalyticsRequestTests.java create mode 100644 client/rest-high-level/src/test/java/org/elasticsearch/client/ml/StopDataFrameAnalyticsRequestTests.java rename server/src/main/java/org/elasticsearch/action/admin/indices/exists/indices/IndicesExistsResponse.java => client/rest-high-level/src/test/java/org/elasticsearch/client/ml/StopDataFrameAnalyticsResponseTests.java (53%) create mode 100644 client/rest-high-level/src/test/java/org/elasticsearch/client/ml/dataframe/DataFrameAnalyticsConfigTests.java create mode 100644 client/rest-high-level/src/test/java/org/elasticsearch/client/ml/dataframe/DataFrameAnalyticsDestTests.java create mode 100644 client/rest-high-level/src/test/java/org/elasticsearch/client/ml/dataframe/DataFrameAnalyticsSourceTests.java create mode 100644 client/rest-high-level/src/test/java/org/elasticsearch/client/ml/dataframe/DataFrameAnalyticsStatsTests.java create mode 100644 client/rest-high-level/src/test/java/org/elasticsearch/client/ml/dataframe/OutlierDetectionTests.java create mode 100644 client/rest-high-level/src/test/java/org/elasticsearch/client/ml/dataframe/QueryConfigTests.java create mode 100644 client/rest-high-level/src/test/java/org/elasticsearch/client/ml/job/process/TimingStatsTests.java delete mode 100644 client/rest-high-level/src/test/java/org/elasticsearch/client/watcher/ExecuteWatchResponseTests.java delete mode 100644 client/transport/build.gradle delete mode 100644 client/transport/src/main/java/org/elasticsearch/transport/client/PreBuiltTransportClient.java delete mode 100644 client/transport/src/test/java/org/elasticsearch/transport/client/PreBuiltTransportClientTests.java create mode 100644 distribution/docker/docker-build-context/src/docker/config/log4j2.properties rename distribution/docker/{ => oss-docker-build-context}/src/docker/config/log4j2.properties (100%) delete mode 100644 docs/groovy-api/anatomy.asciidoc delete mode 100644 docs/groovy-api/client.asciidoc delete mode 100644 docs/groovy-api/delete.asciidoc delete mode 100644 docs/groovy-api/get.asciidoc delete mode 100644 docs/groovy-api/index.asciidoc delete mode 100644 docs/groovy-api/index_.asciidoc delete mode 100644 docs/groovy-api/search.asciidoc delete mode 100644 docs/java-api/admin/cluster/health.asciidoc delete mode 100644 docs/java-api/admin/cluster/index.asciidoc delete mode 100644 docs/java-api/admin/cluster/stored-scripts.asciidoc delete mode 100644 docs/java-api/admin/index.asciidoc delete mode 100644 docs/java-api/admin/indices/create-index.asciidoc delete mode 100644 docs/java-api/admin/indices/get-settings.asciidoc delete mode 100644 docs/java-api/admin/indices/index.asciidoc delete mode 100644 docs/java-api/admin/indices/put-mapping.asciidoc delete mode 100644 docs/java-api/admin/indices/refresh.asciidoc delete mode 100644 docs/java-api/admin/indices/update-settings.asciidoc delete mode 100644 docs/java-api/aggregations/bucket.asciidoc delete mode 100644 docs/java-api/aggregations/bucket/children-aggregation.asciidoc delete mode 100644 docs/java-api/aggregations/bucket/datehistogram-aggregation.asciidoc delete mode 100644 docs/java-api/aggregations/bucket/daterange-aggregation.asciidoc delete mode 100644 docs/java-api/aggregations/bucket/filter-aggregation.asciidoc delete mode 100644 docs/java-api/aggregations/bucket/filters-aggregation.asciidoc delete mode 100644 docs/java-api/aggregations/bucket/geodistance-aggregation.asciidoc delete mode 100644 docs/java-api/aggregations/bucket/geohashgrid-aggregation.asciidoc delete mode 100644 docs/java-api/aggregations/bucket/global-aggregation.asciidoc delete mode 100644 docs/java-api/aggregations/bucket/histogram-aggregation.asciidoc delete mode 100644 docs/java-api/aggregations/bucket/iprange-aggregation.asciidoc delete mode 100644 docs/java-api/aggregations/bucket/missing-aggregation.asciidoc delete mode 100644 docs/java-api/aggregations/bucket/nested-aggregation.asciidoc delete mode 100644 docs/java-api/aggregations/bucket/range-aggregation.asciidoc delete mode 100644 docs/java-api/aggregations/bucket/reverse-nested-aggregation.asciidoc delete mode 100644 docs/java-api/aggregations/bucket/significantterms-aggregation.asciidoc delete mode 100644 docs/java-api/aggregations/bucket/terms-aggregation.asciidoc delete mode 100644 docs/java-api/aggregations/metrics.asciidoc delete mode 100644 docs/java-api/aggregations/metrics/avg-aggregation.asciidoc delete mode 100644 docs/java-api/aggregations/metrics/cardinality-aggregation.asciidoc delete mode 100644 docs/java-api/aggregations/metrics/extendedstats-aggregation.asciidoc delete mode 100644 docs/java-api/aggregations/metrics/geobounds-aggregation.asciidoc delete mode 100644 docs/java-api/aggregations/metrics/max-aggregation.asciidoc delete mode 100644 docs/java-api/aggregations/metrics/min-aggregation.asciidoc delete mode 100644 docs/java-api/aggregations/metrics/percentile-aggregation.asciidoc delete mode 100644 docs/java-api/aggregations/metrics/percentile-rank-aggregation.asciidoc delete mode 100644 docs/java-api/aggregations/metrics/scripted-metric-aggregation.asciidoc delete mode 100644 docs/java-api/aggregations/metrics/stats-aggregation.asciidoc delete mode 100644 docs/java-api/aggregations/metrics/sum-aggregation.asciidoc delete mode 100644 docs/java-api/aggregations/metrics/tophits-aggregation.asciidoc delete mode 100644 docs/java-api/aggregations/metrics/valuecount-aggregation.asciidoc delete mode 100644 docs/java-api/aggs.asciidoc delete mode 100644 docs/java-api/client.asciidoc delete mode 100644 docs/java-api/docs.asciidoc delete mode 100644 docs/java-api/docs/bulk.asciidoc delete mode 100644 docs/java-api/docs/delete.asciidoc delete mode 100644 docs/java-api/docs/get.asciidoc delete mode 100644 docs/java-api/docs/index_.asciidoc delete mode 100644 docs/java-api/docs/multi-get.asciidoc delete mode 100644 docs/java-api/docs/reindex.asciidoc delete mode 100644 docs/java-api/docs/update-by-query.asciidoc delete mode 100644 docs/java-api/docs/update.asciidoc delete mode 100644 docs/java-api/index.asciidoc delete mode 100644 docs/java-api/query-dsl.asciidoc delete mode 100644 docs/java-api/query-dsl/bool-query.asciidoc delete mode 100644 docs/java-api/query-dsl/boosting-query.asciidoc delete mode 100644 docs/java-api/query-dsl/common-terms-query.asciidoc delete mode 100644 docs/java-api/query-dsl/compound-queries.asciidoc delete mode 100644 docs/java-api/query-dsl/constant-score-query.asciidoc delete mode 100644 docs/java-api/query-dsl/dis-max-query.asciidoc delete mode 100644 docs/java-api/query-dsl/exists-query.asciidoc delete mode 100644 docs/java-api/query-dsl/full-text-queries.asciidoc delete mode 100644 docs/java-api/query-dsl/function-score-query.asciidoc delete mode 100644 docs/java-api/query-dsl/fuzzy-query.asciidoc delete mode 100644 docs/java-api/query-dsl/geo-bounding-box-query.asciidoc delete mode 100644 docs/java-api/query-dsl/geo-distance-query.asciidoc delete mode 100644 docs/java-api/query-dsl/geo-polygon-query.asciidoc delete mode 100644 docs/java-api/query-dsl/geo-queries.asciidoc delete mode 100644 docs/java-api/query-dsl/geo-shape-query.asciidoc delete mode 100644 docs/java-api/query-dsl/has-child-query.asciidoc delete mode 100644 docs/java-api/query-dsl/has-parent-query.asciidoc delete mode 100644 docs/java-api/query-dsl/ids-query.asciidoc delete mode 100644 docs/java-api/query-dsl/joining-queries.asciidoc delete mode 100644 docs/java-api/query-dsl/match-all-query.asciidoc delete mode 100644 docs/java-api/query-dsl/match-query.asciidoc delete mode 100644 docs/java-api/query-dsl/mlt-query.asciidoc delete mode 100644 docs/java-api/query-dsl/multi-match-query.asciidoc delete mode 100644 docs/java-api/query-dsl/nested-query.asciidoc delete mode 100644 docs/java-api/query-dsl/percolate-query.asciidoc delete mode 100644 docs/java-api/query-dsl/prefix-query.asciidoc delete mode 100644 docs/java-api/query-dsl/query-string-query.asciidoc delete mode 100644 docs/java-api/query-dsl/range-query.asciidoc delete mode 100644 docs/java-api/query-dsl/regexp-query.asciidoc delete mode 100644 docs/java-api/query-dsl/script-query.asciidoc delete mode 100644 docs/java-api/query-dsl/simple-query-string-query.asciidoc delete mode 100644 docs/java-api/query-dsl/span-containing-query.asciidoc delete mode 100644 docs/java-api/query-dsl/span-first-query.asciidoc delete mode 100644 docs/java-api/query-dsl/span-multi-term-query.asciidoc delete mode 100644 docs/java-api/query-dsl/span-near-query.asciidoc delete mode 100644 docs/java-api/query-dsl/span-not-query.asciidoc delete mode 100644 docs/java-api/query-dsl/span-or-query.asciidoc delete mode 100644 docs/java-api/query-dsl/span-queries.asciidoc delete mode 100644 docs/java-api/query-dsl/span-term-query.asciidoc delete mode 100644 docs/java-api/query-dsl/span-within-query.asciidoc delete mode 100644 docs/java-api/query-dsl/special-queries.asciidoc delete mode 100644 docs/java-api/query-dsl/term-level-queries.asciidoc delete mode 100644 docs/java-api/query-dsl/term-query.asciidoc delete mode 100644 docs/java-api/query-dsl/terms-query.asciidoc delete mode 100644 docs/java-api/query-dsl/wildcard-query.asciidoc delete mode 100644 docs/java-api/query-dsl/wrapper-query.asciidoc delete mode 100644 docs/java-api/search.asciidoc create mode 100644 docs/java-rest/high-level/indices/reload_analyzers.asciidoc create mode 100644 docs/java-rest/high-level/ml/delete-data-frame-analytics.asciidoc create mode 100644 docs/java-rest/high-level/ml/evaluate-data-frame.asciidoc create mode 100644 docs/java-rest/high-level/ml/get-data-frame-analytics-stats.asciidoc create mode 100644 docs/java-rest/high-level/ml/get-data-frame-analytics.asciidoc create mode 100644 docs/java-rest/high-level/ml/put-data-frame-analytics.asciidoc create mode 100644 docs/java-rest/high-level/ml/start-data-frame-analytics.asciidoc create mode 100644 docs/java-rest/high-level/ml/stop-data-frame-analytics.asciidoc create mode 100644 docs/java-rest/high-level/security/get-builtin-privileges.asciidoc create mode 100644 docs/painless/painless-guide/painless-datetime.asciidoc create mode 100644 docs/painless/redirects.asciidoc create mode 100644 docs/reference/administering.asciidoc create mode 100644 docs/reference/aggregations/bucket/rare-terms-aggregation.asciidoc create mode 100644 docs/reference/analysis/tokenizers/pathhierarchy-tokenizer-examples.asciidoc create mode 100644 docs/reference/data-frames/apis/pivotresource.asciidoc create mode 100644 docs/reference/images/rare_terms/accuracy_0001.png create mode 100644 docs/reference/images/rare_terms/accuracy_001.png create mode 100644 docs/reference/images/rare_terms/accuracy_01.png create mode 100644 docs/reference/images/rare_terms/memory.png create mode 100644 docs/reference/indices/apis/reload-analyzers.asciidoc create mode 100644 docs/reference/intro.asciidoc create mode 100644 docs/reference/mapping/types/flattened.asciidoc create mode 100644 docs/reference/migration/migrate_8_0/rollup.asciidoc delete mode 100644 docs/reference/modules/memcached.asciidoc delete mode 100644 docs/reference/modules/ml-node.asciidoc delete mode 100644 docs/reference/modules/thrift.asciidoc create mode 100644 docs/reference/monitoring/configuring-filebeat.asciidoc delete mode 100644 docs/reference/query-dsl/common-terms-query.asciidoc rename docs/reference/{modules => }/scripting.asciidoc (86%) rename docs/reference/{modules => }/scripting/engine.asciidoc (97%) rename docs/reference/{modules => }/scripting/expression.asciidoc (99%) rename docs/reference/{modules => }/scripting/fields.asciidoc (98%) rename docs/reference/{modules => }/scripting/painless.asciidoc (97%) rename docs/reference/{modules => }/scripting/security.asciidoc (99%) rename docs/reference/{modules => }/scripting/using.asciidoc (97%) delete mode 100644 docs/reference/security/securing-communications/separating-node-client-traffic.asciidoc create mode 100644 docs/reference/setup/install/brew.asciidoc delete mode 100644 libs/geo/src/main/java/org/elasticsearch/geo/geometry/GeometryUtils.java create mode 100644 libs/geo/src/main/java/org/elasticsearch/geo/utils/GeographyValidator.java create mode 100644 libs/geo/src/main/java/org/elasticsearch/geo/utils/GeometryValidator.java create mode 100644 libs/geo/src/test/java/org/elasticsearch/geo/geometry/GeometryValidatorTests.java delete mode 100644 libs/grok/licenses/jcodings-1.0.12.jar.sha1 create mode 100644 libs/grok/licenses/jcodings-1.0.44.jar.sha1 create mode 100644 libs/nio/src/main/java/org/elasticsearch/nio/DelegatingHandler.java rename libs/nio/src/main/java/org/elasticsearch/nio/{ReadWriteHandler.java => NioChannelHandler.java} (90%) delete mode 100644 modules/analysis-common/src/main/java/org/elasticsearch/analysis/common/StandardHtmlStripAnalyzer.java create mode 100644 modules/analysis-common/src/test/java/org/elasticsearch/analysis/common/EdgeNGramTokenizerTests.java create mode 100644 modules/analysis-common/src/test/java/org/elasticsearch/analysis/common/ElisionFilterFactoryTests.java rename server/src/main/java/org/elasticsearch/action/admin/indices/exists/types/TypesExistsAction.java => modules/lang-painless/spi/src/main/java/org/elasticsearch/painless/spi/annotation/DeprecatedAnnotation.java (63%) create mode 100644 modules/lang-painless/spi/src/main/java/org/elasticsearch/painless/spi/annotation/DeprecatedAnnotationParser.java rename modules/{mapper-extras/src/test/java/org/elasticsearch/index/mapper/SparseVectorFieldTypeTests.java => lang-painless/spi/src/main/java/org/elasticsearch/painless/spi/annotation/NoImportAnnotation.java} (75%) rename server/src/main/java/org/elasticsearch/action/admin/indices/exists/indices/IndicesExistsAction.java => modules/lang-painless/spi/src/main/java/org/elasticsearch/painless/spi/annotation/NoImportAnnotationParser.java (59%) create mode 100644 modules/lang-painless/spi/src/main/java/org/elasticsearch/painless/spi/annotation/WhitelistAnnotationParser.java create mode 100644 modules/lang-painless/src/test/java/org/elasticsearch/painless/AnnotationTestObject.java create mode 100644 modules/lang-painless/src/test/java/org/elasticsearch/painless/DateTimeTests.java create mode 100644 modules/lang-painless/src/test/java/org/elasticsearch/painless/GetByPathAugmentationTests.java delete mode 100644 modules/lang-painless/src/test/java/org/elasticsearch/painless/PainlessDocGenerator.java create mode 100644 modules/lang-painless/src/test/java/org/elasticsearch/painless/WhitelistLoaderTests.java create mode 100644 modules/lang-painless/src/test/resources/org/elasticsearch/painless/spi/org.elasticsearch.painless.annotation rename {rest-api-spec/src/main => modules/lang-painless/src/test}/resources/rest-api-spec/api/scripts_painless_context.json (88%) create mode 100644 modules/lang-painless/src/test/resources/rest-api-spec/test/painless/100_terms_agg.yml delete mode 100644 modules/mapper-extras/src/main/resources/META-INF/services/org.elasticsearch.painless.spi.PainlessExtension delete mode 100644 modules/mapper-extras/src/main/resources/org/elasticsearch/index/query/docvalues_whitelist.txt delete mode 100644 modules/mapper-extras/src/test/java/org/elasticsearch/index/mapper/DenseVectorFieldMapperTests.java create mode 100644 modules/reindex/src/test/java/org/elasticsearch/index/reindex/remote/RemoteResponseParsersTests.java delete mode 100644 modules/transport-netty4/src/main/java/org/elasticsearch/http/netty4/cors/Netty4CorsConfig.java delete mode 100644 modules/transport-netty4/src/main/java/org/elasticsearch/http/netty4/cors/Netty4CorsConfigBuilder.java delete mode 100644 modules/transport-netty4/src/main/java/org/elasticsearch/transport/netty4/Netty4InternalESLogger.java create mode 100644 plugins/analysis-smartcn/src/main/java/org/elasticsearch/index/analysis/SmartChineseStopTokenFilterFactory.java create mode 100644 plugins/analysis-stempel/src/main/java/org/elasticsearch/index/analysis/pl/PolishStopTokenFilterFactory.java rename {server => plugins/discovery-azure-classic}/licenses/log4j-1.2-api-2.11.1.jar.sha1 (100%) rename {server => plugins/discovery-azure-classic}/licenses/log4j-LICENSE.txt (100%) rename {server => plugins/discovery-azure-classic}/licenses/log4j-NOTICE.txt (100%) delete mode 100644 plugins/discovery-ec2/licenses/aws-java-sdk-core-1.11.505.jar.sha1 create mode 100644 plugins/discovery-ec2/licenses/aws-java-sdk-core-1.11.562.jar.sha1 delete mode 100644 plugins/discovery-ec2/licenses/aws-java-sdk-ec2-1.11.505.jar.sha1 create mode 100644 plugins/discovery-ec2/licenses/aws-java-sdk-ec2-1.11.562.jar.sha1 create mode 100644 plugins/discovery-ec2/licenses/log4j-1.2-api-2.11.1.jar.sha1 create mode 100644 plugins/discovery-ec2/licenses/log4j-LICENSE.txt create mode 100644 plugins/discovery-ec2/licenses/log4j-NOTICE.txt create mode 100644 plugins/discovery-gce/licenses/log4j-1.2-api-2.11.1.jar.sha1 create mode 100644 plugins/discovery-gce/licenses/log4j-LICENSE.txt create mode 100644 plugins/discovery-gce/licenses/log4j-NOTICE.txt rename server/src/main/java/org/elasticsearch/action/admin/indices/alias/exists/AliasesExistRequestBuilder.java => plugins/examples/painless-whitelist/src/main/java/org/elasticsearch/example/painlesswhitelist/ExamplePainlessAnnotation.java (62%) create mode 100644 plugins/examples/painless-whitelist/src/main/java/org/elasticsearch/example/painlesswhitelist/ExampleWhitelistAnnotationParser.java create mode 100644 plugins/repository-azure/src/test/java/org/elasticsearch/repositories/azure/AzureStorageCleanupThirdPartyTests.java delete mode 100644 plugins/repository-gcs/licenses/api-common-1.7.0.jar.sha1 create mode 100644 plugins/repository-gcs/licenses/api-common-1.8.1.jar.sha1 delete mode 100644 plugins/repository-gcs/licenses/gax-1.30.0.jar.sha1 create mode 100644 plugins/repository-gcs/licenses/gax-1.45.0.jar.sha1 delete mode 100644 plugins/repository-gcs/licenses/gax-httpjson-0.47.0.jar.sha1 create mode 100644 plugins/repository-gcs/licenses/gax-httpjson-0.62.0.jar.sha1 delete mode 100644 plugins/repository-gcs/licenses/google-api-client-1.24.1.jar.sha1 create mode 100644 plugins/repository-gcs/licenses/google-api-client-1.28.0.jar.sha1 delete mode 100644 plugins/repository-gcs/licenses/google-api-services-storage-v1-rev135-1.24.1.jar.sha1 create mode 100644 plugins/repository-gcs/licenses/google-api-services-storage-v1-rev20190426-1.28.0.jar.sha1 delete mode 100644 plugins/repository-gcs/licenses/google-auth-library-credentials-0.10.0.jar.sha1 create mode 100644 plugins/repository-gcs/licenses/google-auth-library-credentials-0.16.1.jar.sha1 delete mode 100644 plugins/repository-gcs/licenses/google-auth-library-oauth2-http-0.10.0.jar.sha1 create mode 100644 plugins/repository-gcs/licenses/google-auth-library-oauth2-http-0.16.1.jar.sha1 delete mode 100644 plugins/repository-gcs/licenses/google-cloud-core-1.59.0.jar.sha1 create mode 100644 plugins/repository-gcs/licenses/google-cloud-core-1.77.0.jar.sha1 delete mode 100644 plugins/repository-gcs/licenses/google-cloud-core-http-1.59.0.jar.sha1 create mode 100644 plugins/repository-gcs/licenses/google-cloud-core-http-1.77.0.jar.sha1 delete mode 100644 plugins/repository-gcs/licenses/google-cloud-storage-1.59.0.jar.sha1 create mode 100644 plugins/repository-gcs/licenses/google-cloud-storage-1.77.0.jar.sha1 delete mode 100644 plugins/repository-gcs/licenses/google-http-client-1.24.1.jar.sha1 create mode 100644 plugins/repository-gcs/licenses/google-http-client-1.30.1.jar.sha1 delete mode 100644 plugins/repository-gcs/licenses/google-http-client-appengine-1.24.1.jar.sha1 create mode 100644 plugins/repository-gcs/licenses/google-http-client-appengine-1.29.2.jar.sha1 delete mode 100644 plugins/repository-gcs/licenses/google-http-client-jackson-1.24.1.jar.sha1 delete mode 100644 plugins/repository-gcs/licenses/google-http-client-jackson2-1.24.1.jar.sha1 create mode 100644 plugins/repository-gcs/licenses/google-http-client-jackson2-1.29.2.jar.sha1 delete mode 100644 plugins/repository-gcs/licenses/google-oauth-client-1.24.1.jar.sha1 create mode 100644 plugins/repository-gcs/licenses/google-oauth-client-1.28.0.jar.sha1 delete mode 100644 plugins/repository-gcs/licenses/guava-20.0.jar.sha1 create mode 100644 plugins/repository-gcs/licenses/guava-26.0-jre.jar.sha1 delete mode 100644 plugins/repository-gcs/licenses/jackson-LICENSE delete mode 100644 plugins/repository-gcs/licenses/jackson-NOTICE delete mode 100644 plugins/repository-gcs/licenses/jackson-core-asl-1.9.11.jar.sha1 create mode 100644 plugins/repository-gcs/licenses/log4j-1.2-api-2.11.1.jar.sha1 create mode 100644 plugins/repository-gcs/licenses/log4j-LICENSE.txt create mode 100644 plugins/repository-gcs/licenses/log4j-NOTICE.txt delete mode 100644 plugins/repository-gcs/licenses/opencensus-api-0.15.0.jar.sha1 create mode 100644 plugins/repository-gcs/licenses/opencensus-api-0.18.0.jar.sha1 delete mode 100644 plugins/repository-gcs/licenses/opencensus-contrib-http-util-0.15.0.jar.sha1 create mode 100644 plugins/repository-gcs/licenses/opencensus-contrib-http-util-0.18.0.jar.sha1 delete mode 100644 plugins/repository-gcs/licenses/proto-google-common-protos-1.12.0.jar.sha1 create mode 100644 plugins/repository-gcs/licenses/proto-google-common-protos-1.16.0.jar.sha1 delete mode 100644 plugins/repository-gcs/licenses/protobuf-java-3.6.0.jar.sha1 create mode 100644 plugins/repository-gcs/licenses/protobuf-java-3.7.1.jar.sha1 delete mode 100644 plugins/repository-gcs/licenses/protobuf-java-util-3.6.0.jar.sha1 create mode 100644 plugins/repository-gcs/licenses/protobuf-java-util-3.7.1.jar.sha1 create mode 100644 plugins/repository-gcs/src/test/java/org/elasticsearch/repositories/gcs/GoogleCloudStorageThirdPartyTests.java create mode 100644 plugins/repository-hdfs/licenses/log4j-1.2-api-2.11.1.jar.sha1 create mode 100644 plugins/repository-hdfs/licenses/log4j-LICENSE.txt create mode 100644 plugins/repository-hdfs/licenses/log4j-NOTICE.txt create mode 100644 plugins/repository-hdfs/src/test/java/org/elasticsearch/repositories/hdfs/HdfsRepositoryTests.java delete mode 100644 plugins/repository-s3/licenses/aws-java-sdk-core-1.11.505.jar.sha1 create mode 100644 plugins/repository-s3/licenses/aws-java-sdk-core-1.11.562.jar.sha1 delete mode 100644 plugins/repository-s3/licenses/aws-java-sdk-kms-1.11.505.jar.sha1 delete mode 100644 plugins/repository-s3/licenses/aws-java-sdk-s3-1.11.505.jar.sha1 create mode 100644 plugins/repository-s3/licenses/aws-java-sdk-s3-1.11.562.jar.sha1 delete mode 100644 plugins/repository-s3/licenses/jmespath-java-1.11.505.jar.sha1 create mode 100644 plugins/repository-s3/licenses/jmespath-java-1.11.562.jar.sha1 create mode 100644 plugins/repository-s3/licenses/log4j-1.2-api-2.11.1.jar.sha1 create mode 100644 plugins/repository-s3/licenses/log4j-LICENSE.txt create mode 100644 plugins/repository-s3/licenses/log4j-NOTICE.txt create mode 100644 plugins/repository-s3/src/test/java/org/elasticsearch/repositories/s3/S3RepositoryThirdPartyTests.java delete mode 100644 plugins/transport-nio/src/main/java/org/elasticsearch/http/nio/cors/NioCorsConfig.java delete mode 100644 plugins/transport-nio/src/main/java/org/elasticsearch/http/nio/cors/NioCorsConfigBuilder.java delete mode 100644 qa/smoke-test-client/build.gradle delete mode 100644 qa/smoke-test-client/src/test/java/org/elasticsearch/smoketest/ESSmokeClientTestCase.java delete mode 100644 qa/smoke-test-client/src/test/java/org/elasticsearch/smoketest/SmokeTestClientIT.java delete mode 100644 rest-api-spec/src/main/resources/rest-api-spec/test/count/11_basic_with_types.yml delete mode 100644 rest-api-spec/src/main/resources/rest-api-spec/test/msearch/12_basic_with_types.yml create mode 100644 rest-api-spec/src/main/resources/rest-api-spec/test/search.aggregation/280_rare_terms.yml delete mode 100644 server/licenses/hppc-0.7.1.jar.sha1 create mode 100644 server/licenses/hppc-0.8.1.jar.sha1 delete mode 100644 server/src/main/java/org/apache/lucene/queries/ExtendedCommonTermsQuery.java create mode 100644 server/src/main/java/org/apache/lucene/util/CombinedBitSet.java rename server/src/main/java/org/elasticsearch/action/{Action.java => ActionType.java} (75%) create mode 100644 server/src/main/java/org/elasticsearch/action/RequestValidators.java create mode 100644 server/src/main/java/org/elasticsearch/action/StreamableResponseActionType.java delete mode 100644 server/src/main/java/org/elasticsearch/action/admin/indices/alias/exists/TransportAliasesExistAction.java delete mode 100644 server/src/main/java/org/elasticsearch/action/admin/indices/analyze/AnalyzeRequest.java delete mode 100644 server/src/main/java/org/elasticsearch/action/admin/indices/analyze/AnalyzeResponse.java delete mode 100644 server/src/main/java/org/elasticsearch/action/admin/indices/analyze/DetailAnalyzeResponse.java delete mode 100644 server/src/main/java/org/elasticsearch/action/admin/indices/exists/indices/IndicesExistsRequest.java delete mode 100644 server/src/main/java/org/elasticsearch/action/admin/indices/exists/indices/IndicesExistsRequestBuilder.java delete mode 100644 server/src/main/java/org/elasticsearch/action/admin/indices/exists/indices/TransportIndicesExistsAction.java delete mode 100644 server/src/main/java/org/elasticsearch/action/admin/indices/exists/types/TransportTypesExistsAction.java delete mode 100644 server/src/main/java/org/elasticsearch/action/admin/indices/exists/types/TypesExistsRequest.java delete mode 100644 server/src/main/java/org/elasticsearch/action/admin/indices/exists/types/TypesExistsRequestBuilder.java delete mode 100644 server/src/main/java/org/elasticsearch/action/admin/indices/mapping/put/MappingRequestValidator.java delete mode 100644 server/src/main/java/org/elasticsearch/client/transport/TransportClient.java delete mode 100644 server/src/main/java/org/elasticsearch/client/transport/TransportClientNodesService.java delete mode 100644 server/src/main/java/org/elasticsearch/client/transport/TransportProxyClient.java create mode 100644 server/src/main/java/org/elasticsearch/cluster/coordination/ElectionStrategy.java create mode 100644 server/src/main/java/org/elasticsearch/cluster/node/DiscoveryNodeRole.java create mode 100644 server/src/main/java/org/elasticsearch/cluster/routing/BatchedRerouteService.java create mode 100644 server/src/main/java/org/elasticsearch/cluster/routing/LazilyInitializedRerouteService.java create mode 100644 server/src/main/java/org/elasticsearch/cluster/routing/RerouteService.java delete mode 100644 server/src/main/java/org/elasticsearch/cluster/routing/RoutingService.java rename buildSrc/src/main/groovy/org/elasticsearch/gradle/EmptyDirTask.groovy => server/src/main/java/org/elasticsearch/common/TriConsumer.java (59%) rename server/src/main/java/org/{apache/lucene/search/XIndexSearcher.java => elasticsearch/common/geo/GeometryFormat.java} (52%) create mode 100644 server/src/main/java/org/elasticsearch/common/settings/ConsistentSettingsService.java create mode 100644 server/src/main/java/org/elasticsearch/common/util/CuckooFilter.java create mode 100644 server/src/main/java/org/elasticsearch/common/util/SetBackedScalingCuckooFilter.java create mode 100644 server/src/main/java/org/elasticsearch/http/CorsHandler.java create mode 100644 server/src/main/java/org/elasticsearch/index/analysis/AnalyzerComponents.java rename modules/mapper-extras/src/test/java/org/elasticsearch/index/mapper/DenseVectorFieldTypeTests.java => server/src/main/java/org/elasticsearch/index/analysis/AnalyzerComponentsProvider.java (76%) create mode 100644 server/src/main/java/org/elasticsearch/index/analysis/NameOrDefinition.java create mode 100644 server/src/main/java/org/elasticsearch/index/analysis/ReloadableCustomAnalyzer.java create mode 100644 server/src/main/java/org/elasticsearch/index/engine/CombinedDocValues.java create mode 100644 server/src/main/java/org/elasticsearch/index/engine/PrunePostingsMergePolicy.java create mode 100644 server/src/main/java/org/elasticsearch/index/fieldvisitor/IdOnlyFieldVisitor.java create mode 100644 server/src/main/java/org/elasticsearch/index/mapper/DynamicKeyFieldMapper.java delete mode 100644 server/src/main/java/org/elasticsearch/index/query/CommonTermsQueryBuilder.java delete mode 100644 server/src/main/java/org/elasticsearch/index/shard/IndexSearcherWrapper.java create mode 100644 server/src/main/java/org/elasticsearch/search/aggregations/bucket/terms/AbstractRareTermsAggregator.java create mode 100644 server/src/main/java/org/elasticsearch/search/aggregations/bucket/terms/InternalMappedRareTerms.java create mode 100644 server/src/main/java/org/elasticsearch/search/aggregations/bucket/terms/InternalRareTerms.java create mode 100644 server/src/main/java/org/elasticsearch/search/aggregations/bucket/terms/LongRareTerms.java create mode 100644 server/src/main/java/org/elasticsearch/search/aggregations/bucket/terms/LongRareTermsAggregator.java create mode 100644 server/src/main/java/org/elasticsearch/search/aggregations/bucket/terms/RareTerms.java create mode 100644 server/src/main/java/org/elasticsearch/search/aggregations/bucket/terms/RareTermsAggregationBuilder.java create mode 100644 server/src/main/java/org/elasticsearch/search/aggregations/bucket/terms/RareTermsAggregatorFactory.java create mode 100644 server/src/main/java/org/elasticsearch/search/aggregations/bucket/terms/StringRareTerms.java create mode 100644 server/src/main/java/org/elasticsearch/search/aggregations/bucket/terms/StringRareTermsAggregator.java create mode 100644 server/src/main/java/org/elasticsearch/search/aggregations/bucket/terms/UnmappedRareTerms.java create mode 100644 server/src/test/java/org/apache/lucene/util/CombinedBitSetTests.java create mode 100644 server/src/test/java/org/elasticsearch/action/RequestValidatorsTests.java create mode 100644 server/src/test/java/org/elasticsearch/action/admin/indices/alias/ValidateIndicesAliasesRequestIT.java delete mode 100644 server/src/test/java/org/elasticsearch/action/admin/indices/exists/IndicesExistsIT.java create mode 100644 server/src/test/java/org/elasticsearch/action/support/master/ShardsAcknowledgedResponseTests.java create mode 100644 server/src/test/java/org/elasticsearch/cluster/node/DiscoveryNodeRoleIT.java create mode 100644 server/src/test/java/org/elasticsearch/cluster/node/DiscoveryNodeRoleTests.java create mode 100644 server/src/test/java/org/elasticsearch/cluster/routing/BatchedRerouteServiceTests.java create mode 100644 server/src/test/java/org/elasticsearch/cluster/routing/allocation/RetryFailedAllocationTests.java create mode 100644 server/src/test/java/org/elasticsearch/common/settings/ConsistentSettingsIT.java create mode 100644 server/src/test/java/org/elasticsearch/common/settings/ConsistentSettingsServiceTests.java create mode 100644 server/src/test/java/org/elasticsearch/common/util/CuckooFilterTests.java create mode 100644 server/src/test/java/org/elasticsearch/common/util/SetBackedScalingCuckooFilterTests.java create mode 100644 server/src/test/java/org/elasticsearch/discovery/DiskDisruptionIT.java create mode 100644 server/src/test/java/org/elasticsearch/http/CorsHandlerTests.java create mode 100644 server/src/test/java/org/elasticsearch/index/analysis/ReloadableCustomAnalyzerTests.java create mode 100644 server/src/test/java/org/elasticsearch/index/engine/PrunePostingsMergePolicyTests.java delete mode 100644 server/src/test/java/org/elasticsearch/index/query/CommonTermsQueryBuilderTests.java delete mode 100644 server/src/test/java/org/elasticsearch/index/query/CommonTermsQueryParserTests.java delete mode 100644 server/src/test/java/org/elasticsearch/index/query/FullTextQueryTestCase.java rename server/src/test/java/org/elasticsearch/index/shard/{IndexSearcherWrapperTests.java => IndexReaderWrapperTests.java} (65%) create mode 100644 server/src/test/java/org/elasticsearch/indices/cluster/IndicesClusterStateServiceTests.java delete mode 100644 server/src/test/java/org/elasticsearch/indices/exists/indices/IndicesExistsIT.java delete mode 100644 server/src/test/java/org/elasticsearch/indices/exists/types/TypesExistsIT.java delete mode 100644 server/src/test/java/org/elasticsearch/rest/action/search/RestCountActionTests.java create mode 100644 server/src/test/java/org/elasticsearch/search/aggregations/bucket/RareTermsTests.java create mode 100644 server/src/test/java/org/elasticsearch/search/aggregations/bucket/histogram/AutoDateHistogramAggregationBuilderTests.java create mode 100644 server/src/test/java/org/elasticsearch/search/aggregations/bucket/terms/RareTermsAggregatorTests.java create mode 100644 server/src/test/java/org/elasticsearch/search/aggregations/support/ValuesSourceTypeTests.java create mode 100644 server/src/test/java/org/elasticsearch/search/internal/ContextIndexSearcherTests.java create mode 100644 server/src/test/java/org/elasticsearch/snapshots/SnapshotInfoTests.java delete mode 100644 server/src/test/resources/org/elasticsearch/index/query/commonTerms-query1.json delete mode 100644 server/src/test/resources/org/elasticsearch/index/query/commonTerms-query2.json delete mode 100644 server/src/test/resources/org/elasticsearch/index/query/commonTerms-query3.json create mode 100644 server/src/test/resources/org/elasticsearch/index/translog/translog-v2.binary create mode 100644 test/framework/src/main/java/org/elasticsearch/cluster/coordination/AbstractCoordinatorTestCase.java create mode 100644 test/framework/src/main/java/org/elasticsearch/cluster/coordination/CoordinationStateTestCluster.java rename {server/src/test => test/framework/src/main}/java/org/elasticsearch/cluster/coordination/LinearizabilityChecker.java (100%) rename {server/src/test => test/framework/src/main}/java/org/elasticsearch/gateway/MockGatewayMetaState.java (95%) rename {server/src/test => test/framework/src/main}/java/org/elasticsearch/indices/cluster/FakeThreadPoolMasterService.java (100%) create mode 100644 test/framework/src/main/java/org/elasticsearch/repositories/AbstractThirdPartyRepositoryTestCase.java create mode 100644 test/framework/src/main/java/org/elasticsearch/repositories/blobstore/BlobStoreTestUtil.java rename {server/src/test/java/org/elasticsearch/action/support/broadcast => test/framework/src/main/java/org/elasticsearch/test}/AbstractBroadcastResponseTestCase.java (98%) create mode 100644 test/framework/src/main/java/org/elasticsearch/test/hamcrest/OptionalMatchers.java delete mode 100644 test/framework/src/main/java/org/elasticsearch/transport/MockTransportClient.java create mode 100644 x-pack/docs/en/rest-api/security/get-builtin-privileges.asciidoc rename x-pack/plugin/ccr/src/test/java/org/elasticsearch/xpack/ccr/{CCRFeatureSetTests.java => CCRInfoTransportActionTests.java} (75%) create mode 100644 x-pack/plugin/ccr/src/test/java/org/elasticsearch/xpack/ccr/CcrAliasesIT.java create mode 100644 x-pack/plugin/core/licenses/log4j-1.2-api-2.11.1.jar.sha1 create mode 100644 x-pack/plugin/core/licenses/log4j-LICENSE.txt create mode 100644 x-pack/plugin/core/licenses/log4j-NOTICE.txt rename x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/{core/ccr/CCRFeatureSet.java => ccr/CCRInfoTransportAction.java} (65%) create mode 100644 x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/ccr/CCRUsageTransportAction.java delete mode 100644 x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/EmptyXPackFeatureSet.java delete mode 100644 x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/XPackClient.java create mode 100644 x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/action/ReloadAnalyzerAction.java create mode 100644 x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/action/ReloadAnalyzersRequest.java create mode 100644 x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/action/ReloadAnalyzersResponse.java create mode 100644 x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/action/TransportReloadAnalyzersAction.java create mode 100644 x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/action/XPackInfoFeatureAction.java create mode 100644 x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/action/XPackInfoFeatureResponse.java create mode 100644 x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/action/XPackInfoFeatureTransportAction.java create mode 100644 x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/action/XPackUsageFeatureAction.java create mode 100644 x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/action/XPackUsageFeatureResponse.java create mode 100644 x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/action/XPackUsageFeatureTransportAction.java delete mode 100644 x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/ccr/client/CcrClient.java rename x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/{ml/utils => common}/time/TimeUtils.java (87%) create mode 100644 x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/dataframe/DataFrameNamedXContentProvider.java create mode 100644 x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/dataframe/transforms/NodeAttributes.java create mode 100644 x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/dataframe/transforms/SyncConfig.java create mode 100644 x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/dataframe/transforms/TimeSyncConfig.java create mode 100644 x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/dataframe/utils/DataFrameStrings.java delete mode 100644 x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/dataframe/utils/TimeUtils.java create mode 100644 x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/flattened/FlattenedFeatureSetUsage.java delete mode 100644 x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/indexlifecycle/client/ILMClient.java create mode 100644 x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/ml/action/DeleteDataFrameAnalyticsAction.java create mode 100644 x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/ml/action/EvaluateDataFrameAction.java create mode 100644 x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/ml/action/GetDataFrameAnalyticsAction.java create mode 100644 x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/ml/action/GetDataFrameAnalyticsStatsAction.java create mode 100644 x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/ml/action/PutDataFrameAnalyticsAction.java create mode 100644 x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/ml/action/StartDataFrameAnalyticsAction.java create mode 100644 x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/ml/action/StopDataFrameAnalyticsAction.java delete mode 100644 x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/ml/client/MachineLearningClient.java create mode 100644 x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/ml/dataframe/DataFrameAnalyticsConfig.java create mode 100644 x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/ml/dataframe/DataFrameAnalyticsDest.java create mode 100644 x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/ml/dataframe/DataFrameAnalyticsSource.java create mode 100644 x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/ml/dataframe/DataFrameAnalyticsState.java create mode 100644 x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/ml/dataframe/DataFrameAnalyticsTaskState.java create mode 100644 x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/ml/dataframe/analyses/DataFrameAnalysis.java create mode 100644 x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/ml/dataframe/analyses/MlDataFrameAnalysisNamedXContentProvider.java create mode 100644 x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/ml/dataframe/analyses/OutlierDetection.java create mode 100644 x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/ml/dataframe/evaluation/Evaluation.java create mode 100644 x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/ml/dataframe/evaluation/EvaluationMetricResult.java create mode 100644 x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/ml/dataframe/evaluation/MlEvaluationNamedXContentProvider.java create mode 100644 x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/ml/dataframe/evaluation/softclassification/AbstractConfusionMatrixMetric.java create mode 100644 x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/ml/dataframe/evaluation/softclassification/AucRoc.java create mode 100644 x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/ml/dataframe/evaluation/softclassification/BinarySoftClassification.java create mode 100644 x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/ml/dataframe/evaluation/softclassification/ConfusionMatrix.java create mode 100644 x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/ml/dataframe/evaluation/softclassification/Precision.java create mode 100644 x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/ml/dataframe/evaluation/softclassification/Recall.java create mode 100644 x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/ml/dataframe/evaluation/softclassification/ScoreByThresholdResult.java create mode 100644 x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/ml/dataframe/evaluation/softclassification/SoftClassificationMetric.java create mode 100644 x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/ml/job/process/autodetect/state/TimingStats.java rename x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/ml/{datafeed => utils}/QueryProvider.java (81%) delete mode 100644 x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/monitoring/client/MonitoringClient.java delete mode 100644 x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/rest/XPackRestHandler.java create mode 100644 x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/rest/action/RestReloadAnalyzersAction.java create mode 100644 x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/security/action/privilege/GetBuiltinPrivilegesAction.java create mode 100644 x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/security/action/privilege/GetBuiltinPrivilegesRequest.java create mode 100644 x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/security/action/privilege/GetBuiltinPrivilegesResponse.java create mode 100644 x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/security/authz/accesscontrol/DocumentSubsetBitsetCache.java create mode 100644 x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/security/authz/accesscontrol/SecurityIndexReaderWrapper.java delete mode 100644 x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/security/authz/accesscontrol/SecurityIndexSearcherWrapper.java delete mode 100644 x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/security/client/SecurityClient.java create mode 100644 x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/security/support/CacheIteratorHelper.java create mode 100644 x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/vectors/VectorsFeatureSetUsage.java create mode 100644 x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/votingonly/VotingOnlyNodeFeatureSetUsage.java delete mode 100644 x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/watcher/client/WatcherClient.java create mode 100644 x-pack/plugin/core/src/test/java/org/elasticsearch/xpack/core/action/ReloadAnalyzersResponseTests.java create mode 100644 x-pack/plugin/core/src/test/java/org/elasticsearch/xpack/core/action/ReloadSynonymAnalyzerTests.java rename x-pack/plugin/{ml/src/test/java/org/elasticsearch/xpack/ml/utils => core/src/test/java/org/elasticsearch/xpack/core/common}/time/TimeUtilsTests.java (64%) create mode 100644 x-pack/plugin/core/src/test/java/org/elasticsearch/xpack/core/dataframe/transforms/DataFrameTransformTests.java create mode 100644 x-pack/plugin/core/src/test/java/org/elasticsearch/xpack/core/dataframe/transforms/NodeAttributeTests.java create mode 100644 x-pack/plugin/core/src/test/java/org/elasticsearch/xpack/core/dataframe/transforms/TimeSyncConfigTests.java create mode 100644 x-pack/plugin/core/src/test/java/org/elasticsearch/xpack/core/dataframe/utils/DataFrameStringsTests.java create mode 100644 x-pack/plugin/core/src/test/java/org/elasticsearch/xpack/core/ml/action/EvaluateDataFrameActionRequestTests.java create mode 100644 x-pack/plugin/core/src/test/java/org/elasticsearch/xpack/core/ml/action/GetDataFrameAnalyticsActionResponseTests.java create mode 100644 x-pack/plugin/core/src/test/java/org/elasticsearch/xpack/core/ml/action/GetDataFrameAnalyticsRequestTests.java create mode 100644 x-pack/plugin/core/src/test/java/org/elasticsearch/xpack/core/ml/action/GetDataFrameAnalyticsStatsActionResponseTests.java create mode 100644 x-pack/plugin/core/src/test/java/org/elasticsearch/xpack/core/ml/action/GetDataFrameAnalyticsStatsRequestTests.java create mode 100644 x-pack/plugin/core/src/test/java/org/elasticsearch/xpack/core/ml/action/PutDataFrameAnalyticsActionRequestTests.java create mode 100644 x-pack/plugin/core/src/test/java/org/elasticsearch/xpack/core/ml/action/PutDataFrameAnalyticsActionResponseTests.java create mode 100644 x-pack/plugin/core/src/test/java/org/elasticsearch/xpack/core/ml/action/StartDataFrameAnalyticsActionTaskParamsTests.java create mode 100644 x-pack/plugin/core/src/test/java/org/elasticsearch/xpack/core/ml/action/StartDataFrameAnalyticsRequestTests.java create mode 100644 x-pack/plugin/core/src/test/java/org/elasticsearch/xpack/core/ml/action/StopDataFrameAnalyticsActionResponseTests.java create mode 100644 x-pack/plugin/core/src/test/java/org/elasticsearch/xpack/core/ml/action/StopDataFrameAnalyticsRequestTests.java create mode 100644 x-pack/plugin/core/src/test/java/org/elasticsearch/xpack/core/ml/dataframe/DataFrameAnalyticsConfigTests.java create mode 100644 x-pack/plugin/core/src/test/java/org/elasticsearch/xpack/core/ml/dataframe/DataFrameAnalyticsDestTests.java create mode 100644 x-pack/plugin/core/src/test/java/org/elasticsearch/xpack/core/ml/dataframe/DataFrameAnalyticsSourceTests.java create mode 100644 x-pack/plugin/core/src/test/java/org/elasticsearch/xpack/core/ml/dataframe/DataFrameAnalyticsStateTests.java create mode 100644 x-pack/plugin/core/src/test/java/org/elasticsearch/xpack/core/ml/dataframe/DataFrameAnalyticsTaskStateTests.java create mode 100644 x-pack/plugin/core/src/test/java/org/elasticsearch/xpack/core/ml/dataframe/analyses/OutlierDetectionTests.java create mode 100644 x-pack/plugin/core/src/test/java/org/elasticsearch/xpack/core/ml/dataframe/evaluation/softclassification/AucRocTests.java create mode 100644 x-pack/plugin/core/src/test/java/org/elasticsearch/xpack/core/ml/dataframe/evaluation/softclassification/BinarySoftClassificationTests.java create mode 100644 x-pack/plugin/core/src/test/java/org/elasticsearch/xpack/core/ml/dataframe/evaluation/softclassification/ConfusionMatrixTests.java create mode 100644 x-pack/plugin/core/src/test/java/org/elasticsearch/xpack/core/ml/dataframe/evaluation/softclassification/PrecisionTests.java create mode 100644 x-pack/plugin/core/src/test/java/org/elasticsearch/xpack/core/ml/dataframe/evaluation/softclassification/RecallTests.java create mode 100644 x-pack/plugin/core/src/test/java/org/elasticsearch/xpack/core/ml/job/process/autodetect/state/TimingStatsTests.java rename x-pack/plugin/core/src/test/java/org/elasticsearch/xpack/core/ml/{datafeed => utils}/QueryProviderTests.java (92%) create mode 100644 x-pack/plugin/core/src/test/java/org/elasticsearch/xpack/core/rest/action/ReloadSynonymAnalyzerIT.java create mode 100644 x-pack/plugin/core/src/test/java/org/elasticsearch/xpack/core/security/action/privilege/GetBuiltinPrivilegesResponseTests.java create mode 100644 x-pack/plugin/core/src/test/java/org/elasticsearch/xpack/core/security/authz/accesscontrol/DocumentSubsetBitsetCacheTests.java rename x-pack/plugin/core/src/test/java/org/elasticsearch/xpack/core/security/authz/accesscontrol/{SecurityIndexSearcherWrapperIntegrationTests.java => SecurityIndexReaderWrapperIntegrationTests.java} (84%) create mode 100644 x-pack/plugin/core/src/test/java/org/elasticsearch/xpack/core/security/authz/accesscontrol/SecurityIndexReaderWrapperUnitTests.java delete mode 100644 x-pack/plugin/core/src/test/java/org/elasticsearch/xpack/core/security/authz/accesscontrol/SecurityIndexSearcherWrapperUnitTests.java rename x-pack/plugin/data-frame/src/main/java/org/elasticsearch/xpack/dataframe/{DataFrameFeatureSet.java => DataFrameInfoTransportAction.java} (54%) create mode 100644 x-pack/plugin/data-frame/src/main/java/org/elasticsearch/xpack/dataframe/DataFrameUsageTransportAction.java rename x-pack/plugin/data-frame/src/test/java/org/elasticsearch/xpack/dataframe/{DataFrameFeatureSetTests.java => DataFrameInfoTransportActionTests.java} (74%) create mode 100644 x-pack/plugin/data-frame/src/test/java/org/elasticsearch/xpack/dataframe/persistence/DataframeIndexTests.java rename x-pack/plugin/graph/src/main/java/org/elasticsearch/xpack/graph/{GraphFeatureSet.java => GraphInfoTransportAction.java} (57%) create mode 100644 x-pack/plugin/graph/src/main/java/org/elasticsearch/xpack/graph/GraphUsageTransportAction.java rename x-pack/plugin/graph/src/test/java/org/elasticsearch/xpack/graph/{GraphFeatureSetTests.java => GraphInfoTransportActionTests.java} (62%) delete mode 100644 x-pack/plugin/ilm/src/main/java/org/elasticsearch/xpack/indexlifecycle/IndexLifecycleFeatureSet.java create mode 100644 x-pack/plugin/ilm/src/main/java/org/elasticsearch/xpack/indexlifecycle/IndexLifecycleInfoTransportAction.java create mode 100644 x-pack/plugin/ilm/src/main/java/org/elasticsearch/xpack/indexlifecycle/IndexLifecycleUsageTransportAction.java rename x-pack/plugin/ilm/src/test/java/org/elasticsearch/xpack/indexlifecycle/{IndexLifecycleFeatureSetTests.java => IndexLifecycleInfoTransportActionTests.java} (79%) rename x-pack/plugin/logstash/src/main/java/org/elasticsearch/xpack/logstash/{LogstashFeatureSet.java => LogstashInfoTransportAction.java} (53%) create mode 100644 x-pack/plugin/logstash/src/main/java/org/elasticsearch/xpack/logstash/LogstashUsageTransportAction.java rename x-pack/plugin/logstash/src/test/java/org/elasticsearch/xpack/logstash/{LogstashFeatureSetTests.java => LogstashInfoTransportActionTests.java} (55%) create mode 100644 x-pack/plugin/mapper-flattened/build.gradle create mode 100644 x-pack/plugin/mapper-flattened/src/main/java/org/elasticsearch/xpack/flattened/FlattenedInfoTransportAction.java create mode 100644 x-pack/plugin/mapper-flattened/src/main/java/org/elasticsearch/xpack/flattened/FlattenedMapperPlugin.java create mode 100644 x-pack/plugin/mapper-flattened/src/main/java/org/elasticsearch/xpack/flattened/FlattenedUsageTransportAction.java create mode 100644 x-pack/plugin/mapper-flattened/src/main/java/org/elasticsearch/xpack/flattened/mapper/FlatObjectFieldMapper.java create mode 100644 x-pack/plugin/mapper-flattened/src/main/java/org/elasticsearch/xpack/flattened/mapper/FlatObjectFieldParser.java create mode 100644 x-pack/plugin/mapper-flattened/src/main/java/org/elasticsearch/xpack/flattened/mapper/KeyedFlatObjectAtomicFieldData.java create mode 100644 x-pack/plugin/mapper-flattened/src/test/java/org/elasticsearch/index/mapper/FlatObjectFieldLookupTests.java create mode 100644 x-pack/plugin/mapper-flattened/src/test/java/org/elasticsearch/xpack/flattened/mapper/FlatObjectFieldMapperTests.java create mode 100644 x-pack/plugin/mapper-flattened/src/test/java/org/elasticsearch/xpack/flattened/mapper/FlatObjectFieldParserTests.java create mode 100644 x-pack/plugin/mapper-flattened/src/test/java/org/elasticsearch/xpack/flattened/mapper/FlatObjectIndexFieldDataTests.java create mode 100644 x-pack/plugin/mapper-flattened/src/test/java/org/elasticsearch/xpack/flattened/mapper/FlatObjectSearchTests.java create mode 100644 x-pack/plugin/mapper-flattened/src/test/java/org/elasticsearch/xpack/flattened/mapper/KeyedFlatObjectAtomicFieldDataTests.java create mode 100644 x-pack/plugin/mapper-flattened/src/test/java/org/elasticsearch/xpack/flattened/mapper/KeyedFlatObjectFieldTypeTests.java create mode 100644 x-pack/plugin/mapper-flattened/src/test/java/org/elasticsearch/xpack/flattened/mapper/RootFlatObjectFieldTypeTests.java create mode 100644 x-pack/plugin/ml/qa/native-multi-node-tests/src/test/java/org/elasticsearch/xpack/ml/integration/MlNativeDataFrameAnalyticsIntegTestCase.java create mode 100644 x-pack/plugin/ml/qa/native-multi-node-tests/src/test/java/org/elasticsearch/xpack/ml/integration/RunDataFrameAnalyticsIT.java delete mode 100644 x-pack/plugin/ml/src/main/java/org/elasticsearch/xpack/ml/MachineLearningFeatureSet.java create mode 100644 x-pack/plugin/ml/src/main/java/org/elasticsearch/xpack/ml/MachineLearningInfoTransportAction.java create mode 100644 x-pack/plugin/ml/src/main/java/org/elasticsearch/xpack/ml/MachineLearningUsageTransportAction.java create mode 100644 x-pack/plugin/ml/src/main/java/org/elasticsearch/xpack/ml/action/TransportDeleteDataFrameAnalyticsAction.java create mode 100644 x-pack/plugin/ml/src/main/java/org/elasticsearch/xpack/ml/action/TransportEvaluateDataFrameAction.java create mode 100644 x-pack/plugin/ml/src/main/java/org/elasticsearch/xpack/ml/action/TransportGetDataFrameAnalyticsAction.java create mode 100644 x-pack/plugin/ml/src/main/java/org/elasticsearch/xpack/ml/action/TransportGetDataFrameAnalyticsStatsAction.java create mode 100644 x-pack/plugin/ml/src/main/java/org/elasticsearch/xpack/ml/action/TransportPutDataFrameAnalyticsAction.java create mode 100644 x-pack/plugin/ml/src/main/java/org/elasticsearch/xpack/ml/action/TransportStartDataFrameAnalyticsAction.java create mode 100644 x-pack/plugin/ml/src/main/java/org/elasticsearch/xpack/ml/action/TransportStopDataFrameAnalyticsAction.java create mode 100644 x-pack/plugin/ml/src/main/java/org/elasticsearch/xpack/ml/dataframe/DataFrameAnalyticsIndex.java create mode 100644 x-pack/plugin/ml/src/main/java/org/elasticsearch/xpack/ml/dataframe/DataFrameAnalyticsManager.java create mode 100644 x-pack/plugin/ml/src/main/java/org/elasticsearch/xpack/ml/dataframe/MappingsMerger.java create mode 100644 x-pack/plugin/ml/src/main/java/org/elasticsearch/xpack/ml/dataframe/SourceDestValidator.java create mode 100644 x-pack/plugin/ml/src/main/java/org/elasticsearch/xpack/ml/dataframe/extractor/DataFrameDataExtractor.java create mode 100644 x-pack/plugin/ml/src/main/java/org/elasticsearch/xpack/ml/dataframe/extractor/DataFrameDataExtractorContext.java create mode 100644 x-pack/plugin/ml/src/main/java/org/elasticsearch/xpack/ml/dataframe/extractor/DataFrameDataExtractorFactory.java create mode 100644 x-pack/plugin/ml/src/main/java/org/elasticsearch/xpack/ml/dataframe/extractor/ExtractedFieldsDetector.java create mode 100644 x-pack/plugin/ml/src/main/java/org/elasticsearch/xpack/ml/dataframe/persistence/DataFrameAnalyticsConfigProvider.java create mode 100644 x-pack/plugin/ml/src/main/java/org/elasticsearch/xpack/ml/dataframe/process/AnalyticsBuilder.java create mode 100644 x-pack/plugin/ml/src/main/java/org/elasticsearch/xpack/ml/dataframe/process/AnalyticsControlMessageWriter.java create mode 100644 x-pack/plugin/ml/src/main/java/org/elasticsearch/xpack/ml/dataframe/process/AnalyticsProcess.java create mode 100644 x-pack/plugin/ml/src/main/java/org/elasticsearch/xpack/ml/dataframe/process/AnalyticsProcessConfig.java create mode 100644 x-pack/plugin/ml/src/main/java/org/elasticsearch/xpack/ml/dataframe/process/AnalyticsProcessFactory.java create mode 100644 x-pack/plugin/ml/src/main/java/org/elasticsearch/xpack/ml/dataframe/process/AnalyticsProcessManager.java create mode 100644 x-pack/plugin/ml/src/main/java/org/elasticsearch/xpack/ml/dataframe/process/AnalyticsResult.java create mode 100644 x-pack/plugin/ml/src/main/java/org/elasticsearch/xpack/ml/dataframe/process/AnalyticsResultProcessor.java create mode 100644 x-pack/plugin/ml/src/main/java/org/elasticsearch/xpack/ml/dataframe/process/DataFrameRowsJoiner.java create mode 100644 x-pack/plugin/ml/src/main/java/org/elasticsearch/xpack/ml/dataframe/process/NativeAnalyticsProcess.java create mode 100644 x-pack/plugin/ml/src/main/java/org/elasticsearch/xpack/ml/dataframe/process/NativeAnalyticsProcessFactory.java create mode 100644 x-pack/plugin/ml/src/main/java/org/elasticsearch/xpack/ml/dataframe/process/results/RowResults.java create mode 100644 x-pack/plugin/ml/src/main/java/org/elasticsearch/xpack/ml/job/JobNodeSelector.java create mode 100644 x-pack/plugin/ml/src/main/java/org/elasticsearch/xpack/ml/job/persistence/TimingStatsReporter.java rename x-pack/plugin/ml/src/main/java/org/elasticsearch/xpack/ml/job/process/autodetect/output/{AutoDetectResultProcessor.java => AutodetectResultProcessor.java} (79%) create mode 100644 x-pack/plugin/ml/src/main/java/org/elasticsearch/xpack/ml/process/DummyController.java create mode 100644 x-pack/plugin/ml/src/main/java/org/elasticsearch/xpack/ml/process/MlController.java create mode 100644 x-pack/plugin/ml/src/main/java/org/elasticsearch/xpack/ml/process/MlControllerHolder.java delete mode 100644 x-pack/plugin/ml/src/main/java/org/elasticsearch/xpack/ml/process/NativeControllerHolder.java rename x-pack/plugin/ml/src/main/java/org/elasticsearch/xpack/ml/{job/process/autodetect/output/AutodetectResultsParser.java => process/ProcessResultsParser.java} (72%) create mode 100644 x-pack/plugin/ml/src/main/java/org/elasticsearch/xpack/ml/rest/dataframe/RestDeleteDataFrameAnalyticsAction.java create mode 100644 x-pack/plugin/ml/src/main/java/org/elasticsearch/xpack/ml/rest/dataframe/RestEvaluateDataFrameAction.java create mode 100644 x-pack/plugin/ml/src/main/java/org/elasticsearch/xpack/ml/rest/dataframe/RestGetDataFrameAnalyticsAction.java create mode 100644 x-pack/plugin/ml/src/main/java/org/elasticsearch/xpack/ml/rest/dataframe/RestGetDataFrameAnalyticsStatsAction.java create mode 100644 x-pack/plugin/ml/src/main/java/org/elasticsearch/xpack/ml/rest/dataframe/RestPutDataFrameAnalyticsAction.java create mode 100644 x-pack/plugin/ml/src/main/java/org/elasticsearch/xpack/ml/rest/dataframe/RestStartDataFrameAnalyticsAction.java create mode 100644 x-pack/plugin/ml/src/main/java/org/elasticsearch/xpack/ml/rest/dataframe/RestStopDataFrameAnalyticsAction.java create mode 100644 x-pack/plugin/ml/src/main/plugin-metadata/plugin-security-test.policy rename x-pack/plugin/ml/src/test/java/org/elasticsearch/xpack/ml/{MachineLearningFeatureSetTests.java => MachineLearningInfoTransportActionTests.java} (82%) create mode 100644 x-pack/plugin/ml/src/test/java/org/elasticsearch/xpack/ml/action/TransportStopDataFrameAnalyticsActionTests.java create mode 100644 x-pack/plugin/ml/src/test/java/org/elasticsearch/xpack/ml/dataframe/DataFrameAnalyticsIndexTests.java create mode 100644 x-pack/plugin/ml/src/test/java/org/elasticsearch/xpack/ml/dataframe/MappingsMergerTests.java create mode 100644 x-pack/plugin/ml/src/test/java/org/elasticsearch/xpack/ml/dataframe/SourceDestValidatorTests.java create mode 100644 x-pack/plugin/ml/src/test/java/org/elasticsearch/xpack/ml/dataframe/extractor/DataFrameDataExtractorTests.java create mode 100644 x-pack/plugin/ml/src/test/java/org/elasticsearch/xpack/ml/dataframe/extractor/ExtractedFieldsDetectorTests.java create mode 100644 x-pack/plugin/ml/src/test/java/org/elasticsearch/xpack/ml/dataframe/process/AnalyticsControlMessageWriterTests.java create mode 100644 x-pack/plugin/ml/src/test/java/org/elasticsearch/xpack/ml/dataframe/process/AnalyticsResultProcessorTests.java create mode 100644 x-pack/plugin/ml/src/test/java/org/elasticsearch/xpack/ml/dataframe/process/AnalyticsResultTests.java create mode 100644 x-pack/plugin/ml/src/test/java/org/elasticsearch/xpack/ml/dataframe/process/DataFrameRowsJoinerTests.java create mode 100644 x-pack/plugin/ml/src/test/java/org/elasticsearch/xpack/ml/dataframe/process/results/RowResultsTests.java create mode 100644 x-pack/plugin/ml/src/test/java/org/elasticsearch/xpack/ml/job/JobNodeSelectorTests.java create mode 100644 x-pack/plugin/ml/src/test/java/org/elasticsearch/xpack/ml/job/persistence/TimingStatsReporterTests.java create mode 100644 x-pack/plugin/ml/src/test/java/org/elasticsearch/xpack/ml/job/process/autodetect/NativeAutodetectProcessFactoryTests.java rename x-pack/plugin/ml/src/test/java/org/elasticsearch/xpack/ml/job/process/autodetect/output/{AutoDetectResultProcessorTests.java => AutodetectResultProcessorTests.java} (52%) delete mode 100644 x-pack/plugin/ml/src/test/java/org/elasticsearch/xpack/ml/job/process/autodetect/output/AutodetectResultsParserTests.java create mode 100644 x-pack/plugin/ml/src/test/java/org/elasticsearch/xpack/ml/job/process/autodetect/params/AutodetectParamsTests.java create mode 100644 x-pack/plugin/ml/src/test/java/org/elasticsearch/xpack/ml/process/ProcessResultsParserTests.java delete mode 100644 x-pack/plugin/monitoring/src/main/java/org/elasticsearch/xpack/monitoring/MonitoringFeatureSet.java create mode 100644 x-pack/plugin/monitoring/src/main/java/org/elasticsearch/xpack/monitoring/MonitoringInfoTransportAction.java create mode 100644 x-pack/plugin/monitoring/src/main/java/org/elasticsearch/xpack/monitoring/MonitoringUsageServices.java create mode 100644 x-pack/plugin/monitoring/src/main/java/org/elasticsearch/xpack/monitoring/MonitoringUsageTransportAction.java rename x-pack/plugin/monitoring/src/test/java/org/elasticsearch/xpack/monitoring/{MonitoringFeatureSetTests.java => MonitoringInfoTransportActionTests.java} (82%) rename x-pack/plugin/rollup/src/main/java/org/elasticsearch/xpack/rollup/{RollupFeatureSet.java => RollupInfoTransportAction.java} (50%) create mode 100644 x-pack/plugin/rollup/src/main/java/org/elasticsearch/xpack/rollup/RollupUsageTransportAction.java rename x-pack/plugin/rollup/src/test/java/org/elasticsearch/xpack/rollup/{RollupFeatureSetTests.java => RollupInfoTransportActionTests.java} (58%) create mode 100644 x-pack/plugin/security/src/main/java/org/elasticsearch/xpack/security/SecurityInfoTransportAction.java create mode 100644 x-pack/plugin/security/src/main/java/org/elasticsearch/xpack/security/SecurityUsageServices.java rename x-pack/plugin/security/src/main/java/org/elasticsearch/xpack/security/{SecurityFeatureSet.java => SecurityUsageTransportAction.java} (73%) create mode 100644 x-pack/plugin/security/src/main/java/org/elasticsearch/xpack/security/action/privilege/TransportGetBuiltinPrivilegesAction.java create mode 100644 x-pack/plugin/security/src/main/java/org/elasticsearch/xpack/security/rest/action/privilege/RestGetBuiltinPrivilegesAction.java create mode 100644 x-pack/plugin/security/src/main/java/org/elasticsearch/xpack/security/support/SecurityStatusChangeListener.java rename x-pack/plugin/security/src/test/java/org/elasticsearch/xpack/security/{SecurityFeatureSetTests.java => SecurityInfoTransportActionTests.java} (88%) create mode 100644 x-pack/plugin/security/src/test/java/org/elasticsearch/xpack/security/support/SecurityStatusChangeListenerTests.java delete mode 100644 x-pack/plugin/security/src/test/java/org/elasticsearch/xpack/security/transport/ServerTransportFilterIntegrationTests.java delete mode 100644 x-pack/plugin/sql/qa/src/main/java/org/elasticsearch/xpack/sql/qa/jdbc/DebugCsvSpec.java delete mode 100644 x-pack/plugin/sql/src/main/java/org/elasticsearch/xpack/sql/SqlFeatureSet.java create mode 100644 x-pack/plugin/sql/src/main/java/org/elasticsearch/xpack/sql/SqlInfoTransportAction.java create mode 100644 x-pack/plugin/sql/src/main/java/org/elasticsearch/xpack/sql/SqlUsageTransportAction.java rename x-pack/plugin/sql/src/test/java/org/elasticsearch/xpack/sql/{SqlFeatureSetTests.java => SqlInfoTransportActionTests.java} (77%) create mode 100644 x-pack/plugin/sql/src/test/java/org/elasticsearch/xpack/sql/execution/search/QuerierTests.java create mode 100644 x-pack/plugin/src/test/resources/rest-api-spec/api/indices.reload_search_analyzers.json create mode 100644 x-pack/plugin/src/test/resources/rest-api-spec/api/ml.delete_data_frame_analytics.json create mode 100644 x-pack/plugin/src/test/resources/rest-api-spec/api/ml.evaluate_data_frame.json create mode 100644 x-pack/plugin/src/test/resources/rest-api-spec/api/ml.get_data_frame_analytics.json create mode 100644 x-pack/plugin/src/test/resources/rest-api-spec/api/ml.get_data_frame_analytics_stats.json create mode 100644 x-pack/plugin/src/test/resources/rest-api-spec/api/ml.put_data_frame_analytics.json create mode 100644 x-pack/plugin/src/test/resources/rest-api-spec/api/ml.start_data_frame_analytics.json create mode 100644 x-pack/plugin/src/test/resources/rest-api-spec/api/ml.stop_data_frame_analytics.json create mode 100644 x-pack/plugin/src/test/resources/rest-api-spec/api/security.get_builtin_privileges.json create mode 100644 x-pack/plugin/src/test/resources/rest-api-spec/test/flattened/10_basic.yml create mode 100644 x-pack/plugin/src/test/resources/rest-api-spec/test/indices.freeze/20_stats.yml create mode 100644 x-pack/plugin/src/test/resources/rest-api-spec/test/ml/data_frame_analytics_crud.yml create mode 100644 x-pack/plugin/src/test/resources/rest-api-spec/test/ml/evaluate_data_frame.yml create mode 100644 x-pack/plugin/src/test/resources/rest-api-spec/test/ml/start_data_frame_analytics.yml create mode 100644 x-pack/plugin/src/test/resources/rest-api-spec/test/ml/stop_data_frame_analytics.yml create mode 100644 x-pack/plugin/src/test/resources/rest-api-spec/test/privileges/11_builtin.yml rename modules/mapper-extras/src/test/resources/rest-api-spec/test/dense-vector/10_basic.yml => x-pack/plugin/src/test/resources/rest-api-spec/test/vectors/10_dense_vector_basic.yml (95%) rename modules/mapper-extras/src/test/resources/rest-api-spec/test/dense-vector/20_special_cases.yml => x-pack/plugin/src/test/resources/rest-api-spec/test/vectors/20_dense_vector_special_cases.yml (60%) rename modules/mapper-extras/src/test/resources/rest-api-spec/test/sparse-vector/10_basic.yml => x-pack/plugin/src/test/resources/rest-api-spec/test/vectors/30_sparse_vector_basic.yml (96%) rename modules/mapper-extras/src/test/resources/rest-api-spec/test/sparse-vector/20_special_cases.yml => x-pack/plugin/src/test/resources/rest-api-spec/test/vectors/40_sparse_vector_special_cases.yml (98%) create mode 100644 x-pack/plugin/src/test/resources/rest-api-spec/test/voting_only_node/10_basic.yml create mode 100644 x-pack/plugin/src/test/resources/rest-api-spec/test/watcher/execute_watch/80_foreach.yml create mode 100644 x-pack/plugin/vectors/build.gradle create mode 100644 x-pack/plugin/vectors/src/main/java/org/elasticsearch/xpack/vectors/Vectors.java create mode 100644 x-pack/plugin/vectors/src/main/java/org/elasticsearch/xpack/vectors/VectorsInfoTransportAction.java create mode 100644 x-pack/plugin/vectors/src/main/java/org/elasticsearch/xpack/vectors/VectorsUsageTransportAction.java rename {modules/mapper-extras/src/main/java/org/elasticsearch/index => x-pack/plugin/vectors/src/main/java/org/elasticsearch/xpack/vectors}/mapper/DenseVectorFieldMapper.java (67%) rename {modules/mapper-extras/src/main/java/org/elasticsearch/index => x-pack/plugin/vectors/src/main/java/org/elasticsearch/xpack/vectors}/mapper/SparseVectorFieldMapper.java (89%) rename {modules/mapper-extras/src/main/java/org/elasticsearch/index => x-pack/plugin/vectors/src/main/java/org/elasticsearch/xpack/vectors}/mapper/VectorEncoderDecoder.java (84%) create mode 100644 x-pack/plugin/vectors/src/main/java/org/elasticsearch/xpack/vectors/query/DocValuesWhitelistExtension.java rename {modules/mapper-extras/src/main/java/org/elasticsearch/index => x-pack/plugin/vectors/src/main/java/org/elasticsearch/xpack/vectors}/query/ScoreScriptUtils.java (87%) rename {modules/mapper-extras/src/main/java/org/elasticsearch/index => x-pack/plugin/vectors/src/main/java/org/elasticsearch/xpack/vectors}/query/VectorDVAtomicFieldData.java (69%) rename {modules/mapper-extras/src/main/java/org/elasticsearch/index => x-pack/plugin/vectors/src/main/java/org/elasticsearch/xpack/vectors}/query/VectorDVIndexFieldData.java (72%) rename {modules/mapper-extras/src/main/java/org/elasticsearch/index => x-pack/plugin/vectors/src/main/java/org/elasticsearch/xpack/vectors}/query/VectorScriptDocValues.java (67%) create mode 100644 x-pack/plugin/vectors/src/main/resources/META-INF/services/org.elasticsearch.painless.spi.PainlessExtension create mode 100644 x-pack/plugin/vectors/src/main/resources/org/elasticsearch/xpack/vectors/query/whitelist.txt create mode 100644 x-pack/plugin/vectors/src/test/java/org/elasticsearch/xpack/vectors/VectorsInfoTransportActionTests.java create mode 100644 x-pack/plugin/vectors/src/test/java/org/elasticsearch/xpack/vectors/mapper/DenseVectorFieldMapperTests.java create mode 100644 x-pack/plugin/vectors/src/test/java/org/elasticsearch/xpack/vectors/mapper/DenseVectorFieldTypeTests.java rename {modules/mapper-extras/src/test/java/org/elasticsearch/index => x-pack/plugin/vectors/src/test/java/org/elasticsearch/xpack/vectors}/mapper/SparseVectorFieldMapperTests.java (90%) create mode 100644 x-pack/plugin/vectors/src/test/java/org/elasticsearch/xpack/vectors/mapper/SparseVectorFieldTypeTests.java rename {modules/mapper-extras/src/test/java/org/elasticsearch/index => x-pack/plugin/vectors/src/test/java/org/elasticsearch/xpack/vectors}/mapper/VectorEncoderDecoderTests.java (69%) rename {modules/mapper-extras/src/test/java/org/elasticsearch/index => x-pack/plugin/vectors/src/test/java/org/elasticsearch/xpack/vectors}/query/ScoreScriptUtilsTests.java (61%) create mode 100644 x-pack/plugin/voting-only-node/build.gradle create mode 100644 x-pack/plugin/voting-only-node/src/main/java/org/elasticsearch/cluster/coordination/VotingOnlyNodeFeatureSet.java create mode 100644 x-pack/plugin/voting-only-node/src/main/java/org/elasticsearch/cluster/coordination/VotingOnlyNodePlugin.java create mode 100644 x-pack/plugin/voting-only-node/src/test/java/org/elasticsearch/cluster/coordination/LocalStateVotingOnlyNodePlugin.java create mode 100644 x-pack/plugin/voting-only-node/src/test/java/org/elasticsearch/cluster/coordination/VotingOnlyNodeCoordinationStateTests.java create mode 100644 x-pack/plugin/voting-only-node/src/test/java/org/elasticsearch/cluster/coordination/VotingOnlyNodeCoordinatorTests.java create mode 100644 x-pack/plugin/voting-only-node/src/test/java/org/elasticsearch/cluster/coordination/VotingOnlyNodePluginTests.java create mode 100644 x-pack/plugin/watcher/src/main/java/org/elasticsearch/xpack/watcher/ClockHolder.java delete mode 100644 x-pack/plugin/watcher/src/main/java/org/elasticsearch/xpack/watcher/WatcherFeatureSet.java create mode 100644 x-pack/plugin/watcher/src/main/java/org/elasticsearch/xpack/watcher/WatcherInfoTransportAction.java create mode 100644 x-pack/plugin/watcher/src/main/java/org/elasticsearch/xpack/watcher/WatcherUsageTransportAction.java delete mode 100644 x-pack/plugin/watcher/src/main/java/org/elasticsearch/xpack/watcher/rest/WatcherRestHandler.java rename x-pack/plugin/watcher/src/test/java/org/elasticsearch/xpack/watcher/{WatcherFeatureSetTests.java => WatcherInfoTransportActionTests.java} (81%) delete mode 100644 x-pack/plugin/watcher/src/test/java/org/elasticsearch/xpack/watcher/WatcherXpackUsageStatsTests.java create mode 100644 x-pack/qa/rolling-upgrade/src/test/java/org/elasticsearch/upgrades/DataFrameSurvivesUpgradeIT.java create mode 100644 x-pack/qa/rolling-upgrade/src/test/resources/rest-api-spec/test/mixed_cluster/80_data_frame_jobs_crud.yml create mode 100644 x-pack/qa/rolling-upgrade/src/test/resources/rest-api-spec/test/old_cluster/80_data_frame_jobs_crud.yml create mode 100644 x-pack/qa/rolling-upgrade/src/test/resources/rest-api-spec/test/upgraded_cluster/80_data_frame_jobs_crud.yml diff --git a/.ci/java-versions.properties b/.ci/java-versions.properties index e40470bf51b89..16b760507324d 100644 --- a/.ci/java-versions.properties +++ b/.ci/java-versions.properties @@ -8,3 +8,4 @@ ES_BUILD_JAVA=openjdk12 ES_RUNTIME_JAVA=java11 GRADLE_TASK=build + diff --git a/.ci/matrix-runtime-javas.yml b/.ci/matrix-runtime-javas.yml index 61746ea59e1d0..583acce078055 100644 --- a/.ci/matrix-runtime-javas.yml +++ b/.ci/matrix-runtime-javas.yml @@ -9,6 +9,7 @@ ES_RUNTIME_JAVA: - java11 - java12 - openjdk12 + - openjdk13 - zulu11 - zulu12 - corretto11 diff --git a/.github/PULL_REQUEST_TEMPLATE.md b/.github/PULL_REQUEST_TEMPLATE.md index 6a4531f1bdefa..57f0992b9172d 100644 --- a/.github/PULL_REQUEST_TEMPLATE.md +++ b/.github/PULL_REQUEST_TEMPLATE.md @@ -10,5 +10,5 @@ attention. - Have you followed the [contributor guidelines](https://github.com/elastic/elasticsearch/blob/master/CONTRIBUTING.md)? - If submitting code, have you built your formula locally prior to submission with `gradle check`? - If submitting code, is your pull request against master? Unless there is a good reason otherwise, we prefer pull requests against master and will backport as needed. -- If submitting code, have you checked that your submission is for an [OS that we support](https://www.elastic.co/support/matrix#show_os)? +- If submitting code, have you checked that your submission is for an [OS and architecture that we support](https://www.elastic.co/support/matrix#show_os)? - If you are submitting this code for a class then read our [policy](https://github.com/elastic/elasticsearch/blob/master/CONTRIBUTING.md#contributing-as-part-of-a-class) for that. diff --git a/CONTRIBUTING.md b/CONTRIBUTING.md index 9bf11f2b49fd5..c2bae4e206256 100644 --- a/CONTRIBUTING.md +++ b/CONTRIBUTING.md @@ -325,8 +325,7 @@ Commercially licensed code that integrates with the rest of Elasticsearch. The `docs` subdirectory functions just like the top level `docs` subdirectory and the `qa` subdirectory functions just like the top level `qa` subdirectory. The `plugin` subdirectory contains the x-pack module which runs inside the -Elasticsearch process. The `transport-client` subdirectory contains extensions -to Elasticsearch's standard transport client to work properly with x-pack. +Elasticsearch process. ### Gradle Build diff --git a/README.textile b/README.textile index 5912c0371fa2e..6f907f370e7a5 100644 --- a/README.textile +++ b/README.textile @@ -150,13 +150,13 @@ curl -XPUT 'http://localhost:9200/kimchy/_doc/2?pretty' -H 'Content-Type: applic The above will index information into the @kimchy@ index. Each user will get their own special index. -Complete control on the index level is allowed. As an example, in the above case, we would want to change from the default 5 shards with 1 replica per index, to only 1 shard with 1 replica per index (== per twitter user). Here is how this can be done (the configuration can be in yaml as well): +Complete control on the index level is allowed. As an example, in the above case, we might want to change from the default 1 shard with 1 replica per index, to 2 shards with 1 replica per index (because this user tweets a lot). Here is how this can be done (the configuration can be in yaml as well):
 curl -XPUT http://localhost:9200/another_user?pretty -H 'Content-Type: application/json' -d '
 {
     "index" : {
-        "number_of_shards" : 1,
+        "number_of_shards" : 2,
         "number_of_replicas" : 1
     }
 }'
@@ -191,7 +191,7 @@ h3. Distributed, Highly Available
 
 Let's face it, things will fail....
 
-Elasticsearch is a highly available and distributed search engine. Each index is broken down into shards, and each shard can have one or more replicas. By default, an index is created with 5 shards and 1 replica per shard (5/1). There are many topologies that can be used, including 1/10 (improve search performance), or 20/1 (improve indexing performance, with search executed in a map reduce fashion across shards).
+Elasticsearch is a highly available and distributed search engine. Each index is broken down into shards, and each shard can have one or more replicas. By default, an index is created with 1 shards and 1 replica per shard (1/1). There are many topologies that can be used, including 1/10 (improve search performance), or 20/1 (improve indexing performance, with search executed in a map reduce fashion across shards).
 
 In order to play with the distributed nature of Elasticsearch, simply bring more nodes up and shut down nodes. The system will continue to serve requests (make sure you use the correct http port) with the latest data indexed.
 
diff --git a/TESTING.asciidoc b/TESTING.asciidoc
index 5a443127745e6..022ebc47f7ec0 100644
--- a/TESTING.asciidoc
+++ b/TESTING.asciidoc
@@ -305,12 +305,6 @@ e.g. -Dtests.rest.suite=index,get,create/10_with_id
 blacklisted and need to be skipped
 e.g. -Dtests.rest.blacklist=index/*/Index document,get/10_basic/*
 
-Note that the REST tests, like all the integration tests, can be run against an external
-cluster by specifying the `tests.cluster` property, which if present needs to contain a
-comma separated list of nodes to connect to (e.g. localhost:9300). A transport client will
-be created based on that and used for all the before|after test operations, and to extract
-the http addresses of the nodes so that REST requests can be sent to them.
-
 == Testing packaging
 
 The packaging tests use Vagrant virtual machines to verify that installing
diff --git a/benchmarks/build.gradle b/benchmarks/build.gradle
index e85f9a5608644..376ad4d4e6716 100644
--- a/benchmarks/build.gradle
+++ b/benchmarks/build.gradle
@@ -27,7 +27,7 @@ archivesBaseName = 'elasticsearch-benchmarks'
 test.enabled = false
 
 dependencies {
-    compile("org.elasticsearch:elasticsearch:${version}") {
+    compile(project(":server")) {
         // JMH ships with the conflicting version 4.6. This prevents us from using jopt-simple in benchmarks (which should be ok) but allows
         // us to invoke the JMH uberjar as usual.
         exclude group: 'net.sf.jopt-simple', module: 'jopt-simple'
diff --git a/benchmarks/src/main/java/org/elasticsearch/benchmark/routing/allocation/Allocators.java b/benchmarks/src/main/java/org/elasticsearch/benchmark/routing/allocation/Allocators.java
index 39a2bdfca0953..90f03c26dcb27 100644
--- a/benchmarks/src/main/java/org/elasticsearch/benchmark/routing/allocation/Allocators.java
+++ b/benchmarks/src/main/java/org/elasticsearch/benchmark/routing/allocation/Allocators.java
@@ -22,6 +22,7 @@
 import org.elasticsearch.cluster.ClusterModule;
 import org.elasticsearch.cluster.EmptyClusterInfoService;
 import org.elasticsearch.cluster.node.DiscoveryNode;
+import org.elasticsearch.cluster.node.DiscoveryNodeRole;
 import org.elasticsearch.cluster.routing.ShardRouting;
 import org.elasticsearch.cluster.routing.allocation.AllocationService;
 import org.elasticsearch.cluster.routing.allocation.FailedShard;
@@ -92,7 +93,7 @@ public static AllocationDeciders defaultAllocationDeciders(Settings settings, Cl
 
     public static DiscoveryNode newNode(String nodeId, Map attributes) {
         return new DiscoveryNode("", nodeId, new TransportAddress(TransportAddress.META_ADDRESS,
-            portGenerator.incrementAndGet()), attributes, Sets.newHashSet(DiscoveryNode.Role.MASTER,
-            DiscoveryNode.Role.DATA), Version.CURRENT);
+            portGenerator.incrementAndGet()), attributes, Sets.newHashSet(DiscoveryNodeRole.MASTER_ROLE,
+            DiscoveryNodeRole.DATA_ROLE), Version.CURRENT);
     }
 }
diff --git a/build.gradle b/build.gradle
index 1de3f919d9c49..5c1fe80668283 100644
--- a/build.gradle
+++ b/build.gradle
@@ -104,10 +104,7 @@ subprojects {
 
 /* Introspect all versions of ES that may be tested against for backwards
  * compatibility. It is *super* important that this logic is the same as the
- * logic in VersionUtils.java, throwing out alphas because they don't have any
- * backwards compatibility guarantees and only keeping the latest beta or rc
- * in a branch if there are only betas and rcs in the branch so we have
- * *something* to test against. */
+ * logic in VersionUtils.java. */
 BwcVersions versions = new BwcVersions(file('server/src/main/java/org/elasticsearch/Version.java').readLines('UTF-8'))
 
 // build metadata from previous build, contains eg hashes for bwc builds
@@ -209,68 +206,7 @@ allprojects {
     javadoc.options.addStringOption('Xdoclint:all,-missing', '-quiet')
   }
 
-  /* Sets up the dependencies that we build as part of this project but
-    register as though they were external to resolve internally. We register
-    them as external dependencies so the build plugin that we use can be used
-    to build elasticsearch plugins outside of the elasticsearch source tree. */
-  ext.projectSubstitutions = [
-    "org.elasticsearch.gradle:build-tools:${version}": ':build-tools',
-    "org.elasticsearch:rest-api-spec:${version}": ':rest-api-spec',
-    "org.elasticsearch:elasticsearch:${version}": ':server',
-    "org.elasticsearch:elasticsearch-cli:${version}": ':libs:elasticsearch-cli',
-    "org.elasticsearch:elasticsearch-core:${version}": ':libs:core',
-    "org.elasticsearch:elasticsearch-nio:${version}": ':libs:nio',
-    "org.elasticsearch:elasticsearch-x-content:${version}": ':libs:x-content',
-    "org.elasticsearch:elasticsearch-geo:${version}": ':libs:elasticsearch-geo',
-    "org.elasticsearch:elasticsearch-secure-sm:${version}": ':libs:secure-sm',
-    "org.elasticsearch:elasticsearch-ssl-config:${version}": ':libs:elasticsearch-ssl-config',
-    "org.elasticsearch.client:elasticsearch-rest-client:${version}": ':client:rest',
-    "org.elasticsearch.client:elasticsearch-rest-client-sniffer:${version}": ':client:sniffer',
-    "org.elasticsearch.client:elasticsearch-rest-high-level-client:${version}": ':client:rest-high-level',
-    "org.elasticsearch.client:test:${version}": ':client:test',
-    "org.elasticsearch.client:transport:${version}": ':client:transport',
-    "org.elasticsearch.plugin:elasticsearch-scripting-painless-spi:${version}": ':modules:lang-painless:spi',
-    "org.elasticsearch.test:framework:${version}": ':test:framework',
-    "org.elasticsearch.test:logger-usage:${version}": ':test:logger-usage',
-    "org.elasticsearch.xpack.test:feature-aware:${version}": ':x-pack:test:feature-aware',
-    // for transport client
-    "org.elasticsearch.plugin:transport-netty4-client:${version}": ':modules:transport-netty4',
-    "org.elasticsearch.plugin:reindex-client:${version}": ':modules:reindex',
-    "org.elasticsearch.plugin:lang-mustache-client:${version}": ':modules:lang-mustache',
-    "org.elasticsearch.plugin:parent-join-client:${version}": ':modules:parent-join',
-    "org.elasticsearch.plugin:aggs-matrix-stats-client:${version}": ':modules:aggs-matrix-stats',
-    "org.elasticsearch.plugin:percolator-client:${version}": ':modules:percolator',
-    "org.elasticsearch.plugin:rank-eval-client:${version}": ':modules:rank-eval',
-    // for security example plugins
-    "org.elasticsearch.plugin:x-pack-core:${version}": ':x-pack:plugin:core'
-  ]
-
-  /*
-   * Gradle only resolve project substitutions during dependency resolution but
-   * we sometimes want to do the resolution at other times. This creates a
-   * convenient method we can call to do it.
-   */
-  ext.dependencyToProject = { Dependency dep ->
-    if (dep instanceof ProjectDependency) {
-      return dep.dependencyProject
-    } else {
-      String substitution = projectSubstitutions.get("${dep.group}:${dep.name}:${dep.version}")
-      if (substitution != null) {
-        return findProject(substitution)
-      }
-      return null
-    }
-  }
-
   project.afterEvaluate {
-    configurations.matching { it.canBeResolved }.all {
-      resolutionStrategy.dependencySubstitution { DependencySubstitutions subs ->
-        projectSubstitutions.each { k,v ->
-          subs.substitute(subs.module(k)).with(subs.project(v))
-        }
-      }
-    }
-
     // Handle javadoc dependencies across projects. Order matters: the linksOffline for
     // org.elasticsearch:elasticsearch must be the last one or all the links for the
     // other packages (e.g org.elasticsearch.client) will point to server rather than
@@ -279,10 +215,10 @@ allprojects {
       String artifactsHost = VersionProperties.elasticsearch.endsWith("-SNAPSHOT") ? "https://snapshots.elastic.co" : "https://artifacts.elastic.co"
       Closure sortClosure = { a, b -> b.group <=> a.group }
       Closure depJavadocClosure = { shadowed, dep ->
-        if (dep.group == null || false == dep.group.startsWith('org.elasticsearch')) {
+        if ((dep instanceof ProjectDependency) == false) {
           return
         }
-        Project upstreamProject = project.ext.dependencyToProject(dep)
+        Project upstreamProject = dep.dependencyProject
         if (upstreamProject == null) {
           return
         }
@@ -338,8 +274,8 @@ gradle.projectsEvaluated {
       integTest.mustRunAfter test
     }
     configurations.matching { it.canBeResolved }.all { Configuration configuration ->
-      dependencies.all { Dependency dep ->
-        Project upstreamProject = dependencyToProject(dep)
+      dependencies.matching { it instanceof ProjectDependency }.all { ProjectDependency dep ->
+        Project upstreamProject = dep.dependencyProject
         if (upstreamProject != null) {
           if (project.path == upstreamProject.path) {
             // TODO: distribution integ tests depend on themselves (!), fix that
@@ -552,31 +488,6 @@ gradle.projectsEvaluated {
   }
 }
 
-if (System.properties.get("build.compare") != null) {
-  apply plugin: 'compare-gradle-builds'
-  compareGradleBuilds {
-    ext.referenceProject = System.properties.get("build.compare")
-    doFirst {
-      if (file(referenceProject).exists() == false) {
-        throw new GradleException(
-                "Use git worktree to check out a version to compare against to ../elasticsearch_build_reference"
-        )
-      }
-    }
-    sourceBuild {
-      gradleVersion = gradle.getGradleVersion()
-      projectDir = referenceProject
-      tasks = ["clean", "assemble"]
-      arguments = ["-Dbuild.compare_friendly=true"]
-    }
-    targetBuild {
-      tasks = ["clean", "assemble"]
-      // use -Dorg.gradle.java.home= to alter jdk versions
-      arguments = ["-Dbuild.compare_friendly=true"]
-    }
-  }
-}
-
 allprojects {
   task resolveAllDependencies {
       dependsOn tasks.matching { it.name == "pullFixture"}
diff --git a/buildSrc/build.gradle b/buildSrc/build.gradle
index d3a16f55277d7..a3ca1c7ffa348 100644
--- a/buildSrc/build.gradle
+++ b/buildSrc/build.gradle
@@ -67,8 +67,26 @@ if (JavaVersion.current() < JavaVersion.VERSION_11) {
 }
 
 // Keep compatibility with Java 8 for external users of build-tools that haven't migrated to Java 11
-targetCompatibility = '8'
-sourceCompatibility = '8'
+targetCompatibility = '11'
+sourceCompatibility = '11'
+
+sourceSets {
+    // We have a few classes that need to be compiled for older java versions
+    minimumRuntime { }
+}
+
+compileMinimumRuntimeJava {
+    targetCompatibility = 8
+    sourceCompatibility = 8
+}
+
+jar {
+    from sourceSets.minimumRuntime.output
+}
+
+javadoc {
+    source sourceSets.minimumRuntime.allSource
+}
 
 /*****************************************************************************
  *                    Dependencies used by the entire build                  *
@@ -79,8 +97,15 @@ repositories {
 }
 
 dependencies {
+  if (project.ext.has("isEclipse") == false || project.ext.isEclipse == false) {
+    // eclipse is confused if this is set explicitly
+    compile sourceSets.minimumRuntime.output
+  }
+
   compile localGroovy()
   
+  compile 'commons-codec:commons-codec:1.12'
+
   compile 'com.netflix.nebula:gradle-extra-configurations-plugin:3.0.3'
   compile 'com.netflix.nebula:nebula-publishing-plugin:4.4.4'
   compile 'com.netflix.nebula:gradle-info-plugin:3.0.3'
@@ -94,6 +119,9 @@ dependencies {
   testCompile "junit:junit:${props.getProperty('junit')}"
   testCompile "com.carrotsearch.randomizedtesting:randomizedtesting-runner:${props.getProperty('randomizedrunner')}"
   testCompile 'com.github.tomakehurst:wiremock-jre8-standalone:2.23.2'
+  minimumRuntimeCompile "junit:junit:${props.getProperty('junit')}"
+  minimumRuntimeCompile localGroovy()
+  minimumRuntimeCompile gradleApi()
 }
 
 /*****************************************************************************
@@ -121,10 +149,6 @@ if (project != rootProject) {
   apply plugin: 'nebula.maven-base-publish'
   apply plugin: 'nebula.maven-scm'
 
-  // we need to apply these again to override the build plugin
-  targetCompatibility = "10"
-  sourceCompatibility = "10"
-
   // groovydoc succeeds, but has some weird internal exception...
   groovydoc.enabled = false
 
@@ -132,6 +156,7 @@ if (project != rootProject) {
   dependencyLicenses.enabled = false
   dependenciesInfo.enabled = false
   forbiddenApisMain.enabled = false
+  forbiddenApisMinimumRuntime.enabled = false
   forbiddenApisTest.enabled = false
   jarHell.enabled = false
   thirdPartyAudit.enabled = false
@@ -148,6 +173,11 @@ if (project != rootProject) {
     distribution project(':distribution:archives:linux-tar')
     distribution project(':distribution:archives:oss-linux-tar')
   }
+    
+  // for external projects we want to remove the marker file indicating we are running the Elasticsearch project
+  processResources {
+      exclude 'buildSrc.marker'
+  }
 
   String localDownloads = "${rootProject.buildDir}/local-downloads"
   task setupLocalDownloads(type:Copy) {
@@ -178,6 +208,8 @@ if (project != rootProject) {
       systemProperty 'test.lucene-snapshot-revision', isLuceneSnapshot[0][1]
     }
     maxParallelForks System.getProperty('tests.jvms', project.rootProject.ext.defaultParallel.toString()) as Integer
+    // These tests run Gradle which doesn't have FIPS support
+    onlyIf { project.inFipsJvm == false }
   }
   check.dependsOn(integTest)
 
diff --git a/buildSrc/src/main/groovy/org/elasticsearch/gradle/BuildPlugin.groovy b/buildSrc/src/main/groovy/org/elasticsearch/gradle/BuildPlugin.groovy
index b5c69a418cceb..72af6b8c330f8 100644
--- a/buildSrc/src/main/groovy/org/elasticsearch/gradle/BuildPlugin.groovy
+++ b/buildSrc/src/main/groovy/org/elasticsearch/gradle/BuildPlugin.groovy
@@ -18,6 +18,7 @@
  */
 package org.elasticsearch.gradle
 
+import com.github.jengelman.gradle.plugins.shadow.ShadowExtension
 import com.github.jengelman.gradle.plugins.shadow.ShadowPlugin
 import com.github.jengelman.gradle.plugins.shadow.tasks.ShadowJar
 import groovy.transform.CompileDynamic
@@ -32,6 +33,7 @@ import org.elasticsearch.gradle.precommit.DependencyLicensesTask
 import org.elasticsearch.gradle.precommit.PrecommitTasks
 import org.elasticsearch.gradle.test.ErrorReportingTestListener
 import org.elasticsearch.gradle.testclusters.ElasticsearchCluster
+import org.elasticsearch.gradle.testclusters.TestClustersPlugin
 import org.gradle.api.Action
 import org.gradle.api.GradleException
 import org.gradle.api.InvalidUserDataException
@@ -77,7 +79,6 @@ import org.gradle.authentication.http.HttpHeaderAuthentication
 import org.gradle.external.javadoc.CoreJavadocOptions
 import org.gradle.internal.jvm.Jvm
 import org.gradle.language.base.plugins.LifecycleBasePlugin
-import org.gradle.process.CommandLineArgumentProvider
 import org.gradle.process.ExecResult
 import org.gradle.process.ExecSpec
 import org.gradle.util.GradleVersion
@@ -99,13 +100,17 @@ class BuildPlugin implements Plugin {
         project.rootProject.pluginManager.apply(GlobalBuildInfoPlugin)
 
         if (project.pluginManager.hasPlugin('elasticsearch.standalone-rest-test')) {
-              throw new InvalidUserDataException('elasticsearch.standalone-test, '
-                + 'elasticsearch.standalone-rest-test, and elasticsearch.build '
-                + 'are mutually exclusive')
+            throw new InvalidUserDataException('elasticsearch.standalone-test, '
+                    + 'elasticsearch.standalone-rest-test, and elasticsearch.build '
+                    + 'are mutually exclusive')
         }
         String minimumGradleVersion = null
         InputStream is = getClass().getResourceAsStream("/minimumGradleVersion")
-        try { minimumGradleVersion = IOUtils.toString(is, StandardCharsets.UTF_8.toString()) } finally { is.close() }
+        try {
+            minimumGradleVersion = IOUtils.toString(is, StandardCharsets.UTF_8.toString())
+        } finally {
+            is.close()
+        }
         if (GradleVersion.current() < GradleVersion.version(minimumGradleVersion.trim())) {
             throw new GradleException(
                     "Gradle ${minimumGradleVersion}+ is required to use elasticsearch.build plugin"
@@ -139,23 +144,25 @@ class BuildPlugin implements Plugin {
         configurePrecommit(project)
         configureDependenciesInfo(project)
 
-        // Common config when running with a FIPS-140 runtime JVM
+
+        configureFips140(project)
+    }
+
+    public static void configureFips140(Project project) {
         // Need to do it here to support external plugins
-        if (project == project.rootProject) {
-            GlobalInfoExtension globalInfo = project.extensions.getByType(GlobalInfoExtension)
-
-            // wait until global info is populated because we don't know if we are running in a fips jvm until execution time
-            globalInfo.ready {
-                project.subprojects { Project subproject ->
-                    ExtraPropertiesExtension ext = subproject.extensions.getByType(ExtraPropertiesExtension)
-                    // Common config when running with a FIPS-140 runtime JVM
-                    if (ext.has('inFipsJvm') && ext.get('inFipsJvm')) {
-                        subproject.tasks.withType(Test) { Test task ->
-                            task.systemProperty 'javax.net.ssl.trustStorePassword', 'password'
-                            task.systemProperty 'javax.net.ssl.keyStorePassword', 'password'
-                        }
-                        project.pluginManager.withPlugin("elasticsearch.testclusters") {
-                            NamedDomainObjectContainer testClusters = subproject.extensions.getByName('testClusters') as NamedDomainObjectContainer
+        GlobalInfoExtension globalInfo = project.rootProject.extensions.getByType(GlobalInfoExtension)
+        // wait until global info is populated because we don't know if we are running in a fips jvm until execution time
+        globalInfo.ready {
+                ExtraPropertiesExtension ext = project.extensions.getByType(ExtraPropertiesExtension)
+                // Common config when running with a FIPS-140 runtime JVM
+                if (ext.has('inFipsJvm') && ext.get('inFipsJvm')) {
+                    project.tasks.withType(Test) { Test task ->
+                        task.systemProperty 'javax.net.ssl.trustStorePassword', 'password'
+                        task.systemProperty 'javax.net.ssl.keyStorePassword', 'password'
+                    }
+                    project.pluginManager.withPlugin("elasticsearch.testclusters") {
+                        NamedDomainObjectContainer testClusters = project.extensions.findByName(TestClustersPlugin.EXTENSION_NAME) as NamedDomainObjectContainer
+                        if (testClusters != null) {
                             testClusters.all { ElasticsearchCluster cluster ->
                                 cluster.systemProperty 'javax.net.ssl.trustStorePassword', 'password'
                                 cluster.systemProperty 'javax.net.ssl.keyStorePassword', 'password'
@@ -163,7 +170,6 @@ class BuildPlugin implements Plugin {
                         }
                     }
                 }
-            }
         }
     }
 
@@ -256,15 +262,11 @@ class BuildPlugin implements Plugin {
             }
         }
 
-        if (ext.get('buildDocker')) {
-            (ext.get('requiresDocker') as List).add(task)
-        } else {
-            task.enabled = false
-        }
+        (ext.get('requiresDocker') as List).add(task)
     }
 
     protected static void checkDockerVersionRecent(String dockerVersion) {
-        final Matcher matcher = dockerVersion =~ /Docker version (\d+\.\d+)\.\d+(?:-ce)?, build [0-9a-f]{7,40}/
+        final Matcher matcher = dockerVersion =~ /Docker version (\d+\.\d+)\.\d+(?:-[a-zA-Z0-9]+)?, build [0-9a-f]{7,40}/
         assert matcher.matches(): dockerVersion
         final dockerMajorMinorVersion = matcher.group(1)
         final String[] majorMinor = dockerMajorMinorVersion.split("\\.")
@@ -530,39 +532,43 @@ class BuildPlugin implements Plugin {
     static void configurePomGeneration(Project project) {
         // Only works with  `enableFeaturePreview('STABLE_PUBLISHING')`
         // https://github.com/gradle/gradle/issues/5696#issuecomment-396965185
-        project.tasks.withType(GenerateMavenPom.class) { GenerateMavenPom generatePOMTask ->
-            // The GenerateMavenPom task is aggressive about setting the destination, instead of fighting it,
-            // just make a copy.
-            ExtraPropertiesExtension ext = generatePOMTask.extensions.getByType(ExtraPropertiesExtension)
-            ext.set('pomFileName', null)
-            generatePOMTask.doLast {
-                project.copy { CopySpec spec ->
-                    spec.from generatePOMTask.destination
-                    spec.into "${project.buildDir}/distributions"
-                    spec.rename {
-                        ext.has('pomFileName') && ext.get('pomFileName') == null ?
-                            "${project.convention.getPlugin(BasePluginConvention).archivesBaseName}-${project.version}.pom" :
-                            ext.get('pomFileName')
+        // dummy task to depend on the real pom generation
+        project.plugins.withType(MavenPublishPlugin).whenPluginAdded {
+            Task generatePomTask = project.tasks.create("generatePom")
+            Task assemble = project.tasks.findByName('assemble')
+            if (assemble) {
+                assemble.dependsOn(generatePomTask)
+            }
+            project.tasks.withType(GenerateMavenPom.class) { GenerateMavenPom pomTask ->
+                // The GenerateMavenPom task is aggressive about setting the destination, instead of fighting it,
+                // just make a copy.
+                ExtraPropertiesExtension ext = pomTask.extensions.getByType(ExtraPropertiesExtension)
+                ext.set('pomFileName', null)
+                pomTask.doLast {
+                    project.copy { CopySpec spec ->
+                        spec.from pomTask.destination
+                        spec.into "${project.buildDir}/distributions"
+                        spec.rename {
+                            ext.has('pomFileName') && ext.get('pomFileName') == null ?
+                                    "${project.convention.getPlugin(BasePluginConvention).archivesBaseName}-${project.version}.pom" :
+                                    ext.get('pomFileName')
+                        }
                     }
                 }
             }
-            // build poms with assemble (if the assemble task exists)
-            Task assemble = project.tasks.findByName('assemble')
-            if (assemble && assemble.enabled) {
-                assemble.dependsOn(generatePOMTask)
-            }
-        }
-        project.plugins.withType(MavenPublishPlugin).whenPluginAdded {
+            generatePomTask.dependsOn = ['generatePomFileForNebulaPublication']
             PublishingExtension publishing = project.extensions.getByType(PublishingExtension)
             publishing.publications.all { MavenPublication publication -> // we only deal with maven
                 // add exclusions to the pom directly, for each of the transitive deps of this project's deps
                 publication.pom.withXml(fixupDependencies(project))
             }
             project.plugins.withType(ShadowPlugin).whenPluginAdded {
-                MavenPublication publication = publishing.publications.maybeCreate('nebula', MavenPublication)
+                MavenPublication publication = publishing.publications.maybeCreate('shadow', MavenPublication)
                 publication.with {
-                    artifacts = [ project.tasks.getByName('shadowJar') ]
+                    ShadowExtension shadow = project.extensions.getByType(ShadowExtension)
+                    shadow.component(publication)
                 }
+                generatePomTask.dependsOn = ['generatePomFileForShadowPublication']
             }
         }
     }
@@ -690,6 +696,15 @@ class BuildPlugin implements Plugin {
         project.tasks.withType(Jar) { Jar jarTask ->
             // we put all our distributable files under distributions
             jarTask.destinationDir = new File(project.buildDir, 'distributions')
+            project.plugins.withType(ShadowPlugin).whenPluginAdded {
+                /*
+                 * Ensure the original jar task places its output in 'libs' so that we don't overwrite it with the shadow jar. We only do
+                 * this for tasks named jar to exclude javadoc and sources jars.
+                 */
+                if (jarTask instanceof ShadowJar == false && jarTask.name == JavaPlugin.JAR_TASK_NAME) {
+                    jarTask.destinationDir = new File(project.buildDir, 'libs')
+                }
+            }
             // fixup the jar manifest
             jarTask.doFirst {
                 // this doFirst is added before the info plugin, therefore it will run
@@ -716,10 +731,6 @@ class BuildPlugin implements Plugin {
                         jarTask.manifest.attributes('Change': shortHash)
                     }
                 }
-                // Force manifest entries that change by nature to a constant to be able to compare builds more effectively
-                if (System.properties.getProperty("build.compare_friendly", "false") == "true") {
-                    jarTask.manifest.getAttributes().clear()
-                }
             }
 
             // add license/notice files
@@ -744,12 +755,6 @@ class BuildPlugin implements Plugin {
             }
         }
         project.plugins.withType(ShadowPlugin).whenPluginAdded {
-            /*
-             * When we use the shadow plugin we entirely replace the
-             * normal jar with the shadow jar so we no longer want to run
-             * the jar task.
-             */
-            project.tasks.getByName(JavaPlugin.JAR_TASK_NAME).enabled = false
             project.tasks.getByName('shadowJar').configure { ShadowJar shadowJar ->
                 /*
                  * Replace the default "shadow" classifier with null
@@ -768,7 +773,6 @@ class BuildPlugin implements Plugin {
             }
             // Make sure we assemble the shadow jar
             project.tasks.getByName(BasePlugin.ASSEMBLE_TASK_NAME).dependsOn project.tasks.getByName('shadowJar')
-            project.artifacts.add('apiElements', project.tasks.getByName('shadowJar'))
         }
     }
 
@@ -817,7 +821,7 @@ class BuildPlugin implements Plugin {
                 }
 
                 test.jvmArgumentProviders.add(nonInputProperties)
-                test.extensions.getByType(ExtraPropertiesExtension).set('nonInputProperties', nonInputProperties)
+                test.extensions.add('nonInputProperties', nonInputProperties)
 
                 test.executable = "${ext.get('runtimeJavaHome')}/bin/java"
                 test.workingDir = project.file("${project.buildDir}/testrun/${test.name}")
@@ -842,17 +846,25 @@ class BuildPlugin implements Plugin {
 
                 // we use './temp' since this is per JVM and tests are forbidden from writing to CWD
                 test.systemProperties 'gradle.dist.lib': new File(project.class.location.toURI()).parent,
-                        'gradle.worker.jar': "${project.gradle.getGradleUserHomeDir()}/caches/${project.gradle.gradleVersion}/workerMain/gradle-worker.jar",
-                        'gradle.user.home': project.gradle.getGradleUserHomeDir(),
                         'java.io.tmpdir': './temp',
                         'java.awt.headless': 'true',
                         'tests.gradle': 'true',
                         'tests.artifact': project.name,
                         'tests.task': test.path,
                         'tests.security.manager': 'true',
-                        'tests.seed': project.property('testSeed'),
                         'jna.nosys': 'true'
 
+                // ignore changing test seed when build is passed -Dignore.tests.seed for cacheability experimentation
+                if (System.getProperty('ignore.tests.seed') != null) {
+                    nonInputProperties.systemProperty('tests.seed', project.property('testSeed'))
+                } else {
+                    test.systemProperty('tests.seed', project.property('testSeed'))
+                }
+
+                // don't track these as inputs since they contain absolute paths and break cache relocatability
+                nonInputProperties.systemProperty('gradle.worker.jar', "${project.gradle.getGradleUserHomeDir()}/caches/${project.gradle.gradleVersion}/workerMain/gradle-worker.jar")
+                nonInputProperties.systemProperty('gradle.user.home', project.gradle.getGradleUserHomeDir())
+
                 nonInputProperties.systemProperty('compiler.java', "${-> (ext.get('compilerJavaVersion') as JavaVersion).getMajorVersion()}")
 
                 // TODO: remove setting logging level via system property
@@ -874,6 +886,7 @@ class BuildPlugin implements Plugin {
 
                 project.plugins.withType(ShadowPlugin).whenPluginAdded {
                     // Test against a shadow jar if we made one
+                    test.classpath -= project.configurations.getByName('bundle')
                     test.classpath -= project.tasks.getByName('compileJava').outputs.files
                     test.classpath += project.tasks.getByName('shadowJar').outputs.files
 
@@ -965,19 +978,4 @@ class BuildPlugin implements Plugin {
             })
         }
     }
-
-    private static class SystemPropertyCommandLineArgumentProvider implements CommandLineArgumentProvider {
-        private final Map systemProperties = [:]
-
-        void systemProperty(String key, Object value) {
-            systemProperties.put(key, value)
-        }
-
-        @Override
-        Iterable asArguments() {
-            return systemProperties.collect { key, value ->
-                "-D${key}=${value.toString()}".toString()
-            }
-        }
-    }
 }
diff --git a/buildSrc/src/main/groovy/org/elasticsearch/gradle/doc/SnippetsTask.groovy b/buildSrc/src/main/groovy/org/elasticsearch/gradle/doc/SnippetsTask.groovy
index 83a6a05ec5df7..c1dbddd9e9d52 100644
--- a/buildSrc/src/main/groovy/org/elasticsearch/gradle/doc/SnippetsTask.groovy
+++ b/buildSrc/src/main/groovy/org/elasticsearch/gradle/doc/SnippetsTask.groovy
@@ -43,7 +43,7 @@ public class SnippetsTask extends DefaultTask {
     private static final String SKIP = /skip:([^\]]+)/
     private static final String SETUP = /setup:([^ \]]+)/
     private static final String WARNING = /warning:(.+)/
-    private static final String CAT = /(_cat)/
+    private static final String NON_JSON = /(non_json)/
     private static final String TEST_SYNTAX =
         /(?:$CATCH|$SUBSTITUTION|$SKIP|(continued)|$SETUP|$WARNING|(skip_shard_failures)) ?/
 
@@ -255,12 +255,12 @@ public class SnippetsTask extends DefaultTask {
                             substitutions = []
                         }
                         String loc = "$file:$lineNumber"
-                        parse(loc, matcher.group(2), /(?:$SUBSTITUTION|$CAT|$SKIP) ?/) {
+                        parse(loc, matcher.group(2), /(?:$SUBSTITUTION|$NON_JSON|$SKIP) ?/) {
                             if (it.group(1) != null) {
                                 // TESTRESPONSE[s/adsf/jkl/]
                                 substitutions.add([it.group(1), it.group(2)])
                             } else if (it.group(3) != null) {
-                                // TESTRESPONSE[_cat]
+                                // TESTRESPONSE[non_json]
                                 substitutions.add(['^', '/'])
                                 substitutions.add(['\n$', '\\\\s*/'])
                                 substitutions.add(['( +)', '$1\\\\s+'])
diff --git a/buildSrc/src/main/groovy/org/elasticsearch/gradle/plugin/PluginBuildPlugin.groovy b/buildSrc/src/main/groovy/org/elasticsearch/gradle/plugin/PluginBuildPlugin.groovy
index e04d0966c412d..a25c0415f6483 100644
--- a/buildSrc/src/main/groovy/org/elasticsearch/gradle/plugin/PluginBuildPlugin.groovy
+++ b/buildSrc/src/main/groovy/org/elasticsearch/gradle/plugin/PluginBuildPlugin.groovy
@@ -19,7 +19,6 @@
 package org.elasticsearch.gradle.plugin
 
 import com.github.jengelman.gradle.plugins.shadow.ShadowPlugin
-import nebula.plugin.publishing.maven.MavenScmPlugin
 import org.elasticsearch.gradle.BuildPlugin
 import org.elasticsearch.gradle.NoticeTask
 import org.elasticsearch.gradle.Version
@@ -27,17 +26,16 @@ import org.elasticsearch.gradle.VersionProperties
 import org.elasticsearch.gradle.test.RestIntegTestTask
 import org.elasticsearch.gradle.test.RunTask
 import org.elasticsearch.gradle.testclusters.TestClustersPlugin
+import org.elasticsearch.gradle.tool.ClasspathUtils
 import org.gradle.api.InvalidUserDataException
 import org.gradle.api.Plugin
 import org.gradle.api.Project
 import org.gradle.api.Task
 import org.gradle.api.publish.maven.MavenPublication
 import org.gradle.api.publish.maven.plugins.MavenPublishPlugin
-import org.gradle.api.publish.maven.tasks.GenerateMavenPom
 import org.gradle.api.tasks.Copy
 import org.gradle.api.tasks.SourceSet
 import org.gradle.api.tasks.bundling.Zip
-import org.gradle.jvm.tasks.Jar
 
 import java.util.regex.Matcher
 import java.util.regex.Pattern
@@ -131,31 +129,21 @@ class PluginBuildPlugin implements Plugin {
     }
 
     private void configurePublishing(Project project, PluginPropertiesExtension extension) {
-        // Only configure publishing if applied externally
-        if (extension.hasClientJar) {
-            project.plugins.apply(MavenScmPlugin.class)
-            // Only change Jar tasks, we don't want a -client zip so we can't change archivesBaseName
-            project.tasks.withType(Jar) {
-                baseName = baseName + "-client"
-            }
-            // always configure publishing for client jars
-            project.plugins.apply(MavenScmPlugin.class)
-            project.publishing.publications.nebula(MavenPublication).artifactId(extension.name + "-client")
-            project.tasks.withType(GenerateMavenPom.class) { GenerateMavenPom generatePOMTask ->
-                generatePOMTask.ext.pomFileName = "${project.archivesBaseName}-client-${project.versions.elasticsearch}.pom"
-            }
-        } else {
-            if (project.plugins.hasPlugin(MavenPublishPlugin)) {
-                project.publishing.publications.nebula(MavenPublication).artifactId(extension.name)
-            }
-
+        if (project.plugins.hasPlugin(MavenPublishPlugin)) {
+            project.publishing.publications.nebula(MavenPublication).artifactId(extension.name)
         }
+
     }
 
     private static void configureDependencies(Project project) {
         project.dependencies {
-            compileOnly "org.elasticsearch:elasticsearch:${project.versions.elasticsearch}"
-            testCompile "org.elasticsearch.test:framework:${project.versions.elasticsearch}"
+            if (ClasspathUtils.isElasticsearchProject()) {
+                compileOnly project.project(':server')
+                testCompile project.project(':test:framework')
+            } else {
+                compileOnly "org.elasticsearch:elasticsearch:${project.versions.elasticsearch}"
+                testCompile "org.elasticsearch.test:framework:${project.versions.elasticsearch}"
+            }
             // we "upgrade" these optional deps to provided for plugins, since they will run
             // with a full elasticsearch server that includes optional deps
             compileOnly "org.locationtech.spatial4j:spatial4j:${project.versions.spatial4j}"
@@ -260,8 +248,6 @@ class PluginBuildPlugin implements Plugin {
         project.artifacts.add('zip', bundle)
     }
 
-    /** Adds a task to move jar and associated files to a "-client" name. */
-
     static final Pattern GIT_PATTERN = Pattern.compile(/git@([^:]+):([^\.]+)\.git/)
 
     /** Find the reponame. */
diff --git a/buildSrc/src/main/groovy/org/elasticsearch/gradle/plugin/PluginPropertiesExtension.groovy b/buildSrc/src/main/groovy/org/elasticsearch/gradle/plugin/PluginPropertiesExtension.groovy
deleted file mode 100644
index c250d7695a832..0000000000000
--- a/buildSrc/src/main/groovy/org/elasticsearch/gradle/plugin/PluginPropertiesExtension.groovy
+++ /dev/null
@@ -1,94 +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.gradle.plugin
-
-import org.gradle.api.Project
-import org.gradle.api.tasks.Input
-import org.gradle.api.tasks.InputFile
-
-/**
- * A container for plugin properties that will be written to the plugin descriptor, for easy
- * manipulation in the gradle DSL.
- */
-class PluginPropertiesExtension {
-
-    @Input
-    String name
-
-    @Input
-    String version
-
-    @Input
-    String description
-
-    @Input
-    String classname
-
-    /** Other plugins this plugin extends through SPI */
-    @Input
-    List extendedPlugins = []
-
-    @Input
-    boolean hasNativeController = false
-
-    /** Indicates whether the plugin jar should be made available for the transport client. */
-    @Input
-    boolean hasClientJar = false
-
-    /** True if the plugin requires the elasticsearch keystore to exist, false otherwise. */
-    @Input
-    boolean requiresKeystore = false
-
-    /** A license file that should be included in the built plugin zip. */
-    private File licenseFile = null
-
-    /**
-     * A notice file that should be included in the built plugin zip. This will be
-     * extended with notices from the {@code licenses/} directory.
-     */
-    private File noticeFile = null
-
-    Project project = null
-
-    PluginPropertiesExtension(Project project) {
-        name = project.name
-        version = project.version
-        this.project = project
-    }
-
-    @InputFile
-    File getLicenseFile() {
-        return licenseFile
-    }
-
-    void setLicenseFile(File licenseFile) {
-        project.ext.licenseFile = licenseFile
-        this.licenseFile = licenseFile
-    }
-
-    @InputFile
-    File getNoticeFile() {
-        return noticeFile
-    }
-
-    void setNoticeFile(File noticeFile) {
-        project.ext.noticeFile = noticeFile
-        this.noticeFile = noticeFile
-    }
-}
diff --git a/buildSrc/src/main/groovy/org/elasticsearch/gradle/precommit/DependencyLicensesTask.groovy b/buildSrc/src/main/groovy/org/elasticsearch/gradle/precommit/DependencyLicensesTask.groovy
deleted file mode 100644
index 04fb023e2051a..0000000000000
--- a/buildSrc/src/main/groovy/org/elasticsearch/gradle/precommit/DependencyLicensesTask.groovy
+++ /dev/null
@@ -1,268 +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.gradle.precommit
-
-import org.gradle.api.DefaultTask
-import org.gradle.api.GradleException
-import org.gradle.api.InvalidUserDataException
-import org.gradle.api.file.FileCollection
-import org.gradle.api.tasks.Input
-import org.gradle.api.tasks.InputDirectory
-import org.gradle.api.tasks.InputFiles
-import org.gradle.api.tasks.TaskAction
-
-import java.nio.file.Files
-import java.security.MessageDigest
-import java.util.regex.Matcher
-import java.util.regex.Pattern
-
-/**
- * A task to check licenses for dependencies.
- *
- * There are two parts to the check:
- * 
    - *
  • LICENSE and NOTICE files
  • - *
  • SHA checksums for each dependency jar
  • - *
- * - * The directory to find the license and sha files in defaults to the dir @{code licenses} - * in the project directory for this task. You can override this directory: - *
- *   dependencyLicenses {
- *     licensesDir = project.file('mybetterlicensedir')
- *   }
- * 
- * - * The jar files to check default to the dependencies from the default configuration. You - * can override this, for example, to only check compile dependencies: - *
- *   dependencyLicenses {
- *     dependencies = project.configurations.compile
- *   }
- * 
- * - * Every jar must have a {@code .sha1} file in the licenses dir. These can be managed - * automatically using the {@code updateShas} helper task that is created along - * with this task. It will add {@code .sha1} files for new jars that are in dependencies - * and remove old {@code .sha1} files that are no longer needed. - * - * Every jar must also have a LICENSE and NOTICE file. However, multiple jars can share - * LICENSE and NOTICE files by mapping a pattern to the same name. - *
- *   dependencyLicenses {
- *     mapping from: /lucene-.*/, to: 'lucene'
- *   }
- * 
- */ -public class DependencyLicensesTask extends DefaultTask { - static final String SHA_EXTENSION = '.sha1' - - // TODO: we should be able to default this to eg compile deps, but we need to move the licenses - // check from distribution to core (ie this should only be run on java projects) - /** A collection of jar files that should be checked. */ - @InputFiles - public FileCollection dependencies - - /** The directory to find the license and sha files in. */ - @InputDirectory - public File licensesDir = new File(project.projectDir, 'licenses') - - /** A map of patterns to prefix, used to find the LICENSE and NOTICE file. */ - private LinkedHashMap mappings = new LinkedHashMap<>() - - /** Names of dependencies whose shas should not exist. */ - private Set ignoreShas = new HashSet<>() - - /** - * Add a mapping from a regex pattern for the jar name, to a prefix to find - * the LICENSE and NOTICE file for that jar. - */ - @Input - public void mapping(Map props) { - String from = props.remove('from') - if (from == null) { - throw new InvalidUserDataException('Missing "from" setting for license name mapping') - } - String to = props.remove('to') - if (to == null) { - throw new InvalidUserDataException('Missing "to" setting for license name mapping') - } - if (props.isEmpty() == false) { - throw new InvalidUserDataException("Unknown properties for mapping on dependencyLicenses: ${props.keySet()}") - } - mappings.put(from, to) - } - - public LinkedHashMap getMappings() { - return new LinkedHashMap<>(mappings) - } - - /** - * Add a rule which will skip SHA checking for the given dependency name. This should be used for - * locally build dependencies, which cause the sha to change constantly. - */ - @Input - public void ignoreSha(String dep) { - ignoreShas.add(dep) - } - - @TaskAction - public void checkDependencies() { - if (dependencies.isEmpty()) { - if (licensesDir.exists()) { - throw new GradleException("Licenses dir ${licensesDir} exists, but there are no dependencies") - } - return // no dependencies to check - } else if (licensesDir.exists() == false) { - throw new GradleException("Licences dir ${licensesDir} does not exist, but there are dependencies") - } - - Map licenses = new HashMap<>() - Map notices = new HashMap<>() - Set shaFiles = new HashSet() - - licensesDir.eachFile { - String name = it.getName() - if (name.endsWith(SHA_EXTENSION)) { - shaFiles.add(it) - } else if (name.endsWith('-LICENSE') || name.endsWith('-LICENSE.txt')) { - // TODO: why do we support suffix of LICENSE *and* LICENSE.txt?? - licenses.put(name, 0) - } else if (name.contains('-NOTICE') || name.contains('-NOTICE.txt')) { - notices.put(name, 0) - } - } - - for (File dependency : dependencies) { - String jarName = dependency.getName() - String depName = jarName - ~/\-v?\d+.*/ - if (ignoreShas.contains(depName)) { - // local deps should not have sha files! - if (getShaFile(jarName).exists()) { - throw new GradleException("SHA file ${getShaFile(jarName)} exists for ignored dependency ${depName}") - } - } else { - logger.info("Checking sha for " + jarName) - checkSha(dependency, jarName, shaFiles) - } - - final String dependencyName = getDependencyName(mappings, depName) - logger.info("mapped dependency name ${depName} to ${dependencyName} for license/notice check") - checkFile(dependencyName, jarName, licenses, 'LICENSE') - checkFile(dependencyName, jarName, notices, 'NOTICE') - } - - licenses.each { license, count -> - if (count == 0) { - throw new GradleException("Unused license ${license}") - } - } - notices.each { notice, count -> - if (count == 0) { - throw new GradleException("Unused notice ${notice}") - } - } - if (shaFiles.isEmpty() == false) { - throw new GradleException("Unused sha files found: \n${shaFiles.join('\n')}") - } - } - - public static String getDependencyName(final LinkedHashMap mappings, final String dependencyName) { - // order is the same for keys and values iteration since we use a linked hashmap - List mapped = new ArrayList<>(mappings.values()) - Pattern mappingsPattern = Pattern.compile('(' + mappings.keySet().join(')|(') + ')') - Matcher match = mappingsPattern.matcher(dependencyName) - if (match.matches()) { - int i = 0 - while (i < match.groupCount() && match.group(i + 1) == null) ++i; - return mapped.get(i) - } - return dependencyName - } - - private File getShaFile(String jarName) { - return new File(licensesDir, jarName + SHA_EXTENSION) - } - - private void checkSha(File jar, String jarName, Set shaFiles) { - File shaFile = getShaFile(jarName) - if (shaFile.exists() == false) { - throw new GradleException("Missing SHA for ${jarName}. Run 'gradle updateSHAs' to create") - } - // TODO: shouldn't have to trim, sha files should not have trailing newline - String expectedSha = shaFile.getText('UTF-8').trim() - String sha = MessageDigest.getInstance("SHA-1").digest(jar.getBytes()).encodeHex().toString() - if (expectedSha.equals(sha) == false) { - throw new GradleException("SHA has changed! Expected ${expectedSha} for ${jarName} but got ${sha}. " + - "\nThis usually indicates a corrupt dependency cache or artifacts changed upstream." + - "\nEither wipe your cache, fix the upstream artifact, or delete ${shaFile} and run updateShas") - } - shaFiles.remove(shaFile) - } - - private void checkFile(String name, String jarName, Map counters, String type) { - String fileName = "${name}-${type}" - Integer count = counters.get(fileName) - if (count == null) { - // try the other suffix...TODO: get rid of this, just support ending in .txt - fileName = "${fileName}.txt" - counters.get(fileName) - } - count = counters.get(fileName) - if (count == null) { - throw new GradleException("Missing ${type} for ${jarName}, expected in ${fileName}") - } - counters.put(fileName, count + 1) - } - - /** A helper task to update the sha files in the license dir. */ - public static class UpdateShasTask extends DefaultTask { - private DependencyLicensesTask parentTask - - @TaskAction - public void updateShas() { - Set shaFiles = new HashSet() - parentTask.licensesDir.eachFile { - String name = it.getName() - if (name.endsWith(SHA_EXTENSION)) { - shaFiles.add(it) - } - } - for (File dependency : parentTask.dependencies) { - String jarName = dependency.getName() - String depName = jarName - ~/\-\d+.*/ - if (parentTask.ignoreShas.contains(depName)) { - continue - } - File shaFile = new File(parentTask.licensesDir, jarName + SHA_EXTENSION) - if (shaFile.exists() == false) { - logger.lifecycle("Adding sha for ${jarName}") - String sha = MessageDigest.getInstance("SHA-1").digest(dependency.getBytes()).encodeHex().toString() - shaFile.setText(sha, 'UTF-8') - } else { - shaFiles.remove(shaFile) - } - } - shaFiles.each { shaFile -> - logger.lifecycle("Removing unused sha ${shaFile.getName()}") - Files.delete(shaFile.toPath()) - } - } - } -} diff --git a/buildSrc/src/main/groovy/org/elasticsearch/gradle/precommit/PrecommitTasks.groovy b/buildSrc/src/main/groovy/org/elasticsearch/gradle/precommit/PrecommitTasks.groovy index f656f177ce67b..a5d4f3fcd9495 100644 --- a/buildSrc/src/main/groovy/org/elasticsearch/gradle/precommit/PrecommitTasks.groovy +++ b/buildSrc/src/main/groovy/org/elasticsearch/gradle/precommit/PrecommitTasks.groovy @@ -23,11 +23,13 @@ import de.thetaphi.forbiddenapis.gradle.CheckForbiddenApis import de.thetaphi.forbiddenapis.gradle.ForbiddenApisPlugin import org.elasticsearch.gradle.ExportElasticsearchBuildResourcesTask import org.elasticsearch.gradle.VersionProperties +import org.elasticsearch.gradle.tool.ClasspathUtils import org.gradle.api.JavaVersion import org.gradle.api.Project import org.gradle.api.Task import org.gradle.api.plugins.JavaBasePlugin import org.gradle.api.plugins.quality.Checkstyle + /** * Validation tasks which should be run before committing. These run before tests. */ @@ -40,18 +42,18 @@ class PrecommitTasks { public static Task create(Project project, boolean includeDependencyLicenses) { project.configurations.create("forbiddenApisCliJar") project.dependencies { - forbiddenApisCliJar ('de.thetaphi:forbiddenapis:2.6') + forbiddenApisCliJar('de.thetaphi:forbiddenapis:2.6') } List precommitTasks = [ - configureCheckstyle(project), - configureForbiddenApisCli(project), - project.tasks.create('forbiddenPatterns', ForbiddenPatternsTask.class), - project.tasks.create('licenseHeaders', LicenseHeadersTask.class), - project.tasks.create('filepermissions', FilePermissionsTask.class), - configureJarHell(project), - configureThirdPartyAudit(project), - configureTestingConventions(project) + configureCheckstyle(project), + configureForbiddenApisCli(project), + project.tasks.create('forbiddenPatterns', ForbiddenPatternsTask.class), + project.tasks.create('licenseHeaders', LicenseHeadersTask.class), + project.tasks.create('filepermissions', FilePermissionsTask.class), + configureJarHell(project), + configureThirdPartyAudit(project), + configureTestingConventions(project) ] // tasks with just tests don't need dependency licenses, so this flag makes adding @@ -85,10 +87,10 @@ class PrecommitTasks { } return project.tasks.create([ - name: 'precommit', - group: JavaBasePlugin.VERIFICATION_GROUP, - description: 'Runs all non-test checks.', - dependsOn: precommitTasks + name : 'precommit', + group : JavaBasePlugin.VERIFICATION_GROUP, + description: 'Runs all non-test checks.', + dependsOn : precommitTasks ]) } @@ -168,7 +170,7 @@ class PrecommitTasks { ) } } - Task forbiddenApis = project.tasks.getByName("forbiddenApis") + Task forbiddenApis = project.tasks.getByName("forbiddenApis") forbiddenApis.group = "" return forbiddenApis } @@ -211,7 +213,7 @@ class PrecommitTasks { project.checkstyle { config = project.resources.text.fromFile(checkstyleConf, 'UTF-8') configProperties = [ - suppressions: checkstyleSuppressions + suppressions: checkstyleSuppressions ] toolVersion = CHECKSTYLE_VERSION } @@ -229,9 +231,11 @@ class PrecommitTasks { } private static Task configureLoggerUsage(Project project) { + Object dependency = ClasspathUtils.isElasticsearchProject() ? project.project(':test:logger-usage') : + "org.elasticsearch.test:logger-usage:${VersionProperties.elasticsearch}" + project.configurations.create('loggerUsagePlugin') - project.dependencies.add('loggerUsagePlugin', - "org.elasticsearch.test:logger-usage:${VersionProperties.elasticsearch}") + project.dependencies.add('loggerUsagePlugin', dependency) return project.tasks.create('loggerUsageCheck', LoggerUsageTask.class) { classpath = project.configurations.loggerUsagePlugin } diff --git a/buildSrc/src/main/groovy/org/elasticsearch/gradle/precommit/UpdateShasTask.groovy b/buildSrc/src/main/groovy/org/elasticsearch/gradle/precommit/UpdateShasTask.groovy deleted file mode 100644 index 4a174688aa1e5..0000000000000 --- a/buildSrc/src/main/groovy/org/elasticsearch/gradle/precommit/UpdateShasTask.groovy +++ /dev/null @@ -1,66 +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.gradle.precommit - -import org.gradle.api.DefaultTask -import org.gradle.api.tasks.TaskAction - -import java.nio.file.Files -import java.security.MessageDigest - -/** - * A task to update shas used by {@code DependencyLicensesCheck} - */ -public class UpdateShasTask extends DefaultTask { - - /** The parent dependency licenses task to use configuration from */ - public DependencyLicensesTask parentTask - - public UpdateShasTask() { - description = 'Updates the sha files for the dependencyLicenses check' - onlyIf { parentTask.licensesDir.exists() } - } - - @TaskAction - public void updateShas() { - Set shaFiles = new HashSet() - parentTask.licensesDir.eachFile { - String name = it.getName() - if (name.endsWith(DependencyLicensesTask.SHA_EXTENSION)) { - shaFiles.add(it) - } - } - for (File dependency : parentTask.dependencies) { - String jarName = dependency.getName() - File shaFile = new File(parentTask.licensesDir, jarName + DependencyLicensesTask.SHA_EXTENSION) - if (shaFile.exists() == false) { - logger.lifecycle("Adding sha for ${jarName}") - String sha = MessageDigest.getInstance("SHA-1").digest(dependency.getBytes()).encodeHex().toString() - shaFile.setText(sha, 'UTF-8') - } else { - shaFiles.remove(shaFile) - } - } - shaFiles.each { shaFile -> - logger.lifecycle("Removing unused sha ${shaFile.getName()}") - Files.delete(shaFile.toPath()) - } - } -} diff --git a/buildSrc/src/main/groovy/org/elasticsearch/gradle/test/ClusterFormationTasks.groovy b/buildSrc/src/main/groovy/org/elasticsearch/gradle/test/ClusterFormationTasks.groovy index bc5c7ff0871bb..f70f258b80a7d 100644 --- a/buildSrc/src/main/groovy/org/elasticsearch/gradle/test/ClusterFormationTasks.groovy +++ b/buildSrc/src/main/groovy/org/elasticsearch/gradle/test/ClusterFormationTasks.groovy @@ -657,8 +657,8 @@ class ClusterFormationTasks { static Task configureExecTask(String name, Project project, Task setup, NodeInfo node, Object[] execArgs) { return project.tasks.create(name: name, type: LoggedExec, dependsOn: setup) { Exec exec -> exec.workingDir node.cwd - if (project.isRuntimeJavaHomeSet || node.nodeVersion.before(Version.fromString("7.0.0")) || - node.config.distribution == 'integ-test-zip') { + if ((project.isRuntimeJavaHomeSet && node.isBwcNode == false) // runtime Java might not be compatible with old nodes + || node.config.distribution == 'integ-test-zip') { exec.environment.put('JAVA_HOME', project.runtimeJavaHome) } else { // force JAVA_HOME to *not* be set @@ -683,8 +683,8 @@ class ClusterFormationTasks { ant.exec(executable: node.executable, spawn: node.config.daemonize, newenvironment: true, dir: node.cwd, taskname: 'elasticsearch') { node.env.each { key, value -> env(key: key, value: value) } - if (project.isRuntimeJavaHomeSet || node.nodeVersion.before(Version.fromString("7.0.0")) || - node.config.distribution == 'integ-test-zip') { + if ((project.isRuntimeJavaHomeSet && node.isBwcNode == false) // runtime Java might not be compatible with old nodes + || node.config.distribution == 'integ-test-zip') { env(key: 'JAVA_HOME', value: project.runtimeJavaHome) } node.args.each { arg(value: it) } diff --git a/buildSrc/src/main/groovy/org/elasticsearch/gradle/test/MessyTestPlugin.groovy b/buildSrc/src/main/groovy/org/elasticsearch/gradle/test/MessyTestPlugin.groovy deleted file mode 100644 index 1c0aec1bc00f3..0000000000000 --- a/buildSrc/src/main/groovy/org/elasticsearch/gradle/test/MessyTestPlugin.groovy +++ /dev/null @@ -1,63 +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.gradle.test - -import org.elasticsearch.gradle.plugin.PluginBuildPlugin -import org.gradle.api.Project -import org.gradle.api.artifacts.Dependency -import org.gradle.api.artifacts.ProjectDependency -import org.gradle.api.tasks.Copy - -/** - * A plugin to run messy tests, which are generally tests that depend on plugins. - * - * This plugin will add the same test configuration as standalone tests, except - * also add the plugin-metadata and properties files for each plugin project - * dependency. - */ -class MessyTestPlugin extends StandaloneTestPlugin { - @Override - public void apply(Project project) { - super.apply(project) - - project.configurations.testCompile.dependencies.all { Dependency dep -> - // this closure is run every time a compile dependency is added - if (dep instanceof ProjectDependency && dep.dependencyProject.plugins.hasPlugin(PluginBuildPlugin)) { - project.gradle.projectsEvaluated { - addPluginResources(project, dep.dependencyProject) - } - } - } - } - - private static addPluginResources(Project project, Project pluginProject) { - String outputDir = "${project.buildDir}/generated-resources/${pluginProject.name}" - String taskName = ClusterFormationTasks.pluginTaskName("copy", pluginProject.name, "Metadata") - Copy copyPluginMetadata = project.tasks.create(taskName, Copy.class) - copyPluginMetadata.into(outputDir) - copyPluginMetadata.from(pluginProject.tasks.pluginProperties) - copyPluginMetadata.from(pluginProject.file('src/main/plugin-metadata')) - project.sourceSets.test.output.dir(outputDir, builtBy: taskName) - - // add each generated dir to the test classpath in IDEs - project.idea.module.singleEntryLibraries= ['TEST': [project.file(outputDir)]] - // Eclipse doesn't need this because it gets the entire module as a dependency - } -} diff --git a/buildSrc/src/main/groovy/org/elasticsearch/gradle/test/NodeInfo.groovy b/buildSrc/src/main/groovy/org/elasticsearch/gradle/test/NodeInfo.groovy index ae365038ccf3f..cb7a8397ed011 100644 --- a/buildSrc/src/main/groovy/org/elasticsearch/gradle/test/NodeInfo.groovy +++ b/buildSrc/src/main/groovy/org/elasticsearch/gradle/test/NodeInfo.groovy @@ -23,6 +23,7 @@ import com.sun.jna.Native import com.sun.jna.WString import org.apache.tools.ant.taskdefs.condition.Os import org.elasticsearch.gradle.Version +import org.elasticsearch.gradle.VersionProperties import org.gradle.api.Project import java.nio.file.Files @@ -107,6 +108,9 @@ class NodeInfo { /** the version of elasticsearch that this node runs */ Version nodeVersion + /** true if the node is not the current version */ + boolean isBwcNode + /** Holds node configuration for part of a test cluster. */ NodeInfo(ClusterConfiguration config, int nodeNum, Project project, String prefix, String nodeVersion, File sharedDir) { this.config = config @@ -121,6 +125,7 @@ class NodeInfo { baseDir = new File(project.buildDir, "cluster/${prefix} node${nodeNum}") pidFile = new File(baseDir, 'es.pid') this.nodeVersion = Version.fromString(nodeVersion) + this.isBwcNode = this.nodeVersion.before(VersionProperties.elasticsearch) homeDir = new File(baseDir, "elasticsearch-${nodeVersion}") pathConf = new File(homeDir, 'config') if (config.dataDir != null) { diff --git a/buildSrc/src/main/groovy/org/elasticsearch/gradle/test/RestIntegTestTask.groovy b/buildSrc/src/main/groovy/org/elasticsearch/gradle/test/RestIntegTestTask.groovy index 40cefdcc25fb9..9857a1bc29ed7 100644 --- a/buildSrc/src/main/groovy/org/elasticsearch/gradle/test/RestIntegTestTask.groovy +++ b/buildSrc/src/main/groovy/org/elasticsearch/gradle/test/RestIntegTestTask.groovy @@ -26,13 +26,13 @@ import org.gradle.api.Task import org.gradle.api.execution.TaskExecutionAdapter import org.gradle.api.logging.Logger import org.gradle.api.logging.Logging +import org.gradle.api.specs.Specs import org.gradle.api.tasks.Copy import org.gradle.api.tasks.Input import org.gradle.api.tasks.TaskState import org.gradle.api.tasks.options.Option import org.gradle.api.tasks.testing.Test import org.gradle.plugins.ide.idea.IdeaPlugin -import org.gradle.process.CommandLineArgumentProvider import java.nio.charset.StandardCharsets import java.nio.file.Files @@ -59,7 +59,7 @@ class RestIntegTestTask extends DefaultTask { Boolean includePackaged = false RestIntegTestTask() { - runner = project.tasks.create("${name}Runner", Test.class) + runner = project.tasks.create("${name}Runner", RestTestRunnerTask.class) super.dependsOn(runner) clusterInit = project.tasks.create(name: "${name}Cluster#init", dependsOn: project.testClasses) runner.dependsOn(clusterInit) @@ -77,10 +77,6 @@ class RestIntegTestTask extends DefaultTask { runner.useCluster project.testClusters."$name" } - // disable the build cache for rest test tasks - // there are a number of inputs we aren't properly tracking here so we'll just not cache these for now - runner.outputs.doNotCacheIf('Caching is disabled for REST integration tests') { true } - // override/add more for rest tests runner.maxParallelForks = 1 runner.include('**/*IT.class') @@ -252,7 +248,7 @@ class RestIntegTestTask extends DefaultTask { restSpec } project.dependencies { - restSpec "org.elasticsearch:rest-api-spec:${VersionProperties.elasticsearch}" + restSpec project.project(':rest-api-spec') } Task copyRestSpec = project.tasks.findByName('copyRestSpec') if (copyRestSpec != null) { @@ -285,4 +281,5 @@ class RestIntegTestTask extends DefaultTask { } return copyRestSpec } + } diff --git a/buildSrc/src/main/groovy/org/elasticsearch/gradle/test/StandaloneRestTestPlugin.groovy b/buildSrc/src/main/groovy/org/elasticsearch/gradle/test/StandaloneRestTestPlugin.groovy index c9a26eb74b54d..cfe610fa568c3 100644 --- a/buildSrc/src/main/groovy/org/elasticsearch/gradle/test/StandaloneRestTestPlugin.groovy +++ b/buildSrc/src/main/groovy/org/elasticsearch/gradle/test/StandaloneRestTestPlugin.groovy @@ -25,6 +25,7 @@ import groovy.transform.CompileStatic import org.elasticsearch.gradle.BuildPlugin import org.elasticsearch.gradle.ExportElasticsearchBuildResourcesTask import org.elasticsearch.gradle.VersionProperties +import org.elasticsearch.gradle.info.GlobalBuildInfoPlugin import org.elasticsearch.gradle.precommit.PrecommitTasks import org.gradle.api.InvalidUserDataException import org.gradle.api.JavaVersion @@ -57,12 +58,14 @@ class StandaloneRestTestPlugin implements Plugin { + 'elasticsearch.standalone-rest-test, and elasticsearch.build ' + 'are mutually exclusive') } + project.rootProject.pluginManager.apply(GlobalBuildInfoPlugin) project.pluginManager.apply(JavaBasePlugin) project.getTasks().create("buildResources", ExportElasticsearchBuildResourcesTask) BuildPlugin.configureRepositories(project) BuildPlugin.configureTestTasks(project) BuildPlugin.configureInputNormalization(project) + BuildPlugin.configureFips140(project) ExtraPropertiesExtension ext = project.extensions.getByType(ExtraPropertiesExtension) project.extensions.getByType(JavaPluginExtension).sourceCompatibility = ext.get('minimumRuntimeVersion') as JavaVersion @@ -79,7 +82,7 @@ class StandaloneRestTestPlugin implements Plugin { // create a compileOnly configuration as others might expect it project.configurations.create("compileOnly") - project.dependencies.add('testCompile', "org.elasticsearch.test:framework:${VersionProperties.elasticsearch}") + project.dependencies.add('testCompile', project.project(':test:framework')) EclipseModel eclipse = project.extensions.getByType(EclipseModel) eclipse.classpath.sourceSets = [testSourceSet] diff --git a/buildSrc/src/main/java/org/elasticsearch/gradle/AbstractLazyPropertyCollection.java b/buildSrc/src/main/java/org/elasticsearch/gradle/AbstractLazyPropertyCollection.java new file mode 100644 index 0000000000000..94cdb091eac1b --- /dev/null +++ b/buildSrc/src/main/java/org/elasticsearch/gradle/AbstractLazyPropertyCollection.java @@ -0,0 +1,27 @@ +package org.elasticsearch.gradle; + +import java.util.List; + +public abstract class AbstractLazyPropertyCollection { + + final String name; + final Object owner; + + public AbstractLazyPropertyCollection(String name) { + this(name, null); + } + + public AbstractLazyPropertyCollection(String name, Object owner) { + this.name = name; + this.owner = owner; + } + + abstract List getNormalizedCollection(); + + void assertNotNull(Object value, String description) { + if (value == null) { + throw new NullPointerException(name + " " + description + " was null" + (owner != null ? " when configuring " + owner : "")); + } + } + +} diff --git a/buildSrc/src/main/java/org/elasticsearch/gradle/BwcVersions.java b/buildSrc/src/main/java/org/elasticsearch/gradle/BwcVersions.java index daa8c4130dd3c..b5f1aa5f551dc 100644 --- a/buildSrc/src/main/java/org/elasticsearch/gradle/BwcVersions.java +++ b/buildSrc/src/main/java/org/elasticsearch/gradle/BwcVersions.java @@ -109,7 +109,7 @@ public BwcVersions(List versionLines) { } protected BwcVersions(List versionLines, Version currentVersionProperty) { - SortedSet allVersions = versionLines.stream() + this(versionLines.stream() .map(LINE_PATTERN::matcher) .filter(Matcher::matches) .map(match -> new Version( @@ -117,8 +117,11 @@ protected BwcVersions(List versionLines, Version currentVersionProperty) Integer.parseInt(match.group(2)), Integer.parseInt(match.group(3)) )) - .collect(Collectors.toCollection(TreeSet::new)); + .collect(Collectors.toCollection(TreeSet::new)), currentVersionProperty); + } + // for testkit tests, until BwcVersions is extracted into an extension + public BwcVersions(SortedSet allVersions, Version currentVersionProperty) { if (allVersions.isEmpty()) { throw new IllegalArgumentException("Could not parse any versions"); } diff --git a/buildSrc/src/main/java/org/elasticsearch/gradle/DistributionDownloadPlugin.java b/buildSrc/src/main/java/org/elasticsearch/gradle/DistributionDownloadPlugin.java new file mode 100644 index 0000000000000..28748c00f46e1 --- /dev/null +++ b/buildSrc/src/main/java/org/elasticsearch/gradle/DistributionDownloadPlugin.java @@ -0,0 +1,275 @@ +/* + * 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.gradle; + +import org.elasticsearch.gradle.ElasticsearchDistribution.Flavor; +import org.elasticsearch.gradle.ElasticsearchDistribution.Platform; +import org.elasticsearch.gradle.ElasticsearchDistribution.Type; +import org.gradle.api.GradleException; +import org.gradle.api.NamedDomainObjectContainer; +import org.gradle.api.Plugin; +import org.gradle.api.Project; +import org.gradle.api.UnknownTaskException; +import org.gradle.api.artifacts.Configuration; +import org.gradle.api.artifacts.ConfigurationContainer; +import org.gradle.api.artifacts.Dependency; +import org.gradle.api.artifacts.dsl.DependencyHandler; +import org.gradle.api.artifacts.repositories.IvyArtifactRepository; +import org.gradle.api.credentials.HttpHeaderCredentials; +import org.gradle.api.file.FileTree; +import org.gradle.api.plugins.ExtraPropertiesExtension; +import org.gradle.api.tasks.Sync; +import org.gradle.api.tasks.TaskProvider; +import org.gradle.authentication.http.HttpHeaderAuthentication; + +import java.io.File; +import java.util.HashMap; +import java.util.Locale; +import java.util.Map; +import java.util.concurrent.Callable; +import java.util.function.Supplier; + +/** + * A plugin to manage getting and extracting distributions of Elasticsearch. + * + * The source of the distribution could be from a local snapshot, a locally built + * bwc snapshot, or the Elastic downloads service. + */ +public class DistributionDownloadPlugin implements Plugin { + + private static final String FAKE_IVY_GROUP = "elasticsearch-distribution"; + private static final String DOWNLOAD_REPO_NAME = "elasticsearch-downloads"; + + private BwcVersions bwcVersions; + private NamedDomainObjectContainer distributionsContainer; + + @Override + public void apply(Project project) { + distributionsContainer = project.container(ElasticsearchDistribution.class, name -> { + Configuration fileConfiguration = project.getConfigurations().create("es_distro_file_" + name); + Configuration extractedConfiguration = project.getConfigurations().create("es_distro_extracted_" + name); + return new ElasticsearchDistribution(name, project.getObjects(), fileConfiguration, extractedConfiguration); + }); + project.getExtensions().add("elasticsearch_distributions", distributionsContainer); + + setupDownloadServiceRepo(project); + + ExtraPropertiesExtension extraProperties = project.getExtensions().getExtraProperties(); + this.bwcVersions = (BwcVersions) extraProperties.get("bwcVersions"); + // TODO: setup snapshot dependency instead of pointing to bwc distribution projects for external projects + + project.afterEvaluate(this::setupDistributions); + } + + // pkg private for tests + void setupDistributions(Project project) { + for (ElasticsearchDistribution distribution : distributionsContainer) { + distribution.finalizeValues(); + + DependencyHandler dependencies = project.getDependencies(); + // for the distribution as a file, just depend on the artifact directly + dependencies.add(distribution.configuration.getName(), dependencyNotation(project, distribution)); + + // no extraction allowed for rpm or deb + if (distribution.getType() != Type.RPM && distribution.getType() != Type.DEB) { + // for the distribution extracted, add a root level task that does the extraction, and depend on that + // extracted configuration as an artifact consisting of the extracted distribution directory + dependencies.add(distribution.getExtracted().configuration.getName(), + projectDependency(project, ":", configName("extracted_elasticsearch", distribution))); + // ensure a root level download task exists + setupRootDownload(project.getRootProject(), distribution); + } + } + } + + private void setupRootDownload(Project rootProject, ElasticsearchDistribution distribution) { + String extractTaskName = extractTaskName(distribution); + // NOTE: this is *horrendous*, but seems to be the only way to check for the existence of a registered task + try { + rootProject.getTasks().named(extractTaskName); + // already setup this version + return; + } catch (UnknownTaskException e) { + // fall through: register the task + } + setupDownloadServiceRepo(rootProject); + + final ConfigurationContainer configurations = rootProject.getConfigurations(); + String downloadConfigName = configName("elasticsearch", distribution); + String extractedConfigName = "extracted_" + downloadConfigName; + final Configuration downloadConfig = configurations.create(downloadConfigName); + configurations.create(extractedConfigName); + rootProject.getDependencies().add(downloadConfigName, dependencyNotation(rootProject, distribution)); + + // add task for extraction, delaying resolving config until runtime + if (distribution.getType() == Type.ARCHIVE || distribution.getType() == Type.INTEG_TEST_ZIP) { + Supplier archiveGetter = downloadConfig::getSingleFile; + String extractDir = rootProject.getBuildDir().toPath().resolve("elasticsearch-distros").resolve(extractedConfigName).toString(); + TaskProvider extractTask = rootProject.getTasks().register(extractTaskName, Sync.class, syncTask -> { + syncTask.dependsOn(downloadConfig); + syncTask.into(extractDir); + syncTask.from((Callable)() -> { + File archiveFile = archiveGetter.get(); + String archivePath = archiveFile.toString(); + if (archivePath.endsWith(".zip")) { + return rootProject.zipTree(archiveFile); + } else if (archivePath.endsWith(".tar.gz")) { + return rootProject.tarTree(rootProject.getResources().gzip(archiveFile)); + } + throw new IllegalStateException("unexpected file extension on [" + archivePath + "]"); + }); + }); + rootProject.getArtifacts().add(extractedConfigName, + rootProject.getLayout().getProjectDirectory().dir(extractDir), + artifact -> artifact.builtBy(extractTask)); + } + } + + private static void setupDownloadServiceRepo(Project project) { + if (project.getRepositories().findByName(DOWNLOAD_REPO_NAME) != null) { + return; + } + project.getRepositories().ivy(ivyRepo -> { + ivyRepo.setName(DOWNLOAD_REPO_NAME); + ivyRepo.setUrl("https://artifacts.elastic.co"); + ivyRepo.metadataSources(IvyArtifactRepository.MetadataSources::artifact); + // this header is not a credential but we hack the capability to send this header to avoid polluting our download stats + ivyRepo.credentials(HttpHeaderCredentials.class, creds -> { + creds.setName("X-Elastic-No-KPI"); + creds.setValue("1"); + }); + ivyRepo.getAuthentication().create("header", HttpHeaderAuthentication.class); + ivyRepo.patternLayout(layout -> layout.artifact("/downloads/elasticsearch/[module]-[revision](-[classifier]).[ext]")); + ivyRepo.content(content -> content.includeGroup(FAKE_IVY_GROUP)); + }); + project.getRepositories().all(repo -> { + if (repo.getName().equals(DOWNLOAD_REPO_NAME) == false) { + // all other repos should ignore the special group name + repo.content(content -> content.excludeGroup(FAKE_IVY_GROUP)); + } + }); + // TODO: need maven repo just for integ-test-zip, but only in external cases + } + + /** + * Returns a dependency object representing the given distribution. + * + * The returned object is suitable to be passed to {@link DependencyHandler}. + * The concrete type of the object will either be a project {@link Dependency} or + * a set of maven coordinates as a {@link String}. Project dependencies point to + * a project in the Elasticsearch repo either under `:distribution:bwc`, + * `:distribution:archives` or :distribution:packages`. Maven coordinates point to + * either the integ-test-zip coordinates on maven central, or a set of artificial + * coordinates that resolve to the Elastic download service through an ivy repository. + */ + private Object dependencyNotation(Project project, ElasticsearchDistribution distribution) { + + if (Version.fromString(VersionProperties.getElasticsearch()).equals(distribution.getVersion())) { + return projectDependency(project, distributionProjectPath(distribution), "default"); + // TODO: snapshot dep when not in ES repo + } + BwcVersions.UnreleasedVersionInfo unreleasedInfo = bwcVersions.unreleasedInfo(distribution.getVersion()); + if (unreleasedInfo != null) { + assert distribution.getBundledJdk(); + return projectDependency(project, unreleasedInfo.gradleProjectPath, distributionProjectName(distribution)); + } + + if (distribution.getType() == Type.INTEG_TEST_ZIP) { + return "org.elasticsearch.distribution.integ-test-zip:elasticsearch:" + distribution.getVersion(); + } + + String extension = distribution.getType().toString(); + String classifier = "x86_64"; + if (distribution.getType() == Type.ARCHIVE) { + extension = distribution.getPlatform() == Platform.WINDOWS ? "zip" : "tar.gz"; + classifier = distribution.getPlatform() + "-" + classifier; + } + return FAKE_IVY_GROUP + ":elasticsearch" + (distribution.getFlavor() == Flavor.OSS ? "-oss:" : ":") + + distribution.getVersion() + ":" + classifier + "@" + extension; + } + + private static Dependency projectDependency(Project project, String projectPath, String projectConfig) { + + if (project.findProject(projectPath) == null) { + throw new GradleException("no project [" + projectPath + "], project names: " + project.getRootProject().getAllprojects()); + } + Map depConfig = new HashMap<>(); + depConfig.put("path", projectPath); + depConfig.put("configuration", projectConfig); + return project.getDependencies().project(depConfig); + } + + private static String distributionProjectPath(ElasticsearchDistribution distribution) { + String projectPath = ":distribution"; + if (distribution.getType() == Type.INTEG_TEST_ZIP) { + projectPath += ":archives:integ-test-zip"; + } else { + projectPath += distribution.getType() == Type.ARCHIVE ? ":archives:" : ":packages:"; + projectPath += distributionProjectName(distribution); + } + return projectPath; + } + + private static String distributionProjectName(ElasticsearchDistribution distribution) { + String projectName = ""; + if (distribution.getFlavor() == Flavor.OSS) { + projectName += "oss-"; + } + if (distribution.getBundledJdk() == false) { + projectName += "no-jdk-"; + } + if (distribution.getType() == Type.ARCHIVE) { + Platform platform = distribution.getPlatform(); + projectName += platform.toString() + (platform == Platform.WINDOWS ? "-zip" : "-tar"); + } else { + projectName += distribution.getType(); + } + return projectName; + } + + private static String configName(String prefix, ElasticsearchDistribution distribution) { + return prefix + "_" + distribution.getVersion() + "_" + distribution.getType() + "_" + + (distribution.getPlatform() == null ? "" : distribution.getPlatform() + "_") + + distribution.getFlavor() + (distribution.getBundledJdk() ? "" : "_nojdk"); + } + + private static String capitalize(String s) { + return s.substring(0, 1).toUpperCase(Locale.ROOT) + s.substring(1); + } + + private static String extractTaskName(ElasticsearchDistribution distribution) { + String taskName = "extractElasticsearch"; + if (distribution.getType() != Type.INTEG_TEST_ZIP) { + if (distribution.getFlavor() == Flavor.OSS) { + taskName += "Oss"; + } + if (distribution.getBundledJdk() == false) { + taskName += "NoJdk"; + } + } + if (distribution.getType() == Type.ARCHIVE) { + taskName += capitalize(distribution.getPlatform().toString()); + } else if (distribution.getType() != Type.INTEG_TEST_ZIP) { + taskName += capitalize(distribution.getType().toString()); + } + taskName += distribution.getVersion(); + return taskName; + } +} diff --git a/buildSrc/src/main/java/org/elasticsearch/gradle/ElasticsearchDistribution.java b/buildSrc/src/main/java/org/elasticsearch/gradle/ElasticsearchDistribution.java new file mode 100644 index 0000000000000..ac18209a43373 --- /dev/null +++ b/buildSrc/src/main/java/org/elasticsearch/gradle/ElasticsearchDistribution.java @@ -0,0 +1,230 @@ +/* + * 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.gradle; + +import org.gradle.api.Buildable; +import org.gradle.api.artifacts.Configuration; +import org.gradle.api.model.ObjectFactory; +import org.gradle.api.provider.Property; +import org.gradle.api.tasks.TaskDependency; + +import java.io.File; +import java.util.Iterator; +import java.util.Locale; + +public class ElasticsearchDistribution implements Buildable { + + public enum Platform { + LINUX, + WINDOWS, + DARWIN; + + @Override + public String toString() { + return super.toString().toLowerCase(Locale.ROOT); + } + } + + public enum Type { + INTEG_TEST_ZIP, + ARCHIVE, + RPM, + DEB; + + @Override + public String toString() { + return super.toString().toLowerCase(Locale.ROOT); + } + } + + public enum Flavor { + DEFAULT, + OSS; + + @Override + public String toString() { + return super.toString().toLowerCase(Locale.ROOT); + } + } + + // package private to tests can use + static final Platform CURRENT_PLATFORM = OS.conditional() + .onLinux(() -> Platform.LINUX) + .onWindows(() -> Platform.WINDOWS) + .onMac(() -> Platform.DARWIN) + .supply(); + + public static final class Extracted implements Buildable, Iterable { + + // pkg private so plugin can configure + final Configuration configuration; + + private Extracted(Configuration configuration) { + this.configuration = configuration; + } + + @Override + public Iterator iterator() { + return configuration.iterator(); + } + + @Override + public TaskDependency getBuildDependencies() { + return configuration.getBuildDependencies(); + } + + @Override + public String toString() { + return configuration.getSingleFile().toString(); + } + } + + private final String name; + // pkg private so plugin can configure + final Configuration configuration; + private final Extracted extracted; + + private final Property version; + private final Property type; + private final Property platform; + private final Property flavor; + private final Property bundledJdk; + + ElasticsearchDistribution(String name, ObjectFactory objectFactory, Configuration fileConfiguration, + Configuration extractedConfiguration) { + this.name = name; + this.configuration = fileConfiguration; + this.version = objectFactory.property(Version.class); + this.version.convention(Version.fromString(VersionProperties.getElasticsearch())); + this.type = objectFactory.property(Type.class); + this.type.convention(Type.ARCHIVE); + this.platform = objectFactory.property(Platform.class); + this.flavor = objectFactory.property(Flavor.class); + this.bundledJdk = objectFactory.property(Boolean.class); + this.extracted = new Extracted(extractedConfiguration); + } + + public String getName() { + return name; + } + + public Version getVersion() { + return version.get(); + } + + public void setVersion(String version) { + this.version.set(Version.fromString(version)); + } + + public Platform getPlatform() { + return platform.getOrNull(); + } + + public void setPlatform(Platform platform) { + this.platform.set(platform); + } + + public Type getType() { + return type.get(); + } + + public void setType(Type type) { + this.type.set(type); + } + + public Flavor getFlavor() { + return flavor.getOrNull(); + } + + public void setFlavor(Flavor flavor) { + this.flavor.set(flavor); + } + + public boolean getBundledJdk() { + return bundledJdk.getOrElse(true); + } + + public void setBundledJdk(boolean bundledJdk) { + this.bundledJdk.set(bundledJdk); + } + + @Override + public String toString() { + return configuration.getSingleFile().toString(); + } + + public Extracted getExtracted() { + if (getType() == Type.RPM || getType() == Type.DEB) { + throw new UnsupportedOperationException("distribution type [" + getType() + "] for " + + "elasticsearch distribution [" + name + "] cannot be extracted"); + } + return extracted; + } + + @Override + public TaskDependency getBuildDependencies() { + return configuration.getBuildDependencies(); + } + + // internal, make this distribution's configuration unmodifiable + void finalizeValues() { + + if (getType() == Type.INTEG_TEST_ZIP) { + if (platform.isPresent()) { + throw new IllegalArgumentException( + "platform not allowed for elasticsearch distribution [" + name + "] of type [integ_test_zip]"); + } + if (flavor.isPresent()) { + throw new IllegalArgumentException( + "flavor not allowed for elasticsearch distribution [" + name + "] of type [integ_test_zip]"); + } + if (bundledJdk.isPresent()) { + throw new IllegalArgumentException( + "bundledJdk not allowed for elasticsearch distribution [" + name + "] of type [integ_test_zip]"); + } + return; + } + + if (getType() == Type.ARCHIVE) { + // defaults for archive, set here instead of via convention so integ-test-zip can verify they are not set + if (platform.isPresent() == false) { + platform.set(CURRENT_PLATFORM); + } + } else { // rpm or deb + if (platform.isPresent()) { + throw new IllegalArgumentException("platform not allowed for elasticsearch distribution [" + + name + "] of type [" + getType() + "]"); + } + } + + if (flavor.isPresent() == false) { + flavor.set(Flavor.DEFAULT); + } + if (bundledJdk.isPresent() == false) { + bundledJdk.set(true); + } + + version.finalizeValue(); + platform.finalizeValue(); + type.finalizeValue(); + flavor.finalizeValue(); + bundledJdk.finalizeValue(); + } +} diff --git a/buildSrc/src/main/java/org/elasticsearch/gradle/EmptyDirTask.java b/buildSrc/src/main/java/org/elasticsearch/gradle/EmptyDirTask.java new file mode 100644 index 0000000000000..ec9b774767cd1 --- /dev/null +++ b/buildSrc/src/main/java/org/elasticsearch/gradle/EmptyDirTask.java @@ -0,0 +1,83 @@ +/* + * 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.gradle; + +import java.io.File; + +import javax.inject.Inject; + +import org.gradle.api.DefaultTask; +import org.gradle.api.tasks.Input; +import org.gradle.api.tasks.TaskAction; +import org.gradle.internal.nativeintegration.filesystem.Chmod; + +/** + * Creates an empty directory. + */ +public class EmptyDirTask extends DefaultTask { + + private File dir; + private int dirMode = 0755; + + /** + * Creates an empty directory with the configured permissions. + */ + @TaskAction + public void create() { + dir.mkdirs(); + getChmod().chmod(dir, dirMode); + } + + @Inject + public Chmod getChmod() { + throw new UnsupportedOperationException(); + } + + @Input + public File getDir() { + return dir; + } + + /** + * @param dir The directory to create + */ + public void setDir(File dir) { + this.dir = dir; + } + + /** + * @param dir The path of the directory to create. Takes a String and coerces it to a file. + */ + public void setDir(String dir) { + this.dir = getProject().file(dir); + } + + @Input + public int getDirMode() { + return dirMode; + } + + /** + * @param dirMode The permissions to apply to the new directory + */ + public void setDirMode(int dirMode) { + this.dirMode = dirMode; + } + +} diff --git a/buildSrc/src/main/java/org/elasticsearch/gradle/JdkDownloadPlugin.java b/buildSrc/src/main/java/org/elasticsearch/gradle/JdkDownloadPlugin.java index a408b66ec817d..d4f0d9941dacf 100644 --- a/buildSrc/src/main/java/org/elasticsearch/gradle/JdkDownloadPlugin.java +++ b/buildSrc/src/main/java/org/elasticsearch/gradle/JdkDownloadPlugin.java @@ -23,6 +23,7 @@ import org.gradle.api.NamedDomainObjectContainer; import org.gradle.api.Plugin; import org.gradle.api.Project; +import org.gradle.api.Task; import org.gradle.api.UnknownTaskException; import org.gradle.api.artifacts.Configuration; import org.gradle.api.artifacts.ConfigurationContainer; @@ -165,7 +166,12 @@ private static void setupRootJdkDownload(Project rootProject, String platform, S } String extractDir = rootProject.getBuildDir().toPath().resolve("jdks/openjdk-" + jdkVersion + "_" + platform).toString(); TaskProvider extractTask = rootProject.getTasks().register(extractTaskName, Copy.class, copyTask -> { - copyTask.doFirst(t -> rootProject.delete(extractDir)); + copyTask.doFirst(new Action() { + @Override + public void execute(Task t) { + rootProject.delete(extractDir); + } + }); copyTask.into(extractDir); copyTask.from(fileGetter, removeRootDir); }); diff --git a/buildSrc/src/main/java/org/elasticsearch/gradle/LazyPropertyList.java b/buildSrc/src/main/java/org/elasticsearch/gradle/LazyPropertyList.java new file mode 100644 index 0000000000000..d23c9b9e748d5 --- /dev/null +++ b/buildSrc/src/main/java/org/elasticsearch/gradle/LazyPropertyList.java @@ -0,0 +1,205 @@ +package org.elasticsearch.gradle; + +import org.gradle.api.tasks.Input; +import org.gradle.api.tasks.Nested; + +import java.util.ArrayList; +import java.util.Collection; +import java.util.Iterator; +import java.util.List; +import java.util.ListIterator; +import java.util.function.Supplier; +import java.util.stream.Collectors; + +public class LazyPropertyList extends AbstractLazyPropertyCollection implements List { + + private final List> delegate = new ArrayList<>(); + + public LazyPropertyList(String name) { + super(name); + } + + public LazyPropertyList(String name, Object owner) { + super(name, owner); + } + + @Override + public int size() { + return delegate.size(); + } + + @Override + public boolean isEmpty() { + return delegate.isEmpty(); + } + + @Override + public boolean contains(Object o) { + return delegate.stream().anyMatch(entry -> entry.getValue().equals(o)); + } + + @Override + public Iterator iterator() { + return delegate.stream().peek(this::validate).map(PropertyListEntry::getValue).iterator(); + } + + @Override + public Object[] toArray() { + return delegate.stream().peek(this::validate).map(PropertyListEntry::getValue).toArray(); + } + + @Override + public T1[] toArray(T1[] a) { + return delegate.stream().peek(this::validate).map(PropertyListEntry::getValue).collect(Collectors.toList()).toArray(a); + } + + @Override + public boolean add(T t) { + return delegate.add(new PropertyListEntry<>(() -> t, PropertyNormalization.DEFAULT)); + } + + public boolean add(Supplier supplier) { + return delegate.add(new PropertyListEntry<>(supplier, PropertyNormalization.DEFAULT)); + } + + public boolean add(Supplier supplier, PropertyNormalization normalization) { + return delegate.add(new PropertyListEntry<>(supplier, normalization)); + } + + @Override + public boolean remove(Object o) { + throw new UnsupportedOperationException(this.getClass().getName() + " does not support remove()"); + } + + @Override + public boolean containsAll(Collection c) { + return delegate.stream().map(PropertyListEntry::getValue).collect(Collectors.toList()).containsAll(c); + } + + @Override + public boolean addAll(Collection c) { + c.forEach(this::add); + return true; + } + + @Override + public boolean addAll(int index, Collection c) { + int i = index; + for (T item : c) { + this.add(i++, item); + } + return true; + } + + @Override + public boolean removeAll(Collection c) { + throw new UnsupportedOperationException(this.getClass().getName() + " does not support removeAll()"); + } + + @Override + public boolean retainAll(Collection c) { + throw new UnsupportedOperationException(this.getClass().getName() + " does not support retainAll()"); + } + + @Override + public void clear() { + delegate.clear(); + } + + @Override + public T get(int index) { + PropertyListEntry entry = delegate.get(index); + validate(entry); + return entry.getValue(); + } + + @Override + public T set(int index, T element) { + return delegate.set(index, new PropertyListEntry<>(() -> element, PropertyNormalization.DEFAULT)).getValue(); + } + + @Override + public void add(int index, T element) { + delegate.add(index, new PropertyListEntry<>(() -> element, PropertyNormalization.DEFAULT)); + } + + @Override + public T remove(int index) { + return delegate.remove(index).getValue(); + } + + @Override + public int indexOf(Object o) { + for (int i = 0; i < delegate.size(); i++) { + if (delegate.get(i).getValue().equals(o)) { + return i; + } + } + + return -1; + } + + @Override + public int lastIndexOf(Object o) { + int lastIndex = -1; + for (int i = 0; i < delegate.size(); i++) { + if (delegate.get(i).getValue().equals(o)) { + lastIndex = i; + } + } + + return lastIndex; + } + + @Override + public ListIterator listIterator() { + return delegate.stream().map(PropertyListEntry::getValue).collect(Collectors.toList()).listIterator(); + } + + @Override + public ListIterator listIterator(int index) { + return delegate.stream().peek(this::validate).map(PropertyListEntry::getValue).collect(Collectors.toList()).listIterator(index); + } + + @Override + public List subList(int fromIndex, int toIndex) { + return delegate.stream() + .peek(this::validate) + .map(PropertyListEntry::getValue) + .collect(Collectors.toList()) + .subList(fromIndex, toIndex); + } + + @Override + @Nested + List getNormalizedCollection() { + return delegate.stream() + .peek(this::validate) + .filter(entry -> entry.getNormalization() != PropertyNormalization.IGNORE_VALUE) + .collect(Collectors.toList()); + } + + private void validate(PropertyListEntry entry) { + assertNotNull(entry.getValue(), "entry"); + } + + private class PropertyListEntry { + private final Supplier supplier; + private final PropertyNormalization normalization; + + PropertyListEntry(Supplier supplier, PropertyNormalization normalization) { + this.supplier = supplier; + this.normalization = normalization; + } + + public PropertyNormalization getNormalization() { + return normalization; + } + + @Input + public T getValue() { + assertNotNull(supplier, "supplier"); + return supplier.get(); + } + } +} diff --git a/buildSrc/src/main/java/org/elasticsearch/gradle/LazyPropertyMap.java b/buildSrc/src/main/java/org/elasticsearch/gradle/LazyPropertyMap.java new file mode 100644 index 0000000000000..781bb095bb5cb --- /dev/null +++ b/buildSrc/src/main/java/org/elasticsearch/gradle/LazyPropertyMap.java @@ -0,0 +1,167 @@ +package org.elasticsearch.gradle; + +import org.gradle.api.Named; +import org.gradle.api.tasks.Input; +import org.gradle.api.tasks.Nested; + +import java.util.Collection; +import java.util.LinkedHashMap; +import java.util.List; +import java.util.Map; +import java.util.Set; +import java.util.function.BiFunction; +import java.util.function.Supplier; +import java.util.stream.Collectors; + +public class LazyPropertyMap extends AbstractLazyPropertyCollection implements Map { + + private final Map> delegate = new LinkedHashMap<>(); + private final BiFunction normalizationMapper; + + public LazyPropertyMap(String name) { + this(name, null); + } + + public LazyPropertyMap(String name, Object owner) { + this(name, owner, null); + } + + public LazyPropertyMap(String name, Object owner, BiFunction normalizationMapper) { + super(name, owner); + this.normalizationMapper = normalizationMapper; + } + + @Override + public int size() { + return delegate.size(); + } + + @Override + public boolean isEmpty() { + return delegate.isEmpty(); + } + + @Override + public boolean containsKey(Object key) { + return delegate.containsKey(key); + } + + @Override + public boolean containsValue(Object value) { + return delegate.values().stream().map(PropertyMapEntry::getValue).anyMatch(v -> v.equals(value)); + } + + @Override + public V get(Object key) { + PropertyMapEntry entry = delegate.get(key); + if (entry != null) { + V value = entry.getValue(); + assertNotNull(value, "value for key '" + key + "'"); + return value; + } else { + return null; + } + } + + @Override + public V put(K key, V value) { + return put(key, value, PropertyNormalization.DEFAULT); + } + + public V put(K key, V value, PropertyNormalization normalization) { + assertNotNull(value, "value for key '" + key + "'"); + return put(key, () -> value, normalization); + } + + public V put(K key, Supplier supplier) { + return put(key, supplier, PropertyNormalization.DEFAULT); + } + + public V put(K key, Supplier supplier, PropertyNormalization normalization) { + assertNotNull(supplier, "supplier for key '" + key + "'"); + PropertyMapEntry previous = delegate.put(key, new PropertyMapEntry<>(key, supplier, normalization)); + return previous == null ? null : previous.getValue(); + } + + @Override + public V remove(Object key) { + PropertyMapEntry previous = delegate.remove(key); + return previous == null ? null : previous.getValue(); + } + + @Override + public void putAll(Map m) { + throw new UnsupportedOperationException(this.getClass().getName() + " does not support putAll()"); + } + + @Override + public void clear() { + delegate.clear(); + } + + @Override + public Set keySet() { + return delegate.keySet(); + } + + @Override + public Collection values() { + return delegate.values().stream().peek(this::validate).map(PropertyMapEntry::getValue).collect(Collectors.toList()); + } + + @Override + public Set> entrySet() { + return delegate.entrySet().stream() + .peek(this::validate) + .collect(Collectors.toMap(Entry::getKey, entry -> entry.getValue().getValue())).entrySet(); + } + + @Override + @Nested + List getNormalizedCollection() { + return delegate.values().stream() + .peek(this::validate) + .filter(entry -> entry.getNormalization() != PropertyNormalization.IGNORE_VALUE) + .map(entry -> normalizationMapper == null ? entry : normalizationMapper.apply(entry.getKey(), entry.getValue())) + .collect(Collectors.toList()); + } + + private void validate(Map.Entry> entry) { + validate(entry.getValue()); + } + + private void validate(PropertyMapEntry supplier) { + assertNotNull(supplier, "key '" + supplier.getKey() + "' supplier value"); + } + + private static class PropertyMapEntry implements Named { + private final K key; + private final Supplier value; + private final PropertyNormalization normalization; + + PropertyMapEntry(K key, Supplier value, PropertyNormalization normalization) { + this.key = key; + this.value = value; + this.normalization = normalization; + } + + public PropertyNormalization getNormalization() { + return normalization; + } + + @Override + public String getName() { + return getKey().toString(); + } + + @Input + public K getKey() { + return key; + } + + @Input + public V getValue() { + return value.get(); + } + } +} diff --git a/buildSrc/src/main/java/org/elasticsearch/gradle/PropertyNormalization.java b/buildSrc/src/main/java/org/elasticsearch/gradle/PropertyNormalization.java new file mode 100644 index 0000000000000..85fdede56c392 --- /dev/null +++ b/buildSrc/src/main/java/org/elasticsearch/gradle/PropertyNormalization.java @@ -0,0 +1,13 @@ +package org.elasticsearch.gradle; + +public enum PropertyNormalization { + /** + * Uses default strategy based on runtime property type. + */ + DEFAULT, + + /** + * Ignores property value completely for the purposes of input snapshotting. + */ + IGNORE_VALUE +} diff --git a/buildSrc/src/main/java/org/elasticsearch/gradle/SystemPropertyCommandLineArgumentProvider.java b/buildSrc/src/main/java/org/elasticsearch/gradle/SystemPropertyCommandLineArgumentProvider.java new file mode 100644 index 0000000000000..7e808724035df --- /dev/null +++ b/buildSrc/src/main/java/org/elasticsearch/gradle/SystemPropertyCommandLineArgumentProvider.java @@ -0,0 +1,30 @@ +package org.elasticsearch.gradle; + +import org.gradle.api.tasks.Input; +import org.gradle.process.CommandLineArgumentProvider; + +import java.util.LinkedHashMap; +import java.util.Map; +import java.util.stream.Collectors; + +public class SystemPropertyCommandLineArgumentProvider implements CommandLineArgumentProvider { + private final Map systemProperties = new LinkedHashMap<>(); + + public void systemProperty(String key, Object value) { + systemProperties.put(key, value); + } + + @Override + public Iterable asArguments() { + return systemProperties.entrySet() + .stream() + .map(entry -> "-D" + entry.getKey() + "=" + entry.getValue()) + .collect(Collectors.toList()); + } + + // Track system property keys as an input so our build cache key will change if we add properties but values are still ignored + @Input + public Iterable getPropertyNames() { + return systemProperties.keySet(); + } +} diff --git a/buildSrc/src/main/java/org/elasticsearch/gradle/http/WaitForHttpResource.java b/buildSrc/src/main/java/org/elasticsearch/gradle/http/WaitForHttpResource.java index 3ac4a53910c26..52447b6bc20a1 100644 --- a/buildSrc/src/main/java/org/elasticsearch/gradle/http/WaitForHttpResource.java +++ b/buildSrc/src/main/java/org/elasticsearch/gradle/http/WaitForHttpResource.java @@ -129,18 +129,14 @@ public boolean wait(int durationInMs) throws GeneralSecurityException, Interrupt } protected void checkResource(SSLContext ssl) throws IOException { - try { - final HttpURLConnection connection = buildConnection(ssl); - connection.connect(); - final Integer response = connection.getResponseCode(); - if (validResponseCodes.contains(response)) { - logger.info("Got successful response [{}] from URL [{}]", response, url); - return; - } else { - throw new IOException(response + " " + connection.getResponseMessage()); - } - } catch (IOException e) { - throw e; + final HttpURLConnection connection = buildConnection(ssl); + connection.connect(); + final Integer response = connection.getResponseCode(); + if (validResponseCodes.contains(response)) { + logger.info("Got successful response [{}] from URL [{}]", response, url); + return; + } else { + throw new IOException(response + " " + connection.getResponseMessage()); } } diff --git a/buildSrc/src/main/java/org/elasticsearch/gradle/info/GlobalBuildInfoPlugin.java b/buildSrc/src/main/java/org/elasticsearch/gradle/info/GlobalBuildInfoPlugin.java index f0f34e84261c6..32f9e8e15f928 100644 --- a/buildSrc/src/main/java/org/elasticsearch/gradle/info/GlobalBuildInfoPlugin.java +++ b/buildSrc/src/main/java/org/elasticsearch/gradle/info/GlobalBuildInfoPlugin.java @@ -116,7 +116,7 @@ private static String findJavaHome(String version) { String versionedJavaHome = System.getenv(getJavaHomeEnvVarName(version)); if (versionedJavaHome == null) { throw new GradleException( - "$versionedVarName must be set to build Elasticsearch. " + + "$" + getJavaHomeEnvVarName(version) + " must be set to build Elasticsearch. " + "Note that if the variable was just set you might have to run `./gradlew --stop` for " + "it to be picked up. See https://github.com/elastic/elasticsearch/issues/31399 details." ); diff --git a/buildSrc/src/main/java/org/elasticsearch/gradle/plugin/PluginPropertiesExtension.java b/buildSrc/src/main/java/org/elasticsearch/gradle/plugin/PluginPropertiesExtension.java new file mode 100644 index 0000000000000..28f9a2f3d426e --- /dev/null +++ b/buildSrc/src/main/java/org/elasticsearch/gradle/plugin/PluginPropertiesExtension.java @@ -0,0 +1,141 @@ +/* + * 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.gradle.plugin; + +import org.gradle.api.Project; + +import java.io.File; +import java.util.ArrayList; +import java.util.List; + +/** + * A container for plugin properties that will be written to the plugin descriptor, for easy + * manipulation in the gradle DSL. + */ +public class PluginPropertiesExtension { + private String name; + + private String version; + + private String description; + + private String classname; + + /** Other plugins this plugin extends through SPI */ + private List extendedPlugins = new ArrayList<>(); + + private boolean hasNativeController; + + /** True if the plugin requires the elasticsearch keystore to exist, false otherwise. */ + private boolean requiresKeystore; + + /** A license file that should be included in the built plugin zip. */ + private File licenseFile; + + /** + * A notice file that should be included in the built plugin zip. This will be + * extended with notices from the {@code licenses/} directory. + */ + private File noticeFile; + + private final Project project; + + public PluginPropertiesExtension(Project project) { + this.project = project; + } + + public String getName() { + return name == null ? project.getName() : name; + } + + public void setName(String name) { + this.name = name; + } + + public String getVersion() { + return version == null ? project.getVersion().toString() : version; + } + + public void setVersion(String version) { + this.version = version; + } + + public String getDescription() { + return description; + } + + public void setDescription(String description) { + this.description = description; + } + + public String getClassname() { + return classname; + } + + public void setClassname(String classname) { + this.classname = classname; + } + + public List getExtendedPlugins() { + return this.extendedPlugins; + } + + public boolean isHasNativeController() { + return hasNativeController; + } + + public void setHasNativeController(boolean hasNativeController) { + this.hasNativeController = hasNativeController; + } + + public boolean isRequiresKeystore() { + return requiresKeystore; + } + + public void setRequiresKeystore(boolean requiresKeystore) { + this.requiresKeystore = requiresKeystore; + } + + public File getLicenseFile() { + return licenseFile; + } + + public void setLicenseFile(File licenseFile) { + this.project.getExtensions().getExtraProperties().set("licenseFile", licenseFile); + this.licenseFile = licenseFile; + } + + public File getNoticeFile() { + return noticeFile; + } + + public void setNoticeFile(File noticeFile) { + this.project.getExtensions().getExtraProperties().set("noticeFile", noticeFile); + this.noticeFile = noticeFile; + } + + public Project getProject() { + return project; + } + + public void setExtendedPlugins(List extendedPlugins) { + this.extendedPlugins = extendedPlugins; + } +} diff --git a/buildSrc/src/main/java/org/elasticsearch/gradle/precommit/DependencyLicensesTask.java b/buildSrc/src/main/java/org/elasticsearch/gradle/precommit/DependencyLicensesTask.java new file mode 100644 index 0000000000000..d884207d590d3 --- /dev/null +++ b/buildSrc/src/main/java/org/elasticsearch/gradle/precommit/DependencyLicensesTask.java @@ -0,0 +1,328 @@ +/* + * 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.gradle.precommit; + +import org.apache.commons.codec.binary.Hex; +import org.gradle.api.DefaultTask; +import org.gradle.api.GradleException; +import org.gradle.api.InvalidUserDataException; +import org.gradle.api.file.FileCollection; +import org.gradle.api.logging.Logger; +import org.gradle.api.logging.Logging; +import org.gradle.api.tasks.Input; +import org.gradle.api.tasks.InputDirectory; +import org.gradle.api.tasks.InputFiles; +import org.gradle.api.tasks.Optional; +import org.gradle.api.tasks.TaskAction; + +import java.io.File; +import java.io.IOException; +import java.nio.charset.StandardCharsets; +import java.nio.file.Files; +import java.security.MessageDigest; +import java.security.NoSuchAlgorithmException; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.HashMap; +import java.util.HashSet; +import java.util.LinkedHashMap; +import java.util.List; +import java.util.Map; +import java.util.Set; +import java.util.regex.Matcher; +import java.util.regex.Pattern; +import java.util.stream.Collectors; + +/** + * A task to check licenses for dependencies. + * + * There are two parts to the check: + *
    + *
  • LICENSE and NOTICE files
  • + *
  • SHA checksums for each dependency jar
  • + *
+ * + * The directory to find the license and sha files in defaults to the dir @{code licenses} + * in the project directory for this task. You can override this directory: + *
+ *   dependencyLicenses {
+ *     licensesDir = getProject().file("mybetterlicensedir")
+ *   }
+ * 
+ * + * The jar files to check default to the dependencies from the default configuration. You + * can override this, for example, to only check compile dependencies: + *
+ *   dependencyLicenses {
+ *     dependencies = getProject().configurations.compile
+ *   }
+ * 
+ * + * Every jar must have a {@code .sha1} file in the licenses dir. These can be managed + * automatically using the {@code updateShas} helper task that is created along + * with this task. It will add {@code .sha1} files for new jars that are in dependencies + * and remove old {@code .sha1} files that are no longer needed. + * + * Every jar must also have a LICENSE and NOTICE file. However, multiple jars can share + * LICENSE and NOTICE files by mapping a pattern to the same name. + *
+ *   dependencyLicenses {
+ *     mapping from: /lucene-.*/, to: "lucene"
+ *   }
+ * 
+ */ +public class DependencyLicensesTask extends DefaultTask { + + private final Pattern regex = Pattern.compile("-v?\\d+.*"); + + private final Logger logger = Logging.getLogger(getClass()); + + private static final String SHA_EXTENSION = ".sha1"; + + // TODO: we should be able to default this to eg compile deps, but we need to move the licenses + // check from distribution to core (ie this should only be run on java projects) + /** A collection of jar files that should be checked. */ + private FileCollection dependencies; + + /** The directory to find the license and sha files in. */ + private File licensesDir = new File(getProject().getProjectDir(), "licenses"); + + /** A map of patterns to prefix, used to find the LICENSE and NOTICE file. */ + private Map mappings = new LinkedHashMap<>(); + + /** Names of dependencies whose shas should not exist. */ + private Set ignoreShas = new HashSet<>(); + + /** + * Add a mapping from a regex pattern for the jar name, to a prefix to find + * the LICENSE and NOTICE file for that jar. + */ + public void mapping(Map props) { + String from = props.remove("from"); + if (from == null) { + throw new InvalidUserDataException("Missing \"from\" setting for license name mapping"); + } + String to = props.remove("to"); + if (to == null) { + throw new InvalidUserDataException("Missing \"to\" setting for license name mapping"); + } + if (props.isEmpty() == false) { + throw new InvalidUserDataException("Unknown properties for mapping on dependencyLicenses: " + props.keySet()); + } + mappings.put(from, to); + } + + @InputFiles + public FileCollection getDependencies() { + return dependencies; + } + + public void setDependencies(FileCollection dependencies) { + this.dependencies = dependencies; + } + + @Optional + @InputDirectory + public File getLicensesDir() { + if (licensesDir.exists()) { + return licensesDir; + } + + return null; + } + + public void setLicensesDir(File licensesDir) { + this.licensesDir = licensesDir; + } + + /** + * Add a rule which will skip SHA checking for the given dependency name. This should be used for + * locally build dependencies, which cause the sha to change constantly. + */ + public void ignoreSha(String dep) { + ignoreShas.add(dep); + } + + @TaskAction + public void checkDependencies() throws IOException, NoSuchAlgorithmException { + if (dependencies == null) { + throw new GradleException("No dependencies variable defined."); + } + + if (dependencies.isEmpty()) { + if (licensesDir.exists()) { + throw new GradleException("Licenses dir " + licensesDir + " exists, but there are no dependencies"); + } + return; // no dependencies to check + } else if (licensesDir.exists() == false) { + throw new GradleException("Licences dir " + licensesDir + " does not exist, but there are dependencies"); + } + + Map licenses = new HashMap<>(); + Map notices = new HashMap<>(); + Set shaFiles = new HashSet<>(); + + for (File file : licensesDir.listFiles()) { + String name = file.getName(); + if (name.endsWith(SHA_EXTENSION)) { + shaFiles.add(file); + } else if (name.endsWith("-LICENSE") || name.endsWith("-LICENSE.txt")) { + // TODO: why do we support suffix of LICENSE *and* LICENSE.txt?? + licenses.put(name, false); + } else if (name.contains("-NOTICE") || name.contains("-NOTICE.txt")) { + notices.put(name, false); + } + } + + checkDependencies(licenses, notices, shaFiles); + + licenses.forEach((item, exists) -> failIfAnyMissing(item, exists, "license")); + + notices.forEach((item, exists) -> failIfAnyMissing(item, exists, "notice")); + + if (shaFiles.isEmpty() == false) { + throw new GradleException("Unused sha files found: \n" + joinFilenames(shaFiles)); + } + } + + private void failIfAnyMissing(String item, Boolean exists, String type) { + if (exists == false) { + throw new GradleException("Unused " + type + " " + item); + } + } + + private void checkDependencies(Map licenses, Map notices, Set shaFiles) + throws NoSuchAlgorithmException, IOException { + for (File dependency : dependencies) { + String jarName = dependency.getName(); + String depName = regex.matcher(jarName).replaceFirst(""); + + validateSha(shaFiles, dependency, jarName, depName); + + String dependencyName = getDependencyName(mappings, depName); + logger.info("mapped dependency name {} to {} for license/notice check", depName, dependencyName); + checkFile(dependencyName, jarName, licenses, "LICENSE"); + checkFile(dependencyName, jarName, notices, "NOTICE"); + } + } + + private void validateSha(Set shaFiles, File dependency, String jarName, String depName) + throws NoSuchAlgorithmException, IOException { + if (ignoreShas.contains(depName)) { + // local deps should not have sha files! + if (getShaFile(jarName).exists()) { + throw new GradleException("SHA file " + getShaFile(jarName) + " exists for ignored dependency " + depName); + } + } else { + logger.info("Checking sha for {}", jarName); + checkSha(dependency, jarName, shaFiles); + } + } + + private String joinFilenames(Set shaFiles) { + List names = shaFiles.stream().map(File::getName).collect(Collectors.toList()); + return String.join("\n", names); + } + + public static String getDependencyName(Map mappings, String dependencyName) { + // order is the same for keys and values iteration since we use a linked hashmap + List mapped = new ArrayList<>(mappings.values()); + Pattern mappingsPattern = Pattern.compile("(" + String.join(")|(", mappings.keySet()) + ")"); + Matcher match = mappingsPattern.matcher(dependencyName); + if (match.matches()) { + int i = 0; + while (i < match.groupCount() && match.group(i + 1) == null) { + ++i; + } + return mapped.get(i); + } + return dependencyName; + } + + private void checkSha(File jar, String jarName, Set shaFiles) throws NoSuchAlgorithmException, IOException { + File shaFile = getShaFile(jarName); + if (shaFile.exists() == false) { + throw new GradleException("Missing SHA for " + jarName + ". Run \"gradle updateSHAs\" to create them"); + } + + // TODO: shouldn't have to trim, sha files should not have trailing newline + byte[] fileBytes = Files.readAllBytes(shaFile.toPath()); + String expectedSha = new String(fileBytes, StandardCharsets.UTF_8).trim(); + + String sha = getSha1(jar); + + if (expectedSha.equals(sha) == false) { + throw new GradleException( + "SHA has changed! Expected " + expectedSha + " for " + jarName + " but got " + sha + ". " + + "\nThis usually indicates a corrupt dependency cache or artifacts changed upstream." + + "\nEither wipe your cache, fix the upstream artifact, or delete " + shaFile + " and run updateShas"); + } + shaFiles.remove(shaFile); + } + + private void checkFile(String name, String jarName, Map counters, String type) { + String fileName = getFileName(name, counters, type); + + if (counters.containsKey(fileName) == false) { + throw new GradleException("Missing " + type + " for " + jarName + ", expected in " + fileName); + } + + counters.put(fileName, true); + } + + private String getFileName(String name, Map counters, String type) { + String fileName = name + "-" + type; + + if (counters.containsKey(fileName) == false) { + // try the other suffix...TODO: get rid of this, just support ending in .txt + return fileName + ".txt"; + } + + return fileName; + } + + @Input + public LinkedHashMap getMappings() { + return new LinkedHashMap<>(mappings); + } + + File getShaFile(String jarName) { + return new File(licensesDir, jarName + SHA_EXTENSION); + } + + Set getShaFiles() { + File[] array = licensesDir.listFiles(); + if (array == null) { + throw new GradleException("\"" + licensesDir.getPath() + "\" isn't a valid directory"); + } + + return Arrays.stream(array) + .filter(file -> file.getName().endsWith(SHA_EXTENSION)) + .collect(Collectors.toSet()); + } + + String getSha1(File file) throws IOException, NoSuchAlgorithmException { + byte[] bytes = Files.readAllBytes(file.toPath()); + + MessageDigest digest = MessageDigest.getInstance("SHA-1"); + char[] encoded = Hex.encodeHex(digest.digest(bytes)); + return String.copyValueOf(encoded); + } + +} diff --git a/buildSrc/src/main/java/org/elasticsearch/gradle/precommit/UpdateShasTask.java b/buildSrc/src/main/java/org/elasticsearch/gradle/precommit/UpdateShasTask.java new file mode 100644 index 0000000000000..db3148da696a3 --- /dev/null +++ b/buildSrc/src/main/java/org/elasticsearch/gradle/precommit/UpdateShasTask.java @@ -0,0 +1,86 @@ +/* + * 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.gradle.precommit; + +import org.gradle.api.DefaultTask; +import org.gradle.api.logging.Logger; +import org.gradle.api.logging.Logging; +import org.gradle.api.tasks.TaskAction; + +import java.io.File; +import java.io.IOException; +import java.nio.charset.StandardCharsets; +import java.nio.file.Files; +import java.nio.file.StandardOpenOption; +import java.security.NoSuchAlgorithmException; +import java.util.Set; + +/** + * A task to update shas used by {@code DependencyLicensesCheck} + */ +public class UpdateShasTask extends DefaultTask { + + private final Logger logger = Logging.getLogger(getClass()); + + /** The parent dependency licenses task to use configuration from */ + private DependencyLicensesTask parentTask; + + public UpdateShasTask() { + setDescription("Updates the sha files for the dependencyLicenses check"); + setOnlyIf(element -> parentTask.getLicensesDir() != null); + } + + @TaskAction + public void updateShas() throws NoSuchAlgorithmException, IOException { + Set shaFiles = parentTask.getShaFiles(); + + for (File dependency : parentTask.getDependencies()) { + String jarName = dependency.getName(); + File shaFile = parentTask.getShaFile(jarName); + + if (shaFile.exists() == false) { + createSha(dependency, jarName, shaFile); + } else { + shaFiles.remove(shaFile); + } + } + + for (File shaFile : shaFiles) { + logger.lifecycle("Removing unused sha " + shaFile.getName()); + shaFile.delete(); + } + } + + private void createSha(File dependency, String jarName, File shaFile) throws IOException, NoSuchAlgorithmException { + logger.lifecycle("Adding sha for " + jarName); + + String sha = parentTask.getSha1(dependency); + + Files.write(shaFile.toPath(), sha.getBytes(StandardCharsets.UTF_8), StandardOpenOption.CREATE); + } + + public DependencyLicensesTask getParentTask() { + return parentTask; + } + + public void setParentTask(DependencyLicensesTask parentTask) { + this.parentTask = parentTask; + } +} diff --git a/buildSrc/src/main/java/org/elasticsearch/gradle/test/RestTestRunnerTask.java b/buildSrc/src/main/java/org/elasticsearch/gradle/test/RestTestRunnerTask.java new file mode 100644 index 0000000000000..eff05f64f9c33 --- /dev/null +++ b/buildSrc/src/main/java/org/elasticsearch/gradle/test/RestTestRunnerTask.java @@ -0,0 +1,37 @@ +package org.elasticsearch.gradle.test; + +import org.elasticsearch.gradle.testclusters.ElasticsearchCluster; +import org.gradle.api.tasks.CacheableTask; +import org.gradle.api.tasks.Nested; +import org.gradle.api.tasks.testing.Test; + +import java.util.ArrayList; +import java.util.Collection; + +import static org.elasticsearch.gradle.Distribution.INTEG_TEST; + +/** + * Customized version of Gradle {@link Test} task which tracks a collection of {@link ElasticsearchCluster} as a task input. We must do this + * as a custom task type because the current {@link org.gradle.api.tasks.TaskInputs} runtime API does not have a way to register + * {@link Nested} inputs. + */ +@CacheableTask +public class RestTestRunnerTask extends Test { + + private Collection clusters = new ArrayList<>(); + + public RestTestRunnerTask() { + super(); + this.getOutputs().doNotCacheIf("Build cache is only enabled for tests against clusters using the 'integ-test' distribution", + task -> clusters.stream().flatMap(c -> c.getNodes().stream()).anyMatch(n -> n.getDistribution() != INTEG_TEST)); + } + + @Nested + public Collection getClusters() { + return clusters; + } + + public void testCluster(ElasticsearchCluster cluster) { + this.clusters.add(cluster); + } +} diff --git a/buildSrc/src/main/java/org/elasticsearch/gradle/testclusters/ElasticsearchCluster.java b/buildSrc/src/main/java/org/elasticsearch/gradle/testclusters/ElasticsearchCluster.java index e245fb0ead95a..c343f56525aea 100644 --- a/buildSrc/src/main/java/org/elasticsearch/gradle/testclusters/ElasticsearchCluster.java +++ b/buildSrc/src/main/java/org/elasticsearch/gradle/testclusters/ElasticsearchCluster.java @@ -18,15 +18,17 @@ */ package org.elasticsearch.gradle.testclusters; -import org.elasticsearch.GradleServicesAdapter; import org.elasticsearch.gradle.Distribution; import org.elasticsearch.gradle.FileSupplier; +import org.elasticsearch.gradle.PropertyNormalization; import org.elasticsearch.gradle.Version; import org.elasticsearch.gradle.http.WaitForHttpResource; +import org.gradle.api.Named; import org.gradle.api.NamedDomainObjectContainer; import org.gradle.api.Project; import org.gradle.api.logging.Logger; import org.gradle.api.logging.Logging; +import org.gradle.api.tasks.Nested; import java.io.File; import java.io.IOException; @@ -47,7 +49,7 @@ import java.util.function.Supplier; import java.util.stream.Collectors; -public class ElasticsearchCluster implements TestClusterConfiguration { +public class ElasticsearchCluster implements TestClusterConfiguration, Named { private static final Logger LOGGER = Logging.getLogger(ElasticsearchNode.class); private static final int CLUSTER_UP_TIMEOUT = 40; @@ -60,21 +62,23 @@ public class ElasticsearchCluster implements TestClusterConfiguration { private final File workingDirBase; private final File artifactsExtractDir; private final LinkedHashMap> waitConditions = new LinkedHashMap<>(); - private final GradleServicesAdapter services; + private final Project project; public ElasticsearchCluster(String path, String clusterName, Project project, File artifactsExtractDir, File workingDirBase) { this.path = path; this.clusterName = clusterName; + this.project = project; this.workingDirBase = workingDirBase; this.artifactsExtractDir = artifactsExtractDir; - this.services = GradleServicesAdapter.getInstance(project); this.nodes = project.container(ElasticsearchNode.class); this.nodes.add( new ElasticsearchNode( path, clusterName + "-0", - services, artifactsExtractDir, workingDirBase + project, artifactsExtractDir, workingDirBase ) ); + // configure the cluster name eagerly so nodes know about it + this.nodes.all((node) -> node.defaultConfig.put("cluster.name", safeName(clusterName))); addWaitForClusterHealth(); } @@ -94,7 +98,7 @@ public void setNumberOfNodes(int numberOfNodes) { for (int i = nodes.size() ; i < numberOfNodes; i++) { this.nodes.add(new ElasticsearchNode( - path, clusterName + "-" + i, services, artifactsExtractDir, workingDirBase + path, clusterName + "-" + i, project, artifactsExtractDir, workingDirBase )); } } @@ -151,6 +155,11 @@ public void keystore(String key, File value) { nodes.all(each -> each.keystore(key, value)); } + @Override + public void keystore(String key, File value, PropertyNormalization normalization) { + nodes.all(each -> each.keystore(key, value, normalization)); + } + @Override public void keystore(String key, FileSupplier valueSupplier) { nodes.all(each -> each.keystore(key, valueSupplier)); @@ -161,11 +170,21 @@ public void setting(String key, String value) { nodes.all(each -> each.setting(key, value)); } + @Override + public void setting(String key, String value, PropertyNormalization normalization) { + nodes.all(each -> each.setting(key, value, normalization)); + } + @Override public void setting(String key, Supplier valueSupplier) { nodes.all(each -> each.setting(key, valueSupplier)); } + @Override + public void setting(String key, Supplier valueSupplier, PropertyNormalization normalization) { + nodes.all(each -> each.setting(key, valueSupplier, normalization)); + } + @Override public void systemProperty(String key, String value) { nodes.all(each -> each.systemProperty(key, value)); @@ -176,6 +195,11 @@ public void systemProperty(String key, Supplier valueSupplier) { nodes.all(each -> each.systemProperty(key, valueSupplier)); } + @Override + public void systemProperty(String key, Supplier valueSupplier, PropertyNormalization normalization) { + nodes.all(each -> each.systemProperty(key, valueSupplier, normalization)); + } + @Override public void environment(String key, String value) { nodes.all(each -> each.environment(key, value)); @@ -186,6 +210,16 @@ public void environment(String key, Supplier valueSupplier) { nodes.all(each -> each.environment(key, valueSupplier)); } + @Override + public void environment(String key, Supplier valueSupplier, PropertyNormalization normalization) { + nodes.all(each -> each.environment(key, valueSupplier, normalization)); + } + + @Override + public void jvmArgs(String... values) { + nodes.all(each -> each.jvmArgs(values)); + } + @Override public void freeze() { nodes.forEach(ElasticsearchNode::freeze); @@ -205,22 +239,40 @@ public void setJavaHome(File javaHome) { @Override public void start() { - String nodeNames = nodes.stream().map(ElasticsearchNode::getName).collect(Collectors.joining(",")); + final String nodeNames; + if (nodes.stream().map(ElasticsearchNode::getName).anyMatch( name -> name == null)) { + nodeNames = null; + } else { + nodeNames = nodes.stream().map(ElasticsearchNode::getName).collect(Collectors.joining(",")); + }; for (ElasticsearchNode node : nodes) { - node.defaultConfig.put("cluster.name", safeName(clusterName)); - if (Version.fromString(node.getVersion()).getMajor() >= 7) { - node.defaultConfig.put("cluster.initial_master_nodes", "[" + nodeNames + "]"); - node.defaultConfig.put("discovery.seed_providers", "file"); + if (nodeNames != null) { + // Can only configure master nodes if we have node names defined + if (Version.fromString(node.getVersion()).getMajor() >= 7) { + node.defaultConfig.put("cluster.initial_master_nodes", "[" + nodeNames + "]"); + node.defaultConfig.put("discovery.seed_providers", "file"); + node.defaultConfig.put("discovery.seed_hosts", "[]"); + } } node.start(); } } + @Override + public void restart() { + nodes.forEach(ElasticsearchNode::restart); + } + @Override public void extraConfigFile(String destination, File from) { nodes.all(node -> node.extraConfigFile(destination, from)); } + @Override + public void extraConfigFile(String destination, File from, PropertyNormalization normalization) { + nodes.all(node -> node.extraConfigFile(destination, from, normalization)); + } + @Override public void user(Map userSpec) { nodes.all(node -> node.user(userSpec)); @@ -262,14 +314,13 @@ public List getAllTransportPortURI() { } public void waitForAllConditions() { - long startedAt = System.currentTimeMillis(); LOGGER.info("Waiting for nodes"); nodes.forEach(ElasticsearchNode::waitForAllConditions); writeUnicastHostsFiles(); LOGGER.info("Starting to wait for cluster to form"); - waitForConditions(waitConditions, startedAt, CLUSTER_UP_TIMEOUT, CLUSTER_UP_TIMEOUT_UNIT, this); + waitForConditions(waitConditions, System.currentTimeMillis(), CLUSTER_UP_TIMEOUT, CLUSTER_UP_TIMEOUT_UNIT, this); } @Override @@ -305,9 +356,16 @@ public ElasticsearchNode singleNode() { private void addWaitForClusterHealth() { waitConditions.put("cluster health yellow", (node) -> { try { + boolean httpSslEnabled = getFirstNode().isHttpSslEnabled(); WaitForHttpResource wait = new WaitForHttpResource( - "http", getFirstNode().getHttpSocketURI(), nodes.size() + httpSslEnabled ? "https" : "http", + getFirstNode().getHttpSocketURI(), + nodes.size() ); + if (httpSslEnabled) { + + getFirstNode().configureHttpWait(wait); + } List> credentials = getFirstNode().getCredentials(); if (getFirstNode().getCredentials().isEmpty() == false) { wait.setUsername(credentials.get(0).get("useradd")); @@ -315,7 +373,7 @@ private void addWaitForClusterHealth() { } return wait.wait(500); } catch (IOException e) { - throw new IllegalStateException("Connection attempt to " + this + " failed", e); + throw new UncheckedIOException("IO error while waiting cluster", e); } catch (InterruptedException e) { Thread.currentThread().interrupt(); throw new TestClustersException("Interrupted while waiting for " + this, e); @@ -325,6 +383,11 @@ private void addWaitForClusterHealth() { }); } + @Nested + public NamedDomainObjectContainer getNodes() { + return nodes; + } + @Override public boolean equals(Object o) { if (this == o) return true; diff --git a/buildSrc/src/main/java/org/elasticsearch/gradle/testclusters/ElasticsearchNode.java b/buildSrc/src/main/java/org/elasticsearch/gradle/testclusters/ElasticsearchNode.java index bba94f6c7d173..025cee2ffafb1 100644 --- a/buildSrc/src/main/java/org/elasticsearch/gradle/testclusters/ElasticsearchNode.java +++ b/buildSrc/src/main/java/org/elasticsearch/gradle/testclusters/ElasticsearchNode.java @@ -18,13 +18,29 @@ */ package org.elasticsearch.gradle.testclusters; -import org.elasticsearch.GradleServicesAdapter; import org.elasticsearch.gradle.Distribution; import org.elasticsearch.gradle.FileSupplier; +import org.elasticsearch.gradle.LazyPropertyList; +import org.elasticsearch.gradle.LazyPropertyMap; +import org.elasticsearch.gradle.LoggedExec; import org.elasticsearch.gradle.OS; +import org.elasticsearch.gradle.PropertyNormalization; import org.elasticsearch.gradle.Version; +import org.elasticsearch.gradle.http.WaitForHttpResource; +import org.gradle.api.Action; +import org.gradle.api.Named; +import org.gradle.api.Project; +import org.gradle.api.file.FileCollection; import org.gradle.api.logging.Logger; import org.gradle.api.logging.Logging; +import org.gradle.api.tasks.Classpath; +import org.gradle.api.tasks.Input; +import org.gradle.api.tasks.InputFile; +import org.gradle.api.tasks.InputFiles; +import org.gradle.api.tasks.Nested; +import org.gradle.api.tasks.PathSensitive; +import org.gradle.api.tasks.PathSensitivity; +import org.gradle.api.tasks.util.PatternFilterable; import java.io.ByteArrayInputStream; import java.io.File; @@ -36,9 +52,12 @@ import java.nio.file.Files; import java.nio.file.Path; import java.nio.file.StandardCopyOption; +import java.nio.file.StandardOpenOption; +import java.time.Instant; import java.util.ArrayList; import java.util.Arrays; import java.util.Collections; +import java.util.Comparator; import java.util.HashMap; import java.util.HashSet; import java.util.LinkedHashMap; @@ -63,8 +82,10 @@ public class ElasticsearchNode implements TestClusterConfiguration { private static final Logger LOGGER = Logging.getLogger(ElasticsearchNode.class); private static final int ES_DESTROY_TIMEOUT = 20; private static final TimeUnit ES_DESTROY_TIMEOUT_UNIT = TimeUnit.SECONDS; - private static final int NODE_UP_TIMEOUT = 60; - private static final TimeUnit NODE_UP_TIMEOUT_UNIT = TimeUnit.SECONDS; + private static final int NODE_UP_TIMEOUT = 2; + private static final TimeUnit NODE_UP_TIMEOUT_UNIT = TimeUnit.MINUTES; + private static final int ADDITIONAL_CONFIG_TIMEOUT = 15; + private static final TimeUnit ADDITIONAL_CONFIG_TIMEOUT_UNIT = TimeUnit.SECONDS; private static final List OVERRIDABLE_SETTINGS = Arrays.asList( "path.repo", "discovery.seed_providers" @@ -72,23 +93,23 @@ public class ElasticsearchNode implements TestClusterConfiguration { private final String path; private final String name; - private final GradleServicesAdapter services; + private final Project project; private final AtomicBoolean configurationFrozen = new AtomicBoolean(false); private final Path artifactsExtractDir; private final Path workingDir; - private final LinkedHashMap> waitConditions = new LinkedHashMap<>(); private final List plugins = new ArrayList<>(); private final List modules = new ArrayList<>(); - private final Map> settings = new LinkedHashMap<>(); - private final Map> keystoreSettings = new LinkedHashMap<>(); - private final Map keystoreFiles = new LinkedHashMap<>(); - private final Map> systemProperties = new LinkedHashMap<>(); - private final Map> environment = new LinkedHashMap<>(); - private final Map extraConfigFiles = new HashMap<>(); - final LinkedHashMap defaultConfig = new LinkedHashMap<>(); + private final LazyPropertyMap settings = new LazyPropertyMap<>("Settings", this); + private final LazyPropertyMap keystoreSettings = new LazyPropertyMap<>("Keystore", this); + private final LazyPropertyMap keystoreFiles = new LazyPropertyMap<>("Keystore files", this, FileEntry::new); + private final LazyPropertyMap systemProperties = new LazyPropertyMap<>("System properties", this); + private final LazyPropertyMap environment = new LazyPropertyMap<>("Environment", this); + private final LazyPropertyList jvmArgs = new LazyPropertyList<>("JVM arguments", this); + private final LazyPropertyMap extraConfigFiles = new LazyPropertyMap<>("Extra config files", this, FileEntry::new); private final List> credentials = new ArrayList<>(); + final LinkedHashMap defaultConfig = new LinkedHashMap<>(); private final Path confPathRepo; private final Path configFile; @@ -105,11 +126,12 @@ public class ElasticsearchNode implements TestClusterConfiguration { private File javaHome; private volatile Process esProcess; private Function nameCustomization = Function.identity(); + private boolean isWorkingDirConfigured = false; - ElasticsearchNode(String path, String name, GradleServicesAdapter services, File artifactsExtractDir, File workingDirBase) { + ElasticsearchNode(String path, String name, Project project, File artifactsExtractDir, File workingDirBase) { this.path = path; this.name = name; - this.services = services; + this.project = project; this.artifactsExtractDir = artifactsExtractDir.toPath(); this.workingDir = workingDirBase.toPath().resolve(safeName(name)).toAbsolutePath(); confPathRepo = workingDir.resolve("repo"); @@ -128,6 +150,7 @@ public String getName() { return nameCustomization.apply(name); } + @Input public String getVersion() { return version; } @@ -139,6 +162,7 @@ public void setVersion(String version) { this.version = version; } + @Input public Distribution getDistribution() { return distribution; } @@ -169,72 +193,81 @@ public void module(File module) { @Override public void keystore(String key, String value) { - addSupplier("Keystore", keystoreSettings, key, value); + keystoreSettings.put(key, value); } @Override public void keystore(String key, Supplier valueSupplier) { - addSupplier("Keystore", keystoreSettings, key, valueSupplier); + keystoreSettings.put(key, valueSupplier); } @Override public void keystore(String key, File value) { - requireNonNull(value, "keystore value was null when configuring test cluster`" + this + "`"); - keystore(key, () -> value); + keystoreFiles.put(key, value); + } + + @Override + public void keystore(String key, File value, PropertyNormalization normalization) { + keystoreFiles.put(key, value, normalization); } @Override public void keystore(String key, FileSupplier valueSupplier) { - requireNonNull(key, "Keystore" + " key was null when configuring test cluster `" + this + "`"); - requireNonNull(valueSupplier, "Keystore" + " value supplier was null when configuring test cluster `" + this + "`"); keystoreFiles.put(key, valueSupplier); } @Override public void setting(String key, String value) { - addSupplier("Settings", settings, key, value); + settings.put(key, value); + } + + @Override + public void setting(String key, String value, PropertyNormalization normalization) { + settings.put(key, value, normalization); } @Override public void setting(String key, Supplier valueSupplier) { - addSupplier("Setting", settings, key, valueSupplier); + settings.put(key, valueSupplier); + } + + @Override + public void setting(String key, Supplier valueSupplier, PropertyNormalization normalization) { + settings.put(key, valueSupplier, normalization); } @Override public void systemProperty(String key, String value) { - addSupplier("Java System property", systemProperties, key, value); + systemProperties.put(key, value); } @Override public void systemProperty(String key, Supplier valueSupplier) { - addSupplier("Java System property", systemProperties, key, valueSupplier); + systemProperties.put(key, valueSupplier); } @Override - public void environment(String key, String value) { - addSupplier("Environment variable", environment, key, value); + public void systemProperty(String key, Supplier valueSupplier, PropertyNormalization normalization) { + systemProperties.put(key, valueSupplier, normalization); } @Override - public void environment(String key, Supplier valueSupplier) { - addSupplier("Environment variable", environment, key, valueSupplier); + public void environment(String key, String value) { + environment.put(key, value); } - private void addSupplier(String name, Map> collector, String key, Supplier valueSupplier) { - requireNonNull(key, name + " key was null when configuring test cluster `" + this + "`"); - requireNonNull(valueSupplier, name + " value supplier was null when configuring test cluster `" + this + "`"); - collector.put(key, valueSupplier); + @Override + public void environment(String key, Supplier valueSupplier) { + environment.put(key, valueSupplier); } - private void addSupplier(String name, Map> collector, String key, String actualValue) { - requireNonNull(actualValue, name + " value was null when configuring test cluster `" + this + "`"); - addSupplier(name, collector, key, () -> actualValue); + @Override + public void environment(String key, Supplier valueSupplier, PropertyNormalization normalization) { + environment.put(key, valueSupplier, normalization); } - private void checkSuppliers(String name, Map> collector) { - collector.forEach((key, value) -> { - requireNonNull(value.get().toString(), name + " supplied value was null when configuring test cluster `" + this + "`"); - }); + public void jvmArgs(String... values) { + jvmArgs.addAll(Arrays.asList(values)); } public Path getConfigDir() { @@ -277,39 +310,43 @@ public Stream logLines() throws IOException { public synchronized void start() { LOGGER.info("Starting `{}`", this); - Path distroArtifact = artifactsExtractDir - .resolve(distribution.getGroup()) - .resolve("elasticsearch-" + getVersion()); - - if (Files.exists(distroArtifact) == false) { - throw new TestClustersException("Can not start " + this + ", missing: " + distroArtifact); + if (Files.exists(getExtractedDistributionDir()) == false) { + throw new TestClustersException("Can not start " + this + ", missing: " + getExtractedDistributionDir()); } - if (Files.isDirectory(distroArtifact) == false) { - throw new TestClustersException("Can not start " + this + ", is not a directory: " + distroArtifact); + if (Files.isDirectory(getExtractedDistributionDir()) == false) { + throw new TestClustersException("Can not start " + this + ", is not a directory: " + getExtractedDistributionDir()); } try { - createWorkingDir(distroArtifact); + if (isWorkingDirConfigured == false) { + logToProcessStdout("Configuring working directory: " + workingDir); + // Only configure working dir once so we don't loose data on restarts + isWorkingDirConfigured = true; + createWorkingDir(getExtractedDistributionDir()); + } } catch (IOException e) { throw new UncheckedIOException("Failed to create working directory for " + this, e); } createConfiguration(); - plugins.forEach(plugin -> runElaticsearchBinScript( - "elasticsearch-plugin", - "install", "--batch", plugin.toString()) - ); + if (plugins.isEmpty() == false) { + logToProcessStdout("Installing " + plugins.size() + " plugins"); + plugins.forEach(plugin -> runElaticsearchBinScript( + "elasticsearch-plugin", + "install", "--batch", plugin.toString()) + ); + } if (keystoreSettings.isEmpty() == false || keystoreFiles.isEmpty() == false) { + logToProcessStdout("Adding " + keystoreSettings.size() + " keystore settings and " + keystoreFiles.size() + " keystore files"); runElaticsearchBinScript("elasticsearch-keystore", "create"); - checkSuppliers("Keystore", keystoreSettings); keystoreSettings.forEach((key, value) -> - runElaticsearchBinScriptWithInput(value.get().toString(), "elasticsearch-keystore", "add", "-x", key) + runElaticsearchBinScriptWithInput(value.toString(), "elasticsearch-keystore", "add", "-x", key) ); - for (Map.Entry entry : keystoreFiles.entrySet()) { - File file = entry.getValue().get(); + for (Map.Entry entry : keystoreFiles.entrySet()) { + File file = entry.getValue(); requireNonNull(file, "supplied keystoreFile was null when configuring " + this); if (file.exists() == false) { throw new TestClustersException("supplied keystore file " + file + " does not exist, require for " + this); @@ -323,62 +360,95 @@ public synchronized void start() { copyExtraConfigFiles(); if (isSettingMissingOrTrue("xpack.security.enabled")) { + logToProcessStdout("Setting up " + credentials.size() + " users"); if (credentials.isEmpty()) { user(Collections.emptyMap()); } credentials.forEach(paramMap -> runElaticsearchBinScript( "elasticsearch-users", - paramMap.entrySet().stream() - .flatMap(entry -> Stream.of(entry.getKey(), entry.getValue())) - .toArray(String[]::new) + paramMap.entrySet().stream() + .flatMap(entry -> Stream.of(entry.getKey(), entry.getValue())) + .toArray(String[]::new) )); } + logToProcessStdout("Starting Elasticsearch process"); startElasticsearchProcess(); } + private void logToProcessStdout(String message) { + try { + if (Files.exists(esStdoutFile.getParent()) == false) { + Files.createDirectories(esStdoutFile.getParent()); + } + Files.write( + esStdoutFile, + ("[" + Instant.now().toString() + "] [BUILD] " + message + "\n").getBytes(StandardCharsets.UTF_8), + StandardOpenOption.CREATE, StandardOpenOption.APPEND + ); + } catch (IOException e) { + throw new UncheckedIOException(e); + } + } + + @Override + public void restart() { + LOGGER.info("Restarting {}", this); + stop(false); + try { + Files.delete(httpPortsFile); + Files.delete(transportPortFile); + } catch (IOException e) { + throw new UncheckedIOException(e); + } + + start(); + } + private boolean isSettingMissingOrTrue(String name) { - return Boolean.valueOf(settings.getOrDefault(name, () -> "false").get().toString()); + return Boolean.valueOf(settings.getOrDefault(name, "false").toString()); } private void copyExtraConfigFiles() { + if (extraConfigFiles.isEmpty() == false) { + logToProcessStdout("Setting up " + extraConfigFiles.size() + " additional config files"); + } extraConfigFiles.forEach((destination, from) -> { - if (Files.exists(from.toPath()) == false) { - throw new TestClustersException("Can't create extra config file from " + from + " for " + this + - " as it does not exist"); - } - Path dst = configFile.getParent().resolve(destination); - try { - Files.createDirectories(dst); - Files.copy(from.toPath(), dst, StandardCopyOption.REPLACE_EXISTING); - LOGGER.info("Added extra config file {} for {}", destination, this); - } catch (IOException e) { - throw new UncheckedIOException("Can't create extra config file for", e); - } - }); + if (Files.exists(from.toPath()) == false) { + throw new TestClustersException("Can't create extra config file from " + from + " for " + this + + " as it does not exist"); + } + Path dst = configFile.getParent().resolve(destination); + try { + Files.createDirectories(dst.getParent()); + Files.copy(from.toPath(), dst, StandardCopyOption.REPLACE_EXISTING); + LOGGER.info("Added extra config file {} for {}", destination, this); + } catch (IOException e) { + throw new UncheckedIOException("Can't create extra config file for", e); + } + }); } private void installModules() { if (distribution == Distribution.INTEG_TEST) { - modules.forEach(module -> services.copy(spec -> { - if (module.getName().toLowerCase().endsWith(".zip")) { - spec.from(services.zipTree(module)); - } else if (module.isDirectory()) { - spec.from(module); - } else { - throw new IllegalArgumentException("Not a valid module " + module + " for " + this); + logToProcessStdout("Installing " + modules.size() + "modules"); + for (File module : modules) { + Path destination = workingDir.resolve("modules").resolve(module.getName().replace(".zip", "").replace("-" + version, "")); + + // only install modules that are not already bundled with the integ-test distribution + if (Files.exists(destination) == false) { + project.copy(spec -> { + if (module.getName().toLowerCase().endsWith(".zip")) { + spec.from(project.zipTree(module)); + } else if (module.isDirectory()) { + spec.from(module); + } else { + throw new IllegalArgumentException("Not a valid module " + module + " for " + this); + } + spec.into(destination); + }); } - spec.into( - workingDir - .resolve("modules") - .resolve( - module.getName() - .replace(".zip", "") - .replace("-" + version, "") - ) - .toFile() - ); - })); + } } else { LOGGER.info("Not installing " + modules.size() + "(s) since the " + distribution + " distribution already " + "has them"); @@ -394,6 +464,15 @@ public void extraConfigFile(String destination, File from) { extraConfigFiles.put(destination, from); } + @Override + public void extraConfigFile(String destination, File from, PropertyNormalization normalization) { + if (destination.contains("..")) { + throw new IllegalArgumentException("extra config file destination can't be relative, was " + destination + + " for " + this); + } + extraConfigFiles.put(destination, from, normalization); + } + @Override public void user(Map userSpec) { Set keys = new HashSet<>(userSpec.keySet()); @@ -403,16 +482,23 @@ public void user(Map userSpec) { if (keys.isEmpty() == false) { throw new TestClustersException("Unknown keys in user definition " + keys + " for " + this); } - Map cred = new LinkedHashMap<>(); - cred.put("useradd", userSpec.getOrDefault("username","test_user")); - cred.put("-p", userSpec.getOrDefault("password","x-pack-test-password")); + Map cred = new LinkedHashMap<>(); + cred.put("useradd", userSpec.getOrDefault("username", "test_user")); + cred.put("-p", userSpec.getOrDefault("password", "x-pack-test-password")); cred.put("-r", userSpec.getOrDefault("role", "superuser")); credentials.add(cred); } private void runElaticsearchBinScriptWithInput(String input, String tool, String... args) { + if ( + Files.exists(workingDir.resolve("bin").resolve(tool)) == false && + Files.exists(workingDir.resolve("bin").resolve(tool + ".bat")) == false + ) { + throw new TestClustersException("Can't run bin script: `" + tool + "` does not exist. " + + "Is this the distribution you expect it to be ?"); + } try (InputStream byteArrayInputStream = new ByteArrayInputStream(input.getBytes(StandardCharsets.UTF_8))) { - services.loggedExec(spec -> { + LoggedExec.exec(project, spec -> { spec.setEnvironment(getESEnvironment()); spec.workingDir(workingDir); spec.executable( @@ -453,12 +539,25 @@ private Map getESEnvironment() { defaultEnv.put("ES_PATH_CONF", configFile.getParent().toString()); String systemPropertiesString = ""; if (systemProperties.isEmpty() == false) { - checkSuppliers("Java System property", systemProperties); systemPropertiesString = " " + systemProperties.entrySet().stream() - .map(entry -> "-D" + entry.getKey() + "=" + entry.getValue().get()) + .map(entry -> "-D" + entry.getKey() + "=" + entry.getValue()) + .collect(Collectors.joining(" ")); + } + String jvmArgsString = ""; + if (jvmArgs.isEmpty() == false) { + jvmArgsString = " " + jvmArgs.stream() + .peek(argument -> { + if (argument.toString().startsWith("-D")) { + throw new TestClustersException("Invalid jvm argument `" + argument + + "` configure as systemProperty instead for " + this + ); + } + }) .collect(Collectors.joining(" ")); } - defaultEnv.put("ES_JAVA_OPTS", "-Xms512m -Xmx512m -ea -esa" + systemPropertiesString); + defaultEnv.put("ES_JAVA_OPTS", "-Xms512m -Xmx512m -ea -esa" + + systemPropertiesString + jvmArgsString + ); defaultEnv.put("ES_TMPDIR", tmpDir.toString()); // Windows requires this as it defaults to `c:\windows` despite ES_TMPDIR defaultEnv.put("TMP", tmpDir.toString()); @@ -471,8 +570,7 @@ private Map getESEnvironment() { ); } - checkSuppliers("Environment variable", environment); - environment.forEach((key, value) -> defaultEnv.put(key, value.get().toString())); + environment.forEach((key, value) -> defaultEnv.put(key, value.toString())); return defaultEnv; } @@ -520,6 +618,14 @@ public List getAllTransportPortURI() { return getTransportPortInternal(); } + public File getServerLog() { + return confPathLogs.resolve(defaultConfig.get("cluster.name") + "_server.json").toFile(); + } + + public File getAuditLog() { + return confPathLogs.resolve(defaultConfig.get("cluster.name") + "_audit.json").toFile(); + } + @Override public synchronized void stop(boolean tailLogs) { if (esProcess == null && tailLogs) { @@ -588,7 +694,7 @@ private void logProcessInfo(String prefix, ProcessHandle.Info info) { private void logFileContents(String description, Path from) { LOGGER.error("{} `{}`", description, this); - try(Stream lines = Files.lines(from, StandardCharsets.UTF_8)) { + try (Stream lines = Files.lines(from, StandardCharsets.UTF_8)) { lines .map(line -> " " + line) .forEach(LOGGER::error); @@ -624,12 +730,12 @@ private void createWorkingDir(Path distroExtractDir) throws IOException { * We remove write permissions to make sure files are note mistakenly edited ( e.x. the config file ) and changes * reflected across all copies. Permissions are retained to be able to replace the links. * - * @param sourceRoot where to copy from + * @param sourceRoot where to copy from * @param destinationRoot destination to link to */ private void syncWithLinks(Path sourceRoot, Path destinationRoot) { if (Files.exists(destinationRoot)) { - services.delete(destinationRoot); + project.delete(destinationRoot); } try (Stream stream = Files.walk(sourceRoot)) { @@ -662,8 +768,11 @@ private void syncWithLinks(Path sourceRoot, Path destinationRoot) { } } - private void createConfiguration() { - defaultConfig.put("node.name", nameCustomization.apply(safeName(name))); + private void createConfiguration() { + String nodeName = nameCustomization.apply(safeName(name)); + if (nodeName != null) { + defaultConfig.put("node.name", nodeName); + } defaultConfig.put("path.repo", confPathRepo.toAbsolutePath().toString()); defaultConfig.put("path.data", confPathData.toAbsolutePath().toString()); defaultConfig.put("path.logs", confPathLogs.toAbsolutePath().toString()); @@ -688,24 +797,21 @@ private void createConfiguration() { // over and the REST client will not retry on circuit breaking exceptions yet (see #31986 for details). Once the REST client // can retry on circuit breaking exceptions, we can revert again to the default configuration. if (Version.fromString(version).getMajor() >= 7) { - defaultConfig.put("indices.breaker.total.use_real_memory", "false"); + defaultConfig.put("indices.breaker.total.use_real_memory", "false"); } // Don't wait for state, just start up quickly. This will also allow new and old nodes in the BWC case to become the master - defaultConfig.put("discovery.initial_state_timeout", "0s"); + defaultConfig.put("discovery.initial_state_timeout", "0s"); - checkSuppliers("Settings", settings); - Map userConfig = settings.entrySet().stream() - .collect(Collectors.toMap(entry -> entry.getKey(), entry -> entry.getValue().get().toString())); HashSet overriden = new HashSet<>(defaultConfig.keySet()); - overriden.retainAll(userConfig.keySet()); + overriden.retainAll(settings.keySet()); overriden.removeAll(OVERRIDABLE_SETTINGS); - if (overriden.isEmpty() ==false) { + if (overriden.isEmpty() == false) { throw new IllegalArgumentException( "Testclusters does not allow the following settings to be changed:" + overriden + " for " + this ); } // Make sure no duplicate config keys - userConfig.keySet().stream() + settings.keySet().stream() .filter(OVERRIDABLE_SETTINGS::contains) .forEach(defaultConfig::remove); @@ -716,7 +822,7 @@ private void createConfiguration() { Files.write( configFile, Stream.concat( - userConfig.entrySet().stream(), + settings.entrySet().stream(), defaultConfig.entrySet().stream() ) .map(entry -> entry.getKey() + ": " + entry.getValue()) @@ -731,7 +837,7 @@ private void createConfiguration() { private void checkFrozen() { if (configurationFrozen.get()) { - throw new IllegalStateException("Configuration for " + this + " can not be altered, already locked"); + throw new IllegalStateException("Configuration for " + this + " can not be altered, already locked"); } } @@ -756,11 +862,97 @@ private List getHttpPortInternal() { } private List readPortsFile(Path file) throws IOException { - try(Stream lines = Files.lines(file, StandardCharsets.UTF_8)) { + try (Stream lines = Files.lines(file, StandardCharsets.UTF_8)) { return lines.map(String::trim).collect(Collectors.toList()); } } + private Path getExtractedDistributionDir() { + return artifactsExtractDir.resolve(distribution.getGroup()).resolve("elasticsearch-" + getVersion()); + } + + private List getInstalledFileSet(Action filter) { + return Stream.concat( + plugins.stream().filter(uri -> uri.getScheme().equalsIgnoreCase("file")).map(File::new), + modules.stream() + ) + .filter(File::exists) + // TODO: We may be able to simplify this with Gradle 5.6 + // https://docs.gradle.org/nightly/release-notes.html#improved-handling-of-zip-archives-on-classpaths + .map(zipFile -> project.zipTree(zipFile).matching(filter)) + .flatMap(tree -> tree.getFiles().stream()) + .sorted(Comparator.comparing(File::getName)) + .collect(Collectors.toList()); + } + + @Input + private Set getRemotePlugins() { + Set file = plugins.stream().filter(uri -> uri.getScheme().equalsIgnoreCase("file") == false).collect(Collectors.toSet()); + return file; + } + + @Classpath + private List getInstalledClasspath() { + return getInstalledFileSet(filter -> filter.include("**/*.jar")); + } + + @InputFiles + @PathSensitive(PathSensitivity.RELATIVE) + private List getInstalledFiles() { + return getInstalledFileSet(filter -> filter.exclude("**/*.jar")); + } + + @Classpath + private List getDistributionClasspath() { + ArrayList files = new ArrayList<>(project.fileTree(getExtractedDistributionDir()) + .matching(filter -> filter.include("**/*.jar")) + .getFiles()); + files.sort(Comparator.comparing(File::getName)); + + return files; + } + + @InputFiles + @PathSensitive(PathSensitivity.RELATIVE) + private FileCollection getDistributionFiles() { + return project.fileTree(getExtractedDistributionDir()).minus(project.files(getDistributionClasspath())); + } + + @Nested + private Map getKeystoreSettings() { + return keystoreSettings; + } + + @Nested + private Map getKeystoreFiles() { + return keystoreFiles; + } + + @Nested + private Map getSettings() { + return settings; + } + + @Nested + private Map getSystemProperties() { + return systemProperties; + } + + @Nested + private Map getEnvironment() { + return environment; + } + + @Nested + private List getJvmArgs() { + return jvmArgs; + } + + @Nested + private Map getExtraConfigFiles() { + return extraConfigFiles; + } + @Override public boolean isProcessAlive() { requireNonNull( @@ -771,7 +963,23 @@ public boolean isProcessAlive() { } void waitForAllConditions() { - waitForConditions(waitConditions, System.currentTimeMillis(), NODE_UP_TIMEOUT, NODE_UP_TIMEOUT_UNIT, this); + waitForConditions( + waitConditions, + System.currentTimeMillis(), + NODE_UP_TIMEOUT_UNIT.toMillis(NODE_UP_TIMEOUT) + + // Installing plugins at config time and loading them when nods start requires additional time we need to + // account for + ADDITIONAL_CONFIG_TIMEOUT_UNIT.toMillis(ADDITIONAL_CONFIG_TIMEOUT * + ( + plugins.size() + + keystoreFiles.size() + + keystoreSettings.size() + + credentials.size() + ) + ), + TimeUnit.MILLISECONDS, + this + ); } @Override @@ -793,6 +1001,7 @@ public String toString() { return "node{" + path + ":" + name + "}"; } + @Input List> getCredentials() { return credentials; } @@ -809,4 +1018,61 @@ private boolean checkPortsFilesExistWithDelay(TestClusterConfiguration node) { } return Files.exists(httpPortsFile) && Files.exists(transportPortFile); } + + public boolean isHttpSslEnabled() { + return Boolean.valueOf( + settings.getOrDefault("xpack.security.http.ssl.enabled", "false").toString() + ); + } + + void configureHttpWait(WaitForHttpResource wait) { + if (settings.containsKey("xpack.security.http.ssl.certificate_authorities")) { + wait.setCertificateAuthorities( + getConfigDir() + .resolve(settings.get("xpack.security.http.ssl.certificate_authorities").toString()) + .toFile() + ); + } + if (settings.containsKey("xpack.security.http.ssl.certificate")) { + wait.setCertificateAuthorities( + getConfigDir() + .resolve(settings.get("xpack.security.http.ssl.certificate").toString()) + .toFile() + ); + } + if (settings.containsKey("xpack.security.http.ssl.keystore.path")) { + wait.setTrustStoreFile( + getConfigDir() + .resolve(settings.get("xpack.security.http.ssl.keystore.path").toString()) + .toFile() + ); + } + if (keystoreSettings.containsKey("xpack.security.http.ssl.keystore.secure_password")) { + wait.setTrustStorePassword( + keystoreSettings.get("xpack.security.http.ssl.keystore.secure_password").toString() + ); + } + } + + private static class FileEntry implements Named { + private String name; + private File file; + + FileEntry(String name, File file) { + this.name = name; + this.file = file; + } + + @Input + @Override + public String getName() { + return name; + } + + @InputFile + @PathSensitive(PathSensitivity.NONE) + public File getFile() { + return file; + } + } } diff --git a/buildSrc/src/main/java/org/elasticsearch/gradle/testclusters/TestClusterCleanupOnShutdown.java b/buildSrc/src/main/java/org/elasticsearch/gradle/testclusters/TestClusterCleanupOnShutdown.java new file mode 100644 index 0000000000000..0381cece108e2 --- /dev/null +++ b/buildSrc/src/main/java/org/elasticsearch/gradle/testclusters/TestClusterCleanupOnShutdown.java @@ -0,0 +1,59 @@ +package org.elasticsearch.gradle.testclusters; + +import org.gradle.api.logging.Logger; +import org.gradle.api.logging.Logging; + +import java.util.Collection; +import java.util.HashSet; +import java.util.Iterator; +import java.util.Set; + +/** + * Keep an inventory of all running Clusters and stop them when interrupted + * + * This takes advantage of the fact that Gradle interrupts all the threads in the daemon when the build completes. + */ +public class TestClusterCleanupOnShutdown implements Runnable { + + private final Logger logger = Logging.getLogger(TestClusterCleanupOnShutdown.class); + + private Set clustersToWatch = new HashSet<>(); + + public void watch(Collection cluster) { + synchronized (clustersToWatch) { + clustersToWatch.addAll(clustersToWatch); + } + } + + public void unWatch(Collection cluster) { + synchronized (clustersToWatch) { + clustersToWatch.removeAll(clustersToWatch); + } + } + + @Override + public void run() { + try { + while (true) { + Thread.sleep(Long.MAX_VALUE); + } + } catch (InterruptedException interrupted) { + synchronized (clustersToWatch) { + if (clustersToWatch.isEmpty()) { + return; + } + logger.info("Cleanup thread was interrupted, shutting down all clusters"); + Iterator iterator = clustersToWatch.iterator(); + while (iterator.hasNext()) { + ElasticsearchCluster cluster = iterator.next(); + iterator.remove(); + try { + cluster.stop(false); + } catch (Exception e) { + logger.warn("Could not shut down {}", cluster, e); + } + } + } + } + } +} diff --git a/buildSrc/src/main/java/org/elasticsearch/gradle/testclusters/TestClusterConfiguration.java b/buildSrc/src/main/java/org/elasticsearch/gradle/testclusters/TestClusterConfiguration.java index 1ccbeabd4b88a..f290b4aa91b8f 100644 --- a/buildSrc/src/main/java/org/elasticsearch/gradle/testclusters/TestClusterConfiguration.java +++ b/buildSrc/src/main/java/org/elasticsearch/gradle/testclusters/TestClusterConfiguration.java @@ -20,6 +20,7 @@ import org.elasticsearch.gradle.Distribution; import org.elasticsearch.gradle.FileSupplier; +import org.elasticsearch.gradle.PropertyNormalization; import org.gradle.api.logging.Logging; import org.slf4j.Logger; @@ -52,28 +53,44 @@ public interface TestClusterConfiguration { void keystore(String key, File value); + void keystore(String key, File value, PropertyNormalization normalization); + void keystore(String key, FileSupplier valueSupplier); void setting(String key, String value); + void setting(String key, String value, PropertyNormalization normalization); + void setting(String key, Supplier valueSupplier); + void setting(String key, Supplier valueSupplier, PropertyNormalization normalization); + void systemProperty(String key, String value); void systemProperty(String key, Supplier valueSupplier); + void systemProperty(String key, Supplier valueSupplier, PropertyNormalization normalization); + void environment(String key, String value); void environment(String key, Supplier valueSupplier); + void environment(String key, Supplier valueSupplier, PropertyNormalization normalization); + + void jvmArgs(String... values); + void freeze(); void setJavaHome(File javaHome); void start(); + void restart(); + void extraConfigFile(String destination, File from); + void extraConfigFile(String destination, File from, PropertyNormalization normalization); + void user(Map userSpec); String getHttpSocketURI(); @@ -116,11 +133,7 @@ default void waitForConditions( } catch (TestClustersException e) { throw e; } catch (Exception e) { - if (lastException == null) { - lastException = e; - } else { - lastException = e; - } + lastException = e; } } if (conditionMet == false) { @@ -129,7 +142,17 @@ default void waitForConditions( if (lastException == null) { throw new TestClustersException(message); } else { - throw new TestClustersException(message, lastException); + String extraCause = ""; + Throwable cause = lastException; + int ident = 2; + while (cause != null) { + if (cause.getMessage() != null && cause.getMessage().isEmpty() == false) { + extraCause += "\n" + " ".repeat(ident) + cause.getMessage(); + ident += 2; + } + cause = cause.getCause(); + } + throw new TestClustersException(message + extraCause, lastException); } } logger.info( @@ -146,7 +169,5 @@ default String safeName(String name) { .replaceAll("[^a-zA-Z0-9]+", "-"); } - - boolean isProcessAlive(); } diff --git a/buildSrc/src/main/java/org/elasticsearch/gradle/testclusters/TestClustersCleanupExtension.java b/buildSrc/src/main/java/org/elasticsearch/gradle/testclusters/TestClustersCleanupExtension.java new file mode 100644 index 0000000000000..14bdfa952db0f --- /dev/null +++ b/buildSrc/src/main/java/org/elasticsearch/gradle/testclusters/TestClustersCleanupExtension.java @@ -0,0 +1,74 @@ +package org.elasticsearch.gradle.testclusters; + +import org.gradle.api.Project; +import org.gradle.api.logging.Logger; +import org.gradle.api.logging.Logging; + +import java.util.concurrent.ExecutorService; +import java.util.concurrent.Executors; +import java.util.concurrent.TimeUnit; + +/** + * This extensions was meant to be used internally by testclusters + * + * It holds synchronization primitives needed to implement the rate limiting. + * This is tricky because we can't use Gradle workers as there's no way to make sure that tests and their clusters are + * allocated atomically, so we could be in a situation where all workers are tests waiting for clusters to start up. + * + * Also auto configures cleanup of executors to make sure we don't leak threads in the daemon. + */ +public class TestClustersCleanupExtension { + + private static final int EXECUTOR_SHUTDOWN_TIMEOUT = 1; + private static final TimeUnit EXECUTOR_SHUTDOWN_TIMEOUT_UNIT = TimeUnit.MINUTES; + + private static final Logger logger = Logging.getLogger(TestClustersCleanupExtension.class); + + private final ExecutorService executorService; + private final TestClusterCleanupOnShutdown cleanupThread; + + public TestClustersCleanupExtension() { + executorService = Executors.newSingleThreadExecutor(); + cleanupThread = new TestClusterCleanupOnShutdown(); + executorService.submit(cleanupThread); + } + + + public static void createExtension(Project project) { + if (project.getRootProject().getExtensions().findByType(TestClustersCleanupExtension.class) != null) { + return; + } + // Configure the extension on the root project so we have a single instance per run + TestClustersCleanupExtension ext = project.getRootProject().getExtensions().create( + "__testclusters_rate_limit", + TestClustersCleanupExtension.class + ); + Thread shutdownHook = new Thread(ext.cleanupThread::run); + Runtime.getRuntime().addShutdownHook(shutdownHook); + project.getGradle().buildFinished(buildResult -> { + ext.executorService.shutdownNow(); + try { + if (ext.executorService.awaitTermination(EXECUTOR_SHUTDOWN_TIMEOUT, EXECUTOR_SHUTDOWN_TIMEOUT_UNIT) == false) { + throw new IllegalStateException( + "Failed to shut down executor service after " + + EXECUTOR_SHUTDOWN_TIMEOUT + " " + EXECUTOR_SHUTDOWN_TIMEOUT_UNIT + ); + } + } catch (InterruptedException e) { + Thread.currentThread().interrupt(); + } + try { + if (false == Runtime.getRuntime().removeShutdownHook(shutdownHook)) { + logger.warn("Trying to deregister shutdown hook when it was not registered."); + } + } catch (IllegalStateException ese) { + // Thrown when shutdown is in progress + logger.warn("Can't remove shutdown hook", ese); + } + }); + } + + public TestClusterCleanupOnShutdown getCleanupThread() { + return cleanupThread; + } +} diff --git a/buildSrc/src/main/java/org/elasticsearch/gradle/testclusters/TestClustersPlugin.java b/buildSrc/src/main/java/org/elasticsearch/gradle/testclusters/TestClustersPlugin.java index daca1f5ebb191..3c50108d9a179 100644 --- a/buildSrc/src/main/java/org/elasticsearch/gradle/testclusters/TestClustersPlugin.java +++ b/buildSrc/src/main/java/org/elasticsearch/gradle/testclusters/TestClustersPlugin.java @@ -21,6 +21,7 @@ import groovy.lang.Closure; import org.elasticsearch.gradle.BwcVersions; import org.elasticsearch.gradle.Version; +import org.elasticsearch.gradle.test.RestTestRunnerTask; import org.elasticsearch.gradle.tool.Boilerplate; import org.gradle.api.Action; import org.gradle.api.NamedDomainObjectContainer; @@ -43,31 +44,25 @@ import java.util.Collections; import java.util.HashMap; import java.util.HashSet; -import java.util.Iterator; import java.util.List; import java.util.Map; import java.util.Set; -import java.util.concurrent.ExecutorService; -import java.util.concurrent.Executors; -import java.util.concurrent.TimeUnit; import java.util.stream.Collectors; public class TestClustersPlugin implements Plugin { private static final String LIST_TASK_NAME = "listTestClusters"; - private static final String NODE_EXTENSION_NAME = "testClusters"; + public static final String EXTENSION_NAME = "testClusters"; private static final String HELPER_CONFIGURATION_PREFIX = "testclusters"; private static final String SYNC_ARTIFACTS_TASK_NAME = "syncTestClustersArtifacts"; - private static final int EXECUTOR_SHUTDOWN_TIMEOUT = 1; - private static final TimeUnit EXECUTOR_SHUTDOWN_TIMEOUT_UNIT = TimeUnit.MINUTES; private static final Logger logger = Logging.getLogger(TestClustersPlugin.class); + private static final String TESTCLUSTERS_INSPECT_FAILURE = "testclusters.inspect.failure"; private final Map> usedClusters = new HashMap<>(); private final Map claimsInventory = new HashMap<>(); - private final Set runningClusters =new HashSet<>(); - private final Thread shutdownHook = new Thread(this::shutDownAllClusters); - private ExecutorService executorService = Executors.newSingleThreadExecutor(); + private final Set runningClusters = new HashSet<>(); + private final Boolean allowClusterToSurvive = Boolean.valueOf(System.getProperty(TESTCLUSTERS_INSPECT_FAILURE, "false")); public static String getHelperConfigurationName(String version) { return HELPER_CONFIGURATION_PREFIX + "-" + version; @@ -80,6 +75,8 @@ public void apply(Project project) { // enable the DSL to describe clusters NamedDomainObjectContainer container = createTestClustersContainerExtension(project); + TestClustersCleanupExtension.createExtension(project); + // provide a task to be able to list defined clusters. createListClustersTask(project, container); @@ -98,9 +95,6 @@ public void apply(Project project) { // After each task we determine if there are clusters that are no longer needed. configureStopClustersHook(project); - // configure hooks to make sure no test cluster processes survive the build - configureCleanupHooks(project); - // Since we have everything modeled in the DSL, add all the required dependencies e.x. the distribution to the // configuration so the user doesn't have to repeat this. autoConfigureClusterDependencies(project, rootProject, container); @@ -122,7 +116,7 @@ private NamedDomainObjectContainer createTestClustersConta new File(project.getBuildDir(), "testclusters") ) ); - project.getExtensions().add(NODE_EXTENSION_NAME, container); + project.getExtensions().add(EXTENSION_NAME, container); return container; } @@ -162,6 +156,9 @@ public void doCall(ElasticsearchCluster cluster) { ((Task) thisObject).dependsOn( project.getRootProject().getTasks().getByName(SYNC_ARTIFACTS_TASK_NAME) ); + if (thisObject instanceof RestTestRunnerTask) { + ((RestTestRunnerTask) thisObject).testCluster(cluster); + } } }) ); @@ -194,8 +191,19 @@ private void configureStartClustersHook(Project project) { @Override public void beforeActions(Task task) { // we only start the cluster before the actions, so we'll not start it if the task is up-to-date - usedClusters.getOrDefault(task, Collections.emptyList()).stream() - .filter(each -> runningClusters.contains(each) == false) + List neededButNotRunning = usedClusters.getOrDefault( + task, + Collections.emptyList() + ) + .stream() + .filter(cluster -> runningClusters.contains(cluster) == false) + .collect(Collectors.toList()); + + project.getRootProject().getExtensions() + .getByType(TestClustersCleanupExtension.class) + .getCleanupThread() + .watch(neededButNotRunning); + neededButNotRunning .forEach(elasticsearchCluster -> { elasticsearchCluster.start(); runningClusters.add(elasticsearchCluster); @@ -218,22 +226,36 @@ public void afterExecute(Task task, TaskState state) { task, Collections.emptyList() ); + if (clustersUsedByTask.isEmpty()) { + return; + } + logger.info("Clusters were used, stopping and releasing permits"); + final int permitsToRelease; if (state.getFailure() != null) { // If the task fails, and other tasks use this cluster, the other task will likely never be - // executed at all, so we will never get to un-claim and terminate it. - clustersUsedByTask.forEach(each -> each.stop(true)); + // executed at all, so we will never be called again to un-claim and terminate it. + clustersUsedByTask.forEach(cluster -> stopCluster(cluster, true)); + permitsToRelease = clustersUsedByTask.stream() + .map(cluster -> cluster.getNumberOfNodes()) + .reduce(Integer::sum).get(); } else { clustersUsedByTask.forEach( - each -> claimsInventory.put(each, claimsInventory.getOrDefault(each, 0) - 1) + cluster -> claimsInventory.put(cluster, claimsInventory.getOrDefault(cluster, 0) - 1) ); - claimsInventory.entrySet().stream() + List stoppingClusers = claimsInventory.entrySet().stream() .filter(entry -> entry.getValue() == 0) .filter(entry -> runningClusters.contains(entry.getKey())) .map(Map.Entry::getKey) - .forEach(each -> { - each.stop(false); - runningClusters.remove(each); - }); + .collect(Collectors.toList()); + stoppingClusers.forEach(cluster -> { + stopCluster(cluster, false); + runningClusters.remove(cluster); + }); + + project.getRootProject().getExtensions() + .getByType(TestClustersCleanupExtension.class) + .getCleanupThread() + .unWatch(stoppingClusers); } } @Override @@ -242,6 +264,28 @@ public void beforeExecute(Task task) {} ); } + private void stopCluster(ElasticsearchCluster cluster, boolean taskFailed) { + if (allowClusterToSurvive) { + logger.info("Not stopping clusters, disabled by property"); + if (taskFailed) { + // task failed or this is the last one to stop + for (int i=1 ; ; i += i) { + logger.lifecycle( + "No more test clusters left to run, going to sleep because {} was set," + + " interrupt (^C) to stop clusters.", TESTCLUSTERS_INSPECT_FAILURE + ); + try { + Thread.sleep(1000 * i); + } catch (InterruptedException e) { + Thread.currentThread().interrupt(); + return; + } + } + } + } + cluster.stop(taskFailed); + } + /** * Boilerplate to get testClusters container extension * @@ -250,7 +294,7 @@ public void beforeExecute(Task task) {} @SuppressWarnings("unchecked") public static NamedDomainObjectContainer getNodeExtension(Project project) { return (NamedDomainObjectContainer) - project.getExtensions().getByName(NODE_EXTENSION_NAME); + project.getExtensions().getByName(EXTENSION_NAME); } private static void autoConfigureClusterDependencies( @@ -382,59 +426,4 @@ public void execute(Task task) { }))); } - private void configureCleanupHooks(Project project) { - // When the Gradle daemon is used, it will interrupt all threads when the build concludes. - // This is our signal to clean up - executorService.submit(() -> { - while (true) { - try { - Thread.sleep(Long.MAX_VALUE); - } catch (InterruptedException interrupted) { - shutDownAllClusters(); - Thread.currentThread().interrupt(); - return; - } - } - }); - - // When the Daemon is not used, or runs into issues, rely on a shutdown hook - // When the daemon is used, but does not work correctly and eventually dies off (e.x. due to non interruptible - // thread in the build) process will be stopped eventually when the daemon dies. - Runtime.getRuntime().addShutdownHook(shutdownHook); - - // When we don't run into anything out of the ordinary, and the build completes, makes sure to clean up - project.getGradle().buildFinished(buildResult -> { - shutdownExecutorService(); - if (false == Runtime.getRuntime().removeShutdownHook(shutdownHook)) { - logger.info("Trying to deregister shutdown hook when it was not registered."); - } - }); - } - - private void shutdownExecutorService() { - executorService.shutdownNow(); - try { - if (executorService.awaitTermination(EXECUTOR_SHUTDOWN_TIMEOUT, EXECUTOR_SHUTDOWN_TIMEOUT_UNIT) == false) { - throw new IllegalStateException( - "Failed to shut down executor service after " + - EXECUTOR_SHUTDOWN_TIMEOUT + " " + EXECUTOR_SHUTDOWN_TIMEOUT_UNIT - ); - } - } catch (InterruptedException e) { - logger.info("Wait for testclusters shutdown interrupted", e); - Thread.currentThread().interrupt(); - } - } - - private void shutDownAllClusters() { - synchronized (runningClusters) { - Iterator iterator = runningClusters.iterator(); - while (iterator.hasNext()) { - iterator.remove(); - iterator.next().stop(true); - } - } - } - - } diff --git a/buildSrc/src/main/java/org/elasticsearch/gradle/testfixtures/TestFixturesPlugin.java b/buildSrc/src/main/java/org/elasticsearch/gradle/testfixtures/TestFixturesPlugin.java index b930955236fb8..b9eaa2ab979c6 100644 --- a/buildSrc/src/main/java/org/elasticsearch/gradle/testfixtures/TestFixturesPlugin.java +++ b/buildSrc/src/main/java/org/elasticsearch/gradle/testfixtures/TestFixturesPlugin.java @@ -22,9 +22,11 @@ import com.avast.gradle.dockercompose.DockerComposePlugin; import com.avast.gradle.dockercompose.tasks.ComposeUp; import org.elasticsearch.gradle.OS; +import org.elasticsearch.gradle.SystemPropertyCommandLineArgumentProvider; import org.elasticsearch.gradle.precommit.JarHellTask; import org.elasticsearch.gradle.precommit.TestingConventionsTasks; import org.elasticsearch.gradle.precommit.ThirdPartyAuditTask; +import org.gradle.api.Action; import org.gradle.api.DefaultTask; import org.gradle.api.Plugin; import org.gradle.api.Project; @@ -34,6 +36,7 @@ import org.gradle.api.tasks.TaskContainer; import org.gradle.api.tasks.testing.Test; +import java.io.File; import java.util.Collections; import java.util.function.BiConsumer; @@ -56,46 +59,47 @@ public void apply(Project project) { disableTaskByType(tasks, ThirdPartyAuditTask.class); disableTaskByType(tasks, JarHellTask.class); + // the project that defined a test fixture can also use it + extension.fixtures.add(project); + Task buildFixture = project.getTasks().create("buildFixture"); Task pullFixture = project.getTasks().create("pullFixture"); Task preProcessFixture = project.getTasks().create("preProcessFixture"); buildFixture.dependsOn(preProcessFixture); pullFixture.dependsOn(preProcessFixture); Task postProcessFixture = project.getTasks().create("postProcessFixture"); + postProcessFixture.dependsOn(buildFixture); + preProcessFixture.onlyIf(spec -> buildFixture.getEnabled()); + postProcessFixture.onlyIf(spec -> buildFixture.getEnabled()); - if (dockerComposeSupported(project) == false) { + if (dockerComposeSupported() == false) { preProcessFixture.setEnabled(false); postProcessFixture.setEnabled(false); buildFixture.setEnabled(false); pullFixture.setEnabled(false); - return; - } - preProcessFixture.onlyIf(spec -> buildFixture.getEnabled()); - postProcessFixture.onlyIf(spec -> buildFixture.getEnabled()); - - project.apply(spec -> spec.plugin(BasePlugin.class)); - project.apply(spec -> spec.plugin(DockerComposePlugin.class)); - ComposeExtension composeExtension = project.getExtensions().getByType(ComposeExtension.class); - composeExtension.setUseComposeFiles(Collections.singletonList(DOCKER_COMPOSE_YML)); - composeExtension.setRemoveContainers(true); - composeExtension.setExecutable( - project.file("/usr/local/bin/docker-compose").exists() ? - "/usr/local/bin/docker-compose" : "/usr/bin/docker-compose" - ); + } else { + project.apply(spec -> spec.plugin(BasePlugin.class)); + project.apply(spec -> spec.plugin(DockerComposePlugin.class)); + ComposeExtension composeExtension = project.getExtensions().getByType(ComposeExtension.class); + composeExtension.setUseComposeFiles(Collections.singletonList(DOCKER_COMPOSE_YML)); + composeExtension.setRemoveContainers(true); + composeExtension.setExecutable( + project.file("/usr/local/bin/docker-compose").exists() ? + "/usr/local/bin/docker-compose" : "/usr/bin/docker-compose" + ); - buildFixture.dependsOn(tasks.getByName("composeUp")); - pullFixture.dependsOn(tasks.getByName("composePull")); - tasks.getByName("composeUp").mustRunAfter(preProcessFixture); - tasks.getByName("composePull").mustRunAfter(preProcessFixture); - postProcessFixture.dependsOn(buildFixture); + buildFixture.dependsOn(tasks.getByName("composeUp")); + pullFixture.dependsOn(tasks.getByName("composePull")); + tasks.getByName("composeUp").mustRunAfter(preProcessFixture); + tasks.getByName("composePull").mustRunAfter(preProcessFixture); - configureServiceInfoForTask( - postProcessFixture, - project, - (name, port) -> postProcessFixture.getExtensions() - .getByType(ExtraPropertiesExtension.class).set(name, port) - ); - extension.fixtures.add(project); + configureServiceInfoForTask( + postProcessFixture, + project, + (name, port) -> postProcessFixture.getExtensions() + .getByType(ExtraPropertiesExtension.class).set(name, port) + ); + } } extension.fixtures @@ -107,7 +111,7 @@ public void apply(Project project) { conditionTaskByType(tasks, extension, TestingConventionsTasks.class); conditionTaskByType(tasks, extension, ComposeUp.class); - if (dockerComposeSupported(project) == false) { + if (dockerComposeSupported() == false) { project.getLogger().warn( "Tests for {} require docker-compose at /usr/local/bin/docker-compose or /usr/bin/docker-compose " + "but none could be found so these will be skipped", project.getPath() @@ -122,7 +126,8 @@ public void apply(Project project) { configureServiceInfoForTask( task, fixtureProject, - task::systemProperty + (name, host) -> + task.getExtensions().getByType(SystemPropertyCommandLineArgumentProvider.class).systemProperty(name, host) ); task.dependsOn(fixtureProject.getTasks().getByName("postProcessFixture")); }) @@ -135,7 +140,9 @@ private void conditionTaskByType(TaskContainer tasks, TestFixtureExtension exten taskClass, task -> task.onlyIf(spec -> extension.fixtures.stream() - .anyMatch(fixtureProject -> fixtureProject.getTasks().getByName("buildFixture").getEnabled() == false) == false + .anyMatch(fixtureProject -> + fixtureProject.getTasks().getByName("buildFixture").getEnabled() == false + ) == false ) ); } @@ -143,37 +150,41 @@ private void conditionTaskByType(TaskContainer tasks, TestFixtureExtension exten private void configureServiceInfoForTask(Task task, Project fixtureProject, BiConsumer consumer) { // Configure ports for the tests as system properties. // We only know these at execution time so we need to do it in doFirst - task.doFirst(theTask -> - fixtureProject.getExtensions().getByType(ComposeExtension.class).getServicesInfos() - .forEach((service, infos) -> { - infos.getTcpPorts() - .forEach((container, host) -> { - String name = "test.fixtures." + service + ".tcp." + container; - theTask.getLogger().info("port mapping property: {}={}", name, host); - consumer.accept( - name, - host - ); - }); - infos.getUdpPorts() - .forEach((container, host) -> { - String name = "test.fixtures." + service + ".udp." + container; - theTask.getLogger().info("port mapping property: {}={}", name, host); - consumer.accept( - name, - host - ); - }); - }) + task.doFirst(new Action() { + @Override + public void execute(Task theTask) { + fixtureProject.getExtensions().getByType(ComposeExtension.class).getServicesInfos() + .forEach((service, infos) -> { + infos.getTcpPorts() + .forEach((container, host) -> { + String name = "test.fixtures." + service + ".tcp." + container; + theTask.getLogger().info("port mapping property: {}={}", name, host); + consumer.accept( + name, + host + ); + }); + infos.getUdpPorts() + .forEach((container, host) -> { + String name = "test.fixtures." + service + ".udp." + container; + theTask.getLogger().info("port mapping property: {}={}", name, host); + consumer.accept( + name, + host + ); + }); + }); + } + } ); } - public boolean dockerComposeSupported(Project project) { + public static boolean dockerComposeSupported() { if (OS.current().equals(OS.WINDOWS)) { return false; } - final boolean hasDockerCompose = project.file("/usr/local/bin/docker-compose").exists() || - project.file("/usr/bin/docker-compose").exists(); + final boolean hasDockerCompose = (new File("/usr/local/bin/docker-compose")).exists() || + (new File("/usr/bin/docker-compose").exists()); return hasDockerCompose && Boolean.parseBoolean(System.getProperty("tests.fixture.enabled", "true")); } diff --git a/buildSrc/src/main/java/org/elasticsearch/gradle/tool/ClasspathUtils.java b/buildSrc/src/main/java/org/elasticsearch/gradle/tool/ClasspathUtils.java new file mode 100644 index 0000000000000..40ec6bd71830a --- /dev/null +++ b/buildSrc/src/main/java/org/elasticsearch/gradle/tool/ClasspathUtils.java @@ -0,0 +1,23 @@ +package org.elasticsearch.gradle.tool; + +public class ClasspathUtils { + private static boolean isElasticsearchProject; + + static { + // look for buildSrc marker file, if it exists then we are running in the context of the elastic/elasticsearch build + isElasticsearchProject = ClasspathUtils.class.getResource("/buildSrc.marker") != null; + } + + private ClasspathUtils() { + } + + /** + * Determine if we are running in the context of the `elastic/elasticsearch` project. This method will return {@code false} when + * the build-tools project is pulled in as an external dependency. + * + * @return if we are currently running in the `elastic/elasticsearch` project + */ + public static boolean isElasticsearchProject() { + return isElasticsearchProject; + } +} diff --git a/buildSrc/src/main/resources/META-INF/gradle-plugins/elasticsearch.messy-test.properties b/buildSrc/src/main/resources/META-INF/gradle-plugins/elasticsearch.distribution-download.properties similarity index 91% rename from buildSrc/src/main/resources/META-INF/gradle-plugins/elasticsearch.messy-test.properties rename to buildSrc/src/main/resources/META-INF/gradle-plugins/elasticsearch.distribution-download.properties index 507a0f85a0468..f400ea1ad67f1 100644 --- a/buildSrc/src/main/resources/META-INF/gradle-plugins/elasticsearch.messy-test.properties +++ b/buildSrc/src/main/resources/META-INF/gradle-plugins/elasticsearch.distribution-download.properties @@ -17,4 +17,4 @@ # under the License. # -implementation-class=org.elasticsearch.gradle.test.MessyTestPlugin +implementation-class=org.elasticsearch.gradle.DistributionDownloadPlugin \ No newline at end of file diff --git a/buildSrc/src/main/resources/buildSrc.marker b/buildSrc/src/main/resources/buildSrc.marker new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/buildSrc/src/main/resources/minimumGradleVersion b/buildSrc/src/main/resources/minimumGradleVersion index 04edabda285a6..e5e7441d3e938 100644 --- a/buildSrc/src/main/resources/minimumGradleVersion +++ b/buildSrc/src/main/resources/minimumGradleVersion @@ -1 +1 @@ -5.4.1 \ No newline at end of file +5.5 \ No newline at end of file diff --git a/buildSrc/src/main/java/org/elasticsearch/gradle/JdkJarHellCheck.java b/buildSrc/src/minimumRuntime/java/org/elasticsearch/gradle/JdkJarHellCheck.java similarity index 100% rename from buildSrc/src/main/java/org/elasticsearch/gradle/JdkJarHellCheck.java rename to buildSrc/src/minimumRuntime/java/org/elasticsearch/gradle/JdkJarHellCheck.java diff --git a/buildSrc/src/main/java/org/elasticsearch/gradle/LazyFileOutputStream.java b/buildSrc/src/minimumRuntime/java/org/elasticsearch/gradle/LazyFileOutputStream.java similarity index 100% rename from buildSrc/src/main/java/org/elasticsearch/gradle/LazyFileOutputStream.java rename to buildSrc/src/minimumRuntime/java/org/elasticsearch/gradle/LazyFileOutputStream.java diff --git a/buildSrc/src/main/java/org/elasticsearch/gradle/LoggedExec.java b/buildSrc/src/minimumRuntime/java/org/elasticsearch/gradle/LoggedExec.java similarity index 94% rename from buildSrc/src/main/java/org/elasticsearch/gradle/LoggedExec.java rename to buildSrc/src/minimumRuntime/java/org/elasticsearch/gradle/LoggedExec.java index 0921c611895fa..89031916160b2 100644 --- a/buildSrc/src/main/java/org/elasticsearch/gradle/LoggedExec.java +++ b/buildSrc/src/minimumRuntime/java/org/elasticsearch/gradle/LoggedExec.java @@ -17,7 +17,6 @@ import java.io.IOException; import java.io.OutputStream; import java.io.UnsupportedEncodingException; -import java.nio.charset.StandardCharsets; import java.nio.file.Files; import java.util.function.Consumer; import java.util.function.Function; @@ -79,7 +78,13 @@ public void setSpoolOutput(boolean spoolOutput) { }; } else { out = new ByteArrayOutputStream(); - outputLogger = logger -> logger.error(((ByteArrayOutputStream) out).toString(StandardCharsets.UTF_8)); + outputLogger = logger -> { + try { + logger.error(((ByteArrayOutputStream) out).toString("UTF-8")); + } catch (UnsupportedEncodingException e) { + throw new RuntimeException(e); + } + }; } setStandardOutput(out); setErrorOutput(out); diff --git a/buildSrc/src/main/java/org/elasticsearch/gradle/Version.java b/buildSrc/src/minimumRuntime/java/org/elasticsearch/gradle/Version.java similarity index 100% rename from buildSrc/src/main/java/org/elasticsearch/gradle/Version.java rename to buildSrc/src/minimumRuntime/java/org/elasticsearch/gradle/Version.java diff --git a/buildSrc/src/main/java/org/elasticsearch/gradle/VersionProperties.java b/buildSrc/src/minimumRuntime/java/org/elasticsearch/gradle/VersionProperties.java similarity index 100% rename from buildSrc/src/main/java/org/elasticsearch/gradle/VersionProperties.java rename to buildSrc/src/minimumRuntime/java/org/elasticsearch/gradle/VersionProperties.java diff --git a/buildSrc/src/test/java/org/elasticsearch/gradle/DistributionDownloadPluginIT.java b/buildSrc/src/test/java/org/elasticsearch/gradle/DistributionDownloadPluginIT.java new file mode 100644 index 0000000000000..d83de5f21739a --- /dev/null +++ b/buildSrc/src/test/java/org/elasticsearch/gradle/DistributionDownloadPluginIT.java @@ -0,0 +1,136 @@ +/* + * 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.gradle; + +import com.github.tomakehurst.wiremock.WireMockServer; +import org.elasticsearch.gradle.test.GradleIntegrationTestCase; +import org.gradle.testkit.runner.BuildResult; +import org.gradle.testkit.runner.GradleRunner; + +import java.io.IOException; +import java.io.InputStream; +import java.nio.file.Files; +import java.nio.file.Paths; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.List; + +import static com.github.tomakehurst.wiremock.client.WireMock.aResponse; +import static com.github.tomakehurst.wiremock.client.WireMock.get; +import static com.github.tomakehurst.wiremock.client.WireMock.head; +import static com.github.tomakehurst.wiremock.client.WireMock.urlEqualTo; + +public class DistributionDownloadPluginIT extends GradleIntegrationTestCase { + // TODO: check reuse of root task across projects MOVE TO UNIT TEST + // TODO: future: check integ-test-zip to maven, snapshots to snapshot service for external project + + public void testCurrent() throws Exception { + String projectName = ":distribution:archives:linux-tar"; + assertExtractedDistro(VersionProperties.getElasticsearch(), "archive", "linux", null, null, + "tests.local_distro.config", "default", + "tests.local_distro.project", projectName); + } + + public void testBwc() throws Exception { + assertExtractedDistro("1.1.0", "archive", "linux", null, null, + "tests.local_distro.config", "linux-tar", + "tests.local_distro.project", ":distribution:bwc:minor", + "tests.current_version", "2.0.0"); + } + + public void testReleased() throws Exception { + WireMockServer wireMock = new WireMockServer(0); + try { + final byte[] filebytes; + try (InputStream stream = + Files.newInputStream(Paths.get("src/testKit/distribution-download/distribution/files/fake_elasticsearch.zip"))) { + filebytes = stream.readAllBytes(); + } + String urlPath = "/downloads/elasticsearch/elasticsearch-1.0.0-windows-x86_64.zip"; + wireMock.stubFor(head(urlEqualTo(urlPath)).willReturn(aResponse().withStatus(200))); + wireMock.stubFor(get(urlEqualTo(urlPath)).willReturn(aResponse().withStatus(200).withBody(filebytes))); + wireMock.start(); + + assertExtractedDistro("1.0.0", "archive", "windows", null, null, + "tests.download_service", wireMock.baseUrl()); + } catch (Exception e) { + // for debugging + System.err.println("missed requests: " + wireMock.findUnmatchedRequests().getRequests()); + throw e; + } finally { + wireMock.stop(); + } + } + + private void assertFileDistro(String version, String type, String platform, String flavor, Boolean bundledJdk, + String... sysProps) throws IOException { + List finalSysProps = new ArrayList<>(); + addDistroSysProps(finalSysProps, version, type, platform, flavor, bundledJdk); + finalSysProps.addAll(Arrays.asList(sysProps)); + runBuild(":subproj:assertDistroFile", finalSysProps.toArray(new String[0])); + } + + private void assertExtractedDistro(String version, String type, String platform, String flavor, Boolean bundledJdk, + String... sysProps) throws IOException { + List finalSysProps = new ArrayList<>(); + addDistroSysProps(finalSysProps, version, type, platform, flavor, bundledJdk); + finalSysProps.addAll(Arrays.asList(sysProps)); + runBuild(":subproj:assertDistroExtracted", finalSysProps.toArray(new String[0])); + } + + private BuildResult runBuild(String taskname, String... sysProps) throws IOException { + assert sysProps.length % 2 == 0; + List args = new ArrayList<>(); + args.add(taskname); + args.add("-Dlocal.repo.path=" + getLocalTestRepoPath()); + for (int i = 0; i < sysProps.length; i += 2) { + args.add("-D" + sysProps[i] + "=" + sysProps[i + 1]); + } + args.add("-i"); + GradleRunner runner = getGradleRunner("distribution-download").withArguments(args); + + BuildResult result = runner.build(); + System.out.println(result.getOutput()); + return result; + } + + private void addDistroSysProps(List sysProps, String version, String type, String platform, String flavor, Boolean bundledJdk) { + if (version != null) { + sysProps.add("tests.distro.version"); + sysProps.add(version); + } + if (type != null) { + sysProps.add("tests.distro.type"); + sysProps.add(type); + } + if (platform != null) { + sysProps.add("tests.distro.platform"); + sysProps.add(platform); + } + if (flavor != null) { + sysProps.add("tests.distro.flavor"); + sysProps.add(flavor); + } + if (bundledJdk != null) { + sysProps.add("tests.distro.bundledJdk"); + sysProps.add(bundledJdk.toString()); + } + } +} diff --git a/buildSrc/src/test/java/org/elasticsearch/gradle/DistributionDownloadPluginTests.java b/buildSrc/src/test/java/org/elasticsearch/gradle/DistributionDownloadPluginTests.java new file mode 100644 index 0000000000000..69169d13d1977 --- /dev/null +++ b/buildSrc/src/test/java/org/elasticsearch/gradle/DistributionDownloadPluginTests.java @@ -0,0 +1,260 @@ +/* + * 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.gradle; + +import org.elasticsearch.gradle.ElasticsearchDistribution.Flavor; +import org.elasticsearch.gradle.ElasticsearchDistribution.Platform; +import org.elasticsearch.gradle.ElasticsearchDistribution.Type; +import org.elasticsearch.gradle.test.GradleUnitTestCase; +import org.gradle.api.NamedDomainObjectContainer; +import org.gradle.api.Project; +import org.gradle.testfixtures.ProjectBuilder; + +import java.io.File; +import java.util.Arrays; +import java.util.TreeSet; + +import static org.hamcrest.core.StringContains.containsString; + +public class DistributionDownloadPluginTests extends GradleUnitTestCase { + private static Project rootProject; + private static Project archivesProject; + private static Project packagesProject; + private static Project bwcProject; + + private static final Version BWC_MAJOR_VERSION = Version.fromString("2.0.0"); + private static final Version BWC_MINOR_VERSION = Version.fromString("1.1.0"); + private static final Version BWC_STAGED_VERSION = Version.fromString("1.0.0"); + private static final Version BWC_BUGFIX_VERSION = Version.fromString("1.0.1"); + private static final Version BWC_MAINTENANCE_VERSION = Version.fromString("0.90.1"); + private static final BwcVersions BWC_MINOR = + new BwcVersions(new TreeSet<>(Arrays.asList(BWC_BUGFIX_VERSION, BWC_MINOR_VERSION, BWC_MAJOR_VERSION)), BWC_MAJOR_VERSION); + private static final BwcVersions BWC_STAGED = + new BwcVersions(new TreeSet<>(Arrays.asList(BWC_STAGED_VERSION, BWC_MINOR_VERSION, BWC_MAJOR_VERSION)), BWC_MAJOR_VERSION); + private static final BwcVersions BWC_BUGFIX = + new BwcVersions(new TreeSet<>(Arrays.asList(BWC_BUGFIX_VERSION, BWC_MINOR_VERSION, BWC_MAJOR_VERSION)), BWC_MAJOR_VERSION); + private static final BwcVersions BWC_MAINTENANCE = + new BwcVersions(new TreeSet<>(Arrays.asList(BWC_MAINTENANCE_VERSION, BWC_STAGED_VERSION, BWC_MINOR_VERSION)), BWC_MINOR_VERSION); + + public void testVersionDefault() { + ElasticsearchDistribution distro = checkDistro(createProject(null), + "testdistro", null, Type.ARCHIVE, Platform.LINUX, Flavor.OSS, true); + assertEquals(distro.getVersion(), Version.fromString(VersionProperties.getElasticsearch())); + } + + public void testBadVersionFormat() { + assertDistroError(createProject(null), "testdistro", "badversion", Type.ARCHIVE, Platform.LINUX, Flavor.OSS, true, + "Invalid version format: 'badversion'"); + } + + public void testTypeDefault() { + ElasticsearchDistribution distro = checkDistro(createProject(null), + "testdistro", "5.0.0", null, Platform.LINUX, Flavor.OSS, true); + assertEquals(distro.getType(), Type.ARCHIVE); + } + + public void testPlatformDefault() { + ElasticsearchDistribution distro = checkDistro(createProject(null), + "testdistro", "5.0.0", Type.ARCHIVE, null, Flavor.OSS, true); + assertEquals(distro.getPlatform(), ElasticsearchDistribution.CURRENT_PLATFORM); + } + + public void testPlatformForIntegTest() { + assertDistroError(createProject(null), "testdistro", "5.0.0", Type.INTEG_TEST_ZIP, Platform.LINUX, null, null, + "platform not allowed for elasticsearch distribution [testdistro]"); + } + + public void testFlavorDefault() { + ElasticsearchDistribution distro = checkDistro(createProject(null), + "testdistro", "5.0.0", Type.ARCHIVE, Platform.LINUX, null, true); + assertEquals(distro.getFlavor(), Flavor.DEFAULT); + } + + public void testFlavorForIntegTest() { + assertDistroError(createProject(null), + "testdistro", "5.0.0", Type.INTEG_TEST_ZIP, null, Flavor.OSS, null, + "flavor not allowed for elasticsearch distribution [testdistro]"); + } + + public void testBundledJdkDefault() { + ElasticsearchDistribution distro = checkDistro(createProject(null), + "testdistro", "5.0.0", Type.ARCHIVE, Platform.LINUX, null, true); + assertTrue(distro.getBundledJdk()); + } + + public void testBundledJdkForIntegTest() { + assertDistroError(createProject(null), "testdistro", "5.0.0", Type.INTEG_TEST_ZIP, null, null, true, + "bundledJdk not allowed for elasticsearch distribution [testdistro]"); + } + + public void testCurrentVersionIntegTestZip() { + Project project = createProject(null); + Project archiveProject = ProjectBuilder.builder().withParent(archivesProject).withName("integ-test-zip").build(); + archiveProject.getConfigurations().create("default"); + archiveProject.getArtifacts().add("default", new File("doesnotmatter")); + createDistro(project, "distro", + VersionProperties.getElasticsearch(), Type.INTEG_TEST_ZIP, null, null, null); + checkPlugin(project); + } + + public void testCurrentVersionArchives() { + for (Platform platform : Platform.values()) { + for (Flavor flavor : Flavor.values()) { + for (boolean bundledJdk : new boolean[] { true, false}) { + // create a new project in each iteration, so that we know we are resolving the only additional project being created + Project project = createProject(null); + String projectName = projectName(platform.toString(), flavor, bundledJdk); + projectName += (platform == Platform.WINDOWS ? "-zip" : "-tar"); + Project archiveProject = ProjectBuilder.builder().withParent(archivesProject).withName(projectName).build(); + archiveProject.getConfigurations().create("default"); + archiveProject.getArtifacts().add("default", new File("doesnotmatter")); + createDistro(project, "distro", + VersionProperties.getElasticsearch(), Type.ARCHIVE, platform, flavor, bundledJdk); + checkPlugin(project); + } + } + } + } + + public void testCurrentVersionPackages() { + for (Type packageType : new Type[] { Type.RPM, Type.DEB }) { + for (Flavor flavor : Flavor.values()) { + for (boolean bundledJdk : new boolean[] { true, false}) { + Project project = createProject(null); + String projectName = projectName(packageType.toString(), flavor, bundledJdk); + Project packageProject = ProjectBuilder.builder().withParent(packagesProject).withName(projectName).build(); + packageProject.getConfigurations().create("default"); + packageProject.getArtifacts().add("default", new File("doesnotmatter")); + createDistro(project, "distro", + VersionProperties.getElasticsearch(), packageType, null, flavor, bundledJdk); + checkPlugin(project); + } + } + } + } + + public void testLocalBwcArchives() { + for (Platform platform : Platform.values()) { + for (Flavor flavor : Flavor.values()) { + // note: no non bundled jdk for bwc + String configName = projectName(platform.toString(), flavor, true); + configName += (platform == Platform.WINDOWS ? "-zip" : "-tar"); + + checkBwc("minor", configName, BWC_MINOR_VERSION, BWC_MINOR, Type.ARCHIVE, platform, flavor); + checkBwc("staged", configName, BWC_STAGED_VERSION, BWC_STAGED, Type.ARCHIVE, platform, flavor); + checkBwc("bugfix", configName, BWC_BUGFIX_VERSION, BWC_BUGFIX, Type.ARCHIVE, platform, flavor); + checkBwc("maintenance", configName, BWC_MAINTENANCE_VERSION, BWC_MAINTENANCE, Type.ARCHIVE, platform, flavor); + } + } + } + + public void testLocalBwcPackages() { + for (Type packageType : new Type[] { Type.RPM, Type.DEB }) { + for (Flavor flavor : Flavor.values()) { + // note: no non bundled jdk for bwc + String configName = projectName(packageType.toString(), flavor, true); + + checkBwc("minor", configName, BWC_MINOR_VERSION, BWC_MINOR, packageType, null, flavor); + checkBwc("staged", configName, BWC_STAGED_VERSION, BWC_STAGED, packageType, null, flavor); + checkBwc("bugfix", configName, BWC_BUGFIX_VERSION, BWC_BUGFIX, packageType, null, flavor); + checkBwc("maintenance", configName, BWC_MAINTENANCE_VERSION, BWC_MAINTENANCE, packageType, null, flavor); + } + } + } + + private void assertDistroError(Project project, String name, String version, Type type, Platform platform, + Flavor flavor, Boolean bundledJdk, String message) { + IllegalArgumentException e = expectThrows(IllegalArgumentException.class, + () -> checkDistro(project, name, version, type, platform, flavor, bundledJdk)); + assertThat(e.getMessage(), containsString(message)); + } + + private ElasticsearchDistribution createDistro(Project project, String name, String version, Type type, + Platform platform, Flavor flavor, Boolean bundledJdk) { + @SuppressWarnings("unchecked") + NamedDomainObjectContainer distros = + (NamedDomainObjectContainer) project.getExtensions().getByName("elasticsearch_distributions"); + return distros.create(name, distro -> { + if (version != null) { + distro.setVersion(version); + } + if (type != null) { + distro.setType(type); + } + if (platform != null) { + distro.setPlatform(platform); + } + if (flavor != null) { + distro.setFlavor(flavor); + } + if (bundledJdk != null) { + distro.setBundledJdk(bundledJdk); + } + }); + } + + // create a distro and finalize its configuration + private ElasticsearchDistribution checkDistro(Project project, String name, String version, Type type, + Platform platform, Flavor flavor, Boolean bundledJdk) { + ElasticsearchDistribution distribution = createDistro(project, name, version, type, platform, flavor, bundledJdk); + distribution.finalizeValues(); + return distribution; + } + + // check the download plugin can be fully configured + private void checkPlugin(Project project) { + DistributionDownloadPlugin plugin = project.getPlugins().getPlugin(DistributionDownloadPlugin.class); + plugin.setupDistributions(project); + } + + private void checkBwc(String projectName, String config, Version version, BwcVersions bwcVersions, + Type type, Platform platform, Flavor flavor) { + Project project = createProject(bwcVersions); + Project archiveProject = ProjectBuilder.builder().withParent(bwcProject).withName(projectName).build(); + archiveProject.getConfigurations().create(config); + archiveProject.getArtifacts().add(config, new File("doesnotmatter")); + createDistro(project, "distro", version.toString(), type, platform, flavor, true); + checkPlugin(project); + } + + private Project createProject(BwcVersions bwcVersions) { + rootProject = ProjectBuilder.builder().build(); + Project distributionProject = ProjectBuilder.builder().withParent(rootProject).withName("distribution").build(); + archivesProject = ProjectBuilder.builder().withParent(distributionProject).withName("archives").build(); + packagesProject = ProjectBuilder.builder().withParent(distributionProject).withName("packages").build(); + bwcProject = ProjectBuilder.builder().withParent(distributionProject).withName("bwc").build(); + Project project = ProjectBuilder.builder().withParent(rootProject).build(); + project.getExtensions().getExtraProperties().set("bwcVersions", bwcVersions); + project.getPlugins().apply("elasticsearch.distribution-download"); + return project; + } + + private static String projectName(String base, Flavor flavor, boolean bundledJdk) { + String prefix = ""; + if (flavor == Flavor.OSS) { + prefix += "oss-"; + } + if (bundledJdk == false) { + prefix += "no-jdk-"; + } + + return prefix + base; + } +} diff --git a/buildSrc/src/test/java/org/elasticsearch/gradle/EmptyDirTaskTests.java b/buildSrc/src/test/java/org/elasticsearch/gradle/EmptyDirTaskTests.java new file mode 100644 index 0000000000000..9cb49e8efd418 --- /dev/null +++ b/buildSrc/src/test/java/org/elasticsearch/gradle/EmptyDirTaskTests.java @@ -0,0 +1,78 @@ +/* + * 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.gradle; + +import java.io.File; +import java.io.IOException; + +import org.elasticsearch.gradle.test.GradleUnitTestCase; +import org.gradle.api.Project; +import org.gradle.testfixtures.ProjectBuilder; + +public class EmptyDirTaskTests extends GradleUnitTestCase { + + public void testCreateEmptyDir() throws Exception { + Project project = ProjectBuilder.builder().build(); + EmptyDirTask emptyDirTask = project.getTasks().create("emptyDirTask", EmptyDirTask.class); + assertEquals(0755, emptyDirTask.getDirMode()); + + // generate a new temporary folder and make sure it does not exists + File newEmptyFolder = getNewNonExistingTempFolderFile(project); + + emptyDirTask.setDir(newEmptyFolder); + emptyDirTask.create(); + + assertTrue(newEmptyFolder.exists()); + assertTrue(newEmptyFolder.isDirectory()); + assertTrue(newEmptyFolder.canExecute()); + assertTrue(newEmptyFolder.canRead()); + assertTrue(newEmptyFolder.canWrite()); + + // cleanup + newEmptyFolder.delete(); + } + + public void testCreateEmptyDirNoPermissions() throws Exception { + Project project = ProjectBuilder.builder().build(); + EmptyDirTask emptyDirTask = project.getTasks().create("emptyDirTask", EmptyDirTask.class); + emptyDirTask.setDirMode(0000); + + // generate a new temporary folder and make sure it does not exists + File newEmptyFolder = getNewNonExistingTempFolderFile(project); + + emptyDirTask.setDir(newEmptyFolder); + emptyDirTask.create(); + + assertTrue(newEmptyFolder.exists()); + assertTrue(newEmptyFolder.isDirectory()); + assertFalse(newEmptyFolder.canExecute()); + assertFalse(newEmptyFolder.canRead()); + assertFalse(newEmptyFolder.canWrite()); + + // cleanup + newEmptyFolder.delete(); + } + + private File getNewNonExistingTempFolderFile(Project project) throws IOException { + File newEmptyFolder = new File(project.getBuildDir(), "empty-dir"); + assertFalse(newEmptyFolder.exists()); + return newEmptyFolder; + } + +} diff --git a/buildSrc/src/test/java/org/elasticsearch/gradle/plugin/PluginPropertiesExtensionTests.java b/buildSrc/src/test/java/org/elasticsearch/gradle/plugin/PluginPropertiesExtensionTests.java new file mode 100644 index 0000000000000..b93d400f3e057 --- /dev/null +++ b/buildSrc/src/test/java/org/elasticsearch/gradle/plugin/PluginPropertiesExtensionTests.java @@ -0,0 +1,58 @@ +/* + * 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.gradle.plugin; + +import org.elasticsearch.gradle.test.GradleUnitTestCase; +import org.gradle.api.Project; +import org.gradle.api.plugins.JavaPlugin; +import org.gradle.testfixtures.ProjectBuilder; + +public class PluginPropertiesExtensionTests extends GradleUnitTestCase { + + public void testCreatingPluginPropertiesExtensionWithNameAndVersion() { + String projectName = "Test"; + String projectVersion = "5.0"; + + PluginPropertiesExtension pluginPropertiesExtension = + new PluginPropertiesExtension(this.createProject(projectName, projectVersion)); + + assertEquals(projectName, pluginPropertiesExtension.getName()); + assertEquals(projectVersion, pluginPropertiesExtension.getVersion()); + } + + public void testCreatingPluginPropertiesExtensionWithNameWithoutVersion() { + String projectName = "Test"; + + PluginPropertiesExtension pluginPropertiesExtension = + new PluginPropertiesExtension(this.createProject(projectName, null)); + + assertEquals(projectName, pluginPropertiesExtension.getName()); + assertEquals("unspecified", pluginPropertiesExtension.getVersion()); + } + + private Project createProject(String projectName, String version) { + Project project = ProjectBuilder.builder().withName(projectName).build(); + project.setVersion(version); + + project.getPlugins().apply(JavaPlugin.class); + + return project; + } +} diff --git a/buildSrc/src/test/java/org/elasticsearch/gradle/precommit/DependencyLicensesTaskTests.java b/buildSrc/src/test/java/org/elasticsearch/gradle/precommit/DependencyLicensesTaskTests.java new file mode 100644 index 0000000000000..397c5938fba23 --- /dev/null +++ b/buildSrc/src/test/java/org/elasticsearch/gradle/precommit/DependencyLicensesTaskTests.java @@ -0,0 +1,268 @@ +package org.elasticsearch.gradle.precommit; + +import org.elasticsearch.gradle.test.GradleUnitTestCase; +import org.gradle.api.GradleException; +import org.gradle.api.Project; +import org.gradle.api.artifacts.Dependency; +import org.gradle.api.file.FileCollection; +import org.gradle.api.plugins.JavaPlugin; +import org.gradle.testfixtures.ProjectBuilder; +import org.junit.Before; +import org.junit.Rule; +import org.junit.Test; +import org.junit.rules.ExpectedException; + +import java.io.File; +import java.io.IOException; +import java.nio.charset.StandardCharsets; +import java.nio.file.Files; +import java.nio.file.Path; +import java.nio.file.StandardOpenOption; +import java.security.NoSuchAlgorithmException; +import java.util.HashMap; +import java.util.Map; + +import static org.hamcrest.CoreMatchers.containsString; + +public class DependencyLicensesTaskTests extends GradleUnitTestCase { + + @Rule + public ExpectedException expectedException = ExpectedException.none(); + + private UpdateShasTask updateShas; + + private DependencyLicensesTask task; + + private Project project; + + private Dependency dependency; + + @Before + public void prepare() { + project = createProject(); + task = createDependencyLicensesTask(project); + updateShas = createUpdateShasTask(project, task); + dependency = project.getDependencies().localGroovy(); + } + + @Test + public void givenProjectWithLicensesDirButNoDependenciesThenShouldThrowException() throws Exception { + expectedException.expect(GradleException.class); + expectedException.expectMessage(containsString("exists, but there are no dependencies")); + + getLicensesDir(project).mkdir(); + task.checkDependencies(); + } + + @Test + public void givenProjectWithoutLicensesDirButWithDependenciesThenShouldThrowException() throws Exception { + expectedException.expect(GradleException.class); + expectedException.expectMessage(containsString("does not exist, but there are dependencies")); + + project.getDependencies().add("compile", dependency); + task.checkDependencies(); + } + + @Test + public void givenProjectWithoutLicensesDirNorDependenciesThenShouldReturnSilently() throws Exception { + task.checkDependencies(); + } + + @Test + public void givenProjectWithDependencyButNoShaFileThenShouldReturnException() throws Exception { + expectedException.expect(GradleException.class); + expectedException.expectMessage(containsString("Missing SHA for ")); + + File licensesDir = getLicensesDir(project); + createFileIn(licensesDir, "groovy-all-LICENSE.txt", ""); + createFileIn(licensesDir, "groovy-all-NOTICE.txt", ""); + + project.getDependencies().add("compile", project.getDependencies().localGroovy()); + task.checkDependencies(); + } + + @Test + public void givenProjectWithDependencyButNoLicenseFileThenShouldReturnException() throws Exception { + expectedException.expect(GradleException.class); + expectedException.expectMessage(containsString("Missing LICENSE for ")); + + project.getDependencies().add("compile", project.getDependencies().localGroovy()); + + getLicensesDir(project).mkdir(); + updateShas.updateShas(); + task.checkDependencies(); + } + + @Test + public void givenProjectWithDependencyButNoNoticeFileThenShouldReturnException() throws Exception { + expectedException.expect(GradleException.class); + expectedException.expectMessage(containsString("Missing NOTICE for ")); + + project.getDependencies().add("compile", dependency); + + createFileIn(getLicensesDir(project), "groovy-all-LICENSE.txt", ""); + + updateShas.updateShas(); + task.checkDependencies(); + } + + @Test + public void givenProjectWithDependencyAndEverythingInOrderThenShouldReturnSilently() throws Exception { + project.getDependencies().add("compile", dependency); + + File licensesDir = getLicensesDir(project); + + createAllDefaultDependencyFiles(licensesDir, "groovy-all"); + task.checkDependencies(); + } + + @Test + public void givenProjectWithALicenseButWithoutTheDependencyThenShouldThrowException() throws Exception { + expectedException.expect(GradleException.class); + expectedException.expectMessage(containsString("Unused license ")); + + project.getDependencies().add("compile", dependency); + + File licensesDir = getLicensesDir(project); + createAllDefaultDependencyFiles(licensesDir, "groovy-all"); + createFileIn(licensesDir, "non-declared-LICENSE.txt", ""); + + task.checkDependencies(); + } + + @Test + public void givenProjectWithANoticeButWithoutTheDependencyThenShouldThrowException() throws Exception { + expectedException.expect(GradleException.class); + expectedException.expectMessage(containsString("Unused notice ")); + + project.getDependencies().add("compile", dependency); + + File licensesDir = getLicensesDir(project); + createAllDefaultDependencyFiles(licensesDir, "groovy-all"); + createFileIn(licensesDir, "non-declared-NOTICE.txt", ""); + + task.checkDependencies(); + } + + @Test + public void givenProjectWithAShaButWithoutTheDependencyThenShouldThrowException() throws Exception { + expectedException.expect(GradleException.class); + expectedException.expectMessage(containsString("Unused sha files found: \n")); + + project.getDependencies().add("compile", dependency); + + File licensesDir = getLicensesDir(project); + createAllDefaultDependencyFiles(licensesDir, "groovy-all"); + createFileIn(licensesDir, "non-declared.sha1", ""); + + task.checkDependencies(); + } + + @Test + public void givenProjectWithADependencyWithWrongShaThenShouldThrowException() throws Exception { + expectedException.expect(GradleException.class); + expectedException.expectMessage(containsString("SHA has changed! Expected ")); + + project.getDependencies().add("compile", dependency); + + File licensesDir = getLicensesDir(project); + createAllDefaultDependencyFiles(licensesDir, "groovy-all"); + + Path groovySha = Files + .list(licensesDir.toPath()) + .filter(file -> file.toFile().getName().contains("sha")) + .findFirst().get(); + + Files.write(groovySha, new byte[] { 1 }, StandardOpenOption.CREATE); + + task.checkDependencies(); + } + + @Test + public void givenProjectWithADependencyMappingThenShouldReturnSilently() throws Exception { + project.getDependencies().add("compile", dependency); + + File licensesDir = getLicensesDir(project); + createAllDefaultDependencyFiles(licensesDir, "groovy"); + + Map mappings = new HashMap<>(); + mappings.put("from", "groovy-all"); + mappings.put("to", "groovy"); + + task.mapping(mappings); + task.checkDependencies(); + } + + @Test + public void givenProjectWithAIgnoreShaConfigurationAndNoShaFileThenShouldReturnSilently() throws Exception { + project.getDependencies().add("compile", dependency); + + File licensesDir = getLicensesDir(project); + createFileIn(licensesDir, "groovy-all-LICENSE.txt", ""); + createFileIn(licensesDir, "groovy-all-NOTICE.txt", ""); + + task.ignoreSha("groovy-all"); + task.checkDependencies(); + } + + @Test + public void givenProjectWithoutLicensesDirWhenAskingForShaFilesThenShouldThrowException() { + expectedException.expect(GradleException.class); + expectedException.expectMessage(containsString("isn't a valid directory")); + + task.getShaFiles(); + } + + private Project createProject() { + Project project = ProjectBuilder.builder().build(); + project.getPlugins().apply(JavaPlugin.class); + + return project; + } + + private void createAllDefaultDependencyFiles(File licensesDir, String dependencyName) throws IOException, NoSuchAlgorithmException { + createFileIn(licensesDir, dependencyName + "-LICENSE.txt", ""); + createFileIn(licensesDir, dependencyName + "-NOTICE.txt", ""); + + updateShas.updateShas(); + } + + private File getLicensesDir(Project project) { + return getFile(project, "licenses"); + } + + private File getFile(Project project, String fileName) { + return project.getProjectDir().toPath().resolve(fileName).toFile(); + } + + private void createFileIn(File parent, String name, String content) throws IOException { + parent.mkdir(); + + Path file = parent.toPath().resolve(name); + file.toFile().createNewFile(); + + Files.write(file, content.getBytes(StandardCharsets.UTF_8)); + } + + private UpdateShasTask createUpdateShasTask(Project project, DependencyLicensesTask dependencyLicensesTask) { + UpdateShasTask task = project.getTasks() + .register("updateShas", UpdateShasTask.class) + .get(); + + task.setParentTask(dependencyLicensesTask); + return task; + } + + private DependencyLicensesTask createDependencyLicensesTask(Project project) { + DependencyLicensesTask task = project.getTasks() + .register("dependencyLicenses", DependencyLicensesTask.class) + .get(); + + task.setDependencies(getDependencies(project)); + return task; + } + + private FileCollection getDependencies(Project project) { + return project.getConfigurations().getByName("compile"); + } +} diff --git a/buildSrc/src/test/java/org/elasticsearch/gradle/precommit/UpdateShasTaskTests.java b/buildSrc/src/test/java/org/elasticsearch/gradle/precommit/UpdateShasTaskTests.java new file mode 100644 index 0000000000000..62ac9600a8307 --- /dev/null +++ b/buildSrc/src/test/java/org/elasticsearch/gradle/precommit/UpdateShasTaskTests.java @@ -0,0 +1,140 @@ +package org.elasticsearch.gradle.precommit; + +import org.apache.commons.io.FileUtils; +import org.elasticsearch.gradle.test.GradleUnitTestCase; +import org.gradle.api.GradleException; +import org.gradle.api.Project; +import org.gradle.api.artifacts.Dependency; +import org.gradle.api.file.FileCollection; +import org.gradle.api.plugins.JavaPlugin; +import org.gradle.testfixtures.ProjectBuilder; +import org.junit.Before; +import org.junit.Rule; +import org.junit.Test; +import org.junit.rules.ExpectedException; + +import java.io.File; +import java.io.IOException; +import java.nio.file.Files; +import java.nio.file.Path; +import java.nio.file.StandardOpenOption; +import java.security.NoSuchAlgorithmException; + +import static org.hamcrest.CoreMatchers.containsString; +import static org.hamcrest.CoreMatchers.equalTo; + +public class UpdateShasTaskTests extends GradleUnitTestCase { + + @Rule + public ExpectedException expectedException = ExpectedException.none(); + + private UpdateShasTask task; + + private Project project; + + private Dependency dependency; + + @Before + public void prepare() throws IOException { + project = createProject(); + task = createUpdateShasTask(project); + dependency = project.getDependencies().localGroovy(); + + } + + @Test + public void whenDependencyDoesntExistThenShouldDeleteDependencySha() + throws IOException, NoSuchAlgorithmException { + + File unusedSha = createFileIn(getLicensesDir(project), "test.sha1", ""); + task.updateShas(); + + assertFalse(unusedSha.exists()); + } + + @Test + public void whenDependencyExistsButShaNotThenShouldCreateNewShaFile() + throws IOException, NoSuchAlgorithmException { + project.getDependencies().add("compile", dependency); + + getLicensesDir(project).mkdir(); + task.updateShas(); + + Path groovySha = Files + .list(getLicensesDir(project).toPath()) + .findFirst().get(); + + assertTrue(groovySha.toFile().getName().startsWith("groovy-all")); + } + + @Test + public void whenDependencyAndWrongShaExistsThenShouldNotOverwriteShaFile() + throws IOException, NoSuchAlgorithmException { + project.getDependencies().add("compile", dependency); + + File groovyJar = task.getParentTask().getDependencies().getFiles().iterator().next(); + String groovyShaName = groovyJar.getName() + ".sha1"; + + File groovySha = createFileIn(getLicensesDir(project), groovyShaName, "content"); + task.updateShas(); + + assertThat(FileUtils.readFileToString(groovySha), equalTo("content")); + } + + @Test + public void whenLicensesDirDoesntExistThenShouldThrowException() + throws IOException, NoSuchAlgorithmException { + expectedException.expect(GradleException.class); + expectedException.expectMessage(containsString("isn't a valid directory")); + + task.updateShas(); + } + + private Project createProject() { + Project project = ProjectBuilder.builder().build(); + project.getPlugins().apply(JavaPlugin.class); + + return project; + } + + private File getLicensesDir(Project project) { + return getFile(project, "licenses"); + } + + private File getFile(Project project, String fileName) { + return project.getProjectDir().toPath().resolve(fileName).toFile(); + } + + private File createFileIn(File parent, String name, String content) throws IOException { + parent.mkdir(); + + Path path = parent.toPath().resolve(name); + File file = path.toFile(); + + Files.write(path, content.getBytes(), StandardOpenOption.CREATE); + + return file; + } + + private UpdateShasTask createUpdateShasTask(Project project) { + UpdateShasTask task = project.getTasks() + .register("updateShas", UpdateShasTask.class) + .get(); + + task.setParentTask(createDependencyLicensesTask(project)); + return task; + } + + private DependencyLicensesTask createDependencyLicensesTask(Project project) { + DependencyLicensesTask task = project.getTasks() + .register("dependencyLicenses", DependencyLicensesTask.class) + .get(); + + task.setDependencies(getDependencies(project)); + return task; + } + + private FileCollection getDependencies(Project project) { + return project.getConfigurations().getByName("compile"); + } +} diff --git a/buildSrc/src/testKit/distribution-download/build.gradle b/buildSrc/src/testKit/distribution-download/build.gradle new file mode 100644 index 0000000000000..82f76ff65acb7 --- /dev/null +++ b/buildSrc/src/testKit/distribution-download/build.gradle @@ -0,0 +1,44 @@ +import org.elasticsearch.gradle.BwcVersions +import org.elasticsearch.gradle.Version +/* + * 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. + */ + +// bring in build-tools onto the classpath +plugins { + id 'elasticsearch.global-build-info' apply false +} +project.gradle.projectsEvaluated { + // wire the download service url to wiremock + String fakeDownloadService = System.getProperty('tests.download_service') + if (fakeDownloadService != null) { + println rootProject.repositories.asMap.keySet() + IvyArtifactRepository repository = (IvyArtifactRepository) rootProject.repositories.getByName("elasticsearch-downloads") + repository.setUrl(fakeDownloadService) + repository = (IvyArtifactRepository) project('subproj').repositories.getByName("elasticsearch-downloads") + repository.setUrl(fakeDownloadService) + } +} + +Version currentVersion = Version.fromString("2.0.0") +BwcVersions versions = new BwcVersions(new TreeSet<>( + Arrays.asList(Version.fromString("1.0.0"), Version.fromString("1.0.1"), Version.fromString("1.1.0"), currentVersion)), + currentVersion) +allprojects { + ext.bwcVersions = versions +} \ No newline at end of file diff --git a/buildSrc/src/testKit/distribution-download/distribution/build.gradle b/buildSrc/src/testKit/distribution-download/distribution/build.gradle new file mode 100644 index 0000000000000..d0fdf038a602d --- /dev/null +++ b/buildSrc/src/testKit/distribution-download/distribution/build.gradle @@ -0,0 +1,35 @@ + +String distroConfig = System.getProperty('tests.local_distro.config') +if (distroConfig != null) { + // setup the test distribution as an artifact of this project + String distroType = System.getProperty('tests.distro.type') + + Task buildDistro + File buildFile + if (['rpm', 'deb'].contains(distroType)) { + buildDistro = project.tasks.create("build" + distroType.capitalize(), Copy) { + from 'files' + into 'build/files' + include 'fake_elasticsearch.tar.gz' + // this shouldn't be extracted so we just rename the file so it is a dummy package + rename { filename -> filename.replace('tar.gz', distroType) } + } + buildFile = project.file("build/files/fake_elasticsearch." + distroType) + } else { + String distroPlatform = System.getProperty('tests.distro.platform') + String extension = "tar.gz" + if (distroType == 'archive' && distroPlatform == 'windows' || distroType == 'integ-test-zip') { + extension = "zip" + } + // copy file as is + buildDistro = project.tasks.create("buildArchive", Copy) { + from 'files' + include "fake_elasticsearch.${extension}" + into 'build/files' + } + buildFile = project.file("build/files/fake_elasticsearch.${extension}") + } + + configurations.create(distroConfig) + artifacts.add(distroConfig, [file: buildFile, builtBy: buildDistro]) +} diff --git a/buildSrc/src/testKit/distribution-download/distribution/files/fake_elasticsearch.tar.gz b/buildSrc/src/testKit/distribution-download/distribution/files/fake_elasticsearch.tar.gz new file mode 100644 index 0000000000000000000000000000000000000000..a9c43bc71c5ce6ea28b33720cbbbfcf2ab49e474 GIT binary patch literal 188 zcmV;t07L&DiwFSabnRRK1MSl73c?^5hT$EGC(v?!%;QSTt*0uLnd(eb+BmJ8QqBlj*~*zh*xZdQ zxy7*wsg-c?VTj}Ey0on8-t05q<5_pYFXVsfhpc~KOa3~IQ~qg8{=cx6|I7TBXQcP2 qjoFaDlE&IX$SgPf;Lr0100000000000002MzTE*MbZ*7~C;$N2s$rf0 literal 0 HcmV?d00001 diff --git a/buildSrc/src/testKit/distribution-download/distribution/files/fake_elasticsearch.zip b/buildSrc/src/testKit/distribution-download/distribution/files/fake_elasticsearch.zip new file mode 100644 index 0000000000000000000000000000000000000000..42b572e37f3fbfbb90dee93ecbff745d37988879 GIT binary patch literal 568 zcmWIWW@h1H00E1zJ$_&Yln`N%VMxtMEH24RE>2A>O3u(V)HBjE)(;KgWMJM?@;2u7 zG^W_n3T_5QmamKq3@jo*eF1ROtk6u8z+qZaW*)@E1EiX$ipNA0cY_@aGJYa92QxCs zG2;pV320CVFuZjHF;R?#2L~%8I52_&VKQc5Ae#&db!;X>f(B?h$n7MUuF3#2oq=IV e;{rUUqu3f6g3BsaHlP^{%s{9NjOhO$4g&zPL2_UK literal 0 HcmV?d00001 diff --git a/buildSrc/src/testKit/distribution-download/settings.gradle b/buildSrc/src/testKit/distribution-download/settings.gradle new file mode 100644 index 0000000000000..1a64261480d6f --- /dev/null +++ b/buildSrc/src/testKit/distribution-download/settings.gradle @@ -0,0 +1,8 @@ +include 'subproj' + +String distroProject = System.getProperty('tests.local_distro.project') +System.out.println("local distro project: " + distroProject); +if (distroProject != null) { + include distroProject + project(distroProject).projectDir = new File(rootDir, 'distribution') +} diff --git a/buildSrc/src/testKit/distribution-download/subproj/build.gradle b/buildSrc/src/testKit/distribution-download/subproj/build.gradle new file mode 100644 index 0000000000000..7584ce9a76d1c --- /dev/null +++ b/buildSrc/src/testKit/distribution-download/subproj/build.gradle @@ -0,0 +1,58 @@ + +plugins { + id 'elasticsearch.distribution-download' +} + +String distroVersion = System.getProperty('tests.distro.version') +String distroType = System.getProperty('tests.distro.type') +String distroPlatform = System.getProperty('tests.distro.platform') +String distroFlavor = System.getProperty('tests.distro.flavor') +String distroBundledJdk = System.getProperty('tests.distro.bundledJdk') + +elasticsearch_distributions { + test_distro { + if (distroVersion != null) { + version = distroVersion + } + if (distroType != null) { + type = distroType + } + if (distroPlatform != null) { + platform = distroPlatform + } + if (distroFlavor != null) { + flavor = distroFlavor + } + if (distroBundledJdk != null) { + bundledJdk = Boolean.parseBoolean(distroBundledJdk) + } + } +} + +task assertDistroFile { + dependsOn elasticsearch_distributions.test_distro + doLast { + File distroFile = new File(elasticsearch_distributions.test_distro.toString()) + if (distroFile.exists() == false) { + throw new GradleException("distro file does not exist: ${distroFile}") + } + if (distroFile.isFile() == false) { + throw new GradleException("distro file is not a regular file: ${distroFile}") + } + } +} + +if (['rpm', 'deb'].contains(distroType) == false) { + task assertDistroExtracted { + dependsOn elasticsearch_distributions.test_distro.extracted, assertDistroFile + doLast { + File distroExtracted = new File(elasticsearch_distributions.test_distro.extracted.toString()) + if (distroExtracted.exists() == false) { + throw new GradleException("extracted does not exist: ${distroExtracted}") + } + if (distroExtracted.isDirectory() == false) { + throw new GradleException("extracted distro is not a directory: ${distroExtracted}") + } + } + } +} diff --git a/client/benchmark/README.md b/client/benchmark/README.md index 2562a139bec0d..31ac1ea8ae548 100644 --- a/client/benchmark/README.md +++ b/client/benchmark/README.md @@ -54,9 +54,9 @@ Example invocation: The parameters are in order: -* Client type: Use either "rest" or "transport" +* Client type: Always "rest" * Benchmark type: Use either "bulk" or "search" * Benchmark target host IP (the host where Elasticsearch is running) * name of the index -* a search request body (remember to escape double quotes). The `TransportClientBenchmark` uses `QueryBuilders.wrapperQuery()` internally which automatically adds a root key `query`, so it must not be present in the command line parameter. +* a search request body (remember to escape double quotes). * A comma-separated list of target throughput rates diff --git a/client/benchmark/build.gradle b/client/benchmark/build.gradle index a53f102034009..41e166558d522 100644 --- a/client/benchmark/build.gradle +++ b/client/benchmark/build.gradle @@ -34,16 +34,10 @@ test.enabled = false dependencies { compile 'org.apache.commons:commons-math3:3.2' - compile("org.elasticsearch.client:elasticsearch-rest-client:${version}") + compile project(':server') + compile project(':client:rest') // bottleneck should be the client, not Elasticsearch compile project(path: ':client:client-benchmark-noop-api-plugin') - // for transport client - compile("org.elasticsearch:elasticsearch:${version}") - compile("org.elasticsearch.client:transport:${version}") - compile project(path: ':modules:transport-netty4', configuration: 'runtime') - compile project(path: ':modules:reindex', configuration: 'runtime') - compile project(path: ':modules:lang-mustache', configuration: 'runtime') - compile project(path: ':modules:percolator', configuration: 'runtime') } // No licenses for our benchmark deps (we don't ship benchmarks) diff --git a/client/benchmark/src/main/java/org/elasticsearch/client/benchmark/BenchmarkMain.java b/client/benchmark/src/main/java/org/elasticsearch/client/benchmark/BenchmarkMain.java index 317f0bf479779..d1363c0811190 100644 --- a/client/benchmark/src/main/java/org/elasticsearch/client/benchmark/BenchmarkMain.java +++ b/client/benchmark/src/main/java/org/elasticsearch/client/benchmark/BenchmarkMain.java @@ -19,7 +19,6 @@ package org.elasticsearch.client.benchmark; import org.elasticsearch.client.benchmark.rest.RestClientBenchmark; -import org.elasticsearch.client.benchmark.transport.TransportClientBenchmark; import org.elasticsearch.common.SuppressForbidden; import java.util.Arrays; @@ -30,9 +29,6 @@ public static void main(String[] args) throws Exception { String type = args[0]; AbstractBenchmark benchmark = null; switch (type) { - case "transport": - benchmark = new TransportClientBenchmark(); - break; case "rest": benchmark = new RestClientBenchmark(); break; diff --git a/client/benchmark/src/main/java/org/elasticsearch/client/benchmark/transport/TransportClientBenchmark.java b/client/benchmark/src/main/java/org/elasticsearch/client/benchmark/transport/TransportClientBenchmark.java deleted file mode 100644 index 66b3d7bf060ad..0000000000000 --- a/client/benchmark/src/main/java/org/elasticsearch/client/benchmark/transport/TransportClientBenchmark.java +++ /dev/null @@ -1,127 +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.client.benchmark.transport; - -import org.elasticsearch.ElasticsearchException; -import org.elasticsearch.action.bulk.BulkRequest; -import org.elasticsearch.action.bulk.BulkResponse; -import org.elasticsearch.action.index.IndexRequest; -import org.elasticsearch.action.search.SearchRequest; -import org.elasticsearch.action.search.SearchResponse; -import org.elasticsearch.client.benchmark.AbstractBenchmark; -import org.elasticsearch.client.benchmark.ops.bulk.BulkRequestExecutor; -import org.elasticsearch.client.benchmark.ops.search.SearchRequestExecutor; -import org.elasticsearch.client.transport.TransportClient; -import org.elasticsearch.common.settings.Settings; -import org.elasticsearch.common.transport.TransportAddress; -import org.elasticsearch.common.xcontent.XContentType; -import org.elasticsearch.index.query.QueryBuilders; -import org.elasticsearch.plugin.noop.NoopPlugin; -import org.elasticsearch.plugin.noop.action.bulk.NoopBulkAction; -import org.elasticsearch.plugin.noop.action.search.NoopSearchAction; -import org.elasticsearch.rest.RestStatus; -import org.elasticsearch.search.builder.SearchSourceBuilder; -import org.elasticsearch.transport.client.PreBuiltTransportClient; - -import java.net.InetAddress; -import java.nio.charset.StandardCharsets; -import java.util.List; -import java.util.concurrent.ExecutionException; - -public final class TransportClientBenchmark extends AbstractBenchmark { - public static void main(String[] args) throws Exception { - TransportClientBenchmark benchmark = new TransportClientBenchmark(); - benchmark.run(args); - } - - @Override - protected TransportClient client(String benchmarkTargetHost) throws Exception { - TransportClient client = new PreBuiltTransportClient(Settings.EMPTY, NoopPlugin.class); - client.addTransportAddress(new TransportAddress(InetAddress.getByName(benchmarkTargetHost), 9300)); - return client; - } - - @Override - protected BulkRequestExecutor bulkRequestExecutor(TransportClient client, String indexName, String typeName) { - return new TransportBulkRequestExecutor(client, indexName, typeName); - } - - @Override - protected SearchRequestExecutor searchRequestExecutor(TransportClient client, String indexName) { - return new TransportSearchRequestExecutor(client, indexName); - } - - private static final class TransportBulkRequestExecutor implements BulkRequestExecutor { - private final TransportClient client; - private final String indexName; - private final String typeName; - - TransportBulkRequestExecutor(TransportClient client, String indexName, String typeName) { - this.client = client; - this.indexName = indexName; - this.typeName = typeName; - } - - @Override - public boolean bulkIndex(List bulkData) { - BulkRequest bulkRequest = new BulkRequest(); - for (String bulkItem : bulkData) { - bulkRequest.add(new IndexRequest(indexName, typeName).source(bulkItem.getBytes(StandardCharsets.UTF_8), XContentType.JSON)); - } - BulkResponse bulkResponse; - try { - bulkResponse = client.execute(NoopBulkAction.INSTANCE, bulkRequest).get(); - } catch (InterruptedException e) { - Thread.currentThread().interrupt(); - return false; - } catch (ExecutionException e) { - throw new ElasticsearchException(e); - } - return !bulkResponse.hasFailures(); - } - } - - private static final class TransportSearchRequestExecutor implements SearchRequestExecutor { - private final TransportClient client; - private final String indexName; - - private TransportSearchRequestExecutor(TransportClient client, String indexName) { - this.client = client; - this.indexName = indexName; - } - - @Override - public boolean search(String source) { - final SearchResponse response; - try { - final SearchRequest searchRequest = new SearchRequest(indexName); - SearchSourceBuilder searchSourceBuilder = new SearchSourceBuilder(); - searchRequest.source(searchSourceBuilder); - searchSourceBuilder.query(QueryBuilders.wrapperQuery(source)); - response = client.execute(NoopSearchAction.INSTANCE, searchRequest).get(); - return response.status() == RestStatus.OK; - } catch (InterruptedException e) { - Thread.currentThread().interrupt(); - return false; - } catch (ExecutionException e) { - throw new ElasticsearchException(e); - } - } - } -} diff --git a/client/client-benchmark-noop-api-plugin/src/main/java/org/elasticsearch/plugin/noop/action/bulk/NoopBulkAction.java b/client/client-benchmark-noop-api-plugin/src/main/java/org/elasticsearch/plugin/noop/action/bulk/NoopBulkAction.java index 2bfd3b0cc8ed4..e6412099fee72 100644 --- a/client/client-benchmark-noop-api-plugin/src/main/java/org/elasticsearch/plugin/noop/action/bulk/NoopBulkAction.java +++ b/client/client-benchmark-noop-api-plugin/src/main/java/org/elasticsearch/plugin/noop/action/bulk/NoopBulkAction.java @@ -18,10 +18,10 @@ */ package org.elasticsearch.plugin.noop.action.bulk; -import org.elasticsearch.action.Action; +import org.elasticsearch.action.StreamableResponseActionType; import org.elasticsearch.action.bulk.BulkResponse; -public class NoopBulkAction extends Action { +public class NoopBulkAction extends StreamableResponseActionType { public static final String NAME = "mock:data/write/bulk"; public static final NoopBulkAction INSTANCE = new NoopBulkAction(); diff --git a/client/client-benchmark-noop-api-plugin/src/main/java/org/elasticsearch/plugin/noop/action/search/NoopSearchAction.java b/client/client-benchmark-noop-api-plugin/src/main/java/org/elasticsearch/plugin/noop/action/search/NoopSearchAction.java index 9b390e1ffddbc..fb83bda148b11 100644 --- a/client/client-benchmark-noop-api-plugin/src/main/java/org/elasticsearch/plugin/noop/action/search/NoopSearchAction.java +++ b/client/client-benchmark-noop-api-plugin/src/main/java/org/elasticsearch/plugin/noop/action/search/NoopSearchAction.java @@ -18,11 +18,11 @@ */ package org.elasticsearch.plugin.noop.action.search; -import org.elasticsearch.action.Action; +import org.elasticsearch.action.ActionType; import org.elasticsearch.action.search.SearchResponse; import org.elasticsearch.common.io.stream.Writeable; -public class NoopSearchAction extends Action { +public class NoopSearchAction extends ActionType { public static final NoopSearchAction INSTANCE = new NoopSearchAction(); public static final String NAME = "mock:data/read/search"; @@ -30,11 +30,6 @@ private NoopSearchAction() { super(NAME); } - @Override - public SearchResponse newResponse() { - throw new UnsupportedOperationException("usage of Streamable is to be replaced by Writeable"); - } - @Override public Writeable.Reader getResponseReader() { return SearchResponse::new; diff --git a/client/rest-high-level/build.gradle b/client/rest-high-level/build.gradle index c9a3fc486f1da..0e05eb567687a 100644 --- a/client/rest-high-level/build.gradle +++ b/client/rest-high-level/build.gradle @@ -16,6 +16,7 @@ * specific language governing permissions and limitations * under the License. */ +apply plugin: 'elasticsearch.testclusters' apply plugin: 'elasticsearch.build' apply plugin: 'elasticsearch.rest-test' apply plugin: 'nebula.maven-base-publish' @@ -46,32 +47,34 @@ idea { } dependencies { - /* - * Everything in the "shadow" configuration is *not* copied into the - * shadowJar. - */ - compile "org.elasticsearch:elasticsearch:${version}" - compile "org.elasticsearch.client:elasticsearch-rest-client:${version}" - compile "org.elasticsearch.plugin:parent-join-client:${version}" - compile "org.elasticsearch.plugin:aggs-matrix-stats-client:${version}" - compile "org.elasticsearch.plugin:rank-eval-client:${version}" - compile "org.elasticsearch.plugin:lang-mustache-client:${version}" + compile project(':server') + compile project(':client:rest') + compile project(':modules:parent-join') + compile project(':modules:aggs-matrix-stats') + compile project(':modules:rank-eval') + compile project(':modules:lang-mustache') + shadow project(':server') + shadow project(':client:rest') + bundle project(':modules:parent-join') + bundle project(':modules:aggs-matrix-stats') + bundle project(':modules:rank-eval') + bundle project(':modules:lang-mustache') - testCompile "org.elasticsearch.client:test:${version}" - testCompile "org.elasticsearch.test:framework:${version}" + testCompile project(':client:test') + testCompile project(':test:framework') testCompile "com.carrotsearch.randomizedtesting:randomizedtesting-runner:${versions.randomizedrunner}" testCompile "junit:junit:${versions.junit}" //this is needed to make RestHighLevelClientTests#testApiNamingConventions work from IDEs - testCompile "org.elasticsearch:rest-api-spec:${version}" + testCompile project(":rest-api-spec") // Needed for serialization tests: // (In order to serialize a server side class to a client side class or the other way around) if (isEclipse == false || project.path == ":client:rest-high-level-tests") { - testCompile("org.elasticsearch.plugin:x-pack-core:${version}") { + testCompile(project(':x-pack:plugin:core')) { exclude group: 'org.elasticsearch', module: 'elasticsearch-rest-high-level-client' } } - restSpec "org.elasticsearch:rest-api-spec:${version}" + restSpec project(':rest-api-spec') } //we need to copy the yaml spec so we can check naming (see RestHighlevelClientTests#testApiNamingConventions) @@ -113,14 +116,15 @@ if (isEclipse) { File nodeCert = file("./testnode.crt") File nodeTrustStore = file("./testnode.jks") -integTestRunner { +integTest.runner { systemProperty 'tests.rest.cluster.username', System.getProperty('tests.rest.cluster.username', 'test_user') systemProperty 'tests.rest.cluster.password', System.getProperty('tests.rest.cluster.password', 'test-password') } -integTestCluster { +testClusters.integTest { + distribution = "DEFAULT" systemProperty 'es.scripting.update.ctx_in_params', 'false' - setting 'reindex.remote.whitelist', ['"[::1]:*"', '"127.0.0.1:*"'] + setting 'reindex.remote.whitelist', '[ "[::1]:*", "127.0.0.1:*" ]' setting 'xpack.license.self_generated.type', 'trial' setting 'xpack.security.enabled', 'true' setting 'xpack.security.authc.token.enabled', 'true' @@ -129,22 +133,10 @@ integTestCluster { setting 'xpack.security.http.ssl.certificate_authorities', 'testnode.crt' setting 'xpack.security.transport.ssl.truststore.path', 'testnode.jks' setting 'indices.lifecycle.poll_interval', '1000ms' - keystoreSetting 'xpack.security.transport.ssl.truststore.secure_password', 'testnode' - setupCommand 'setupDummyUser', - 'bin/elasticsearch-users', - 'useradd', System.getProperty('tests.rest.cluster.username', 'test_user'), - '-p', System.getProperty('tests.rest.cluster.password', 'test-password'), - '-r', 'superuser' + keystore 'xpack.security.transport.ssl.truststore.secure_password', 'testnode' + user username: System.getProperty('tests.rest.cluster.username', 'test_user'), + password: System.getProperty('tests.rest.cluster.password', 'test-password') + extraConfigFile nodeCert.name, nodeCert extraConfigFile nodeTrustStore.name, nodeTrustStore - waitCondition = { node, ant -> - File tmpFile = new File(node.cwd, 'wait.success') - ant.get(src: "http://${node.httpUri()}/_cluster/health?wait_for_nodes=>=${numNodes}&wait_for_status=yellow", - dest: tmpFile.toString(), - username: System.getProperty('tests.rest.cluster.username', 'test_user'), - password: System.getProperty('tests.rest.cluster.password', 'test-password'), - ignoreerrors: true, - retries: 10) - return tmpFile.exists() - } } diff --git a/client/rest-high-level/src/main/java/org/elasticsearch/client/CcrRequestConverters.java b/client/rest-high-level/src/main/java/org/elasticsearch/client/CcrRequestConverters.java index a3f5d7e79fda7..8272e5d73bb4f 100644 --- a/client/rest-high-level/src/main/java/org/elasticsearch/client/CcrRequestConverters.java +++ b/client/rest-high-level/src/main/java/org/elasticsearch/client/CcrRequestConverters.java @@ -48,9 +48,10 @@ static Request putFollow(PutFollowRequest putFollowRequest) throws IOException { .addPathPartAsIs("_ccr", "follow") .build(); Request request = new Request(HttpPut.METHOD_NAME, endpoint); - RequestConverters.Params parameters = new RequestConverters.Params(request); + RequestConverters.Params parameters = new RequestConverters.Params(); parameters.withWaitForActiveShards(putFollowRequest.waitForActiveShards()); request.setEntity(createEntity(putFollowRequest, REQUEST_BODY_CONTENT_TYPE)); + request.addParameters(parameters.asMap()); return request; } diff --git a/client/rest-high-level/src/main/java/org/elasticsearch/client/ClusterRequestConverters.java b/client/rest-high-level/src/main/java/org/elasticsearch/client/ClusterRequestConverters.java index 4da8d128b98d1..a246402b505cc 100644 --- a/client/rest-high-level/src/main/java/org/elasticsearch/client/ClusterRequestConverters.java +++ b/client/rest-high-level/src/main/java/org/elasticsearch/client/ClusterRequestConverters.java @@ -36,22 +36,21 @@ private ClusterRequestConverters() {} static Request clusterPutSettings(ClusterUpdateSettingsRequest clusterUpdateSettingsRequest) throws IOException { Request request = new Request(HttpPut.METHOD_NAME, "/_cluster/settings"); - RequestConverters.Params parameters = new RequestConverters.Params(request); + RequestConverters.Params parameters = new RequestConverters.Params(); parameters.withTimeout(clusterUpdateSettingsRequest.timeout()); parameters.withMasterTimeout(clusterUpdateSettingsRequest.masterNodeTimeout()); - + request.addParameters(parameters.asMap()); request.setEntity(RequestConverters.createEntity(clusterUpdateSettingsRequest, RequestConverters.REQUEST_BODY_CONTENT_TYPE)); return request; } static Request clusterGetSettings(ClusterGetSettingsRequest clusterGetSettingsRequest) throws IOException { Request request = new Request(HttpGet.METHOD_NAME, "/_cluster/settings"); - - RequestConverters.Params parameters = new RequestConverters.Params(request); + RequestConverters.Params parameters = new RequestConverters.Params(); parameters.withLocal(clusterGetSettingsRequest.local()); parameters.withIncludeDefaults(clusterGetSettingsRequest.includeDefaults()); parameters.withMasterTimeout(clusterGetSettingsRequest.masterNodeTimeout()); - + request.addParameters(parameters.asMap()); return request; } @@ -63,7 +62,7 @@ static Request clusterHealth(ClusterHealthRequest healthRequest) { .build(); Request request = new Request(HttpGet.METHOD_NAME, endpoint); - new RequestConverters.Params(request) + RequestConverters.Params params = new RequestConverters.Params() .withWaitForStatus(healthRequest.waitForStatus()) .withWaitForNoRelocatingShards(healthRequest.waitForNoRelocatingShards()) .withWaitForNoInitializingShards(healthRequest.waitForNoInitializingShards()) @@ -74,6 +73,7 @@ static Request clusterHealth(ClusterHealthRequest healthRequest) { .withMasterTimeout(healthRequest.masterNodeTimeout()) .withLocal(healthRequest.local()) .withLevel(healthRequest.level()); + request.addParameters(params.asMap()); return request; } } diff --git a/client/rest-high-level/src/main/java/org/elasticsearch/client/DataFrameRequestConverters.java b/client/rest-high-level/src/main/java/org/elasticsearch/client/DataFrameRequestConverters.java index 0e887994da7bf..18dfc2305575d 100644 --- a/client/rest-high-level/src/main/java/org/elasticsearch/client/DataFrameRequestConverters.java +++ b/client/rest-high-level/src/main/java/org/elasticsearch/client/DataFrameRequestConverters.java @@ -37,6 +37,7 @@ import static org.elasticsearch.client.RequestConverters.REQUEST_BODY_CONTENT_TYPE; import static org.elasticsearch.client.RequestConverters.createEntity; +import static org.elasticsearch.client.dataframe.GetDataFrameTransformRequest.ALLOW_NO_MATCH; final class DataFrameRequestConverters { @@ -64,6 +65,9 @@ static Request getDataFrameTransform(GetDataFrameTransformRequest getRequest) { if (getRequest.getPageParams() != null && getRequest.getPageParams().getSize() != null) { request.addParameter(PageParams.SIZE.getPreferredName(), getRequest.getPageParams().getSize().toString()); } + if (getRequest.getAllowNoMatch() != null) { + request.addParameter(ALLOW_NO_MATCH, getRequest.getAllowNoMatch().toString()); + } return request; } @@ -82,28 +86,33 @@ static Request startDataFrameTransform(StartDataFrameTransformRequest startReque .addPathPartAsIs("_start") .build(); Request request = new Request(HttpPost.METHOD_NAME, endpoint); - RequestConverters.Params params = new RequestConverters.Params(request); + RequestConverters.Params params = new RequestConverters.Params(); if (startRequest.getTimeout() != null) { params.withTimeout(startRequest.getTimeout()); } + request.addParameters(params.asMap()); return request; } static Request stopDataFrameTransform(StopDataFrameTransformRequest stopRequest) { - String endpoint = new RequestConverters.EndpointBuilder() - .addPathPartAsIs("_data_frame", "transforms") - .addPathPart(stopRequest.getId()) - .addPathPartAsIs("_stop") - .build(); - Request request = new Request(HttpPost.METHOD_NAME, endpoint); - RequestConverters.Params params = new RequestConverters.Params(request); - if (stopRequest.getWaitForCompletion() != null) { - params.withWaitForCompletion(stopRequest.getWaitForCompletion()); - } - if (stopRequest.getTimeout() != null) { - params.withTimeout(stopRequest.getTimeout()); - } - return request; + String endpoint = new RequestConverters.EndpointBuilder() + .addPathPartAsIs("_data_frame", "transforms") + .addPathPart(stopRequest.getId()) + .addPathPartAsIs("_stop") + .build(); + Request request = new Request(HttpPost.METHOD_NAME, endpoint); + RequestConverters.Params params = new RequestConverters.Params(); + if (stopRequest.getWaitForCompletion() != null) { + params.withWaitForCompletion(stopRequest.getWaitForCompletion()); + } + if (stopRequest.getTimeout() != null) { + params.withTimeout(stopRequest.getTimeout()); + } + if (stopRequest.getAllowNoMatch() != null) { + request.addParameter(ALLOW_NO_MATCH, stopRequest.getAllowNoMatch().toString()); + } + request.addParameters(params.asMap()); + return request; } static Request previewDataFrameTransform(PreviewDataFrameTransformRequest previewRequest) throws IOException { @@ -128,6 +137,9 @@ static Request getDataFrameTransformStats(GetDataFrameTransformStatsRequest stat if (statsRequest.getPageParams() != null && statsRequest.getPageParams().getSize() != null) { request.addParameter(PageParams.SIZE.getPreferredName(), statsRequest.getPageParams().getSize().toString()); } + if (statsRequest.getAllowNoMatch() != null) { + request.addParameter(ALLOW_NO_MATCH, statsRequest.getAllowNoMatch().toString()); + } return request; } } diff --git a/client/rest-high-level/src/main/java/org/elasticsearch/client/IndexLifecycleRequestConverters.java b/client/rest-high-level/src/main/java/org/elasticsearch/client/IndexLifecycleRequestConverters.java index ff6d8a90cdb2d..f39f2b36cebc0 100644 --- a/client/rest-high-level/src/main/java/org/elasticsearch/client/IndexLifecycleRequestConverters.java +++ b/client/rest-high-level/src/main/java/org/elasticsearch/client/IndexLifecycleRequestConverters.java @@ -44,9 +44,10 @@ static Request getLifecyclePolicy(GetLifecyclePolicyRequest getLifecyclePolicyRe String endpoint = new RequestConverters.EndpointBuilder().addPathPartAsIs("_ilm/policy") .addCommaSeparatedPathParts(getLifecyclePolicyRequest.getPolicyNames()).build(); Request request = new Request(HttpGet.METHOD_NAME, endpoint); - RequestConverters.Params params = new RequestConverters.Params(request); + RequestConverters.Params params = new RequestConverters.Params(); params.withMasterTimeout(getLifecyclePolicyRequest.masterNodeTimeout()); params.withTimeout(getLifecyclePolicyRequest.timeout()); + request.addParameters(params.asMap()); return request; } @@ -56,9 +57,10 @@ static Request putLifecyclePolicy(PutLifecyclePolicyRequest putLifecycleRequest) .addPathPartAsIs(putLifecycleRequest.getName()) .build(); Request request = new Request(HttpPut.METHOD_NAME, endpoint); - RequestConverters.Params params = new RequestConverters.Params(request); + RequestConverters.Params params = new RequestConverters.Params(); params.withMasterTimeout(putLifecycleRequest.masterNodeTimeout()); params.withTimeout(putLifecycleRequest.timeout()); + request.addParameters(params.asMap()); request.setEntity(RequestConverters.createEntity(putLifecycleRequest, RequestConverters.REQUEST_BODY_CONTENT_TYPE)); return request; } @@ -69,9 +71,10 @@ static Request deleteLifecyclePolicy(DeleteLifecyclePolicyRequest deleteLifecycl .addPathPartAsIs("_ilm/policy") .addPathPartAsIs(deleteLifecyclePolicyRequest.getLifecyclePolicy()) .build()); - RequestConverters.Params params = new RequestConverters.Params(request); + RequestConverters.Params params = new RequestConverters.Params(); params.withMasterTimeout(deleteLifecyclePolicyRequest.masterNodeTimeout()); params.withTimeout(deleteLifecyclePolicyRequest.timeout()); + request.addParameters(params.asMap()); return request; } @@ -83,9 +86,10 @@ static Request removeIndexLifecyclePolicy(RemoveIndexLifecyclePolicyRequest remo .addCommaSeparatedPathParts(indices) .addPathPartAsIs("_ilm", "remove") .build()); - RequestConverters.Params params = new RequestConverters.Params(request); + RequestConverters.Params params = new RequestConverters.Params(); params.withIndicesOptions(removePolicyRequest.indicesOptions()); params.withMasterTimeout(removePolicyRequest.masterNodeTimeout()); + request.addParameters(params.asMap()); return request; } @@ -95,9 +99,10 @@ static Request startILM(StartILMRequest startILMRequest) { .addPathPartAsIs("_ilm") .addPathPartAsIs("start") .build()); - RequestConverters.Params params = new RequestConverters.Params(request); + RequestConverters.Params params = new RequestConverters.Params(); params.withMasterTimeout(startILMRequest.masterNodeTimeout()); params.withTimeout(startILMRequest.timeout()); + request.addParameters(params.asMap()); return request; } @@ -107,9 +112,10 @@ static Request stopILM(StopILMRequest stopILMRequest) { .addPathPartAsIs("_ilm") .addPathPartAsIs("stop") .build()); - RequestConverters.Params params = new RequestConverters.Params(request); + RequestConverters.Params params = new RequestConverters.Params(); params.withMasterTimeout(stopILMRequest.masterNodeTimeout()); params.withTimeout(stopILMRequest.timeout()); + request.addParameters(params.asMap()); return request; } @@ -119,9 +125,10 @@ static Request lifecycleManagementStatus(LifecycleManagementStatusRequest lifecy .addPathPartAsIs("_ilm") .addPathPartAsIs("status") .build()); - RequestConverters.Params params = new RequestConverters.Params(request); + RequestConverters.Params params = new RequestConverters.Params(); params.withMasterTimeout(lifecycleManagementStatusRequest.masterNodeTimeout()); params.withTimeout(lifecycleManagementStatusRequest.timeout()); + request.addParameters(params.asMap()); return request; } @@ -132,9 +139,10 @@ static Request explainLifecycle(ExplainLifecycleRequest explainLifecycleRequest) .addPathPartAsIs("_ilm") .addPathPartAsIs("explain") .build()); - RequestConverters.Params params = new RequestConverters.Params(request); + RequestConverters.Params params = new RequestConverters.Params(); params.withIndicesOptions(explainLifecycleRequest.indicesOptions()); params.withMasterTimeout(explainLifecycleRequest.masterNodeTimeout()); + request.addParameters(params.asMap()); return request; } @@ -145,9 +153,10 @@ static Request retryLifecycle(RetryLifecyclePolicyRequest retryLifecyclePolicyRe .addPathPartAsIs("_ilm") .addPathPartAsIs("retry") .build()); - RequestConverters.Params params = new RequestConverters.Params(request); + RequestConverters.Params params = new RequestConverters.Params(); params.withMasterTimeout(retryLifecyclePolicyRequest.masterNodeTimeout()); params.withTimeout(retryLifecyclePolicyRequest.timeout()); + request.addParameters(params.asMap()); return request; } } diff --git a/client/rest-high-level/src/main/java/org/elasticsearch/client/IndicesClient.java b/client/rest-high-level/src/main/java/org/elasticsearch/client/IndicesClient.java index cbb1d95feae1b..584bdad745026 100644 --- a/client/rest-high-level/src/main/java/org/elasticsearch/client/IndicesClient.java +++ b/client/rest-high-level/src/main/java/org/elasticsearch/client/IndicesClient.java @@ -22,8 +22,6 @@ import org.elasticsearch.action.ActionListener; import org.elasticsearch.action.admin.indices.alias.IndicesAliasesRequest; import org.elasticsearch.action.admin.indices.alias.get.GetAliasesRequest; -import org.elasticsearch.action.admin.indices.analyze.AnalyzeRequest; -import org.elasticsearch.action.admin.indices.analyze.AnalyzeResponse; import org.elasticsearch.action.admin.indices.cache.clear.ClearIndicesCacheRequest; import org.elasticsearch.action.admin.indices.cache.clear.ClearIndicesCacheResponse; import org.elasticsearch.action.admin.indices.close.CloseIndexRequest; @@ -47,6 +45,8 @@ import org.elasticsearch.action.admin.indices.validate.query.ValidateQueryResponse; import org.elasticsearch.action.support.master.AcknowledgedResponse; import org.elasticsearch.client.core.ShardsAcknowledgedResponse; +import org.elasticsearch.client.indices.AnalyzeRequest; +import org.elasticsearch.client.indices.AnalyzeResponse; import org.elasticsearch.client.indices.CreateIndexRequest; import org.elasticsearch.client.indices.CreateIndexResponse; import org.elasticsearch.client.indices.FreezeIndexRequest; @@ -61,6 +61,8 @@ import org.elasticsearch.client.indices.IndexTemplatesExistRequest; import org.elasticsearch.client.indices.PutIndexTemplateRequest; import org.elasticsearch.client.indices.PutMappingRequest; +import org.elasticsearch.client.indices.ReloadAnalyzersRequest; +import org.elasticsearch.client.indices.ReloadAnalyzersResponse; import org.elasticsearch.client.indices.UnfreezeIndexRequest; import org.elasticsearch.client.indices.rollover.RolloverRequest; import org.elasticsearch.client.indices.rollover.RolloverResponse; @@ -1328,4 +1330,28 @@ public void deleteTemplateAsync(DeleteIndexTemplateRequest request, RequestOptio restHighLevelClient.performRequestAsyncAndParseEntity(request, IndicesRequestConverters::deleteTemplate, options, AcknowledgedResponse::fromXContent, listener, emptySet()); } + + /** + * Synchronously calls the _reload_search_analyzers API + * + * @param request the request + * @param options the request options (e.g. headers), use {@link RequestOptions#DEFAULT} if nothing needs to be customized + */ + public ReloadAnalyzersResponse reloadAnalyzers(ReloadAnalyzersRequest request, RequestOptions options) throws IOException { + return restHighLevelClient.performRequestAndParseEntity(request, IndicesRequestConverters::reloadAnalyzers, options, + ReloadAnalyzersResponse::fromXContent, emptySet()); + } + + /** + * Asynchronously calls the _reload_search_analyzers API + * + * @param request the request + * @param options the request options (e.g. headers), use {@link RequestOptions#DEFAULT} if nothing needs to be customized + * @param listener the listener to be notified upon request completion + */ + public void reloadAnalyzersAsync(ReloadAnalyzersRequest request, RequestOptions options, + ActionListener listener) { + restHighLevelClient.performRequestAsyncAndParseEntity(request, IndicesRequestConverters::reloadAnalyzers, options, + ReloadAnalyzersResponse::fromXContent, listener, emptySet()); + } } diff --git a/client/rest-high-level/src/main/java/org/elasticsearch/client/IndicesRequestConverters.java b/client/rest-high-level/src/main/java/org/elasticsearch/client/IndicesRequestConverters.java index cc5adffd33483..7de53510b5080 100644 --- a/client/rest-high-level/src/main/java/org/elasticsearch/client/IndicesRequestConverters.java +++ b/client/rest-high-level/src/main/java/org/elasticsearch/client/IndicesRequestConverters.java @@ -26,7 +26,6 @@ import org.apache.http.client.methods.HttpPut; import org.elasticsearch.action.admin.indices.alias.IndicesAliasesRequest; import org.elasticsearch.action.admin.indices.alias.get.GetAliasesRequest; -import org.elasticsearch.action.admin.indices.analyze.AnalyzeRequest; import org.elasticsearch.action.admin.indices.cache.clear.ClearIndicesCacheRequest; import org.elasticsearch.action.admin.indices.close.CloseIndexRequest; import org.elasticsearch.action.admin.indices.delete.DeleteIndexRequest; @@ -41,6 +40,7 @@ import org.elasticsearch.action.admin.indices.shrink.ResizeType; import org.elasticsearch.action.admin.indices.template.delete.DeleteIndexTemplateRequest; import org.elasticsearch.action.admin.indices.validate.query.ValidateQueryRequest; +import org.elasticsearch.client.indices.AnalyzeRequest; import org.elasticsearch.client.indices.CreateIndexRequest; import org.elasticsearch.client.indices.FreezeIndexRequest; import org.elasticsearch.client.indices.GetFieldMappingsRequest; @@ -50,6 +50,7 @@ import org.elasticsearch.client.indices.IndexTemplatesExistRequest; import org.elasticsearch.client.indices.PutIndexTemplateRequest; import org.elasticsearch.client.indices.PutMappingRequest; +import org.elasticsearch.client.indices.ReloadAnalyzersRequest; import org.elasticsearch.client.indices.UnfreezeIndexRequest; import org.elasticsearch.client.indices.rollover.RolloverRequest; import org.elasticsearch.common.Strings; @@ -67,10 +68,11 @@ static Request deleteIndex(DeleteIndexRequest deleteIndexRequest) { String endpoint = RequestConverters.endpoint(deleteIndexRequest.indices()); Request request = new Request(HttpDelete.METHOD_NAME, endpoint); - RequestConverters.Params parameters = new RequestConverters.Params(request); + RequestConverters.Params parameters = new RequestConverters.Params(); parameters.withTimeout(deleteIndexRequest.timeout()); parameters.withMasterTimeout(deleteIndexRequest.masterNodeTimeout()); parameters.withIndicesOptions(deleteIndexRequest.indicesOptions()); + request.addParameters(parameters.asMap()); return request; } @@ -78,11 +80,12 @@ static Request openIndex(OpenIndexRequest openIndexRequest) { String endpoint = RequestConverters.endpoint(openIndexRequest.indices(), "_open"); Request request = new Request(HttpPost.METHOD_NAME, endpoint); - RequestConverters.Params parameters = new RequestConverters.Params(request); + RequestConverters.Params parameters = new RequestConverters.Params(); parameters.withTimeout(openIndexRequest.timeout()); parameters.withMasterTimeout(openIndexRequest.masterNodeTimeout()); parameters.withWaitForActiveShards(openIndexRequest.waitForActiveShards()); parameters.withIndicesOptions(openIndexRequest.indicesOptions()); + request.addParameters(parameters.asMap()); return request; } @@ -90,10 +93,11 @@ static Request closeIndex(CloseIndexRequest closeIndexRequest) { String endpoint = RequestConverters.endpoint(closeIndexRequest.indices(), "_close"); Request request = new Request(HttpPost.METHOD_NAME, endpoint); - RequestConverters.Params parameters = new RequestConverters.Params(request); + RequestConverters.Params parameters = new RequestConverters.Params(); parameters.withTimeout(closeIndexRequest.timeout()); parameters.withMasterTimeout(closeIndexRequest.masterNodeTimeout()); parameters.withIndicesOptions(closeIndexRequest.indicesOptions()); + request.addParameters(parameters.asMap()); return request; } @@ -102,11 +106,11 @@ static Request createIndex(CreateIndexRequest createIndexRequest) throws IOExcep .addPathPart(createIndexRequest.index()).build(); Request request = new Request(HttpPut.METHOD_NAME, endpoint); - RequestConverters.Params parameters = new RequestConverters.Params(request); + RequestConverters.Params parameters = new RequestConverters.Params(); parameters.withTimeout(createIndexRequest.timeout()); parameters.withMasterTimeout(createIndexRequest.masterNodeTimeout()); parameters.withWaitForActiveShards(createIndexRequest.waitForActiveShards()); - + request.addParameters(parameters.asMap()); request.setEntity(RequestConverters.createEntity(createIndexRequest, RequestConverters.REQUEST_BODY_CONTENT_TYPE)); return request; } @@ -116,12 +120,12 @@ static Request createIndex(org.elasticsearch.action.admin.indices.create.CreateI String endpoint = RequestConverters.endpoint(createIndexRequest.indices()); Request request = new Request(HttpPut.METHOD_NAME, endpoint); - RequestConverters.Params parameters = new RequestConverters.Params(request); + RequestConverters.Params parameters = new RequestConverters.Params(); parameters.withTimeout(createIndexRequest.timeout()); parameters.withMasterTimeout(createIndexRequest.masterNodeTimeout()); parameters.withWaitForActiveShards(createIndexRequest.waitForActiveShards()); parameters.putParam(INCLUDE_TYPE_NAME_PARAMETER, "true"); - + request.addParameters(parameters.asMap()); request.setEntity(RequestConverters.createEntity(createIndexRequest, RequestConverters.REQUEST_BODY_CONTENT_TYPE)); return request; } @@ -129,10 +133,10 @@ static Request createIndex(org.elasticsearch.action.admin.indices.create.CreateI static Request updateAliases(IndicesAliasesRequest indicesAliasesRequest) throws IOException { Request request = new Request(HttpPost.METHOD_NAME, "/_aliases"); - RequestConverters.Params parameters = new RequestConverters.Params(request); + RequestConverters.Params parameters = new RequestConverters.Params(); parameters.withTimeout(indicesAliasesRequest.timeout()); parameters.withMasterTimeout(indicesAliasesRequest.masterNodeTimeout()); - + request.addParameters(parameters.asMap()); request.setEntity(RequestConverters.createEntity(indicesAliasesRequest, RequestConverters.REQUEST_BODY_CONTENT_TYPE)); return request; } @@ -141,9 +145,10 @@ static Request updateAliases(IndicesAliasesRequest indicesAliasesRequest) throws static Request putMapping(PutMappingRequest putMappingRequest) throws IOException { Request request = new Request(HttpPut.METHOD_NAME, RequestConverters.endpoint(putMappingRequest.indices(), "_mapping")); - RequestConverters.Params parameters = new RequestConverters.Params(request); + RequestConverters.Params parameters = new RequestConverters.Params(); parameters.withTimeout(putMappingRequest.timeout()); parameters.withMasterTimeout(putMappingRequest.masterNodeTimeout()); + request.addParameters(parameters.asMap()); request.setEntity(RequestConverters.createEntity(putMappingRequest, RequestConverters.REQUEST_BODY_CONTENT_TYPE)); return request; } @@ -162,11 +167,11 @@ static Request putMapping(org.elasticsearch.action.admin.indices.mapping.put.Put Request request = new Request(HttpPut.METHOD_NAME, RequestConverters.endpoint(putMappingRequest.indices(), "_mapping", putMappingRequest.type())); - RequestConverters.Params parameters = new RequestConverters.Params(request); + RequestConverters.Params parameters = new RequestConverters.Params(); parameters.withTimeout(putMappingRequest.timeout()); parameters.withMasterTimeout(putMappingRequest.masterNodeTimeout()); parameters.putParam(INCLUDE_TYPE_NAME_PARAMETER, "true"); - + request.addParameters(parameters.asMap()); request.setEntity(RequestConverters.createEntity(putMappingRequest, RequestConverters.REQUEST_BODY_CONTENT_TYPE)); return request; } @@ -176,11 +181,11 @@ static Request getMappings(GetMappingsRequest getMappingsRequest) { Request request = new Request(HttpGet.METHOD_NAME, RequestConverters.endpoint(indices, "_mapping")); - RequestConverters.Params parameters = new RequestConverters.Params(request); + RequestConverters.Params parameters = new RequestConverters.Params(); parameters.withMasterTimeout(getMappingsRequest.masterNodeTimeout()); parameters.withIndicesOptions(getMappingsRequest.indicesOptions()); parameters.withLocal(getMappingsRequest.local()); - + request.addParameters(parameters.asMap()); return request; } @@ -191,12 +196,12 @@ static Request getMappings(org.elasticsearch.action.admin.indices.mapping.get.Ge Request request = new Request(HttpGet.METHOD_NAME, RequestConverters.endpoint(indices, "_mapping", types)); - RequestConverters.Params parameters = new RequestConverters.Params(request); + RequestConverters.Params parameters = new RequestConverters.Params(); parameters.withMasterTimeout(getMappingsRequest.masterNodeTimeout()); parameters.withIndicesOptions(getMappingsRequest.indicesOptions()); parameters.withLocal(getMappingsRequest.local()); parameters.putParam(INCLUDE_TYPE_NAME_PARAMETER, "true"); - + request.addParameters(parameters.asMap()); return request; } @@ -213,11 +218,11 @@ static Request getFieldMapping(GetFieldMappingsRequest getFieldMappingsRequest) Request request = new Request(HttpGet.METHOD_NAME, endpoint); - RequestConverters.Params parameters = new RequestConverters.Params(request); + RequestConverters.Params parameters = new RequestConverters.Params(); parameters.withIndicesOptions(getFieldMappingsRequest.indicesOptions()); parameters.withIncludeDefaults(getFieldMappingsRequest.includeDefaults()); parameters.withLocal(getFieldMappingsRequest.local()); - + request.addParameters(parameters.asMap()); return request; } @@ -234,12 +239,12 @@ static Request getFieldMapping(org.elasticsearch.action.admin.indices.mapping.ge Request request = new Request(HttpGet.METHOD_NAME, endpoint); - RequestConverters.Params parameters = new RequestConverters.Params(request); + RequestConverters.Params parameters = new RequestConverters.Params(); parameters.withIndicesOptions(getFieldMappingsRequest.indicesOptions()); parameters.withIncludeDefaults(getFieldMappingsRequest.includeDefaults()); parameters.withLocal(getFieldMappingsRequest.local()); parameters.putParam(INCLUDE_TYPE_NAME_PARAMETER, "true"); - + request.addParameters(parameters.asMap()); return request; } @@ -247,8 +252,9 @@ static Request refresh(RefreshRequest refreshRequest) { String[] indices = refreshRequest.indices() == null ? Strings.EMPTY_ARRAY : refreshRequest.indices(); Request request = new Request(HttpPost.METHOD_NAME, RequestConverters.endpoint(indices, "_refresh")); - RequestConverters.Params parameters = new RequestConverters.Params(request); + RequestConverters.Params parameters = new RequestConverters.Params(); parameters.withIndicesOptions(refreshRequest.indicesOptions()); + request.addParameters(parameters.asMap()); return request; } @@ -256,18 +262,20 @@ static Request flush(FlushRequest flushRequest) { String[] indices = flushRequest.indices() == null ? Strings.EMPTY_ARRAY : flushRequest.indices(); Request request = new Request(HttpPost.METHOD_NAME, RequestConverters.endpoint(indices, "_flush")); - RequestConverters.Params parameters = new RequestConverters.Params(request); + RequestConverters.Params parameters = new RequestConverters.Params(); parameters.withIndicesOptions(flushRequest.indicesOptions()); parameters.putParam("wait_if_ongoing", Boolean.toString(flushRequest.waitIfOngoing())); parameters.putParam("force", Boolean.toString(flushRequest.force())); + request.addParameters(parameters.asMap()); return request; } static Request flushSynced(SyncedFlushRequest syncedFlushRequest) { String[] indices = syncedFlushRequest.indices() == null ? Strings.EMPTY_ARRAY : syncedFlushRequest.indices(); Request request = new Request(HttpPost.METHOD_NAME, RequestConverters.endpoint(indices, "_flush/synced")); - RequestConverters.Params parameters = new RequestConverters.Params(request); + RequestConverters.Params parameters = new RequestConverters.Params(); parameters.withIndicesOptions(syncedFlushRequest.indicesOptions()); + request.addParameters(parameters.asMap()); return request; } @@ -275,11 +283,12 @@ static Request forceMerge(ForceMergeRequest forceMergeRequest) { String[] indices = forceMergeRequest.indices() == null ? Strings.EMPTY_ARRAY : forceMergeRequest.indices(); Request request = new Request(HttpPost.METHOD_NAME, RequestConverters.endpoint(indices, "_forcemerge")); - RequestConverters.Params parameters = new RequestConverters.Params(request); + RequestConverters.Params parameters = new RequestConverters.Params(); parameters.withIndicesOptions(forceMergeRequest.indicesOptions()); parameters.putParam("max_num_segments", Integer.toString(forceMergeRequest.maxNumSegments())); parameters.putParam("only_expunge_deletes", Boolean.toString(forceMergeRequest.onlyExpungeDeletes())); parameters.putParam("flush", Boolean.toString(forceMergeRequest.flush())); + request.addParameters(parameters.asMap()); return request; } @@ -287,12 +296,13 @@ static Request clearCache(ClearIndicesCacheRequest clearIndicesCacheRequest) { String[] indices = clearIndicesCacheRequest.indices() == null ? Strings.EMPTY_ARRAY :clearIndicesCacheRequest.indices(); Request request = new Request(HttpPost.METHOD_NAME, RequestConverters.endpoint(indices, "_cache/clear")); - RequestConverters.Params parameters = new RequestConverters.Params(request); + RequestConverters.Params parameters = new RequestConverters.Params(); parameters.withIndicesOptions(clearIndicesCacheRequest.indicesOptions()); parameters.putParam("query", Boolean.toString(clearIndicesCacheRequest.queryCache())); parameters.putParam("fielddata", Boolean.toString(clearIndicesCacheRequest.fieldDataCache())); parameters.putParam("request", Boolean.toString(clearIndicesCacheRequest.requestCache())); parameters.putParam("fields", String.join(",", clearIndicesCacheRequest.fields())); + request.addParameters(parameters.asMap()); return request; } @@ -306,9 +316,10 @@ static Request existsAlias(GetAliasesRequest getAliasesRequest) { Request request = new Request(HttpHead.METHOD_NAME, RequestConverters.endpoint(indices, "_alias", aliases)); - RequestConverters.Params params = new RequestConverters.Params(request); + RequestConverters.Params params = new RequestConverters.Params(); params.withIndicesOptions(getAliasesRequest.indicesOptions()); params.withLocal(getAliasesRequest.local()); + request.addParameters(params.asMap()); return request; } @@ -332,11 +343,11 @@ private static Request resize(ResizeRequest resizeRequest) throws IOException { .addPathPart(resizeRequest.getTargetIndexRequest().index()).build(); Request request = new Request(HttpPut.METHOD_NAME, endpoint); - RequestConverters.Params params = new RequestConverters.Params(request); + RequestConverters.Params params = new RequestConverters.Params(); params.withTimeout(resizeRequest.timeout()); params.withMasterTimeout(resizeRequest.masterNodeTimeout()); params.withWaitForActiveShards(resizeRequest.getTargetIndexRequest().waitForActiveShards()); - + request.addParameters(params.asMap()); request.setEntity(RequestConverters.createEntity(resizeRequest, RequestConverters.REQUEST_BODY_CONTENT_TYPE)); return request; } @@ -346,14 +357,14 @@ static Request rollover(RolloverRequest rolloverRequest) throws IOException { .addPathPart(rolloverRequest.getNewIndexName()).build(); Request request = new Request(HttpPost.METHOD_NAME, endpoint); - RequestConverters.Params params = new RequestConverters.Params(request); + RequestConverters.Params params = new RequestConverters.Params(); params.withTimeout(rolloverRequest.timeout()); params.withMasterTimeout(rolloverRequest.masterNodeTimeout()); params.withWaitForActiveShards(rolloverRequest.getCreateIndexRequest().waitForActiveShards()); if (rolloverRequest.isDryRun()) { params.putParam("dry_run", Boolean.TRUE.toString()); } - + request.addParameters(params.asMap()); request.setEntity(RequestConverters.createEntity(rolloverRequest, RequestConverters.REQUEST_BODY_CONTENT_TYPE)); return request; } @@ -364,7 +375,7 @@ static Request rollover(org.elasticsearch.action.admin.indices.rollover.Rollover .addPathPart(rolloverRequest.getNewIndexName()).build(); Request request = new Request(HttpPost.METHOD_NAME, endpoint); - RequestConverters.Params params = new RequestConverters.Params(request); + RequestConverters.Params params = new RequestConverters.Params(); params.withTimeout(rolloverRequest.timeout()); params.withMasterTimeout(rolloverRequest.masterNodeTimeout()); params.withWaitForActiveShards(rolloverRequest.getCreateIndexRequest().waitForActiveShards()); @@ -373,7 +384,7 @@ static Request rollover(org.elasticsearch.action.admin.indices.rollover.Rollover } params.putParam(INCLUDE_TYPE_NAME_PARAMETER, "true"); request.setEntity(RequestConverters.createEntity(rolloverRequest, RequestConverters.REQUEST_BODY_CONTENT_TYPE)); - + request.addParameters(params.asMap()); return request; } @@ -384,12 +395,12 @@ static Request getSettings(GetSettingsRequest getSettingsRequest) { String endpoint = RequestConverters.endpoint(indices, "_settings", names); Request request = new Request(HttpGet.METHOD_NAME, endpoint); - RequestConverters.Params params = new RequestConverters.Params(request); + RequestConverters.Params params = new RequestConverters.Params(); params.withIndicesOptions(getSettingsRequest.indicesOptions()); params.withLocal(getSettingsRequest.local()); params.withIncludeDefaults(getSettingsRequest.includeDefaults()); params.withMasterTimeout(getSettingsRequest.masterNodeTimeout()); - + request.addParameters(params.asMap()); return request; } @@ -404,14 +415,14 @@ static Request getIndex(org.elasticsearch.action.admin.indices.get.GetIndexReque String endpoint = RequestConverters.endpoint(indices); Request request = new Request(HttpGet.METHOD_NAME, endpoint); - RequestConverters.Params params = new RequestConverters.Params(request); + RequestConverters.Params params = new RequestConverters.Params(); params.withIndicesOptions(getIndexRequest.indicesOptions()); params.withLocal(getIndexRequest.local()); params.withIncludeDefaults(getIndexRequest.includeDefaults()); params.withHuman(getIndexRequest.humanReadable()); params.withMasterTimeout(getIndexRequest.masterNodeTimeout()); params.putParam(INCLUDE_TYPE_NAME_PARAMETER, "true"); - + request.addParameters(params.asMap()); return request; } @@ -421,13 +432,13 @@ static Request getIndex(GetIndexRequest getIndexRequest) { String endpoint = RequestConverters.endpoint(indices); Request request = new Request(HttpGet.METHOD_NAME, endpoint); - RequestConverters.Params params = new RequestConverters.Params(request); + RequestConverters.Params params = new RequestConverters.Params(); params.withIndicesOptions(getIndexRequest.indicesOptions()); params.withLocal(getIndexRequest.local()); params.withIncludeDefaults(getIndexRequest.includeDefaults()); params.withHuman(getIndexRequest.humanReadable()); params.withMasterTimeout(getIndexRequest.masterNodeTimeout()); - + request.addParameters(params.asMap()); return request; } @@ -437,35 +448,35 @@ static Request getIndex(GetIndexRequest getIndexRequest) { */ @Deprecated static Request indicesExist(org.elasticsearch.action.admin.indices.get.GetIndexRequest getIndexRequest) { - // this can be called with no indices as argument by transport client, not via REST though if (getIndexRequest.indices() == null || getIndexRequest.indices().length == 0) { throw new IllegalArgumentException("indices are mandatory"); } String endpoint = RequestConverters.endpoint(getIndexRequest.indices(), ""); Request request = new Request(HttpHead.METHOD_NAME, endpoint); - RequestConverters.Params params = new RequestConverters.Params(request); + RequestConverters.Params params = new RequestConverters.Params(); params.withLocal(getIndexRequest.local()); params.withHuman(getIndexRequest.humanReadable()); params.withIndicesOptions(getIndexRequest.indicesOptions()); params.withIncludeDefaults(getIndexRequest.includeDefaults()); params.putParam(INCLUDE_TYPE_NAME_PARAMETER, "true"); + request.addParameters(params.asMap()); return request; } static Request indicesExist(GetIndexRequest getIndexRequest) { - // this can be called with no indices as argument by transport client, not via REST though if (getIndexRequest.indices() == null || getIndexRequest.indices().length == 0) { throw new IllegalArgumentException("indices are mandatory"); } String endpoint = RequestConverters.endpoint(getIndexRequest.indices(), ""); Request request = new Request(HttpHead.METHOD_NAME, endpoint); - RequestConverters.Params params = new RequestConverters.Params(request); + RequestConverters.Params params = new RequestConverters.Params(); params.withLocal(getIndexRequest.local()); params.withHuman(getIndexRequest.humanReadable()); params.withIndicesOptions(getIndexRequest.indicesOptions()); params.withIncludeDefaults(getIndexRequest.includeDefaults()); + request.addParameters(params.asMap()); return request; } @@ -473,12 +484,12 @@ static Request indexPutSettings(UpdateSettingsRequest updateSettingsRequest) thr String[] indices = updateSettingsRequest.indices() == null ? Strings.EMPTY_ARRAY : updateSettingsRequest.indices(); Request request = new Request(HttpPut.METHOD_NAME, RequestConverters.endpoint(indices, "_settings")); - RequestConverters.Params parameters = new RequestConverters.Params(request); + RequestConverters.Params parameters = new RequestConverters.Params(); parameters.withTimeout(updateSettingsRequest.timeout()); parameters.withMasterTimeout(updateSettingsRequest.masterNodeTimeout()); parameters.withIndicesOptions(updateSettingsRequest.indicesOptions()); parameters.withPreserveExisting(updateSettingsRequest.isPreserveExisting()); - + request.addParameters(parameters.asMap()); request.setEntity(RequestConverters.createEntity(updateSettingsRequest, RequestConverters.REQUEST_BODY_CONTENT_TYPE)); return request; } @@ -493,7 +504,7 @@ static Request putTemplate(org.elasticsearch.action.admin.indices.template.put.P String endpoint = new RequestConverters.EndpointBuilder().addPathPartAsIs("_template") .addPathPart(putIndexTemplateRequest.name()).build(); Request request = new Request(HttpPut.METHOD_NAME, endpoint); - RequestConverters.Params params = new RequestConverters.Params(request); + RequestConverters.Params params = new RequestConverters.Params(); params.withMasterTimeout(putIndexTemplateRequest.masterNodeTimeout()); if (putIndexTemplateRequest.create()) { params.putParam("create", Boolean.TRUE.toString()); @@ -502,6 +513,7 @@ static Request putTemplate(org.elasticsearch.action.admin.indices.template.put.P params.putParam("cause", putIndexTemplateRequest.cause()); } params.putParam(INCLUDE_TYPE_NAME_PARAMETER, "true"); + request.addParameters(params.asMap()); request.setEntity(RequestConverters.createEntity(putIndexTemplateRequest, RequestConverters.REQUEST_BODY_CONTENT_TYPE)); return request; } @@ -510,7 +522,7 @@ static Request putTemplate(PutIndexTemplateRequest putIndexTemplateRequest) thro String endpoint = new RequestConverters.EndpointBuilder().addPathPartAsIs("_template") .addPathPart(putIndexTemplateRequest.name()).build(); Request request = new Request(HttpPut.METHOD_NAME, endpoint); - RequestConverters.Params params = new RequestConverters.Params(request); + RequestConverters.Params params = new RequestConverters.Params(); params.withMasterTimeout(putIndexTemplateRequest.masterNodeTimeout()); if (putIndexTemplateRequest.create()) { params.putParam("create", Boolean.TRUE.toString()); @@ -518,6 +530,7 @@ static Request putTemplate(PutIndexTemplateRequest putIndexTemplateRequest) thro if (Strings.hasText(putIndexTemplateRequest.cause())) { params.putParam("cause", putIndexTemplateRequest.cause()); } + request.addParameters(params.asMap()); request.setEntity(RequestConverters.createEntity(putIndexTemplateRequest, RequestConverters.REQUEST_BODY_CONTENT_TYPE)); return request; } @@ -527,11 +540,12 @@ static Request validateQuery(ValidateQueryRequest validateQueryRequest) throws I String[] types = validateQueryRequest.types() == null || indices.length <= 0 ? Strings.EMPTY_ARRAY : validateQueryRequest.types(); String endpoint = RequestConverters.endpoint(indices, types, "_validate/query"); Request request = new Request(HttpGet.METHOD_NAME, endpoint); - RequestConverters.Params params = new RequestConverters.Params(request); + RequestConverters.Params params = new RequestConverters.Params(); params.withIndicesOptions(validateQueryRequest.indicesOptions()); params.putParam("explain", Boolean.toString(validateQueryRequest.explain())); params.putParam("all_shards", Boolean.toString(validateQueryRequest.allShards())); params.putParam("rewrite", Boolean.toString(validateQueryRequest.rewrite())); + request.addParameters(params.asMap()); request.setEntity(RequestConverters.createEntity(validateQueryRequest, RequestConverters.REQUEST_BODY_CONTENT_TYPE)); return request; } @@ -541,9 +555,10 @@ static Request getAlias(GetAliasesRequest getAliasesRequest) { String[] aliases = getAliasesRequest.aliases() == null ? Strings.EMPTY_ARRAY : getAliasesRequest.aliases(); String endpoint = RequestConverters.endpoint(indices, "_alias", aliases); Request request = new Request(HttpGet.METHOD_NAME, endpoint); - RequestConverters.Params params = new RequestConverters.Params(request); + RequestConverters.Params params = new RequestConverters.Params(); params.withIndicesOptions(getAliasesRequest.indicesOptions()); params.withLocal(getAliasesRequest.local()); + request.addParameters(params.asMap()); return request; } @@ -562,12 +577,13 @@ private static Request getTemplates(GetIndexTemplatesRequest getIndexTemplatesRe .addCommaSeparatedPathParts(getIndexTemplatesRequest.names()) .build(); final Request request = new Request(HttpGet.METHOD_NAME, endpoint); - final RequestConverters.Params params = new RequestConverters.Params(request); + final RequestConverters.Params params = new RequestConverters.Params(); params.withLocal(getIndexTemplatesRequest.isLocal()); params.withMasterTimeout(getIndexTemplatesRequest.getMasterNodeTimeout()); if (includeTypeName) { params.putParam(INCLUDE_TYPE_NAME_PARAMETER, "true"); } + request.addParameters(params.asMap()); return request; } @@ -577,9 +593,10 @@ static Request templatesExist(IndexTemplatesExistRequest indexTemplatesExistRequ .addCommaSeparatedPathParts(indexTemplatesExistRequest.names()) .build(); final Request request = new Request(HttpHead.METHOD_NAME, endpoint); - final RequestConverters.Params params = new RequestConverters.Params(request); + final RequestConverters.Params params = new RequestConverters.Params(); params.withLocal(indexTemplatesExistRequest.isLocal()); params.withMasterTimeout(indexTemplatesExistRequest.getMasterNodeTimeout()); + request.addParameters(params.asMap()); return request; } @@ -598,22 +615,24 @@ static Request analyze(AnalyzeRequest request) throws IOException { static Request freezeIndex(FreezeIndexRequest freezeIndexRequest) { String endpoint = RequestConverters.endpoint(freezeIndexRequest.getIndices(), "_freeze"); Request request = new Request(HttpPost.METHOD_NAME, endpoint); - RequestConverters.Params parameters = new RequestConverters.Params(request); + RequestConverters.Params parameters = new RequestConverters.Params(); parameters.withTimeout(freezeIndexRequest.timeout()); parameters.withMasterTimeout(freezeIndexRequest.masterNodeTimeout()); parameters.withIndicesOptions(freezeIndexRequest.indicesOptions()); parameters.withWaitForActiveShards(freezeIndexRequest.getWaitForActiveShards()); + request.addParameters(parameters.asMap()); return request; } static Request unfreezeIndex(UnfreezeIndexRequest unfreezeIndexRequest) { String endpoint = RequestConverters.endpoint(unfreezeIndexRequest.getIndices(), "_unfreeze"); Request request = new Request(HttpPost.METHOD_NAME, endpoint); - RequestConverters.Params parameters = new RequestConverters.Params(request); + RequestConverters.Params parameters = new RequestConverters.Params(); parameters.withTimeout(unfreezeIndexRequest.timeout()); parameters.withMasterTimeout(unfreezeIndexRequest.masterNodeTimeout()); parameters.withIndicesOptions(unfreezeIndexRequest.indicesOptions()); parameters.withWaitForActiveShards(unfreezeIndexRequest.getWaitForActiveShards()); + request.addParameters(parameters.asMap()); return request; } @@ -621,8 +640,18 @@ static Request deleteTemplate(DeleteIndexTemplateRequest deleteIndexTemplateRequ String name = deleteIndexTemplateRequest.name(); String endpoint = new RequestConverters.EndpointBuilder().addPathPartAsIs("_template").addPathPart(name).build(); Request request = new Request(HttpDelete.METHOD_NAME, endpoint); - RequestConverters.Params params = new RequestConverters.Params(request); + RequestConverters.Params params = new RequestConverters.Params(); params.withMasterTimeout(deleteIndexTemplateRequest.masterNodeTimeout()); + request.addParameters(params.asMap()); + return request; + } + + static Request reloadAnalyzers(ReloadAnalyzersRequest reloadAnalyzersRequest) { + String endpoint = RequestConverters.endpoint(reloadAnalyzersRequest.getIndices(), "_reload_search_analyzers"); + Request request = new Request(HttpPost.METHOD_NAME, endpoint); + RequestConverters.Params parameters = new RequestConverters.Params(); + parameters.withIndicesOptions(reloadAnalyzersRequest.indicesOptions()); + request.addParameters(parameters.asMap()); return request; } } diff --git a/client/rest-high-level/src/main/java/org/elasticsearch/client/IngestRequestConverters.java b/client/rest-high-level/src/main/java/org/elasticsearch/client/IngestRequestConverters.java index 06b4c0fd62ac0..c2ffc4ee5da4d 100644 --- a/client/rest-high-level/src/main/java/org/elasticsearch/client/IngestRequestConverters.java +++ b/client/rest-high-level/src/main/java/org/elasticsearch/client/IngestRequestConverters.java @@ -41,8 +41,9 @@ static Request getPipeline(GetPipelineRequest getPipelineRequest) { .build(); Request request = new Request(HttpGet.METHOD_NAME, endpoint); - RequestConverters.Params parameters = new RequestConverters.Params(request); + RequestConverters.Params parameters = new RequestConverters.Params(); parameters.withMasterTimeout(getPipelineRequest.masterNodeTimeout()); + request.addParameters(parameters.asMap()); return request; } @@ -53,10 +54,10 @@ static Request putPipeline(PutPipelineRequest putPipelineRequest) throws IOExcep .build(); Request request = new Request(HttpPut.METHOD_NAME, endpoint); - RequestConverters.Params parameters = new RequestConverters.Params(request); + RequestConverters.Params parameters = new RequestConverters.Params(); parameters.withTimeout(putPipelineRequest.timeout()); parameters.withMasterTimeout(putPipelineRequest.masterNodeTimeout()); - + request.addParameters(parameters.asMap()); request.setEntity(RequestConverters.createEntity(putPipelineRequest, RequestConverters.REQUEST_BODY_CONTENT_TYPE)); return request; } @@ -68,10 +69,10 @@ static Request deletePipeline(DeletePipelineRequest deletePipelineRequest) { .build(); Request request = new Request(HttpDelete.METHOD_NAME, endpoint); - RequestConverters.Params parameters = new RequestConverters.Params(request); + RequestConverters.Params parameters = new RequestConverters.Params(); parameters.withTimeout(deletePipelineRequest.timeout()); parameters.withMasterTimeout(deletePipelineRequest.masterNodeTimeout()); - + request.addParameters(parameters.asMap()); return request; } @@ -83,8 +84,9 @@ static Request simulatePipeline(SimulatePipelineRequest simulatePipelineRequest) builder.addPathPartAsIs("_simulate"); String endpoint = builder.build(); Request request = new Request(HttpPost.METHOD_NAME, endpoint); - RequestConverters.Params params = new RequestConverters.Params(request); + RequestConverters.Params params = new RequestConverters.Params(); params.putParam("verbose", Boolean.toString(simulatePipelineRequest.isVerbose())); + request.addParameters(params.asMap()); request.setEntity(RequestConverters.createEntity(simulatePipelineRequest, RequestConverters.REQUEST_BODY_CONTENT_TYPE)); return request; } diff --git a/client/rest-high-level/src/main/java/org/elasticsearch/client/LicenseRequestConverters.java b/client/rest-high-level/src/main/java/org/elasticsearch/client/LicenseRequestConverters.java index 73ecce0f0467c..ec5ae8e3a6073 100644 --- a/client/rest-high-level/src/main/java/org/elasticsearch/client/LicenseRequestConverters.java +++ b/client/rest-high-level/src/main/java/org/elasticsearch/client/LicenseRequestConverters.java @@ -36,12 +36,13 @@ private LicenseRequestConverters() {} static Request putLicense(PutLicenseRequest putLicenseRequest) { String endpoint = new RequestConverters.EndpointBuilder().addPathPartAsIs("_license").build(); Request request = new Request(HttpPut.METHOD_NAME, endpoint); - RequestConverters.Params parameters = new RequestConverters.Params(request); + RequestConverters.Params parameters = new RequestConverters.Params(); parameters.withTimeout(putLicenseRequest.timeout()); parameters.withMasterTimeout(putLicenseRequest.masterNodeTimeout()); if (putLicenseRequest.isAcknowledge()) { parameters.putParam("acknowledge", "true"); } + request.addParameters(parameters.asMap()); request.setJsonEntity(putLicenseRequest.getLicenseDefinition()); return request; } @@ -49,17 +50,19 @@ static Request putLicense(PutLicenseRequest putLicenseRequest) { static Request getLicense(GetLicenseRequest getLicenseRequest) { String endpoint = new RequestConverters.EndpointBuilder().addPathPartAsIs("_license").build(); Request request = new Request(HttpGet.METHOD_NAME, endpoint); - RequestConverters.Params parameters = new RequestConverters.Params(request); + RequestConverters.Params parameters = new RequestConverters.Params(); parameters.withLocal(getLicenseRequest.isLocal()); + request.addParameters(parameters.asMap()); return request; } static Request deleteLicense(DeleteLicenseRequest deleteLicenseRequest) { String endpoint = new RequestConverters.EndpointBuilder().addPathPartAsIs("_license").build(); Request request = new Request(HttpDelete.METHOD_NAME, endpoint); - RequestConverters.Params parameters = new RequestConverters.Params(request); + RequestConverters.Params parameters = new RequestConverters.Params(); parameters.withTimeout(deleteLicenseRequest.timeout()); parameters.withMasterTimeout(deleteLicenseRequest.masterNodeTimeout()); + request.addParameters(parameters.asMap()); return request; } @@ -67,11 +70,12 @@ static Request startTrial(StartTrialRequest startTrialRequest) { final String endpoint = new RequestConverters.EndpointBuilder().addPathPartAsIs("_license", "start_trial").build(); final Request request = new Request(HttpPost.METHOD_NAME, endpoint); - RequestConverters.Params parameters = new RequestConverters.Params(request); + RequestConverters.Params parameters = new RequestConverters.Params(); parameters.putParam("acknowledge", Boolean.toString(startTrialRequest.isAcknowledge())); if (startTrialRequest.getLicenseType() != null) { parameters.putParam("type", startTrialRequest.getLicenseType()); } + request.addParameters(parameters.asMap()); return request; } @@ -80,12 +84,13 @@ static Request startBasic(StartBasicRequest startBasicRequest) { .addPathPartAsIs("_license", "start_basic") .build(); Request request = new Request(HttpPost.METHOD_NAME, endpoint); - RequestConverters.Params parameters = new RequestConverters.Params(request); + RequestConverters.Params parameters = new RequestConverters.Params(); parameters.withTimeout(startBasicRequest.timeout()); parameters.withMasterTimeout(startBasicRequest.masterNodeTimeout()); if (startBasicRequest.isAcknowledge()) { parameters.putParam("acknowledge", "true"); } + request.addParameters(parameters.asMap()); return request; } diff --git a/client/rest-high-level/src/main/java/org/elasticsearch/client/MLRequestConverters.java b/client/rest-high-level/src/main/java/org/elasticsearch/client/MLRequestConverters.java index 99884ee49c868..651851e345df9 100644 --- a/client/rest-high-level/src/main/java/org/elasticsearch/client/MLRequestConverters.java +++ b/client/rest-high-level/src/main/java/org/elasticsearch/client/MLRequestConverters.java @@ -32,12 +32,14 @@ import org.elasticsearch.client.ml.DeleteCalendarEventRequest; import org.elasticsearch.client.ml.DeleteCalendarJobRequest; import org.elasticsearch.client.ml.DeleteCalendarRequest; +import org.elasticsearch.client.ml.DeleteDataFrameAnalyticsRequest; import org.elasticsearch.client.ml.DeleteDatafeedRequest; import org.elasticsearch.client.ml.DeleteExpiredDataRequest; import org.elasticsearch.client.ml.DeleteFilterRequest; import org.elasticsearch.client.ml.DeleteForecastRequest; import org.elasticsearch.client.ml.DeleteJobRequest; import org.elasticsearch.client.ml.DeleteModelSnapshotRequest; +import org.elasticsearch.client.ml.EvaluateDataFrameRequest; import org.elasticsearch.client.ml.FindFileStructureRequest; import org.elasticsearch.client.ml.FlushJobRequest; import org.elasticsearch.client.ml.ForecastJobRequest; @@ -45,6 +47,8 @@ import org.elasticsearch.client.ml.GetCalendarEventsRequest; import org.elasticsearch.client.ml.GetCalendarsRequest; import org.elasticsearch.client.ml.GetCategoriesRequest; +import org.elasticsearch.client.ml.GetDataFrameAnalyticsRequest; +import org.elasticsearch.client.ml.GetDataFrameAnalyticsStatsRequest; import org.elasticsearch.client.ml.GetDatafeedRequest; import org.elasticsearch.client.ml.GetDatafeedStatsRequest; import org.elasticsearch.client.ml.GetFiltersRequest; @@ -61,12 +65,15 @@ import org.elasticsearch.client.ml.PreviewDatafeedRequest; import org.elasticsearch.client.ml.PutCalendarJobRequest; import org.elasticsearch.client.ml.PutCalendarRequest; +import org.elasticsearch.client.ml.PutDataFrameAnalyticsRequest; import org.elasticsearch.client.ml.PutDatafeedRequest; import org.elasticsearch.client.ml.PutFilterRequest; import org.elasticsearch.client.ml.PutJobRequest; import org.elasticsearch.client.ml.RevertModelSnapshotRequest; import org.elasticsearch.client.ml.SetUpgradeModeRequest; +import org.elasticsearch.client.ml.StartDataFrameAnalyticsRequest; import org.elasticsearch.client.ml.StartDatafeedRequest; +import org.elasticsearch.client.ml.StopDataFrameAnalyticsRequest; import org.elasticsearch.client.ml.StopDatafeedRequest; import org.elasticsearch.client.ml.UpdateDatafeedRequest; import org.elasticsearch.client.ml.UpdateFilterRequest; @@ -105,11 +112,11 @@ static Request getJob(GetJobRequest getJobRequest) { .build(); Request request = new Request(HttpGet.METHOD_NAME, endpoint); - RequestConverters.Params params = new RequestConverters.Params(request); + RequestConverters.Params params = new RequestConverters.Params(); if (getJobRequest.getAllowNoJobs() != null) { params.putParam("allow_no_jobs", Boolean.toString(getJobRequest.getAllowNoJobs())); } - + request.addParameters(params.asMap()); return request; } @@ -122,10 +129,11 @@ static Request getJobStats(GetJobStatsRequest getJobStatsRequest) { .build(); Request request = new Request(HttpGet.METHOD_NAME, endpoint); - RequestConverters.Params params = new RequestConverters.Params(request); + RequestConverters.Params params = new RequestConverters.Params(); if (getJobStatsRequest.getAllowNoJobs() != null) { params.putParam("allow_no_jobs", Boolean.toString(getJobStatsRequest.getAllowNoJobs())); } + request.addParameters(params.asMap()); return request; } @@ -171,14 +179,14 @@ static Request deleteJob(DeleteJobRequest deleteJobRequest) { .build(); Request request = new Request(HttpDelete.METHOD_NAME, endpoint); - RequestConverters.Params params = new RequestConverters.Params(request); + RequestConverters.Params params = new RequestConverters.Params(); if (deleteJobRequest.getForce() != null) { params.putParam("force", Boolean.toString(deleteJobRequest.getForce())); } if (deleteJobRequest.getWaitForCompletion() != null) { params.putParam("wait_for_completion", Boolean.toString(deleteJobRequest.getWaitForCompletion())); } - + request.addParameters(params.asMap()); return request; } @@ -249,12 +257,12 @@ static Request getDatafeed(GetDatafeedRequest getDatafeedRequest) { .build(); Request request = new Request(HttpGet.METHOD_NAME, endpoint); - RequestConverters.Params params = new RequestConverters.Params(request); + RequestConverters.Params params = new RequestConverters.Params(); if (getDatafeedRequest.getAllowNoDatafeeds() != null) { params.putParam(GetDatafeedRequest.ALLOW_NO_DATAFEEDS.getPreferredName(), Boolean.toString(getDatafeedRequest.getAllowNoDatafeeds())); } - + request.addParameters(params.asMap()); return request; } @@ -265,10 +273,11 @@ static Request deleteDatafeed(DeleteDatafeedRequest deleteDatafeedRequest) { .addPathPart(deleteDatafeedRequest.getDatafeedId()) .build(); Request request = new Request(HttpDelete.METHOD_NAME, endpoint); - RequestConverters.Params params = new RequestConverters.Params(request); + RequestConverters.Params params = new RequestConverters.Params(); if (deleteDatafeedRequest.getForce() != null) { params.putParam("force", Boolean.toString(deleteDatafeedRequest.getForce())); } + request.addParameters(params.asMap()); return request; } @@ -305,10 +314,11 @@ static Request getDatafeedStats(GetDatafeedStatsRequest getDatafeedStatsRequest) .build(); Request request = new Request(HttpGet.METHOD_NAME, endpoint); - RequestConverters.Params params = new RequestConverters.Params(request); + RequestConverters.Params params = new RequestConverters.Params(); if (getDatafeedStatsRequest.getAllowNoDatafeeds() != null) { params.putParam("allow_no_datafeeds", Boolean.toString(getDatafeedStatsRequest.getAllowNoDatafeeds())); } + request.addParameters(params.asMap()); return request; } @@ -331,13 +341,14 @@ static Request deleteForecast(DeleteForecastRequest deleteForecastRequest) { .addPathPart(Strings.collectionToCommaDelimitedString(deleteForecastRequest.getForecastIds())) .build(); Request request = new Request(HttpDelete.METHOD_NAME, endpoint); - RequestConverters.Params params = new RequestConverters.Params(request); + RequestConverters.Params params = new RequestConverters.Params(); if (deleteForecastRequest.getAllowNoForecasts() != null) { params.putParam("allow_no_forecasts", Boolean.toString(deleteForecastRequest.getAllowNoForecasts())); } if (deleteForecastRequest.timeout() != null) { params.putParam("timeout", deleteForecastRequest.timeout().getStringRep()); } + request.addParameters(params.asMap()); return request; } @@ -453,7 +464,7 @@ static Request postData(PostDataRequest postDataRequest) { .build(); Request request = new Request(HttpPost.METHOD_NAME, endpoint); - RequestConverters.Params params = new RequestConverters.Params(request); + RequestConverters.Params params = new RequestConverters.Params(); if (postDataRequest.getResetStart() != null) { params.putParam(PostDataRequest.RESET_START.getPreferredName(), postDataRequest.getResetStart()); } @@ -461,6 +472,7 @@ static Request postData(PostDataRequest postDataRequest) { params.putParam(PostDataRequest.RESET_END.getPreferredName(), postDataRequest.getResetEnd()); } BytesReference content = postDataRequest.getContent(); + request.addParameters(params.asMap()); if (content != null) { BytesRef source = postDataRequest.getContent().toBytesRef(); HttpEntity byteEntity = new NByteArrayEntity(source.bytes, @@ -576,6 +588,118 @@ static Request deleteCalendarEvent(DeleteCalendarEventRequest deleteCalendarEven return new Request(HttpDelete.METHOD_NAME, endpoint); } + static Request putDataFrameAnalytics(PutDataFrameAnalyticsRequest putRequest) throws IOException { + String endpoint = new EndpointBuilder() + .addPathPartAsIs("_ml", "data_frame", "analytics") + .addPathPart(putRequest.getConfig().getId()) + .build(); + Request request = new Request(HttpPut.METHOD_NAME, endpoint); + request.setEntity(createEntity(putRequest, REQUEST_BODY_CONTENT_TYPE)); + return request; + } + + static Request getDataFrameAnalytics(GetDataFrameAnalyticsRequest getRequest) { + String endpoint = new EndpointBuilder() + .addPathPartAsIs("_ml", "data_frame", "analytics") + .addPathPart(Strings.collectionToCommaDelimitedString(getRequest.getIds())) + .build(); + Request request = new Request(HttpGet.METHOD_NAME, endpoint); + RequestConverters.Params params = new RequestConverters.Params(); + if (getRequest.getPageParams() != null) { + PageParams pageParams = getRequest.getPageParams(); + if (pageParams.getFrom() != null) { + params.putParam(PageParams.FROM.getPreferredName(), pageParams.getFrom().toString()); + } + if (pageParams.getSize() != null) { + params.putParam(PageParams.SIZE.getPreferredName(), pageParams.getSize().toString()); + } + } + if (getRequest.getAllowNoMatch() != null) { + params.putParam(GetDataFrameAnalyticsRequest.ALLOW_NO_MATCH.getPreferredName(), Boolean.toString(getRequest.getAllowNoMatch())); + } + request.addParameters(params.asMap()); + return request; + } + + static Request getDataFrameAnalyticsStats(GetDataFrameAnalyticsStatsRequest getStatsRequest) { + String endpoint = new EndpointBuilder() + .addPathPartAsIs("_ml", "data_frame", "analytics") + .addPathPart(Strings.collectionToCommaDelimitedString(getStatsRequest.getIds())) + .addPathPartAsIs("_stats") + .build(); + Request request = new Request(HttpGet.METHOD_NAME, endpoint); + RequestConverters.Params params = new RequestConverters.Params(); + if (getStatsRequest.getPageParams() != null) { + PageParams pageParams = getStatsRequest.getPageParams(); + if (pageParams.getFrom() != null) { + params.putParam(PageParams.FROM.getPreferredName(), pageParams.getFrom().toString()); + } + if (pageParams.getSize() != null) { + params.putParam(PageParams.SIZE.getPreferredName(), pageParams.getSize().toString()); + } + } + if (getStatsRequest.getAllowNoMatch() != null) { + params.putParam(GetDataFrameAnalyticsStatsRequest.ALLOW_NO_MATCH.getPreferredName(), + Boolean.toString(getStatsRequest.getAllowNoMatch())); + } + request.addParameters(params.asMap()); + return request; + } + + static Request startDataFrameAnalytics(StartDataFrameAnalyticsRequest startRequest) { + String endpoint = new EndpointBuilder() + .addPathPartAsIs("_ml", "data_frame", "analytics") + .addPathPart(startRequest.getId()) + .addPathPartAsIs("_start") + .build(); + Request request = new Request(HttpPost.METHOD_NAME, endpoint); + RequestConverters.Params params = new RequestConverters.Params(); + if (startRequest.getTimeout() != null) { + params.withTimeout(startRequest.getTimeout()); + } + request.addParameters(params.asMap()); + return request; + } + + static Request stopDataFrameAnalytics(StopDataFrameAnalyticsRequest stopRequest) { + String endpoint = new EndpointBuilder() + .addPathPartAsIs("_ml", "data_frame", "analytics") + .addPathPart(stopRequest.getId()) + .addPathPartAsIs("_stop") + .build(); + Request request = new Request(HttpPost.METHOD_NAME, endpoint); + RequestConverters.Params params = new RequestConverters.Params(); + if (stopRequest.getTimeout() != null) { + params.withTimeout(stopRequest.getTimeout()); + } + if (stopRequest.getAllowNoMatch() != null) { + params.putParam( + StopDataFrameAnalyticsRequest.ALLOW_NO_MATCH.getPreferredName(), Boolean.toString(stopRequest.getAllowNoMatch())); + } + if (stopRequest.getForce() != null) { + params.putParam(StopDataFrameAnalyticsRequest.FORCE.getPreferredName(), Boolean.toString(stopRequest.getForce())); + } + request.addParameters(params.asMap()); + return request; + } + + static Request deleteDataFrameAnalytics(DeleteDataFrameAnalyticsRequest deleteRequest) { + String endpoint = new EndpointBuilder() + .addPathPartAsIs("_ml", "data_frame", "analytics") + .addPathPart(deleteRequest.getId()) + .build(); + return new Request(HttpDelete.METHOD_NAME, endpoint); + } + + static Request evaluateDataFrame(EvaluateDataFrameRequest evaluateRequest) throws IOException { + String endpoint = new EndpointBuilder() + .addPathPartAsIs("_ml", "data_frame", "_evaluate") + .build(); + Request request = new Request(HttpPost.METHOD_NAME, endpoint); + request.setEntity(createEntity(evaluateRequest, REQUEST_BODY_CONTENT_TYPE)); + return request; + } + static Request putFilter(PutFilterRequest putFilterRequest) throws IOException { String endpoint = new EndpointBuilder() .addPathPartAsIs("_ml") @@ -594,13 +718,14 @@ static Request getFilter(GetFiltersRequest getFiltersRequest) { .addPathPart(getFiltersRequest.getFilterId()) .build(); Request request = new Request(HttpGet.METHOD_NAME, endpoint); - RequestConverters.Params params = new RequestConverters.Params(request); + RequestConverters.Params params = new RequestConverters.Params(); if (getFiltersRequest.getSize() != null) { params.putParam(PageParams.SIZE.getPreferredName(), getFiltersRequest.getSize().toString()); } if (getFiltersRequest.getFrom() != null) { params.putParam(PageParams.FROM.getPreferredName(), getFiltersRequest.getFrom().toString()); } + request.addParameters(params.asMap()); return request; } @@ -628,11 +753,12 @@ static Request deleteFilter(DeleteFilterRequest deleteFilterRequest) { static Request setUpgradeMode(SetUpgradeModeRequest setUpgradeModeRequest) { String endpoint = new EndpointBuilder().addPathPartAsIs("_ml", "set_upgrade_mode").build(); Request request = new Request(HttpPost.METHOD_NAME, endpoint); - RequestConverters.Params params = new RequestConverters.Params(request); + RequestConverters.Params params = new RequestConverters.Params(); params.putParam(SetUpgradeModeRequest.ENABLED.getPreferredName(), Boolean.toString(setUpgradeModeRequest.isEnabled())); if (setUpgradeModeRequest.getTimeout() != null) { params.putParam(SetUpgradeModeRequest.TIMEOUT.getPreferredName(), setUpgradeModeRequest.getTimeout().toString()); } + request.addParameters(params.asMap()); return request; } @@ -650,7 +776,7 @@ static Request findFileStructure(FindFileStructureRequest findFileStructureReque .build(); Request request = new Request(HttpPost.METHOD_NAME, endpoint); - RequestConverters.Params params = new RequestConverters.Params(request); + RequestConverters.Params params = new RequestConverters.Params(); if (findFileStructureRequest.getLinesToSample() != null) { params.putParam(FindFileStructureRequest.LINES_TO_SAMPLE.getPreferredName(), findFileStructureRequest.getLinesToSample().toString()); @@ -695,7 +821,7 @@ static Request findFileStructure(FindFileStructureRequest findFileStructureReque if (findFileStructureRequest.getExplain() != null) { params.putParam(FindFileStructureRequest.EXPLAIN.getPreferredName(), findFileStructureRequest.getExplain().toString()); } - + request.addParameters(params.asMap()); BytesReference sample = findFileStructureRequest.getSample(); BytesRef source = sample.toBytesRef(); HttpEntity byteEntity = new NByteArrayEntity(source.bytes, source.offset, source.length, createContentType(XContentType.JSON)); diff --git a/client/rest-high-level/src/main/java/org/elasticsearch/client/MachineLearningClient.java b/client/rest-high-level/src/main/java/org/elasticsearch/client/MachineLearningClient.java index 2e359931c1025..ea72c355a02e7 100644 --- a/client/rest-high-level/src/main/java/org/elasticsearch/client/MachineLearningClient.java +++ b/client/rest-high-level/src/main/java/org/elasticsearch/client/MachineLearningClient.java @@ -25,6 +25,7 @@ import org.elasticsearch.client.ml.DeleteCalendarEventRequest; import org.elasticsearch.client.ml.DeleteCalendarJobRequest; import org.elasticsearch.client.ml.DeleteCalendarRequest; +import org.elasticsearch.client.ml.DeleteDataFrameAnalyticsRequest; import org.elasticsearch.client.ml.DeleteDatafeedRequest; import org.elasticsearch.client.ml.DeleteExpiredDataRequest; import org.elasticsearch.client.ml.DeleteExpiredDataResponse; @@ -33,6 +34,8 @@ import org.elasticsearch.client.ml.DeleteJobRequest; import org.elasticsearch.client.ml.DeleteJobResponse; import org.elasticsearch.client.ml.DeleteModelSnapshotRequest; +import org.elasticsearch.client.ml.EvaluateDataFrameRequest; +import org.elasticsearch.client.ml.EvaluateDataFrameResponse; import org.elasticsearch.client.ml.FindFileStructureRequest; import org.elasticsearch.client.ml.FindFileStructureResponse; import org.elasticsearch.client.ml.FlushJobRequest; @@ -47,6 +50,10 @@ import org.elasticsearch.client.ml.GetCalendarsResponse; import org.elasticsearch.client.ml.GetCategoriesRequest; import org.elasticsearch.client.ml.GetCategoriesResponse; +import org.elasticsearch.client.ml.GetDataFrameAnalyticsRequest; +import org.elasticsearch.client.ml.GetDataFrameAnalyticsResponse; +import org.elasticsearch.client.ml.GetDataFrameAnalyticsStatsRequest; +import org.elasticsearch.client.ml.GetDataFrameAnalyticsStatsResponse; import org.elasticsearch.client.ml.GetDatafeedRequest; import org.elasticsearch.client.ml.GetDatafeedResponse; import org.elasticsearch.client.ml.GetDatafeedStatsRequest; @@ -78,6 +85,8 @@ import org.elasticsearch.client.ml.PutCalendarJobRequest; import org.elasticsearch.client.ml.PutCalendarRequest; import org.elasticsearch.client.ml.PutCalendarResponse; +import org.elasticsearch.client.ml.PutDataFrameAnalyticsRequest; +import org.elasticsearch.client.ml.PutDataFrameAnalyticsResponse; import org.elasticsearch.client.ml.PutDatafeedRequest; import org.elasticsearch.client.ml.PutDatafeedResponse; import org.elasticsearch.client.ml.PutFilterRequest; @@ -87,8 +96,11 @@ import org.elasticsearch.client.ml.RevertModelSnapshotRequest; import org.elasticsearch.client.ml.RevertModelSnapshotResponse; import org.elasticsearch.client.ml.SetUpgradeModeRequest; +import org.elasticsearch.client.ml.StartDataFrameAnalyticsRequest; import org.elasticsearch.client.ml.StartDatafeedRequest; import org.elasticsearch.client.ml.StartDatafeedResponse; +import org.elasticsearch.client.ml.StopDataFrameAnalyticsRequest; +import org.elasticsearch.client.ml.StopDataFrameAnalyticsResponse; import org.elasticsearch.client.ml.StopDatafeedRequest; import org.elasticsearch.client.ml.StopDatafeedResponse; import org.elasticsearch.client.ml.UpdateDatafeedRequest; @@ -1877,4 +1889,286 @@ public void setUpgradeModeAsync(SetUpgradeModeRequest request, RequestOptions op listener, Collections.emptySet()); } + + /** + * Creates a new Data Frame Analytics config + *

+ * For additional info + * see PUT Data Frame Analytics documentation + * + * @param request The {@link PutDataFrameAnalyticsRequest} containing the + * {@link org.elasticsearch.client.ml.dataframe.DataFrameAnalyticsConfig} + * @param options Additional request options (e.g. headers), use {@link RequestOptions#DEFAULT} if nothing needs to be customized + * @return The {@link PutDataFrameAnalyticsResponse} containing the created + * {@link org.elasticsearch.client.ml.dataframe.DataFrameAnalyticsConfig} + * @throws IOException when there is a serialization issue sending the request or receiving the response + */ + public PutDataFrameAnalyticsResponse putDataFrameAnalytics(PutDataFrameAnalyticsRequest request, + RequestOptions options) throws IOException { + return restHighLevelClient.performRequestAndParseEntity(request, + MLRequestConverters::putDataFrameAnalytics, + options, + PutDataFrameAnalyticsResponse::fromXContent, + Collections.emptySet()); + } + + /** + * Creates a new Data Frame Analytics config asynchronously and notifies listener upon completion + *

+ * For additional info + * see PUT Data Frame Analytics documentation + * + * @param request The {@link PutDataFrameAnalyticsRequest} containing the + * {@link org.elasticsearch.client.ml.dataframe.DataFrameAnalyticsConfig} + * @param options Additional request options (e.g. headers), use {@link RequestOptions#DEFAULT} if nothing needs to be customized + * @param listener Listener to be notified upon request completion + */ + public void putDataFrameAnalyticsAsync(PutDataFrameAnalyticsRequest request, RequestOptions options, + ActionListener listener) { + restHighLevelClient.performRequestAsyncAndParseEntity(request, + MLRequestConverters::putDataFrameAnalytics, + options, + PutDataFrameAnalyticsResponse::fromXContent, + listener, + Collections.emptySet()); + } + + /** + * Gets a single or multiple Data Frame Analytics configs + *

+ * For additional info + * see GET Data Frame Analytics documentation + * + * @param request The {@link GetDataFrameAnalyticsRequest} + * @param options Additional request options (e.g. headers), use {@link RequestOptions#DEFAULT} if nothing needs to be customized + * @return {@link GetDataFrameAnalyticsResponse} response object containing the + * {@link org.elasticsearch.client.ml.dataframe.DataFrameAnalyticsConfig} objects + */ + public GetDataFrameAnalyticsResponse getDataFrameAnalytics(GetDataFrameAnalyticsRequest request, + RequestOptions options) throws IOException { + return restHighLevelClient.performRequestAndParseEntity(request, + MLRequestConverters::getDataFrameAnalytics, + options, + GetDataFrameAnalyticsResponse::fromXContent, + Collections.emptySet()); + } + + /** + * Gets a single or multiple Data Frame Analytics configs asynchronously and notifies listener upon completion + *

+ * For additional info + * see GET Data Frame Analytics documentation + * + * @param request The {@link GetDataFrameAnalyticsRequest} + * @param options Additional request options (e.g. headers), use {@link RequestOptions#DEFAULT} if nothing needs to be customized + * @param listener Listener to be notified upon request completion + */ + public void getDataFrameAnalyticsAsync(GetDataFrameAnalyticsRequest request, RequestOptions options, + ActionListener listener) { + restHighLevelClient.performRequestAsyncAndParseEntity(request, + MLRequestConverters::getDataFrameAnalytics, + options, + GetDataFrameAnalyticsResponse::fromXContent, + listener, + Collections.emptySet()); + } + + /** + * Gets the running statistics of a Data Frame Analytics + *

+ * For additional info + * see GET Data Frame Analytics Stats documentation + * + * @param request The {@link GetDataFrameAnalyticsStatsRequest} + * @param options Additional request options (e.g. headers), use {@link RequestOptions#DEFAULT} if nothing needs to be customized + * @return {@link GetDataFrameAnalyticsStatsResponse} response object + */ + public GetDataFrameAnalyticsStatsResponse getDataFrameAnalyticsStats(GetDataFrameAnalyticsStatsRequest request, + RequestOptions options) throws IOException { + return restHighLevelClient.performRequestAndParseEntity(request, + MLRequestConverters::getDataFrameAnalyticsStats, + options, + GetDataFrameAnalyticsStatsResponse::fromXContent, + Collections.emptySet()); + } + + /** + * Gets the running statistics of a Data Frame Analytics asynchronously and notifies listener upon completion + *

+ * For additional info + * see GET Data Frame Analytics Stats documentation + * + * @param request The {@link GetDataFrameAnalyticsStatsRequest} + * @param options Additional request options (e.g. headers), use {@link RequestOptions#DEFAULT} if nothing needs to be customized + * @param listener Listener to be notified upon request completion + */ + public void getDataFrameAnalyticsStatsAsync(GetDataFrameAnalyticsStatsRequest request, RequestOptions options, + ActionListener listener) { + restHighLevelClient.performRequestAsyncAndParseEntity(request, + MLRequestConverters::getDataFrameAnalyticsStats, + options, + GetDataFrameAnalyticsStatsResponse::fromXContent, + listener, + Collections.emptySet()); + } + + /** + * Starts Data Frame Analytics + *

+ * For additional info + * see Start Data Frame Analytics documentation + * + * @param request The {@link StartDataFrameAnalyticsRequest} + * @param options Additional request options (e.g. headers), use {@link RequestOptions#DEFAULT} if nothing needs to be customized + * @return action acknowledgement + * @throws IOException when there is a serialization issue sending the request or receiving the response + */ + public AcknowledgedResponse startDataFrameAnalytics(StartDataFrameAnalyticsRequest request, + RequestOptions options) throws IOException { + return restHighLevelClient.performRequestAndParseEntity(request, + MLRequestConverters::startDataFrameAnalytics, + options, + AcknowledgedResponse::fromXContent, + Collections.emptySet()); + } + + /** + * Starts Data Frame Analytics asynchronously and notifies listener upon completion + *

+ * For additional info + * see Start Data Frame Analytics documentation + * + * @param request The {@link StartDataFrameAnalyticsRequest} + * @param options Additional request options (e.g. headers), use {@link RequestOptions#DEFAULT} if nothing needs to be customized + * @param listener Listener to be notified upon request completion + */ + public void startDataFrameAnalyticsAsync(StartDataFrameAnalyticsRequest request, RequestOptions options, + ActionListener listener) { + restHighLevelClient.performRequestAsyncAndParseEntity(request, + MLRequestConverters::startDataFrameAnalytics, + options, + AcknowledgedResponse::fromXContent, + listener, + Collections.emptySet()); + } + + /** + * Stops Data Frame Analytics + *

+ * For additional info + * see Stop Data Frame Analytics documentation + * + * @param request The {@link StopDataFrameAnalyticsRequest} + * @param options Additional request options (e.g. headers), use {@link RequestOptions#DEFAULT} if nothing needs to be customized + * @return {@link StopDataFrameAnalyticsResponse} + * @throws IOException when there is a serialization issue sending the request or receiving the response + */ + public StopDataFrameAnalyticsResponse stopDataFrameAnalytics(StopDataFrameAnalyticsRequest request, + RequestOptions options) throws IOException { + return restHighLevelClient.performRequestAndParseEntity(request, + MLRequestConverters::stopDataFrameAnalytics, + options, + StopDataFrameAnalyticsResponse::fromXContent, + Collections.emptySet()); + } + + /** + * Stops Data Frame Analytics asynchronously and notifies listener upon completion + *

+ * For additional info + * see Stop Data Frame Analytics documentation + * + * @param request The {@link StopDataFrameAnalyticsRequest} + * @param options Additional request options (e.g. headers), use {@link RequestOptions#DEFAULT} if nothing needs to be customized + * @param listener Listener to be notified upon request completion + */ + public void stopDataFrameAnalyticsAsync(StopDataFrameAnalyticsRequest request, RequestOptions options, + ActionListener listener) { + restHighLevelClient.performRequestAsyncAndParseEntity(request, + MLRequestConverters::stopDataFrameAnalytics, + options, + StopDataFrameAnalyticsResponse::fromXContent, + listener, + Collections.emptySet()); + } + + /** + * Deletes the given Data Frame Analytics config + *

+ * For additional info + * see DELETE Data Frame Analytics documentation + * + * @param request The {@link DeleteDataFrameAnalyticsRequest} + * @param options Additional request options (e.g. headers), use {@link RequestOptions#DEFAULT} if nothing needs to be customized + * @return action acknowledgement + * @throws IOException when there is a serialization issue sending the request or receiving the response + */ + public AcknowledgedResponse deleteDataFrameAnalytics(DeleteDataFrameAnalyticsRequest request, + RequestOptions options) throws IOException { + return restHighLevelClient.performRequestAndParseEntity(request, + MLRequestConverters::deleteDataFrameAnalytics, + options, + AcknowledgedResponse::fromXContent, + Collections.emptySet()); + } + + /** + * Deletes the given Data Frame Analytics config asynchronously and notifies listener upon completion + *

+ * For additional info + * see DELETE Data Frame Analytics documentation + * + * @param request The {@link DeleteDataFrameAnalyticsRequest} + * @param options Additional request options (e.g. headers), use {@link RequestOptions#DEFAULT} if nothing needs to be customized + * @param listener Listener to be notified upon request completion + */ + public void deleteDataFrameAnalyticsAsync(DeleteDataFrameAnalyticsRequest request, RequestOptions options, + ActionListener listener) { + restHighLevelClient.performRequestAsyncAndParseEntity(request, + MLRequestConverters::deleteDataFrameAnalytics, + options, + AcknowledgedResponse::fromXContent, + listener, + Collections.emptySet()); + } + + /** + * Evaluates the given Data Frame + *

+ * For additional info + * see Evaluate Data Frame documentation + * + * @param request The {@link EvaluateDataFrameRequest} + * @param options Additional request options (e.g. headers), use {@link RequestOptions#DEFAULT} if nothing needs to be customized + * @return {@link EvaluateDataFrameResponse} response object + * @throws IOException when there is a serialization issue sending the request or receiving the response + */ + public EvaluateDataFrameResponse evaluateDataFrame(EvaluateDataFrameRequest request, + RequestOptions options) throws IOException { + return restHighLevelClient.performRequestAndParseEntity(request, + MLRequestConverters::evaluateDataFrame, + options, + EvaluateDataFrameResponse::fromXContent, + Collections.emptySet()); + } + + /** + * Evaluates the given Data Frame asynchronously and notifies listener upon completion + *

+ * For additional info + * see Evaluate Data Frame documentation + * + * @param request The {@link EvaluateDataFrameRequest} + * @param options Additional request options (e.g. headers), use {@link RequestOptions#DEFAULT} if nothing needs to be customized + * @param listener Listener to be notified upon request completion + */ + public void evaluateDataFrameAsync(EvaluateDataFrameRequest request, RequestOptions options, + ActionListener listener) { + restHighLevelClient.performRequestAsyncAndParseEntity(request, + MLRequestConverters::evaluateDataFrame, + options, + EvaluateDataFrameResponse::fromXContent, + listener, + Collections.emptySet()); + } } 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 023bd1fe63786..6adc287645c7c 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 @@ -33,7 +33,6 @@ import org.elasticsearch.action.admin.cluster.storedscripts.DeleteStoredScriptRequest; import org.elasticsearch.action.admin.cluster.storedscripts.GetStoredScriptRequest; import org.elasticsearch.action.admin.cluster.storedscripts.PutStoredScriptRequest; -import org.elasticsearch.action.admin.indices.analyze.AnalyzeRequest; import org.elasticsearch.action.bulk.BulkRequest; import org.elasticsearch.action.delete.DeleteRequest; import org.elasticsearch.action.explain.ExplainRequest; @@ -52,6 +51,7 @@ import org.elasticsearch.client.core.CountRequest; import org.elasticsearch.client.core.MultiTermVectorsRequest; import org.elasticsearch.client.core.TermVectorsRequest; +import org.elasticsearch.client.indices.AnalyzeRequest; import org.elasticsearch.client.security.RefreshPolicy; import org.elasticsearch.cluster.health.ClusterHealthStatus; import org.elasticsearch.common.Nullable; @@ -88,8 +88,10 @@ import java.net.URI; import java.net.URISyntaxException; import java.nio.charset.Charset; +import java.util.HashMap; import java.util.List; import java.util.Locale; +import java.util.Map; import java.util.StringJoiner; final class RequestConverters { @@ -103,7 +105,7 @@ static Request delete(DeleteRequest deleteRequest) { String endpoint = endpoint(deleteRequest.index(), deleteRequest.type(), deleteRequest.id()); Request request = new Request(HttpDelete.METHOD_NAME, endpoint); - Params parameters = new Params(request); + Params parameters = new Params(); parameters.withRouting(deleteRequest.routing()); parameters.withTimeout(deleteRequest.timeout()); parameters.withVersion(deleteRequest.version()); @@ -112,6 +114,7 @@ static Request delete(DeleteRequest deleteRequest) { parameters.withIfPrimaryTerm(deleteRequest.ifPrimaryTerm()); parameters.withRefreshPolicy(deleteRequest.getRefreshPolicy()); parameters.withWaitForActiveShards(deleteRequest.waitForActiveShards()); + request.addParameters(parameters.asMap()); return request; } @@ -122,7 +125,7 @@ static Request info() { static Request bulk(BulkRequest bulkRequest) throws IOException { Request request = new Request(HttpPost.METHOD_NAME, "/_bulk"); - Params parameters = new Params(request); + Params parameters = new Params(); parameters.withTimeout(bulkRequest.timeout()); parameters.withRefreshPolicy(bulkRequest.getRefreshPolicy()); parameters.withPipeline(bulkRequest.pipeline()); @@ -249,6 +252,7 @@ static Request bulk(BulkRequest bulkRequest) throws IOException { content.write(separator); } } + request.addParameters(parameters.asMap()); request.setEntity(new NByteArrayEntity(content.toByteArray(), 0, content.size(), requestContentType)); return request; } @@ -264,7 +268,7 @@ static Request get(GetRequest getRequest) { private static Request getStyleRequest(String method, GetRequest getRequest) { Request request = new Request(method, endpoint(getRequest.index(), getRequest.type(), getRequest.id())); - Params parameters = new Params(request); + Params parameters = new Params(); parameters.withPreference(getRequest.preference()); parameters.withRouting(getRequest.routing()); parameters.withRefresh(getRequest.refresh()); @@ -273,7 +277,7 @@ private static Request getStyleRequest(String method, GetRequest getRequest) { parameters.withVersion(getRequest.version()); parameters.withVersionType(getRequest.versionType()); parameters.withFetchSourceContext(getRequest.fetchSourceContext()); - + request.addParameters(parameters.asMap()); return request; } @@ -286,23 +290,24 @@ static Request sourceExists(GetRequest getRequest) { endpoint = endpoint(getRequest.index(), optionalType, getRequest.id(), "_source"); } Request request = new Request(HttpHead.METHOD_NAME, endpoint); - Params parameters = new Params(request); + Params parameters = new Params(); parameters.withPreference(getRequest.preference()); parameters.withRouting(getRequest.routing()); parameters.withRefresh(getRequest.refresh()); parameters.withRealtime(getRequest.realtime()); // Version params are not currently supported by the source exists API so are not passed + request.addParameters(parameters.asMap()); return request; } static Request multiGet(MultiGetRequest multiGetRequest) throws IOException { Request request = new Request(HttpPost.METHOD_NAME, "/_mget"); - Params parameters = new Params(request); + Params parameters = new Params(); parameters.withPreference(multiGetRequest.preference()); parameters.withRealtime(multiGetRequest.realtime()); parameters.withRefresh(multiGetRequest.refresh()); - + request.addParameters(parameters.asMap()); request.setEntity(createEntity(multiGetRequest, REQUEST_BODY_CONTENT_TYPE)); return request; } @@ -321,7 +326,7 @@ static Request index(IndexRequest indexRequest) { Request request = new Request(method, endpoint); - Params parameters = new Params(request); + Params parameters = new Params(); parameters.withRouting(indexRequest.routing()); parameters.withTimeout(indexRequest.timeout()); parameters.withVersion(indexRequest.version()); @@ -334,6 +339,7 @@ static Request index(IndexRequest indexRequest) { BytesRef source = indexRequest.source().toBytesRef(); ContentType contentType = createContentType(indexRequest.getContentType()); + request.addParameters(parameters.asMap()); request.setEntity(new NByteArrayEntity(source.bytes, source.offset, source.length, contentType)); return request; } @@ -348,7 +354,7 @@ static Request update(UpdateRequest updateRequest) throws IOException { : endpoint(updateRequest.index(), updateRequest.type(), updateRequest.id(), "_update"); Request request = new Request(HttpPost.METHOD_NAME, endpoint); - Params parameters = new Params(request); + Params parameters = new Params(); parameters.withRouting(updateRequest.routing()); parameters.withTimeout(updateRequest.timeout()); parameters.withRefreshPolicy(updateRequest.getRefreshPolicy()); @@ -379,6 +385,7 @@ static Request update(UpdateRequest updateRequest) throws IOException { if (xContentType == null) { xContentType = Requests.INDEX_CONTENT_TYPE; } + request.addParameters(parameters.asMap()); request.setEntity(createEntity(updateRequest, xContentType)); return request; } @@ -391,14 +398,15 @@ static Request update(UpdateRequest updateRequest) throws IOException { * searches. */ static Request search(SearchRequest searchRequest, String searchEndpoint) throws IOException { - Request request = new Request(HttpPost.METHOD_NAME, endpoint(searchRequest.indices(), searchRequest.types(), searchEndpoint)); + Request request = new Request(HttpPost.METHOD_NAME, endpoint(searchRequest.indices(), searchEndpoint)); - Params params = new Params(request); + Params params = new Params(); addSearchRequestParams(params, searchRequest); if (searchRequest.source() != null) { request.setEntity(createEntity(searchRequest.source(), REQUEST_BODY_CONTENT_TYPE)); } + request.addParameters(params.asMap()); return request; } @@ -436,7 +444,7 @@ static Request clearScroll(ClearScrollRequest clearScrollRequest) throws IOExcep static Request multiSearch(MultiSearchRequest multiSearchRequest) throws IOException { Request request = new Request(HttpPost.METHOD_NAME, "/_msearch"); - Params params = new Params(request); + Params params = new Params(); params.putParam(RestSearchAction.TYPED_KEYS_PARAM, "true"); if (multiSearchRequest.maxConcurrentSearchRequests() != MultiSearchRequest.MAX_CONCURRENT_SEARCH_REQUESTS_DEFAULT) { params.putParam("max_concurrent_searches", Integer.toString(multiSearchRequest.maxConcurrentSearchRequests())); @@ -444,6 +452,7 @@ static Request multiSearch(MultiSearchRequest multiSearchRequest) throws IOExcep XContent xContent = REQUEST_BODY_CONTENT_TYPE.xContent(); byte[] source = MultiSearchRequest.writeMultiLineFormat(multiSearchRequest, xContent); + request.addParameters(params.asMap()); request.setEntity(new NByteArrayEntity(source, createContentType(xContent.type()))); return request; } @@ -455,11 +464,12 @@ static Request searchTemplate(SearchTemplateRequest searchTemplateRequest) throw request = new Request(HttpGet.METHOD_NAME, "_render/template"); } else { SearchRequest searchRequest = searchTemplateRequest.getRequest(); - String endpoint = endpoint(searchRequest.indices(), searchRequest.types(), "_search/template"); + String endpoint = endpoint(searchRequest.indices(), "_search/template"); request = new Request(HttpGet.METHOD_NAME, endpoint); - Params params = new Params(request); + Params params = new Params(); addSearchRequestParams(params, searchRequest); + request.addParameters(params.asMap()); } request.setEntity(createEntity(searchTemplateRequest, REQUEST_BODY_CONTENT_TYPE)); @@ -469,7 +479,7 @@ static Request searchTemplate(SearchTemplateRequest searchTemplateRequest) throw static Request multiSearchTemplate(MultiSearchTemplateRequest multiSearchTemplateRequest) throws IOException { Request request = new Request(HttpPost.METHOD_NAME, "/_msearch/template"); - Params params = new Params(request); + Params params = new Params(); params.putParam(RestSearchAction.TYPED_KEYS_PARAM, "true"); if (multiSearchTemplateRequest.maxConcurrentSearchRequests() != MultiSearchRequest.MAX_CONCURRENT_SEARCH_REQUESTS_DEFAULT) { params.putParam("max_concurrent_searches", Integer.toString(multiSearchTemplateRequest.maxConcurrentSearchRequests())); @@ -483,10 +493,11 @@ static Request multiSearchTemplate(MultiSearchTemplateRequest multiSearchTemplat static Request count(CountRequest countRequest) throws IOException { Request request = new Request(HttpPost.METHOD_NAME, endpoint(countRequest.indices(), countRequest.types(), "_count")); - Params params = new Params(request); + Params params = new Params(); params.withRouting(countRequest.routing()); params.withPreference(countRequest.preference()); params.withIndicesOptions(countRequest.indicesOptions()); + request.addParameters(params.asMap()); request.setEntity(createEntity(countRequest.source(), REQUEST_BODY_CONTENT_TYPE)); return request; } @@ -497,11 +508,12 @@ static Request explain(ExplainRequest explainRequest) throws IOException { : endpoint(explainRequest.index(), explainRequest.type(), explainRequest.id(), "_explain"); Request request = new Request(HttpGet.METHOD_NAME, endpoint); - Params params = new Params(request); + Params params = new Params(); params.withStoredFields(explainRequest.storedFields()); params.withFetchSourceContext(explainRequest.fetchSourceContext()); params.withRouting(explainRequest.routing()); params.withPreference(explainRequest.preference()); + request.addParameters(params.asMap()); request.setEntity(createEntity(explainRequest, REQUEST_BODY_CONTENT_TYPE)); return request; } @@ -509,18 +521,19 @@ static Request explain(ExplainRequest explainRequest) throws IOException { static Request fieldCaps(FieldCapabilitiesRequest fieldCapabilitiesRequest) { Request request = new Request(HttpGet.METHOD_NAME, endpoint(fieldCapabilitiesRequest.indices(), "_field_caps")); - Params params = new Params(request); + Params params = new Params(); params.withFields(fieldCapabilitiesRequest.fields()); params.withIndicesOptions(fieldCapabilitiesRequest.indicesOptions()); + request.addParameters(params.asMap()); return request; } static Request rankEval(RankEvalRequest rankEvalRequest) throws IOException { Request request = new Request(HttpGet.METHOD_NAME, endpoint(rankEvalRequest.indices(), Strings.EMPTY_ARRAY, "_rank_eval")); - Params params = new Params(request); + Params params = new Params(); params.withIndicesOptions(rankEvalRequest.indicesOptions()); - + request.addParameters(params.asMap()); request.setEntity(createEntity(rankEvalRequest.getRankEvalSpec(), REQUEST_BODY_CONTENT_TYPE)); return request; } @@ -536,7 +549,7 @@ static Request submitReindex(ReindexRequest reindexRequest) throws IOException { private static Request prepareReindexRequest(ReindexRequest reindexRequest, boolean waitForCompletion) throws IOException { String endpoint = new EndpointBuilder().addPathPart("_reindex").build(); Request request = new Request(HttpPost.METHOD_NAME, endpoint); - Params params = new Params(request) + Params params = new Params() .withWaitForCompletion(waitForCompletion) .withRefresh(reindexRequest.isRefresh()) .withTimeout(reindexRequest.getTimeout()) @@ -546,15 +559,15 @@ private static Request prepareReindexRequest(ReindexRequest reindexRequest, bool if (reindexRequest.getScrollTime() != null) { params.putParam("scroll", reindexRequest.getScrollTime()); } + request.addParameters(params.asMap()); request.setEntity(createEntity(reindexRequest, REQUEST_BODY_CONTENT_TYPE)); return request; } static Request updateByQuery(UpdateByQueryRequest updateByQueryRequest) throws IOException { - String endpoint = - endpoint(updateByQueryRequest.indices(), updateByQueryRequest.getDocTypes(), "_update_by_query"); + String endpoint = endpoint(updateByQueryRequest.indices(), "_update_by_query"); Request request = new Request(HttpPost.METHOD_NAME, endpoint); - Params params = new Params(request) + Params params = new Params() .withRouting(updateByQueryRequest.getRouting()) .withPipeline(updateByQueryRequest.getPipeline()) .withRefresh(updateByQueryRequest.isRefresh()) @@ -571,18 +584,18 @@ static Request updateByQuery(UpdateByQueryRequest updateByQueryRequest) throws I if (updateByQueryRequest.getScrollTime() != AbstractBulkByScrollRequest.DEFAULT_SCROLL_TIMEOUT) { params.putParam("scroll", updateByQueryRequest.getScrollTime()); } - if (updateByQueryRequest.getSize() > 0) { - params.putParam("size", Integer.toString(updateByQueryRequest.getSize())); + if (updateByQueryRequest.getMaxDocs() > 0) { + params.putParam("max_docs", Integer.toString(updateByQueryRequest.getMaxDocs())); } + request.addParameters(params.asMap()); request.setEntity(createEntity(updateByQueryRequest, REQUEST_BODY_CONTENT_TYPE)); return request; } static Request deleteByQuery(DeleteByQueryRequest deleteByQueryRequest) throws IOException { - String endpoint = - endpoint(deleteByQueryRequest.indices(), deleteByQueryRequest.getDocTypes(), "_delete_by_query"); + String endpoint = endpoint(deleteByQueryRequest.indices(), "_delete_by_query"); Request request = new Request(HttpPost.METHOD_NAME, endpoint); - Params params = new Params(request) + Params params = new Params() .withRouting(deleteByQueryRequest.getRouting()) .withRefresh(deleteByQueryRequest.isRefresh()) .withTimeout(deleteByQueryRequest.getTimeout()) @@ -598,9 +611,10 @@ static Request deleteByQuery(DeleteByQueryRequest deleteByQueryRequest) throws I if (deleteByQueryRequest.getScrollTime() != AbstractBulkByScrollRequest.DEFAULT_SCROLL_TIMEOUT) { params.putParam("scroll", deleteByQueryRequest.getScrollTime()); } - if (deleteByQueryRequest.getSize() > 0) { - params.putParam("size", Integer.toString(deleteByQueryRequest.getSize())); + if (deleteByQueryRequest.getMaxDocs() > 0) { + params.putParam("max_docs", Integer.toString(deleteByQueryRequest.getMaxDocs())); } + request.addParameters(params.asMap()); request.setEntity(createEntity(deleteByQueryRequest, REQUEST_BODY_CONTENT_TYPE)); return request; } @@ -621,22 +635,24 @@ private static Request rethrottle(RethrottleRequest rethrottleRequest, String fi String endpoint = new EndpointBuilder().addPathPart(firstPathPart).addPathPart(rethrottleRequest.getTaskId().toString()) .addPathPart("_rethrottle").build(); Request request = new Request(HttpPost.METHOD_NAME, endpoint); - Params params = new Params(request) + Params params = new Params() .withRequestsPerSecond(rethrottleRequest.getRequestsPerSecond()); // we set "group_by" to "none" because this is the response format we can parse back params.putParam("group_by", "none"); + request.addParameters(params.asMap()); return request; } static Request putScript(PutStoredScriptRequest putStoredScriptRequest) throws IOException { String endpoint = new EndpointBuilder().addPathPartAsIs("_scripts").addPathPart(putStoredScriptRequest.id()).build(); Request request = new Request(HttpPost.METHOD_NAME, endpoint); - Params params = new Params(request); + Params params = new Params(); params.withTimeout(putStoredScriptRequest.timeout()); params.withMasterTimeout(putStoredScriptRequest.masterNodeTimeout()); if (Strings.hasText(putStoredScriptRequest.context())) { params.putParam("context", putStoredScriptRequest.context()); } + request.addParameters(params.asMap()); request.setEntity(createEntity(putStoredScriptRequest, REQUEST_BODY_CONTENT_TYPE)); return request; } @@ -667,11 +683,12 @@ static Request termVectors(TermVectorsRequest tvrequest) throws IOException { } Request request = new Request(HttpGet.METHOD_NAME, endpoint); - Params params = new Params(request); + Params params = new Params(); params.withRouting(tvrequest.getRouting()); params.withPreference(tvrequest.getPreference()); params.withFields(tvrequest.getFields()); params.withRealtime(tvrequest.getRealtime()); + request.addParameters(params.asMap()); request.setEntity(createEntity(tvrequest, REQUEST_BODY_CONTENT_TYPE)); return request; } @@ -686,17 +703,19 @@ static Request mtermVectors(MultiTermVectorsRequest mtvrequest) throws IOExcepti static Request getScript(GetStoredScriptRequest getStoredScriptRequest) { String endpoint = new EndpointBuilder().addPathPartAsIs("_scripts").addPathPart(getStoredScriptRequest.id()).build(); Request request = new Request(HttpGet.METHOD_NAME, endpoint); - Params params = new Params(request); + Params params = new Params(); params.withMasterTimeout(getStoredScriptRequest.masterNodeTimeout()); + request.addParameters(params.asMap()); return request; } static Request deleteScript(DeleteStoredScriptRequest deleteStoredScriptRequest) { String endpoint = new EndpointBuilder().addPathPartAsIs("_scripts").addPathPart(deleteStoredScriptRequest.id()).build(); Request request = new Request(HttpDelete.METHOD_NAME, endpoint); - Params params = new Params(request); + Params params = new Params(); params.withTimeout(deleteStoredScriptRequest.timeout()); params.withMasterTimeout(deleteStoredScriptRequest.masterNodeTimeout()); + request.addParameters(params.asMap()); return request; } @@ -710,10 +729,12 @@ static HttpEntity createEntity(ToXContent toXContent, XContentType xContentType, return new NByteArrayEntity(source.bytes, source.offset, source.length, createContentType(xContentType)); } + @Deprecated static String endpoint(String index, String type, String id) { return new EndpointBuilder().addPathPart(index, type, id).build(); } + @Deprecated static String endpoint(String index, String type, String id, String endpoint) { return new EndpointBuilder().addPathPart(index, type, id).addPathPartAsIs(endpoint).build(); } @@ -726,6 +747,7 @@ static String endpoint(String[] indices, String endpoint) { return new EndpointBuilder().addCommaSeparatedPathParts(indices).addPathPartAsIs(endpoint).build(); } + @Deprecated static String endpoint(String[] indices, String[] types, String endpoint) { return new EndpointBuilder().addCommaSeparatedPathParts(indices).addCommaSeparatedPathParts(types) .addPathPartAsIs(endpoint).build(); @@ -736,6 +758,7 @@ static String endpoint(String[] indices, String endpoint, String[] suffixes) { .addCommaSeparatedPathParts(suffixes).build(); } + @Deprecated static String endpoint(String[] indices, String endpoint, String type) { return new EndpointBuilder().addCommaSeparatedPathParts(indices).addPathPartAsIs(endpoint).addPathPart(type).build(); } @@ -756,15 +779,14 @@ public static ContentType createContentType(final XContentType xContentType) { * a {@link Request} and adds the parameters to it directly. */ static class Params { - private final Request request; + private final Map parameters = new HashMap<>(); - Params(Request request) { - this.request = request; + Params() { } Params putParam(String name, String value) { if (Strings.hasLength(value)) { - request.addParameter(name, value); + parameters.put(name,value); } return this; } @@ -776,6 +798,10 @@ Params putParam(String key, TimeValue value) { return this; } + Map asMap(){ + return parameters; + } + Params withDocAsUpsert(boolean docAsUpsert) { if (docAsUpsert) { return putParam("doc_as_upsert", Boolean.TRUE.toString()); @@ -939,6 +965,7 @@ Params withIndicesOptions(IndicesOptions indicesOptions) { expandWildcards = joiner.toString(); } putParam("expand_wildcards", expandWildcards); + putParam("ignore_throttled", Boolean.toString(indicesOptions.ignoreThrottled())); } return this; } @@ -1016,13 +1043,6 @@ Params withParentTaskId(TaskId parentTaskId) { return this; } - Params withVerify(boolean verify) { - if (verify) { - return putParam("verify", Boolean.TRUE.toString()); - } - return this; - } - Params withWaitForStatus(ClusterHealthStatus status) { if (status != null) { return putParam("wait_for_status", status.name().toLowerCase(Locale.ROOT)); diff --git a/client/rest-high-level/src/main/java/org/elasticsearch/client/RollupRequestConverters.java b/client/rest-high-level/src/main/java/org/elasticsearch/client/RollupRequestConverters.java index 4cd3be057a9f5..a9f6ff6d5f9fb 100644 --- a/client/rest-high-level/src/main/java/org/elasticsearch/client/RollupRequestConverters.java +++ b/client/rest-high-level/src/main/java/org/elasticsearch/client/RollupRequestConverters.java @@ -68,11 +68,12 @@ static Request stopJob(final StopRollupJobRequest stopRollupJobRequest) throws I .build(); Request request = new Request(HttpPost.METHOD_NAME, endpoint); - RequestConverters.Params parameters = new RequestConverters.Params(request); + RequestConverters.Params parameters = new RequestConverters.Params(); parameters.withTimeout(stopRollupJobRequest.timeout()); if (stopRollupJobRequest.waitForCompletion() != null) { parameters.withWaitForCompletion(stopRollupJobRequest.waitForCompletion()); } + request.addParameters(parameters.asMap()); return request; } @@ -95,16 +96,6 @@ static Request deleteJob(final DeleteRollupJobRequest deleteRollupJobRequest) th } static Request search(final SearchRequest request) throws IOException { - if (request.types().length > 0) { - /* - * Ideally we'd check this with the standard validation framework - * but we don't have a special request for rollup search so that'd - * be difficult. - */ - ValidationException ve = new ValidationException(); - ve.addValidationError("types are not allowed in rollup search"); - throw ve; - } return RequestConverters.search(request, "_rollup_search"); } diff --git a/client/rest-high-level/src/main/java/org/elasticsearch/client/SecurityClient.java b/client/rest-high-level/src/main/java/org/elasticsearch/client/SecurityClient.java index fefb5771dc801..8c29cfaae54e1 100644 --- a/client/rest-high-level/src/main/java/org/elasticsearch/client/SecurityClient.java +++ b/client/rest-high-level/src/main/java/org/elasticsearch/client/SecurityClient.java @@ -43,6 +43,8 @@ import org.elasticsearch.client.security.EnableUserRequest; import org.elasticsearch.client.security.GetApiKeyRequest; import org.elasticsearch.client.security.GetApiKeyResponse; +import org.elasticsearch.client.security.GetBuiltinPrivilegesRequest; +import org.elasticsearch.client.security.GetBuiltinPrivilegesResponse; import org.elasticsearch.client.security.GetPrivilegesRequest; import org.elasticsearch.client.security.GetPrivilegesResponse; import org.elasticsearch.client.security.GetRoleMappingsRequest; @@ -751,6 +753,34 @@ public void invalidateTokenAsync(InvalidateTokenRequest request, RequestOptions InvalidateTokenResponse::fromXContent, listener, emptySet()); } + /** + * Synchronously get builtin (cluster & index) privilege(s). + * See + * the docs for more. + * + * @param options the request options (e.g. headers), use + * {@link RequestOptions#DEFAULT} if nothing needs to be customized + * @return the response from the get builtin privileges call + * @throws IOException in case there is a problem sending the request or parsing back the response + */ + public GetBuiltinPrivilegesResponse getBuiltinPrivileges(final RequestOptions options) throws IOException { + return restHighLevelClient.performRequestAndParseEntity(GetBuiltinPrivilegesRequest.INSTANCE, + GetBuiltinPrivilegesRequest::getRequest, options, GetBuiltinPrivilegesResponse::fromXContent, emptySet()); + } + + /** + * Asynchronously get builtin (cluster & index) privilege(s). + * See + * the docs for more. + * + * @param options the request options (e.g. headers), use {@link RequestOptions#DEFAULT} if nothing needs to be customized + * @param listener the listener to be notified upon request completion + */ + public void getBuiltinPrivilegesAsync(final RequestOptions options, final ActionListener listener) { + restHighLevelClient.performRequestAsyncAndParseEntity(GetBuiltinPrivilegesRequest.INSTANCE, + GetBuiltinPrivilegesRequest::getRequest, options, GetBuiltinPrivilegesResponse::fromXContent, listener, emptySet()); + } + /** * Synchronously get application privilege(s). * See diff --git a/client/rest-high-level/src/main/java/org/elasticsearch/client/SecurityRequestConverters.java b/client/rest-high-level/src/main/java/org/elasticsearch/client/SecurityRequestConverters.java index f686167e211bb..18ecc2cea281a 100644 --- a/client/rest-high-level/src/main/java/org/elasticsearch/client/SecurityRequestConverters.java +++ b/client/rest-high-level/src/main/java/org/elasticsearch/client/SecurityRequestConverters.java @@ -66,8 +66,9 @@ static Request changePassword(ChangePasswordRequest changePasswordRequest) throw .build(); Request request = new Request(HttpPost.METHOD_NAME, endpoint); request.setEntity(createEntity(changePasswordRequest, REQUEST_BODY_CONTENT_TYPE)); - RequestConverters.Params params = new RequestConverters.Params(request); + RequestConverters.Params params = new RequestConverters.Params(); params.withRefreshPolicy(changePasswordRequest.getRefreshPolicy()); + request.addParameters(params.asMap()); return request; } @@ -87,8 +88,9 @@ static Request putUser(PutUserRequest putUserRequest) throws IOException { .build(); Request request = new Request(HttpPut.METHOD_NAME, endpoint); request.setEntity(createEntity(putUserRequest, REQUEST_BODY_CONTENT_TYPE)); - RequestConverters.Params params = new RequestConverters.Params(request); + RequestConverters.Params params = new RequestConverters.Params(); params.withRefreshPolicy(putUserRequest.getRefreshPolicy()); + request.addParameters(params.asMap()); return request; } @@ -98,8 +100,9 @@ static Request deleteUser(DeleteUserRequest deleteUserRequest) { .addPathPart(deleteUserRequest.getName()) .build(); Request request = new Request(HttpDelete.METHOD_NAME, endpoint); - RequestConverters.Params params = new RequestConverters.Params(request); + RequestConverters.Params params = new RequestConverters.Params(); params.withRefreshPolicy(deleteUserRequest.getRefreshPolicy()); + request.addParameters(params.asMap()); return request; } @@ -110,8 +113,9 @@ static Request putRoleMapping(final PutRoleMappingRequest putRoleMappingRequest) .build(); final Request request = new Request(HttpPut.METHOD_NAME, endpoint); request.setEntity(createEntity(putRoleMappingRequest, REQUEST_BODY_CONTENT_TYPE)); - final RequestConverters.Params params = new RequestConverters.Params(request); + final RequestConverters.Params params = new RequestConverters.Params(); params.withRefreshPolicy(putRoleMappingRequest.getRefreshPolicy()); + request.addParameters(params.asMap()); return request; } @@ -139,8 +143,9 @@ private static Request setUserEnabled(SetUserEnabledRequest setUserEnabledReques .addPathPart(setUserEnabledRequest.isEnabled() ? "_enable" : "_disable") .build(); Request request = new Request(HttpPut.METHOD_NAME, endpoint); - RequestConverters.Params params = new RequestConverters.Params(request); + RequestConverters.Params params = new RequestConverters.Params(); params.withRefreshPolicy(setUserEnabledRequest.getRefreshPolicy()); + request.addParameters(params.asMap()); return request; } @@ -161,8 +166,9 @@ static Request clearRealmCache(ClearRealmCacheRequest clearRealmCacheRequest) { final String endpoint = builder.addPathPartAsIs("_clear_cache").build(); Request request = new Request(HttpPost.METHOD_NAME, endpoint); if (clearRealmCacheRequest.getUsernames().isEmpty() == false) { - RequestConverters.Params params = new RequestConverters.Params(request); + RequestConverters.Params params = new RequestConverters.Params(); params.putParam("usernames", Strings.collectionToCommaDelimitedString(clearRealmCacheRequest.getUsernames())); + request.addParameters(params.asMap()); } return request; } @@ -182,8 +188,9 @@ static Request deleteRoleMapping(DeleteRoleMappingRequest deleteRoleMappingReque .addPathPart(deleteRoleMappingRequest.getName()) .build(); final Request request = new Request(HttpDelete.METHOD_NAME, endpoint); - final RequestConverters.Params params = new RequestConverters.Params(request); + final RequestConverters.Params params = new RequestConverters.Params(); params.withRefreshPolicy(deleteRoleMappingRequest.getRefreshPolicy()); + request.addParameters(params.asMap()); return request; } @@ -193,8 +200,9 @@ static Request deleteRole(DeleteRoleRequest deleteRoleRequest) { .addPathPart(deleteRoleRequest.getName()) .build(); Request request = new Request(HttpDelete.METHOD_NAME, endpoint); - RequestConverters.Params params = new RequestConverters.Params(request); + RequestConverters.Params params = new RequestConverters.Params(); params.withRefreshPolicy(deleteRoleRequest.getRefreshPolicy()); + request.addParameters(params.asMap()); return request; } @@ -231,8 +239,9 @@ static Request getPrivileges(GetPrivilegesRequest getPrivilegesRequest) { static Request putPrivileges(final PutPrivilegesRequest putPrivilegesRequest) throws IOException { Request request = new Request(HttpPut.METHOD_NAME, "/_security/privilege"); request.setEntity(createEntity(putPrivilegesRequest, REQUEST_BODY_CONTENT_TYPE)); - RequestConverters.Params params = new RequestConverters.Params(request); + RequestConverters.Params params = new RequestConverters.Params(); params.withRefreshPolicy(putPrivilegesRequest.getRefreshPolicy()); + request.addParameters(params.asMap()); return request; } @@ -243,8 +252,9 @@ static Request deletePrivileges(DeletePrivilegesRequest deletePrivilegeRequest) .addCommaSeparatedPathParts(deletePrivilegeRequest.getPrivileges()) .build(); Request request = new Request(HttpDelete.METHOD_NAME, endpoint); - RequestConverters.Params params = new RequestConverters.Params(request); + RequestConverters.Params params = new RequestConverters.Params(); params.withRefreshPolicy(deletePrivilegeRequest.getRefreshPolicy()); + request.addParameters(params.asMap()); return request; } @@ -255,16 +265,18 @@ static Request putRole(final PutRoleRequest putRoleRequest) throws IOException { .build(); final Request request = new Request(HttpPut.METHOD_NAME, endpoint); request.setEntity(createEntity(putRoleRequest, REQUEST_BODY_CONTENT_TYPE)); - final RequestConverters.Params params = new RequestConverters.Params(request); + final RequestConverters.Params params = new RequestConverters.Params(); params.withRefreshPolicy(putRoleRequest.getRefreshPolicy()); + request.addParameters(params.asMap()); return request; } static Request createApiKey(final CreateApiKeyRequest createApiKeyRequest) throws IOException { final Request request = new Request(HttpPost.METHOD_NAME, "/_security/api_key"); request.setEntity(createEntity(createApiKeyRequest, REQUEST_BODY_CONTENT_TYPE)); - final RequestConverters.Params params = new RequestConverters.Params(request); + final RequestConverters.Params params = new RequestConverters.Params(); params.withRefreshPolicy(createApiKeyRequest.getRefreshPolicy()); + request.addParameters(params.asMap()); return request; } @@ -282,13 +294,13 @@ static Request getApiKey(final GetApiKeyRequest getApiKeyRequest) throws IOExcep if (Strings.hasText(getApiKeyRequest.getRealmName())) { request.addParameter("realm_name", getApiKeyRequest.getRealmName()); } + return request; } static Request invalidateApiKey(final InvalidateApiKeyRequest invalidateApiKeyRequest) throws IOException { final Request request = new Request(HttpDelete.METHOD_NAME, "/_security/api_key"); request.setEntity(createEntity(invalidateApiKeyRequest, REQUEST_BODY_CONTENT_TYPE)); - final RequestConverters.Params params = new RequestConverters.Params(request); return request; } } diff --git a/client/rest-high-level/src/main/java/org/elasticsearch/client/SnapshotRequestConverters.java b/client/rest-high-level/src/main/java/org/elasticsearch/client/SnapshotRequestConverters.java index 93fb10bd56136..406470ea52cda 100644 --- a/client/rest-high-level/src/main/java/org/elasticsearch/client/SnapshotRequestConverters.java +++ b/client/rest-high-level/src/main/java/org/elasticsearch/client/SnapshotRequestConverters.java @@ -46,9 +46,10 @@ static Request getRepositories(GetRepositoriesRequest getRepositoriesRequest) { .build(); Request request = new Request(HttpGet.METHOD_NAME, endpoint); - RequestConverters.Params parameters = new RequestConverters.Params(request); + RequestConverters.Params parameters = new RequestConverters.Params(); parameters.withMasterTimeout(getRepositoriesRequest.masterNodeTimeout()); parameters.withLocal(getRepositoriesRequest.local()); + request.addParameters(parameters.asMap()); return request; } @@ -56,11 +57,13 @@ static Request createRepository(PutRepositoryRequest putRepositoryRequest) throw String endpoint = new RequestConverters.EndpointBuilder().addPathPart("_snapshot").addPathPart(putRepositoryRequest.name()).build(); Request request = new Request(HttpPut.METHOD_NAME, endpoint); - RequestConverters.Params parameters = new RequestConverters.Params(request); + RequestConverters.Params parameters = new RequestConverters.Params(); parameters.withMasterTimeout(putRepositoryRequest.masterNodeTimeout()); parameters.withTimeout(putRepositoryRequest.timeout()); - parameters.withVerify(putRepositoryRequest.verify()); - + if (putRepositoryRequest.verify() == false) { + parameters.putParam("verify", "false"); + } + request.addParameters(parameters.asMap()); request.setEntity(RequestConverters.createEntity(putRepositoryRequest, RequestConverters.REQUEST_BODY_CONTENT_TYPE)); return request; } @@ -70,9 +73,10 @@ static Request deleteRepository(DeleteRepositoryRequest deleteRepositoryRequest) .build(); Request request = new Request(HttpDelete.METHOD_NAME, endpoint); - RequestConverters.Params parameters = new RequestConverters.Params(request); + RequestConverters.Params parameters = new RequestConverters.Params(); parameters.withMasterTimeout(deleteRepositoryRequest.masterNodeTimeout()); parameters.withTimeout(deleteRepositoryRequest.timeout()); + request.addParameters(parameters.asMap()); return request; } @@ -83,9 +87,10 @@ static Request verifyRepository(VerifyRepositoryRequest verifyRepositoryRequest) .build(); Request request = new Request(HttpPost.METHOD_NAME, endpoint); - RequestConverters.Params parameters = new RequestConverters.Params(request); + RequestConverters.Params parameters = new RequestConverters.Params(); parameters.withMasterTimeout(verifyRepositoryRequest.masterNodeTimeout()); parameters.withTimeout(verifyRepositoryRequest.timeout()); + request.addParameters(parameters.asMap()); return request; } @@ -95,16 +100,17 @@ static Request createSnapshot(CreateSnapshotRequest createSnapshotRequest) throw .addPathPart(createSnapshotRequest.snapshot()) .build(); Request request = new Request(HttpPut.METHOD_NAME, endpoint); - RequestConverters.Params params = new RequestConverters.Params(request); + RequestConverters.Params params = new RequestConverters.Params(); params.withMasterTimeout(createSnapshotRequest.masterNodeTimeout()); params.withWaitForCompletion(createSnapshotRequest.waitForCompletion()); + request.addParameters(params.asMap()); request.setEntity(RequestConverters.createEntity(createSnapshotRequest, RequestConverters.REQUEST_BODY_CONTENT_TYPE)); return request; } static Request getSnapshots(GetSnapshotsRequest getSnapshotsRequest) { RequestConverters.EndpointBuilder endpointBuilder = new RequestConverters.EndpointBuilder().addPathPartAsIs("_snapshot") - .addPathPart(getSnapshotsRequest.repository()); + .addCommaSeparatedPathParts(getSnapshotsRequest.repositories()); String endpoint; if (getSnapshotsRequest.snapshots().length == 0) { endpoint = endpointBuilder.addPathPart("_all").build(); @@ -114,11 +120,11 @@ static Request getSnapshots(GetSnapshotsRequest getSnapshotsRequest) { Request request = new Request(HttpGet.METHOD_NAME, endpoint); - RequestConverters.Params parameters = new RequestConverters.Params(request); + RequestConverters.Params parameters = new RequestConverters.Params(); parameters.withMasterTimeout(getSnapshotsRequest.masterNodeTimeout()); parameters.putParam("ignore_unavailable", Boolean.toString(getSnapshotsRequest.ignoreUnavailable())); parameters.putParam("verbose", Boolean.toString(getSnapshotsRequest.verbose())); - + request.addParameters(parameters.asMap()); return request; } @@ -130,9 +136,10 @@ static Request snapshotsStatus(SnapshotsStatusRequest snapshotsStatusRequest) { .build(); Request request = new Request(HttpGet.METHOD_NAME, endpoint); - RequestConverters.Params parameters = new RequestConverters.Params(request); + RequestConverters.Params parameters = new RequestConverters.Params(); parameters.withMasterTimeout(snapshotsStatusRequest.masterNodeTimeout()); parameters.withIgnoreUnavailable(snapshotsStatusRequest.ignoreUnavailable()); + request.addParameters(parameters.asMap()); return request; } @@ -143,9 +150,10 @@ static Request restoreSnapshot(RestoreSnapshotRequest restoreSnapshotRequest) th .addPathPartAsIs("_restore") .build(); Request request = new Request(HttpPost.METHOD_NAME, endpoint); - RequestConverters.Params parameters = new RequestConverters.Params(request); + RequestConverters.Params parameters = new RequestConverters.Params(); parameters.withMasterTimeout(restoreSnapshotRequest.masterNodeTimeout()); parameters.withWaitForCompletion(restoreSnapshotRequest.waitForCompletion()); + request.addParameters(parameters.asMap()); request.setEntity(RequestConverters.createEntity(restoreSnapshotRequest, RequestConverters.REQUEST_BODY_CONTENT_TYPE)); return request; } @@ -157,8 +165,9 @@ static Request deleteSnapshot(DeleteSnapshotRequest deleteSnapshotRequest) { .build(); Request request = new Request(HttpDelete.METHOD_NAME, endpoint); - RequestConverters.Params parameters = new RequestConverters.Params(request); + RequestConverters.Params parameters = new RequestConverters.Params(); parameters.withMasterTimeout(deleteSnapshotRequest.masterNodeTimeout()); + request.addParameters(parameters.asMap()); return request; } } diff --git a/client/rest-high-level/src/main/java/org/elasticsearch/client/TasksRequestConverters.java b/client/rest-high-level/src/main/java/org/elasticsearch/client/TasksRequestConverters.java index f0e9cf4e025f6..f30efabc823e3 100644 --- a/client/rest-high-level/src/main/java/org/elasticsearch/client/TasksRequestConverters.java +++ b/client/rest-high-level/src/main/java/org/elasticsearch/client/TasksRequestConverters.java @@ -32,12 +32,13 @@ private TasksRequestConverters() {} static Request cancelTasks(CancelTasksRequest cancelTasksRequest) { Request request = new Request(HttpPost.METHOD_NAME, "/_tasks/_cancel"); - RequestConverters.Params params = new RequestConverters.Params(request); + RequestConverters.Params params = new RequestConverters.Params(); params.withTimeout(cancelTasksRequest.getTimeout()) .withTaskId(cancelTasksRequest.getTaskId()) .withNodes(cancelTasksRequest.getNodes()) .withParentTaskId(cancelTasksRequest.getParentTaskId()) .withActions(cancelTasksRequest.getActions()); + request.addParameters(params.asMap()); return request; } @@ -46,7 +47,7 @@ static Request listTasks(ListTasksRequest listTaskRequest) { throw new IllegalArgumentException("TaskId cannot be used for list tasks request"); } Request request = new Request(HttpGet.METHOD_NAME, "/_tasks"); - RequestConverters.Params params = new RequestConverters.Params(request); + RequestConverters.Params params = new RequestConverters.Params(); params.withTimeout(listTaskRequest.getTimeout()) .withDetailed(listTaskRequest.getDetailed()) .withWaitForCompletion(listTaskRequest.getWaitForCompletion()) @@ -54,6 +55,7 @@ static Request listTasks(ListTasksRequest listTaskRequest) { .withNodes(listTaskRequest.getNodes()) .withActions(listTaskRequest.getActions()) .putParam("group_by", "none"); + request.addParameters(params.asMap()); return request; } @@ -62,9 +64,10 @@ static Request getTask(GetTaskRequest getTaskRequest) { .addPathPartAsIs(getTaskRequest.getNodeId() + ":" + Long.toString(getTaskRequest.getTaskId())) .build(); Request request = new Request(HttpGet.METHOD_NAME, endpoint); - RequestConverters.Params params = new RequestConverters.Params(request); + RequestConverters.Params params = new RequestConverters.Params(); params.withTimeout(getTaskRequest.getTimeout()) .withWaitForCompletion(getTaskRequest.getWaitForCompletion()); + request.addParameters(params.asMap()); return request; } diff --git a/client/rest-high-level/src/main/java/org/elasticsearch/client/WatcherRequestConverters.java b/client/rest-high-level/src/main/java/org/elasticsearch/client/WatcherRequestConverters.java index 9718607d8b80e..84559fb182306 100644 --- a/client/rest-high-level/src/main/java/org/elasticsearch/client/WatcherRequestConverters.java +++ b/client/rest-high-level/src/main/java/org/elasticsearch/client/WatcherRequestConverters.java @@ -69,12 +69,13 @@ static Request putWatch(PutWatchRequest putWatchRequest) { .build(); Request request = new Request(HttpPut.METHOD_NAME, endpoint); - RequestConverters.Params params = new RequestConverters.Params(request) + RequestConverters.Params params = new RequestConverters.Params() .withIfSeqNo(putWatchRequest.ifSeqNo()) .withIfPrimaryTerm(putWatchRequest.ifPrimaryTerm()); if (putWatchRequest.isActive() == false) { params.putParam("active", "false"); } + request.addParameters(params.asMap()); ContentType contentType = RequestConverters.createContentType(putWatchRequest.xContentType()); BytesReference source = putWatchRequest.getSource(); request.setEntity(new NByteArrayEntity(source.toBytesRef().bytes, 0, source.length(), contentType)); @@ -118,7 +119,7 @@ static Request executeWatch(ExecuteWatchRequest executeWatchRequest) throws IOEx .addPathPartAsIs("_execute").build(); Request request = new Request(HttpPost.METHOD_NAME, endpoint); - RequestConverters.Params params = new RequestConverters.Params(request); + RequestConverters.Params params = new RequestConverters.Params(); if (executeWatchRequest.isDebug()) { params.putParam("debug", "true"); } @@ -128,7 +129,7 @@ static Request executeWatch(ExecuteWatchRequest executeWatchRequest) throws IOEx if (executeWatchRequest.recordExecution()) { params.putParam("record_execution", "true"); } - + request.addParameters(params.asMap()); request.setEntity(RequestConverters.createEntity(executeWatchRequest, XContentType.JSON)); return request; } @@ -158,7 +159,7 @@ static Request watcherStats(WatcherStatsRequest watcherStatsRequest) { RequestConverters.EndpointBuilder builder = new RequestConverters.EndpointBuilder().addPathPartAsIs("_watcher", "stats"); String endpoint = builder.build(); Request request = new Request(HttpGet.METHOD_NAME, endpoint); - RequestConverters.Params parameters = new RequestConverters.Params(request); + RequestConverters.Params parameters = new RequestConverters.Params(); StringBuilder metric = new StringBuilder(); if (watcherStatsRequest.includeCurrentWatches()) { metric.append("current_watches"); @@ -172,6 +173,7 @@ static Request watcherStats(WatcherStatsRequest watcherStatsRequest) { if (metric.length() > 0) { parameters.putParam("metric", metric.toString()); } + request.addParameters(parameters.asMap()); return request; } } diff --git a/client/rest-high-level/src/main/java/org/elasticsearch/client/XPackClient.java b/client/rest-high-level/src/main/java/org/elasticsearch/client/XPackClient.java index a20dfd1ba328a..69cdd329e3984 100644 --- a/client/rest-high-level/src/main/java/org/elasticsearch/client/XPackClient.java +++ b/client/rest-high-level/src/main/java/org/elasticsearch/client/XPackClient.java @@ -35,8 +35,8 @@ * default distribution of Elasticsearch. All of these APIs will 404 if run * against the OSS distribution of Elasticsearch. *

- * See the - * X-Pack APIs on elastic.co for more information. + * See the + * REST APIs on elastic.co for more information. */ public final class XPackClient { diff --git a/client/rest-high-level/src/main/java/org/elasticsearch/client/XPackRequestConverters.java b/client/rest-high-level/src/main/java/org/elasticsearch/client/XPackRequestConverters.java index 9e0c1527403d5..e4742e8c1cbe2 100644 --- a/client/rest-high-level/src/main/java/org/elasticsearch/client/XPackRequestConverters.java +++ b/client/rest-high-level/src/main/java/org/elasticsearch/client/XPackRequestConverters.java @@ -46,8 +46,9 @@ static Request info(XPackInfoRequest infoRequest) { static Request usage(XPackUsageRequest usageRequest) { Request request = new Request(HttpGet.METHOD_NAME, "/_xpack/usage"); - RequestConverters.Params parameters = new RequestConverters.Params(request); + RequestConverters.Params parameters = new RequestConverters.Params(); parameters.withMasterTimeout(usageRequest.masterNodeTimeout()); + request.addParameters(parameters.asMap()); return request; } } diff --git a/client/rest-high-level/src/main/java/org/elasticsearch/client/ccr/IndicesFollowStats.java b/client/rest-high-level/src/main/java/org/elasticsearch/client/ccr/IndicesFollowStats.java index 7d3af08577b16..ae6217d167447 100644 --- a/client/rest-high-level/src/main/java/org/elasticsearch/client/ccr/IndicesFollowStats.java +++ b/client/rest-high-level/src/main/java/org/elasticsearch/client/ccr/IndicesFollowStats.java @@ -101,6 +101,7 @@ public static final class ShardFollowStats { static final ParseField WRITE_BUFFER_SIZE_IN_BYTES_FIELD = new ParseField("write_buffer_size_in_bytes"); static final ParseField FOLLOWER_MAPPING_VERSION_FIELD = new ParseField("follower_mapping_version"); static final ParseField FOLLOWER_SETTINGS_VERSION_FIELD = new ParseField("follower_settings_version"); + static final ParseField FOLLOWER_ALIASES_VERSION_FIELD = new ParseField("follower_aliases_version"); static final ParseField TOTAL_READ_TIME_MILLIS_FIELD = new ParseField("total_read_time_millis"); static final ParseField TOTAL_READ_REMOTE_EXEC_TIME_MILLIS_FIELD = new ParseField("total_read_remote_exec_time_millis"); static final ParseField SUCCESSFUL_READ_REQUESTS_FIELD = new ParseField("successful_read_requests"); @@ -117,41 +118,42 @@ public static final class ShardFollowStats { @SuppressWarnings("unchecked") static final ConstructingObjectParser PARSER = - new ConstructingObjectParser<>( - "shard-follow-stats", - true, - args -> new ShardFollowStats( - (String) args[0], - (String) args[1], - (String) args[2], - (int) args[3], - (long) args[4], - (long) args[5], - (long) args[6], - (long) args[7], - (long) args[8], - (int) args[9], - (int) args[10], - (int) args[11], - (long) args[12], - (long) args[13], - (long) args[14], - (long) args[15], - (long) args[16], - (long) args[17], - (long) args[18], - (long) args[19], - (long) args[20], - (long) args[21], - (long) args[22], - (long) args[23], - (long) args[24], - (long) args[25], - new TreeMap<>( - ((List>>) args[26]) - .stream() - .collect(Collectors.toMap(Map.Entry::getKey, Map.Entry::getValue))), - (ElasticsearchException) args[27])); + new ConstructingObjectParser<>( + "shard-follow-stats", + true, + args -> new ShardFollowStats( + (String) args[0], + (String) args[1], + (String) args[2], + (int) args[3], + (long) args[4], + (long) args[5], + (long) args[6], + (long) args[7], + (long) args[8], + (int) args[9], + (int) args[10], + (int) args[11], + (long) args[12], + (long) args[13], + (long) args[14], + (long) args[15], + (long) args[16], + (long) args[17], + (long) args[18], + (long) args[19], + (long) args[20], + (long) args[21], + (long) args[22], + (long) args[23], + (long) args[24], + (long) args[25], + (long) args[26], + new TreeMap<>( + ((List>>) args[27]) + .stream() + .collect(Collectors.toMap(Map.Entry::getKey, Map.Entry::getValue))), + (ElasticsearchException) args[28])); static final ConstructingObjectParser>, Void> READ_EXCEPTIONS_ENTRY_PARSER = new ConstructingObjectParser<>( @@ -175,6 +177,7 @@ public static final class ShardFollowStats { PARSER.declareLong(ConstructingObjectParser.constructorArg(), WRITE_BUFFER_SIZE_IN_BYTES_FIELD); PARSER.declareLong(ConstructingObjectParser.constructorArg(), FOLLOWER_MAPPING_VERSION_FIELD); PARSER.declareLong(ConstructingObjectParser.constructorArg(), FOLLOWER_SETTINGS_VERSION_FIELD); + PARSER.declareLong(ConstructingObjectParser.constructorArg(), FOLLOWER_ALIASES_VERSION_FIELD); PARSER.declareLong(ConstructingObjectParser.constructorArg(), TOTAL_READ_TIME_MILLIS_FIELD); PARSER.declareLong(ConstructingObjectParser.constructorArg(), TOTAL_READ_REMOTE_EXEC_TIME_MILLIS_FIELD); PARSER.declareLong(ConstructingObjectParser.constructorArg(), SUCCESSFUL_READ_REQUESTS_FIELD); @@ -220,6 +223,7 @@ public static final class ShardFollowStats { private final long writeBufferSizeInBytes; private final long followerMappingVersion; private final long followerSettingsVersion; + private final long followerAliasesVersion; private final long totalReadTimeMillis; private final long totalReadRemoteExecTimeMillis; private final long successfulReadRequests; @@ -249,6 +253,7 @@ public static final class ShardFollowStats { long writeBufferSizeInBytes, long followerMappingVersion, long followerSettingsVersion, + long followerAliasesVersion, long totalReadTimeMillis, long totalReadRemoteExecTimeMillis, long successfulReadRequests, @@ -277,6 +282,7 @@ public static final class ShardFollowStats { this.writeBufferSizeInBytes = writeBufferSizeInBytes; this.followerMappingVersion = followerMappingVersion; this.followerSettingsVersion = followerSettingsVersion; + this.followerAliasesVersion = followerAliasesVersion; this.totalReadTimeMillis = totalReadTimeMillis; this.totalReadRemoteExecTimeMillis = totalReadRemoteExecTimeMillis; this.successfulReadRequests = successfulReadRequests; @@ -352,6 +358,10 @@ public long getFollowerSettingsVersion() { return followerSettingsVersion; } + public long getFollowerAliasesVersion() { + return followerAliasesVersion; + } + public long getTotalReadTimeMillis() { return totalReadTimeMillis; } diff --git a/client/rest-high-level/src/main/java/org/elasticsearch/client/core/BroadcastResponse.java b/client/rest-high-level/src/main/java/org/elasticsearch/client/core/BroadcastResponse.java index 3665ba5bf5009..35ce0f55d717b 100644 --- a/client/rest-high-level/src/main/java/org/elasticsearch/client/core/BroadcastResponse.java +++ b/client/rest-high-level/src/main/java/org/elasticsearch/client/core/BroadcastResponse.java @@ -45,7 +45,7 @@ public Shards shards() { return shards; } - BroadcastResponse(final Shards shards) { + protected BroadcastResponse(final Shards shards) { this.shards = Objects.requireNonNull(shards); } @@ -56,7 +56,7 @@ public Shards shards() { a -> new BroadcastResponse((Shards) a[0])); static { - PARSER.declareObject(ConstructingObjectParser.constructorArg(), Shards.SHARDS_PARSER, SHARDS_FIELD); + declareShardsField(PARSER); } /** @@ -70,6 +70,10 @@ public static BroadcastResponse fromXContent(final XContentParser parser) throws return PARSER.parse(parser, null); } + protected static void declareShardsField(ConstructingObjectParser PARSER) { + PARSER.declareObject(ConstructingObjectParser.constructorArg(), Shards.SHARDS_PARSER, SHARDS_FIELD); + } + /** * Represents the results of a collection of shards on which a request was executed against. */ diff --git a/client/rest-high-level/src/main/java/org/elasticsearch/client/core/TermVectorsResponse.java b/client/rest-high-level/src/main/java/org/elasticsearch/client/core/TermVectorsResponse.java index eef76ed7cb043..8f4232cca6e2c 100644 --- a/client/rest-high-level/src/main/java/org/elasticsearch/client/core/TermVectorsResponse.java +++ b/client/rest-high-level/src/main/java/org/elasticsearch/client/core/TermVectorsResponse.java @@ -33,17 +33,14 @@ public class TermVectorsResponse { private final String index; - private final String type; private final String id; private final long docVersion; private final boolean found; private final long tookInMillis; private final List termVectorList; - public TermVectorsResponse( - String index, String type, String id, long version, boolean found, long tookInMillis, List termVectorList) { + public TermVectorsResponse(String index, String id, long version, boolean found, long tookInMillis, List termVectorList) { this.index = index; - this.type = type; this.id = id; this.docVersion = version; this.found = found; @@ -53,18 +50,17 @@ public TermVectorsResponse( private static ConstructingObjectParser PARSER = new ConstructingObjectParser<>("term_vectors", true, args -> { - // as the response comes from server, we are sure that args[6] will be a list of TermVector - @SuppressWarnings("unchecked") List termVectorList = (List) args[6]; + // as the response comes from server, we are sure that args[5] will be a list of TermVector + @SuppressWarnings("unchecked") List termVectorList = (List) args[5]; if (termVectorList != null) { Collections.sort(termVectorList, Comparator.comparing(TermVector::getFieldName)); } return new TermVectorsResponse( (String) args[0], (String) args[1], - (String) args[2], - (long) args[3], - (boolean) args[4], - (long) args[5], + (long) args[2], + (boolean) args[3], + (long) args[4], termVectorList ); } @@ -72,7 +68,6 @@ public TermVectorsResponse( static { PARSER.declareString(constructorArg(), new ParseField("_index")); - PARSER.declareString(constructorArg(), new ParseField("_type")); PARSER.declareString(optionalConstructorArg(), new ParseField("_id")); PARSER.declareLong(constructorArg(), new ParseField("_version")); PARSER.declareBoolean(constructorArg(), new ParseField("found")); @@ -92,16 +87,6 @@ public String getIndex() { return index; } - /** - * Returns the type for the response - * - * @deprecated Types are in the process of being removed. - */ - @Deprecated - public String getType() { - return type; - } - /** * Returns the id of the request * can be NULL if there is no document ID @@ -145,7 +130,6 @@ public boolean equals(Object obj) { if (!(obj instanceof TermVectorsResponse)) return false; TermVectorsResponse other = (TermVectorsResponse) obj; return index.equals(other.index) - && type.equals(other.type) && Objects.equals(id, other.id) && docVersion == other.docVersion && found == other.found @@ -155,7 +139,7 @@ public boolean equals(Object obj) { @Override public int hashCode() { - return Objects.hash(index, type, id, docVersion, found, tookInMillis, termVectorList); + return Objects.hash(index, id, docVersion, found, tookInMillis, termVectorList); } diff --git a/modules/mapper-extras/src/main/java/org/elasticsearch/index/query/DocValuesWhitelistExtension.java b/client/rest-high-level/src/main/java/org/elasticsearch/client/dataframe/DataFrameNamedXContentProvider.java similarity index 52% rename from modules/mapper-extras/src/main/java/org/elasticsearch/index/query/DocValuesWhitelistExtension.java rename to client/rest-high-level/src/main/java/org/elasticsearch/client/dataframe/DataFrameNamedXContentProvider.java index f463135d69f71..940b136c93daa 100644 --- a/modules/mapper-extras/src/main/java/org/elasticsearch/index/query/DocValuesWhitelistExtension.java +++ b/client/rest-high-level/src/main/java/org/elasticsearch/client/dataframe/DataFrameNamedXContentProvider.java @@ -17,26 +17,25 @@ * under the License. */ -package org.elasticsearch.index.query; +package org.elasticsearch.client.dataframe; +import org.elasticsearch.client.dataframe.transforms.SyncConfig; +import org.elasticsearch.client.dataframe.transforms.TimeSyncConfig; +import org.elasticsearch.common.ParseField; +import org.elasticsearch.common.xcontent.NamedXContentRegistry; +import org.elasticsearch.plugins.spi.NamedXContentProvider; -import org.elasticsearch.painless.spi.PainlessExtension; -import org.elasticsearch.painless.spi.Whitelist; -import org.elasticsearch.painless.spi.WhitelistLoader; -import org.elasticsearch.script.ScoreScript; -import org.elasticsearch.script.ScriptContext; - -import java.util.Collections; +import java.util.Arrays; import java.util.List; -import java.util.Map; - -public class DocValuesWhitelistExtension implements PainlessExtension { - private static final Whitelist WHITELIST = - WhitelistLoader.loadFromResourceFiles(DocValuesWhitelistExtension.class, "docvalues_whitelist.txt"); +public class DataFrameNamedXContentProvider implements NamedXContentProvider { @Override - public Map, List> getContextWhitelists() { - return Collections.singletonMap(ScoreScript.CONTEXT, Collections.singletonList(WHITELIST)); + public List getNamedXContentParsers() { + return Arrays.asList( + new NamedXContentRegistry.Entry(SyncConfig.class, + new ParseField(TimeSyncConfig.NAME), + TimeSyncConfig::fromXContent)); } + } diff --git a/client/rest-high-level/src/main/java/org/elasticsearch/client/dataframe/GetDataFrameTransformRequest.java b/client/rest-high-level/src/main/java/org/elasticsearch/client/dataframe/GetDataFrameTransformRequest.java index c50f37a27c885..cc69e0bd4cd4e 100644 --- a/client/rest-high-level/src/main/java/org/elasticsearch/client/dataframe/GetDataFrameTransformRequest.java +++ b/client/rest-high-level/src/main/java/org/elasticsearch/client/dataframe/GetDataFrameTransformRequest.java @@ -30,6 +30,7 @@ public class GetDataFrameTransformRequest implements Validatable { + public static final String ALLOW_NO_MATCH = "allow_no_match"; /** * Helper method to create a request that will get ALL Data Frame Transforms * @return new {@link GetDataFrameTransformRequest} object for the id "_all" @@ -40,6 +41,7 @@ public static GetDataFrameTransformRequest getAllDataFrameTransformsRequest() { private final List ids; private PageParams pageParams; + private Boolean allowNoMatch; public GetDataFrameTransformRequest(String... ids) { this.ids = Arrays.asList(ids); @@ -57,6 +59,14 @@ public void setPageParams(PageParams pageParams) { this.pageParams = pageParams; } + public Boolean getAllowNoMatch() { + return allowNoMatch; + } + + public void setAllowNoMatch(Boolean allowNoMatch) { + this.allowNoMatch = allowNoMatch; + } + @Override public Optional validate() { if (ids == null || ids.isEmpty()) { @@ -70,7 +80,7 @@ public Optional validate() { @Override public int hashCode() { - return Objects.hash(ids, pageParams); + return Objects.hash(ids, pageParams, allowNoMatch); } @Override @@ -83,6 +93,8 @@ public boolean equals(Object obj) { return false; } GetDataFrameTransformRequest other = (GetDataFrameTransformRequest) obj; - return Objects.equals(ids, other.ids) && Objects.equals(pageParams, other.pageParams); + return Objects.equals(ids, other.ids) + && Objects.equals(pageParams, other.pageParams) + && Objects.equals(allowNoMatch, other.allowNoMatch); } } diff --git a/client/rest-high-level/src/main/java/org/elasticsearch/client/dataframe/GetDataFrameTransformStatsRequest.java b/client/rest-high-level/src/main/java/org/elasticsearch/client/dataframe/GetDataFrameTransformStatsRequest.java index 4a105f7b40c7e..7522ae0d67c26 100644 --- a/client/rest-high-level/src/main/java/org/elasticsearch/client/dataframe/GetDataFrameTransformStatsRequest.java +++ b/client/rest-high-level/src/main/java/org/elasticsearch/client/dataframe/GetDataFrameTransformStatsRequest.java @@ -29,6 +29,7 @@ public class GetDataFrameTransformStatsRequest implements Validatable { private final String id; private PageParams pageParams; + private Boolean allowNoMatch; public GetDataFrameTransformStatsRequest(String id) { this.id = id; @@ -46,6 +47,14 @@ public void setPageParams(PageParams pageParams) { this.pageParams = pageParams; } + public Boolean getAllowNoMatch() { + return allowNoMatch; + } + + public void setAllowNoMatch(Boolean allowNoMatch) { + this.allowNoMatch = allowNoMatch; + } + @Override public Optional validate() { if (id == null) { @@ -59,7 +68,7 @@ public Optional validate() { @Override public int hashCode() { - return Objects.hash(id, pageParams); + return Objects.hash(id, pageParams, allowNoMatch); } @Override @@ -72,6 +81,8 @@ public boolean equals(Object obj) { return false; } GetDataFrameTransformStatsRequest other = (GetDataFrameTransformStatsRequest) obj; - return Objects.equals(id, other.id) && Objects.equals(pageParams, other.pageParams); + return Objects.equals(id, other.id) + && Objects.equals(pageParams, other.pageParams) + && Objects.equals(allowNoMatch, other.allowNoMatch); } } diff --git a/client/rest-high-level/src/main/java/org/elasticsearch/client/dataframe/PreviewDataFrameTransformResponse.java b/client/rest-high-level/src/main/java/org/elasticsearch/client/dataframe/PreviewDataFrameTransformResponse.java index 5d5f67dd65ed2..40e87b5768b5d 100644 --- a/client/rest-high-level/src/main/java/org/elasticsearch/client/dataframe/PreviewDataFrameTransformResponse.java +++ b/client/rest-high-level/src/main/java/org/elasticsearch/client/dataframe/PreviewDataFrameTransformResponse.java @@ -29,23 +29,32 @@ public class PreviewDataFrameTransformResponse { private static final String PREVIEW = "preview"; + private static final String MAPPINGS = "mappings"; @SuppressWarnings("unchecked") public static PreviewDataFrameTransformResponse fromXContent(final XContentParser parser) throws IOException { - Object previewDocs = parser.map().get(PREVIEW); - return new PreviewDataFrameTransformResponse((List>) previewDocs); + Map previewMap = parser.mapOrdered(); + Object previewDocs = previewMap.get(PREVIEW); + Object mappings = previewMap.get(MAPPINGS); + return new PreviewDataFrameTransformResponse((List>) previewDocs, (Map) mappings); } private List> docs; + private Map mappings; - public PreviewDataFrameTransformResponse(List> docs) { + public PreviewDataFrameTransformResponse(List> docs, Map mappings) { this.docs = docs; + this.mappings = mappings; } public List> getDocs() { return docs; } + public Map getMappings() { + return mappings; + } + @Override public boolean equals(Object obj) { if (obj == this) { @@ -57,12 +66,12 @@ public boolean equals(Object obj) { } PreviewDataFrameTransformResponse other = (PreviewDataFrameTransformResponse) obj; - return Objects.equals(other.docs, docs); + return Objects.equals(other.docs, docs) && Objects.equals(other.mappings, mappings); } @Override public int hashCode() { - return Objects.hashCode(docs); + return Objects.hash(docs, mappings); } } diff --git a/client/rest-high-level/src/main/java/org/elasticsearch/client/dataframe/StartDataFrameTransformResponse.java b/client/rest-high-level/src/main/java/org/elasticsearch/client/dataframe/StartDataFrameTransformResponse.java index f11ecd096c16e..9b358ffdfa151 100644 --- a/client/rest-high-level/src/main/java/org/elasticsearch/client/dataframe/StartDataFrameTransformResponse.java +++ b/client/rest-high-level/src/main/java/org/elasticsearch/client/dataframe/StartDataFrameTransformResponse.java @@ -30,21 +30,19 @@ public class StartDataFrameTransformResponse extends AcknowledgedTasksResponse { - private static final String STARTED = "started"; + private static final String ACKNOWLEDGED = "acknowledged"; private static final ConstructingObjectParser PARSER = - AcknowledgedTasksResponse.generateParser("start_data_frame_transform_response", StartDataFrameTransformResponse::new, STARTED); + AcknowledgedTasksResponse.generateParser("start_data_frame_transform_response", StartDataFrameTransformResponse::new, + ACKNOWLEDGED); public static StartDataFrameTransformResponse fromXContent(final XContentParser parser) throws IOException { return PARSER.parse(parser, null); } - public StartDataFrameTransformResponse(boolean started, @Nullable List taskFailures, + public StartDataFrameTransformResponse(boolean acknowledged, @Nullable List taskFailures, @Nullable List nodeFailures) { - super(started, taskFailures, nodeFailures); + super(acknowledged, taskFailures, nodeFailures); } - public boolean isStarted() { - return isAcknowledged(); - } } diff --git a/client/rest-high-level/src/main/java/org/elasticsearch/client/dataframe/StopDataFrameTransformRequest.java b/client/rest-high-level/src/main/java/org/elasticsearch/client/dataframe/StopDataFrameTransformRequest.java index 0bc690ad79076..4fb6164f2cca9 100644 --- a/client/rest-high-level/src/main/java/org/elasticsearch/client/dataframe/StopDataFrameTransformRequest.java +++ b/client/rest-high-level/src/main/java/org/elasticsearch/client/dataframe/StopDataFrameTransformRequest.java @@ -31,6 +31,7 @@ public class StopDataFrameTransformRequest implements Validatable { private final String id; private Boolean waitForCompletion; private TimeValue timeout; + private Boolean allowNoMatch; public StopDataFrameTransformRequest(String id) { this.id = id; @@ -64,6 +65,14 @@ public TimeValue getTimeout() { return timeout; } + public Boolean getAllowNoMatch() { + return allowNoMatch; + } + + public void setAllowNoMatch(Boolean allowNoMatch) { + this.allowNoMatch = allowNoMatch; + } + @Override public Optional validate() { if (id == null) { @@ -77,7 +86,7 @@ public Optional validate() { @Override public int hashCode() { - return Objects.hash(id, waitForCompletion, timeout); + return Objects.hash(id, waitForCompletion, timeout, allowNoMatch); } @Override @@ -92,7 +101,8 @@ public boolean equals(Object obj) { StopDataFrameTransformRequest other = (StopDataFrameTransformRequest) obj; return Objects.equals(this.id, other.id) && Objects.equals(this.waitForCompletion, other.waitForCompletion) - && Objects.equals(this.timeout, other.timeout); + && Objects.equals(this.timeout, other.timeout) + && Objects.equals(this.allowNoMatch, other.allowNoMatch); } } diff --git a/client/rest-high-level/src/main/java/org/elasticsearch/client/dataframe/StopDataFrameTransformResponse.java b/client/rest-high-level/src/main/java/org/elasticsearch/client/dataframe/StopDataFrameTransformResponse.java index 3224dfb4703e5..6d32474f70c1a 100644 --- a/client/rest-high-level/src/main/java/org/elasticsearch/client/dataframe/StopDataFrameTransformResponse.java +++ b/client/rest-high-level/src/main/java/org/elasticsearch/client/dataframe/StopDataFrameTransformResponse.java @@ -30,21 +30,18 @@ public class StopDataFrameTransformResponse extends AcknowledgedTasksResponse { - private static final String STOPPED = "stopped"; + private static final String ACKNOWLEDGED = "acknowledged"; - private static final ConstructingObjectParser PARSER = - AcknowledgedTasksResponse.generateParser("stop_data_frame_transform_response", StopDataFrameTransformResponse::new, STOPPED); + private static final ConstructingObjectParser PARSER = AcknowledgedTasksResponse + .generateParser("stop_data_frame_transform_response", StopDataFrameTransformResponse::new, ACKNOWLEDGED); public static StopDataFrameTransformResponse fromXContent(final XContentParser parser) throws IOException { return PARSER.parse(parser, null); } - public StopDataFrameTransformResponse(boolean stopped, @Nullable List taskFailures, + public StopDataFrameTransformResponse(boolean acknowledged, @Nullable List taskFailures, @Nullable List nodeFailures) { - super(stopped, taskFailures, nodeFailures); + super(acknowledged, taskFailures, nodeFailures); } - public boolean isStopped() { - return isAcknowledged(); - } } diff --git a/client/rest-high-level/src/main/java/org/elasticsearch/client/dataframe/transforms/DataFrameTransformConfig.java b/client/rest-high-level/src/main/java/org/elasticsearch/client/dataframe/transforms/DataFrameTransformConfig.java index 8465ae8342827..355e3ad9bbc0f 100644 --- a/client/rest-high-level/src/main/java/org/elasticsearch/client/dataframe/transforms/DataFrameTransformConfig.java +++ b/client/rest-high-level/src/main/java/org/elasticsearch/client/dataframe/transforms/DataFrameTransformConfig.java @@ -19,16 +19,21 @@ package org.elasticsearch.client.dataframe.transforms; +import org.elasticsearch.Version; import org.elasticsearch.client.dataframe.transforms.pivot.PivotConfig; +import org.elasticsearch.client.dataframe.transforms.util.TimeUtil; import org.elasticsearch.common.Nullable; import org.elasticsearch.common.ParseField; import org.elasticsearch.common.Strings; import org.elasticsearch.common.xcontent.ConstructingObjectParser; +import org.elasticsearch.common.xcontent.ObjectParser; import org.elasticsearch.common.xcontent.ToXContentObject; import org.elasticsearch.common.xcontent.XContentBuilder; import org.elasticsearch.common.xcontent.XContentParser; +import org.elasticsearch.common.xcontent.XContentParserUtils; import java.io.IOException; +import java.time.Instant; import java.util.Objects; import static org.elasticsearch.common.xcontent.ConstructingObjectParser.constructorArg; @@ -40,14 +45,20 @@ public class DataFrameTransformConfig implements ToXContentObject { public static final ParseField SOURCE = new ParseField("source"); public static final ParseField DEST = new ParseField("dest"); public static final ParseField DESCRIPTION = new ParseField("description"); + public static final ParseField SYNC = new ParseField("sync"); + public static final ParseField VERSION = new ParseField("version"); + public static final ParseField CREATE_TIME = new ParseField("create_time"); // types of transforms public static final ParseField PIVOT_TRANSFORM = new ParseField("pivot"); private final String id; private final SourceConfig source; private final DestConfig dest; + private final SyncConfig syncConfig; private final PivotConfig pivotConfig; private final String description; + private final Version transformVersion; + private final Instant createTime; public static final ConstructingObjectParser PARSER = new ConstructingObjectParser<>("data_frame_transform", true, @@ -55,19 +66,42 @@ public class DataFrameTransformConfig implements ToXContentObject { String id = (String) args[0]; SourceConfig source = (SourceConfig) args[1]; DestConfig dest = (DestConfig) args[2]; - PivotConfig pivotConfig = (PivotConfig) args[3]; - String description = (String)args[4]; - return new DataFrameTransformConfig(id, source, dest, pivotConfig, description); + SyncConfig syncConfig = (SyncConfig) args[3]; + PivotConfig pivotConfig = (PivotConfig) args[4]; + String description = (String)args[5]; + Instant createTime = (Instant)args[6]; + String transformVersion = (String)args[7]; + return new DataFrameTransformConfig(id, + source, + dest, + syncConfig, + pivotConfig, + description, + createTime, + transformVersion); }); static { PARSER.declareString(constructorArg(), ID); PARSER.declareObject(constructorArg(), (p, c) -> SourceConfig.PARSER.apply(p, null), SOURCE); PARSER.declareObject(constructorArg(), (p, c) -> DestConfig.PARSER.apply(p, null), DEST); + PARSER.declareObject(optionalConstructorArg(), (p, c) -> parseSyncConfig(p), SYNC); PARSER.declareObject(optionalConstructorArg(), (p, c) -> PivotConfig.fromXContent(p), PIVOT_TRANSFORM); PARSER.declareString(optionalConstructorArg(), DESCRIPTION); + PARSER.declareField(optionalConstructorArg(), + p -> TimeUtil.parseTimeFieldToInstant(p, CREATE_TIME.getPreferredName()), CREATE_TIME, ObjectParser.ValueType.VALUE); + PARSER.declareString(optionalConstructorArg(), VERSION); } + private static SyncConfig parseSyncConfig(XContentParser parser) throws IOException { + XContentParserUtils.ensureExpectedToken(XContentParser.Token.START_OBJECT, parser.currentToken(), parser::getTokenLocation); + XContentParserUtils.ensureExpectedToken(XContentParser.Token.FIELD_NAME, parser.nextToken(), parser::getTokenLocation); + SyncConfig syncConfig = parser.namedObject(SyncConfig.class, parser.currentName(), true); + XContentParserUtils.ensureExpectedToken(XContentParser.Token.END_OBJECT, parser.nextToken(), parser::getTokenLocation); + return syncConfig; + } + + public static DataFrameTransformConfig fromXContent(final XContentParser parser) { return PARSER.apply(parser, null); } @@ -84,19 +118,25 @@ public static DataFrameTransformConfig fromXContent(final XContentParser parser) * @return A DataFrameTransformConfig to preview, NOTE it will have a {@code null} id, destination and index. */ public static DataFrameTransformConfig forPreview(final SourceConfig source, final PivotConfig pivotConfig) { - return new DataFrameTransformConfig(null, source, null, pivotConfig, null); + return new DataFrameTransformConfig(null, source, null, null, pivotConfig, null, null, null); } DataFrameTransformConfig(final String id, final SourceConfig source, final DestConfig dest, + final SyncConfig syncConfig, final PivotConfig pivotConfig, - final String description) { + final String description, + final Instant createTime, + final String version) { this.id = id; this.source = source; this.dest = dest; + this.syncConfig = syncConfig; this.pivotConfig = pivotConfig; this.description = description; + this.createTime = createTime == null ? null : Instant.ofEpochMilli(createTime.toEpochMilli()); + this.transformVersion = version == null ? null : Version.fromString(version); } public String getId() { @@ -111,10 +151,22 @@ public DestConfig getDestination() { return dest; } + public SyncConfig getSyncConfig() { + return syncConfig; + } + public PivotConfig getPivotConfig() { return pivotConfig; } + public Version getVersion() { + return transformVersion; + } + + public Instant getCreateTime() { + return createTime; + } + @Nullable public String getDescription() { return description; @@ -132,12 +184,23 @@ public XContentBuilder toXContent(final XContentBuilder builder, final Params pa if (dest != null) { builder.field(DEST.getPreferredName(), dest); } + if (syncConfig != null) { + builder.startObject(SYNC.getPreferredName()); + builder.field(syncConfig.getName(), syncConfig); + builder.endObject(); + } if (pivotConfig != null) { builder.field(PIVOT_TRANSFORM.getPreferredName(), pivotConfig); } if (description != null) { builder.field(DESCRIPTION.getPreferredName(), description); } + if (createTime != null) { + builder.timeField(CREATE_TIME.getPreferredName(), CREATE_TIME.getPreferredName() + "_string", createTime.toEpochMilli()); + } + if (transformVersion != null) { + builder.field(VERSION.getPreferredName(), transformVersion); + } builder.endObject(); return builder; } @@ -155,15 +218,18 @@ public boolean equals(Object other) { final DataFrameTransformConfig that = (DataFrameTransformConfig) other; return Objects.equals(this.id, that.id) - && Objects.equals(this.source, that.source) - && Objects.equals(this.dest, that.dest) - && Objects.equals(this.description, that.description) - && Objects.equals(this.pivotConfig, that.pivotConfig); + && Objects.equals(this.source, that.source) + && Objects.equals(this.dest, that.dest) + && Objects.equals(this.description, that.description) + && Objects.equals(this.syncConfig, that.syncConfig) + && Objects.equals(this.transformVersion, that.transformVersion) + && Objects.equals(this.createTime, that.createTime) + && Objects.equals(this.pivotConfig, that.pivotConfig); } @Override public int hashCode() { - return Objects.hash(id, source, dest, pivotConfig, description); + return Objects.hash(id, source, dest, syncConfig, pivotConfig, description); } @Override @@ -180,6 +246,7 @@ public static class Builder { private String id; private SourceConfig source; private DestConfig dest; + private SyncConfig syncConfig; private PivotConfig pivotConfig; private String description; @@ -198,6 +265,11 @@ public Builder setDest(DestConfig dest) { return this; } + public Builder setSyncConfig(SyncConfig syncConfig) { + this.syncConfig = syncConfig; + return this; + } + public Builder setPivotConfig(PivotConfig pivotConfig) { this.pivotConfig = pivotConfig; return this; @@ -209,7 +281,7 @@ public Builder setDescription(String description) { } public DataFrameTransformConfig build() { - return new DataFrameTransformConfig(id, source, dest, pivotConfig, description); + return new DataFrameTransformConfig(id, source, dest, syncConfig, pivotConfig, description, null, null); } } } diff --git a/client/rest-high-level/src/main/java/org/elasticsearch/client/dataframe/transforms/DataFrameTransformState.java b/client/rest-high-level/src/main/java/org/elasticsearch/client/dataframe/transforms/DataFrameTransformState.java index 352cbfb67fcde..186c67bf42ce2 100644 --- a/client/rest-high-level/src/main/java/org/elasticsearch/client/dataframe/transforms/DataFrameTransformState.java +++ b/client/rest-high-level/src/main/java/org/elasticsearch/client/dataframe/transforms/DataFrameTransformState.java @@ -43,6 +43,7 @@ public class DataFrameTransformState { private static final ParseField CHECKPOINT = new ParseField("checkpoint"); private static final ParseField REASON = new ParseField("reason"); private static final ParseField PROGRESS = new ParseField("progress"); + private static final ParseField NODE = new ParseField("node"); @SuppressWarnings("unchecked") public static final ConstructingObjectParser PARSER = @@ -52,7 +53,8 @@ public class DataFrameTransformState { (Map) args[2], (long) args[3], (String) args[4], - (DataFrameTransformProgress) args[5])); + (DataFrameTransformProgress) args[5], + (NodeAttributes) args[6])); static { PARSER.declareField(constructorArg(), p -> DataFrameTransformTaskState.fromString(p.text()), TASK_STATE, ValueType.STRING); @@ -61,6 +63,7 @@ public class DataFrameTransformState { PARSER.declareLong(ConstructingObjectParser.optionalConstructorArg(), CHECKPOINT); PARSER.declareString(ConstructingObjectParser.optionalConstructorArg(), REASON); PARSER.declareField(optionalConstructorArg(), DataFrameTransformProgress::fromXContent, PROGRESS, ValueType.OBJECT); + PARSER.declareField(optionalConstructorArg(), NodeAttributes.PARSER::apply, NODE, ValueType.OBJECT); } public static DataFrameTransformState fromXContent(XContentParser parser) throws IOException { @@ -73,19 +76,22 @@ public static DataFrameTransformState fromXContent(XContentParser parser) throws private final Map currentPosition; private final String reason; private final DataFrameTransformProgress progress; + private final NodeAttributes node; public DataFrameTransformState(DataFrameTransformTaskState taskState, IndexerState indexerState, @Nullable Map position, long checkpoint, @Nullable String reason, - @Nullable DataFrameTransformProgress progress) { + @Nullable DataFrameTransformProgress progress, + @Nullable NodeAttributes node) { this.taskState = taskState; this.indexerState = indexerState; this.currentPosition = position == null ? null : Collections.unmodifiableMap(new LinkedHashMap<>(position)); this.checkpoint = checkpoint; this.reason = reason; this.progress = progress; + this.node = node; } public IndexerState getIndexerState() { @@ -115,6 +121,11 @@ public DataFrameTransformProgress getProgress() { return progress; } + @Nullable + public NodeAttributes getNode() { + return node; + } + @Override public boolean equals(Object other) { if (this == other) { @@ -132,12 +143,13 @@ public boolean equals(Object other) { Objects.equals(this.currentPosition, that.currentPosition) && Objects.equals(this.progress, that.progress) && this.checkpoint == that.checkpoint && + Objects.equals(this.node, that.node) && Objects.equals(this.reason, that.reason); } @Override public int hashCode() { - return Objects.hash(taskState, indexerState, currentPosition, checkpoint, reason, progress); + return Objects.hash(taskState, indexerState, currentPosition, checkpoint, reason, progress, node); } } diff --git a/client/rest-high-level/src/main/java/org/elasticsearch/client/dataframe/transforms/DestConfig.java b/client/rest-high-level/src/main/java/org/elasticsearch/client/dataframe/transforms/DestConfig.java index 5e81a368f66b4..f808fa867209a 100644 --- a/client/rest-high-level/src/main/java/org/elasticsearch/client/dataframe/transforms/DestConfig.java +++ b/client/rest-high-level/src/main/java/org/elasticsearch/client/dataframe/transforms/DestConfig.java @@ -28,6 +28,7 @@ import java.util.Objects; import static org.elasticsearch.common.xcontent.ConstructingObjectParser.constructorArg; +import static org.elasticsearch.common.xcontent.ConstructingObjectParser.optionalConstructorArg; /** * Configuration containing the destination index for the {@link DataFrameTransformConfig} @@ -35,29 +36,40 @@ public class DestConfig implements ToXContentObject { public static final ParseField INDEX = new ParseField("index"); + public static final ParseField PIPELINE = new ParseField("pipeline"); public static final ConstructingObjectParser PARSER = new ConstructingObjectParser<>("data_frame_config_dest", true, - args -> new DestConfig((String)args[0])); + args -> new DestConfig((String)args[0], (String)args[1])); static { PARSER.declareString(constructorArg(), INDEX); + PARSER.declareString(optionalConstructorArg(), PIPELINE); } private final String index; + private final String pipeline; - public DestConfig(String index) { + DestConfig(String index, String pipeline) { this.index = Objects.requireNonNull(index, INDEX.getPreferredName()); + this.pipeline = pipeline; } public String getIndex() { return index; } + public String getPipeline() { + return pipeline; + } + @Override public XContentBuilder toXContent(XContentBuilder builder, Params params) throws IOException { builder.startObject(); builder.field(INDEX.getPreferredName(), index); + if (pipeline != null) { + builder.field(PIPELINE.getPreferredName(), pipeline); + } builder.endObject(); return builder; } @@ -72,11 +84,45 @@ public boolean equals(Object other) { } DestConfig that = (DestConfig) other; - return Objects.equals(index, that.index); + return Objects.equals(index, that.index) && + Objects.equals(pipeline, that.pipeline); } @Override public int hashCode(){ - return Objects.hash(index); + return Objects.hash(index, pipeline); + } + + public static Builder builder() { + return new Builder(); + } + + public static class Builder { + private String index; + private String pipeline; + + /** + * Sets which index to which to write the data + * @param index where to write the data + * @return The {@link Builder} with index set + */ + public Builder setIndex(String index) { + this.index = Objects.requireNonNull(index, INDEX.getPreferredName()); + return this; + } + + /** + * Sets the pipeline through which the indexed documents should be processed + * @param pipeline The pipeline ID + * @return The {@link Builder} with pipeline set + */ + public Builder setPipeline(String pipeline) { + this.pipeline = pipeline; + return this; + } + + public DestConfig build() { + return new DestConfig(index, pipeline); + } } } diff --git a/client/rest-high-level/src/main/java/org/elasticsearch/client/dataframe/transforms/NodeAttributes.java b/client/rest-high-level/src/main/java/org/elasticsearch/client/dataframe/transforms/NodeAttributes.java new file mode 100644 index 0000000000000..85c2b9644c2fd --- /dev/null +++ b/client/rest-high-level/src/main/java/org/elasticsearch/client/dataframe/transforms/NodeAttributes.java @@ -0,0 +1,156 @@ +/* + * 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.client.dataframe.transforms; + +import org.elasticsearch.common.ParseField; +import org.elasticsearch.common.Strings; +import org.elasticsearch.common.xcontent.ConstructingObjectParser; +import org.elasticsearch.common.xcontent.ObjectParser; +import org.elasticsearch.common.xcontent.ToXContentObject; +import org.elasticsearch.common.xcontent.XContentBuilder; + +import java.io.IOException; +import java.util.Collections; +import java.util.Map; +import java.util.Objects; + +/** + * A Pojo class containing an Elastic Node's attributes + */ +public class NodeAttributes implements ToXContentObject { + + public static final ParseField ID = new ParseField("id"); + public static final ParseField NAME = new ParseField("name"); + public static final ParseField EPHEMERAL_ID = new ParseField("ephemeral_id"); + public static final ParseField TRANSPORT_ADDRESS = new ParseField("transport_address"); + public static final ParseField ATTRIBUTES = new ParseField("attributes"); + + @SuppressWarnings("unchecked") + public static final ConstructingObjectParser PARSER = + new ConstructingObjectParser<>("node", true, + (a) -> { + int i = 0; + String id = (String) a[i++]; + String name = (String) a[i++]; + String ephemeralId = (String) a[i++]; + String transportAddress = (String) a[i++]; + Map attributes = (Map) a[i]; + return new NodeAttributes(id, name, ephemeralId, transportAddress, attributes); + }); + + static { + PARSER.declareString(ConstructingObjectParser.constructorArg(), ID); + PARSER.declareString(ConstructingObjectParser.constructorArg(), NAME); + PARSER.declareString(ConstructingObjectParser.constructorArg(), EPHEMERAL_ID); + PARSER.declareString(ConstructingObjectParser.constructorArg(), TRANSPORT_ADDRESS); + PARSER.declareField(ConstructingObjectParser.constructorArg(), + (p, c) -> p.mapStrings(), + ATTRIBUTES, + ObjectParser.ValueType.OBJECT); + } + + private final String id; + private final String name; + private final String ephemeralId; + private final String transportAddress; + private final Map attributes; + + public NodeAttributes(String id, String name, String ephemeralId, String transportAddress, Map attributes) { + this.id = id; + this.name = name; + this.ephemeralId = ephemeralId; + this.transportAddress = transportAddress; + this.attributes = Collections.unmodifiableMap(attributes); + } + + /** + * The unique identifier of the node. + */ + public String getId() { + return id; + } + + /** + * The node name. + */ + public String getName() { + return name; + } + + /** + * The ephemeral id of the node. + */ + public String getEphemeralId() { + return ephemeralId; + } + + /** + * The host and port where transport HTTP connections are accepted. + */ + public String getTransportAddress() { + return transportAddress; + } + + /** + * Additional attributes related to this node + */ + public Map getAttributes() { + return attributes; + } + + @Override + public XContentBuilder toXContent(XContentBuilder builder, Params params) throws IOException { + builder.startObject(); + builder.field(ID.getPreferredName(), id); + builder.field(NAME.getPreferredName(), name); + builder.field(EPHEMERAL_ID.getPreferredName(), ephemeralId); + builder.field(TRANSPORT_ADDRESS.getPreferredName(), transportAddress); + builder.field(ATTRIBUTES.getPreferredName(), attributes); + builder.endObject(); + return builder; + } + + @Override + public int hashCode() { + return Objects.hash(id, name, ephemeralId, transportAddress, attributes); + } + + @Override + public boolean equals(Object other) { + if (this == other) { + return true; + } + + if (other == null || getClass() != other.getClass()) { + return false; + } + + NodeAttributes that = (NodeAttributes) other; + return Objects.equals(id, that.id) && + Objects.equals(name, that.name) && + Objects.equals(ephemeralId, that.ephemeralId) && + Objects.equals(transportAddress, that.transportAddress) && + Objects.equals(attributes, that.attributes); + } + + @Override + public String toString() { + return Strings.toString(this); + } +} diff --git a/client/rest-high-level/src/main/java/org/elasticsearch/client/dataframe/transforms/SyncConfig.java b/client/rest-high-level/src/main/java/org/elasticsearch/client/dataframe/transforms/SyncConfig.java new file mode 100644 index 0000000000000..3ead35d0a491a --- /dev/null +++ b/client/rest-high-level/src/main/java/org/elasticsearch/client/dataframe/transforms/SyncConfig.java @@ -0,0 +1,30 @@ +/* + * 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.client.dataframe.transforms; + +import org.elasticsearch.common.xcontent.ToXContentObject; + +public interface SyncConfig extends ToXContentObject { + + /** + * Returns the name of the writeable object + */ + String getName(); +} diff --git a/client/rest-high-level/src/main/java/org/elasticsearch/client/dataframe/transforms/TimeSyncConfig.java b/client/rest-high-level/src/main/java/org/elasticsearch/client/dataframe/transforms/TimeSyncConfig.java new file mode 100644 index 0000000000000..797ca3f896138 --- /dev/null +++ b/client/rest-high-level/src/main/java/org/elasticsearch/client/dataframe/transforms/TimeSyncConfig.java @@ -0,0 +1,108 @@ +/* + * 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.client.dataframe.transforms; + +import org.elasticsearch.common.ParseField; +import org.elasticsearch.common.unit.TimeValue; +import org.elasticsearch.common.xcontent.ConstructingObjectParser; +import org.elasticsearch.common.xcontent.ObjectParser; +import org.elasticsearch.common.xcontent.XContentBuilder; +import org.elasticsearch.common.xcontent.XContentParser; + +import java.io.IOException; +import java.util.Objects; + +import static org.elasticsearch.common.xcontent.ConstructingObjectParser.constructorArg; +import static org.elasticsearch.common.xcontent.ConstructingObjectParser.optionalConstructorArg; + +public class TimeSyncConfig implements SyncConfig { + + public static final String NAME = "time"; + + private static final ParseField FIELD = new ParseField("field"); + private static final ParseField DELAY = new ParseField("delay"); + + private final String field; + private final TimeValue delay; + + private static final ConstructingObjectParser PARSER = new ConstructingObjectParser<>("time_sync_config", true, + args -> new TimeSyncConfig((String) args[0], args[1] != null ? (TimeValue) args[1] : TimeValue.ZERO)); + + static { + PARSER.declareString(constructorArg(), FIELD); + PARSER.declareField(optionalConstructorArg(), (p, c) -> TimeValue.parseTimeValue(p.textOrNull(), DELAY.getPreferredName()), DELAY, + ObjectParser.ValueType.STRING_OR_NULL); + } + + public static TimeSyncConfig fromXContent(XContentParser parser) { + return PARSER.apply(parser, null); + } + + public TimeSyncConfig(String field, TimeValue delay) { + this.field = field; + this.delay = delay; + } + + @Override + public XContentBuilder toXContent(XContentBuilder builder, Params params) throws IOException { + builder.startObject(); + builder.field(FIELD.getPreferredName(), field); + if (delay.duration() > 0) { + builder.field(DELAY.getPreferredName(), delay.getStringRep()); + } + builder.endObject(); + return builder; + } + + public String getField() { + return field; + } + + public TimeValue getDelay() { + return delay; + } + + @Override + public boolean equals(Object other) { + if (this == other) { + return true; + } + + if (other == null || getClass() != other.getClass()) { + return false; + } + + final TimeSyncConfig that = (TimeSyncConfig) other; + + return Objects.equals(this.field, that.field) + && Objects.equals(this.delay, that.delay); + } + + @Override + public int hashCode() { + return Objects.hash(field, delay); + } + + @Override + public String getName() { + return NAME; + } + +} diff --git a/client/rest-high-level/src/main/java/org/elasticsearch/client/dataframe/transforms/pivot/DateHistogramGroupSource.java b/client/rest-high-level/src/main/java/org/elasticsearch/client/dataframe/transforms/pivot/DateHistogramGroupSource.java index d880bfd82140b..c8fb885896d9c 100644 --- a/client/rest-high-level/src/main/java/org/elasticsearch/client/dataframe/transforms/pivot/DateHistogramGroupSource.java +++ b/client/rest-high-level/src/main/java/org/elasticsearch/client/dataframe/transforms/pivot/DateHistogramGroupSource.java @@ -45,7 +45,6 @@ public class DateHistogramGroupSource extends SingleGroupSource implements ToXContentObject { private static final ParseField TIME_ZONE = new ParseField("time_zone"); - private static final ParseField FORMAT = new ParseField("format"); // From DateHistogramAggregationBuilder in core, transplanted and modified to a set // so we don't need to import a dependency on the class @@ -195,8 +194,7 @@ public int hashCode() { } ZoneId zoneId = (ZoneId) args[3]; - String format = (String) args[4]; - return new DateHistogramGroupSource(field, interval, format, zoneId); + return new DateHistogramGroupSource(field, interval, zoneId); }); static { @@ -212,8 +210,6 @@ public int hashCode() { return ZoneOffset.ofHours(p.intValue()); } }, TIME_ZONE, ObjectParser.ValueType.LONG); - - PARSER.declareString(optionalConstructorArg(), FORMAT); } public static DateHistogramGroupSource fromXContent(final XContentParser parser) { @@ -221,13 +217,11 @@ public static DateHistogramGroupSource fromXContent(final XContentParser parser) } private final Interval interval; - private final String format; private final ZoneId timeZone; - DateHistogramGroupSource(String field, Interval interval, String format, ZoneId timeZone) { + DateHistogramGroupSource(String field, Interval interval, ZoneId timeZone) { super(field); this.interval = interval; - this.format = format; this.timeZone = timeZone; } @@ -240,10 +234,6 @@ public Interval getInterval() { return interval; } - public String getFormat() { - return format; - } - public ZoneId getTimeZone() { return timeZone; } @@ -258,9 +248,6 @@ public XContentBuilder toXContent(XContentBuilder builder, Params params) throws if (timeZone != null) { builder.field(TIME_ZONE.getPreferredName(), timeZone.toString()); } - if (format != null) { - builder.field(FORMAT.getPreferredName(), format); - } builder.endObject(); return builder; } @@ -279,13 +266,12 @@ public boolean equals(Object other) { return Objects.equals(this.field, that.field) && Objects.equals(this.interval, that.interval) && - Objects.equals(this.timeZone, that.timeZone) && - Objects.equals(this.format, that.format); + Objects.equals(this.timeZone, that.timeZone); } @Override public int hashCode() { - return Objects.hash(field, interval, timeZone, format); + return Objects.hash(field, interval, timeZone); } public static Builder builder() { @@ -296,7 +282,6 @@ public static class Builder { private String field; private Interval interval; - private String format; private ZoneId timeZone; /** @@ -319,16 +304,6 @@ public Builder setInterval(Interval interval) { return this; } - /** - * Set the optional String formatting for the time interval. - * @param format The format of the output for the time interval key - * @return The {@link Builder} with the format set. - */ - public Builder setFormat(String format) { - this.format = format; - return this; - } - /** * Sets the time zone to use for this aggregation * @param timeZone The zoneId for the timeZone @@ -340,7 +315,7 @@ public Builder setTimeZone(ZoneId timeZone) { } public DateHistogramGroupSource build() { - return new DateHistogramGroupSource(field, interval, format, timeZone); + return new DateHistogramGroupSource(field, interval, timeZone); } } } diff --git a/client/rest-high-level/src/main/java/org/elasticsearch/client/dataframe/transforms/util/TimeUtil.java b/client/rest-high-level/src/main/java/org/elasticsearch/client/dataframe/transforms/util/TimeUtil.java new file mode 100644 index 0000000000000..2470c3f7a4aa8 --- /dev/null +++ b/client/rest-high-level/src/main/java/org/elasticsearch/client/dataframe/transforms/util/TimeUtil.java @@ -0,0 +1,59 @@ +/* + * 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.client.dataframe.transforms.util; + +import org.elasticsearch.common.time.DateFormatters; +import org.elasticsearch.common.xcontent.XContentParser; + +import java.io.IOException; +import java.time.Instant; +import java.time.format.DateTimeFormatter; +import java.util.Date; + +public final class TimeUtil { + + /** + * Parse out a Date object given the current parser and field name. + * + * @param parser current XContentParser + * @param fieldName the field's preferred name (utilized in exception) + * @return parsed Date object + * @throws IOException from XContentParser + */ + public static Date parseTimeField(XContentParser parser, String fieldName) throws IOException { + if (parser.currentToken() == XContentParser.Token.VALUE_NUMBER) { + return new Date(parser.longValue()); + } else if (parser.currentToken() == XContentParser.Token.VALUE_STRING) { + return new Date(DateFormatters.from(DateTimeFormatter.ISO_INSTANT.parse(parser.text())).toInstant().toEpochMilli()); + } + throw new IllegalArgumentException( + "unexpected token [" + parser.currentToken() + "] for [" + fieldName + "]"); + } + + public static Instant parseTimeFieldToInstant(XContentParser parser, String fieldName) throws IOException { + if (parser.currentToken() == XContentParser.Token.VALUE_NUMBER) { + return Instant.ofEpochMilli(parser.longValue()); + } else if (parser.currentToken() == XContentParser.Token.VALUE_STRING) { + return DateFormatters.from(DateTimeFormatter.ISO_INSTANT.parse(parser.text())).toInstant(); + } + throw new IllegalArgumentException( + "unexpected token [" + parser.currentToken() + "] for [" + fieldName + "]"); + } + +} diff --git a/client/rest-high-level/src/main/java/org/elasticsearch/client/indices/AnalyzeRequest.java b/client/rest-high-level/src/main/java/org/elasticsearch/client/indices/AnalyzeRequest.java new file mode 100644 index 0000000000000..1aed59227e8bd --- /dev/null +++ b/client/rest-high-level/src/main/java/org/elasticsearch/client/indices/AnalyzeRequest.java @@ -0,0 +1,343 @@ +/* + * 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.client.indices; + +import org.elasticsearch.client.Validatable; +import org.elasticsearch.common.Strings; +import org.elasticsearch.common.settings.Settings; +import org.elasticsearch.common.xcontent.ToXContentFragment; +import org.elasticsearch.common.xcontent.ToXContentObject; +import org.elasticsearch.common.xcontent.XContentBuilder; +import org.elasticsearch.common.xcontent.XContentFactory; +import org.elasticsearch.common.xcontent.XContentType; + +import java.io.IOException; +import java.util.ArrayList; +import java.util.List; +import java.util.Map; +import java.util.Objects; + +/** + * A request to analyze text + */ +public class AnalyzeRequest implements Validatable, ToXContentObject { + + private String index; + + private String[] text; + + private String analyzer; + + private NameOrDefinition tokenizer; + + private final List tokenFilters = new ArrayList<>(); + + private final List charFilters = new ArrayList<>(); + + private String field; + + private boolean explain = false; + + private String[] attributes = Strings.EMPTY_ARRAY; + + private String normalizer; + + /** + * Analyzes text using a global analyzer + */ + public static AnalyzeRequest withGlobalAnalyzer(String analyzer, String... text) { + return new AnalyzeRequest(null, analyzer, null, null, text); + } + + /** + * Analyzes text using a custom analyzer built from global components + */ + public static CustomAnalyzerBuilder buildCustomAnalyzer(String tokenizer) { + return new CustomAnalyzerBuilder(null, new NameOrDefinition(tokenizer)); + } + + /** + * Analyzes text using a custom analyzer built from global components + */ + public static CustomAnalyzerBuilder buildCustomAnalyzer(Map tokenizerSettings) { + return new CustomAnalyzerBuilder(null, new NameOrDefinition(tokenizerSettings)); + } + + /** + * Analyzes text using a custom analyzer built from components defined on an index + */ + public static CustomAnalyzerBuilder buildCustomAnalyzer(String index, String tokenizer) { + return new CustomAnalyzerBuilder(index, new NameOrDefinition(tokenizer)); + } + + /** + * Analyzes text using a custom analyzer built from components defined on an index + */ + public static CustomAnalyzerBuilder buildCustomAnalyzer(String index, Map tokenizerSettings) { + return new CustomAnalyzerBuilder(index, new NameOrDefinition(tokenizerSettings)); + } + + /** + * Analyzes text using a named analyzer on an index + */ + public static AnalyzeRequest withIndexAnalyzer(String index, String analyzer, String... text) { + return new AnalyzeRequest(index, analyzer, null, null, text); + } + + /** + * Analyzes text using the analyzer defined on a specific field within an index + */ + public static AnalyzeRequest withField(String index, String field, String... text) { + return new AnalyzeRequest(index, null, null, field, text); + } + + /** + * Analyzes text using a named normalizer on an index + */ + public static AnalyzeRequest withNormalizer(String index, String normalizer, String... text) { + return new AnalyzeRequest(index, null, normalizer, null, text); + } + + /** + * Analyzes text using a custom normalizer built from global components + */ + public static CustomAnalyzerBuilder buildCustomNormalizer() { + return new CustomAnalyzerBuilder(null, null); + } + + /** + * Analyzes text using a custom normalizer built from components defined on an index + */ + public static CustomAnalyzerBuilder buildCustomNormalizer(String index) { + return new CustomAnalyzerBuilder(index, null); + } + + /** + * Helper class to build custom analyzer definitions + */ + public static class CustomAnalyzerBuilder { + + final NameOrDefinition tokenizer; + final String index; + List charFilters = new ArrayList<>(); + List tokenFilters = new ArrayList<>(); + + CustomAnalyzerBuilder(String index, NameOrDefinition tokenizer) { + this.tokenizer = tokenizer; + this.index = index; + } + + public CustomAnalyzerBuilder addCharFilter(String name) { + charFilters.add(new NameOrDefinition(name)); + return this; + } + + public CustomAnalyzerBuilder addCharFilter(Map settings) { + charFilters.add(new NameOrDefinition(settings)); + return this; + } + + public CustomAnalyzerBuilder addTokenFilter(String name) { + tokenFilters.add(new NameOrDefinition(name)); + return this; + } + + public CustomAnalyzerBuilder addTokenFilter(Map settings) { + tokenFilters.add(new NameOrDefinition(settings)); + return this; + } + + public AnalyzeRequest build(String... text) { + return new AnalyzeRequest(index, tokenizer, charFilters, tokenFilters, text); + } + } + + private AnalyzeRequest(String index, String analyzer, String normalizer, String field, String... text) { + this.index = index; + this.analyzer = analyzer; + this.normalizer = normalizer; + this.field = field; + this.text = text; + } + + private AnalyzeRequest(String index, NameOrDefinition tokenizer, List charFilters, + List tokenFilters, String... text) { + this.index = index; + this.analyzer = null; + this.normalizer = null; + this.field = null; + this.tokenizer = tokenizer; + this.charFilters.addAll(charFilters); + this.tokenFilters.addAll(tokenFilters); + this.text = text; + } + + static class NameOrDefinition implements ToXContentFragment { + // exactly one of these two members is not null + public final String name; + public final Settings definition; + + NameOrDefinition(String name) { + this.name = Objects.requireNonNull(name); + this.definition = null; + } + + NameOrDefinition(Settings settings) { + this.name = null; + this.definition = Objects.requireNonNull(settings); + } + + NameOrDefinition(Map definition) { + this.name = null; + Objects.requireNonNull(definition); + try { + XContentBuilder builder = XContentFactory.contentBuilder(XContentType.JSON); + builder.map(definition); + this.definition = Settings.builder().loadFromSource(Strings.toString(builder), builder.contentType()).build(); + } catch (IOException e) { + throw new IllegalArgumentException("Failed to parse [" + definition + "]", e); + } + } + + @Override + public XContentBuilder toXContent(XContentBuilder builder, Params params) throws IOException { + if (definition == null) { + return builder.value(name); + } + builder.startObject(); + definition.toXContent(builder, params); + builder.endObject(); + return builder; + } + + } + + /** + * Returns the index that the request should be executed against, or {@code null} if + * no index is specified + */ + public String index() { + return this.index; + } + + /** + * Returns the text to be analyzed + */ + public String[] text() { + return this.text; + } + + /** + * Returns the named analyzer used for analysis, if defined + */ + public String analyzer() { + return this.analyzer; + } + + /** + * Returns the named tokenizer used for analysis, if defined + */ + public String normalizer() { + return this.normalizer; + } + + /** + * Returns a custom Tokenizer used for analysis, if defined + */ + public NameOrDefinition tokenizer() { + return this.tokenizer; + } + + /** + * Returns the custom token filters used for analysis, if defined + */ + public List tokenFilters() { + return this.tokenFilters; + } + + /** + * Returns the custom character filters used for analysis, if defined + */ + public List charFilters() { + return this.charFilters; + } + + /** + * Returns the field to take an Analyzer from, if defined + */ + public String field() { + return this.field; + } + + /** + * Set whether or not detailed explanations of analysis should be returned + */ + public AnalyzeRequest explain(boolean explain) { + this.explain = explain; + return this; + } + + public boolean explain() { + return this.explain; + } + + public AnalyzeRequest attributes(String... attributes) { + if (attributes == null) { + throw new IllegalArgumentException("attributes must not be null"); + } + this.attributes = attributes; + return this; + } + + public String[] attributes() { + return this.attributes; + } + + @Override + public XContentBuilder toXContent(XContentBuilder builder, Params params) throws IOException { + builder.startObject(); + builder.field("text", text); + if (Strings.isNullOrEmpty(analyzer) == false) { + builder.field("analyzer", analyzer); + } + if (tokenizer != null) { + builder.field("tokenizer", tokenizer); + } + if (tokenFilters.size() > 0) { + builder.field("filter", tokenFilters); + } + if (charFilters.size() > 0) { + builder.field("char_filter", charFilters); + } + if (Strings.isNullOrEmpty(field) == false) { + builder.field("field", field); + } + if (explain) { + builder.field("explain", true); + } + if (attributes.length > 0) { + builder.field("attributes", attributes); + } + if (Strings.isNullOrEmpty(normalizer) == false) { + builder.field("normalizer", normalizer); + } + return builder.endObject(); + } +} diff --git a/client/rest-high-level/src/main/java/org/elasticsearch/client/indices/AnalyzeResponse.java b/client/rest-high-level/src/main/java/org/elasticsearch/client/indices/AnalyzeResponse.java new file mode 100644 index 0000000000000..aaba8653dee84 --- /dev/null +++ b/client/rest-high-level/src/main/java/org/elasticsearch/client/indices/AnalyzeResponse.java @@ -0,0 +1,183 @@ +/* + * 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.client.indices; + +import org.elasticsearch.common.ParseField; +import org.elasticsearch.common.xcontent.ConstructingObjectParser; +import org.elasticsearch.common.xcontent.ObjectParser; +import org.elasticsearch.common.xcontent.XContentParser; + +import java.io.IOException; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.Objects; + +import static org.elasticsearch.common.xcontent.ConstructingObjectParser.optionalConstructorArg; + +public class AnalyzeResponse { + + private static final String TOKENS = "tokens"; + private static final String DETAIL = "detail"; + + public static class AnalyzeToken { + private String term; + private int startOffset; + private int endOffset; + private int position; + private int positionLength = 1; + private String type; + private final Map attributes = new HashMap<>(); + + @Override + public boolean equals(Object o) { + if (this == o) return true; + if (o == null || getClass() != o.getClass()) return false; + AnalyzeResponse.AnalyzeToken that = (AnalyzeResponse.AnalyzeToken) o; + return startOffset == that.startOffset && + endOffset == that.endOffset && + position == that.position && + positionLength == that.positionLength && + Objects.equals(term, that.term) && + Objects.equals(attributes, that.attributes) && + Objects.equals(type, that.type); + } + + @Override + public int hashCode() { + return Objects.hash(term, startOffset, endOffset, position, positionLength, attributes, type); + } + + public String getTerm() { + return this.term; + } + + private void setTerm(String term) { + this.term = term; + } + + public int getStartOffset() { + return this.startOffset; + } + + private void setStartOffset(int startOffset) { + this.startOffset = startOffset; + } + + public int getEndOffset() { + return this.endOffset; + } + + private void setEndOffset(int endOffset) { + this.endOffset = endOffset; + } + + public int getPosition() { + return this.position; + } + + private void setPosition(int position) { + this.position = position; + } + + public int getPositionLength() { + return this.positionLength; + } + + private void setPositionLength(int positionLength) { + this.positionLength = positionLength; + } + + public String getType() { + return this.type; + } + + private void setType(String type) { + this.type = type; + } + + public Map getAttributes() { + return this.attributes; + } + + private void setAttribute(String key, Object value) { + this.attributes.put(key, value); + } + + private static final ObjectParser PARSER + = new ObjectParser<>("analyze_token", AnalyzeToken::setAttribute, AnalyzeToken::new); + static { + PARSER.declareString(AnalyzeToken::setTerm, new ParseField("token")); + PARSER.declareString(AnalyzeToken::setType, new ParseField("type")); + PARSER.declareInt(AnalyzeToken::setPosition, new ParseField("position")); + PARSER.declareInt(AnalyzeToken::setStartOffset, new ParseField("start_offset")); + PARSER.declareInt(AnalyzeToken::setEndOffset, new ParseField("end_offset")); + PARSER.declareInt(AnalyzeToken::setPositionLength, new ParseField("positionLength")); + } + + public static AnalyzeToken fromXContent(XContentParser parser) throws IOException { + return PARSER.parse(parser, null); + } + } + + private final DetailAnalyzeResponse detail; + private final List tokens; + + private AnalyzeResponse(List tokens, DetailAnalyzeResponse detail) { + this.tokens = tokens; + this.detail = detail; + } + + public List getTokens() { + return this.tokens; + } + + public DetailAnalyzeResponse detail() { + return this.detail; + } + + @SuppressWarnings("unchecked") + private static final ConstructingObjectParser PARSER = new ConstructingObjectParser<>("analyze_response", + true, args -> new AnalyzeResponse((List) args[0], (DetailAnalyzeResponse) args[1])); + + static { + PARSER.declareObjectArray(optionalConstructorArg(), AnalyzeToken.PARSER, new ParseField(TOKENS)); + PARSER.declareObject(optionalConstructorArg(), DetailAnalyzeResponse.PARSER, new ParseField(DETAIL)); + } + + public static AnalyzeResponse fromXContent(XContentParser parser) throws IOException { + return PARSER.parse(parser, null); + } + + @Override + public boolean equals(Object o) { + if (this == o) return true; + if (o == null || getClass() != o.getClass()) return false; + AnalyzeResponse that = (AnalyzeResponse) o; + return Objects.equals(detail, that.detail) && + Objects.equals(tokens, that.tokens); + } + + @Override + public int hashCode() { + return Objects.hash(detail, tokens); + } + +} diff --git a/client/rest-high-level/src/main/java/org/elasticsearch/client/indices/DetailAnalyzeResponse.java b/client/rest-high-level/src/main/java/org/elasticsearch/client/indices/DetailAnalyzeResponse.java new file mode 100644 index 0000000000000..36cf8afad0d58 --- /dev/null +++ b/client/rest-high-level/src/main/java/org/elasticsearch/client/indices/DetailAnalyzeResponse.java @@ -0,0 +1,214 @@ +/* + * 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.client.indices; + +import org.elasticsearch.common.ParseField; +import org.elasticsearch.common.Strings; +import org.elasticsearch.common.xcontent.ConstructingObjectParser; +import org.elasticsearch.common.xcontent.XContentParser; + +import java.io.IOException; +import java.util.Arrays; +import java.util.List; +import java.util.Objects; + +import static org.elasticsearch.common.xcontent.ConstructingObjectParser.constructorArg; +import static org.elasticsearch.common.xcontent.ConstructingObjectParser.optionalConstructorArg; + +public class DetailAnalyzeResponse { + + private final boolean customAnalyzer; + private final AnalyzeTokenList analyzer; + private final CharFilteredText[] charfilters; + private final AnalyzeTokenList tokenizer; + private final AnalyzeTokenList[] tokenfilters; + + private DetailAnalyzeResponse(boolean customAnalyzer, + AnalyzeTokenList analyzer, + List charfilters, + AnalyzeTokenList tokenizer, + List tokenfilters) { + this.customAnalyzer = customAnalyzer; + this.analyzer = analyzer; + this.charfilters = charfilters == null ? null : charfilters.toArray(new CharFilteredText[]{}); + this.tokenizer = tokenizer; + this.tokenfilters = tokenfilters == null ? null : tokenfilters.toArray(new AnalyzeTokenList[]{}); + } + + public AnalyzeTokenList analyzer() { + return this.analyzer; + } + + public CharFilteredText[] charfilters() { + return this.charfilters; + } + + public AnalyzeTokenList tokenizer() { + return tokenizer; + } + + public AnalyzeTokenList[] tokenfilters() { + return tokenfilters; + } + + @Override + public boolean equals(Object o) { + if (this == o) return true; + if (o == null || getClass() != o.getClass()) return false; + DetailAnalyzeResponse that = (DetailAnalyzeResponse) o; + return customAnalyzer == that.customAnalyzer && + Objects.equals(analyzer, that.analyzer) && + Arrays.equals(charfilters, that.charfilters) && + Objects.equals(tokenizer, that.tokenizer) && + Arrays.equals(tokenfilters, that.tokenfilters); + } + + @Override + public int hashCode() { + int result = Objects.hash(customAnalyzer, analyzer, tokenizer); + result = 31 * result + Arrays.hashCode(charfilters); + result = 31 * result + Arrays.hashCode(tokenfilters); + return result; + } + + @SuppressWarnings("unchecked") + static final ConstructingObjectParser PARSER = new ConstructingObjectParser<>("detail", + true, args -> new DetailAnalyzeResponse( + (boolean) args[0], + (AnalyzeTokenList) args[1], + (List)args[2], + (AnalyzeTokenList) args[3], + (List)args[4])); + + static { + PARSER.declareBoolean(constructorArg(), new ParseField("custom_analyzer")); + PARSER.declareObject(optionalConstructorArg(), AnalyzeTokenList.PARSER, new ParseField("analyzer")); + PARSER.declareObjectArray(optionalConstructorArg(), CharFilteredText.PARSER, new ParseField("charfilters")); + PARSER.declareObject(optionalConstructorArg(), AnalyzeTokenList.PARSER, new ParseField("tokenizer")); + PARSER.declareObjectArray(optionalConstructorArg(), AnalyzeTokenList.PARSER, new ParseField("tokenfilters")); + } + + public static DetailAnalyzeResponse fromXContent(XContentParser parser) throws IOException { + return PARSER.parse(parser, null); + } + + public static class AnalyzeTokenList { + private final String name; + private final AnalyzeResponse.AnalyzeToken[] tokens; + + private static final String TOKENS = "tokens"; + + @Override + public boolean equals(Object o) { + if (this == o) return true; + if (o == null || getClass() != o.getClass()) return false; + AnalyzeTokenList that = (AnalyzeTokenList) o; + return Objects.equals(name, that.name) && + Arrays.equals(tokens, that.tokens); + } + + @Override + public int hashCode() { + int result = Objects.hash(name); + result = 31 * result + Arrays.hashCode(tokens); + return result; + } + + public AnalyzeTokenList(String name, List tokens) { + this.name = name; + this.tokens = tokens.toArray(new AnalyzeResponse.AnalyzeToken[]{}); + } + + public String getName() { + return name; + } + + public AnalyzeResponse.AnalyzeToken[] getTokens() { + return tokens; + } + + @SuppressWarnings("unchecked") + private static final ConstructingObjectParser PARSER = new ConstructingObjectParser<>("token_list", + true, args -> new AnalyzeTokenList((String) args[0], + (List)args[1])); + + static { + PARSER.declareString(constructorArg(), new ParseField("name")); + PARSER.declareObjectArray(constructorArg(), (p, c) -> AnalyzeResponse.AnalyzeToken.fromXContent(p), + new ParseField("tokens")); + } + + public static AnalyzeTokenList fromXContent(XContentParser parser) throws IOException { + return PARSER.parse(parser, null); + } + + } + + public static class CharFilteredText { + private final String name; + private final String[] texts; + + CharFilteredText(String name, String[] texts) { + this.name = name; + if (texts != null) { + this.texts = texts; + } else { + this.texts = Strings.EMPTY_ARRAY; + } + } + + public String getName() { + return name; + } + + public String[] getTexts() { + return texts; + } + + @SuppressWarnings("unchecked") + private static final ConstructingObjectParser PARSER = new ConstructingObjectParser<>("char_filtered_text", + true, args -> new CharFilteredText((String) args[0], ((List) args[1]).toArray(new String[0]))); + + static { + PARSER.declareString(constructorArg(), new ParseField("name")); + PARSER.declareStringArray(constructorArg(), new ParseField("filtered_text")); + } + + public static CharFilteredText fromXContent(XContentParser parser) throws IOException { + return PARSER.parse(parser, null); + } + + @Override + public boolean equals(Object o) { + if (this == o) return true; + if (o == null || getClass() != o.getClass()) return false; + CharFilteredText that = (CharFilteredText) o; + return Objects.equals(name, that.name) && + Arrays.equals(texts, that.texts); + } + + @Override + public int hashCode() { + int result = Objects.hash(name); + result = 31 * result + Arrays.hashCode(texts); + return result; + } + } +} diff --git a/client/rest-high-level/src/main/java/org/elasticsearch/client/indices/ReloadAnalyzersRequest.java b/client/rest-high-level/src/main/java/org/elasticsearch/client/indices/ReloadAnalyzersRequest.java new file mode 100644 index 0000000000000..e815d91bbe8f5 --- /dev/null +++ b/client/rest-high-level/src/main/java/org/elasticsearch/client/indices/ReloadAnalyzersRequest.java @@ -0,0 +1,68 @@ +/* + * 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.client.indices; + +import org.elasticsearch.action.support.IndicesOptions; +import org.elasticsearch.client.Validatable; + +import java.util.Objects; + +/** + * Request for the _reload_search_analyzers API + */ +public final class ReloadAnalyzersRequest implements Validatable { + + private final String[] indices; + private IndicesOptions indicesOptions = IndicesOptions.strictExpandOpen(); + + /** + * Creates a new reload analyzers request + * @param indices the index for which to reload analyzers + */ + public ReloadAnalyzersRequest(String... indices) { + this.indices = Objects.requireNonNull(indices); + } + + /** + * Returns the indices + */ + public String[] getIndices() { + return indices; + } + + /** + * Specifies what type of requested indices to ignore and how to deal with wildcard expressions. + * For example indices that don't exist. + * + * @return the current behaviour when it comes to index names and wildcard indices expressions + */ + public IndicesOptions indicesOptions() { + return indicesOptions; + } + + /** + * Specifies what type of requested indices to ignore and how to deal with wildcard expressions. + * For example indices that don't exist. + * + * @param indicesOptions the desired behaviour regarding indices to ignore and wildcard indices expressions + */ + public void setIndicesOptions(IndicesOptions indicesOptions) { + this.indicesOptions = indicesOptions; + } +} diff --git a/client/rest-high-level/src/main/java/org/elasticsearch/client/indices/ReloadAnalyzersResponse.java b/client/rest-high-level/src/main/java/org/elasticsearch/client/indices/ReloadAnalyzersResponse.java new file mode 100644 index 0000000000000..e2c39d0a7aeba --- /dev/null +++ b/client/rest-high-level/src/main/java/org/elasticsearch/client/indices/ReloadAnalyzersResponse.java @@ -0,0 +1,108 @@ +/* + * 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.client.indices; + +import org.elasticsearch.client.core.BroadcastResponse; +import org.elasticsearch.common.ParseField; +import org.elasticsearch.common.collect.Tuple; +import org.elasticsearch.common.xcontent.ConstructingObjectParser; +import org.elasticsearch.common.xcontent.XContentParser; + +import java.util.HashMap; +import java.util.HashSet; +import java.util.List; +import java.util.Map; +import java.util.Set; + +import static org.elasticsearch.common.xcontent.ConstructingObjectParser.constructorArg; + +/** + * The response object that will be returned when reloading analyzers + */ +public class ReloadAnalyzersResponse extends BroadcastResponse { + + private final Map reloadDetails; + + ReloadAnalyzersResponse(final Shards shards, Map reloadDetails) { + super(shards); + this.reloadDetails = reloadDetails; + } + + @SuppressWarnings({ "unchecked" }) + private static final ConstructingObjectParser PARSER = new ConstructingObjectParser<>("reload_analyzer", + true, arg -> { + Shards shards = (Shards) arg[0]; + List> results = (List>) arg[1]; + Map reloadDetails = new HashMap<>(); + for (Tuple result : results) { + reloadDetails.put(result.v1(), result.v2()); + } + return new ReloadAnalyzersResponse(shards, reloadDetails); + }); + + @SuppressWarnings({ "unchecked" }) + private static final ConstructingObjectParser, Void> ENTRY_PARSER = new ConstructingObjectParser<>( + "reload_analyzer.entry", true, arg -> { + String index = (String) arg[0]; + Set nodeIds = new HashSet<>((List) arg[1]); + Set analyzers = new HashSet<>((List) arg[2]); + return new Tuple<>(index, new ReloadDetails(index, nodeIds, analyzers)); + }); + + static { + declareShardsField(PARSER); + PARSER.declareObjectArray(constructorArg(), ENTRY_PARSER, new ParseField("reload_details")); + ENTRY_PARSER.declareString(constructorArg(), new ParseField("index")); + ENTRY_PARSER.declareStringArray(constructorArg(), new ParseField("reloaded_node_ids")); + ENTRY_PARSER.declareStringArray(constructorArg(), new ParseField("reloaded_analyzers")); + } + + public static ReloadAnalyzersResponse fromXContent(XContentParser parser) { + return PARSER.apply(parser, null); + } + + public Map getReloadedDetails() { + return reloadDetails; + } + + public static class ReloadDetails { + + private final String indexName; + private final Set reloadedIndicesNodes; + private final Set reloadedAnalyzers; + + public ReloadDetails(String name, Set reloadedIndicesNodes, Set reloadedAnalyzers) { + this.indexName = name; + this.reloadedIndicesNodes = reloadedIndicesNodes; + this.reloadedAnalyzers = reloadedAnalyzers; + } + + public String getIndexName() { + return indexName; + } + + public Set getReloadedIndicesNodes() { + return reloadedIndicesNodes; + } + + public Set getReloadedAnalyzers() { + return reloadedAnalyzers; + } + } +} diff --git a/client/rest-high-level/src/main/java/org/elasticsearch/client/ml/DeleteDataFrameAnalyticsRequest.java b/client/rest-high-level/src/main/java/org/elasticsearch/client/ml/DeleteDataFrameAnalyticsRequest.java new file mode 100644 index 0000000000000..f03466632304d --- /dev/null +++ b/client/rest-high-level/src/main/java/org/elasticsearch/client/ml/DeleteDataFrameAnalyticsRequest.java @@ -0,0 +1,64 @@ +/* + * 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.client.ml; + +import org.elasticsearch.client.Validatable; +import org.elasticsearch.client.ValidationException; + +import java.util.Objects; +import java.util.Optional; + +/** + * Request to delete a data frame analytics config + */ +public class DeleteDataFrameAnalyticsRequest implements Validatable { + + private final String id; + + public DeleteDataFrameAnalyticsRequest(String id) { + this.id = id; + } + + public String getId() { + return id; + } + + @Override + public Optional validate() { + if (id == null) { + return Optional.of(ValidationException.withError("data frame analytics id must not be null")); + } + return Optional.empty(); + } + + @Override + public boolean equals(Object o) { + if (this == o) return true; + if (o == null || getClass() != o.getClass()) return false; + + DeleteDataFrameAnalyticsRequest other = (DeleteDataFrameAnalyticsRequest) o; + return Objects.equals(id, other.id); + } + + @Override + public int hashCode() { + return Objects.hash(id); + } +} diff --git a/client/rest-high-level/src/main/java/org/elasticsearch/client/ml/EvaluateDataFrameRequest.java b/client/rest-high-level/src/main/java/org/elasticsearch/client/ml/EvaluateDataFrameRequest.java new file mode 100644 index 0000000000000..2e3bbb170509c --- /dev/null +++ b/client/rest-high-level/src/main/java/org/elasticsearch/client/ml/EvaluateDataFrameRequest.java @@ -0,0 +1,136 @@ +/* + * 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.client.ml; + +import org.elasticsearch.client.Validatable; +import org.elasticsearch.client.ValidationException; +import org.elasticsearch.client.ml.dataframe.evaluation.Evaluation; +import org.elasticsearch.common.ParseField; +import org.elasticsearch.common.xcontent.ConstructingObjectParser; +import org.elasticsearch.common.xcontent.ToXContentObject; +import org.elasticsearch.common.xcontent.XContentBuilder; +import org.elasticsearch.common.xcontent.XContentParser; + +import java.io.IOException; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.Collections; +import java.util.List; +import java.util.Objects; +import java.util.Optional; + +import static org.elasticsearch.common.xcontent.ConstructingObjectParser.constructorArg; +import static org.elasticsearch.common.xcontent.XContentParserUtils.ensureExpectedToken; + +public class EvaluateDataFrameRequest implements ToXContentObject, Validatable { + + private static final ParseField INDEX = new ParseField("index"); + private static final ParseField EVALUATION = new ParseField("evaluation"); + + @SuppressWarnings("unchecked") + private static final ConstructingObjectParser PARSER = + new ConstructingObjectParser<>( + "evaluate_data_frame_request", true, args -> new EvaluateDataFrameRequest((List) args[0], (Evaluation) args[1])); + + static { + PARSER.declareStringArray(constructorArg(), INDEX); + PARSER.declareObject(constructorArg(), (p, c) -> parseEvaluation(p), EVALUATION); + } + + private static Evaluation parseEvaluation(XContentParser parser) throws IOException { + ensureExpectedToken(XContentParser.Token.START_OBJECT, parser.currentToken(), parser::getTokenLocation); + ensureExpectedToken(XContentParser.Token.FIELD_NAME, parser.nextToken(), parser::getTokenLocation); + Evaluation evaluation = parser.namedObject(Evaluation.class, parser.currentName(), null); + ensureExpectedToken(XContentParser.Token.END_OBJECT, parser.nextToken(), parser::getTokenLocation); + return evaluation; + } + + public static EvaluateDataFrameRequest fromXContent(XContentParser parser) { + return PARSER.apply(parser, null); + } + + private List indices; + private Evaluation evaluation; + + public EvaluateDataFrameRequest(String index, Evaluation evaluation) { + this(Arrays.asList(index), evaluation); + } + + public EvaluateDataFrameRequest(List indices, Evaluation evaluation) { + setIndices(indices); + setEvaluation(evaluation); + } + + public List getIndices() { + return Collections.unmodifiableList(indices); + } + + public final void setIndices(List indices) { + Objects.requireNonNull(indices); + this.indices = new ArrayList<>(indices); + } + + public Evaluation getEvaluation() { + return evaluation; + } + + public final void setEvaluation(Evaluation evaluation) { + this.evaluation = evaluation; + } + + @Override + public Optional validate() { + List errors = new ArrayList<>(); + if (indices.isEmpty()) { + errors.add("At least one index must be specified"); + } + if (evaluation == null) { + errors.add("evaluation must not be null"); + } + return errors.isEmpty() + ? Optional.empty() + : Optional.of(ValidationException.withErrors(errors)); + } + + @Override + public XContentBuilder toXContent(XContentBuilder builder, Params params) throws IOException { + return builder + .startObject() + .array(INDEX.getPreferredName(), indices.toArray()) + .startObject(EVALUATION.getPreferredName()) + .field(evaluation.getName(), evaluation) + .endObject() + .endObject(); + } + + @Override + public int hashCode() { + return Objects.hash(indices, evaluation); + } + + @Override + public boolean equals(Object o) { + if (this == o) return true; + if (o == null || getClass() != o.getClass()) return false; + EvaluateDataFrameRequest that = (EvaluateDataFrameRequest) o; + return Objects.equals(indices, that.indices) + && Objects.equals(evaluation, that.evaluation); + } +} diff --git a/client/rest-high-level/src/main/java/org/elasticsearch/client/ml/EvaluateDataFrameResponse.java b/client/rest-high-level/src/main/java/org/elasticsearch/client/ml/EvaluateDataFrameResponse.java new file mode 100644 index 0000000000000..0709021ed4bd5 --- /dev/null +++ b/client/rest-high-level/src/main/java/org/elasticsearch/client/ml/EvaluateDataFrameResponse.java @@ -0,0 +1,117 @@ +/* + * 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.client.ml; + +import org.elasticsearch.client.ml.dataframe.evaluation.EvaluationMetric; +import org.elasticsearch.common.Strings; +import org.elasticsearch.common.xcontent.NamedObjectNotFoundException; +import org.elasticsearch.common.xcontent.ToXContent; +import org.elasticsearch.common.xcontent.ToXContentObject; +import org.elasticsearch.common.xcontent.XContentBuilder; +import org.elasticsearch.common.xcontent.XContentParser; + +import java.io.IOException; +import java.util.LinkedHashMap; +import java.util.List; +import java.util.Map; +import java.util.Objects; +import java.util.stream.Collectors; + +import static org.elasticsearch.common.xcontent.XContentParserUtils.ensureExpectedToken; + +public class EvaluateDataFrameResponse implements ToXContentObject { + + public static EvaluateDataFrameResponse fromXContent(XContentParser parser) throws IOException { + if (parser.currentToken() == null) { + parser.nextToken(); + } + ensureExpectedToken(XContentParser.Token.START_OBJECT, parser.currentToken(), parser::getTokenLocation); + ensureExpectedToken(XContentParser.Token.FIELD_NAME, parser.nextToken(), parser::getTokenLocation); + String evaluationName = parser.currentName(); + parser.nextToken(); + Map metrics = parser.map(LinkedHashMap::new, EvaluateDataFrameResponse::parseMetric); + List knownMetrics = + metrics.values().stream() + .filter(Objects::nonNull) // Filter out null values returned by {@link EvaluateDataFrameResponse::parseMetric}. + .collect(Collectors.toList()); + ensureExpectedToken(XContentParser.Token.END_OBJECT, parser.nextToken(), parser::getTokenLocation); + return new EvaluateDataFrameResponse(evaluationName, knownMetrics); + } + + private static EvaluationMetric.Result parseMetric(XContentParser parser) throws IOException { + String metricName = parser.currentName(); + try { + return parser.namedObject(EvaluationMetric.Result.class, metricName, null); + } catch (NamedObjectNotFoundException e) { + parser.skipChildren(); + // Metric name not recognized. Return {@code null} value here and filter it out later. + return null; + } + } + + private final String evaluationName; + private final Map metrics; + + public EvaluateDataFrameResponse(String evaluationName, List metrics) { + this.evaluationName = Objects.requireNonNull(evaluationName); + this.metrics = Objects.requireNonNull(metrics).stream().collect(Collectors.toUnmodifiableMap(m -> m.getMetricName(), m -> m)); + } + + public String getEvaluationName() { + return evaluationName; + } + + public List getMetrics() { + return metrics.values().stream().collect(Collectors.toList()); + } + + @SuppressWarnings("unchecked") + public T getMetricByName(String metricName) { + Objects.requireNonNull(metricName); + return (T) metrics.get(metricName); + } + + @Override + public XContentBuilder toXContent(XContentBuilder builder, ToXContent.Params params) throws IOException { + return builder + .startObject() + .field(evaluationName, metrics) + .endObject(); + } + + @Override + public boolean equals(Object o) { + if (o == this) return true; + if (o == null || getClass() != o.getClass()) return false; + EvaluateDataFrameResponse that = (EvaluateDataFrameResponse) o; + return Objects.equals(evaluationName, that.evaluationName) + && Objects.equals(metrics, that.metrics); + } + + @Override + public int hashCode() { + return Objects.hash(evaluationName, metrics); + } + + @Override + public final String toString() { + return Strings.toString(this); + } +} diff --git a/client/rest-high-level/src/main/java/org/elasticsearch/client/ml/FindFileStructureRequest.java b/client/rest-high-level/src/main/java/org/elasticsearch/client/ml/FindFileStructureRequest.java index adfee92bd6171..fed417e9582d8 100644 --- a/client/rest-high-level/src/main/java/org/elasticsearch/client/ml/FindFileStructureRequest.java +++ b/client/rest-high-level/src/main/java/org/elasticsearch/client/ml/FindFileStructureRequest.java @@ -37,6 +37,7 @@ public class FindFileStructureRequest implements Validatable, ToXContentFragment { public static final ParseField LINES_TO_SAMPLE = new ParseField("lines_to_sample"); + public static final ParseField LINE_MERGE_SIZE_LIMIT = new ParseField("line_merge_size_limit"); public static final ParseField TIMEOUT = new ParseField("timeout"); public static final ParseField CHARSET = FileStructure.CHARSET; public static final ParseField FORMAT = FileStructure.FORMAT; @@ -52,6 +53,7 @@ public class FindFileStructureRequest implements Validatable, ToXContentFragment public static final ParseField EXPLAIN = new ParseField("explain"); private Integer linesToSample; + private Integer lineMergeSizeLimit; private TimeValue timeout; private String charset; private FileStructure.Format format; @@ -77,6 +79,14 @@ public void setLinesToSample(Integer linesToSample) { this.linesToSample = linesToSample; } + public Integer getLineMergeSizeLimit() { + return lineMergeSizeLimit; + } + + public void setLineMergeSizeLimit(Integer lineMergeSizeLimit) { + this.lineMergeSizeLimit = lineMergeSizeLimit; + } + public TimeValue getTimeout() { return timeout; } @@ -228,6 +238,9 @@ public XContentBuilder toXContent(XContentBuilder builder, Params params) throws if (linesToSample != null) { builder.field(LINES_TO_SAMPLE.getPreferredName(), linesToSample); } + if (lineMergeSizeLimit != null) { + builder.field(LINE_MERGE_SIZE_LIMIT.getPreferredName(), lineMergeSizeLimit); + } if (timeout != null) { builder.field(TIMEOUT.getPreferredName(), timeout); } @@ -270,8 +283,8 @@ public XContentBuilder toXContent(XContentBuilder builder, Params params) throws @Override public int hashCode() { - return Objects.hash(linesToSample, timeout, charset, format, columnNames, hasHeaderRow, delimiter, grokPattern, timestampFormat, - timestampField, explain, sample); + return Objects.hash(linesToSample, lineMergeSizeLimit, timeout, charset, format, columnNames, hasHeaderRow, delimiter, grokPattern, + timestampFormat, timestampField, explain, sample); } @Override @@ -287,6 +300,7 @@ public boolean equals(Object other) { FindFileStructureRequest that = (FindFileStructureRequest) other; return Objects.equals(this.linesToSample, that.linesToSample) && + Objects.equals(this.lineMergeSizeLimit, that.lineMergeSizeLimit) && Objects.equals(this.timeout, that.timeout) && Objects.equals(this.charset, that.charset) && Objects.equals(this.format, that.format) && diff --git a/client/rest-high-level/src/main/java/org/elasticsearch/client/ml/GetDataFrameAnalyticsRequest.java b/client/rest-high-level/src/main/java/org/elasticsearch/client/ml/GetDataFrameAnalyticsRequest.java new file mode 100644 index 0000000000000..40698c4b528fa --- /dev/null +++ b/client/rest-high-level/src/main/java/org/elasticsearch/client/ml/GetDataFrameAnalyticsRequest.java @@ -0,0 +1,104 @@ +/* + * 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.client.ml; + +import org.elasticsearch.client.Validatable; +import org.elasticsearch.client.ValidationException; +import org.elasticsearch.client.core.PageParams; +import org.elasticsearch.common.Nullable; +import org.elasticsearch.common.ParseField; + +import java.util.Arrays; +import java.util.List; +import java.util.Objects; +import java.util.Optional; + +public class GetDataFrameAnalyticsRequest implements Validatable { + + public static final ParseField ALLOW_NO_MATCH = new ParseField("allow_no_match"); + + private final List ids; + private Boolean allowNoMatch; + private PageParams pageParams; + + /** + * Helper method to create a request that will get ALL Data Frame Analytics + * @return new {@link GetDataFrameAnalyticsRequest} object for the id "_all" + */ + public static GetDataFrameAnalyticsRequest getAllDataFrameAnalyticsRequest() { + return new GetDataFrameAnalyticsRequest("_all"); + } + + public GetDataFrameAnalyticsRequest(String... ids) { + this.ids = Arrays.asList(ids); + } + + public List getIds() { + return ids; + } + + public Boolean getAllowNoMatch() { + return allowNoMatch; + } + + /** + * Whether to ignore if a wildcard expression matches no data frame analytics. + * + * @param allowNoMatch If this is {@code false}, then an error is returned when a wildcard (or {@code _all}) + * does not match any data frame analytics + */ + public GetDataFrameAnalyticsRequest setAllowNoMatch(boolean allowNoMatch) { + this.allowNoMatch = allowNoMatch; + return this; + } + + public PageParams getPageParams() { + return pageParams; + } + + public GetDataFrameAnalyticsRequest setPageParams(@Nullable PageParams pageParams) { + this.pageParams = pageParams; + return this; + } + + @Override + public Optional validate() { + if (ids == null || ids.isEmpty()) { + return Optional.of(ValidationException.withError("data frame analytics id must not be null")); + } + return Optional.empty(); + } + + @Override + public boolean equals(Object o) { + if (this == o) return true; + if (o == null || getClass() != o.getClass()) return false; + + GetDataFrameAnalyticsRequest other = (GetDataFrameAnalyticsRequest) o; + return Objects.equals(ids, other.ids) + && Objects.equals(allowNoMatch, other.allowNoMatch) + && Objects.equals(pageParams, other.pageParams); + } + + @Override + public int hashCode() { + return Objects.hash(ids, allowNoMatch, pageParams); + } +} diff --git a/client/rest-high-level/src/main/java/org/elasticsearch/client/ml/GetDataFrameAnalyticsResponse.java b/client/rest-high-level/src/main/java/org/elasticsearch/client/ml/GetDataFrameAnalyticsResponse.java new file mode 100644 index 0000000000000..76996e9d4d0b6 --- /dev/null +++ b/client/rest-high-level/src/main/java/org/elasticsearch/client/ml/GetDataFrameAnalyticsResponse.java @@ -0,0 +1,74 @@ +/* + * 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.client.ml; + +import org.elasticsearch.client.ml.dataframe.DataFrameAnalyticsConfig; +import org.elasticsearch.common.ParseField; +import org.elasticsearch.common.xcontent.ConstructingObjectParser; +import org.elasticsearch.common.xcontent.XContentParser; + +import java.util.List; +import java.util.Objects; + +import static org.elasticsearch.common.xcontent.ConstructingObjectParser.constructorArg; + +public class GetDataFrameAnalyticsResponse { + + public static final ParseField DATA_FRAME_ANALYTICS = new ParseField("data_frame_analytics"); + + @SuppressWarnings("unchecked") + static final ConstructingObjectParser PARSER = + new ConstructingObjectParser<>( + "get_data_frame_analytics", + true, + args -> new GetDataFrameAnalyticsResponse((List) args[0])); + + static { + PARSER.declareObjectArray(constructorArg(), (p, c) -> DataFrameAnalyticsConfig.fromXContent(p), DATA_FRAME_ANALYTICS); + } + + public static GetDataFrameAnalyticsResponse fromXContent(final XContentParser parser) { + return PARSER.apply(parser, null); + } + + private List analytics; + + public GetDataFrameAnalyticsResponse(List analytics) { + this.analytics = analytics; + } + + public List getAnalytics() { + return analytics; + } + + @Override + public boolean equals(Object o) { + if (this == o) return true; + if (o == null || getClass() != o.getClass()) return false; + + GetDataFrameAnalyticsResponse other = (GetDataFrameAnalyticsResponse) o; + return Objects.equals(this.analytics, other.analytics); + } + + @Override + public int hashCode() { + return Objects.hash(analytics); + } +} diff --git a/client/rest-high-level/src/main/java/org/elasticsearch/client/ml/GetDataFrameAnalyticsStatsRequest.java b/client/rest-high-level/src/main/java/org/elasticsearch/client/ml/GetDataFrameAnalyticsStatsRequest.java new file mode 100644 index 0000000000000..f1e4a35fb661b --- /dev/null +++ b/client/rest-high-level/src/main/java/org/elasticsearch/client/ml/GetDataFrameAnalyticsStatsRequest.java @@ -0,0 +1,99 @@ +/* + * 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.client.ml; + +import org.elasticsearch.client.Validatable; +import org.elasticsearch.client.ValidationException; +import org.elasticsearch.client.core.PageParams; +import org.elasticsearch.common.Nullable; +import org.elasticsearch.common.ParseField; + +import java.util.Arrays; +import java.util.List; +import java.util.Objects; +import java.util.Optional; + +/** + * Request to get data frame analytics stats + */ +public class GetDataFrameAnalyticsStatsRequest implements Validatable { + + public static final ParseField ALLOW_NO_MATCH = new ParseField("allow_no_match"); + + private final List ids; + private Boolean allowNoMatch; + private PageParams pageParams; + + public GetDataFrameAnalyticsStatsRequest(String... ids) { + this.ids = Arrays.asList(ids); + } + + public List getIds() { + return ids; + } + + public Boolean getAllowNoMatch() { + return allowNoMatch; + } + + /** + * Whether to ignore if a wildcard expression matches no data frame analytics. + * + * @param allowNoMatch If this is {@code false}, then an error is returned when a wildcard (or {@code _all}) + * does not match any data frame analytics + */ + public GetDataFrameAnalyticsStatsRequest setAllowNoMatch(boolean allowNoMatch) { + this.allowNoMatch = allowNoMatch; + return this; + } + + public PageParams getPageParams() { + return pageParams; + } + + public GetDataFrameAnalyticsStatsRequest setPageParams(@Nullable PageParams pageParams) { + this.pageParams = pageParams; + return this; + } + + @Override + public Optional validate() { + if (ids == null || ids.isEmpty()) { + return Optional.of(ValidationException.withError("data frame analytics id must not be null")); + } + return Optional.empty(); + } + + @Override + public boolean equals(Object o) { + if (this == o) return true; + if (o == null || getClass() != o.getClass()) return false; + + GetDataFrameAnalyticsStatsRequest other = (GetDataFrameAnalyticsStatsRequest) o; + return Objects.equals(ids, other.ids) + && Objects.equals(allowNoMatch, other.allowNoMatch) + && Objects.equals(pageParams, other.pageParams); + } + + @Override + public int hashCode() { + return Objects.hash(ids, allowNoMatch, pageParams); + } +} diff --git a/client/rest-high-level/src/main/java/org/elasticsearch/client/ml/GetDataFrameAnalyticsStatsResponse.java b/client/rest-high-level/src/main/java/org/elasticsearch/client/ml/GetDataFrameAnalyticsStatsResponse.java new file mode 100644 index 0000000000000..5391a576e98b0 --- /dev/null +++ b/client/rest-high-level/src/main/java/org/elasticsearch/client/ml/GetDataFrameAnalyticsStatsResponse.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.client.ml; + +import org.elasticsearch.ElasticsearchException; +import org.elasticsearch.action.TaskOperationFailure; +import org.elasticsearch.client.dataframe.AcknowledgedTasksResponse; +import org.elasticsearch.client.ml.dataframe.DataFrameAnalyticsStats; +import org.elasticsearch.common.Nullable; +import org.elasticsearch.common.ParseField; +import org.elasticsearch.common.xcontent.ConstructingObjectParser; +import org.elasticsearch.common.xcontent.XContentParser; + +import java.util.Collections; +import java.util.List; +import java.util.Objects; + +import static org.elasticsearch.common.xcontent.ConstructingObjectParser.constructorArg; +import static org.elasticsearch.common.xcontent.ConstructingObjectParser.optionalConstructorArg; + +public class GetDataFrameAnalyticsStatsResponse { + + public static GetDataFrameAnalyticsStatsResponse fromXContent(XContentParser parser) { + return GetDataFrameAnalyticsStatsResponse.PARSER.apply(parser, null); + } + + private static final ParseField DATA_FRAME_ANALYTICS = new ParseField("data_frame_analytics"); + + @SuppressWarnings("unchecked") + private static final ConstructingObjectParser PARSER = + new ConstructingObjectParser<>( + "get_data_frame_analytics_stats_response", true, + args -> new GetDataFrameAnalyticsStatsResponse( + (List) args[0], + (List) args[1], + (List) args[2])); + + static { + PARSER.declareObjectArray(constructorArg(), (p, c) -> DataFrameAnalyticsStats.fromXContent(p), DATA_FRAME_ANALYTICS); + PARSER.declareObjectArray( + optionalConstructorArg(), (p, c) -> TaskOperationFailure.fromXContent(p), AcknowledgedTasksResponse.TASK_FAILURES); + PARSER.declareObjectArray( + optionalConstructorArg(), (p, c) -> ElasticsearchException.fromXContent(p), AcknowledgedTasksResponse.NODE_FAILURES); + } + + private final List analyticsStats; + private final List taskFailures; + private final List nodeFailures; + + public GetDataFrameAnalyticsStatsResponse(List analyticsStats, + @Nullable List taskFailures, + @Nullable List nodeFailures) { + this.analyticsStats = analyticsStats; + this.taskFailures = taskFailures == null ? Collections.emptyList() : Collections.unmodifiableList(taskFailures); + this.nodeFailures = nodeFailures == null ? Collections.emptyList() : Collections.unmodifiableList(nodeFailures); + } + + public List getAnalyticsStats() { + return analyticsStats; + } + + public List getNodeFailures() { + return nodeFailures; + } + + public List getTaskFailures() { + return taskFailures; + } + + @Override + public boolean equals(Object o) { + if (this == o) return true; + if (o == null || getClass() != o.getClass()) return false; + + GetDataFrameAnalyticsStatsResponse other = (GetDataFrameAnalyticsStatsResponse) o; + return Objects.equals(analyticsStats, other.analyticsStats) + && Objects.equals(nodeFailures, other.nodeFailures) + && Objects.equals(taskFailures, other.taskFailures); + } + + @Override + public int hashCode() { + return Objects.hash(analyticsStats, nodeFailures, taskFailures); + } +} diff --git a/client/rest-high-level/src/main/java/org/elasticsearch/client/ml/NodeAttributes.java b/client/rest-high-level/src/main/java/org/elasticsearch/client/ml/NodeAttributes.java index 892df340abd6b..a0f0d25f2ca01 100644 --- a/client/rest-high-level/src/main/java/org/elasticsearch/client/ml/NodeAttributes.java +++ b/client/rest-high-level/src/main/java/org/elasticsearch/client/ml/NodeAttributes.java @@ -19,6 +19,7 @@ package org.elasticsearch.client.ml; import org.elasticsearch.common.ParseField; +import org.elasticsearch.common.Strings; import org.elasticsearch.common.xcontent.ConstructingObjectParser; import org.elasticsearch.common.xcontent.ObjectParser; import org.elasticsearch.common.xcontent.ToXContentObject; @@ -147,4 +148,9 @@ public boolean equals(Object other) { Objects.equals(transportAddress, that.transportAddress) && Objects.equals(attributes, that.attributes); } + + @Override + public String toString() { + return Strings.toString(this); + } } diff --git a/client/rest-high-level/src/main/java/org/elasticsearch/client/ml/PutDataFrameAnalyticsRequest.java b/client/rest-high-level/src/main/java/org/elasticsearch/client/ml/PutDataFrameAnalyticsRequest.java new file mode 100644 index 0000000000000..2624b68a98318 --- /dev/null +++ b/client/rest-high-level/src/main/java/org/elasticsearch/client/ml/PutDataFrameAnalyticsRequest.java @@ -0,0 +1,76 @@ +/* + * 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.client.ml; + +import org.elasticsearch.client.Validatable; +import org.elasticsearch.client.ValidationException; +import org.elasticsearch.client.ml.dataframe.DataFrameAnalyticsConfig; +import org.elasticsearch.common.Strings; +import org.elasticsearch.common.xcontent.ToXContentObject; +import org.elasticsearch.common.xcontent.XContentBuilder; + +import java.io.IOException; +import java.util.Objects; +import java.util.Optional; + +public class PutDataFrameAnalyticsRequest implements ToXContentObject, Validatable { + + private final DataFrameAnalyticsConfig config; + + public PutDataFrameAnalyticsRequest(DataFrameAnalyticsConfig config) { + this.config = config; + } + + public DataFrameAnalyticsConfig getConfig() { + return config; + } + + @Override + public Optional validate() { + if (config == null) { + return Optional.of(ValidationException.withError("put requires a non-null data frame analytics config")); + } + return Optional.empty(); + } + + @Override + public XContentBuilder toXContent(XContentBuilder builder, Params params) throws IOException { + return config.toXContent(builder, params); + } + + @Override + public boolean equals(Object o) { + if (this == o) return true; + if (o == null || getClass() != o.getClass()) return false; + + PutDataFrameAnalyticsRequest other = (PutDataFrameAnalyticsRequest) o; + return Objects.equals(config, other.config); + } + + @Override + public int hashCode() { + return Objects.hash(config); + } + + @Override + public String toString() { + return Strings.toString(this); + } +} diff --git a/client/rest-high-level/src/main/java/org/elasticsearch/client/ml/PutDataFrameAnalyticsResponse.java b/client/rest-high-level/src/main/java/org/elasticsearch/client/ml/PutDataFrameAnalyticsResponse.java new file mode 100644 index 0000000000000..e6c4be15987d4 --- /dev/null +++ b/client/rest-high-level/src/main/java/org/elasticsearch/client/ml/PutDataFrameAnalyticsResponse.java @@ -0,0 +1,57 @@ +/* + * 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.client.ml; + +import org.elasticsearch.client.ml.dataframe.DataFrameAnalyticsConfig; +import org.elasticsearch.common.xcontent.XContentParser; + +import java.io.IOException; +import java.util.Objects; + +public class PutDataFrameAnalyticsResponse { + + public static PutDataFrameAnalyticsResponse fromXContent(XContentParser parser) throws IOException { + return new PutDataFrameAnalyticsResponse(DataFrameAnalyticsConfig.fromXContent(parser)); + } + + private final DataFrameAnalyticsConfig config; + + public PutDataFrameAnalyticsResponse(DataFrameAnalyticsConfig config) { + this.config = config; + } + + public DataFrameAnalyticsConfig getConfig() { + return config; + } + + @Override + public boolean equals(Object o) { + if (this == o) return true; + if (o == null || getClass() != o.getClass()) return false; + + PutDataFrameAnalyticsResponse other = (PutDataFrameAnalyticsResponse) o; + return Objects.equals(config, other.config); + } + + @Override + public int hashCode() { + return Objects.hash(config); + } +} diff --git a/client/rest-high-level/src/main/java/org/elasticsearch/client/ml/StartDataFrameAnalyticsRequest.java b/client/rest-high-level/src/main/java/org/elasticsearch/client/ml/StartDataFrameAnalyticsRequest.java new file mode 100644 index 0000000000000..68a925d15019a --- /dev/null +++ b/client/rest-high-level/src/main/java/org/elasticsearch/client/ml/StartDataFrameAnalyticsRequest.java @@ -0,0 +1,74 @@ +/* + * 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.client.ml; + +import org.elasticsearch.client.Validatable; +import org.elasticsearch.client.ValidationException; +import org.elasticsearch.common.Nullable; +import org.elasticsearch.common.unit.TimeValue; + +import java.util.Objects; +import java.util.Optional; + +public class StartDataFrameAnalyticsRequest implements Validatable { + + private final String id; + private TimeValue timeout; + + public StartDataFrameAnalyticsRequest(String id) { + this.id = id; + } + + public String getId() { + return id; + } + + public TimeValue getTimeout() { + return timeout; + } + + public StartDataFrameAnalyticsRequest setTimeout(@Nullable TimeValue timeout) { + this.timeout = timeout; + return this; + } + + @Override + public Optional validate() { + if (id == null) { + return Optional.of(ValidationException.withError("data frame analytics id must not be null")); + } + return Optional.empty(); + } + + @Override + public boolean equals(Object o) { + if (this == o) return true; + if (o == null || getClass() != o.getClass()) return false; + + StartDataFrameAnalyticsRequest other = (StartDataFrameAnalyticsRequest) o; + return Objects.equals(id, other.id) + && Objects.equals(timeout, other.timeout); + } + + @Override + public int hashCode() { + return Objects.hash(id, timeout); + } +} diff --git a/client/rest-high-level/src/main/java/org/elasticsearch/client/ml/StopDataFrameAnalyticsRequest.java b/client/rest-high-level/src/main/java/org/elasticsearch/client/ml/StopDataFrameAnalyticsRequest.java new file mode 100644 index 0000000000000..4ba6af852f61c --- /dev/null +++ b/client/rest-high-level/src/main/java/org/elasticsearch/client/ml/StopDataFrameAnalyticsRequest.java @@ -0,0 +1,100 @@ +/* + * 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.client.ml; + +import org.elasticsearch.client.Validatable; +import org.elasticsearch.client.ValidationException; +import org.elasticsearch.common.Nullable; +import org.elasticsearch.common.ParseField; +import org.elasticsearch.common.unit.TimeValue; + +import java.util.Objects; +import java.util.Optional; + +public class StopDataFrameAnalyticsRequest implements Validatable { + + public static final ParseField ALLOW_NO_MATCH = new ParseField("allow_no_match"); + public static final ParseField FORCE = new ParseField("force"); + + private final String id; + private Boolean allowNoMatch; + private Boolean force; + private TimeValue timeout; + + public StopDataFrameAnalyticsRequest(String id) { + this.id = id; + } + + public String getId() { + return id; + } + + public TimeValue getTimeout() { + return timeout; + } + + public StopDataFrameAnalyticsRequest setTimeout(@Nullable TimeValue timeout) { + this.timeout = timeout; + return this; + } + + public Boolean getAllowNoMatch() { + return allowNoMatch; + } + + public StopDataFrameAnalyticsRequest setAllowNoMatch(boolean allowNoMatch) { + this.allowNoMatch = allowNoMatch; + return this; + } + + public Boolean getForce() { + return force; + } + + public StopDataFrameAnalyticsRequest setForce(boolean force) { + this.force = force; + return this; + } + + @Override + public Optional validate() { + if (id == null) { + return Optional.of(ValidationException.withError("data frame analytics id must not be null")); + } + return Optional.empty(); + } + + @Override + public boolean equals(Object o) { + if (this == o) return true; + if (o == null || getClass() != o.getClass()) return false; + + StopDataFrameAnalyticsRequest other = (StopDataFrameAnalyticsRequest) o; + return Objects.equals(id, other.id) + && Objects.equals(timeout, other.timeout) + && Objects.equals(allowNoMatch, other.allowNoMatch) + && Objects.equals(force, other.force); + } + + @Override + public int hashCode() { + return Objects.hash(id, timeout, allowNoMatch, force); + } +} diff --git a/client/rest-high-level/src/main/java/org/elasticsearch/client/ml/StopDataFrameAnalyticsResponse.java b/client/rest-high-level/src/main/java/org/elasticsearch/client/ml/StopDataFrameAnalyticsResponse.java new file mode 100644 index 0000000000000..5f45c6f9ea51f --- /dev/null +++ b/client/rest-high-level/src/main/java/org/elasticsearch/client/ml/StopDataFrameAnalyticsResponse.java @@ -0,0 +1,87 @@ +/* + * 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.client.ml; + +import org.elasticsearch.common.ParseField; +import org.elasticsearch.common.xcontent.ConstructingObjectParser; +import org.elasticsearch.common.xcontent.ToXContentObject; +import org.elasticsearch.common.xcontent.XContentBuilder; +import org.elasticsearch.common.xcontent.XContentParser; + +import java.io.IOException; +import java.util.Objects; + +/** + * Response indicating if the Machine Learning Data Frame Analytics is now stopped or not + */ +public class StopDataFrameAnalyticsResponse implements ToXContentObject { + + private static final ParseField STOPPED = new ParseField("stopped"); + + public static final ConstructingObjectParser PARSER = + new ConstructingObjectParser<>( + "stop_data_frame_analytics_response", + true, + args -> new StopDataFrameAnalyticsResponse((Boolean) args[0])); + + static { + PARSER.declareBoolean(ConstructingObjectParser.constructorArg(), STOPPED); + } + + public static StopDataFrameAnalyticsResponse fromXContent(XContentParser parser) throws IOException { + return PARSER.parse(parser, null); + } + + private final boolean stopped; + + public StopDataFrameAnalyticsResponse(boolean stopped) { + this.stopped = stopped; + } + + /** + * Has the Data Frame Analytics stopped or not + * + * @return boolean value indicating the Data Frame Analytics stopped status + */ + public boolean isStopped() { + return stopped; + } + + @Override + public boolean equals(Object o) { + if (this == o) return true; + if (o == null || getClass() != o.getClass()) return false; + + StopDataFrameAnalyticsResponse other = (StopDataFrameAnalyticsResponse) o; + return stopped == other.stopped; + } + + @Override + public int hashCode() { + return Objects.hash(stopped); + } + + @Override + public XContentBuilder toXContent(XContentBuilder builder, Params params) throws IOException { + return builder + .startObject() + .field(STOPPED.getPreferredName(), stopped) + .endObject(); + } +} diff --git a/client/rest-high-level/src/main/java/org/elasticsearch/client/ml/dataframe/DataFrameAnalysis.java b/client/rest-high-level/src/main/java/org/elasticsearch/client/ml/dataframe/DataFrameAnalysis.java new file mode 100644 index 0000000000000..81b19eefce573 --- /dev/null +++ b/client/rest-high-level/src/main/java/org/elasticsearch/client/ml/dataframe/DataFrameAnalysis.java @@ -0,0 +1,27 @@ +/* + * 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.client.ml.dataframe; + +import org.elasticsearch.common.xcontent.ToXContentObject; + +public interface DataFrameAnalysis extends ToXContentObject { + + String getName(); +} diff --git a/client/rest-high-level/src/main/java/org/elasticsearch/client/ml/dataframe/DataFrameAnalyticsConfig.java b/client/rest-high-level/src/main/java/org/elasticsearch/client/ml/dataframe/DataFrameAnalyticsConfig.java new file mode 100644 index 0000000000000..62adb06294558 --- /dev/null +++ b/client/rest-high-level/src/main/java/org/elasticsearch/client/ml/dataframe/DataFrameAnalyticsConfig.java @@ -0,0 +1,259 @@ +/* + * 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.client.ml.dataframe; + +import org.elasticsearch.Version; +import org.elasticsearch.client.dataframe.transforms.util.TimeUtil; +import org.elasticsearch.common.Nullable; +import org.elasticsearch.common.ParseField; +import org.elasticsearch.common.Strings; +import org.elasticsearch.common.unit.ByteSizeValue; +import org.elasticsearch.common.xcontent.ObjectParser; +import org.elasticsearch.common.xcontent.ObjectParser.ValueType; +import org.elasticsearch.common.xcontent.ToXContentObject; +import org.elasticsearch.common.xcontent.XContentBuilder; +import org.elasticsearch.common.xcontent.XContentParser; +import org.elasticsearch.common.xcontent.XContentParserUtils; +import org.elasticsearch.search.fetch.subphase.FetchSourceContext; + +import java.io.IOException; +import java.time.Instant; +import java.util.Objects; + +public class DataFrameAnalyticsConfig implements ToXContentObject { + + public static DataFrameAnalyticsConfig fromXContent(XContentParser parser) { + return PARSER.apply(parser, null).build(); + } + + public static Builder builder(String id) { + return new Builder().setId(id); + } + + private static final ParseField ID = new ParseField("id"); + private static final ParseField SOURCE = new ParseField("source"); + private static final ParseField DEST = new ParseField("dest"); + private static final ParseField ANALYSIS = new ParseField("analysis"); + private static final ParseField ANALYZED_FIELDS = new ParseField("analyzed_fields"); + private static final ParseField MODEL_MEMORY_LIMIT = new ParseField("model_memory_limit"); + private static final ParseField CREATE_TIME = new ParseField("create_time"); + private static final ParseField VERSION = new ParseField("version"); + + private static ObjectParser PARSER = new ObjectParser<>("data_frame_analytics_config", true, Builder::new); + + static { + PARSER.declareString(Builder::setId, ID); + PARSER.declareObject(Builder::setSource, (p, c) -> DataFrameAnalyticsSource.fromXContent(p), SOURCE); + PARSER.declareObject(Builder::setDest, (p, c) -> DataFrameAnalyticsDest.fromXContent(p), DEST); + PARSER.declareObject(Builder::setAnalysis, (p, c) -> parseAnalysis(p), ANALYSIS); + PARSER.declareField(Builder::setAnalyzedFields, + (p, c) -> FetchSourceContext.fromXContent(p), + ANALYZED_FIELDS, + ValueType.OBJECT_ARRAY_BOOLEAN_OR_STRING); + PARSER.declareField(Builder::setModelMemoryLimit, + (p, c) -> ByteSizeValue.parseBytesSizeValue(p.text(), MODEL_MEMORY_LIMIT.getPreferredName()), + MODEL_MEMORY_LIMIT, + ValueType.VALUE); + PARSER.declareField(Builder::setCreateTime, + p -> TimeUtil.parseTimeFieldToInstant(p, CREATE_TIME.getPreferredName()), + CREATE_TIME, + ValueType.VALUE); + PARSER.declareField(Builder::setVersion, + p -> { + if (p.currentToken() == XContentParser.Token.VALUE_STRING) { + return Version.fromString(p.text()); + } + throw new IllegalArgumentException("Unsupported token [" + p.currentToken() + "]"); + }, + VERSION, + ValueType.STRING); + } + + private static DataFrameAnalysis parseAnalysis(XContentParser parser) throws IOException { + XContentParserUtils.ensureExpectedToken(XContentParser.Token.START_OBJECT, parser.currentToken(), parser::getTokenLocation); + XContentParserUtils.ensureExpectedToken(XContentParser.Token.FIELD_NAME, parser.nextToken(), parser::getTokenLocation); + DataFrameAnalysis analysis = parser.namedObject(DataFrameAnalysis.class, parser.currentName(), true); + XContentParserUtils.ensureExpectedToken(XContentParser.Token.END_OBJECT, parser.nextToken(), parser::getTokenLocation); + return analysis; + } + + private final String id; + private final DataFrameAnalyticsSource source; + private final DataFrameAnalyticsDest dest; + private final DataFrameAnalysis analysis; + private final FetchSourceContext analyzedFields; + private final ByteSizeValue modelMemoryLimit; + private final Instant createTime; + private final Version version; + + private DataFrameAnalyticsConfig(String id, DataFrameAnalyticsSource source, DataFrameAnalyticsDest dest, DataFrameAnalysis analysis, + @Nullable FetchSourceContext analyzedFields, @Nullable ByteSizeValue modelMemoryLimit, + @Nullable Instant createTime, @Nullable Version version) { + this.id = Objects.requireNonNull(id); + this.source = Objects.requireNonNull(source); + this.dest = Objects.requireNonNull(dest); + this.analysis = Objects.requireNonNull(analysis); + this.analyzedFields = analyzedFields; + this.modelMemoryLimit = modelMemoryLimit; + this.createTime = createTime == null ? null : Instant.ofEpochMilli(createTime.toEpochMilli());; + this.version = version; + } + + public String getId() { + return id; + } + + public DataFrameAnalyticsSource getSource() { + return source; + } + + public DataFrameAnalyticsDest getDest() { + return dest; + } + + public DataFrameAnalysis getAnalysis() { + return analysis; + } + + public FetchSourceContext getAnalyzedFields() { + return analyzedFields; + } + + public ByteSizeValue getModelMemoryLimit() { + return modelMemoryLimit; + } + + public Instant getCreateTime() { + return createTime; + } + + public Version getVersion() { + return version; + } + + @Override + public XContentBuilder toXContent(XContentBuilder builder, Params params) throws IOException { + builder.startObject(); + builder.field(ID.getPreferredName(), id); + builder.field(SOURCE.getPreferredName(), source); + builder.field(DEST.getPreferredName(), dest); + builder.startObject(ANALYSIS.getPreferredName()); + builder.field(analysis.getName(), analysis); + builder.endObject(); + if (analyzedFields != null) { + builder.field(ANALYZED_FIELDS.getPreferredName(), analyzedFields); + } + if (modelMemoryLimit != null) { + builder.field(MODEL_MEMORY_LIMIT.getPreferredName(), modelMemoryLimit.getStringRep()); + } + if (createTime != null) { + builder.timeField(CREATE_TIME.getPreferredName(), CREATE_TIME.getPreferredName() + "_string", createTime.toEpochMilli()); + } + if (version != null) { + builder.field(VERSION.getPreferredName(), version); + } + builder.endObject(); + return builder; + } + + @Override + public boolean equals(Object o) { + if (o == this) return true; + if (o == null || getClass() != o.getClass()) return false; + + DataFrameAnalyticsConfig other = (DataFrameAnalyticsConfig) o; + return Objects.equals(id, other.id) + && Objects.equals(source, other.source) + && Objects.equals(dest, other.dest) + && Objects.equals(analysis, other.analysis) + && Objects.equals(analyzedFields, other.analyzedFields) + && Objects.equals(modelMemoryLimit, other.modelMemoryLimit) + && Objects.equals(createTime, other.createTime) + && Objects.equals(version, other.version); + } + + @Override + public int hashCode() { + return Objects.hash(id, source, dest, analysis, analyzedFields, modelMemoryLimit, createTime, version); + } + + @Override + public String toString() { + return Strings.toString(this); + } + + public static class Builder { + + private String id; + private DataFrameAnalyticsSource source; + private DataFrameAnalyticsDest dest; + private DataFrameAnalysis analysis; + private FetchSourceContext analyzedFields; + private ByteSizeValue modelMemoryLimit; + private Instant createTime; + private Version version; + + private Builder() {} + + public Builder setId(String id) { + this.id = Objects.requireNonNull(id); + return this; + } + + public Builder setSource(DataFrameAnalyticsSource source) { + this.source = Objects.requireNonNull(source); + return this; + } + + public Builder setDest(DataFrameAnalyticsDest dest) { + this.dest = Objects.requireNonNull(dest); + return this; + } + + public Builder setAnalysis(DataFrameAnalysis analysis) { + this.analysis = Objects.requireNonNull(analysis); + return this; + } + + public Builder setAnalyzedFields(FetchSourceContext fields) { + this.analyzedFields = fields; + return this; + } + + public Builder setModelMemoryLimit(ByteSizeValue modelMemoryLimit) { + this.modelMemoryLimit = modelMemoryLimit; + return this; + } + + public Builder setCreateTime(Instant createTime) { + this.createTime = createTime; + return this; + } + + public Builder setVersion(Version version) { + this.version = version; + return this; + } + + public DataFrameAnalyticsConfig build() { + return new DataFrameAnalyticsConfig(id, source, dest, analysis, analyzedFields, modelMemoryLimit, createTime, version); + } + } +} diff --git a/client/rest-high-level/src/main/java/org/elasticsearch/client/ml/dataframe/DataFrameAnalyticsDest.java b/client/rest-high-level/src/main/java/org/elasticsearch/client/ml/dataframe/DataFrameAnalyticsDest.java new file mode 100644 index 0000000000000..4123f85ee2f43 --- /dev/null +++ b/client/rest-high-level/src/main/java/org/elasticsearch/client/ml/dataframe/DataFrameAnalyticsDest.java @@ -0,0 +1,123 @@ +/* + * 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.client.ml.dataframe; + +import org.elasticsearch.common.Nullable; +import org.elasticsearch.common.ParseField; +import org.elasticsearch.common.Strings; +import org.elasticsearch.common.xcontent.ObjectParser; +import org.elasticsearch.common.xcontent.ToXContentObject; +import org.elasticsearch.common.xcontent.XContentBuilder; +import org.elasticsearch.common.xcontent.XContentParser; + +import java.io.IOException; +import java.util.Objects; + +import static java.util.Objects.requireNonNull; + +public class DataFrameAnalyticsDest implements ToXContentObject { + + public static DataFrameAnalyticsDest fromXContent(XContentParser parser) { + return PARSER.apply(parser, null).build(); + } + + public static Builder builder() { + return new Builder(); + } + + private static final ParseField INDEX = new ParseField("index"); + private static final ParseField RESULTS_FIELD = new ParseField("results_field"); + + private static ObjectParser PARSER = new ObjectParser<>("data_frame_analytics_dest", true, Builder::new); + + static { + PARSER.declareString(Builder::setIndex, INDEX); + PARSER.declareString(Builder::setResultsField, RESULTS_FIELD); + } + + private final String index; + private final String resultsField; + + private DataFrameAnalyticsDest(String index, @Nullable String resultsField) { + this.index = requireNonNull(index); + this.resultsField = resultsField; + } + + public String getIndex() { + return index; + } + + public String getResultsField() { + return resultsField; + } + + @Override + public XContentBuilder toXContent(XContentBuilder builder, Params params) throws IOException { + builder.startObject(); + builder.field(INDEX.getPreferredName(), index); + if (resultsField != null) { + builder.field(RESULTS_FIELD.getPreferredName(), resultsField); + } + builder.endObject(); + return builder; + } + + @Override + public boolean equals(Object o) { + if (o == this) return true; + if (o == null || getClass() != o.getClass()) return false; + + DataFrameAnalyticsDest other = (DataFrameAnalyticsDest) o; + return Objects.equals(index, other.index) + && Objects.equals(resultsField, other.resultsField); + } + + @Override + public int hashCode() { + return Objects.hash(index, resultsField); + } + + @Override + public String toString() { + return Strings.toString(this); + } + + public static class Builder { + + private String index; + private String resultsField; + + private Builder() {} + + public Builder setIndex(String index) { + this.index = index; + return this; + } + + public Builder setResultsField(String resultsField) { + this.resultsField = resultsField; + return this; + } + + public DataFrameAnalyticsDest build() { + return new DataFrameAnalyticsDest(index, resultsField); + } + } +} diff --git a/client/rest-high-level/src/main/java/org/elasticsearch/client/ml/dataframe/DataFrameAnalyticsSource.java b/client/rest-high-level/src/main/java/org/elasticsearch/client/ml/dataframe/DataFrameAnalyticsSource.java new file mode 100644 index 0000000000000..9a6de159bea3e --- /dev/null +++ b/client/rest-high-level/src/main/java/org/elasticsearch/client/ml/dataframe/DataFrameAnalyticsSource.java @@ -0,0 +1,128 @@ +/* + * 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.client.ml.dataframe; + +import org.elasticsearch.common.Nullable; +import org.elasticsearch.common.ParseField; +import org.elasticsearch.common.Strings; +import org.elasticsearch.common.xcontent.ObjectParser; +import org.elasticsearch.common.xcontent.ToXContentObject; +import org.elasticsearch.common.xcontent.XContentBuilder; +import org.elasticsearch.common.xcontent.XContentParser; + +import java.io.IOException; +import java.util.Arrays; +import java.util.List; +import java.util.Objects; + +public class DataFrameAnalyticsSource implements ToXContentObject { + + public static DataFrameAnalyticsSource fromXContent(XContentParser parser) { + return PARSER.apply(parser, null).build(); + } + + public static Builder builder() { + return new Builder(); + } + + private static final ParseField INDEX = new ParseField("index"); + private static final ParseField QUERY = new ParseField("query"); + + private static ObjectParser PARSER = new ObjectParser<>("data_frame_analytics_source", true, Builder::new); + + static { + PARSER.declareStringArray(Builder::setIndex, INDEX); + PARSER.declareObject(Builder::setQueryConfig, (p, c) -> QueryConfig.fromXContent(p), QUERY); + } + + private final String[] index; + private final QueryConfig queryConfig; + + private DataFrameAnalyticsSource(String[] index, @Nullable QueryConfig queryConfig) { + this.index = Objects.requireNonNull(index); + this.queryConfig = queryConfig; + } + + public String[] getIndex() { + return index; + } + + public QueryConfig getQueryConfig() { + return queryConfig; + } + + @Override + public XContentBuilder toXContent(XContentBuilder builder, Params params) throws IOException { + builder.startObject(); + builder.field(INDEX.getPreferredName(), index); + if (queryConfig != null) { + builder.field(QUERY.getPreferredName(), queryConfig.getQuery()); + } + builder.endObject(); + return builder; + } + + @Override + public boolean equals(Object o) { + if (o == this) return true; + if (o == null || getClass() != o.getClass()) return false; + + DataFrameAnalyticsSource other = (DataFrameAnalyticsSource) o; + return Arrays.equals(index, other.index) + && Objects.equals(queryConfig, other.queryConfig); + } + + @Override + public int hashCode() { + return Objects.hash(Arrays.asList(index), queryConfig); + } + + @Override + public String toString() { + return Strings.toString(this); + } + + public static class Builder { + + private String[] index; + private QueryConfig queryConfig; + + private Builder() {} + + public Builder setIndex(String... index) { + this.index = index; + return this; + } + + public Builder setIndex(List index) { + this.index = index.toArray(new String[0]); + return this; + } + + public Builder setQueryConfig(QueryConfig queryConfig) { + this.queryConfig = queryConfig; + return this; + } + + public DataFrameAnalyticsSource build() { + return new DataFrameAnalyticsSource(index, queryConfig); + } + } +} diff --git a/client/rest-high-level/src/main/java/org/elasticsearch/client/ml/dataframe/DataFrameAnalyticsState.java b/client/rest-high-level/src/main/java/org/elasticsearch/client/ml/dataframe/DataFrameAnalyticsState.java new file mode 100644 index 0000000000000..6ee349b8e8d38 --- /dev/null +++ b/client/rest-high-level/src/main/java/org/elasticsearch/client/ml/dataframe/DataFrameAnalyticsState.java @@ -0,0 +1,34 @@ +/* + * 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.client.ml.dataframe; + +import java.util.Locale; + +public enum DataFrameAnalyticsState { + STARTED, REINDEXING, ANALYZING, STOPPING, STOPPED; + + public static DataFrameAnalyticsState fromString(String name) { + return valueOf(name.trim().toUpperCase(Locale.ROOT)); + } + + public String value() { + return name().toLowerCase(Locale.ROOT); + } +} diff --git a/client/rest-high-level/src/main/java/org/elasticsearch/client/ml/dataframe/DataFrameAnalyticsStats.java b/client/rest-high-level/src/main/java/org/elasticsearch/client/ml/dataframe/DataFrameAnalyticsStats.java new file mode 100644 index 0000000000000..4e04204e65021 --- /dev/null +++ b/client/rest-high-level/src/main/java/org/elasticsearch/client/ml/dataframe/DataFrameAnalyticsStats.java @@ -0,0 +1,145 @@ +/* + * 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.client.ml.dataframe; + +import org.elasticsearch.client.ml.NodeAttributes; +import org.elasticsearch.common.Nullable; +import org.elasticsearch.common.ParseField; +import org.elasticsearch.common.inject.internal.ToStringBuilder; +import org.elasticsearch.common.xcontent.ConstructingObjectParser; +import org.elasticsearch.common.xcontent.ObjectParser; +import org.elasticsearch.common.xcontent.XContentParser; + +import java.io.IOException; +import java.util.Objects; + +import static org.elasticsearch.common.xcontent.ConstructingObjectParser.constructorArg; +import static org.elasticsearch.common.xcontent.ConstructingObjectParser.optionalConstructorArg; + +public class DataFrameAnalyticsStats { + + public static DataFrameAnalyticsStats fromXContent(XContentParser parser) throws IOException { + return PARSER.parse(parser, null); + } + + static final ParseField ID = new ParseField("id"); + static final ParseField STATE = new ParseField("state"); + static final ParseField FAILURE_REASON = new ParseField("failure_reason"); + static final ParseField PROGRESS_PERCENT = new ParseField("progress_percent"); + static final ParseField NODE = new ParseField("node"); + static final ParseField ASSIGNMENT_EXPLANATION = new ParseField("assignment_explanation"); + + private static final ConstructingObjectParser PARSER = + new ConstructingObjectParser<>("data_frame_analytics_stats", true, + args -> new DataFrameAnalyticsStats( + (String) args[0], + (DataFrameAnalyticsState) args[1], + (String) args[2], + (Integer) args[3], + (NodeAttributes) args[4], + (String) args[5])); + + static { + PARSER.declareString(constructorArg(), ID); + PARSER.declareField(constructorArg(), p -> { + if (p.currentToken() == XContentParser.Token.VALUE_STRING) { + return DataFrameAnalyticsState.fromString(p.text()); + } + throw new IllegalArgumentException("Unsupported token [" + p.currentToken() + "]"); + }, STATE, ObjectParser.ValueType.STRING); + PARSER.declareString(optionalConstructorArg(), FAILURE_REASON); + PARSER.declareInt(optionalConstructorArg(), PROGRESS_PERCENT); + PARSER.declareObject(optionalConstructorArg(), NodeAttributes.PARSER, NODE); + PARSER.declareString(optionalConstructorArg(), ASSIGNMENT_EXPLANATION); + } + + private final String id; + private final DataFrameAnalyticsState state; + private final String failureReason; + private final Integer progressPercent; + private final NodeAttributes node; + private final String assignmentExplanation; + + public DataFrameAnalyticsStats(String id, DataFrameAnalyticsState state, @Nullable String failureReason, + @Nullable Integer progressPercent, @Nullable NodeAttributes node, + @Nullable String assignmentExplanation) { + this.id = id; + this.state = state; + this.failureReason = failureReason; + this.progressPercent = progressPercent; + this.node = node; + this.assignmentExplanation = assignmentExplanation; + } + + public String getId() { + return id; + } + + public DataFrameAnalyticsState getState() { + return state; + } + + public String getFailureReason() { + return failureReason; + } + + public Integer getProgressPercent() { + return progressPercent; + } + + public NodeAttributes getNode() { + return node; + } + + public String getAssignmentExplanation() { + return assignmentExplanation; + } + + @Override + public boolean equals(Object o) { + if (this == o) return true; + if (o == null || getClass() != o.getClass()) return false; + + DataFrameAnalyticsStats other = (DataFrameAnalyticsStats) o; + return Objects.equals(id, other.id) + && Objects.equals(state, other.state) + && Objects.equals(failureReason, other.failureReason) + && Objects.equals(progressPercent, other.progressPercent) + && Objects.equals(node, other.node) + && Objects.equals(assignmentExplanation, other.assignmentExplanation); + } + + @Override + public int hashCode() { + return Objects.hash(id, state, failureReason, progressPercent, node, assignmentExplanation); + } + + @Override + public String toString() { + return new ToStringBuilder(getClass()) + .add("id", id) + .add("state", state) + .add("failureReason", failureReason) + .add("progressPercent", progressPercent) + .add("node", node) + .add("assignmentExplanation", assignmentExplanation) + .toString(); + } +} diff --git a/client/rest-high-level/src/main/java/org/elasticsearch/client/ml/dataframe/MlDataFrameAnalysisNamedXContentProvider.java b/client/rest-high-level/src/main/java/org/elasticsearch/client/ml/dataframe/MlDataFrameAnalysisNamedXContentProvider.java new file mode 100644 index 0000000000000..3b78c60be91fd --- /dev/null +++ b/client/rest-high-level/src/main/java/org/elasticsearch/client/ml/dataframe/MlDataFrameAnalysisNamedXContentProvider.java @@ -0,0 +1,37 @@ +/* + * 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.client.ml.dataframe; + +import org.elasticsearch.common.xcontent.NamedXContentRegistry; +import org.elasticsearch.plugins.spi.NamedXContentProvider; + +import java.util.Arrays; +import java.util.List; + +public class MlDataFrameAnalysisNamedXContentProvider implements NamedXContentProvider { + + @Override + public List getNamedXContentParsers() { + return Arrays.asList( + new NamedXContentRegistry.Entry( + DataFrameAnalysis.class, + OutlierDetection.NAME, + (p, c) -> OutlierDetection.fromXContent(p))); + } +} diff --git a/client/rest-high-level/src/main/java/org/elasticsearch/client/ml/dataframe/OutlierDetection.java b/client/rest-high-level/src/main/java/org/elasticsearch/client/ml/dataframe/OutlierDetection.java new file mode 100644 index 0000000000000..fe5094fb7190a --- /dev/null +++ b/client/rest-high-level/src/main/java/org/elasticsearch/client/ml/dataframe/OutlierDetection.java @@ -0,0 +1,175 @@ +/* + * 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.client.ml.dataframe; + +import org.elasticsearch.common.Nullable; +import org.elasticsearch.common.ParseField; +import org.elasticsearch.common.Strings; +import org.elasticsearch.common.xcontent.ObjectParser; +import org.elasticsearch.common.xcontent.XContentBuilder; +import org.elasticsearch.common.xcontent.XContentParser; + +import java.io.IOException; +import java.util.Locale; +import java.util.Objects; + +public class OutlierDetection implements DataFrameAnalysis { + + public static OutlierDetection fromXContent(XContentParser parser) { + return PARSER.apply(parser, null).build(); + } + + public static OutlierDetection createDefault() { + return builder().build(); + } + + public static Builder builder() { + return new Builder(); + } + + public static final ParseField NAME = new ParseField("outlier_detection"); + static final ParseField N_NEIGHBORS = new ParseField("n_neighbors"); + static final ParseField METHOD = new ParseField("method"); + public static final ParseField FEATURE_INFLUENCE_THRESHOLD = new ParseField("feature_influence_threshold"); + + private static ObjectParser PARSER = new ObjectParser<>(NAME.getPreferredName(), true, Builder::new); + + static { + PARSER.declareInt(Builder::setNNeighbors, N_NEIGHBORS); + PARSER.declareField(Builder::setMethod, p -> { + if (p.currentToken() == XContentParser.Token.VALUE_STRING) { + return Method.fromString(p.text()); + } + throw new IllegalArgumentException("Unsupported token [" + p.currentToken() + "]"); + }, METHOD, ObjectParser.ValueType.STRING); + PARSER.declareDouble(Builder::setFeatureInfluenceThreshold, FEATURE_INFLUENCE_THRESHOLD); + } + + private final Integer nNeighbors; + private final Method method; + private final Double featureInfluenceThreshold; + + /** + * Constructs the outlier detection configuration + * @param nNeighbors The number of neighbors. Leave unspecified for dynamic detection. + * @param method The method. Leave unspecified for a dynamic mixture of methods. + * @param featureInfluenceThreshold The min outlier score required to calculate feature influence. Defaults to 0.1. + */ + private OutlierDetection(@Nullable Integer nNeighbors, @Nullable Method method, @Nullable Double featureInfluenceThreshold) { + this.nNeighbors = nNeighbors; + this.method = method; + this.featureInfluenceThreshold = featureInfluenceThreshold; + } + + @Override + public String getName() { + return NAME.getPreferredName(); + } + + public Integer getNNeighbors() { + return nNeighbors; + } + + public Method getMethod() { + return method; + } + + public Double getFeatureInfluenceThreshold() { + return featureInfluenceThreshold; + } + + @Override + public XContentBuilder toXContent(XContentBuilder builder, Params params) throws IOException { + builder.startObject(); + if (nNeighbors != null) { + builder.field(N_NEIGHBORS.getPreferredName(), nNeighbors); + } + if (method != null) { + builder.field(METHOD.getPreferredName(), method); + } + if (featureInfluenceThreshold != null) { + builder.field(FEATURE_INFLUENCE_THRESHOLD.getPreferredName(), featureInfluenceThreshold); + } + builder.endObject(); + return builder; + } + + @Override + public boolean equals(Object o) { + if (this == o) return true; + if (o == null || getClass() != o.getClass()) return false; + + OutlierDetection other = (OutlierDetection) o; + return Objects.equals(nNeighbors, other.nNeighbors) + && Objects.equals(method, other.method) + && Objects.equals(featureInfluenceThreshold, other.featureInfluenceThreshold); + } + + @Override + public int hashCode() { + return Objects.hash(nNeighbors, method, featureInfluenceThreshold); + } + + @Override + public String toString() { + return Strings.toString(this); + } + + public enum Method { + LOF, LDOF, DISTANCE_KTH_NN, DISTANCE_KNN; + + public static Method fromString(String value) { + return Method.valueOf(value.toUpperCase(Locale.ROOT)); + } + + @Override + public String toString() { + return name().toLowerCase(Locale.ROOT); + } + } + + public static class Builder { + + private Integer nNeighbors; + private Method method; + private Double featureInfluenceThreshold; + + private Builder() {} + + public Builder setNNeighbors(Integer nNeighbors) { + this.nNeighbors = nNeighbors; + return this; + } + + public Builder setMethod(Method method) { + this.method = method; + return this; + } + + public Builder setFeatureInfluenceThreshold(Double featureInfluenceThreshold) { + this.featureInfluenceThreshold = featureInfluenceThreshold; + return this; + } + + public OutlierDetection build() { + return new OutlierDetection(nNeighbors, method, featureInfluenceThreshold); + } + } +} diff --git a/client/rest-high-level/src/main/java/org/elasticsearch/client/ml/dataframe/QueryConfig.java b/client/rest-high-level/src/main/java/org/elasticsearch/client/ml/dataframe/QueryConfig.java new file mode 100644 index 0000000000000..ae704db9f800e --- /dev/null +++ b/client/rest-high-level/src/main/java/org/elasticsearch/client/ml/dataframe/QueryConfig.java @@ -0,0 +1,82 @@ +/* + * 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.client.ml.dataframe; + +import org.elasticsearch.common.Strings; +import org.elasticsearch.common.xcontent.ToXContentObject; +import org.elasticsearch.common.xcontent.XContentBuilder; +import org.elasticsearch.common.xcontent.XContentParser; +import org.elasticsearch.index.query.AbstractQueryBuilder; +import org.elasticsearch.index.query.QueryBuilder; + +import java.io.IOException; +import java.util.Objects; + +import static java.util.Objects.requireNonNull; + +/** + * Object for encapsulating the desired Query for a DataFrameAnalysis + */ +public class QueryConfig implements ToXContentObject { + + public static QueryConfig fromXContent(XContentParser parser) throws IOException { + QueryBuilder query = AbstractQueryBuilder.parseInnerQueryBuilder(parser); + return new QueryConfig(query); + } + + private final QueryBuilder query; + + public QueryConfig(QueryBuilder query) { + this.query = requireNonNull(query); + } + + public QueryConfig(QueryConfig queryConfig) { + this(requireNonNull(queryConfig).query); + } + + @Override + public XContentBuilder toXContent(XContentBuilder builder, Params params) throws IOException { + query.toXContent(builder, params); + return builder; + } + + public QueryBuilder getQuery() { + return query; + } + + @Override + public boolean equals(Object o) { + if (this == o) return true; + if (o == null || getClass() != o.getClass()) return false; + + QueryConfig other = (QueryConfig) o; + return Objects.equals(query, other.query); + } + + @Override + public int hashCode() { + return Objects.hash(query); + } + + @Override + public String toString() { + return Strings.toString(this); + } +} diff --git a/client/rest-high-level/src/main/java/org/elasticsearch/client/ml/dataframe/evaluation/Evaluation.java b/client/rest-high-level/src/main/java/org/elasticsearch/client/ml/dataframe/evaluation/Evaluation.java new file mode 100644 index 0000000000000..78578597e195b --- /dev/null +++ b/client/rest-high-level/src/main/java/org/elasticsearch/client/ml/dataframe/evaluation/Evaluation.java @@ -0,0 +1,32 @@ +/* + * 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.client.ml.dataframe.evaluation; + +import org.elasticsearch.common.xcontent.ToXContentObject; + +/** + * Defines an evaluation + */ +public interface Evaluation extends ToXContentObject { + + /** + * Returns the evaluation name + */ + String getName(); +} diff --git a/server/src/main/java/org/elasticsearch/action/admin/indices/alias/exists/AliasesExistAction.java b/client/rest-high-level/src/main/java/org/elasticsearch/client/ml/dataframe/evaluation/EvaluationMetric.java similarity index 57% rename from server/src/main/java/org/elasticsearch/action/admin/indices/alias/exists/AliasesExistAction.java rename to client/rest-high-level/src/main/java/org/elasticsearch/client/ml/dataframe/evaluation/EvaluationMetric.java index dfaebab076c66..a0f77838f1fd0 100644 --- a/server/src/main/java/org/elasticsearch/action/admin/indices/alias/exists/AliasesExistAction.java +++ b/client/rest-high-level/src/main/java/org/elasticsearch/client/ml/dataframe/evaluation/EvaluationMetric.java @@ -7,7 +7,7 @@ * 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 + * 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 @@ -16,22 +16,28 @@ * specific language governing permissions and limitations * under the License. */ +package org.elasticsearch.client.ml.dataframe.evaluation; -package org.elasticsearch.action.admin.indices.alias.exists; +import org.elasticsearch.common.xcontent.ToXContentObject; -import org.elasticsearch.action.Action; - -public class AliasesExistAction extends Action { +/** + * Defines an evaluation metric + */ +public interface EvaluationMetric extends ToXContentObject { - public static final AliasesExistAction INSTANCE = new AliasesExistAction(); - public static final String NAME = "indices:admin/aliases/exists"; + /** + * Returns the name of the metric + */ + String getName(); - private AliasesExistAction() { - super(NAME); - } + /** + * The result of an evaluation metric + */ + interface Result extends ToXContentObject { - @Override - public AliasesExistResponse newResponse() { - return new AliasesExistResponse(); + /** + * Returns the name of the metric + */ + String getMetricName(); } } diff --git a/client/rest-high-level/src/main/java/org/elasticsearch/client/ml/dataframe/evaluation/MlEvaluationNamedXContentProvider.java b/client/rest-high-level/src/main/java/org/elasticsearch/client/ml/dataframe/evaluation/MlEvaluationNamedXContentProvider.java new file mode 100644 index 0000000000000..764ff41de86e0 --- /dev/null +++ b/client/rest-high-level/src/main/java/org/elasticsearch/client/ml/dataframe/evaluation/MlEvaluationNamedXContentProvider.java @@ -0,0 +1,57 @@ +/* + * 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.client.ml.dataframe.evaluation; + +import org.elasticsearch.client.ml.dataframe.evaluation.softclassification.BinarySoftClassification; +import org.elasticsearch.common.ParseField; +import org.elasticsearch.common.xcontent.NamedXContentRegistry; +import org.elasticsearch.plugins.spi.NamedXContentProvider; +import org.elasticsearch.client.ml.dataframe.evaluation.softclassification.AucRocMetric; +import org.elasticsearch.client.ml.dataframe.evaluation.softclassification.ConfusionMatrixMetric; +import org.elasticsearch.client.ml.dataframe.evaluation.softclassification.PrecisionMetric; +import org.elasticsearch.client.ml.dataframe.evaluation.softclassification.RecallMetric; + +import java.util.Arrays; +import java.util.List; + +public class MlEvaluationNamedXContentProvider implements NamedXContentProvider { + + @Override + public List getNamedXContentParsers() { + return Arrays.asList( + // Evaluations + new NamedXContentRegistry.Entry( + Evaluation.class, new ParseField(BinarySoftClassification.NAME), BinarySoftClassification::fromXContent), + // Evaluation metrics + new NamedXContentRegistry.Entry(EvaluationMetric.class, new ParseField(AucRocMetric.NAME), AucRocMetric::fromXContent), + new NamedXContentRegistry.Entry(EvaluationMetric.class, new ParseField(PrecisionMetric.NAME), PrecisionMetric::fromXContent), + new NamedXContentRegistry.Entry(EvaluationMetric.class, new ParseField(RecallMetric.NAME), RecallMetric::fromXContent), + new NamedXContentRegistry.Entry( + EvaluationMetric.class, new ParseField(ConfusionMatrixMetric.NAME), ConfusionMatrixMetric::fromXContent), + // Evaluation metrics results + new NamedXContentRegistry.Entry( + EvaluationMetric.Result.class, new ParseField(AucRocMetric.NAME), AucRocMetric.Result::fromXContent), + new NamedXContentRegistry.Entry( + EvaluationMetric.Result.class, new ParseField(PrecisionMetric.NAME), PrecisionMetric.Result::fromXContent), + new NamedXContentRegistry.Entry( + EvaluationMetric.Result.class, new ParseField(RecallMetric.NAME), RecallMetric.Result::fromXContent), + new NamedXContentRegistry.Entry( + EvaluationMetric.Result.class, new ParseField(ConfusionMatrixMetric.NAME), ConfusionMatrixMetric.Result::fromXContent)); + } +} diff --git a/client/rest-high-level/src/main/java/org/elasticsearch/client/ml/dataframe/evaluation/softclassification/AbstractConfusionMatrixMetric.java b/client/rest-high-level/src/main/java/org/elasticsearch/client/ml/dataframe/evaluation/softclassification/AbstractConfusionMatrixMetric.java new file mode 100644 index 0000000000000..f41c13f248ab9 --- /dev/null +++ b/client/rest-high-level/src/main/java/org/elasticsearch/client/ml/dataframe/evaluation/softclassification/AbstractConfusionMatrixMetric.java @@ -0,0 +1,47 @@ +/* + * 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.client.ml.dataframe.evaluation.softclassification; + +import org.elasticsearch.client.ml.dataframe.evaluation.EvaluationMetric; +import org.elasticsearch.common.ParseField; +import org.elasticsearch.common.xcontent.ToXContent; +import org.elasticsearch.common.xcontent.XContentBuilder; + +import java.io.IOException; +import java.util.List; +import java.util.Objects; + +abstract class AbstractConfusionMatrixMetric implements EvaluationMetric { + + protected static final ParseField AT = new ParseField("at"); + + protected final double[] thresholds; + + protected AbstractConfusionMatrixMetric(List at) { + this.thresholds = Objects.requireNonNull(at).stream().mapToDouble(Double::doubleValue).toArray(); + } + + @Override + public XContentBuilder toXContent(XContentBuilder builder, ToXContent.Params params) throws IOException { + return builder + .startObject() + .field(AT.getPreferredName(), thresholds) + .endObject(); + } +} diff --git a/client/rest-high-level/src/main/java/org/elasticsearch/client/ml/dataframe/evaluation/softclassification/AucRocMetric.java b/client/rest-high-level/src/main/java/org/elasticsearch/client/ml/dataframe/evaluation/softclassification/AucRocMetric.java new file mode 100644 index 0000000000000..78c713c592581 --- /dev/null +++ b/client/rest-high-level/src/main/java/org/elasticsearch/client/ml/dataframe/evaluation/softclassification/AucRocMetric.java @@ -0,0 +1,241 @@ +/* + * 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.client.ml.dataframe.evaluation.softclassification; + +import org.elasticsearch.client.ml.dataframe.evaluation.EvaluationMetric; +import org.elasticsearch.common.Nullable; +import org.elasticsearch.common.ParseField; +import org.elasticsearch.common.Strings; +import org.elasticsearch.common.xcontent.ConstructingObjectParser; +import org.elasticsearch.common.xcontent.ToXContent; +import org.elasticsearch.common.xcontent.ToXContentObject; +import org.elasticsearch.common.xcontent.XContentBuilder; +import org.elasticsearch.common.xcontent.XContentParser; + +import java.io.IOException; +import java.util.Collections; +import java.util.List; +import java.util.Objects; + +import static org.elasticsearch.common.xcontent.ConstructingObjectParser.constructorArg; +import static org.elasticsearch.common.xcontent.ConstructingObjectParser.optionalConstructorArg; + +/** + * Area under the curve (AUC) of the receiver operating characteristic (ROC). + * The ROC curve is a plot of the TPR (true positive rate) against + * the FPR (false positive rate) over a varying threshold. + */ +public class AucRocMetric implements EvaluationMetric { + + public static final String NAME = "auc_roc"; + + public static final ParseField INCLUDE_CURVE = new ParseField("include_curve"); + + @SuppressWarnings("unchecked") + public static final ConstructingObjectParser PARSER = + new ConstructingObjectParser<>(NAME, args -> new AucRocMetric((Boolean) args[0])); + + static { + PARSER.declareBoolean(optionalConstructorArg(), INCLUDE_CURVE); + } + + public static AucRocMetric fromXContent(XContentParser parser) { + return PARSER.apply(parser, null); + } + + public static AucRocMetric withCurve() { + return new AucRocMetric(true); + } + + private final boolean includeCurve; + + public AucRocMetric(Boolean includeCurve) { + this.includeCurve = includeCurve == null ? false : includeCurve; + } + + @Override + public XContentBuilder toXContent(XContentBuilder builder, ToXContent.Params params) throws IOException { + return builder + .startObject() + .field(INCLUDE_CURVE.getPreferredName(), includeCurve) + .endObject(); + } + + @Override + public String getName() { + return NAME; + } + + @Override + public boolean equals(Object o) { + if (this == o) return true; + if (o == null || getClass() != o.getClass()) return false; + AucRocMetric that = (AucRocMetric) o; + return Objects.equals(includeCurve, that.includeCurve); + } + + @Override + public int hashCode() { + return Objects.hash(includeCurve); + } + + public static class Result implements EvaluationMetric.Result { + + public static Result fromXContent(XContentParser parser) { + return PARSER.apply(parser, null); + } + + private static final ParseField SCORE = new ParseField("score"); + private static final ParseField CURVE = new ParseField("curve"); + + @SuppressWarnings("unchecked") + private static final ConstructingObjectParser PARSER = + new ConstructingObjectParser<>("auc_roc_result", true, args -> new Result((double) args[0], (List) args[1])); + + static { + PARSER.declareDouble(constructorArg(), SCORE); + PARSER.declareObjectArray(optionalConstructorArg(), (p, c) -> AucRocPoint.fromXContent(p), CURVE); + } + + private final double score; + private final List curve; + + public Result(double score, @Nullable List curve) { + this.score = score; + this.curve = curve; + } + + @Override + public String getMetricName() { + return NAME; + } + + public double getScore() { + return score; + } + + public List getCurve() { + return curve == null ? null : Collections.unmodifiableList(curve); + } + + @Override + public XContentBuilder toXContent(XContentBuilder builder, ToXContent.Params params) throws IOException { + builder.startObject(); + builder.field(SCORE.getPreferredName(), score); + if (curve != null && curve.isEmpty() == false) { + builder.field(CURVE.getPreferredName(), curve); + } + builder.endObject(); + return builder; + } + + @Override + public boolean equals(Object o) { + if (this == o) return true; + if (o == null || getClass() != o.getClass()) return false; + Result that = (Result) o; + return Objects.equals(score, that.score) + && Objects.equals(curve, that.curve); + } + + @Override + public int hashCode() { + return Objects.hash(score, curve); + } + + @Override + public String toString() { + return Strings.toString(this); + } + } + + public static final class AucRocPoint implements ToXContentObject { + + public static AucRocPoint fromXContent(XContentParser parser) { + return PARSER.apply(parser, null); + } + + private static final ParseField TPR = new ParseField("tpr"); + private static final ParseField FPR = new ParseField("fpr"); + private static final ParseField THRESHOLD = new ParseField("threshold"); + + @SuppressWarnings("unchecked") + private static final ConstructingObjectParser PARSER = + new ConstructingObjectParser<>( + "auc_roc_point", + true, + args -> new AucRocPoint((double) args[0], (double) args[1], (double) args[2])); + + static { + PARSER.declareDouble(constructorArg(), TPR); + PARSER.declareDouble(constructorArg(), FPR); + PARSER.declareDouble(constructorArg(), THRESHOLD); + } + + private final double tpr; + private final double fpr; + private final double threshold; + + public AucRocPoint(double tpr, double fpr, double threshold) { + this.tpr = tpr; + this.fpr = fpr; + this.threshold = threshold; + } + + public double getTruePositiveRate() { + return tpr; + } + + public double getFalsePositiveRate() { + return fpr; + } + + public double getThreshold() { + return threshold; + } + + @Override + public XContentBuilder toXContent(XContentBuilder builder, Params params) throws IOException { + return builder + .startObject() + .field(TPR.getPreferredName(), tpr) + .field(FPR.getPreferredName(), fpr) + .field(THRESHOLD.getPreferredName(), threshold) + .endObject(); + } + + @Override + public boolean equals(Object o) { + if (this == o) return true; + if (o == null || getClass() != o.getClass()) return false; + AucRocPoint that = (AucRocPoint) o; + return tpr == that.tpr && fpr == that.fpr && threshold == that.threshold; + } + + @Override + public int hashCode() { + return Objects.hash(tpr, fpr, threshold); + } + + @Override + public String toString() { + return Strings.toString(this); + } + } +} diff --git a/client/rest-high-level/src/main/java/org/elasticsearch/client/ml/dataframe/evaluation/softclassification/BinarySoftClassification.java b/client/rest-high-level/src/main/java/org/elasticsearch/client/ml/dataframe/evaluation/softclassification/BinarySoftClassification.java new file mode 100644 index 0000000000000..6d5fa04da38e5 --- /dev/null +++ b/client/rest-high-level/src/main/java/org/elasticsearch/client/ml/dataframe/evaluation/softclassification/BinarySoftClassification.java @@ -0,0 +1,129 @@ +/* + * 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.client.ml.dataframe.evaluation.softclassification; + +import org.elasticsearch.client.ml.dataframe.evaluation.Evaluation; +import org.elasticsearch.client.ml.dataframe.evaluation.EvaluationMetric; +import org.elasticsearch.common.Nullable; +import org.elasticsearch.common.ParseField; +import org.elasticsearch.common.xcontent.ConstructingObjectParser; +import org.elasticsearch.common.xcontent.ToXContent; +import org.elasticsearch.common.xcontent.XContentBuilder; +import org.elasticsearch.common.xcontent.XContentParser; + +import java.io.IOException; +import java.util.Arrays; +import java.util.List; +import java.util.Objects; + +import static org.elasticsearch.common.xcontent.ConstructingObjectParser.constructorArg; +import static org.elasticsearch.common.xcontent.ConstructingObjectParser.optionalConstructorArg; + +/** + * Evaluation of binary soft classification methods, e.g. outlier detection. + * This is useful to evaluate problems where a model outputs a probability of whether + * a data frame row belongs to one of two groups. + */ +public class BinarySoftClassification implements Evaluation { + + public static final String NAME = "binary_soft_classification"; + + private static final ParseField ACTUAL_FIELD = new ParseField("actual_field"); + private static final ParseField PREDICTED_PROBABILITY_FIELD = new ParseField("predicted_probability_field"); + private static final ParseField METRICS = new ParseField("metrics"); + + @SuppressWarnings("unchecked") + public static final ConstructingObjectParser PARSER = + new ConstructingObjectParser<>( + NAME, + args -> new BinarySoftClassification((String) args[0], (String) args[1], (List) args[2])); + + static { + PARSER.declareString(constructorArg(), ACTUAL_FIELD); + PARSER.declareString(constructorArg(), PREDICTED_PROBABILITY_FIELD); + PARSER.declareNamedObjects(optionalConstructorArg(), (p, c, n) -> p.namedObject(EvaluationMetric.class, n, null), METRICS); + } + + public static BinarySoftClassification fromXContent(XContentParser parser) { + return PARSER.apply(parser, null); + } + + /** + * The field where the actual class is marked up. + * The value of this field is assumed to either be 1 or 0, or true or false. + */ + private final String actualField; + + /** + * The field of the predicted probability in [0.0, 1.0]. + */ + private final String predictedProbabilityField; + + /** + * The list of metrics to calculate + */ + private final List metrics; + + public BinarySoftClassification(String actualField, String predictedProbabilityField, EvaluationMetric... metric) { + this(actualField, predictedProbabilityField, Arrays.asList(metric)); + } + + public BinarySoftClassification(String actualField, String predictedProbabilityField, + @Nullable List metrics) { + this.actualField = Objects.requireNonNull(actualField); + this.predictedProbabilityField = Objects.requireNonNull(predictedProbabilityField); + this.metrics = Objects.requireNonNull(metrics); + } + + @Override + public String getName() { + return NAME; + } + + @Override + public XContentBuilder toXContent(XContentBuilder builder, ToXContent.Params params) throws IOException { + builder.startObject(); + builder.field(ACTUAL_FIELD.getPreferredName(), actualField); + builder.field(PREDICTED_PROBABILITY_FIELD.getPreferredName(), predictedProbabilityField); + + builder.startObject(METRICS.getPreferredName()); + for (EvaluationMetric metric : metrics) { + builder.field(metric.getName(), metric); + } + builder.endObject(); + + builder.endObject(); + return builder; + } + + @Override + public boolean equals(Object o) { + if (this == o) return true; + if (o == null || getClass() != o.getClass()) return false; + BinarySoftClassification that = (BinarySoftClassification) o; + return Objects.equals(actualField, that.actualField) + && Objects.equals(predictedProbabilityField, that.predictedProbabilityField) + && Objects.equals(metrics, that.metrics); + } + + @Override + public int hashCode() { + return Objects.hash(actualField, predictedProbabilityField, metrics); + } +} diff --git a/client/rest-high-level/src/main/java/org/elasticsearch/client/ml/dataframe/evaluation/softclassification/ConfusionMatrixMetric.java b/client/rest-high-level/src/main/java/org/elasticsearch/client/ml/dataframe/evaluation/softclassification/ConfusionMatrixMetric.java new file mode 100644 index 0000000000000..d5e4307c9cc74 --- /dev/null +++ b/client/rest-high-level/src/main/java/org/elasticsearch/client/ml/dataframe/evaluation/softclassification/ConfusionMatrixMetric.java @@ -0,0 +1,206 @@ +/* + * 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.client.ml.dataframe.evaluation.softclassification; + +import org.elasticsearch.client.ml.dataframe.evaluation.EvaluationMetric; +import org.elasticsearch.common.ParseField; +import org.elasticsearch.common.Strings; +import org.elasticsearch.common.xcontent.ConstructingObjectParser; +import org.elasticsearch.common.xcontent.ToXContent; +import org.elasticsearch.common.xcontent.ToXContentObject; +import org.elasticsearch.common.xcontent.XContentBuilder; +import org.elasticsearch.common.xcontent.XContentParser; + +import java.io.IOException; +import java.util.Arrays; +import java.util.LinkedHashMap; +import java.util.List; +import java.util.Map; +import java.util.Objects; + +import static org.elasticsearch.common.xcontent.ConstructingObjectParser.constructorArg; + +public class ConfusionMatrixMetric extends AbstractConfusionMatrixMetric { + + public static final String NAME = "confusion_matrix"; + + @SuppressWarnings("unchecked") + private static final ConstructingObjectParser PARSER = + new ConstructingObjectParser<>(NAME, args -> new ConfusionMatrixMetric((List) args[0])); + + static { + PARSER.declareDoubleArray(constructorArg(), AT); + } + + public static ConfusionMatrixMetric fromXContent(XContentParser parser) { + return PARSER.apply(parser, null); + } + + public static ConfusionMatrixMetric at(Double... at) { + return new ConfusionMatrixMetric(Arrays.asList(at)); + } + + public ConfusionMatrixMetric(List at) { + super(at); + } + + @Override + public String getName() { + return NAME; + } + + @Override + public boolean equals(Object o) { + if (this == o) return true; + if (o == null || getClass() != o.getClass()) return false; + ConfusionMatrixMetric that = (ConfusionMatrixMetric) o; + return Arrays.equals(thresholds, that.thresholds); + } + + @Override + public int hashCode() { + return Arrays.hashCode(thresholds); + } + + public static class Result implements EvaluationMetric.Result { + + public static Result fromXContent(XContentParser parser) throws IOException { + return new Result(parser.map(LinkedHashMap::new, ConfusionMatrix::fromXContent)); + } + + private final Map results; + + public Result(Map results) { + this.results = Objects.requireNonNull(results); + } + + @Override + public String getMetricName() { + return NAME; + } + + public ConfusionMatrix getScoreByThreshold(String threshold) { + return results.get(threshold); + } + + @Override + public XContentBuilder toXContent(XContentBuilder builder, ToXContent.Params params) throws IOException { + return builder.map(results); + } + + @Override + public boolean equals(Object o) { + if (this == o) return true; + if (o == null || getClass() != o.getClass()) return false; + Result that = (Result) o; + return Objects.equals(results, that.results); + } + + @Override + public int hashCode() { + return Objects.hash(results); + } + + @Override + public String toString() { + return Strings.toString(this); + } + } + + public static final class ConfusionMatrix implements ToXContentObject { + + public static ConfusionMatrix fromXContent(XContentParser parser) { + return PARSER.apply(parser, null); + } + + private static final ParseField TP = new ParseField("tp"); + private static final ParseField FP = new ParseField("fp"); + private static final ParseField TN = new ParseField("tn"); + private static final ParseField FN = new ParseField("fn"); + + @SuppressWarnings("unchecked") + private static final ConstructingObjectParser PARSER = + new ConstructingObjectParser<>( + "confusion_matrix", true, args -> new ConfusionMatrix((long) args[0], (long) args[1], (long) args[2], (long) args[3])); + + static { + PARSER.declareLong(constructorArg(), TP); + PARSER.declareLong(constructorArg(), FP); + PARSER.declareLong(constructorArg(), TN); + PARSER.declareLong(constructorArg(), FN); + } + + private final long tp; + private final long fp; + private final long tn; + private final long fn; + + public ConfusionMatrix(long tp, long fp, long tn, long fn) { + this.tp = tp; + this.fp = fp; + this.tn = tn; + this.fn = fn; + } + + public long getTruePositives() { + return tp; + } + + public long getFalsePositives() { + return fp; + } + + public long getTrueNegatives() { + return tn; + } + + public long getFalseNegatives() { + return fn; + } + + @Override + public XContentBuilder toXContent(XContentBuilder builder, Params params) throws IOException { + return builder + .startObject() + .field(TP.getPreferredName(), tp) + .field(FP.getPreferredName(), fp) + .field(TN.getPreferredName(), tn) + .field(FN.getPreferredName(), fn) + .endObject(); + } + + @Override + public boolean equals(Object o) { + if (this == o) return true; + if (o == null || getClass() != o.getClass()) return false; + ConfusionMatrix that = (ConfusionMatrix) o; + return tp == that.tp && fp == that.fp && tn == that.tn && fn == that.fn; + } + + @Override + public int hashCode() { + return Objects.hash(tp, fp, tn, fn); + } + + @Override + public String toString() { + return Strings.toString(this); + } + } +} diff --git a/client/rest-high-level/src/main/java/org/elasticsearch/client/ml/dataframe/evaluation/softclassification/PrecisionMetric.java b/client/rest-high-level/src/main/java/org/elasticsearch/client/ml/dataframe/evaluation/softclassification/PrecisionMetric.java new file mode 100644 index 0000000000000..2a0f1499461d6 --- /dev/null +++ b/client/rest-high-level/src/main/java/org/elasticsearch/client/ml/dataframe/evaluation/softclassification/PrecisionMetric.java @@ -0,0 +1,123 @@ +/* + * 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.client.ml.dataframe.evaluation.softclassification; + +import org.elasticsearch.client.ml.dataframe.evaluation.EvaluationMetric; +import org.elasticsearch.common.Strings; +import org.elasticsearch.common.xcontent.ConstructingObjectParser; +import org.elasticsearch.common.xcontent.ToXContent; +import org.elasticsearch.common.xcontent.XContentBuilder; +import org.elasticsearch.common.xcontent.XContentParser; + +import java.io.IOException; +import java.util.Arrays; +import java.util.LinkedHashMap; +import java.util.List; +import java.util.Map; +import java.util.Objects; + +import static org.elasticsearch.common.xcontent.ConstructingObjectParser.constructorArg; + +public class PrecisionMetric extends AbstractConfusionMatrixMetric { + + public static final String NAME = "precision"; + + @SuppressWarnings("unchecked") + private static final ConstructingObjectParser PARSER = + new ConstructingObjectParser<>(NAME, args -> new PrecisionMetric((List) args[0])); + + static { + PARSER.declareDoubleArray(constructorArg(), AT); + } + + public static PrecisionMetric fromXContent(XContentParser parser) { + return PARSER.apply(parser, null); + } + + public static PrecisionMetric at(Double... at) { + return new PrecisionMetric(Arrays.asList(at)); + } + + public PrecisionMetric(List at) { + super(at); + } + + @Override + public String getName() { + return NAME; + } + + @Override + public boolean equals(Object o) { + if (this == o) return true; + if (o == null || getClass() != o.getClass()) return false; + PrecisionMetric that = (PrecisionMetric) o; + return Arrays.equals(thresholds, that.thresholds); + } + + @Override + public int hashCode() { + return Arrays.hashCode(thresholds); + } + + public static class Result implements EvaluationMetric.Result { + + public static Result fromXContent(XContentParser parser) throws IOException { + return new Result(parser.map(LinkedHashMap::new, p -> p.doubleValue())); + } + + private final Map results; + + public Result(Map results) { + this.results = Objects.requireNonNull(results); + } + + @Override + public String getMetricName() { + return NAME; + } + + public Double getScoreByThreshold(String threshold) { + return results.get(threshold); + } + + @Override + public XContentBuilder toXContent(XContentBuilder builder, ToXContent.Params params) throws IOException { + return builder.map(results); + } + + @Override + public boolean equals(Object o) { + if (this == o) return true; + if (o == null || getClass() != o.getClass()) return false; + Result that = (Result) o; + return Objects.equals(results, that.results); + } + + @Override + public int hashCode() { + return Objects.hash(results); + } + + @Override + public String toString() { + return Strings.toString(this); + } + } +} diff --git a/client/rest-high-level/src/main/java/org/elasticsearch/client/ml/dataframe/evaluation/softclassification/RecallMetric.java b/client/rest-high-level/src/main/java/org/elasticsearch/client/ml/dataframe/evaluation/softclassification/RecallMetric.java new file mode 100644 index 0000000000000..505ff1b34d7c5 --- /dev/null +++ b/client/rest-high-level/src/main/java/org/elasticsearch/client/ml/dataframe/evaluation/softclassification/RecallMetric.java @@ -0,0 +1,123 @@ +/* + * 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.client.ml.dataframe.evaluation.softclassification; + +import org.elasticsearch.client.ml.dataframe.evaluation.EvaluationMetric; +import org.elasticsearch.common.Strings; +import org.elasticsearch.common.xcontent.ConstructingObjectParser; +import org.elasticsearch.common.xcontent.ToXContent; +import org.elasticsearch.common.xcontent.XContentBuilder; +import org.elasticsearch.common.xcontent.XContentParser; + +import java.io.IOException; +import java.util.Arrays; +import java.util.LinkedHashMap; +import java.util.List; +import java.util.Map; +import java.util.Objects; + +import static org.elasticsearch.common.xcontent.ConstructingObjectParser.constructorArg; + +public class RecallMetric extends AbstractConfusionMatrixMetric { + + public static final String NAME = "recall"; + + @SuppressWarnings("unchecked") + private static final ConstructingObjectParser PARSER = + new ConstructingObjectParser<>(NAME, args -> new RecallMetric((List) args[0])); + + static { + PARSER.declareDoubleArray(constructorArg(), AT); + } + + public static RecallMetric fromXContent(XContentParser parser) { + return PARSER.apply(parser, null); + } + + public static RecallMetric at(Double... at) { + return new RecallMetric(Arrays.asList(at)); + } + + public RecallMetric(List at) { + super(at); + } + + @Override + public String getName() { + return NAME; + } + + @Override + public boolean equals(Object o) { + if (this == o) return true; + if (o == null || getClass() != o.getClass()) return false; + RecallMetric that = (RecallMetric) o; + return Arrays.equals(thresholds, that.thresholds); + } + + @Override + public int hashCode() { + return Arrays.hashCode(thresholds); + } + + public static class Result implements EvaluationMetric.Result { + + public static Result fromXContent(XContentParser parser) throws IOException { + return new Result(parser.map(LinkedHashMap::new, p -> p.doubleValue())); + } + + private final Map results; + + public Result(Map results) { + this.results = Objects.requireNonNull(results); + } + + @Override + public String getMetricName() { + return NAME; + } + + public Double getScoreByThreshold(String threshold) { + return results.get(threshold); + } + + @Override + public XContentBuilder toXContent(XContentBuilder builder, ToXContent.Params params) throws IOException { + return builder.map(results); + } + + @Override + public boolean equals(Object o) { + if (this == o) return true; + if (o == null || getClass() != o.getClass()) return false; + Result that = (Result) o; + return Objects.equals(results, that.results); + } + + @Override + public int hashCode() { + return Objects.hash(results); + } + + @Override + public String toString() { + return Strings.toString(this); + } + } +} diff --git a/client/rest-high-level/src/main/java/org/elasticsearch/client/ml/filestructurefinder/FieldStats.java b/client/rest-high-level/src/main/java/org/elasticsearch/client/ml/filestructurefinder/FieldStats.java index 4391d03f6d940..adb8e68393e82 100644 --- a/client/rest-high-level/src/main/java/org/elasticsearch/client/ml/filestructurefinder/FieldStats.java +++ b/client/rest-high-level/src/main/java/org/elasticsearch/client/ml/filestructurefinder/FieldStats.java @@ -38,12 +38,14 @@ public class FieldStats implements ToXContentObject { public static final ParseField MAX_VALUE = new ParseField("max_value"); public static final ParseField MEAN_VALUE = new ParseField("mean_value"); public static final ParseField MEDIAN_VALUE = new ParseField("median_value"); + public static final ParseField EARLIEST = new ParseField("earliest"); + public static final ParseField LATEST = new ParseField("latest"); public static final ParseField TOP_HITS = new ParseField("top_hits"); @SuppressWarnings("unchecked") public static final ConstructingObjectParser PARSER = new ConstructingObjectParser<>("field_stats", true, a -> new FieldStats((long) a[0], (int) a[1], (Double) a[2], (Double) a[3], (Double) a[4], (Double) a[5], - (List>) a[6])); + (String) a[6], (String) a[7], (List>) a[8])); static { PARSER.declareLong(ConstructingObjectParser.constructorArg(), COUNT); @@ -52,6 +54,8 @@ public class FieldStats implements ToXContentObject { PARSER.declareDouble(ConstructingObjectParser.optionalConstructorArg(), MAX_VALUE); PARSER.declareDouble(ConstructingObjectParser.optionalConstructorArg(), MEAN_VALUE); PARSER.declareDouble(ConstructingObjectParser.optionalConstructorArg(), MEDIAN_VALUE); + PARSER.declareString(ConstructingObjectParser.optionalConstructorArg(), EARLIEST); + PARSER.declareString(ConstructingObjectParser.optionalConstructorArg(), LATEST); PARSER.declareObjectArray(ConstructingObjectParser.optionalConstructorArg(), (p, c) -> p.mapOrdered(), TOP_HITS); } @@ -61,16 +65,20 @@ public class FieldStats implements ToXContentObject { private final Double maxValue; private final Double meanValue; private final Double medianValue; + private final String earliestTimestamp; + private final String latestTimestamp; private final List> topHits; FieldStats(long count, int cardinality, Double minValue, Double maxValue, Double meanValue, Double medianValue, - List> topHits) { + String earliestTimestamp, String latestTimestamp, List> topHits) { this.count = count; this.cardinality = cardinality; this.minValue = minValue; this.maxValue = maxValue; this.meanValue = meanValue; this.medianValue = medianValue; + this.earliestTimestamp = earliestTimestamp; + this.latestTimestamp = latestTimestamp; this.topHits = (topHits == null) ? Collections.emptyList() : Collections.unmodifiableList(topHits); } @@ -98,6 +106,14 @@ public Double getMedianValue() { return medianValue; } + public String getEarliestTimestamp() { + return earliestTimestamp; + } + + public String getLatestTimestamp() { + return latestTimestamp; + } + public List> getTopHits() { return topHits; } @@ -120,6 +136,12 @@ public XContentBuilder toXContent(XContentBuilder builder, ToXContent.Params par if (medianValue != null) { builder.field(MEDIAN_VALUE.getPreferredName(), toIntegerIfInteger(medianValue)); } + if (earliestTimestamp != null) { + builder.field(EARLIEST.getPreferredName(), earliestTimestamp); + } + if (latestTimestamp != null) { + builder.field(LATEST.getPreferredName(), latestTimestamp); + } if (topHits.isEmpty() == false) { builder.field(TOP_HITS.getPreferredName(), topHits); } @@ -140,7 +162,7 @@ static Number toIntegerIfInteger(double d) { @Override public int hashCode() { - return Objects.hash(count, cardinality, minValue, maxValue, meanValue, medianValue, topHits); + return Objects.hash(count, cardinality, minValue, maxValue, meanValue, medianValue, earliestTimestamp, latestTimestamp, topHits); } @Override @@ -161,6 +183,8 @@ public boolean equals(Object other) { Objects.equals(this.maxValue, that.maxValue) && Objects.equals(this.meanValue, that.meanValue) && Objects.equals(this.medianValue, that.medianValue) && + Objects.equals(this.earliestTimestamp, that.earliestTimestamp) && + Objects.equals(this.latestTimestamp, that.latestTimestamp) && Objects.equals(this.topHits, that.topHits); } } diff --git a/client/rest-high-level/src/main/java/org/elasticsearch/client/ml/job/process/TimingStats.java b/client/rest-high-level/src/main/java/org/elasticsearch/client/ml/job/process/TimingStats.java new file mode 100644 index 0000000000000..73393140f30af --- /dev/null +++ b/client/rest-high-level/src/main/java/org/elasticsearch/client/ml/job/process/TimingStats.java @@ -0,0 +1,159 @@ +/* + * 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.client.ml.job.process; + +import org.elasticsearch.client.ml.job.config.Job; +import org.elasticsearch.common.Nullable; +import org.elasticsearch.common.ParseField; +import org.elasticsearch.common.Strings; +import org.elasticsearch.common.xcontent.ConstructingObjectParser; +import org.elasticsearch.common.xcontent.ToXContent; +import org.elasticsearch.common.xcontent.ToXContentObject; +import org.elasticsearch.common.xcontent.XContentBuilder; + +import java.io.IOException; +import java.util.Objects; + +import static org.elasticsearch.common.xcontent.ConstructingObjectParser.constructorArg; +import static org.elasticsearch.common.xcontent.ConstructingObjectParser.optionalConstructorArg; + +/** + * Stats that give more insight into timing of various operations performed as part of anomaly detection job. + */ +public class TimingStats implements ToXContentObject { + + public static final ParseField BUCKET_COUNT = new ParseField("bucket_count"); + public static final ParseField MIN_BUCKET_PROCESSING_TIME_MS = new ParseField("minimum_bucket_processing_time_ms"); + public static final ParseField MAX_BUCKET_PROCESSING_TIME_MS = new ParseField("maximum_bucket_processing_time_ms"); + public static final ParseField AVG_BUCKET_PROCESSING_TIME_MS = new ParseField("average_bucket_processing_time_ms"); + public static final ParseField EXPONENTIAL_AVG_BUCKET_PROCESSING_TIME_MS = + new ParseField("exponential_average_bucket_processing_time_ms"); + + public static final ConstructingObjectParser PARSER = + new ConstructingObjectParser<>( + "timing_stats", + true, + args -> + new TimingStats((String) args[0], (long) args[1], (Double) args[2], (Double) args[3], (Double) args[4], (Double) args[5])); + + static { + PARSER.declareString(constructorArg(), Job.ID); + PARSER.declareLong(constructorArg(), BUCKET_COUNT); + PARSER.declareDouble(optionalConstructorArg(), MIN_BUCKET_PROCESSING_TIME_MS); + PARSER.declareDouble(optionalConstructorArg(), MAX_BUCKET_PROCESSING_TIME_MS); + PARSER.declareDouble(optionalConstructorArg(), AVG_BUCKET_PROCESSING_TIME_MS); + PARSER.declareDouble(optionalConstructorArg(), EXPONENTIAL_AVG_BUCKET_PROCESSING_TIME_MS); + } + + private final String jobId; + private long bucketCount; + private Double minBucketProcessingTimeMs; + private Double maxBucketProcessingTimeMs; + private Double avgBucketProcessingTimeMs; + private Double exponentialAvgBucketProcessingTimeMs; + + public TimingStats( + String jobId, + long bucketCount, + @Nullable Double minBucketProcessingTimeMs, + @Nullable Double maxBucketProcessingTimeMs, + @Nullable Double avgBucketProcessingTimeMs, + @Nullable Double exponentialAvgBucketProcessingTimeMs) { + this.jobId = jobId; + this.bucketCount = bucketCount; + this.minBucketProcessingTimeMs = minBucketProcessingTimeMs; + this.maxBucketProcessingTimeMs = maxBucketProcessingTimeMs; + this.avgBucketProcessingTimeMs = avgBucketProcessingTimeMs; + this.exponentialAvgBucketProcessingTimeMs = exponentialAvgBucketProcessingTimeMs; + } + + public String getJobId() { + return jobId; + } + + public long getBucketCount() { + return bucketCount; + } + + public Double getMinBucketProcessingTimeMs() { + return minBucketProcessingTimeMs; + } + + public Double getMaxBucketProcessingTimeMs() { + return maxBucketProcessingTimeMs; + } + + public Double getAvgBucketProcessingTimeMs() { + return avgBucketProcessingTimeMs; + } + + public Double getExponentialAvgBucketProcessingTimeMs() { + return exponentialAvgBucketProcessingTimeMs; + } + + @Override + public XContentBuilder toXContent(XContentBuilder builder, ToXContent.Params params) throws IOException { + builder.startObject(); + builder.field(Job.ID.getPreferredName(), jobId); + builder.field(BUCKET_COUNT.getPreferredName(), bucketCount); + if (minBucketProcessingTimeMs != null) { + builder.field(MIN_BUCKET_PROCESSING_TIME_MS.getPreferredName(), minBucketProcessingTimeMs); + } + if (maxBucketProcessingTimeMs != null) { + builder.field(MAX_BUCKET_PROCESSING_TIME_MS.getPreferredName(), maxBucketProcessingTimeMs); + } + if (avgBucketProcessingTimeMs != null) { + builder.field(AVG_BUCKET_PROCESSING_TIME_MS.getPreferredName(), avgBucketProcessingTimeMs); + } + if (exponentialAvgBucketProcessingTimeMs != null) { + builder.field(EXPONENTIAL_AVG_BUCKET_PROCESSING_TIME_MS.getPreferredName(), exponentialAvgBucketProcessingTimeMs); + } + builder.endObject(); + return builder; + } + + @Override + public boolean equals(Object o) { + if (o == this) return true; + if (o == null || getClass() != o.getClass()) return false; + TimingStats that = (TimingStats) o; + return Objects.equals(this.jobId, that.jobId) + && this.bucketCount == that.bucketCount + && Objects.equals(this.minBucketProcessingTimeMs, that.minBucketProcessingTimeMs) + && Objects.equals(this.maxBucketProcessingTimeMs, that.maxBucketProcessingTimeMs) + && Objects.equals(this.avgBucketProcessingTimeMs, that.avgBucketProcessingTimeMs) + && Objects.equals(this.exponentialAvgBucketProcessingTimeMs, that.exponentialAvgBucketProcessingTimeMs); + } + + @Override + public int hashCode() { + return Objects.hash( + jobId, + bucketCount, + minBucketProcessingTimeMs, + maxBucketProcessingTimeMs, + avgBucketProcessingTimeMs, + exponentialAvgBucketProcessingTimeMs); + } + + @Override + public String toString() { + return Strings.toString(this); + } +} diff --git a/client/rest-high-level/src/main/java/org/elasticsearch/client/ml/job/stats/JobStats.java b/client/rest-high-level/src/main/java/org/elasticsearch/client/ml/job/stats/JobStats.java index df5be4aa4c5cc..078f781d4b21d 100644 --- a/client/rest-high-level/src/main/java/org/elasticsearch/client/ml/job/stats/JobStats.java +++ b/client/rest-high-level/src/main/java/org/elasticsearch/client/ml/job/stats/JobStats.java @@ -22,6 +22,7 @@ import org.elasticsearch.client.ml.job.config.JobState; import org.elasticsearch.client.ml.job.process.DataCounts; import org.elasticsearch.client.ml.job.process.ModelSizeStats; +import org.elasticsearch.client.ml.job.process.TimingStats; import org.elasticsearch.common.Nullable; import org.elasticsearch.common.ParseField; import org.elasticsearch.common.unit.TimeValue; @@ -42,6 +43,7 @@ public class JobStats implements ToXContentObject { private static final ParseField DATA_COUNTS = new ParseField("data_counts"); private static final ParseField MODEL_SIZE_STATS = new ParseField("model_size_stats"); + private static final ParseField TIMING_STATS = new ParseField("timing_stats"); private static final ParseField FORECASTS_STATS = new ParseField("forecasts_stats"); private static final ParseField STATE = new ParseField("state"); private static final ParseField NODE = new ParseField("node"); @@ -58,6 +60,7 @@ public class JobStats implements ToXContentObject { JobState jobState = (JobState) a[i++]; ModelSizeStats.Builder modelSizeStatsBuilder = (ModelSizeStats.Builder) a[i++]; ModelSizeStats modelSizeStats = modelSizeStatsBuilder == null ? null : modelSizeStatsBuilder.build(); + TimingStats timingStats = (TimingStats) a[i++]; ForecastStats forecastStats = (ForecastStats) a[i++]; NodeAttributes node = (NodeAttributes) a[i++]; String assignmentExplanation = (String) a[i++]; @@ -66,6 +69,7 @@ public class JobStats implements ToXContentObject { dataCounts, jobState, modelSizeStats, + timingStats, forecastStats, node, assignmentExplanation, @@ -80,6 +84,7 @@ public class JobStats implements ToXContentObject { STATE, ObjectParser.ValueType.VALUE); PARSER.declareObject(ConstructingObjectParser.optionalConstructorArg(), ModelSizeStats.PARSER, MODEL_SIZE_STATS); + PARSER.declareObject(ConstructingObjectParser.optionalConstructorArg(), TimingStats.PARSER, TIMING_STATS); PARSER.declareObject(ConstructingObjectParser.optionalConstructorArg(), ForecastStats.PARSER, FORECASTS_STATS); PARSER.declareObject(ConstructingObjectParser.optionalConstructorArg(), NodeAttributes.PARSER, NODE); PARSER.declareString(ConstructingObjectParser.optionalConstructorArg(), ASSIGNMENT_EXPLANATION); @@ -94,22 +99,24 @@ public class JobStats implements ToXContentObject { private final DataCounts dataCounts; private final JobState state; private final ModelSizeStats modelSizeStats; + private final TimingStats timingStats; private final ForecastStats forecastStats; private final NodeAttributes node; private final String assignmentExplanation; private final TimeValue openTime; JobStats(String jobId, DataCounts dataCounts, JobState state, @Nullable ModelSizeStats modelSizeStats, - @Nullable ForecastStats forecastStats, @Nullable NodeAttributes node, - @Nullable String assignmentExplanation, @Nullable TimeValue opentime) { + @Nullable TimingStats timingStats, @Nullable ForecastStats forecastStats, @Nullable NodeAttributes node, + @Nullable String assignmentExplanation, @Nullable TimeValue openTime) { this.jobId = Objects.requireNonNull(jobId); this.dataCounts = Objects.requireNonNull(dataCounts); this.state = Objects.requireNonNull(state); this.modelSizeStats = modelSizeStats; + this.timingStats = timingStats; this.forecastStats = forecastStats; this.node = node; this.assignmentExplanation = assignmentExplanation; - this.openTime = opentime; + this.openTime = openTime; } /** @@ -135,6 +142,10 @@ public ModelSizeStats getModelSizeStats() { return modelSizeStats; } + public TimingStats getTimingStats() { + return timingStats; + } + /** * An object that provides statistical information about forecasts of this job. * See {@link ForecastStats} @@ -182,6 +193,9 @@ public XContentBuilder toXContent(XContentBuilder builder, Params params) throws if (modelSizeStats != null) { builder.field(MODEL_SIZE_STATS.getPreferredName(), modelSizeStats); } + if (timingStats != null) { + builder.field(TIMING_STATS.getPreferredName(), timingStats); + } if (forecastStats != null) { builder.field(FORECASTS_STATS.getPreferredName(), forecastStats); } @@ -199,7 +213,7 @@ public XContentBuilder toXContent(XContentBuilder builder, Params params) throws @Override public int hashCode() { - return Objects.hash(jobId, dataCounts, modelSizeStats, forecastStats, state, node, assignmentExplanation, openTime); + return Objects.hash(jobId, dataCounts, modelSizeStats, timingStats, forecastStats, state, node, assignmentExplanation, openTime); } @Override @@ -216,6 +230,7 @@ public boolean equals(Object obj) { return Objects.equals(jobId, other.jobId) && Objects.equals(this.dataCounts, other.dataCounts) && Objects.equals(this.modelSizeStats, other.modelSizeStats) && + Objects.equals(this.timingStats, other.timingStats) && Objects.equals(this.forecastStats, other.forecastStats) && Objects.equals(this.state, other.state) && Objects.equals(this.node, other.node) && diff --git a/client/rest-high-level/src/main/java/org/elasticsearch/client/security/CreateTokenRequest.java b/client/rest-high-level/src/main/java/org/elasticsearch/client/security/CreateTokenRequest.java index 6c1b394355e1e..4f1302533d9b5 100644 --- a/client/rest-high-level/src/main/java/org/elasticsearch/client/security/CreateTokenRequest.java +++ b/client/rest-high-level/src/main/java/org/elasticsearch/client/security/CreateTokenRequest.java @@ -40,6 +40,7 @@ public final class CreateTokenRequest implements Validatable, ToXContentObject { private final String username; private final char[] password; private final String refreshToken; + private final char[] kerberosTicket; /** * General purpose constructor. This constructor is typically not useful, and one of the following factory methods should be used @@ -48,10 +49,11 @@ public final class CreateTokenRequest implements Validatable, ToXContentObject { *

  • {@link #passwordGrant(String, char[])}
  • *
  • {@link #refreshTokenGrant(String)}
  • *
  • {@link #clientCredentialsGrant()}
  • + *
  • {@link #kerberosGrant(char[])}
  • * */ public CreateTokenRequest(String grantType, @Nullable String scope, @Nullable String username, @Nullable char[] password, - @Nullable String refreshToken) { + @Nullable String refreshToken, @Nullable char[] kerberosTicket) { if (Strings.isNullOrEmpty(grantType)) { throw new IllegalArgumentException("grant_type is required"); } @@ -60,6 +62,7 @@ public CreateTokenRequest(String grantType, @Nullable String scope, @Nullable St this.password = password; this.scope = scope; this.refreshToken = refreshToken; + this.kerberosTicket = kerberosTicket; } public static CreateTokenRequest passwordGrant(String username, char[] password) { @@ -69,18 +72,25 @@ public static CreateTokenRequest passwordGrant(String username, char[] password) if (password == null || password.length == 0) { throw new IllegalArgumentException("password is required"); } - return new CreateTokenRequest("password", null, username, password, null); + return new CreateTokenRequest("password", null, username, password, null, null); } public static CreateTokenRequest refreshTokenGrant(String refreshToken) { if (Strings.isNullOrEmpty(refreshToken)) { throw new IllegalArgumentException("refresh_token is required"); } - return new CreateTokenRequest("refresh_token", null, null, null, refreshToken); + return new CreateTokenRequest("refresh_token", null, null, null, refreshToken, null); } public static CreateTokenRequest clientCredentialsGrant() { - return new CreateTokenRequest("client_credentials", null, null, null, null); + return new CreateTokenRequest("client_credentials", null, null, null, null, null); + } + + public static CreateTokenRequest kerberosGrant(char[] kerberosTicket) { + if (kerberosTicket == null || kerberosTicket.length == 0) { + throw new IllegalArgumentException("kerberos ticket is required"); + } + return new CreateTokenRequest("_kerberos", null, null, null, null, kerberosTicket); } public String getGrantType() { @@ -103,6 +113,10 @@ public String getRefreshToken() { return refreshToken; } + public char[] getKerberosTicket() { + return kerberosTicket; + } + @Override public XContentBuilder toXContent(XContentBuilder builder, Params params) throws IOException { builder.startObject() @@ -124,6 +138,14 @@ public XContentBuilder toXContent(XContentBuilder builder, Params params) throws if (refreshToken != null) { builder.field("refresh_token", refreshToken); } + if (kerberosTicket != null) { + byte[] kerberosTicketBytes = CharArrays.toUtf8Bytes(kerberosTicket); + try { + builder.field("kerberos_ticket").utf8Value(kerberosTicketBytes, 0, kerberosTicketBytes.length); + } finally { + Arrays.fill(kerberosTicketBytes, (byte) 0); + } + } return builder.endObject(); } @@ -140,13 +162,15 @@ public boolean equals(Object o) { Objects.equals(scope, that.scope) && Objects.equals(username, that.username) && Arrays.equals(password, that.password) && - Objects.equals(refreshToken, that.refreshToken); + Objects.equals(refreshToken, that.refreshToken) && + Arrays.equals(kerberosTicket, that.kerberosTicket); } @Override public int hashCode() { int result = Objects.hash(grantType, scope, username, refreshToken); result = 31 * result + Arrays.hashCode(password); + result = 31 * result + Arrays.hashCode(kerberosTicket); return result; } } diff --git a/client/rest-high-level/src/main/java/org/elasticsearch/client/security/CreateTokenResponse.java b/client/rest-high-level/src/main/java/org/elasticsearch/client/security/CreateTokenResponse.java index 32d298d1a9bc0..dc71d49f4b770 100644 --- a/client/rest-high-level/src/main/java/org/elasticsearch/client/security/CreateTokenResponse.java +++ b/client/rest-high-level/src/main/java/org/elasticsearch/client/security/CreateTokenResponse.java @@ -41,13 +41,16 @@ public final class CreateTokenResponse { private final TimeValue expiresIn; private final String scope; private final String refreshToken; + private final String kerberosAuthenticationResponseToken; - public CreateTokenResponse(String accessToken, String type, TimeValue expiresIn, String scope, String refreshToken) { + public CreateTokenResponse(String accessToken, String type, TimeValue expiresIn, String scope, String refreshToken, + String kerberosAuthenticationResponseToken) { this.accessToken = accessToken; this.type = type; this.expiresIn = expiresIn; this.scope = scope; this.refreshToken = refreshToken; + this.kerberosAuthenticationResponseToken = kerberosAuthenticationResponseToken; } public String getAccessToken() { @@ -70,6 +73,10 @@ public String getRefreshToken() { return refreshToken; } + public String getKerberosAuthenticationResponseToken() { + return kerberosAuthenticationResponseToken; + } + @Override public boolean equals(Object o) { if (this == o) { @@ -83,17 +90,18 @@ public boolean equals(Object o) { Objects.equals(type, that.type) && Objects.equals(expiresIn, that.expiresIn) && Objects.equals(scope, that.scope) && - Objects.equals(refreshToken, that.refreshToken); + Objects.equals(refreshToken, that.refreshToken) && + Objects.equals(kerberosAuthenticationResponseToken, that.kerberosAuthenticationResponseToken); } @Override public int hashCode() { - return Objects.hash(accessToken, type, expiresIn, scope, refreshToken); + return Objects.hash(accessToken, type, expiresIn, scope, refreshToken, kerberosAuthenticationResponseToken); } private static final ConstructingObjectParser PARSER = new ConstructingObjectParser<>( - "create_token_response", true, args -> new CreateTokenResponse( - (String) args[0], (String) args[1], TimeValue.timeValueSeconds((Long) args[2]), (String) args[3], (String) args[4])); + "create_token_response", true, args -> new CreateTokenResponse((String) args[0], (String) args[1], + TimeValue.timeValueSeconds((Long) args[2]), (String) args[3], (String) args[4], (String) args[5])); static { PARSER.declareString(constructorArg(), new ParseField("access_token")); @@ -101,6 +109,7 @@ public int hashCode() { PARSER.declareLong(constructorArg(), new ParseField("expires_in")); PARSER.declareStringOrNull(optionalConstructorArg(), new ParseField("scope")); PARSER.declareStringOrNull(optionalConstructorArg(), new ParseField("refresh_token")); + PARSER.declareStringOrNull(optionalConstructorArg(), new ParseField("kerberos_authentication_response_token")); } public static CreateTokenResponse fromXContent(XContentParser parser) throws IOException { diff --git a/client/rest-high-level/src/main/java/org/elasticsearch/client/security/GetBuiltinPrivilegesRequest.java b/client/rest-high-level/src/main/java/org/elasticsearch/client/security/GetBuiltinPrivilegesRequest.java new file mode 100644 index 0000000000000..543665580756c --- /dev/null +++ b/client/rest-high-level/src/main/java/org/elasticsearch/client/security/GetBuiltinPrivilegesRequest.java @@ -0,0 +1,40 @@ +/* + * 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.client.security; + +import org.apache.http.client.methods.HttpGet; +import org.elasticsearch.client.Request; +import org.elasticsearch.client.Validatable; + +/** + * Request object to retrieve the privilege that are builtin to the Elasticsearch cluster. + */ +public final class GetBuiltinPrivilegesRequest implements Validatable { + + public static final GetBuiltinPrivilegesRequest INSTANCE = new GetBuiltinPrivilegesRequest(); + + private GetBuiltinPrivilegesRequest() { + } + + public Request getRequest() { + return new Request(HttpGet.METHOD_NAME, "/_security/privilege/_builtin"); + } + +} diff --git a/client/rest-high-level/src/main/java/org/elasticsearch/client/security/GetBuiltinPrivilegesResponse.java b/client/rest-high-level/src/main/java/org/elasticsearch/client/security/GetBuiltinPrivilegesResponse.java new file mode 100644 index 0000000000000..a900ee4f976f4 --- /dev/null +++ b/client/rest-high-level/src/main/java/org/elasticsearch/client/security/GetBuiltinPrivilegesResponse.java @@ -0,0 +1,82 @@ +/* + * 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.client.security; + +import org.elasticsearch.common.ParseField; +import org.elasticsearch.common.xcontent.ConstructingObjectParser; +import org.elasticsearch.common.xcontent.XContentParser; + +import java.io.IOException; +import java.util.Collection; +import java.util.Objects; +import java.util.Set; + +import static org.elasticsearch.common.xcontent.ConstructingObjectParser.constructorArg; + +/** + * Get builtin privileges response + */ +public final class GetBuiltinPrivilegesResponse { + + private final Set clusterPrivileges; + private final Set indexPrivileges; + + public GetBuiltinPrivilegesResponse(Collection cluster, Collection index) { + this.clusterPrivileges = Set.copyOf(cluster); + this.indexPrivileges = Set.copyOf(index); + } + + public Set getClusterPrivileges() { + return clusterPrivileges; + } + + public Set getIndexPrivileges() { + return indexPrivileges; + } + + public static GetBuiltinPrivilegesResponse fromXContent(XContentParser parser) throws IOException { + return PARSER.parse(parser, null); + } + + @Override + public boolean equals(Object o) { + if (this == o) return true; + if (o == null || getClass() != o.getClass()) return false; + GetBuiltinPrivilegesResponse that = (GetBuiltinPrivilegesResponse) o; + return Objects.equals(this.clusterPrivileges, that.clusterPrivileges) + && Objects.equals(this.indexPrivileges, that.indexPrivileges); + } + + @Override + public int hashCode() { + return Objects.hash(clusterPrivileges, indexPrivileges); + } + + + @SuppressWarnings("unchecked") + private static final ConstructingObjectParser PARSER = new ConstructingObjectParser<>( + "get_builtin_privileges", true, + args -> new GetBuiltinPrivilegesResponse((Collection) args[0], (Collection) args[1])); + + static { + PARSER.declareStringArray(constructorArg(), new ParseField("cluster")); + PARSER.declareStringArray(constructorArg(), new ParseField("index")); + } +} diff --git a/client/rest-high-level/src/main/java/org/elasticsearch/client/security/user/privileges/Role.java b/client/rest-high-level/src/main/java/org/elasticsearch/client/security/user/privileges/Role.java index a3263e7f6e920..e8e1a104d8c2c 100644 --- a/client/rest-high-level/src/main/java/org/elasticsearch/client/security/user/privileges/Role.java +++ b/client/rest-high-level/src/main/java/org/elasticsearch/client/security/user/privileges/Role.java @@ -299,10 +299,12 @@ public static class ClusterPrivilegeName { public static final String NONE = "none"; public static final String ALL = "all"; public static final String MONITOR = "monitor"; + public static final String MONITOR_DATA_FRAME_TRANSFORMS = "monitor_data_frame_transforms"; public static final String MONITOR_ML = "monitor_ml"; public static final String MONITOR_WATCHER = "monitor_watcher"; public static final String MONITOR_ROLLUP = "monitor_rollup"; public static final String MANAGE = "manage"; + public static final String MANAGE_DATA_FRAME_TRANSFORMS = "manage_data_frame_transforms"; public static final String MANAGE_ML = "manage_ml"; public static final String MANAGE_WATCHER = "manage_watcher"; public static final String MANAGE_ROLLUP = "manage_rollup"; @@ -318,8 +320,9 @@ public static class ClusterPrivilegeName { public static final String READ_CCR = "read_ccr"; public static final String MANAGE_ILM = "manage_ilm"; public static final String READ_ILM = "read_ilm"; - public static final String[] ALL_ARRAY = new String[] { NONE, ALL, MONITOR, MONITOR_ML, MONITOR_WATCHER, MONITOR_ROLLUP, MANAGE, - MANAGE_ML, MANAGE_WATCHER, MANAGE_ROLLUP, MANAGE_INDEX_TEMPLATES, MANAGE_INGEST_PIPELINES, TRANSPORT_CLIENT, + public static final String[] ALL_ARRAY = new String[] { NONE, ALL, MONITOR, MONITOR_DATA_FRAME_TRANSFORMS, MONITOR_ML, + MONITOR_WATCHER, MONITOR_ROLLUP, MANAGE, MANAGE_DATA_FRAME_TRANSFORMS, + MANAGE_ML, MANAGE_WATCHER, MANAGE_ROLLUP, MANAGE_INDEX_TEMPLATES, MANAGE_INGEST_PIPELINES, TRANSPORT_CLIENT, MANAGE_SECURITY, MANAGE_SAML, MANAGE_OIDC, MANAGE_TOKEN, MANAGE_PIPELINE, MANAGE_CCR, READ_CCR, MANAGE_ILM, READ_ILM}; } diff --git a/client/rest-high-level/src/main/java/org/elasticsearch/client/xpack/XPackInfoResponse.java b/client/rest-high-level/src/main/java/org/elasticsearch/client/xpack/XPackInfoResponse.java index e8fc2ce89b251..62f19570c6bc8 100644 --- a/client/rest-high-level/src/main/java/org/elasticsearch/client/xpack/XPackInfoResponse.java +++ b/client/rest-high-level/src/main/java/org/elasticsearch/client/xpack/XPackInfoResponse.java @@ -318,11 +318,6 @@ public String name() { return name; } - @Nullable - public String description() { - return description; - } - public boolean available() { return available; } @@ -331,6 +326,11 @@ public boolean enabled() { return enabled; } + /** + * Return native code info + * @deprecated Use ML info api to find native code info + */ + @Deprecated @Nullable public Map nativeCodeInfo() { return nativeCodeInfo; diff --git a/client/rest-high-level/src/main/resources/META-INF/services/org.elasticsearch.plugins.spi.NamedXContentProvider b/client/rest-high-level/src/main/resources/META-INF/services/org.elasticsearch.plugins.spi.NamedXContentProvider index 4204a868246a5..dde81e43867d8 100644 --- a/client/rest-high-level/src/main/resources/META-INF/services/org.elasticsearch.plugins.spi.NamedXContentProvider +++ b/client/rest-high-level/src/main/resources/META-INF/services/org.elasticsearch.plugins.spi.NamedXContentProvider @@ -1 +1,4 @@ -org.elasticsearch.client.indexlifecycle.IndexLifecycleNamedXContentProvider \ No newline at end of file +org.elasticsearch.client.dataframe.DataFrameNamedXContentProvider +org.elasticsearch.client.indexlifecycle.IndexLifecycleNamedXContentProvider +org.elasticsearch.client.ml.dataframe.MlDataFrameAnalysisNamedXContentProvider +org.elasticsearch.client.ml.dataframe.evaluation.MlEvaluationNamedXContentProvider \ No newline at end of file diff --git a/client/rest-high-level/src/test/java/org/elasticsearch/client/AbstractHlrcWriteableXContentTestCase.java b/client/rest-high-level/src/test/java/org/elasticsearch/client/AbstractHlrcWriteableXContentTestCase.java new file mode 100644 index 0000000000000..432c2db99643b --- /dev/null +++ b/client/rest-high-level/src/test/java/org/elasticsearch/client/AbstractHlrcWriteableXContentTestCase.java @@ -0,0 +1,70 @@ +/* + * 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.client; + +import org.elasticsearch.common.io.stream.Writeable; +import org.elasticsearch.common.xcontent.ToXContent; +import org.elasticsearch.common.xcontent.XContentParser; +import org.elasticsearch.test.AbstractSerializingTestCase; + +import java.io.IOException; + +import static org.elasticsearch.test.AbstractXContentTestCase.xContentTester; + +/** + * @deprecated Use {@link AbstractResponseTestCase} instead of this class. + */ +// TODO: Remove and change subclasses to use AbstractResponseTestCase instead +@Deprecated +public abstract class AbstractHlrcWriteableXContentTestCase + extends AbstractSerializingTestCase { + + /** + * Generic test that creates new instance of HLRC request/response from the test instance and checks + * both for equality and asserts equality on the two queries. + */ + public final void testHlrcFromXContent() throws IOException { + xContentTester(this::createParser, this::createTestInstance, getToXContentParams(), + p -> convertHlrcToInternal(doHlrcParseInstance(p))) + .numberOfTestRuns(NUMBER_OF_TEST_RUNS) + .supportsUnknownFields(supportsUnknownFields()) + .shuffleFieldsExceptions(getShuffleFieldsExceptions()) + .randomFieldsExcludeFilter(getRandomFieldsExcludeFilter()) + .assertEqualsConsumer(this::assertEqualInstances) + .assertToXContentEquivalence(true) + .test(); + } + + /** + * Parses to a new HLRC instance using the provided {@link XContentParser} + */ + public abstract H doHlrcParseInstance(XContentParser parser) throws IOException; + + /** + * Converts a HLRC instance to a XPack instance + */ + public abstract T convertHlrcToInternal(H instance); + + //TODO this would be final ideally: why do both responses need to parse from xcontent, only one (H) should? I think that T#fromXContent + //are only there for testing and could go away? Then the additional testHlrcFromXContent is also no longer needed. + @Override + protected T doParseInstance(XContentParser parser) throws IOException { + return convertHlrcToInternal(doHlrcParseInstance(parser)); + } +} diff --git a/client/rest-high-level/src/test/java/org/elasticsearch/client/AbstractResponseTestCase.java b/client/rest-high-level/src/test/java/org/elasticsearch/client/AbstractResponseTestCase.java index 8565ca14a908a..2a91a639a5ac3 100644 --- a/client/rest-high-level/src/test/java/org/elasticsearch/client/AbstractResponseTestCase.java +++ b/client/rest-high-level/src/test/java/org/elasticsearch/client/AbstractResponseTestCase.java @@ -45,7 +45,7 @@ public final void testFromXContent() throws IOException { final S serverTestInstance = createServerTestInstance(); final XContentType xContentType = randomFrom(XContentType.values()); - final BytesReference bytes = toShuffledXContent(serverTestInstance, xContentType, ToXContent.EMPTY_PARAMS, randomBoolean()); + final BytesReference bytes = toShuffledXContent(serverTestInstance, xContentType, getParams(), randomBoolean()); final XContent xContent = XContentFactory.xContent(xContentType); final XContentParser parser = xContent.createParser( @@ -62,4 +62,8 @@ public final void testFromXContent() throws IOException { protected abstract void assertInstances(S serverTestInstance, C clientInstance); + protected ToXContent.Params getParams() { + return ToXContent.EMPTY_PARAMS; + } + } diff --git a/client/rest-high-level/src/test/java/org/elasticsearch/client/BulkProcessorIT.java b/client/rest-high-level/src/test/java/org/elasticsearch/client/BulkProcessorIT.java index 762e927551b8b..2aa9457bcd897 100644 --- a/client/rest-high-level/src/test/java/org/elasticsearch/client/BulkProcessorIT.java +++ b/client/rest-high-level/src/test/java/org/elasticsearch/client/BulkProcessorIT.java @@ -291,7 +291,6 @@ public void testBulkProcessorConcurrentRequestsReadOnlyIndex() throws Exception assertMultiGetResponse(highLevelClient().mget(multiGetRequest, RequestOptions.DEFAULT), testDocs); } - @SuppressWarnings("unchecked") public void testGlobalParametersAndSingleRequest() throws Exception { createIndexWithMultipleShards("test"); @@ -326,7 +325,6 @@ public void testGlobalParametersAndSingleRequest() throws Exception { assertThat(blogs, everyItem(hasProperty(fieldFromSource("fieldNameXYZ"), equalTo("valueXYZ")))); } - @SuppressWarnings("unchecked") public void testGlobalParametersAndBulkProcessor() throws Exception { createIndexWithMultipleShards("test"); diff --git a/client/rest-high-level/src/test/java/org/elasticsearch/client/BulkRequestWithGlobalParametersIT.java b/client/rest-high-level/src/test/java/org/elasticsearch/client/BulkRequestWithGlobalParametersIT.java index 3020eb0329b5c..dc49e6f88a6e4 100644 --- a/client/rest-high-level/src/test/java/org/elasticsearch/client/BulkRequestWithGlobalParametersIT.java +++ b/client/rest-high-level/src/test/java/org/elasticsearch/client/BulkRequestWithGlobalParametersIT.java @@ -44,7 +44,6 @@ public class BulkRequestWithGlobalParametersIT extends ESRestHighLevelClientTestCase { - @SuppressWarnings("unchecked") public void testGlobalPipelineOnBulkRequest() throws IOException { createFieldAddingPipleine("xyz", "fieldNameXYZ", "valueXYZ"); @@ -83,7 +82,6 @@ public void testPipelineOnRequestOverridesGlobalPipeline() throws IOException { assertThat(hits, everyItem(hasProperty(fieldFromSource("fieldXYZ"), nullValue()))); } - @SuppressWarnings("unchecked") public void testMixPipelineOnRequestAndGlobal() throws IOException { createFieldAddingPipleine("globalId", "fieldXYZ", "valueXYZ"); createFieldAddingPipleine("perIndexId", "someNewField", "someValue"); @@ -153,7 +151,6 @@ public void testGlobalType() throws IOException { assertThat(hits, everyItem(hasType("global_type"))); } - @SuppressWarnings("unchecked") public void testTypeGlobalAndPerRequest() throws IOException { BulkRequest request = new BulkRequest(null, "global_type"); request.add(new IndexRequest("index1", "local_type", "1") @@ -171,7 +168,6 @@ public void testTypeGlobalAndPerRequest() throws IOException { .and(hasType("global_type")))); } - @SuppressWarnings("unchecked") public void testGlobalRouting() throws IOException { createIndexWithMultipleShards("index"); BulkRequest request = new BulkRequest(null); @@ -189,7 +185,6 @@ public void testGlobalRouting() throws IOException { assertThat(hits, containsInAnyOrder(hasId("1"), hasId("2"))); } - @SuppressWarnings("unchecked") public void testMixLocalAndGlobalRouting() throws IOException { BulkRequest request = new BulkRequest(null); request.routing("globalRouting"); diff --git a/client/rest-high-level/src/test/java/org/elasticsearch/client/DataFrameRequestConvertersTests.java b/client/rest-high-level/src/test/java/org/elasticsearch/client/DataFrameRequestConvertersTests.java index 7a1e5e2389316..db111904f4704 100644 --- a/client/rest-high-level/src/test/java/org/elasticsearch/client/DataFrameRequestConvertersTests.java +++ b/client/rest-high-level/src/test/java/org/elasticsearch/client/DataFrameRequestConvertersTests.java @@ -23,6 +23,7 @@ import org.apache.http.client.methods.HttpGet; import org.apache.http.client.methods.HttpPost; import org.apache.http.client.methods.HttpPut; +import org.elasticsearch.client.dataframe.DataFrameNamedXContentProvider; import org.elasticsearch.client.core.PageParams; import org.elasticsearch.client.dataframe.DeleteDataFrameTransformRequest; import org.elasticsearch.client.dataframe.GetDataFrameTransformRequest; @@ -43,7 +44,9 @@ import java.io.IOException; import java.util.Collections; +import java.util.List; +import static org.elasticsearch.client.dataframe.GetDataFrameTransformRequest.ALLOW_NO_MATCH; import static org.hamcrest.Matchers.allOf; import static org.hamcrest.Matchers.equalTo; import static org.hamcrest.Matchers.hasEntry; @@ -52,8 +55,11 @@ public class DataFrameRequestConvertersTests extends ESTestCase { @Override protected NamedXContentRegistry xContentRegistry() { - SearchModule searchModule = new SearchModule(Settings.EMPTY, false, Collections.emptyList()); - return new NamedXContentRegistry(searchModule.getNamedXContents()); + SearchModule searchModule = new SearchModule(Settings.EMPTY, Collections.emptyList()); + List namedXContents = searchModule.getNamedXContents(); + namedXContents.addAll(new DataFrameNamedXContentProvider().getNamedXContentParsers()); + + return new NamedXContentRegistry(namedXContents); } public void testPutDataFrameTransform() throws IOException { @@ -110,7 +116,6 @@ public void testStopDataFrameTransform() { } StopDataFrameTransformRequest stopRequest = new StopDataFrameTransformRequest(id, waitForCompletion, timeValue); - Request request = DataFrameRequestConverters.stopDataFrameTransform(stopRequest); assertEquals(HttpPost.METHOD_NAME, request.getMethod()); assertThat(request.getEndpoint(), equalTo("/_data_frame/transforms/" + stopRequest.getId() + "/_stop")); @@ -128,6 +133,11 @@ public void testStopDataFrameTransform() { } else { assertFalse(request.getParameters().containsKey("timeout")); } + + assertFalse(request.getParameters().containsKey(ALLOW_NO_MATCH)); + stopRequest.setAllowNoMatch(randomBoolean()); + request = DataFrameRequestConverters.stopDataFrameTransform(stopRequest); + assertEquals(stopRequest.getAllowNoMatch(), Boolean.parseBoolean(request.getParameters().get(ALLOW_NO_MATCH))); } public void testPreviewDataFrameTransform() throws IOException { @@ -153,6 +163,7 @@ public void testGetDataFrameTransformStats() { assertFalse(request.getParameters().containsKey("from")); assertFalse(request.getParameters().containsKey("size")); + assertFalse(request.getParameters().containsKey(ALLOW_NO_MATCH)); getStatsRequest.setPageParams(new PageParams(0, null)); request = DataFrameRequestConverters.getDataFrameTransformStats(getStatsRequest); @@ -167,6 +178,10 @@ public void testGetDataFrameTransformStats() { getStatsRequest.setPageParams(new PageParams(0, 10)); request = DataFrameRequestConverters.getDataFrameTransformStats(getStatsRequest); assertThat(request.getParameters(), allOf(hasEntry("from", "0"), hasEntry("size", "10"))); + + getStatsRequest.setAllowNoMatch(false); + request = DataFrameRequestConverters.getDataFrameTransformStats(getStatsRequest); + assertThat(request.getParameters(), hasEntry("allow_no_match", "false")); } public void testGetDataFrameTransform() { @@ -178,6 +193,7 @@ public void testGetDataFrameTransform() { assertFalse(request.getParameters().containsKey("from")); assertFalse(request.getParameters().containsKey("size")); + assertFalse(request.getParameters().containsKey(ALLOW_NO_MATCH)); getRequest.setPageParams(new PageParams(0, null)); request = DataFrameRequestConverters.getDataFrameTransform(getRequest); @@ -192,6 +208,10 @@ public void testGetDataFrameTransform() { getRequest.setPageParams(new PageParams(0, 10)); request = DataFrameRequestConverters.getDataFrameTransform(getRequest); assertThat(request.getParameters(), allOf(hasEntry("from", "0"), hasEntry("size", "10"))); + + getRequest.setAllowNoMatch(false); + request = DataFrameRequestConverters.getDataFrameTransform(getRequest); + assertThat(request.getParameters(), hasEntry("allow_no_match", "false")); } public void testGetDataFrameTransform_givenMulitpleIds() { diff --git a/client/rest-high-level/src/test/java/org/elasticsearch/client/DataFrameTransformIT.java b/client/rest-high-level/src/test/java/org/elasticsearch/client/DataFrameTransformIT.java index 40cd6f454cdab..d7371468b1506 100644 --- a/client/rest-high-level/src/test/java/org/elasticsearch/client/DataFrameTransformIT.java +++ b/client/rest-high-level/src/test/java/org/elasticsearch/client/DataFrameTransformIT.java @@ -66,10 +66,12 @@ import java.util.Optional; import static org.elasticsearch.common.xcontent.XContentFactory.jsonBuilder; +import static org.hamcrest.CoreMatchers.nullValue; import static org.hamcrest.Matchers.containsString; import static org.hamcrest.Matchers.empty; import static org.hamcrest.Matchers.equalTo; import static org.hamcrest.Matchers.greaterThan; +import static org.hamcrest.Matchers.hasKey; import static org.hamcrest.Matchers.hasSize; import static org.hamcrest.Matchers.is; import static org.hamcrest.Matchers.oneOf; @@ -140,7 +142,7 @@ private void indexData(String indexName) throws IOException { } @After - public void cleanUpTransforms() throws IOException { + public void cleanUpTransforms() throws Exception { for (String transformId : transformsToClean) { highLevelClient().dataFrame().stopDataFrameTransform( new StopDataFrameTransformRequest(transformId, Boolean.TRUE, null), RequestOptions.DEFAULT); @@ -152,6 +154,7 @@ public void cleanUpTransforms() throws IOException { } transformsToClean = new ArrayList<>(); + waitForPendingTasks(adminClient()); } public void testCreateDelete() throws IOException { @@ -185,16 +188,14 @@ public void testGetTransform() throws IOException { DataFrameTransformConfig transform = validDataFrameTransformConfig(id, sourceIndex, "pivot-dest"); DataFrameClient client = highLevelClient().dataFrame(); - AcknowledgedResponse ack = execute(new PutDataFrameTransformRequest(transform), client::putDataFrameTransform, - client::putDataFrameTransformAsync); - assertTrue(ack.isAcknowledged()); + putTransform(transform); GetDataFrameTransformRequest getRequest = new GetDataFrameTransformRequest(id); GetDataFrameTransformResponse getResponse = execute(getRequest, client::getDataFrameTransform, client::getDataFrameTransformAsync); assertNull(getResponse.getInvalidTransforms()); assertThat(getResponse.getTransformConfigurations(), hasSize(1)); - assertEquals(transform, getResponse.getTransformConfigurations().get(0)); + assertEquals(transform.getId(), getResponse.getTransformConfigurations().get(0).getId()); } public void testGetAllAndPageTransforms() throws IOException { @@ -204,21 +205,17 @@ public void testGetAllAndPageTransforms() throws IOException { DataFrameClient client = highLevelClient().dataFrame(); DataFrameTransformConfig transform = validDataFrameTransformConfig("test-get-all-1", sourceIndex, "pivot-dest-1"); - AcknowledgedResponse ack = execute(new PutDataFrameTransformRequest(transform), client::putDataFrameTransform, - client::putDataFrameTransformAsync); - assertTrue(ack.isAcknowledged()); + putTransform(transform); transform = validDataFrameTransformConfig("test-get-all-2", sourceIndex, "pivot-dest-2"); - ack = execute(new PutDataFrameTransformRequest(transform), client::putDataFrameTransform, - client::putDataFrameTransformAsync); - assertTrue(ack.isAcknowledged()); + putTransform(transform); GetDataFrameTransformRequest getRequest = new GetDataFrameTransformRequest("_all"); GetDataFrameTransformResponse getResponse = execute(getRequest, client::getDataFrameTransform, client::getDataFrameTransformAsync); assertNull(getResponse.getInvalidTransforms()); assertThat(getResponse.getTransformConfigurations(), hasSize(2)); - assertEquals(transform, getResponse.getTransformConfigurations().get(1)); + assertEquals(transform.getId(), getResponse.getTransformConfigurations().get(1).getId()); getRequest.setPageParams(new PageParams(0,1)); getResponse = execute(getRequest, client::getDataFrameTransform, @@ -250,32 +247,38 @@ public void testStartStop() throws IOException { DataFrameTransformConfig transform = validDataFrameTransformConfig(id, sourceIndex, "pivot-dest"); DataFrameClient client = highLevelClient().dataFrame(); - AcknowledgedResponse ack = execute(new PutDataFrameTransformRequest(transform), client::putDataFrameTransform, - client::putDataFrameTransformAsync); - assertTrue(ack.isAcknowledged()); - transformsToClean.add(id); + putTransform(transform); StartDataFrameTransformRequest startRequest = new StartDataFrameTransformRequest(id); StartDataFrameTransformResponse startResponse = execute(startRequest, client::startDataFrameTransform, client::startDataFrameTransformAsync); - assertTrue(startResponse.isStarted()); + assertTrue(startResponse.isAcknowledged()); assertThat(startResponse.getNodeFailures(), empty()); assertThat(startResponse.getTaskFailures(), empty()); GetDataFrameTransformStatsResponse statsResponse = execute(new GetDataFrameTransformStatsRequest(id), client::getDataFrameTransformStats, client::getDataFrameTransformStatsAsync); assertThat(statsResponse.getTransformsStateAndStats(), hasSize(1)); - IndexerState indexerState = statsResponse.getTransformsStateAndStats().get(0).getTransformState().getIndexerState(); - assertThat(indexerState, is(oneOf(IndexerState.STARTED, IndexerState.INDEXING))); + DataFrameTransformTaskState taskState = statsResponse.getTransformsStateAndStats().get(0).getTransformState().getTaskState(); + + // Since we are non-continuous, the transform could auto-stop between being started earlier and us gathering the statistics + assertThat(taskState, is(oneOf(DataFrameTransformTaskState.STARTED, DataFrameTransformTaskState.STOPPED))); StopDataFrameTransformRequest stopRequest = new StopDataFrameTransformRequest(id, Boolean.TRUE, null); StopDataFrameTransformResponse stopResponse = execute(stopRequest, client::stopDataFrameTransform, client::stopDataFrameTransformAsync); - assertTrue(stopResponse.isStopped()); + assertTrue(stopResponse.isAcknowledged()); assertThat(stopResponse.getNodeFailures(), empty()); assertThat(stopResponse.getTaskFailures(), empty()); + + // Calling stop with wait_for_completion assures that we will be in the `STOPPED` state for the transform task + statsResponse = execute(new GetDataFrameTransformStatsRequest(id), + client::getDataFrameTransformStats, client::getDataFrameTransformStatsAsync); + taskState = statsResponse.getTransformsStateAndStats().get(0).getTransformState().getTaskState(); + assertThat(taskState, is(DataFrameTransformTaskState.STOPPED)); } + @SuppressWarnings("unchecked") public void testPreview() throws IOException { String sourceIndex = "transform-source"; createIndex(sourceIndex); @@ -297,6 +300,12 @@ public void testPreview() throws IOException { Optional> michel = docs.stream().filter(doc -> "michel".equals(doc.get("reviewer"))).findFirst(); assertTrue(michel.isPresent()); assertEquals(3.6d, (double) michel.get().get("avg_rating"), 0.1d); + + Map mappings = preview.getMappings(); + assertThat(mappings, hasKey("properties")); + Map fields = (Map)mappings.get("properties"); + assertThat(fields.get("reviewer"), equalTo(Map.of("type", "keyword"))); + assertThat(fields.get("avg_rating"), equalTo(Map.of("type", "double"))); } private DataFrameTransformConfig validDataFrameTransformConfig(String id, String source, String destination) { @@ -306,7 +315,7 @@ private DataFrameTransformConfig validDataFrameTransformConfig(String id, String aggBuilder.addAggregator(AggregationBuilders.avg("avg_rating").field("stars")); PivotConfig pivotConfig = PivotConfig.builder().setGroups(groupConfig).setAggregations(aggBuilder).build(); - DestConfig destConfig = (destination != null) ? new DestConfig(destination) : null; + DestConfig destConfig = (destination != null) ? DestConfig.builder().setIndex(destination).build() : null; return DataFrameTransformConfig.builder() .setId(id) @@ -317,6 +326,7 @@ private DataFrameTransformConfig validDataFrameTransformConfig(String id, String .build(); } + // TODO add tests to cover continuous situations public void testGetStats() throws Exception { String sourceIndex = "transform-source"; createIndex(sourceIndex); @@ -332,16 +342,13 @@ public void testGetStats() throws Exception { DataFrameTransformConfig transform = DataFrameTransformConfig.builder() .setId(id) .setSource(SourceConfig.builder().setIndex(sourceIndex).setQuery(new MatchAllQueryBuilder()).build()) - .setDest(new DestConfig("pivot-dest")) + .setDest(DestConfig.builder().setIndex("pivot-dest").build()) .setPivotConfig(pivotConfig) .setDescription("transform for testing stats") .build(); DataFrameClient client = highLevelClient().dataFrame(); - AcknowledgedResponse ack = execute(new PutDataFrameTransformRequest(transform), client::putDataFrameTransform, - client::putDataFrameTransformAsync); - assertTrue(ack.isAcknowledged()); - transformsToClean.add(id); + putTransform(transform); GetDataFrameTransformStatsResponse statsResponse = execute(new GetDataFrameTransformStatsRequest(id), client::getDataFrameTransformStats, client::getDataFrameTransformStatsAsync); @@ -358,20 +365,30 @@ public void testGetStats() throws Exception { StartDataFrameTransformResponse startTransformResponse = execute(new StartDataFrameTransformRequest(id), client::startDataFrameTransform, client::startDataFrameTransformAsync); - assertThat(startTransformResponse.isStarted(), is(true)); + assertThat(startTransformResponse.isAcknowledged(), is(true)); assertBusy(() -> { GetDataFrameTransformStatsResponse response = execute(new GetDataFrameTransformStatsRequest(id), client::getDataFrameTransformStats, client::getDataFrameTransformStatsAsync); DataFrameTransformStateAndStats stateAndStats = response.getTransformsStateAndStats().get(0); - assertEquals(IndexerState.STARTED, stateAndStats.getTransformState().getIndexerState()); - assertEquals(DataFrameTransformTaskState.STARTED, stateAndStats.getTransformState().getTaskState()); - assertEquals(null, stateAndStats.getTransformState().getReason()); assertNotEquals(zeroIndexerStats, stateAndStats.getTransformStats()); assertNotNull(stateAndStats.getTransformState().getProgress()); + assertThat(stateAndStats.getTransformState().getTaskState(), + is(oneOf(DataFrameTransformTaskState.STARTED, DataFrameTransformTaskState.STOPPED))); + assertThat(stateAndStats.getTransformState().getIndexerState(), + is(oneOf(IndexerState.STARTED, IndexerState.STOPPED))); assertThat(stateAndStats.getTransformState().getProgress().getPercentComplete(), equalTo(100.0)); assertThat(stateAndStats.getTransformState().getProgress().getTotalDocs(), greaterThan(0L)); assertThat(stateAndStats.getTransformState().getProgress().getRemainingDocs(), equalTo(0L)); + assertThat(stateAndStats.getTransformState().getReason(), is(nullValue())); }); } + + void putTransform(DataFrameTransformConfig config) throws IOException { + DataFrameClient client = highLevelClient().dataFrame(); + AcknowledgedResponse ack = execute(new PutDataFrameTransformRequest(config), client::putDataFrameTransform, + client::putDataFrameTransformAsync); + assertTrue(ack.isAcknowledged()); + transformsToClean.add(config.getId()); + } } diff --git a/client/rest-high-level/src/test/java/org/elasticsearch/client/ESRestHighLevelClientTestCase.java b/client/rest-high-level/src/test/java/org/elasticsearch/client/ESRestHighLevelClientTestCase.java index f758156c222a8..afbdb9fe60058 100644 --- a/client/rest-high-level/src/test/java/org/elasticsearch/client/ESRestHighLevelClientTestCase.java +++ b/client/rest-high-level/src/test/java/org/elasticsearch/client/ESRestHighLevelClientTestCase.java @@ -131,7 +131,7 @@ protected interface AsyncMethodNoRequest { private static class HighLevelClient extends RestHighLevelClient { private HighLevelClient(RestClient restClient) { - super(restClient, (client) -> {}, new SearchModule(Settings.EMPTY, false, Collections.emptyList()).getNamedXContents()); + super(restClient, (client) -> {}, new SearchModule(Settings.EMPTY, Collections.emptyList()).getNamedXContents()); } } diff --git a/client/rest-high-level/src/test/java/org/elasticsearch/client/IndicesClientIT.java b/client/rest-high-level/src/test/java/org/elasticsearch/client/IndicesClientIT.java index d9adf61782b3d..59d76142566e6 100644 --- a/client/rest-high-level/src/test/java/org/elasticsearch/client/IndicesClientIT.java +++ b/client/rest-high-level/src/test/java/org/elasticsearch/client/IndicesClientIT.java @@ -28,8 +28,6 @@ import org.elasticsearch.action.admin.indices.alias.IndicesAliasesRequest; import org.elasticsearch.action.admin.indices.alias.IndicesAliasesRequest.AliasActions; import org.elasticsearch.action.admin.indices.alias.get.GetAliasesRequest; -import org.elasticsearch.action.admin.indices.analyze.AnalyzeRequest; -import org.elasticsearch.action.admin.indices.analyze.AnalyzeResponse; import org.elasticsearch.action.admin.indices.cache.clear.ClearIndicesCacheRequest; import org.elasticsearch.action.admin.indices.cache.clear.ClearIndicesCacheResponse; import org.elasticsearch.action.admin.indices.close.CloseIndexRequest; @@ -58,6 +56,8 @@ import org.elasticsearch.action.support.broadcast.BroadcastResponse; import org.elasticsearch.action.support.master.AcknowledgedResponse; import org.elasticsearch.client.core.ShardsAcknowledgedResponse; +import org.elasticsearch.client.indices.AnalyzeRequest; +import org.elasticsearch.client.indices.AnalyzeResponse; import org.elasticsearch.client.indices.CreateIndexRequest; import org.elasticsearch.client.indices.CreateIndexResponse; import org.elasticsearch.client.indices.FreezeIndexRequest; @@ -73,6 +73,8 @@ import org.elasticsearch.client.indices.IndexTemplatesExistRequest; import org.elasticsearch.client.indices.PutIndexTemplateRequest; import org.elasticsearch.client.indices.PutMappingRequest; +import org.elasticsearch.client.indices.ReloadAnalyzersRequest; +import org.elasticsearch.client.indices.ReloadAnalyzersResponse; import org.elasticsearch.client.indices.UnfreezeIndexRequest; import org.elasticsearch.client.indices.rollover.RolloverRequest; import org.elasticsearch.client.indices.rollover.RolloverResponse; @@ -1852,12 +1854,12 @@ public void testAnalyze() throws Exception { RestHighLevelClient client = highLevelClient(); - AnalyzeRequest noindexRequest = new AnalyzeRequest().text("One two three").analyzer("english"); + AnalyzeRequest noindexRequest = AnalyzeRequest.withGlobalAnalyzer("english", "One two three"); AnalyzeResponse noindexResponse = execute(noindexRequest, client.indices()::analyze, client.indices()::analyzeAsync); assertThat(noindexResponse.getTokens(), hasSize(3)); - AnalyzeRequest detailsRequest = new AnalyzeRequest().text("One two three").analyzer("english").explain(true); + AnalyzeRequest detailsRequest = AnalyzeRequest.withGlobalAnalyzer("english", "One two three").explain(true); AnalyzeResponse detailsResponse = execute(detailsRequest, client.indices()::analyze, client.indices()::analyzeAsync); assertNotNull(detailsResponse.detail()); @@ -1877,4 +1879,14 @@ public void testFreezeAndUnfreeze() throws IOException { assertTrue(unfreeze.isShardsAcknowledged()); assertTrue(unfreeze.isAcknowledged()); } + + public void testReloadAnalyzer() throws IOException { + createIndex("test", Settings.EMPTY); + RestHighLevelClient client = highLevelClient(); + + ReloadAnalyzersResponse reloadResponse = execute(new ReloadAnalyzersRequest("test"), client.indices()::reloadAnalyzers, + client.indices()::reloadAnalyzersAsync); + assertNotNull(reloadResponse.shards()); + assertTrue(reloadResponse.getReloadedDetails().containsKey("test")); + } } diff --git a/client/rest-high-level/src/test/java/org/elasticsearch/client/IndicesRequestConvertersTests.java b/client/rest-high-level/src/test/java/org/elasticsearch/client/IndicesRequestConvertersTests.java index f7d5ac51a73ac..d0f4177635797 100644 --- a/client/rest-high-level/src/test/java/org/elasticsearch/client/IndicesRequestConvertersTests.java +++ b/client/rest-high-level/src/test/java/org/elasticsearch/client/IndicesRequestConvertersTests.java @@ -29,7 +29,6 @@ import org.elasticsearch.action.admin.indices.alias.Alias; import org.elasticsearch.action.admin.indices.alias.IndicesAliasesRequest; import org.elasticsearch.action.admin.indices.alias.get.GetAliasesRequest; -import org.elasticsearch.action.admin.indices.analyze.AnalyzeRequest; import org.elasticsearch.action.admin.indices.cache.clear.ClearIndicesCacheRequest; import org.elasticsearch.action.admin.indices.close.CloseIndexRequest; import org.elasticsearch.action.admin.indices.delete.DeleteIndexRequest; @@ -45,6 +44,7 @@ import org.elasticsearch.action.admin.indices.template.delete.DeleteIndexTemplateRequest; import org.elasticsearch.action.admin.indices.validate.query.ValidateQueryRequest; import org.elasticsearch.action.support.master.AcknowledgedRequest; +import org.elasticsearch.client.indices.AnalyzeRequest; import org.elasticsearch.client.indices.CreateIndexRequest; import org.elasticsearch.client.indices.GetFieldMappingsRequest; import org.elasticsearch.client.indices.GetIndexRequest; @@ -54,6 +54,7 @@ import org.elasticsearch.client.indices.PutIndexTemplateRequest; import org.elasticsearch.client.indices.PutMappingRequest; import org.elasticsearch.client.indices.RandomCreateIndexGenerator; +import org.elasticsearch.client.indices.ReloadAnalyzersRequest; import org.elasticsearch.client.indices.rollover.RolloverRequest; import org.elasticsearch.common.CheckedFunction; import org.elasticsearch.common.Strings; @@ -86,18 +87,14 @@ public class IndicesRequestConvertersTests extends ESTestCase { public void testAnalyzeRequest() throws Exception { - AnalyzeRequest indexAnalyzeRequest = new AnalyzeRequest() - .text("Here is some text") - .index("test_index") - .analyzer("test_analyzer"); + AnalyzeRequest indexAnalyzeRequest + = AnalyzeRequest.withIndexAnalyzer("test_index", "test_analyzer", "Here is some text"); Request request = IndicesRequestConverters.analyze(indexAnalyzeRequest); assertThat(request.getEndpoint(), equalTo("/test_index/_analyze")); RequestConvertersTests.assertToXContentBody(indexAnalyzeRequest, request.getEntity()); - AnalyzeRequest analyzeRequest = new AnalyzeRequest() - .text("more text") - .analyzer("test_analyzer"); + AnalyzeRequest analyzeRequest = AnalyzeRequest.withGlobalAnalyzer("test_analyzer", "more text"); assertThat(IndicesRequestConverters.analyze(analyzeRequest).getEndpoint(), equalTo("/_analyze")); } @@ -1219,4 +1216,21 @@ public void testDeleteTemplateRequest() { Assert.assertThat(request.getParameters(), equalTo(expectedParams)); Assert.assertThat(request.getEntity(), nullValue()); } + + public void testReloadAnalyzers() { + String[] indices = RequestConvertersTests.randomIndicesNames(1, 5); + StringJoiner endpoint = new StringJoiner("/", "/", ""); + if (indices != null && indices.length > 0) { + endpoint.add(String.join(",", indices)); + } + ReloadAnalyzersRequest reloadRequest = new ReloadAnalyzersRequest(indices); + Map expectedParams = new HashMap<>(); + RequestConvertersTests.setRandomIndicesOptions(reloadRequest::setIndicesOptions, reloadRequest::indicesOptions, + expectedParams); + Request request = IndicesRequestConverters.reloadAnalyzers(reloadRequest); + Assert.assertThat(request.getMethod(), equalTo(HttpPost.METHOD_NAME)); + Assert.assertThat(request.getEndpoint(), equalTo(endpoint + "/_reload_search_analyzers")); + Assert.assertThat(request.getParameters(), equalTo(expectedParams)); + Assert.assertThat(request.getEntity(), nullValue()); + } } diff --git a/client/rest-high-level/src/test/java/org/elasticsearch/client/MLRequestConvertersTests.java b/client/rest-high-level/src/test/java/org/elasticsearch/client/MLRequestConvertersTests.java index fd867a12204d0..aff25b4aa2d3b 100644 --- a/client/rest-high-level/src/test/java/org/elasticsearch/client/MLRequestConvertersTests.java +++ b/client/rest-high-level/src/test/java/org/elasticsearch/client/MLRequestConvertersTests.java @@ -28,12 +28,14 @@ import org.elasticsearch.client.ml.DeleteCalendarEventRequest; import org.elasticsearch.client.ml.DeleteCalendarJobRequest; import org.elasticsearch.client.ml.DeleteCalendarRequest; +import org.elasticsearch.client.ml.DeleteDataFrameAnalyticsRequest; import org.elasticsearch.client.ml.DeleteDatafeedRequest; import org.elasticsearch.client.ml.DeleteExpiredDataRequest; import org.elasticsearch.client.ml.DeleteFilterRequest; import org.elasticsearch.client.ml.DeleteForecastRequest; import org.elasticsearch.client.ml.DeleteJobRequest; import org.elasticsearch.client.ml.DeleteModelSnapshotRequest; +import org.elasticsearch.client.ml.EvaluateDataFrameRequest; import org.elasticsearch.client.ml.FindFileStructureRequest; import org.elasticsearch.client.ml.FindFileStructureRequestTests; import org.elasticsearch.client.ml.FlushJobRequest; @@ -42,6 +44,8 @@ import org.elasticsearch.client.ml.GetCalendarEventsRequest; import org.elasticsearch.client.ml.GetCalendarsRequest; import org.elasticsearch.client.ml.GetCategoriesRequest; +import org.elasticsearch.client.ml.GetDataFrameAnalyticsRequest; +import org.elasticsearch.client.ml.GetDataFrameAnalyticsStatsRequest; import org.elasticsearch.client.ml.GetDatafeedRequest; import org.elasticsearch.client.ml.GetDatafeedStatsRequest; import org.elasticsearch.client.ml.GetFiltersRequest; @@ -58,13 +62,16 @@ import org.elasticsearch.client.ml.PreviewDatafeedRequest; import org.elasticsearch.client.ml.PutCalendarJobRequest; import org.elasticsearch.client.ml.PutCalendarRequest; +import org.elasticsearch.client.ml.PutDataFrameAnalyticsRequest; import org.elasticsearch.client.ml.PutDatafeedRequest; import org.elasticsearch.client.ml.PutFilterRequest; import org.elasticsearch.client.ml.PutJobRequest; import org.elasticsearch.client.ml.RevertModelSnapshotRequest; import org.elasticsearch.client.ml.SetUpgradeModeRequest; +import org.elasticsearch.client.ml.StartDataFrameAnalyticsRequest; import org.elasticsearch.client.ml.StartDatafeedRequest; import org.elasticsearch.client.ml.StartDatafeedRequestTests; +import org.elasticsearch.client.ml.StopDataFrameAnalyticsRequest; import org.elasticsearch.client.ml.StopDatafeedRequest; import org.elasticsearch.client.ml.UpdateFilterRequest; import org.elasticsearch.client.ml.UpdateJobRequest; @@ -75,6 +82,12 @@ import org.elasticsearch.client.ml.calendars.ScheduledEventTests; import org.elasticsearch.client.ml.datafeed.DatafeedConfig; import org.elasticsearch.client.ml.datafeed.DatafeedConfigTests; +import org.elasticsearch.client.ml.dataframe.DataFrameAnalyticsConfig; +import org.elasticsearch.client.ml.dataframe.MlDataFrameAnalysisNamedXContentProvider; +import org.elasticsearch.client.ml.dataframe.evaluation.MlEvaluationNamedXContentProvider; +import org.elasticsearch.client.ml.dataframe.evaluation.softclassification.BinarySoftClassification; +import org.elasticsearch.client.ml.dataframe.evaluation.softclassification.PrecisionMetric; +import org.elasticsearch.client.ml.dataframe.evaluation.softclassification.RecallMetric; import org.elasticsearch.client.ml.filestructurefinder.FileStructure; import org.elasticsearch.client.ml.job.config.AnalysisConfig; import org.elasticsearch.client.ml.job.config.Detector; @@ -84,23 +97,30 @@ import org.elasticsearch.client.ml.job.config.MlFilter; import org.elasticsearch.client.ml.job.config.MlFilterTests; import org.elasticsearch.common.Strings; +import org.elasticsearch.common.settings.Settings; import org.elasticsearch.common.unit.TimeValue; +import org.elasticsearch.common.xcontent.NamedXContentRegistry; import org.elasticsearch.common.xcontent.XContentBuilder; import org.elasticsearch.common.xcontent.XContentParser; import org.elasticsearch.common.xcontent.XContentType; import org.elasticsearch.common.xcontent.json.JsonXContent; +import org.elasticsearch.search.SearchModule; import org.elasticsearch.test.ESTestCase; import java.io.ByteArrayOutputStream; import java.io.IOException; import java.nio.charset.StandardCharsets; +import java.util.ArrayList; import java.util.Arrays; import java.util.Collections; import java.util.HashMap; import java.util.List; import java.util.Map; +import static org.elasticsearch.client.ml.dataframe.DataFrameAnalyticsConfigTests.randomDataFrameAnalyticsConfig; +import static org.hamcrest.Matchers.allOf; import static org.hamcrest.Matchers.equalTo; +import static org.hamcrest.Matchers.hasEntry; import static org.hamcrest.Matchers.is; import static org.hamcrest.core.IsNull.nullValue; @@ -154,7 +174,6 @@ public void testGetJobStats() { assertEquals(Boolean.toString(true), request.getParameters().get("allow_no_jobs")); } - public void testOpenJob() throws Exception { String jobId = "some-job-id"; OpenJobRequest openJobRequest = new OpenJobRequest(jobId); @@ -669,6 +688,113 @@ public void testDeleteCalendarEvent() { assertEquals("/_ml/calendars/" + calendarId + "/events/" + eventId, request.getEndpoint()); } + public void testPutDataFrameAnalytics() throws IOException { + PutDataFrameAnalyticsRequest putRequest = new PutDataFrameAnalyticsRequest(randomDataFrameAnalyticsConfig()); + Request request = MLRequestConverters.putDataFrameAnalytics(putRequest); + assertEquals(HttpPut.METHOD_NAME, request.getMethod()); + assertEquals("/_ml/data_frame/analytics/" + putRequest.getConfig().getId(), request.getEndpoint()); + try (XContentParser parser = createParser(JsonXContent.jsonXContent, request.getEntity().getContent())) { + DataFrameAnalyticsConfig parsedConfig = DataFrameAnalyticsConfig.fromXContent(parser); + assertThat(parsedConfig, equalTo(putRequest.getConfig())); + } + } + + public void testGetDataFrameAnalytics() { + String configId1 = randomAlphaOfLength(10); + String configId2 = randomAlphaOfLength(10); + String configId3 = randomAlphaOfLength(10); + GetDataFrameAnalyticsRequest getRequest = new GetDataFrameAnalyticsRequest(configId1, configId2, configId3) + .setAllowNoMatch(false) + .setPageParams(new PageParams(100, 300)); + + Request request = MLRequestConverters.getDataFrameAnalytics(getRequest); + assertEquals(HttpGet.METHOD_NAME, request.getMethod()); + assertEquals("/_ml/data_frame/analytics/" + configId1 + "," + configId2 + "," + configId3, request.getEndpoint()); + assertThat(request.getParameters(), allOf(hasEntry("from", "100"), hasEntry("size", "300"), hasEntry("allow_no_match", "false"))); + assertNull(request.getEntity()); + } + + public void testGetDataFrameAnalyticsStats() { + String configId1 = randomAlphaOfLength(10); + String configId2 = randomAlphaOfLength(10); + String configId3 = randomAlphaOfLength(10); + GetDataFrameAnalyticsStatsRequest getStatsRequest = new GetDataFrameAnalyticsStatsRequest(configId1, configId2, configId3) + .setAllowNoMatch(false) + .setPageParams(new PageParams(100, 300)); + + Request request = MLRequestConverters.getDataFrameAnalyticsStats(getStatsRequest); + assertEquals(HttpGet.METHOD_NAME, request.getMethod()); + assertEquals("/_ml/data_frame/analytics/" + configId1 + "," + configId2 + "," + configId3 + "/_stats", request.getEndpoint()); + assertThat(request.getParameters(), allOf(hasEntry("from", "100"), hasEntry("size", "300"), hasEntry("allow_no_match", "false"))); + assertNull(request.getEntity()); + } + + public void testStartDataFrameAnalytics() { + StartDataFrameAnalyticsRequest startRequest = new StartDataFrameAnalyticsRequest(randomAlphaOfLength(10)); + Request request = MLRequestConverters.startDataFrameAnalytics(startRequest); + assertEquals(HttpPost.METHOD_NAME, request.getMethod()); + assertEquals("/_ml/data_frame/analytics/" + startRequest.getId() + "/_start", request.getEndpoint()); + assertNull(request.getEntity()); + } + + public void testStartDataFrameAnalytics_WithTimeout() { + StartDataFrameAnalyticsRequest startRequest = new StartDataFrameAnalyticsRequest(randomAlphaOfLength(10)) + .setTimeout(TimeValue.timeValueMinutes(1)); + Request request = MLRequestConverters.startDataFrameAnalytics(startRequest); + assertEquals(HttpPost.METHOD_NAME, request.getMethod()); + assertEquals("/_ml/data_frame/analytics/" + startRequest.getId() + "/_start", request.getEndpoint()); + assertThat(request.getParameters(), hasEntry("timeout", "1m")); + assertNull(request.getEntity()); + } + + public void testStopDataFrameAnalytics() { + StopDataFrameAnalyticsRequest stopRequest = new StopDataFrameAnalyticsRequest(randomAlphaOfLength(10)); + Request request = MLRequestConverters.stopDataFrameAnalytics(stopRequest); + assertEquals(HttpPost.METHOD_NAME, request.getMethod()); + assertEquals("/_ml/data_frame/analytics/" + stopRequest.getId() + "/_stop", request.getEndpoint()); + assertNull(request.getEntity()); + } + + public void testStopDataFrameAnalytics_WithParams() { + StopDataFrameAnalyticsRequest stopRequest = new StopDataFrameAnalyticsRequest(randomAlphaOfLength(10)) + .setTimeout(TimeValue.timeValueMinutes(1)) + .setAllowNoMatch(false) + .setForce(true); + Request request = MLRequestConverters.stopDataFrameAnalytics(stopRequest); + assertEquals(HttpPost.METHOD_NAME, request.getMethod()); + assertEquals("/_ml/data_frame/analytics/" + stopRequest.getId() + "/_stop", request.getEndpoint()); + assertThat(request.getParameters(), allOf( + hasEntry("timeout", "1m"), + hasEntry("allow_no_match", "false"), + hasEntry("force", "true"))); + assertNull(request.getEntity()); + } + + public void testDeleteDataFrameAnalytics() { + DeleteDataFrameAnalyticsRequest deleteRequest = new DeleteDataFrameAnalyticsRequest(randomAlphaOfLength(10)); + Request request = MLRequestConverters.deleteDataFrameAnalytics(deleteRequest); + assertEquals(HttpDelete.METHOD_NAME, request.getMethod()); + assertEquals("/_ml/data_frame/analytics/" + deleteRequest.getId(), request.getEndpoint()); + assertNull(request.getEntity()); + } + + public void testEvaluateDataFrame() throws IOException { + EvaluateDataFrameRequest evaluateRequest = + new EvaluateDataFrameRequest( + Arrays.asList(generateRandomStringArray(1, 10, false, false)), + new BinarySoftClassification( + randomAlphaOfLengthBetween(1, 10), + randomAlphaOfLengthBetween(1, 10), + PrecisionMetric.at(0.5), RecallMetric.at(0.6, 0.7))); + Request request = MLRequestConverters.evaluateDataFrame(evaluateRequest); + assertEquals(HttpPost.METHOD_NAME, request.getMethod()); + assertEquals("/_ml/data_frame/_evaluate", request.getEndpoint()); + try (XContentParser parser = createParser(JsonXContent.jsonXContent, request.getEntity().getContent())) { + EvaluateDataFrameRequest parsedRequest = EvaluateDataFrameRequest.fromXContent(parser); + assertThat(parsedRequest, equalTo(evaluateRequest)); + } + } + public void testPutFilter() throws IOException { MlFilter filter = MlFilterTests.createRandomBuilder("foo").build(); PutFilterRequest putFilterRequest = new PutFilterRequest(filter); @@ -835,6 +961,15 @@ public void testSetUpgradeMode() { assertThat(request.getParameters().get(SetUpgradeModeRequest.TIMEOUT.getPreferredName()), is("1h")); } + @Override + protected NamedXContentRegistry xContentRegistry() { + List namedXContent = new ArrayList<>(); + namedXContent.addAll(new SearchModule(Settings.EMPTY, Collections.emptyList()).getNamedXContents()); + namedXContent.addAll(new MlDataFrameAnalysisNamedXContentProvider().getNamedXContentParsers()); + namedXContent.addAll(new MlEvaluationNamedXContentProvider().getNamedXContentParsers()); + return new NamedXContentRegistry(namedXContent); + } + private static Job createValidJob(String jobId) { AnalysisConfig.Builder analysisConfig = AnalysisConfig.builder(Collections.singletonList( Detector.builder().setFunction("count").build())); diff --git a/client/rest-high-level/src/test/java/org/elasticsearch/client/MachineLearningIT.java b/client/rest-high-level/src/test/java/org/elasticsearch/client/MachineLearningIT.java index 25e610d67814f..93417cab1b393 100644 --- a/client/rest-high-level/src/test/java/org/elasticsearch/client/MachineLearningIT.java +++ b/client/rest-high-level/src/test/java/org/elasticsearch/client/MachineLearningIT.java @@ -29,11 +29,13 @@ import org.elasticsearch.action.update.UpdateRequest; import org.elasticsearch.client.core.PageParams; import org.elasticsearch.client.indices.CreateIndexRequest; +import org.elasticsearch.client.indices.GetIndexRequest; import org.elasticsearch.client.ml.CloseJobRequest; import org.elasticsearch.client.ml.CloseJobResponse; import org.elasticsearch.client.ml.DeleteCalendarEventRequest; import org.elasticsearch.client.ml.DeleteCalendarJobRequest; import org.elasticsearch.client.ml.DeleteCalendarRequest; +import org.elasticsearch.client.ml.DeleteDataFrameAnalyticsRequest; import org.elasticsearch.client.ml.DeleteDatafeedRequest; import org.elasticsearch.client.ml.DeleteExpiredDataRequest; import org.elasticsearch.client.ml.DeleteExpiredDataResponse; @@ -42,6 +44,8 @@ import org.elasticsearch.client.ml.DeleteJobRequest; import org.elasticsearch.client.ml.DeleteJobResponse; import org.elasticsearch.client.ml.DeleteModelSnapshotRequest; +import org.elasticsearch.client.ml.EvaluateDataFrameRequest; +import org.elasticsearch.client.ml.EvaluateDataFrameResponse; import org.elasticsearch.client.ml.FindFileStructureRequest; import org.elasticsearch.client.ml.FindFileStructureResponse; import org.elasticsearch.client.ml.FlushJobRequest; @@ -52,6 +56,10 @@ import org.elasticsearch.client.ml.GetCalendarEventsResponse; import org.elasticsearch.client.ml.GetCalendarsRequest; import org.elasticsearch.client.ml.GetCalendarsResponse; +import org.elasticsearch.client.ml.GetDataFrameAnalyticsRequest; +import org.elasticsearch.client.ml.GetDataFrameAnalyticsResponse; +import org.elasticsearch.client.ml.GetDataFrameAnalyticsStatsRequest; +import org.elasticsearch.client.ml.GetDataFrameAnalyticsStatsResponse; import org.elasticsearch.client.ml.GetDatafeedRequest; import org.elasticsearch.client.ml.GetDatafeedResponse; import org.elasticsearch.client.ml.GetDatafeedStatsRequest; @@ -77,6 +85,8 @@ import org.elasticsearch.client.ml.PutCalendarJobRequest; import org.elasticsearch.client.ml.PutCalendarRequest; import org.elasticsearch.client.ml.PutCalendarResponse; +import org.elasticsearch.client.ml.PutDataFrameAnalyticsRequest; +import org.elasticsearch.client.ml.PutDataFrameAnalyticsResponse; import org.elasticsearch.client.ml.PutDatafeedRequest; import org.elasticsearch.client.ml.PutDatafeedResponse; import org.elasticsearch.client.ml.PutFilterRequest; @@ -86,8 +96,11 @@ import org.elasticsearch.client.ml.RevertModelSnapshotRequest; import org.elasticsearch.client.ml.RevertModelSnapshotResponse; import org.elasticsearch.client.ml.SetUpgradeModeRequest; +import org.elasticsearch.client.ml.StartDataFrameAnalyticsRequest; import org.elasticsearch.client.ml.StartDatafeedRequest; import org.elasticsearch.client.ml.StartDatafeedResponse; +import org.elasticsearch.client.ml.StopDataFrameAnalyticsRequest; +import org.elasticsearch.client.ml.StopDataFrameAnalyticsResponse; import org.elasticsearch.client.ml.StopDatafeedRequest; import org.elasticsearch.client.ml.StopDatafeedResponse; import org.elasticsearch.client.ml.UpdateDatafeedRequest; @@ -103,6 +116,18 @@ import org.elasticsearch.client.ml.datafeed.DatafeedState; import org.elasticsearch.client.ml.datafeed.DatafeedStats; import org.elasticsearch.client.ml.datafeed.DatafeedUpdate; +import org.elasticsearch.client.ml.dataframe.DataFrameAnalyticsConfig; +import org.elasticsearch.client.ml.dataframe.DataFrameAnalyticsDest; +import org.elasticsearch.client.ml.dataframe.DataFrameAnalyticsSource; +import org.elasticsearch.client.ml.dataframe.DataFrameAnalyticsState; +import org.elasticsearch.client.ml.dataframe.DataFrameAnalyticsStats; +import org.elasticsearch.client.ml.dataframe.OutlierDetection; +import org.elasticsearch.client.ml.dataframe.QueryConfig; +import org.elasticsearch.client.ml.dataframe.evaluation.softclassification.AucRocMetric; +import org.elasticsearch.client.ml.dataframe.evaluation.softclassification.BinarySoftClassification; +import org.elasticsearch.client.ml.dataframe.evaluation.softclassification.ConfusionMatrixMetric; +import org.elasticsearch.client.ml.dataframe.evaluation.softclassification.PrecisionMetric; +import org.elasticsearch.client.ml.dataframe.evaluation.softclassification.RecallMetric; import org.elasticsearch.client.ml.filestructurefinder.FileStructure; import org.elasticsearch.client.ml.job.config.AnalysisConfig; import org.elasticsearch.client.ml.job.config.DataDescription; @@ -113,9 +138,12 @@ import org.elasticsearch.client.ml.job.config.MlFilter; import org.elasticsearch.client.ml.job.process.ModelSnapshot; import org.elasticsearch.client.ml.job.stats.JobStats; +import org.elasticsearch.common.unit.ByteSizeValue; import org.elasticsearch.common.unit.TimeValue; +import org.elasticsearch.common.xcontent.XContentBuilder; import org.elasticsearch.common.xcontent.XContentFactory; import org.elasticsearch.common.xcontent.XContentType; +import org.elasticsearch.index.query.MatchAllQueryBuilder; import org.elasticsearch.rest.RestStatus; import org.elasticsearch.search.SearchHit; import org.junit.After; @@ -136,6 +164,7 @@ import static org.hamcrest.CoreMatchers.hasItem; import static org.hamcrest.CoreMatchers.hasItems; import static org.hamcrest.CoreMatchers.not; +import static org.hamcrest.Matchers.closeTo; import static org.hamcrest.Matchers.contains; import static org.hamcrest.Matchers.containsInAnyOrder; import static org.hamcrest.Matchers.greaterThanOrEqualTo; @@ -528,18 +557,7 @@ public void testStartDatafeed() throws Exception { String indexName = "start_data_1"; // Set up the index and docs - CreateIndexRequest createIndexRequest = new CreateIndexRequest(indexName); - createIndexRequest.mapping(XContentFactory.jsonBuilder().startObject() - .startObject("properties") - .startObject("timestamp") - .field("type", "date") - .endObject() - .startObject("total") - .field("type", "long") - .endObject() - .endObject() - .endObject()); - highLevelClient().indices().create(createIndexRequest, RequestOptions.DEFAULT); + createIndex(indexName, defaultMappingForTest()); BulkRequest bulk = new BulkRequest(); bulk.setRefreshPolicy(WriteRequest.RefreshPolicy.IMMEDIATE); long now = (System.currentTimeMillis()/1000)*1000; @@ -611,18 +629,7 @@ public void testStopDatafeed() throws Exception { String indexName = "stop_data_1"; // Set up the index - CreateIndexRequest createIndexRequest = new CreateIndexRequest(indexName); - createIndexRequest.mapping(XContentFactory.jsonBuilder().startObject() - .startObject("properties") - .startObject("timestamp") - .field("type", "date") - .endObject() - .startObject("total") - .field("type", "long") - .endObject() - .endObject() - .endObject()); - highLevelClient().indices().create(createIndexRequest, RequestOptions.DEFAULT); + createIndex(indexName, defaultMappingForTest()); // create the job and the datafeed Job job1 = buildJob(jobId1); @@ -684,18 +691,7 @@ public void testGetDatafeedStats() throws Exception { String indexName = "datafeed_stats_data_1"; // Set up the index - CreateIndexRequest createIndexRequest = new CreateIndexRequest(indexName); - createIndexRequest.mapping(XContentFactory.jsonBuilder().startObject() - .startObject("properties") - .startObject("timestamp") - .field("type", "date") - .endObject() - .startObject("total") - .field("type", "long") - .endObject() - .endObject() - .endObject()); - highLevelClient().indices().create(createIndexRequest, RequestOptions.DEFAULT); + createIndex(indexName, defaultMappingForTest()); // create the job and the datafeed Job job1 = buildJob(jobId1); @@ -762,18 +758,7 @@ public void testPreviewDatafeed() throws Exception { String indexName = "preview_data_1"; // Set up the index and docs - CreateIndexRequest createIndexRequest = new CreateIndexRequest(indexName); - createIndexRequest.mapping(XContentFactory.jsonBuilder().startObject() - .startObject("properties") - .startObject("timestamp") - .field("type", "date") - .endObject() - .startObject("total") - .field("type", "long") - .endObject() - .endObject() - .endObject()); - highLevelClient().indices().create(createIndexRequest, RequestOptions.DEFAULT); + createIndex(indexName, defaultMappingForTest()); BulkRequest bulk = new BulkRequest(); bulk.setRefreshPolicy(WriteRequest.RefreshPolicy.IMMEDIATE); long now = (System.currentTimeMillis()/1000)*1000; @@ -826,21 +811,9 @@ public void testDeleteExpiredDataGivenNothingToDelete() throws Exception { } private String createExpiredData(String jobId) throws Exception { - String indexId = jobId + "-data"; + String indexName = jobId + "-data"; // Set up the index and docs - CreateIndexRequest createIndexRequest = new CreateIndexRequest(indexId); - createIndexRequest.mapping(XContentFactory.jsonBuilder().startObject() - .startObject("properties") - .startObject("timestamp") - .field("type", "date") - .field("format", "epoch_millis") - .endObject() - .startObject("total") - .field("type", "long") - .endObject() - .endObject() - .endObject()); - highLevelClient().indices().create(createIndexRequest, RequestOptions.DEFAULT); + createIndex(indexName, defaultMappingForTest()); BulkRequest bulk = new BulkRequest(); bulk.setRefreshPolicy(WriteRequest.RefreshPolicy.IMMEDIATE); @@ -853,7 +826,7 @@ private String createExpiredData(String jobId) throws Exception { long timestamp = nowMillis - TimeValue.timeValueHours(totalBuckets - bucket).getMillis(); int bucketRate = bucket == anomalousBucket ? anomalousRate : normalRate; for (int point = 0; point < bucketRate; point++) { - IndexRequest indexRequest = new IndexRequest(indexId); + IndexRequest indexRequest = new IndexRequest(indexName); indexRequest.source(XContentType.JSON, "timestamp", timestamp, "total", randomInt(1000)); bulk.add(indexRequest); } @@ -872,7 +845,7 @@ private String createExpiredData(String jobId) throws Exception { Job job = buildJobForExpiredDataTests(jobId); putJob(job); openJob(job); - String datafeedId = createAndPutDatafeed(jobId, indexId); + String datafeedId = createAndPutDatafeed(jobId, indexName); startDatafeed(datafeedId, String.valueOf(0), String.valueOf(nowMillis - TimeValue.timeValueHours(24).getMillis())); @@ -1230,6 +1203,420 @@ public void testDeleteCalendarEvent() throws IOException { assertThat(remainingIds, not(hasItem(deletedEvent))); } + public void testPutDataFrameAnalyticsConfig() throws Exception { + MachineLearningClient machineLearningClient = highLevelClient().machineLearning(); + String configId = "put-test-config"; + DataFrameAnalyticsConfig config = DataFrameAnalyticsConfig.builder(configId) + .setSource(DataFrameAnalyticsSource.builder() + .setIndex("put-test-source-index") + .build()) + .setDest(DataFrameAnalyticsDest.builder() + .setIndex("put-test-dest-index") + .build()) + .setAnalysis(OutlierDetection.createDefault()) + .build(); + + createIndex("put-test-source-index", defaultMappingForTest()); + + PutDataFrameAnalyticsResponse putDataFrameAnalyticsResponse = execute( + new PutDataFrameAnalyticsRequest(config), + machineLearningClient::putDataFrameAnalytics, machineLearningClient::putDataFrameAnalyticsAsync); + DataFrameAnalyticsConfig createdConfig = putDataFrameAnalyticsResponse.getConfig(); + assertThat(createdConfig.getId(), equalTo(config.getId())); + assertThat(createdConfig.getSource().getIndex(), equalTo(config.getSource().getIndex())); + assertThat(createdConfig.getSource().getQueryConfig(), equalTo(new QueryConfig(new MatchAllQueryBuilder()))); // default value + assertThat(createdConfig.getDest().getIndex(), equalTo(config.getDest().getIndex())); + assertThat(createdConfig.getDest().getResultsField(), equalTo("ml")); // default value + assertThat(createdConfig.getAnalysis(), equalTo(config.getAnalysis())); + assertThat(createdConfig.getAnalyzedFields(), equalTo(config.getAnalyzedFields())); + assertThat(createdConfig.getModelMemoryLimit(), equalTo(ByteSizeValue.parseBytesSizeValue("1gb", ""))); // default value + } + + public void testGetDataFrameAnalyticsConfig_SingleConfig() throws Exception { + MachineLearningClient machineLearningClient = highLevelClient().machineLearning(); + String configId = "get-test-config"; + DataFrameAnalyticsConfig config = DataFrameAnalyticsConfig.builder(configId) + .setSource(DataFrameAnalyticsSource.builder() + .setIndex("get-test-source-index") + .build()) + .setDest(DataFrameAnalyticsDest.builder() + .setIndex("get-test-dest-index") + .build()) + .setAnalysis(OutlierDetection.createDefault()) + .build(); + + createIndex("get-test-source-index", defaultMappingForTest()); + + PutDataFrameAnalyticsResponse putDataFrameAnalyticsResponse = execute( + new PutDataFrameAnalyticsRequest(config), + machineLearningClient::putDataFrameAnalytics, machineLearningClient::putDataFrameAnalyticsAsync); + DataFrameAnalyticsConfig createdConfig = putDataFrameAnalyticsResponse.getConfig(); + + GetDataFrameAnalyticsResponse getDataFrameAnalyticsResponse = execute( + new GetDataFrameAnalyticsRequest(configId), + machineLearningClient::getDataFrameAnalytics, machineLearningClient::getDataFrameAnalyticsAsync); + assertThat(getDataFrameAnalyticsResponse.getAnalytics(), hasSize(1)); + assertThat(getDataFrameAnalyticsResponse.getAnalytics(), contains(createdConfig)); + } + + public void testGetDataFrameAnalyticsConfig_MultipleConfigs() throws Exception { + createIndex("get-test-source-index", defaultMappingForTest()); + + MachineLearningClient machineLearningClient = highLevelClient().machineLearning(); + String configIdPrefix = "get-test-config-"; + int numberOfConfigs = 10; + List createdConfigs = new ArrayList<>(); + for (int i = 0; i < numberOfConfigs; ++i) { + String configId = configIdPrefix + i; + DataFrameAnalyticsConfig config = DataFrameAnalyticsConfig.builder(configId) + .setSource(DataFrameAnalyticsSource.builder() + .setIndex("get-test-source-index") + .build()) + .setDest(DataFrameAnalyticsDest.builder() + .setIndex("get-test-dest-index") + .build()) + .setAnalysis(OutlierDetection.createDefault()) + .build(); + + PutDataFrameAnalyticsResponse putDataFrameAnalyticsResponse = execute( + new PutDataFrameAnalyticsRequest(config), + machineLearningClient::putDataFrameAnalytics, machineLearningClient::putDataFrameAnalyticsAsync); + DataFrameAnalyticsConfig createdConfig = putDataFrameAnalyticsResponse.getConfig(); + createdConfigs.add(createdConfig); + } + + { + GetDataFrameAnalyticsResponse getDataFrameAnalyticsResponse = execute( + GetDataFrameAnalyticsRequest.getAllDataFrameAnalyticsRequest(), + machineLearningClient::getDataFrameAnalytics, machineLearningClient::getDataFrameAnalyticsAsync); + assertThat(getDataFrameAnalyticsResponse.getAnalytics(), hasSize(numberOfConfigs)); + assertThat(getDataFrameAnalyticsResponse.getAnalytics(), containsInAnyOrder(createdConfigs.toArray())); + } + { + GetDataFrameAnalyticsResponse getDataFrameAnalyticsResponse = execute( + new GetDataFrameAnalyticsRequest(configIdPrefix + "*"), + machineLearningClient::getDataFrameAnalytics, machineLearningClient::getDataFrameAnalyticsAsync); + assertThat(getDataFrameAnalyticsResponse.getAnalytics(), hasSize(numberOfConfigs)); + assertThat(getDataFrameAnalyticsResponse.getAnalytics(), containsInAnyOrder(createdConfigs.toArray())); + } + { + GetDataFrameAnalyticsResponse getDataFrameAnalyticsResponse = execute( + new GetDataFrameAnalyticsRequest(configIdPrefix + "9", configIdPrefix + "1", configIdPrefix + "4"), + machineLearningClient::getDataFrameAnalytics, machineLearningClient::getDataFrameAnalyticsAsync); + assertThat(getDataFrameAnalyticsResponse.getAnalytics(), hasSize(3)); + assertThat( + getDataFrameAnalyticsResponse.getAnalytics(), + containsInAnyOrder(createdConfigs.get(1), createdConfigs.get(4), createdConfigs.get(9))); + } + { + GetDataFrameAnalyticsRequest getDataFrameAnalyticsRequest = new GetDataFrameAnalyticsRequest(configIdPrefix + "*"); + getDataFrameAnalyticsRequest.setPageParams(new PageParams(3, 4)); + GetDataFrameAnalyticsResponse getDataFrameAnalyticsResponse = execute( + getDataFrameAnalyticsRequest, + machineLearningClient::getDataFrameAnalytics, machineLearningClient::getDataFrameAnalyticsAsync); + assertThat(getDataFrameAnalyticsResponse.getAnalytics(), hasSize(4)); + assertThat( + getDataFrameAnalyticsResponse.getAnalytics(), + containsInAnyOrder(createdConfigs.get(3), createdConfigs.get(4), createdConfigs.get(5), createdConfigs.get(6))); + } + } + + public void testGetDataFrameAnalyticsConfig_ConfigNotFound() { + MachineLearningClient machineLearningClient = highLevelClient().machineLearning(); + GetDataFrameAnalyticsRequest request = new GetDataFrameAnalyticsRequest("config_that_does_not_exist"); + ElasticsearchStatusException exception = expectThrows(ElasticsearchStatusException.class, + () -> execute(request, machineLearningClient::getDataFrameAnalytics, machineLearningClient::getDataFrameAnalyticsAsync)); + assertThat(exception.status().getStatus(), equalTo(404)); + } + + public void testGetDataFrameAnalyticsStats() throws Exception { + String sourceIndex = "get-stats-test-source-index"; + String destIndex = "get-stats-test-dest-index"; + createIndex(sourceIndex, defaultMappingForTest()); + highLevelClient().index(new IndexRequest(sourceIndex).source(XContentType.JSON, "total", 10000), RequestOptions.DEFAULT); + + MachineLearningClient machineLearningClient = highLevelClient().machineLearning(); + String configId = "get-stats-test-config"; + DataFrameAnalyticsConfig config = DataFrameAnalyticsConfig.builder(configId) + .setSource(DataFrameAnalyticsSource.builder() + .setIndex(sourceIndex) + .build()) + .setDest(DataFrameAnalyticsDest.builder() + .setIndex(destIndex) + .build()) + .setAnalysis(OutlierDetection.createDefault()) + .build(); + + execute( + new PutDataFrameAnalyticsRequest(config), + machineLearningClient::putDataFrameAnalytics, machineLearningClient::putDataFrameAnalyticsAsync); + + GetDataFrameAnalyticsStatsResponse statsResponse = execute( + new GetDataFrameAnalyticsStatsRequest(configId), + machineLearningClient::getDataFrameAnalyticsStats, machineLearningClient::getDataFrameAnalyticsStatsAsync); + + assertThat(statsResponse.getAnalyticsStats(), hasSize(1)); + DataFrameAnalyticsStats stats = statsResponse.getAnalyticsStats().get(0); + assertThat(stats.getId(), equalTo(configId)); + assertThat(stats.getState(), equalTo(DataFrameAnalyticsState.STOPPED)); + assertNull(stats.getFailureReason()); + assertNull(stats.getProgressPercent()); + assertNull(stats.getNode()); + assertNull(stats.getAssignmentExplanation()); + assertThat(statsResponse.getNodeFailures(), hasSize(0)); + assertThat(statsResponse.getTaskFailures(), hasSize(0)); + } + + public void testStartDataFrameAnalyticsConfig() throws Exception { + String sourceIndex = "start-test-source-index"; + String destIndex = "start-test-dest-index"; + createIndex(sourceIndex, defaultMappingForTest()); + highLevelClient().index(new IndexRequest(sourceIndex).source(XContentType.JSON, "total", 10000) + .setRefreshPolicy(WriteRequest.RefreshPolicy.IMMEDIATE), RequestOptions.DEFAULT); + + // Verify that the destination index does not exist. Otherwise, analytics' reindexing step would fail. + assertFalse(highLevelClient().indices().exists(new GetIndexRequest(destIndex), RequestOptions.DEFAULT)); + + MachineLearningClient machineLearningClient = highLevelClient().machineLearning(); + String configId = "start-test-config"; + DataFrameAnalyticsConfig config = DataFrameAnalyticsConfig.builder(configId) + .setSource(DataFrameAnalyticsSource.builder() + .setIndex(sourceIndex) + .build()) + .setDest(DataFrameAnalyticsDest.builder() + .setIndex(destIndex) + .build()) + .setAnalysis(OutlierDetection.createDefault()) + .build(); + + execute( + new PutDataFrameAnalyticsRequest(config), + machineLearningClient::putDataFrameAnalytics, machineLearningClient::putDataFrameAnalyticsAsync); + assertThat(getAnalyticsState(configId), equalTo(DataFrameAnalyticsState.STOPPED)); + + AcknowledgedResponse startDataFrameAnalyticsResponse = execute( + new StartDataFrameAnalyticsRequest(configId), + machineLearningClient::startDataFrameAnalytics, machineLearningClient::startDataFrameAnalyticsAsync); + assertTrue(startDataFrameAnalyticsResponse.isAcknowledged()); + + // Wait for the analytics to stop. + assertBusy(() -> assertThat(getAnalyticsState(configId), equalTo(DataFrameAnalyticsState.STOPPED)), 30, TimeUnit.SECONDS); + + // Verify that the destination index got created. + assertTrue(highLevelClient().indices().exists(new GetIndexRequest(destIndex), RequestOptions.DEFAULT)); + } + + @AwaitsFix(bugUrl = "https://github.com/elastic/elasticsearch/issues/43924") + public void testStopDataFrameAnalyticsConfig() throws Exception { + String sourceIndex = "stop-test-source-index"; + String destIndex = "stop-test-dest-index"; + createIndex(sourceIndex, mappingForClassification()); + highLevelClient().index(new IndexRequest(sourceIndex).source(XContentType.JSON, "total", 10000) + .setRefreshPolicy(WriteRequest.RefreshPolicy.IMMEDIATE), RequestOptions.DEFAULT); + + // Verify that the destination index does not exist. Otherwise, analytics' reindexing step would fail. + assertFalse(highLevelClient().indices().exists(new GetIndexRequest(destIndex), RequestOptions.DEFAULT)); + + MachineLearningClient machineLearningClient = highLevelClient().machineLearning(); + String configId = "stop-test-config"; + DataFrameAnalyticsConfig config = DataFrameAnalyticsConfig.builder(configId) + .setSource(DataFrameAnalyticsSource.builder() + .setIndex(sourceIndex) + .build()) + .setDest(DataFrameAnalyticsDest.builder() + .setIndex(destIndex) + .build()) + .setAnalysis(OutlierDetection.createDefault()) + .build(); + + execute( + new PutDataFrameAnalyticsRequest(config), + machineLearningClient::putDataFrameAnalytics, machineLearningClient::putDataFrameAnalyticsAsync); + assertThat(getAnalyticsState(configId), equalTo(DataFrameAnalyticsState.STOPPED)); + + AcknowledgedResponse startDataFrameAnalyticsResponse = execute( + new StartDataFrameAnalyticsRequest(configId), + machineLearningClient::startDataFrameAnalytics, machineLearningClient::startDataFrameAnalyticsAsync); + assertTrue(startDataFrameAnalyticsResponse.isAcknowledged()); + assertThat(getAnalyticsState(configId), equalTo(DataFrameAnalyticsState.STARTED)); + + StopDataFrameAnalyticsResponse stopDataFrameAnalyticsResponse = execute( + new StopDataFrameAnalyticsRequest(configId), + machineLearningClient::stopDataFrameAnalytics, machineLearningClient::stopDataFrameAnalyticsAsync); + assertTrue(stopDataFrameAnalyticsResponse.isStopped()); + assertThat(getAnalyticsState(configId), equalTo(DataFrameAnalyticsState.STOPPED)); + } + + private DataFrameAnalyticsState getAnalyticsState(String configId) throws IOException { + MachineLearningClient machineLearningClient = highLevelClient().machineLearning(); + GetDataFrameAnalyticsStatsResponse statsResponse = + machineLearningClient.getDataFrameAnalyticsStats(new GetDataFrameAnalyticsStatsRequest(configId), RequestOptions.DEFAULT); + assertThat(statsResponse.getAnalyticsStats(), hasSize(1)); + DataFrameAnalyticsStats stats = statsResponse.getAnalyticsStats().get(0); + return stats.getState(); + } + + public void testDeleteDataFrameAnalyticsConfig() throws Exception { + MachineLearningClient machineLearningClient = highLevelClient().machineLearning(); + String configId = "delete-test-config"; + DataFrameAnalyticsConfig config = DataFrameAnalyticsConfig.builder(configId) + .setSource(DataFrameAnalyticsSource.builder() + .setIndex("delete-test-source-index") + .build()) + .setDest(DataFrameAnalyticsDest.builder() + .setIndex("delete-test-dest-index") + .build()) + .setAnalysis(OutlierDetection.createDefault()) + .build(); + + createIndex("delete-test-source-index", defaultMappingForTest()); + + GetDataFrameAnalyticsResponse getDataFrameAnalyticsResponse = execute( + new GetDataFrameAnalyticsRequest(configId + "*"), + machineLearningClient::getDataFrameAnalytics, machineLearningClient::getDataFrameAnalyticsAsync); + assertThat(getDataFrameAnalyticsResponse.getAnalytics(), hasSize(0)); + + execute( + new PutDataFrameAnalyticsRequest(config), + machineLearningClient::putDataFrameAnalytics, machineLearningClient::putDataFrameAnalyticsAsync); + + getDataFrameAnalyticsResponse = execute( + new GetDataFrameAnalyticsRequest(configId + "*"), + machineLearningClient::getDataFrameAnalytics, machineLearningClient::getDataFrameAnalyticsAsync); + assertThat(getDataFrameAnalyticsResponse.getAnalytics(), hasSize(1)); + + AcknowledgedResponse deleteDataFrameAnalyticsResponse = execute( + new DeleteDataFrameAnalyticsRequest(configId), + machineLearningClient::deleteDataFrameAnalytics, machineLearningClient::deleteDataFrameAnalyticsAsync); + assertTrue(deleteDataFrameAnalyticsResponse.isAcknowledged()); + + getDataFrameAnalyticsResponse = execute( + new GetDataFrameAnalyticsRequest(configId + "*"), + machineLearningClient::getDataFrameAnalytics, machineLearningClient::getDataFrameAnalyticsAsync); + assertThat(getDataFrameAnalyticsResponse.getAnalytics(), hasSize(0)); + } + + public void testDeleteDataFrameAnalyticsConfig_ConfigNotFound() { + MachineLearningClient machineLearningClient = highLevelClient().machineLearning(); + DeleteDataFrameAnalyticsRequest request = new DeleteDataFrameAnalyticsRequest("config_that_does_not_exist"); + ElasticsearchStatusException exception = expectThrows(ElasticsearchStatusException.class, + () -> execute( + request, machineLearningClient::deleteDataFrameAnalytics, machineLearningClient::deleteDataFrameAnalyticsAsync)); + assertThat(exception.status().getStatus(), equalTo(404)); + } + + public void testEvaluateDataFrame() throws IOException { + String indexName = "evaluate-test-index"; + createIndex(indexName, mappingForClassification()); + BulkRequest bulk = new BulkRequest() + .setRefreshPolicy(WriteRequest.RefreshPolicy.IMMEDIATE) + .add(docForClassification(indexName, false, 0.1)) // #0 + .add(docForClassification(indexName, false, 0.2)) // #1 + .add(docForClassification(indexName, false, 0.3)) // #2 + .add(docForClassification(indexName, false, 0.4)) // #3 + .add(docForClassification(indexName, false, 0.7)) // #4 + .add(docForClassification(indexName, true, 0.2)) // #5 + .add(docForClassification(indexName, true, 0.3)) // #6 + .add(docForClassification(indexName, true, 0.4)) // #7 + .add(docForClassification(indexName, true, 0.8)) // #8 + .add(docForClassification(indexName, true, 0.9)); // #9 + highLevelClient().bulk(bulk, RequestOptions.DEFAULT); + + MachineLearningClient machineLearningClient = highLevelClient().machineLearning(); + EvaluateDataFrameRequest evaluateDataFrameRequest = + new EvaluateDataFrameRequest( + indexName, + new BinarySoftClassification( + actualField, + probabilityField, + PrecisionMetric.at(0.4, 0.5, 0.6), RecallMetric.at(0.5, 0.7), ConfusionMatrixMetric.at(0.5), AucRocMetric.withCurve())); + + EvaluateDataFrameResponse evaluateDataFrameResponse = + execute(evaluateDataFrameRequest, machineLearningClient::evaluateDataFrame, machineLearningClient::evaluateDataFrameAsync); + assertThat(evaluateDataFrameResponse.getEvaluationName(), equalTo(BinarySoftClassification.NAME)); + assertThat(evaluateDataFrameResponse.getMetrics().size(), equalTo(4)); + + PrecisionMetric.Result precisionResult = evaluateDataFrameResponse.getMetricByName(PrecisionMetric.NAME); + assertThat(precisionResult.getMetricName(), equalTo(PrecisionMetric.NAME)); + // Precision is 3/5=0.6 as there were 3 true examples (#7, #8, #9) among the 5 positive examples (#3, #4, #7, #8, #9) + assertThat(precisionResult.getScoreByThreshold("0.4"), closeTo(0.6, 1e-9)); + // Precision is 2/3=0.(6) as there were 2 true examples (#8, #9) among the 3 positive examples (#4, #8, #9) + assertThat(precisionResult.getScoreByThreshold("0.5"), closeTo(0.666666666, 1e-9)); + // Precision is 2/3=0.(6) as there were 2 true examples (#8, #9) among the 3 positive examples (#4, #8, #9) + assertThat(precisionResult.getScoreByThreshold("0.6"), closeTo(0.666666666, 1e-9)); + assertNull(precisionResult.getScoreByThreshold("0.1")); + + RecallMetric.Result recallResult = evaluateDataFrameResponse.getMetricByName(RecallMetric.NAME); + assertThat(recallResult.getMetricName(), equalTo(RecallMetric.NAME)); + // Recall is 2/5=0.4 as there were 2 true positive examples (#8, #9) among the 5 true examples (#5, #6, #7, #8, #9) + assertThat(recallResult.getScoreByThreshold("0.5"), closeTo(0.4, 1e-9)); + // Recall is 2/5=0.4 as there were 2 true positive examples (#8, #9) among the 5 true examples (#5, #6, #7, #8, #9) + assertThat(recallResult.getScoreByThreshold("0.7"), closeTo(0.4, 1e-9)); + assertNull(recallResult.getScoreByThreshold("0.1")); + + ConfusionMatrixMetric.Result confusionMatrixResult = evaluateDataFrameResponse.getMetricByName(ConfusionMatrixMetric.NAME); + assertThat(confusionMatrixResult.getMetricName(), equalTo(ConfusionMatrixMetric.NAME)); + ConfusionMatrixMetric.ConfusionMatrix confusionMatrix = confusionMatrixResult.getScoreByThreshold("0.5"); + assertThat(confusionMatrix.getTruePositives(), equalTo(2L)); // docs #8 and #9 + assertThat(confusionMatrix.getFalsePositives(), equalTo(1L)); // doc #4 + assertThat(confusionMatrix.getTrueNegatives(), equalTo(4L)); // docs #0, #1, #2 and #3 + assertThat(confusionMatrix.getFalseNegatives(), equalTo(3L)); // docs #5, #6 and #7 + assertNull(confusionMatrixResult.getScoreByThreshold("0.1")); + + AucRocMetric.Result aucRocResult = evaluateDataFrameResponse.getMetricByName(AucRocMetric.NAME); + assertThat(aucRocResult.getMetricName(), equalTo(AucRocMetric.NAME)); + assertThat(aucRocResult.getScore(), closeTo(0.70025, 1e-9)); + assertNotNull(aucRocResult.getCurve()); + List curve = aucRocResult.getCurve(); + AucRocMetric.AucRocPoint curvePointAtThreshold0 = curve.stream().filter(p -> p.getThreshold() == 0.0).findFirst().get(); + assertThat(curvePointAtThreshold0.getTruePositiveRate(), equalTo(1.0)); + assertThat(curvePointAtThreshold0.getFalsePositiveRate(), equalTo(1.0)); + assertThat(curvePointAtThreshold0.getThreshold(), equalTo(0.0)); + AucRocMetric.AucRocPoint curvePointAtThreshold1 = curve.stream().filter(p -> p.getThreshold() == 1.0).findFirst().get(); + assertThat(curvePointAtThreshold1.getTruePositiveRate(), equalTo(0.0)); + assertThat(curvePointAtThreshold1.getFalsePositiveRate(), equalTo(0.0)); + assertThat(curvePointAtThreshold1.getThreshold(), equalTo(1.0)); + } + + private static XContentBuilder defaultMappingForTest() throws IOException { + return XContentFactory.jsonBuilder().startObject() + .startObject("properties") + .startObject("timestamp") + .field("type", "date") + .endObject() + .startObject("total") + .field("type", "long") + .endObject() + .endObject() + .endObject(); + } + + private static final String actualField = "label"; + private static final String probabilityField = "p"; + + private static XContentBuilder mappingForClassification() throws IOException { + return XContentFactory.jsonBuilder().startObject() + .startObject("properties") + .startObject(actualField) + .field("type", "keyword") + .endObject() + .startObject(probabilityField) + .field("type", "double") + .endObject() + .endObject() + .endObject(); + } + + private static IndexRequest docForClassification(String indexName, boolean isTrue, double p) { + return new IndexRequest() + .index(indexName) + .source(XContentType.JSON, actualField, Boolean.toString(isTrue), probabilityField, p); + } + + private void createIndex(String indexName, XContentBuilder mapping) throws IOException { + highLevelClient().indices().create(new CreateIndexRequest(indexName).mapping(mapping), RequestOptions.DEFAULT); + } + public void testPutFilter() throws Exception { String filterId = "filter-job-test"; MlFilter mlFilter = MlFilter.builder(filterId) @@ -1286,7 +1673,7 @@ public void testGetFilters() throws Exception { GetFiltersResponse getFiltersResponse = execute(getFiltersRequest, machineLearningClient::getFilter, machineLearningClient::getFilterAsync); - assertThat(getFiltersResponse.count(), equalTo(2L)); + assertThat(getFiltersResponse.count(), equalTo(3L)); assertThat(getFiltersResponse.filters().size(), equalTo(2)); assertThat(getFiltersResponse.filters().stream().map(MlFilter::getId).collect(Collectors.toList()), containsInAnyOrder("get-filter-test-2", "get-filter-test-3")); diff --git a/client/rest-high-level/src/test/java/org/elasticsearch/client/MlTestStateCleaner.java b/client/rest-high-level/src/test/java/org/elasticsearch/client/MlTestStateCleaner.java index c565af7c37202..f5776e99fd0eb 100644 --- a/client/rest-high-level/src/test/java/org/elasticsearch/client/MlTestStateCleaner.java +++ b/client/rest-high-level/src/test/java/org/elasticsearch/client/MlTestStateCleaner.java @@ -20,14 +20,18 @@ import org.apache.logging.log4j.Logger; import org.elasticsearch.client.ml.CloseJobRequest; +import org.elasticsearch.client.ml.DeleteDataFrameAnalyticsRequest; import org.elasticsearch.client.ml.DeleteDatafeedRequest; import org.elasticsearch.client.ml.DeleteJobRequest; +import org.elasticsearch.client.ml.GetDataFrameAnalyticsRequest; +import org.elasticsearch.client.ml.GetDataFrameAnalyticsResponse; import org.elasticsearch.client.ml.GetDatafeedRequest; import org.elasticsearch.client.ml.GetDatafeedResponse; import org.elasticsearch.client.ml.GetJobRequest; import org.elasticsearch.client.ml.GetJobResponse; import org.elasticsearch.client.ml.StopDatafeedRequest; import org.elasticsearch.client.ml.datafeed.DatafeedConfig; +import org.elasticsearch.client.ml.dataframe.DataFrameAnalyticsConfig; import org.elasticsearch.client.ml.job.config.Job; import java.io.IOException; @@ -48,6 +52,7 @@ public MlTestStateCleaner(Logger logger, MachineLearningClient mlClient) { public void clearMlMetadata() throws IOException { deleteAllDatafeeds(); deleteAllJobs(); + deleteAllDataFrameAnalytics(); } private void deleteAllDatafeeds() throws IOException { @@ -99,4 +104,12 @@ private void closeAllJobs() { throw new RuntimeException("Had to resort to force-closing jobs, something went wrong?", e1); } } + + private void deleteAllDataFrameAnalytics() throws IOException { + GetDataFrameAnalyticsResponse getDataFrameAnalyticsResponse = + mlClient.getDataFrameAnalytics(GetDataFrameAnalyticsRequest.getAllDataFrameAnalyticsRequest(), RequestOptions.DEFAULT); + for (DataFrameAnalyticsConfig config : getDataFrameAnalyticsResponse.getAnalytics()) { + mlClient.deleteDataFrameAnalytics(new DeleteDataFrameAnalyticsRequest(config.getId()), RequestOptions.DEFAULT); + } + } } diff --git a/client/rest-high-level/src/test/java/org/elasticsearch/client/PingAndInfoIT.java b/client/rest-high-level/src/test/java/org/elasticsearch/client/PingAndInfoIT.java index 1b4ca84c47144..eee2491c7d42f 100644 --- a/client/rest-high-level/src/test/java/org/elasticsearch/client/PingAndInfoIT.java +++ b/client/rest-high-level/src/test/java/org/elasticsearch/client/PingAndInfoIT.java @@ -71,31 +71,20 @@ public void testXPackInfo() throws IOException { assertEquals(LicenseStatus.ACTIVE, info.getLicenseInfo().getStatus()); FeatureSet graph = info.getFeatureSetsInfo().getFeatureSets().get("graph"); - assertNotNull(graph.description()); assertTrue(graph.available()); assertTrue(graph.enabled()); - assertNull(graph.nativeCodeInfo()); FeatureSet monitoring = info.getFeatureSetsInfo().getFeatureSets().get("monitoring"); - assertNotNull(monitoring.description()); assertTrue(monitoring.available()); assertTrue(monitoring.enabled()); - assertNull(monitoring.nativeCodeInfo()); FeatureSet ml = info.getFeatureSetsInfo().getFeatureSets().get("ml"); - assertNotNull(ml.description()); assertTrue(ml.available()); assertTrue(ml.enabled()); - assertEquals(mainResponse.getVersion().getNumber(), ml.nativeCodeInfo().get("version").toString()); } public void testXPackInfoEmptyRequest() throws IOException { XPackInfoResponse info = highLevelClient().xpack().info(new XPackInfoRequest(), RequestOptions.DEFAULT); - /* - * The default in the transport client is non-verbose and returning - * no categories which is the opposite of the default when you use - * the API over REST. We don't want to break the transport client - * even though it doesn't feel like a good default. - */ + // TODO: reconsider this leniency now that the transport client is gone assertNull(info.getBuildInfo()); assertNull(info.getLicenseInfo()); assertNull(info.getFeatureSetsInfo()); 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 9c5137d54427a..a1946baa3e0e1 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 @@ -31,7 +31,6 @@ import org.elasticsearch.action.admin.cluster.storedscripts.DeleteStoredScriptRequest; import org.elasticsearch.action.admin.cluster.storedscripts.GetStoredScriptRequest; import org.elasticsearch.action.admin.cluster.storedscripts.PutStoredScriptRequest; -import org.elasticsearch.action.admin.indices.analyze.AnalyzeRequest; import org.elasticsearch.action.bulk.BulkRequest; import org.elasticsearch.action.bulk.BulkShardRequest; import org.elasticsearch.action.delete.DeleteRequest; @@ -56,6 +55,7 @@ import org.elasticsearch.client.core.CountRequest; import org.elasticsearch.client.core.MultiTermVectorsRequest; import org.elasticsearch.client.core.TermVectorsRequest; +import org.elasticsearch.client.indices.AnalyzeRequest; import org.elasticsearch.common.CheckedBiConsumer; import org.elasticsearch.common.Strings; import org.elasticsearch.common.bytes.BytesArray; @@ -404,9 +404,6 @@ public void testReindex() throws IOException { ); reindexRequest.setRemoteInfo(remoteInfo); } - if (randomBoolean()) { - reindexRequest.setSourceDocTypes("doc", "tweet"); - } if (randomBoolean()) { reindexRequest.setSourceBatchSize(randomInt(100)); } @@ -430,7 +427,7 @@ public void testReindex() throws IOException { reindexRequest.setDestRouting("=cat"); } if (randomBoolean()) { - reindexRequest.setSize(randomIntBetween(100, 1000)); + reindexRequest.setMaxDocs(randomIntBetween(100, 1000)); } if (randomBoolean()) { reindexRequest.setAbortOnVersionConflict(false); @@ -457,9 +454,6 @@ public void testUpdateByQuery() throws IOException { UpdateByQueryRequest updateByQueryRequest = new UpdateByQueryRequest(); updateByQueryRequest.indices(randomIndicesNames(1, 5)); Map expectedParams = new HashMap<>(); - if (randomBoolean()) { - updateByQueryRequest.setDocTypes(generateRandomStringArray(5, 5, false, false)); - } if (randomBoolean()) { int batchSize = randomInt(100); updateByQueryRequest.setBatchSize(batchSize); @@ -481,9 +475,9 @@ public void testUpdateByQuery() throws IOException { expectedParams.put("routing", "=cat"); } if (randomBoolean()) { - int size = randomIntBetween(100, 1000); - updateByQueryRequest.setSize(size); - expectedParams.put("size", Integer.toString(size)); + int maxDocs = randomIntBetween(100, 1000); + updateByQueryRequest.setMaxDocs(maxDocs); + expectedParams.put("max_docs", Integer.toString(maxDocs)); } if (randomBoolean()) { updateByQueryRequest.setAbortOnVersionConflict(false); @@ -505,8 +499,6 @@ public void testUpdateByQuery() throws IOException { Request request = RequestConverters.updateByQuery(updateByQueryRequest); StringJoiner joiner = new StringJoiner("/", "/", ""); joiner.add(String.join(",", updateByQueryRequest.indices())); - if (updateByQueryRequest.getDocTypes().length > 0) - joiner.add(String.join(",", updateByQueryRequest.getDocTypes())); joiner.add("_update_by_query"); assertEquals(joiner.toString(), request.getEndpoint()); assertEquals(HttpPost.METHOD_NAME, request.getMethod()); @@ -518,9 +510,6 @@ public void testDeleteByQuery() throws IOException { DeleteByQueryRequest deleteByQueryRequest = new DeleteByQueryRequest(); deleteByQueryRequest.indices(randomIndicesNames(1, 5)); Map expectedParams = new HashMap<>(); - if (randomBoolean()) { - deleteByQueryRequest.setDocTypes(generateRandomStringArray(5, 5, false, false)); - } if (randomBoolean()) { int batchSize = randomInt(100); deleteByQueryRequest.setBatchSize(batchSize); @@ -531,9 +520,9 @@ public void testDeleteByQuery() throws IOException { expectedParams.put("routing", "=cat"); } if (randomBoolean()) { - int size = randomIntBetween(100, 1000); - deleteByQueryRequest.setSize(size); - expectedParams.put("size", Integer.toString(size)); + int maxDocs = randomIntBetween(100, 1000); + deleteByQueryRequest.setMaxDocs(maxDocs); + expectedParams.put("max_docs", Integer.toString(maxDocs)); } if (randomBoolean()) { deleteByQueryRequest.setAbortOnVersionConflict(false); @@ -559,8 +548,6 @@ public void testDeleteByQuery() throws IOException { Request request = RequestConverters.deleteByQuery(deleteByQueryRequest); StringJoiner joiner = new StringJoiner("/", "/", ""); joiner.add(String.join(",", deleteByQueryRequest.indices())); - if (deleteByQueryRequest.getDocTypes().length > 0) - joiner.add(String.join(",", deleteByQueryRequest.getDocTypes())); joiner.add("_delete_by_query"); assertEquals(joiner.toString(), request.getEndpoint()); assertEquals(HttpPost.METHOD_NAME, request.getMethod()); @@ -1065,13 +1052,6 @@ public void testSearch() throws Exception { String[] indices = randomIndicesNames(0, 5); SearchRequest searchRequest = new SearchRequest(indices); - int numTypes = randomIntBetween(0, 5); - String[] types = new String[numTypes]; - for (int i = 0; i < numTypes; i++) { - types[i] = "type-" + randomAlphaOfLengthBetween(2, 5); - } - searchRequest.types(types); - Map expectedParams = new HashMap<>(); setRandomSearchParams(searchRequest, expectedParams); setRandomIndicesOptions(searchRequest::indicesOptions, searchRequest::indicesOptions, expectedParams); @@ -1128,10 +1108,6 @@ public void testSearch() throws Exception { if (Strings.hasLength(index)) { endpoint.add(index); } - String type = String.join(",", types); - if (Strings.hasLength(type)) { - endpoint.add(type); - } endpoint.add(searchEndpoint); assertEquals(HttpPost.METHOD_NAME, request.getMethod()); assertEquals(endpoint.toString(), request.getEndpoint()); @@ -1142,7 +1118,6 @@ public void testSearch() throws Exception { public void testSearchNullIndicesAndTypes() { expectThrows(NullPointerException.class, () -> new SearchRequest((String[]) null)); expectThrows(NullPointerException.class, () -> new SearchRequest().indices((String[]) null)); - expectThrows(NullPointerException.class, () -> new SearchRequest().types((String[]) null)); } public void testCountNotNullSource() throws IOException { @@ -1257,7 +1232,7 @@ public void testMultiSearch() throws IOException { requests.add(searchRequest); }; MultiSearchRequest.readMultiLineFormat(new BytesArray(EntityUtils.toByteArray(request.getEntity())), - REQUEST_BODY_CONTENT_TYPE.xContent(), consumer, null, multiSearchRequest.indicesOptions(), null, null, null, null, + REQUEST_BODY_CONTENT_TYPE.xContent(), consumer, null, multiSearchRequest.indicesOptions(), null, null, null, xContentRegistry(), true); assertEquals(requests, multiSearchRequest.requests()); } @@ -1561,7 +1536,7 @@ public void testFieldCaps() { endpoint.add("_field_caps"); assertEquals(endpoint.toString(), request.getEndpoint()); - assertEquals(4, request.getParameters().size()); + assertEquals(5, request.getParameters().size()); // Note that we don't check the field param value explicitly, as field names are // passed through @@ -1595,7 +1570,7 @@ public void testRankEval() throws Exception { } endpoint.add(RestRankEvalAction.ENDPOINT); assertEquals(endpoint.toString(), request.getEndpoint()); - assertEquals(3, request.getParameters().size()); + assertEquals(4, request.getParameters().size()); assertEquals(expectedParams, request.getParameters()); assertToXContentBody(spec, request.getEntity()); } @@ -1637,18 +1612,14 @@ public void testPutScript() throws Exception { } public void testAnalyzeRequest() throws Exception { - AnalyzeRequest indexAnalyzeRequest = new AnalyzeRequest() - .text("Here is some text") - .index("test_index") - .analyzer("test_analyzer"); + AnalyzeRequest indexAnalyzeRequest + = AnalyzeRequest.withIndexAnalyzer("test_index", "test_analyzer", "Here is some text"); Request request = RequestConverters.analyze(indexAnalyzeRequest); assertThat(request.getEndpoint(), equalTo("/test_index/_analyze")); assertToXContentBody(indexAnalyzeRequest, request.getEntity()); - AnalyzeRequest analyzeRequest = new AnalyzeRequest() - .text("more text") - .analyzer("test_analyzer"); + AnalyzeRequest analyzeRequest = AnalyzeRequest.withGlobalAnalyzer("test_analyzer", "more text"); assertThat(RequestConverters.analyze(analyzeRequest).getEndpoint(), equalTo("/_analyze")); } @@ -1922,7 +1893,8 @@ static void setRandomIndicesOptions(Consumer setter, Supplier expectedParams) { if (randomBoolean()) { - setter.accept(IndicesOptions.fromOptions(randomBoolean(), randomBoolean(), randomBoolean(), randomBoolean())); + setter.accept(IndicesOptions.fromOptions(randomBoolean(), randomBoolean(), randomBoolean(), randomBoolean(), + true, false, false, randomBoolean())); } expectedParams.put("ignore_unavailable", Boolean.toString(getter.get().ignoreUnavailable())); expectedParams.put("allow_no_indices", Boolean.toString(getter.get().allowNoIndices())); @@ -1935,11 +1907,13 @@ static void setRandomIndicesOptions(Consumer setter, Supplier expectedParams) { if (randomBoolean()) { - indicesOptions = IndicesOptions.fromOptions(randomBoolean(), randomBoolean(), randomBoolean(), randomBoolean()); + indicesOptions = IndicesOptions.fromOptions(randomBoolean(), randomBoolean(), randomBoolean(), randomBoolean(), + true, false, false, randomBoolean()); } expectedParams.put("ignore_unavailable", Boolean.toString(indicesOptions.ignoreUnavailable())); expectedParams.put("allow_no_indices", Boolean.toString(indicesOptions.allowNoIndices())); @@ -1952,6 +1926,7 @@ static IndicesOptions setRandomIndicesOptions(IndicesOptions indicesOptions, Map } else { expectedParams.put("expand_wildcards", "none"); } + expectedParams.put("ignore_throttled", Boolean.toString(indicesOptions.ignoreThrottled())); return indicesOptions; } diff --git a/client/rest-high-level/src/test/java/org/elasticsearch/client/RestHighLevelClientTests.java b/client/rest-high-level/src/test/java/org/elasticsearch/client/RestHighLevelClientTests.java index ed5d7b66d80c1..ae1cd5eb45edf 100644 --- a/client/rest-high-level/src/test/java/org/elasticsearch/client/RestHighLevelClientTests.java +++ b/client/rest-high-level/src/test/java/org/elasticsearch/client/RestHighLevelClientTests.java @@ -46,6 +46,8 @@ import org.elasticsearch.action.search.ShardSearchFailure; import org.elasticsearch.client.core.MainRequest; import org.elasticsearch.client.core.MainResponse; +import org.elasticsearch.client.dataframe.transforms.SyncConfig; +import org.elasticsearch.client.dataframe.transforms.TimeSyncConfig; import org.elasticsearch.client.indexlifecycle.AllocateAction; import org.elasticsearch.client.indexlifecycle.DeleteAction; import org.elasticsearch.client.indexlifecycle.ForceMergeAction; @@ -56,6 +58,13 @@ import org.elasticsearch.client.indexlifecycle.SetPriorityAction; import org.elasticsearch.client.indexlifecycle.ShrinkAction; import org.elasticsearch.client.indexlifecycle.UnfollowAction; +import org.elasticsearch.client.ml.dataframe.DataFrameAnalysis; +import org.elasticsearch.client.ml.dataframe.OutlierDetection; +import org.elasticsearch.client.ml.dataframe.evaluation.softclassification.AucRocMetric; +import org.elasticsearch.client.ml.dataframe.evaluation.softclassification.BinarySoftClassification; +import org.elasticsearch.client.ml.dataframe.evaluation.softclassification.ConfusionMatrixMetric; +import org.elasticsearch.client.ml.dataframe.evaluation.softclassification.PrecisionMetric; +import org.elasticsearch.client.ml.dataframe.evaluation.softclassification.RecallMetric; import org.elasticsearch.common.CheckedFunction; import org.elasticsearch.common.bytes.BytesReference; import org.elasticsearch.common.collect.Tuple; @@ -109,6 +118,7 @@ import static org.hamcrest.CoreMatchers.endsWith; import static org.hamcrest.CoreMatchers.equalTo; import static org.hamcrest.CoreMatchers.instanceOf; +import static org.hamcrest.Matchers.hasItems; import static org.mockito.Matchers.any; import static org.mockito.Mockito.mock; import static org.mockito.Mockito.times; @@ -131,7 +141,7 @@ public class RestHighLevelClientTests extends ESTestCase { // core "ping", "info", // security - "security.get_ssl_certificates", "security.authenticate", "security.get_user_privileges", + "security.get_ssl_certificates", "security.authenticate", "security.get_user_privileges", "security.get_builtin_privileges", // license "license.get_trial_status", "license.get_basic_status" @@ -664,7 +674,7 @@ public void testDefaultNamedXContents() { public void testProvidedNamedXContents() { List namedXContents = RestHighLevelClient.getProvidedNamedXContents(); - assertEquals(20, namedXContents.size()); + assertEquals(31, namedXContents.size()); Map, Integer> categories = new HashMap<>(); List names = new ArrayList<>(); for (NamedXContentRegistry.Entry namedXContent : namedXContents) { @@ -674,7 +684,7 @@ public void testProvidedNamedXContents() { categories.put(namedXContent.categoryClass, counter + 1); } } - assertEquals("Had: " + categories, 4, categories.size()); + assertEquals("Had: " + categories, 9, categories.size()); assertEquals(Integer.valueOf(3), categories.get(Aggregation.class)); assertTrue(names.contains(ChildrenAggregationBuilder.NAME)); assertTrue(names.contains(MatrixStatsAggregationBuilder.NAME)); @@ -698,6 +708,16 @@ public void testProvidedNamedXContents() { assertTrue(names.contains(ShrinkAction.NAME)); assertTrue(names.contains(FreezeAction.NAME)); assertTrue(names.contains(SetPriorityAction.NAME)); + assertEquals(Integer.valueOf(1), categories.get(DataFrameAnalysis.class)); + assertTrue(names.contains(OutlierDetection.NAME.getPreferredName())); + assertEquals(Integer.valueOf(1), categories.get(SyncConfig.class)); + assertTrue(names.contains(TimeSyncConfig.NAME)); + assertEquals(Integer.valueOf(1), categories.get(org.elasticsearch.client.ml.dataframe.evaluation.Evaluation.class)); + assertThat(names, hasItems(BinarySoftClassification.NAME)); + assertEquals(Integer.valueOf(4), categories.get(org.elasticsearch.client.ml.dataframe.evaluation.EvaluationMetric.class)); + assertThat(names, hasItems(AucRocMetric.NAME, PrecisionMetric.NAME, RecallMetric.NAME, ConfusionMatrixMetric.NAME)); + assertEquals(Integer.valueOf(4), categories.get(org.elasticsearch.client.ml.dataframe.evaluation.EvaluationMetric.Result.class)); + assertThat(names, hasItems(AucRocMetric.NAME, PrecisionMetric.NAME, RecallMetric.NAME, ConfusionMatrixMetric.NAME)); } public void testApiNamingConventions() throws Exception { @@ -731,7 +751,6 @@ public void testApiNamingConventions() throws Exception { "nodes.hot_threads", "nodes.usage", "nodes.reload_secure_settings", - "scripts_painless_context", "search_shards", }; List booleanReturnMethods = Arrays.asList( @@ -805,6 +824,7 @@ public void testApiNamingConventions() throws Exception { apiName.startsWith("ccr.") == false && apiName.startsWith("data_frame") == false && apiName.endsWith("freeze") == false && + apiName.endsWith("reload_analyzers") == false && // IndicesClientIT.getIndexTemplate should be renamed "getTemplate" in version 8.0 when we // can get rid of 7.0's deprecated "getTemplate" apiName.equals("indices.get_index_template") == false) { diff --git a/client/rest-high-level/src/test/java/org/elasticsearch/client/RollupIT.java b/client/rest-high-level/src/test/java/org/elasticsearch/client/RollupIT.java index d92c6ad73a738..a7c8a85131355 100644 --- a/client/rest-high-level/src/test/java/org/elasticsearch/client/RollupIT.java +++ b/client/rest-high-level/src/test/java/org/elasticsearch/client/RollupIT.java @@ -29,8 +29,8 @@ import org.elasticsearch.action.search.SearchRequest; import org.elasticsearch.action.search.SearchResponse; import org.elasticsearch.action.support.WriteRequest; -import org.elasticsearch.client.core.IndexerState; import org.elasticsearch.client.core.AcknowledgedResponse; +import org.elasticsearch.client.core.IndexerState; import org.elasticsearch.client.rollup.DeleteRollupJobRequest; import org.elasticsearch.client.rollup.GetRollupCapsRequest; import org.elasticsearch.client.rollup.GetRollupCapsResponse; @@ -40,10 +40,10 @@ import org.elasticsearch.client.rollup.GetRollupJobResponse; import org.elasticsearch.client.rollup.GetRollupJobResponse.JobWrapper; import org.elasticsearch.client.rollup.PutRollupJobRequest; -import org.elasticsearch.client.rollup.StartRollupJobRequest; -import org.elasticsearch.client.rollup.StartRollupJobResponse; import org.elasticsearch.client.rollup.RollableIndexCaps; import org.elasticsearch.client.rollup.RollupJobCaps; +import org.elasticsearch.client.rollup.StartRollupJobRequest; +import org.elasticsearch.client.rollup.StartRollupJobResponse; import org.elasticsearch.client.rollup.StopRollupJobRequest; import org.elasticsearch.client.rollup.StopRollupJobResponse; import org.elasticsearch.client.rollup.job.config.DateHistogramGroupConfig; @@ -54,10 +54,10 @@ import org.elasticsearch.rest.RestStatus; import org.elasticsearch.search.SearchHit; import org.elasticsearch.search.aggregations.bucket.histogram.DateHistogramInterval; -import org.elasticsearch.search.aggregations.metrics.NumericMetricsAggregation; import org.elasticsearch.search.aggregations.metrics.AvgAggregationBuilder; import org.elasticsearch.search.aggregations.metrics.MaxAggregationBuilder; import org.elasticsearch.search.aggregations.metrics.MinAggregationBuilder; +import org.elasticsearch.search.aggregations.metrics.NumericMetricsAggregation; import org.elasticsearch.search.aggregations.metrics.SumAggregationBuilder; import org.elasticsearch.search.aggregations.metrics.ValueCountAggregationBuilder; import org.elasticsearch.search.builder.SearchSourceBuilder; @@ -259,20 +259,6 @@ public void testSearch() throws Exception { assertThat(avg.value(), closeTo(sum / numDocs, 0.00000001)); } - public void testSearchWithType() throws Exception { - SearchRequest search = new SearchRequest(rollupIndex); - search.types("a", "b", "c"); - search.source(new SearchSourceBuilder() - .size(0) - .aggregation(new AvgAggregationBuilder("avg").field("value"))); - try { - highLevelClient().rollup().search(search, RequestOptions.DEFAULT); - fail("types are not allowed but didn't fail"); - } catch (ValidationException e) { - assertEquals("Validation Failed: 1: types are not allowed in rollup search;", e.getMessage()); - } - } - public void testGetMissingRollupJob() throws Exception { GetRollupJobRequest getRollupJobRequest = new GetRollupJobRequest("missing"); RollupClient rollupClient = highLevelClient().rollup(); diff --git a/client/rest-high-level/src/test/java/org/elasticsearch/client/SecurityRequestConvertersTests.java b/client/rest-high-level/src/test/java/org/elasticsearch/client/SecurityRequestConvertersTests.java index 99350fc29db8a..4c99cb323969e 100644 --- a/client/rest-high-level/src/test/java/org/elasticsearch/client/SecurityRequestConvertersTests.java +++ b/client/rest-high-level/src/test/java/org/elasticsearch/client/SecurityRequestConvertersTests.java @@ -315,7 +315,7 @@ public void testGetApplicationPrivilege() throws Exception { assertNull(request.getEntity()); } - public void testGetAllApplicationPrivileges() throws Exception { + public void testGetAllPrivilegesForApplication() throws Exception { final String application = randomAlphaOfLength(6); GetPrivilegesRequest getPrivilegesRequest = GetPrivilegesRequest.getApplicationPrivileges(application); Request request = SecurityRequestConverters.getPrivileges(getPrivilegesRequest); @@ -339,7 +339,7 @@ public void testGetMultipleApplicationPrivileges() throws Exception { assertNull(request.getEntity()); } - public void testGetAllPrivileges() throws Exception { + public void testGetAllApplicationPrivileges() throws Exception { GetPrivilegesRequest getPrivilegesRequest = GetPrivilegesRequest.getAllPrivileges(); Request request = SecurityRequestConverters.getPrivileges(getPrivilegesRequest); assertEquals(HttpGet.METHOD_NAME, request.getMethod()); diff --git a/client/rest-high-level/src/test/java/org/elasticsearch/client/SnapshotIT.java b/client/rest-high-level/src/test/java/org/elasticsearch/client/SnapshotIT.java index 616850c513af7..8e4001442b0cc 100644 --- a/client/rest-high-level/src/test/java/org/elasticsearch/client/SnapshotIT.java +++ b/client/rest-high-level/src/test/java/org/elasticsearch/client/SnapshotIT.java @@ -41,14 +41,16 @@ import org.elasticsearch.repositories.fs.FsRepository; import org.elasticsearch.rest.RestStatus; import org.elasticsearch.snapshots.RestoreInfo; +import org.mockito.internal.util.collections.Sets; import java.io.IOException; import java.util.Collections; -import java.util.stream.Collectors; +import java.util.HashMap; +import java.util.Map; -import static org.hamcrest.Matchers.contains; import static org.hamcrest.Matchers.equalTo; import static org.hamcrest.Matchers.greaterThan; +import static org.hamcrest.Matchers.hasSize; import static org.hamcrest.Matchers.is; public class SnapshotIT extends ESRestHighLevelClientTestCase { @@ -58,14 +60,14 @@ private AcknowledgedResponse createTestRepository(String repository, String type request.settings(settings, XContentType.JSON); request.type(type); return execute(request, highLevelClient().snapshot()::createRepository, - highLevelClient().snapshot()::createRepositoryAsync); + highLevelClient().snapshot()::createRepositoryAsync); } private CreateSnapshotResponse createTestSnapshot(CreateSnapshotRequest createSnapshotRequest) throws IOException { // assumes the repository already exists return execute(createSnapshotRequest, highLevelClient().snapshot()::create, - highLevelClient().snapshot()::createAsync); + highLevelClient().snapshot()::createAsync); } public void testCreateRepository() throws IOException { @@ -81,7 +83,7 @@ public void testSnapshotGetRepositoriesUsingParams() throws IOException { GetRepositoriesRequest request = new GetRepositoriesRequest(); request.repositories(new String[]{testRepository}); GetRepositoriesResponse response = execute(request, highLevelClient().snapshot()::getRepository, - highLevelClient().snapshot()::getRepositoryAsync); + highLevelClient().snapshot()::getRepositoryAsync); assertThat(1, equalTo(response.repositories().size())); } @@ -90,7 +92,7 @@ public void testSnapshotGetDefaultRepositories() throws IOException { assertTrue(createTestRepository("test", FsRepository.TYPE, "{\"location\": \".\"}").isAcknowledged()); GetRepositoriesResponse response = execute(new GetRepositoriesRequest(), highLevelClient().snapshot()::getRepository, - highLevelClient().snapshot()::getRepositoryAsync); + highLevelClient().snapshot()::getRepositoryAsync); assertThat(2, equalTo(response.repositories().size())); } @@ -98,11 +100,11 @@ public void testSnapshotGetRepositoriesNonExistent() { String repository = "doesnotexist"; GetRepositoriesRequest request = new GetRepositoriesRequest(new String[]{repository}); ElasticsearchException exception = expectThrows(ElasticsearchException.class, () -> execute(request, - highLevelClient().snapshot()::getRepository, highLevelClient().snapshot()::getRepositoryAsync)); + highLevelClient().snapshot()::getRepository, highLevelClient().snapshot()::getRepositoryAsync)); assertThat(exception.status(), equalTo(RestStatus.NOT_FOUND)); assertThat(exception.getMessage(), equalTo( - "Elasticsearch exception [type=repository_missing_exception, reason=[" + repository + "] missing]")); + "Elasticsearch exception [type=repository_missing_exception, reason=[" + repository + "] missing]")); } public void testSnapshotDeleteRepository() throws IOException { @@ -111,12 +113,12 @@ public void testSnapshotDeleteRepository() throws IOException { GetRepositoriesRequest request = new GetRepositoriesRequest(); GetRepositoriesResponse response = execute(request, highLevelClient().snapshot()::getRepository, - highLevelClient().snapshot()::getRepositoryAsync); + highLevelClient().snapshot()::getRepositoryAsync); assertThat(1, equalTo(response.repositories().size())); DeleteRepositoryRequest deleteRequest = new DeleteRepositoryRequest(repository); AcknowledgedResponse deleteResponse = execute(deleteRequest, highLevelClient().snapshot()::deleteRepository, - highLevelClient().snapshot()::deleteRepositoryAsync); + highLevelClient().snapshot()::deleteRepositoryAsync); assertTrue(deleteResponse.isAcknowledged()); } @@ -127,7 +129,7 @@ public void testVerifyRepository() throws IOException { VerifyRepositoryRequest request = new VerifyRepositoryRequest("test"); VerifyRepositoryResponse response = execute(request, highLevelClient().snapshot()::verifyRepository, - highLevelClient().snapshot()::verifyRepositoryAsync); + highLevelClient().snapshot()::verifyRepositoryAsync); assertThat(response.getNodes().size(), equalTo(1)); } @@ -139,6 +141,9 @@ public void testCreateSnapshot() throws IOException { CreateSnapshotRequest request = new CreateSnapshotRequest(repository, snapshot); boolean waitForCompletion = randomBoolean(); request.waitForCompletion(waitForCompletion); + if (randomBoolean()) { + request.userMetadata(randomUserMetadata()); + } request.partial(randomBoolean()); request.includeGlobalState(randomBoolean()); @@ -147,47 +152,59 @@ public void testCreateSnapshot() throws IOException { if (waitForCompletion == false) { // If we don't wait for the snapshot to complete we have to cancel it to not leak the snapshot task AcknowledgedResponse deleteResponse = execute( - new DeleteSnapshotRequest(repository, snapshot), - highLevelClient().snapshot()::delete, highLevelClient().snapshot()::deleteAsync + new DeleteSnapshotRequest(repository, snapshot), + highLevelClient().snapshot()::delete, highLevelClient().snapshot()::deleteAsync ); assertTrue(deleteResponse.isAcknowledged()); } } public void testGetSnapshots() throws IOException { - String repository = "test_repository"; + String repository1 = "test_repository1"; + String repository2 = "test_repository2"; String snapshot1 = "test_snapshot1"; String snapshot2 = "test_snapshot2"; - AcknowledgedResponse putRepositoryResponse = createTestRepository(repository, FsRepository.TYPE, "{\"location\": \".\"}"); + AcknowledgedResponse putRepositoryResponse = + createTestRepository(repository1, FsRepository.TYPE, "{\"location\": \"loc1\"}"); assertTrue(putRepositoryResponse.isAcknowledged()); - CreateSnapshotRequest createSnapshotRequest1 = new CreateSnapshotRequest(repository, snapshot1); + AcknowledgedResponse putRepositoryResponse2 = + createTestRepository(repository2, FsRepository.TYPE, "{\"location\": \"loc2\"}"); + assertTrue(putRepositoryResponse2.isAcknowledged()); + + CreateSnapshotRequest createSnapshotRequest1 = new CreateSnapshotRequest(repository1, snapshot1); createSnapshotRequest1.waitForCompletion(true); CreateSnapshotResponse putSnapshotResponse1 = createTestSnapshot(createSnapshotRequest1); - CreateSnapshotRequest createSnapshotRequest2 = new CreateSnapshotRequest(repository, snapshot2); + CreateSnapshotRequest createSnapshotRequest2 = new CreateSnapshotRequest(repository2, snapshot2); createSnapshotRequest2.waitForCompletion(true); + Map originalMetadata = randomUserMetadata(); + createSnapshotRequest2.userMetadata(originalMetadata); CreateSnapshotResponse putSnapshotResponse2 = createTestSnapshot(createSnapshotRequest2); // check that the request went ok without parsing JSON here. When using the high level client, check acknowledgement instead. assertEquals(RestStatus.OK, putSnapshotResponse1.status()); assertEquals(RestStatus.OK, putSnapshotResponse2.status()); - GetSnapshotsRequest request; - if (randomBoolean()) { - request = new GetSnapshotsRequest(repository); - } else if (randomBoolean()) { - request = new GetSnapshotsRequest(repository, new String[] {"_all"}); + GetSnapshotsRequest request = new GetSnapshotsRequest( + randomFrom(new String[]{"_all"}, new String[]{"*"}, new String[]{repository1, repository2}), + randomFrom(new String[]{"_all"}, new String[]{"*"}, new String[]{snapshot1, snapshot2}) + ); + request.ignoreUnavailable(true); - } else { - request = new GetSnapshotsRequest(repository, new String[] {snapshot1, snapshot2}); - } GetSnapshotsResponse response = execute(request, highLevelClient().snapshot()::get, highLevelClient().snapshot()::getAsync); - assertEquals(2, response.getSnapshots().size()); - assertThat(response.getSnapshots().stream().map((s) -> s.snapshotId().getName()).collect(Collectors.toList()), - contains("test_snapshot1", "test_snapshot2")); + assertThat(response.isFailed(), is(false)); + assertThat(response.getRepositories(), equalTo(Sets.newSet(repository1, repository2))); + + assertThat(response.getSnapshots(repository1), hasSize(1)); + assertThat(response.getSnapshots(repository1).get(0).snapshotId().getName(), equalTo(snapshot1)); + + assertThat(response.getSnapshots(repository2), hasSize(1)); + assertThat(response.getSnapshots(repository2).get(0).snapshotId().getName(), equalTo(snapshot2)); + assertThat(response.getSnapshots(repository2).get(0).userMetadata(), equalTo(originalMetadata)); } + public void testSnapshotsStatus() throws IOException { String testRepository = "test"; String testSnapshot = "snapshot"; @@ -209,7 +226,7 @@ public void testSnapshotsStatus() throws IOException { request.repository(testRepository); request.snapshots(new String[]{testSnapshot}); SnapshotsStatusResponse response = execute(request, highLevelClient().snapshot()::status, - highLevelClient().snapshot()::statusAsync); + highLevelClient().snapshot()::statusAsync); assertThat(response.getSnapshots().size(), equalTo(1)); assertThat(response.getSnapshots().get(0).getSnapshot().getRepository(), equalTo(testRepository)); assertThat(response.getSnapshots().get(0).getSnapshot().getSnapshotId().getName(), equalTo(testSnapshot)); @@ -231,6 +248,9 @@ public void testRestoreSnapshot() throws IOException { CreateSnapshotRequest createSnapshotRequest = new CreateSnapshotRequest(testRepository, testSnapshot); createSnapshotRequest.indices(testIndex); createSnapshotRequest.waitForCompletion(true); + if (randomBoolean()) { + createSnapshotRequest.userMetadata(randomUserMetadata()); + } CreateSnapshotResponse createSnapshotResponse = createTestSnapshot(createSnapshotRequest); assertEquals(RestStatus.OK, createSnapshotResponse.status()); @@ -243,7 +263,7 @@ public void testRestoreSnapshot() throws IOException { request.renameReplacement(restoredIndex); RestoreSnapshotResponse response = execute(request, highLevelClient().snapshot()::restore, - highLevelClient().snapshot()::restoreAsync); + highLevelClient().snapshot()::restoreAsync); RestoreInfo restoreInfo = response.getRestoreInfo(); assertThat(restoreInfo.name(), equalTo(testSnapshot)); @@ -261,6 +281,9 @@ public void testDeleteSnapshot() throws IOException { CreateSnapshotRequest createSnapshotRequest = new CreateSnapshotRequest(repository, snapshot); createSnapshotRequest.waitForCompletion(true); + if (randomBoolean()) { + createSnapshotRequest.userMetadata(randomUserMetadata()); + } CreateSnapshotResponse createSnapshotResponse = createTestSnapshot(createSnapshotRequest); // check that the request went ok without parsing JSON here. When using the high level client, check acknowledgement instead. assertEquals(RestStatus.OK, createSnapshotResponse.status()); @@ -270,4 +293,28 @@ public void testDeleteSnapshot() throws IOException { assertTrue(response.isAcknowledged()); } -} + + private static Map randomUserMetadata() { + if (randomBoolean()) { + return null; + } + + Map metadata = new HashMap<>(); + long fields = randomLongBetween(0, 4); + for (int i = 0; i < fields; i++) { + if (randomBoolean()) { + metadata.put(randomValueOtherThanMany(metadata::containsKey, () -> randomAlphaOfLengthBetween(2,10)), + randomAlphaOfLengthBetween(5, 5)); + } else { + Map nested = new HashMap<>(); + long nestedFields = randomLongBetween(0, 4); + for (int j = 0; j < nestedFields; j++) { + nested.put(randomValueOtherThanMany(nested::containsKey, () -> randomAlphaOfLengthBetween(2,10)), + randomAlphaOfLengthBetween(5, 5)); + } + metadata.put(randomValueOtherThanMany(metadata::containsKey, () -> randomAlphaOfLengthBetween(2,10)), nested); + } + } + return metadata; + } +} \ No newline at end of file diff --git a/client/rest-high-level/src/test/java/org/elasticsearch/client/SnapshotRequestConvertersTests.java b/client/rest-high-level/src/test/java/org/elasticsearch/client/SnapshotRequestConvertersTests.java index 66720b70ee3a6..23789d390357b 100644 --- a/client/rest-high-level/src/test/java/org/elasticsearch/client/SnapshotRequestConvertersTests.java +++ b/client/rest-high-level/src/test/java/org/elasticsearch/client/SnapshotRequestConvertersTests.java @@ -41,7 +41,6 @@ import java.io.IOException; import java.nio.file.Path; -import java.util.Arrays; import java.util.HashMap; import java.util.Locale; import java.util.Map; @@ -78,7 +77,8 @@ public void testCreateRepository() throws IOException { Path repositoryLocation = PathUtils.get("."); PutRepositoryRequest putRepositoryRequest = new PutRepositoryRequest(repository); putRepositoryRequest.type(FsRepository.TYPE); - putRepositoryRequest.verify(randomBoolean()); + final boolean verify = randomBoolean(); + putRepositoryRequest.verify(verify); putRepositoryRequest.settings( Settings.builder() @@ -90,6 +90,11 @@ public void testCreateRepository() throws IOException { Request request = SnapshotRequestConverters.createRepository(putRepositoryRequest); assertThat(request.getEndpoint(), equalTo(endpoint)); assertThat(request.getMethod(), equalTo(HttpPut.METHOD_NAME)); + if (verify) { + assertThat(request.getParameters().get("verify"), nullValue()); + } else { + assertThat(request.getParameters().get("verify"), equalTo("false")); + } RequestConvertersTests.assertToXContentBody(putRepositoryRequest, request.getEntity()); } @@ -148,15 +153,16 @@ public void testCreateSnapshot() throws IOException { public void testGetSnapshots() { Map expectedParams = new HashMap<>(); - String repository = RequestConvertersTests.randomIndicesNames(1, 1)[0]; + String repository1 = randomAlphaOfLength(10); + String repository2 = randomAlphaOfLength(10); String snapshot1 = "snapshot1-" + randomAlphaOfLengthBetween(2, 5).toLowerCase(Locale.ROOT); String snapshot2 = "snapshot2-" + randomAlphaOfLengthBetween(2, 5).toLowerCase(Locale.ROOT); - String endpoint = String.format(Locale.ROOT, "/_snapshot/%s/%s,%s", repository, snapshot1, snapshot2); + String endpoint = String.format(Locale.ROOT, "/_snapshot/%s,%s/%s,%s", repository1, repository2, snapshot1, snapshot2); GetSnapshotsRequest getSnapshotsRequest = new GetSnapshotsRequest(); - getSnapshotsRequest.repository(repository); - getSnapshotsRequest.snapshots(Arrays.asList(snapshot1, snapshot2).toArray(new String[0])); + getSnapshotsRequest.repositories(repository1, repository2); + getSnapshotsRequest.snapshots(new String[]{snapshot1, snapshot2}); RequestConvertersTests.setRandomMasterTimeout(getSnapshotsRequest, expectedParams); if (randomBoolean()) { diff --git a/client/rest-high-level/src/test/java/org/elasticsearch/client/XPackInfoResponseTests.java b/client/rest-high-level/src/test/java/org/elasticsearch/client/XPackInfoResponseTests.java index 052e700a59f1a..59a7a830a6304 100644 --- a/client/rest-high-level/src/test/java/org/elasticsearch/client/XPackInfoResponseTests.java +++ b/client/rest-high-level/src/test/java/org/elasticsearch/client/XPackInfoResponseTests.java @@ -70,8 +70,7 @@ private LicenseInfo convertHlrcToInternal(org.elasticsearch.client.xpack.XPackIn private FeatureSetsInfo convertHlrcToInternal(org.elasticsearch.client.xpack.XPackInfoResponse.FeatureSetsInfo featureSetsInfo) { return featureSetsInfo != null ? new FeatureSetsInfo(featureSetsInfo.getFeatureSets().values().stream() - .map(fs -> new FeatureSet(fs.name(), fs.description(), fs.available(), fs.enabled(), - fs.nativeCodeInfo())) + .map(fs -> new FeatureSet(fs.name(), fs.available(), fs.enabled())) .collect(Collectors.toSet())) : null; } @@ -169,21 +168,7 @@ private FeatureSetsInfo mutateFeatureSetsInfo(FeatureSetsInfo featureSetsInfo) { private FeatureSet randomFeatureSet() { return new FeatureSet( randomAlphaOfLength(5), - randomBoolean() ? null : randomAlphaOfLength(20), randomBoolean(), - randomBoolean(), - randomNativeCodeInfo()); - } - - private Map randomNativeCodeInfo() { - if (randomBoolean()) { - return null; - } - int size = between(0, 10); - Map nativeCodeInfo = new HashMap<>(size); - while (nativeCodeInfo.size() < size) { - nativeCodeInfo.put(randomAlphaOfLength(5), randomAlphaOfLength(5)); - } - return nativeCodeInfo; + randomBoolean()); } } diff --git a/client/rest-high-level/src/test/java/org/elasticsearch/client/ccr/CcrStatsResponseTests.java b/client/rest-high-level/src/test/java/org/elasticsearch/client/ccr/CcrStatsResponseTests.java index d56b762520c55..eaf6103a0ecfe 100644 --- a/client/rest-high-level/src/test/java/org/elasticsearch/client/ccr/CcrStatsResponseTests.java +++ b/client/rest-high-level/src/test/java/org/elasticsearch/client/ccr/CcrStatsResponseTests.java @@ -106,6 +106,7 @@ static FollowStatsAction.StatsResponses createStatsResponse() { randomNonNegativeLong(), randomNonNegativeLong(), randomNonNegativeLong(), + randomNonNegativeLong(), Collections.emptyNavigableMap(), randomLong(), randomBoolean() ? new ElasticsearchException("fatal error") : null); @@ -190,6 +191,8 @@ protected void assertInstances(CcrStatsAction.Response serverTestInstance, CcrSt equalTo(expectedShardFollowStats.followerMappingVersion())); assertThat(actualShardFollowStats.getFollowerSettingsVersion(), equalTo(expectedShardFollowStats.followerSettingsVersion())); + assertThat(actualShardFollowStats.getFollowerAliasesVersion(), + equalTo(expectedShardFollowStats.followerAliasesVersion())); assertThat(actualShardFollowStats.getTotalReadTimeMillis(), equalTo(expectedShardFollowStats.totalReadTimeMillis())); assertThat(actualShardFollowStats.getSuccessfulReadRequests(), diff --git a/client/rest-high-level/src/test/java/org/elasticsearch/client/ccr/FollowStatsResponseTests.java b/client/rest-high-level/src/test/java/org/elasticsearch/client/ccr/FollowStatsResponseTests.java index cd7257342c724..ff93c8df33eda 100644 --- a/client/rest-high-level/src/test/java/org/elasticsearch/client/ccr/FollowStatsResponseTests.java +++ b/client/rest-high-level/src/test/java/org/elasticsearch/client/ccr/FollowStatsResponseTests.java @@ -93,6 +93,8 @@ protected void assertInstances(FollowStatsAction.StatsResponses serverTestInstan equalTo(expectedShardFollowStats.followerMappingVersion())); assertThat(actualShardFollowStats.getFollowerSettingsVersion(), equalTo(expectedShardFollowStats.followerSettingsVersion())); + assertThat(actualShardFollowStats.getFollowerAliasesVersion(), + equalTo(expectedShardFollowStats.followerAliasesVersion())); assertThat(actualShardFollowStats.getTotalReadTimeMillis(), equalTo(expectedShardFollowStats.totalReadTimeMillis())); assertThat(actualShardFollowStats.getSuccessfulReadRequests(), diff --git a/client/rest-high-level/src/test/java/org/elasticsearch/client/core/TermVectorsResponseTests.java b/client/rest-high-level/src/test/java/org/elasticsearch/client/core/TermVectorsResponseTests.java index 714a7269a19d9..debe08879339d 100644 --- a/client/rest-high-level/src/test/java/org/elasticsearch/client/core/TermVectorsResponseTests.java +++ b/client/rest-high-level/src/test/java/org/elasticsearch/client/core/TermVectorsResponseTests.java @@ -22,11 +22,11 @@ import org.elasticsearch.common.xcontent.XContentBuilder; import org.elasticsearch.test.ESTestCase; -import java.util.ArrayList; -import java.util.List; import java.io.IOException; +import java.util.ArrayList; import java.util.Collections; import java.util.Comparator; +import java.util.List; import static org.elasticsearch.test.AbstractXContentTestCase.xContentTester; @@ -47,7 +47,6 @@ public void testFromXContent() throws IOException { static void toXContent(TermVectorsResponse response, XContentBuilder builder) throws IOException { builder.startObject(); builder.field("_index", response.getIndex()); - builder.field("_type", response.getType()); if (response.getId() != null) { builder.field("_id", response.getId()); } @@ -119,7 +118,6 @@ private static void toXContent(TermVectorsResponse.TermVector tv, XContentBuilde static TermVectorsResponse createTestInstance() { String index = randomAlphaOfLength(5); - String type = randomAlphaOfLength(5); String id = String.valueOf(randomIntBetween(1,100)); long version = randomNonNegativeLong(); long tookInMillis = randomNonNegativeLong(); @@ -142,7 +140,7 @@ static TermVectorsResponse createTestInstance() { fieldName, hasFieldStatistics, hasTermStatistics, hasScores, hasOffsets, hasPositions, hasPayloads)); } } - TermVectorsResponse tvresponse = new TermVectorsResponse(index, type, id, version, found, tookInMillis, tvList); + TermVectorsResponse tvresponse = new TermVectorsResponse(index, id, version, found, tookInMillis, tvList); return tvresponse; } diff --git a/client/rest-high-level/src/test/java/org/elasticsearch/client/dataframe/GetDataFrameTransformResponseTests.java b/client/rest-high-level/src/test/java/org/elasticsearch/client/dataframe/GetDataFrameTransformResponseTests.java index f7386e936301b..c37e8f8997185 100644 --- a/client/rest-high-level/src/test/java/org/elasticsearch/client/dataframe/GetDataFrameTransformResponseTests.java +++ b/client/rest-high-level/src/test/java/org/elasticsearch/client/dataframe/GetDataFrameTransformResponseTests.java @@ -78,7 +78,10 @@ private static void toXContent(GetDataFrameTransformResponse response, XContentB @Override protected NamedXContentRegistry xContentRegistry() { - SearchModule searchModule = new SearchModule(Settings.EMPTY, false, Collections.emptyList()); - return new NamedXContentRegistry(searchModule.getNamedXContents()); + SearchModule searchModule = new SearchModule(Settings.EMPTY, Collections.emptyList()); + List namedXContents = searchModule.getNamedXContents(); + namedXContents.addAll(new DataFrameNamedXContentProvider().getNamedXContentParsers()); + + return new NamedXContentRegistry(namedXContents); } } diff --git a/client/rest-high-level/src/test/java/org/elasticsearch/client/dataframe/PreviewDataFrameTransformRequestTests.java b/client/rest-high-level/src/test/java/org/elasticsearch/client/dataframe/PreviewDataFrameTransformRequestTests.java index c91e1cbb1dd91..d335e6a497a8e 100644 --- a/client/rest-high-level/src/test/java/org/elasticsearch/client/dataframe/PreviewDataFrameTransformRequestTests.java +++ b/client/rest-high-level/src/test/java/org/elasticsearch/client/dataframe/PreviewDataFrameTransformRequestTests.java @@ -31,6 +31,7 @@ import java.io.IOException; import java.util.Collections; +import java.util.List; import java.util.Optional; import static org.elasticsearch.client.dataframe.transforms.SourceConfigTests.randomSourceConfig; @@ -54,8 +55,11 @@ protected boolean supportsUnknownFields() { @Override protected NamedXContentRegistry xContentRegistry() { - SearchModule searchModule = new SearchModule(Settings.EMPTY, false, Collections.emptyList()); - return new NamedXContentRegistry(searchModule.getNamedXContents()); + SearchModule searchModule = new SearchModule(Settings.EMPTY, Collections.emptyList()); + List namedXContents = searchModule.getNamedXContents(); + namedXContents.addAll(new DataFrameNamedXContentProvider().getNamedXContentParsers()); + + return new NamedXContentRegistry(namedXContents); } public void testValidate() { diff --git a/client/rest-high-level/src/test/java/org/elasticsearch/client/dataframe/PreviewDataFrameTransformResponseTests.java b/client/rest-high-level/src/test/java/org/elasticsearch/client/dataframe/PreviewDataFrameTransformResponseTests.java index 31f1a26d6f1d0..c0e4cfb7eeb31 100644 --- a/client/rest-high-level/src/test/java/org/elasticsearch/client/dataframe/PreviewDataFrameTransformResponseTests.java +++ b/client/rest-high-level/src/test/java/org/elasticsearch/client/dataframe/PreviewDataFrameTransformResponseTests.java @@ -53,8 +53,13 @@ private PreviewDataFrameTransformResponse createTestInstance() { } docs.add(doc); } + int numMappingEntries = randomIntBetween(5, 10); + Map mappings = new HashMap<>(numMappingEntries); + for (int i = 0; i < numMappingEntries; i++) { + mappings.put(randomAlphaOfLength(10), Map.of("type", randomAlphaOfLength(10))); + } - return new PreviewDataFrameTransformResponse(docs); + return new PreviewDataFrameTransformResponse(docs, mappings); } private void toXContent(PreviewDataFrameTransformResponse response, XContentBuilder builder) throws IOException { @@ -64,6 +69,7 @@ private void toXContent(PreviewDataFrameTransformResponse response, XContentBuil builder.map(doc); } builder.endArray(); + builder.field("mappings", response.getMappings()); builder.endObject(); } } diff --git a/client/rest-high-level/src/test/java/org/elasticsearch/client/dataframe/PutDataFrameTransformRequestTests.java b/client/rest-high-level/src/test/java/org/elasticsearch/client/dataframe/PutDataFrameTransformRequestTests.java index 28fd92dcf913f..01e1db2cb3823 100644 --- a/client/rest-high-level/src/test/java/org/elasticsearch/client/dataframe/PutDataFrameTransformRequestTests.java +++ b/client/rest-high-level/src/test/java/org/elasticsearch/client/dataframe/PutDataFrameTransformRequestTests.java @@ -31,6 +31,7 @@ import java.io.IOException; import java.util.Collections; +import java.util.List; import java.util.Optional; import static org.hamcrest.Matchers.containsString; @@ -70,7 +71,10 @@ protected boolean supportsUnknownFields() { @Override protected NamedXContentRegistry xContentRegistry() { - SearchModule searchModule = new SearchModule(Settings.EMPTY, false, Collections.emptyList()); - return new NamedXContentRegistry(searchModule.getNamedXContents()); + SearchModule searchModule = new SearchModule(Settings.EMPTY, Collections.emptyList()); + List namedXContents = searchModule.getNamedXContents(); + namedXContents.addAll(new DataFrameNamedXContentProvider().getNamedXContentParsers()); + + return new NamedXContentRegistry(namedXContents); } } diff --git a/client/rest-high-level/src/test/java/org/elasticsearch/client/dataframe/transforms/DataFrameTransformConfigTests.java b/client/rest-high-level/src/test/java/org/elasticsearch/client/dataframe/transforms/DataFrameTransformConfigTests.java index 1b5228d96229f..79b7e85098e04 100644 --- a/client/rest-high-level/src/test/java/org/elasticsearch/client/dataframe/transforms/DataFrameTransformConfigTests.java +++ b/client/rest-high-level/src/test/java/org/elasticsearch/client/dataframe/transforms/DataFrameTransformConfigTests.java @@ -19,6 +19,8 @@ package org.elasticsearch.client.dataframe.transforms; +import org.elasticsearch.client.dataframe.DataFrameNamedXContentProvider; +import org.elasticsearch.Version; import org.elasticsearch.client.dataframe.transforms.pivot.PivotConfigTests; import org.elasticsearch.common.settings.Settings; import org.elasticsearch.common.xcontent.NamedXContentRegistry; @@ -27,7 +29,9 @@ import org.elasticsearch.test.AbstractXContentTestCase; import java.io.IOException; +import java.time.Instant; import java.util.Collections; +import java.util.List; import java.util.function.Predicate; import static org.elasticsearch.client.dataframe.transforms.DestConfigTests.randomDestConfig; @@ -36,8 +40,18 @@ public class DataFrameTransformConfigTests extends AbstractXContentTestCase { public static DataFrameTransformConfig randomDataFrameTransformConfig() { - return new DataFrameTransformConfig(randomAlphaOfLengthBetween(1, 10), randomSourceConfig(), - randomDestConfig(), PivotConfigTests.randomPivotConfig(), randomBoolean() ? null : randomAlphaOfLengthBetween(1, 100)); + return new DataFrameTransformConfig(randomAlphaOfLengthBetween(1, 10), + randomSourceConfig(), + randomDestConfig(), + randomBoolean() ? null : randomSyncConfig(), + PivotConfigTests.randomPivotConfig(), + randomBoolean() ? null : randomAlphaOfLengthBetween(1, 100), + randomBoolean() ? null : Instant.now(), + randomBoolean() ? null : Version.CURRENT.toString()); + } + + public static SyncConfig randomSyncConfig() { + return TimeSyncConfigTests.randomTimeSyncConfig(); } @Override @@ -63,7 +77,10 @@ protected Predicate getRandomFieldsExcludeFilter() { @Override protected NamedXContentRegistry xContentRegistry() { - SearchModule searchModule = new SearchModule(Settings.EMPTY, false, Collections.emptyList()); - return new NamedXContentRegistry(searchModule.getNamedXContents()); + SearchModule searchModule = new SearchModule(Settings.EMPTY, Collections.emptyList()); + List namedXContents = searchModule.getNamedXContents(); + namedXContents.addAll(new DataFrameNamedXContentProvider().getNamedXContentParsers()); + + return new NamedXContentRegistry(namedXContents); } } diff --git a/client/rest-high-level/src/test/java/org/elasticsearch/client/dataframe/transforms/DataFrameTransformStateTests.java b/client/rest-high-level/src/test/java/org/elasticsearch/client/dataframe/transforms/DataFrameTransformStateTests.java index 4ada50c20d219..ebb62890c3cdd 100644 --- a/client/rest-high-level/src/test/java/org/elasticsearch/client/dataframe/transforms/DataFrameTransformStateTests.java +++ b/client/rest-high-level/src/test/java/org/elasticsearch/client/dataframe/transforms/DataFrameTransformStateTests.java @@ -20,6 +20,7 @@ package org.elasticsearch.client.dataframe.transforms; import org.elasticsearch.client.core.IndexerState; +import org.elasticsearch.common.xcontent.ToXContent; import org.elasticsearch.common.xcontent.XContentBuilder; import org.elasticsearch.test.ESTestCase; @@ -37,7 +38,8 @@ public void testFromXContent() throws IOException { DataFrameTransformStateTests::toXContent, DataFrameTransformState::fromXContent) .supportsUnknownFields(true) - .randomFieldsExcludeFilter(field -> field.equals("current_position")) + .randomFieldsExcludeFilter(field -> field.equals("current_position") || + field.equals("node.attributes")) .test(); } @@ -47,7 +49,8 @@ public static DataFrameTransformState randomDataFrameTransformState() { randomPositionMap(), randomLongBetween(0,10), randomBoolean() ? null : randomAlphaOfLength(10), - randomBoolean() ? null : DataFrameTransformProgressTests.randomInstance()); + randomBoolean() ? null : DataFrameTransformProgressTests.randomInstance(), + randomBoolean() ? null : NodeAttributesTests.createRandom()); } public static void toXContent(DataFrameTransformState state, XContentBuilder builder) throws IOException { @@ -65,6 +68,10 @@ public static void toXContent(DataFrameTransformState state, XContentBuilder bui builder.field("progress"); DataFrameTransformProgressTests.toXContent(state.getProgress(), builder); } + if (state.getNode() != null) { + builder.field("node"); + state.getNode().toXContent(builder, ToXContent.EMPTY_PARAMS); + } builder.endObject(); } diff --git a/client/rest-high-level/src/test/java/org/elasticsearch/client/dataframe/transforms/DestConfigTests.java b/client/rest-high-level/src/test/java/org/elasticsearch/client/dataframe/transforms/DestConfigTests.java index f2950b64cf7c9..0dc8f99d7631b 100644 --- a/client/rest-high-level/src/test/java/org/elasticsearch/client/dataframe/transforms/DestConfigTests.java +++ b/client/rest-high-level/src/test/java/org/elasticsearch/client/dataframe/transforms/DestConfigTests.java @@ -27,7 +27,8 @@ public class DestConfigTests extends AbstractXContentTestCase { public static DestConfig randomDestConfig() { - return new DestConfig(randomAlphaOfLength(10)); + return new DestConfig(randomAlphaOfLength(10), + randomBoolean() ? null : randomAlphaOfLength(10)); } @Override diff --git a/client/rest-high-level/src/test/java/org/elasticsearch/client/dataframe/transforms/NodeAttributesTests.java b/client/rest-high-level/src/test/java/org/elasticsearch/client/dataframe/transforms/NodeAttributesTests.java new file mode 100644 index 0000000000000..661aa9f7a30a4 --- /dev/null +++ b/client/rest-high-level/src/test/java/org/elasticsearch/client/dataframe/transforms/NodeAttributesTests.java @@ -0,0 +1,64 @@ +/* + * 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.client.dataframe.transforms; + +import org.elasticsearch.common.xcontent.XContentParser; +import org.elasticsearch.test.AbstractXContentTestCase; + +import java.io.IOException; +import java.util.HashMap; +import java.util.Map; +import java.util.function.Predicate; + +public class NodeAttributesTests extends AbstractXContentTestCase { + + public static NodeAttributes createRandom() { + int numberOfAttributes = randomIntBetween(1, 10); + Map attributes = new HashMap<>(numberOfAttributes); + for(int i = 0; i < numberOfAttributes; i++) { + String val = randomAlphaOfLength(10); + attributes.put("key-"+i, val); + } + return new NodeAttributes(randomAlphaOfLength(10), + randomAlphaOfLength(10), + randomAlphaOfLength(10), + randomAlphaOfLength(10), + attributes); + } + + @Override + protected NodeAttributes createTestInstance() { + return createRandom(); + } + + @Override + protected NodeAttributes doParseInstance(XContentParser parser) throws IOException { + return NodeAttributes.PARSER.parse(parser, null); + } + + @Override + protected Predicate getRandomFieldsExcludeFilter() { + return field -> !field.isEmpty(); + } + + @Override + protected boolean supportsUnknownFields() { + return true; + } +} diff --git a/client/rest-high-level/src/test/java/org/elasticsearch/client/dataframe/transforms/QueryConfigTests.java b/client/rest-high-level/src/test/java/org/elasticsearch/client/dataframe/transforms/QueryConfigTests.java index 644858cab286f..bfa14d4a27a90 100644 --- a/client/rest-high-level/src/test/java/org/elasticsearch/client/dataframe/transforms/QueryConfigTests.java +++ b/client/rest-high-level/src/test/java/org/elasticsearch/client/dataframe/transforms/QueryConfigTests.java @@ -56,7 +56,7 @@ protected boolean supportsUnknownFields() { @Override protected NamedXContentRegistry xContentRegistry() { - SearchModule searchModule = new SearchModule(Settings.EMPTY, false, emptyList()); + SearchModule searchModule = new SearchModule(Settings.EMPTY, emptyList()); return new NamedXContentRegistry(searchModule.getNamedXContents()); } } diff --git a/client/rest-high-level/src/test/java/org/elasticsearch/client/dataframe/transforms/SourceConfigTests.java b/client/rest-high-level/src/test/java/org/elasticsearch/client/dataframe/transforms/SourceConfigTests.java index 722c265c9a05e..e9e5548e93938 100644 --- a/client/rest-high-level/src/test/java/org/elasticsearch/client/dataframe/transforms/SourceConfigTests.java +++ b/client/rest-high-level/src/test/java/org/elasticsearch/client/dataframe/transforms/SourceConfigTests.java @@ -62,7 +62,7 @@ protected SourceConfig createTestInstance() { @Override protected NamedXContentRegistry xContentRegistry() { - SearchModule searchModule = new SearchModule(Settings.EMPTY, false, emptyList()); + SearchModule searchModule = new SearchModule(Settings.EMPTY, emptyList()); return new NamedXContentRegistry(searchModule.getNamedXContents()); } } diff --git a/server/src/main/java/org/elasticsearch/action/admin/indices/exists/types/TypesExistsResponse.java b/client/rest-high-level/src/test/java/org/elasticsearch/client/dataframe/transforms/TimeSyncConfigTests.java similarity index 52% rename from server/src/main/java/org/elasticsearch/action/admin/indices/exists/types/TypesExistsResponse.java rename to client/rest-high-level/src/test/java/org/elasticsearch/client/dataframe/transforms/TimeSyncConfigTests.java index c492c392933e0..dd2a17eb0260d 100644 --- a/server/src/main/java/org/elasticsearch/action/admin/indices/exists/types/TypesExistsResponse.java +++ b/client/rest-high-level/src/test/java/org/elasticsearch/client/dataframe/transforms/TimeSyncConfigTests.java @@ -16,41 +16,34 @@ * specific language governing permissions and limitations * under the License. */ -package org.elasticsearch.action.admin.indices.exists.types; -import org.elasticsearch.action.ActionResponse; -import org.elasticsearch.common.io.stream.StreamInput; -import org.elasticsearch.common.io.stream.StreamOutput; +package org.elasticsearch.client.dataframe.transforms; -import java.io.IOException; +import org.elasticsearch.common.unit.TimeValue; +import org.elasticsearch.common.xcontent.XContentParser; +import org.elasticsearch.test.AbstractXContentTestCase; -/** - * Whether all of the existed types exist. - */ -public class TypesExistsResponse extends ActionResponse { +import java.io.IOException; - private boolean exists; +public class TimeSyncConfigTests extends AbstractXContentTestCase { - TypesExistsResponse() { + public static TimeSyncConfig randomTimeSyncConfig() { + return new TimeSyncConfig(randomAlphaOfLengthBetween(1, 10), new TimeValue(randomNonNegativeLong())); } - public TypesExistsResponse(boolean exists) { - this.exists = exists; - } - - public boolean isExists() { - return this.exists; + @Override + protected TimeSyncConfig createTestInstance() { + return randomTimeSyncConfig(); } @Override - public void readFrom(StreamInput in) throws IOException { - super.readFrom(in); - exists = in.readBoolean(); + protected TimeSyncConfig doParseInstance(XContentParser parser) throws IOException { + return TimeSyncConfig.fromXContent(parser); } @Override - public void writeTo(StreamOutput out) throws IOException { - super.writeTo(out); - out.writeBoolean(exists); + protected boolean supportsUnknownFields() { + return true; } -} \ No newline at end of file + +} diff --git a/client/rest-high-level/src/test/java/org/elasticsearch/client/dataframe/transforms/hlrc/DataFrameTransformStateAndStatsTests.java b/client/rest-high-level/src/test/java/org/elasticsearch/client/dataframe/transforms/hlrc/DataFrameTransformStateAndStatsTests.java index ad08881fb5641..dde44898bf90b 100644 --- a/client/rest-high-level/src/test/java/org/elasticsearch/client/dataframe/transforms/hlrc/DataFrameTransformStateAndStatsTests.java +++ b/client/rest-high-level/src/test/java/org/elasticsearch/client/dataframe/transforms/hlrc/DataFrameTransformStateAndStatsTests.java @@ -64,7 +64,7 @@ protected boolean supportsUnknownFields() { @Override protected Predicate getRandomFieldsExcludeFilter() { - return field -> field.equals("state.current_position"); + return field -> field.equals("state.current_position") || field.equals("state.node") || field.equals("state.node.attributes"); } } diff --git a/client/rest-high-level/src/test/java/org/elasticsearch/client/dataframe/transforms/hlrc/DataFrameTransformStateTests.java b/client/rest-high-level/src/test/java/org/elasticsearch/client/dataframe/transforms/hlrc/DataFrameTransformStateTests.java index 4c80365bc539a..b97e0a72c1fa2 100644 --- a/client/rest-high-level/src/test/java/org/elasticsearch/client/dataframe/transforms/hlrc/DataFrameTransformStateTests.java +++ b/client/rest-high-level/src/test/java/org/elasticsearch/client/dataframe/transforms/hlrc/DataFrameTransformStateTests.java @@ -28,6 +28,7 @@ import org.elasticsearch.xpack.core.dataframe.transforms.DataFrameTransformState; import org.elasticsearch.xpack.core.dataframe.transforms.DataFrameTransformStateAndStats; import org.elasticsearch.xpack.core.dataframe.transforms.DataFrameTransformTaskState; +import org.elasticsearch.xpack.core.dataframe.transforms.NodeAttributes; import org.elasticsearch.xpack.core.indexing.IndexerState; import java.io.IOException; @@ -40,8 +41,20 @@ public class DataFrameTransformStateTests extends AbstractHlrcXContentTestCase getRandomFieldsExcludeFilter() { - return field -> field.equals("current_position"); + return field -> field.equals("current_position") || field.equals("node.attributes"); } public static DataFrameTransformStateAndStats randomDataFrameTransformStateAndStats(String id) { @@ -97,6 +110,20 @@ public static DataFrameTransformProgress randomDataFrameTransformProgress() { return new DataFrameTransformProgress(totalDocs, remainingDocs); } + public static NodeAttributes randomNodeAttributes() { + int numberOfAttributes = randomIntBetween(1, 10); + Map attributes = new HashMap<>(numberOfAttributes); + for(int i = 0; i < numberOfAttributes; i++) { + String val = randomAlphaOfLength(10); + attributes.put("key-"+i, val); + } + return new NodeAttributes(randomAlphaOfLength(10), + randomAlphaOfLength(10), + randomAlphaOfLength(10), + randomAlphaOfLength(10), + attributes); + } + public static DataFrameIndexerTransformStats randomStats(String transformId) { return new DataFrameIndexerTransformStats(transformId, randomLongBetween(10L, 10000L), randomLongBetween(0L, 10000L), randomLongBetween(0L, 10000L), randomLongBetween(0L, 10000L), randomLongBetween(0L, 10000L), @@ -110,7 +137,8 @@ public static DataFrameTransformState randomDataFrameTransformState() { randomPosition(), randomLongBetween(0,10), randomBoolean() ? null : randomAlphaOfLength(10), - randomBoolean() ? null : randomDataFrameTransformProgress()); + randomBoolean() ? null : randomDataFrameTransformProgress(), + randomBoolean() ? null : randomNodeAttributes()); } private static Map randomPosition() { diff --git a/client/rest-high-level/src/test/java/org/elasticsearch/client/dataframe/transforms/hlrc/TimeSyncConfigTests.java b/client/rest-high-level/src/test/java/org/elasticsearch/client/dataframe/transforms/hlrc/TimeSyncConfigTests.java new file mode 100644 index 0000000000000..0c6a0350882a4 --- /dev/null +++ b/client/rest-high-level/src/test/java/org/elasticsearch/client/dataframe/transforms/hlrc/TimeSyncConfigTests.java @@ -0,0 +1,59 @@ +/* + * 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.client.dataframe.transforms.hlrc; + +import org.elasticsearch.client.AbstractResponseTestCase; +import org.elasticsearch.client.dataframe.transforms.TimeSyncConfig; +import org.elasticsearch.common.unit.TimeValue; +import org.elasticsearch.common.xcontent.XContentParser; + +import java.io.IOException; + +public class TimeSyncConfigTests + extends AbstractResponseTestCase { + + public static org.elasticsearch.xpack.core.dataframe.transforms.TimeSyncConfig randomTimeSyncConfig() { + return new org.elasticsearch.xpack.core.dataframe.transforms.TimeSyncConfig(randomAlphaOfLengthBetween(1, 10), + new TimeValue(randomNonNegativeLong())); + } + + public static void assertHlrcEquals(org.elasticsearch.xpack.core.dataframe.transforms.TimeSyncConfig serverTestInstance, + TimeSyncConfig clientInstance) { + assertEquals(serverTestInstance.getField(), clientInstance.getField()); + assertEquals(serverTestInstance.getDelay(), clientInstance.getDelay()); + } + + @Override + protected org.elasticsearch.xpack.core.dataframe.transforms.TimeSyncConfig createServerTestInstance() { + return randomTimeSyncConfig(); + } + + @Override + protected TimeSyncConfig doParseToClientInstance(XContentParser parser) throws IOException { + return TimeSyncConfig.fromXContent(parser); + } + + @Override + protected void assertInstances(org.elasticsearch.xpack.core.dataframe.transforms.TimeSyncConfig serverTestInstance, + TimeSyncConfig clientInstance) { + assertHlrcEquals(serverTestInstance, clientInstance); + } + +} diff --git a/client/rest-high-level/src/test/java/org/elasticsearch/client/dataframe/transforms/pivot/AggregationConfigTests.java b/client/rest-high-level/src/test/java/org/elasticsearch/client/dataframe/transforms/pivot/AggregationConfigTests.java index 210dc59329c02..2408f766b8af8 100644 --- a/client/rest-high-level/src/test/java/org/elasticsearch/client/dataframe/transforms/pivot/AggregationConfigTests.java +++ b/client/rest-high-level/src/test/java/org/elasticsearch/client/dataframe/transforms/pivot/AggregationConfigTests.java @@ -66,7 +66,7 @@ protected boolean supportsUnknownFields() { @Override protected NamedXContentRegistry xContentRegistry() { - SearchModule searchModule = new SearchModule(Settings.EMPTY, false, emptyList()); + SearchModule searchModule = new SearchModule(Settings.EMPTY, emptyList()); return new NamedXContentRegistry(searchModule.getNamedXContents()); } diff --git a/client/rest-high-level/src/test/java/org/elasticsearch/client/dataframe/transforms/pivot/DateHistogramGroupSourceTests.java b/client/rest-high-level/src/test/java/org/elasticsearch/client/dataframe/transforms/pivot/DateHistogramGroupSourceTests.java index 32605f5c286ad..ece1c4fb743ee 100644 --- a/client/rest-high-level/src/test/java/org/elasticsearch/client/dataframe/transforms/pivot/DateHistogramGroupSourceTests.java +++ b/client/rest-high-level/src/test/java/org/elasticsearch/client/dataframe/transforms/pivot/DateHistogramGroupSourceTests.java @@ -39,7 +39,6 @@ public static DateHistogramGroupSource randomDateHistogramGroupSource() { String field = randomAlphaOfLengthBetween(1, 20); return new DateHistogramGroupSource(field, randomDateHistogramInterval(), - randomBoolean() ? randomAlphaOfLength(10) : null, randomBoolean() ? randomZone() : null); } diff --git a/client/rest-high-level/src/test/java/org/elasticsearch/client/dataframe/transforms/pivot/PivotConfigTests.java b/client/rest-high-level/src/test/java/org/elasticsearch/client/dataframe/transforms/pivot/PivotConfigTests.java index 5cafcb9f419b5..a433000f62d1f 100644 --- a/client/rest-high-level/src/test/java/org/elasticsearch/client/dataframe/transforms/pivot/PivotConfigTests.java +++ b/client/rest-high-level/src/test/java/org/elasticsearch/client/dataframe/transforms/pivot/PivotConfigTests.java @@ -60,7 +60,7 @@ protected Predicate getRandomFieldsExcludeFilter() { @Override protected NamedXContentRegistry xContentRegistry() { - SearchModule searchModule = new SearchModule(Settings.EMPTY, false, Collections.emptyList()); + SearchModule searchModule = new SearchModule(Settings.EMPTY, Collections.emptyList()); return new NamedXContentRegistry(searchModule.getNamedXContents()); } } diff --git a/client/rest-high-level/src/test/java/org/elasticsearch/client/dataframe/transforms/pivot/hlrc/DateHistogramGroupSourceTests.java b/client/rest-high-level/src/test/java/org/elasticsearch/client/dataframe/transforms/pivot/hlrc/DateHistogramGroupSourceTests.java index dc31004607dcd..fd98e52a1527e 100644 --- a/client/rest-high-level/src/test/java/org/elasticsearch/client/dataframe/transforms/pivot/hlrc/DateHistogramGroupSourceTests.java +++ b/client/rest-high-level/src/test/java/org/elasticsearch/client/dataframe/transforms/pivot/hlrc/DateHistogramGroupSourceTests.java @@ -44,9 +44,6 @@ public static DateHistogramGroupSource randomDateHistogramGroupSource() { if (randomBoolean()) { dateHistogramGroupSource.setTimeZone(randomZone()); } - if (randomBoolean()) { - dateHistogramGroupSource.setFormat(randomAlphaOfLength(10)); - } return dateHistogramGroupSource; } @@ -64,7 +61,6 @@ protected org.elasticsearch.client.dataframe.transforms.pivot.DateHistogramGroup protected void assertInstances(DateHistogramGroupSource serverTestInstance, org.elasticsearch.client.dataframe.transforms.pivot.DateHistogramGroupSource clientInstance) { assertThat(serverTestInstance.getField(), equalTo(clientInstance.getField())); - assertThat(serverTestInstance.getFormat(), equalTo(clientInstance.getFormat())); assertSameInterval(serverTestInstance.getInterval(), clientInstance.getInterval()); assertThat(serverTestInstance.getTimeZone(), equalTo(clientInstance.getTimeZone())); assertThat(serverTestInstance.getType().name(), equalTo(clientInstance.getType().name())); diff --git a/client/rest-high-level/src/test/java/org/elasticsearch/client/documentation/CRUDDocumentationIT.java b/client/rest-high-level/src/test/java/org/elasticsearch/client/documentation/CRUDDocumentationIT.java index fe003d691a830..062f90c0eb4db 100644 --- a/client/rest-high-level/src/test/java/org/elasticsearch/client/documentation/CRUDDocumentationIT.java +++ b/client/rest-high-level/src/test/java/org/elasticsearch/client/documentation/CRUDDocumentationIT.java @@ -703,7 +703,7 @@ public void testBulk() throws Exception { request.add(new IndexRequest("posts").id("4") // <3> .source(XContentType.JSON,"field", "baz")); // end::bulk-request-with-mixed-operations - BulkResponse bulkResponse = client.bulk(request, RequestOptions.DEFAULT); + BulkResponse bulkResponse = client.bulk(request, RequestOptions.DEFAULT); assertSame(RestStatus.OK, bulkResponse.status()); assertFalse(bulkResponse.hasFailures()); @@ -824,9 +824,9 @@ public void testReindex() throws Exception { // tag::reindex-request-conflicts request.setConflicts("proceed"); // <1> // end::reindex-request-conflicts - // tag::reindex-request-size - request.setSize(10); // <1> - // end::reindex-request-size + // tag::reindex-request-maxDocs + request.setMaxDocs(10); // <1> + // end::reindex-request-maxDocs // tag::reindex-request-sourceSize request.setSourceBatchSize(100); // <1> // end::reindex-request-sourceSize @@ -1026,9 +1026,9 @@ public void testUpdateByQuery() throws Exception { // tag::update-by-query-request-query request.setQuery(new TermQueryBuilder("user", "kimchy")); // <1> // end::update-by-query-request-query - // tag::update-by-query-request-size - request.setSize(10); // <1> - // end::update-by-query-request-size + // tag::update-by-query-request-maxDocs + request.setMaxDocs(10); // <1> + // end::update-by-query-request-maxDocs // tag::update-by-query-request-scrollSize request.setBatchSize(100); // <1> // end::update-by-query-request-scrollSize @@ -1148,9 +1148,9 @@ public void testDeleteByQuery() throws Exception { // tag::delete-by-query-request-query request.setQuery(new TermQueryBuilder("user", "kimchy")); // <1> // end::delete-by-query-request-query - // tag::delete-by-query-request-size - request.setSize(10); // <1> - // end::delete-by-query-request-size + // tag::delete-by-query-request-maxDocs + request.setMaxDocs(10); // <1> + // end::delete-by-query-request-maxDocs // tag::delete-by-query-request-scrollSize request.setBatchSize(100); // <1> // end::delete-by-query-request-scrollSize @@ -1614,9 +1614,8 @@ public void testTermVectors() throws Exception { // tag::term-vectors-response String index = response.getIndex(); // <1> - String type = response.getType(); // <2> - String id = response.getId(); // <3> - boolean found = response.getFound(); // <4> + String id = response.getId(); // <2> + boolean found = response.getFound(); // <3> // end::term-vectors-response if (response.getTermVectorsList() != null) { diff --git a/client/rest-high-level/src/test/java/org/elasticsearch/client/documentation/DataFrameTransformDocumentationIT.java b/client/rest-high-level/src/test/java/org/elasticsearch/client/documentation/DataFrameTransformDocumentationIT.java index 6f7832cbf3cff..731d42f902c50 100644 --- a/client/rest-high-level/src/test/java/org/elasticsearch/client/documentation/DataFrameTransformDocumentationIT.java +++ b/client/rest-high-level/src/test/java/org/elasticsearch/client/documentation/DataFrameTransformDocumentationIT.java @@ -45,6 +45,7 @@ import org.elasticsearch.client.dataframe.transforms.DataFrameTransformStateAndStats; import org.elasticsearch.client.dataframe.transforms.DataFrameTransformTaskState; import org.elasticsearch.client.dataframe.transforms.DestConfig; +import org.elasticsearch.client.dataframe.transforms.NodeAttributes; import org.elasticsearch.client.dataframe.transforms.QueryConfig; import org.elasticsearch.client.dataframe.transforms.SourceConfig; import org.elasticsearch.client.dataframe.transforms.pivot.AggregationConfig; @@ -74,7 +75,7 @@ public class DataFrameTransformDocumentationIT extends ESRestHighLevelClientTest private List transformsToClean = new ArrayList<>(); @After - public void cleanUpTransforms() throws IOException { + public void cleanUpTransforms() throws Exception { for (String transformId : transformsToClean) { highLevelClient().dataFrame().stopDataFrameTransform( new StopDataFrameTransformRequest(transformId, Boolean.TRUE, TimeValue.timeValueSeconds(20)), RequestOptions.DEFAULT); @@ -86,6 +87,7 @@ public void cleanUpTransforms() throws IOException { } transformsToClean = new ArrayList<>(); + waitForPendingTasks(adminClient()); } private void createIndex(String indexName) throws IOException { @@ -124,6 +126,11 @@ public void testPutDataFrameTransform() throws IOException, InterruptedException .setIndex("source-index") .setQueryConfig(queryConfig).build(); // end::put-data-frame-transform-source-config + // tag::put-data-frame-transform-dest-config + DestConfig destConfig = DestConfig.builder() + .setIndex("pivot-destination") + .setPipeline("my-pipeline").build(); + // end::put-data-frame-transform-dest-config // tag::put-data-frame-transform-group-config GroupConfig groupConfig = GroupConfig.builder() .groupBy("reviewer", // <1> @@ -148,7 +155,7 @@ public void testPutDataFrameTransform() throws IOException, InterruptedException .builder() .setId("reviewer-avg-rating") // <1> .setSource(sourceConfig) // <2> - .setDest(new DestConfig("pivot-destination")) // <3> + .setDest(destConfig) // <3> .setPivotConfig(pivotConfig) // <4> .setDescription("This is my test transform") // <5> .build(); @@ -165,6 +172,7 @@ public void testPutDataFrameTransform() throws IOException, InterruptedException client.dataFrame().putDataFrameTransform( request, RequestOptions.DEFAULT); // end::put-data-frame-transform-execute + transformsToClean.add(request.getConfig().getId()); assertTrue(response.isAcknowledged()); } @@ -202,6 +210,7 @@ public void onFailure(Exception e) { // end::put-data-frame-transform-execute-async assertTrue(latch.await(30L, TimeUnit.SECONDS)); + transformsToClean.add(request.getConfig().getId()); } } @@ -221,7 +230,7 @@ public void testStartStop() throws IOException, InterruptedException { DataFrameTransformConfig transformConfig = DataFrameTransformConfig.builder() .setId("mega-transform") .setSource(SourceConfig.builder().setIndex("source-data").setQueryConfig(queryConfig).build()) - .setDest(new DestConfig("pivot-dest")) + .setDest(DestConfig.builder().setIndex("pivot-dest").build()) .setPivotConfig(pivotConfig) .build(); @@ -244,7 +253,7 @@ public void testStartStop() throws IOException, InterruptedException { request, RequestOptions.DEFAULT); // end::start-data-frame-transform-execute - assertTrue(response.isStarted()); + assertTrue(response.isAcknowledged()); } { // tag::stop-data-frame-transform-request @@ -255,6 +264,7 @@ public void testStartStop() throws IOException, InterruptedException { // tag::stop-data-frame-transform-request-options request.setWaitForCompletion(Boolean.TRUE); // <1> request.setTimeout(TimeValue.timeValueSeconds(30)); // <2> + request.setAllowNoMatch(true); // <3> // end::stop-data-frame-transform-request-options // tag::stop-data-frame-transform-execute @@ -263,7 +273,7 @@ public void testStartStop() throws IOException, InterruptedException { request, RequestOptions.DEFAULT); // end::stop-data-frame-transform-execute - assertTrue(response.isStopped()); + assertTrue(response.isAcknowledged()); } { // tag::start-data-frame-transform-execute-listener @@ -343,7 +353,7 @@ public void testDeleteDataFrameTransform() throws IOException, InterruptedExcept .setIndex("source-data") .setQuery(new MatchAllQueryBuilder()) .build()) - .setDest(new DestConfig("pivot-dest")) + .setDest(DestConfig.builder().setIndex("pivot-dest").build()) .setPivotConfig(pivotConfig) .build(); DataFrameTransformConfig transformConfig2 = DataFrameTransformConfig.builder() @@ -352,7 +362,7 @@ public void testDeleteDataFrameTransform() throws IOException, InterruptedExcept .setIndex("source-data") .setQuery(new MatchAllQueryBuilder()) .build()) - .setDest(new DestConfig("pivot-dest2")) + .setDest(DestConfig.builder().setIndex("pivot-dest2").build()) .setPivotConfig(pivotConfig) .build(); @@ -425,6 +435,7 @@ public void testPreview() throws IOException, InterruptedException { .setQueryConfig(queryConfig) .build(), // <1> pivotConfig); // <2> + PreviewDataFrameTransformRequest request = new PreviewDataFrameTransformRequest(transformConfig); // <3> // end::preview-data-frame-transform-request @@ -437,6 +448,7 @@ public void testPreview() throws IOException, InterruptedException { // end::preview-data-frame-transform-execute assertNotNull(response.getDocs()); + assertNotNull(response.getMappings()); } { // tag::preview-data-frame-transform-execute-listener @@ -472,7 +484,6 @@ public void testGetStats() throws IOException, InterruptedException { RestHighLevelClient client = highLevelClient(); - QueryConfig queryConfig = new QueryConfig(new MatchAllQueryBuilder()); GroupConfig groupConfig = GroupConfig.builder().groupBy("reviewer", TermsGroupSource.builder().setField("user_id").build()).build(); AggregatorFactories.Builder aggBuilder = new AggregatorFactories.Builder(); @@ -487,16 +498,22 @@ public void testGetStats() throws IOException, InterruptedException { .setIndex("source-data") .setQuery(new MatchAllQueryBuilder()) .build()) - .setDest(new DestConfig("pivot-dest")) + .setDest(DestConfig.builder().setIndex("pivot-dest").build()) .setPivotConfig(pivotConfig) .build(); client.dataFrame().putDataFrameTransform(new PutDataFrameTransformRequest(transformConfig), RequestOptions.DEFAULT); + transformsToClean.add(id); // tag::get-data-frame-transform-stats-request GetDataFrameTransformStatsRequest request = new GetDataFrameTransformStatsRequest(id); // <1> // end::get-data-frame-transform-stats-request + // tag::get-data-frame-transform-stats-request-options + request.setPageParams(new PageParams(0, 100)); // <1> + request.setAllowNoMatch(true); // <2> + // end::get-data-frame-transform-stats-request-options + { // tag::get-data-frame-transform-stats-execute GetDataFrameTransformStatsResponse response = @@ -517,6 +534,8 @@ public void testGetStats() throws IOException, InterruptedException { stateAndStats.getTransformStats(); // <4> DataFrameTransformProgress progress = stateAndStats.getTransformState().getProgress(); // <5> + NodeAttributes node = + stateAndStats.getTransformState().getNode(); // <6> // end::get-data-frame-transform-stats-response assertEquals(IndexerState.STOPPED, indexerState); @@ -558,7 +577,6 @@ public void onFailure(Exception e) { public void testGetDataFrameTransform() throws IOException, InterruptedException { createIndex("source-data"); - QueryConfig queryConfig = new QueryConfig(new MatchAllQueryBuilder()); GroupConfig groupConfig = GroupConfig.builder().groupBy("reviewer", TermsGroupSource.builder().setField("user_id").build()).build(); AggregatorFactories.Builder aggBuilder = new AggregatorFactories.Builder(); @@ -573,7 +591,7 @@ public void testGetDataFrameTransform() throws IOException, InterruptedException .setIndex("source-data") .setQuery(new MatchAllQueryBuilder()) .build()) - .setDest(new DestConfig("pivot-dest")) + .setDest(DestConfig.builder().setIndex("pivot-dest").build()) .setPivotConfig(pivotConfig) .build(); @@ -589,6 +607,7 @@ public void testGetDataFrameTransform() throws IOException, InterruptedException // tag::get-data-frame-transform-request-options request.setPageParams(new PageParams(0, 100)); // <1> + request.setAllowNoMatch(true); // <2> // end::get-data-frame-transform-request-options // tag::get-data-frame-transform-execute diff --git a/client/rest-high-level/src/test/java/org/elasticsearch/client/documentation/IndicesClientDocumentationIT.java b/client/rest-high-level/src/test/java/org/elasticsearch/client/documentation/IndicesClientDocumentationIT.java index 14def60b277e8..f878f0f6f7d88 100644 --- a/client/rest-high-level/src/test/java/org/elasticsearch/client/documentation/IndicesClientDocumentationIT.java +++ b/client/rest-high-level/src/test/java/org/elasticsearch/client/documentation/IndicesClientDocumentationIT.java @@ -26,9 +26,6 @@ import org.elasticsearch.action.admin.indices.alias.IndicesAliasesRequest; import org.elasticsearch.action.admin.indices.alias.IndicesAliasesRequest.AliasActions; import org.elasticsearch.action.admin.indices.alias.get.GetAliasesRequest; -import org.elasticsearch.action.admin.indices.analyze.AnalyzeRequest; -import org.elasticsearch.action.admin.indices.analyze.AnalyzeResponse; -import org.elasticsearch.action.admin.indices.analyze.DetailAnalyzeResponse; import org.elasticsearch.action.admin.indices.cache.clear.ClearIndicesCacheRequest; import org.elasticsearch.action.admin.indices.cache.clear.ClearIndicesCacheResponse; import org.elasticsearch.action.admin.indices.close.CloseIndexRequest; @@ -61,9 +58,13 @@ import org.elasticsearch.client.RequestOptions; import org.elasticsearch.client.RestHighLevelClient; import org.elasticsearch.client.SyncedFlushResponse; +import org.elasticsearch.client.core.BroadcastResponse.Shards; import org.elasticsearch.client.core.ShardsAcknowledgedResponse; +import org.elasticsearch.client.indices.AnalyzeRequest; +import org.elasticsearch.client.indices.AnalyzeResponse; import org.elasticsearch.client.indices.CreateIndexRequest; import org.elasticsearch.client.indices.CreateIndexResponse; +import org.elasticsearch.client.indices.DetailAnalyzeResponse; import org.elasticsearch.client.indices.FreezeIndexRequest; import org.elasticsearch.client.indices.GetFieldMappingsRequest; import org.elasticsearch.client.indices.GetFieldMappingsResponse; @@ -77,6 +78,9 @@ import org.elasticsearch.client.indices.IndexTemplatesExistRequest; import org.elasticsearch.client.indices.PutIndexTemplateRequest; import org.elasticsearch.client.indices.PutMappingRequest; +import org.elasticsearch.client.indices.ReloadAnalyzersRequest; +import org.elasticsearch.client.indices.ReloadAnalyzersResponse; +import org.elasticsearch.client.indices.ReloadAnalyzersResponse.ReloadDetails; import org.elasticsearch.client.indices.UnfreezeIndexRequest; import org.elasticsearch.client.indices.rollover.RolloverRequest; import org.elasticsearch.client.indices.rollover.RolloverResponse; @@ -2418,32 +2422,29 @@ public void testAnalyze() throws IOException, InterruptedException { { // tag::analyze-builtin-request - AnalyzeRequest request = new AnalyzeRequest(); - request.text("Some text to analyze", "Some more text to analyze"); // <1> - request.analyzer("english"); // <2> + AnalyzeRequest request = AnalyzeRequest.withGlobalAnalyzer("english", // <1> + "Some text to analyze", "Some more text to analyze"); // <2> // end::analyze-builtin-request } { // tag::analyze-custom-request - AnalyzeRequest request = new AnalyzeRequest(); - request.text("Some text to analyze"); - request.addCharFilter("html_strip"); // <1> - request.tokenizer("standard"); // <2> - request.addTokenFilter("lowercase"); // <3> - Map stopFilter = new HashMap<>(); stopFilter.put("type", "stop"); - stopFilter.put("stopwords", new String[]{ "to" }); // <4> - request.addTokenFilter(stopFilter); // <5> + stopFilter.put("stopwords", new String[]{ "to" }); // <1> + AnalyzeRequest request = AnalyzeRequest.buildCustomAnalyzer("standard") // <2> + .addCharFilter("html_strip") // <3> + .addTokenFilter("lowercase") // <4> + .addTokenFilter(stopFilter) // <5> + .build("Some text to analyze"); // end::analyze-custom-request } { // tag::analyze-custom-normalizer-request - AnalyzeRequest request = new AnalyzeRequest(); - request.text("BaR"); - request.addTokenFilter("lowercase"); + AnalyzeRequest request = AnalyzeRequest.buildCustomNormalizer() + .addTokenFilter("lowercase") + .build("BaR"); // end::analyze-custom-normalizer-request // tag::analyze-request-explain @@ -2484,10 +2485,11 @@ public void testAnalyze() throws IOException, InterruptedException { { // tag::analyze-index-request - AnalyzeRequest request = new AnalyzeRequest(); - request.index("my_index"); // <1> - request.analyzer("my_analyzer"); // <2> - request.text("some text to analyze"); + AnalyzeRequest request = AnalyzeRequest.withIndexAnalyzer( + "my_index", // <1> + "my_analyzer", // <2> + "some text to analyze" + ); // end::analyze-index-request // tag::analyze-execute-listener @@ -2505,10 +2507,7 @@ public void onFailure(Exception e) { // end::analyze-execute-listener // use a built-in analyzer in the test - request = new AnalyzeRequest(); - request.index("my_index"); - request.field("my_field"); - request.text("some text to analyze"); + request = AnalyzeRequest.withField("my_index", "my_field", "some text to analyze"); // Use a blocking listener in the test final CountDownLatch latch = new CountDownLatch(1); listener = new LatchedActionListener<>(listener, latch); @@ -2522,19 +2521,17 @@ public void onFailure(Exception e) { { // tag::analyze-index-normalizer-request - AnalyzeRequest request = new AnalyzeRequest(); - request.index("my_index"); // <1> - request.normalizer("my_normalizer"); // <2> - request.text("some text to analyze"); + AnalyzeRequest request = AnalyzeRequest.withNormalizer( + "my_index", // <1> + "my_normalizer", // <2> + "some text to analyze" + ); // end::analyze-index-normalizer-request } { // tag::analyze-field-request - AnalyzeRequest request = new AnalyzeRequest(); - request.index("my_index"); - request.field("my_field"); - request.text("some text to analyze"); + AnalyzeRequest request = AnalyzeRequest.withField("my_index", "my_field", "some text to analyze"); // end::analyze-field-request } @@ -2755,4 +2752,77 @@ public void onFailure(Exception e) { assertTrue(latch.await(30L, TimeUnit.SECONDS)); } + + public void testReloadSearchAnalyzers() throws Exception { + RestHighLevelClient client = highLevelClient(); + { + CreateIndexResponse createIndexResponse = client.indices().create(new CreateIndexRequest("index"), RequestOptions.DEFAULT); + assertTrue(createIndexResponse.isAcknowledged()); + } + + { + // tag::reload-analyzers-request + ReloadAnalyzersRequest request = new ReloadAnalyzersRequest("index"); // <1> + // end::reload-analyzers-request + + // tag::reload-analyzers-request-indicesOptions + request.setIndicesOptions(IndicesOptions.strictExpandOpen()); // <1> + // end::reload-analyzers-request-indicesOptions + + // tag::reload-analyzers-execute + ReloadAnalyzersResponse reloadResponse = client.indices().reloadAnalyzers(request, RequestOptions.DEFAULT); + // end::reload-analyzers-execute + + // tag::reload-analyzers-response + Shards shards = reloadResponse.shards(); // <1> + Map reloadDetails = reloadResponse.getReloadedDetails(); // <2> + ReloadDetails details = reloadDetails.get("index"); // <3> + String indexName = details.getIndexName(); // <4> + Set indicesNodes = details.getReloadedIndicesNodes(); // <5> + Set analyzers = details.getReloadedAnalyzers(); // <6> + // end::reload-analyzers-response + assertNotNull(shards); + assertEquals("index", indexName); + assertEquals(1, indicesNodes.size()); + assertEquals(0, analyzers.size()); + + // tag::reload-analyzers-execute-listener + ActionListener listener = + new ActionListener() { + @Override + public void onResponse(ReloadAnalyzersResponse reloadResponse) { + // <1> + } + + @Override + public void onFailure(Exception e) { + // <2> + } + }; + // end::reload-analyzers-execute-listener + + // Replace the empty listener by a blocking listener in test + final CountDownLatch latch = new CountDownLatch(1); + listener = new LatchedActionListener<>(listener, latch); + + // tag::reload-analyzers-execute-async + client.indices().reloadAnalyzersAsync(request, RequestOptions.DEFAULT, listener); // <1> + // end::reload-analyzers-execute-async + + assertTrue(latch.await(30L, TimeUnit.SECONDS)); + } + + { + // tag::reload-analyzers-notfound + try { + ReloadAnalyzersRequest request = new ReloadAnalyzersRequest("does_not_exist"); + client.indices().reloadAnalyzers(request, RequestOptions.DEFAULT); + } catch (ElasticsearchException exception) { + if (exception.status() == RestStatus.BAD_REQUEST) { + // <1> + } + } + // end::reload-analyzers-notfound + } + } } diff --git a/client/rest-high-level/src/test/java/org/elasticsearch/client/documentation/LicensingDocumentationIT.java b/client/rest-high-level/src/test/java/org/elasticsearch/client/documentation/LicensingDocumentationIT.java index aab21fb14dfe1..0d8fe0a5cd9a7 100644 --- a/client/rest-high-level/src/test/java/org/elasticsearch/client/documentation/LicensingDocumentationIT.java +++ b/client/rest-high-level/src/test/java/org/elasticsearch/client/documentation/LicensingDocumentationIT.java @@ -194,7 +194,7 @@ public void testGetLicense() throws Exception { //end::get-license-response assertThat(currentLicense, containsString("trial")); - assertThat(currentLicense, containsString("client_rest-high-level_integTestCluster")); + assertThat(currentLicense, containsString("integTest")); } { GetLicenseRequest request = new GetLicenseRequest(); @@ -233,7 +233,7 @@ public void onFailure(Exception e) { String currentLicense = response.getLicenseDefinition(); assertThat(currentLicense, startsWith("{")); assertThat(currentLicense, containsString("trial")); - assertThat(currentLicense, containsString("client_rest-high-level_integTestCluster")); + assertThat(currentLicense, containsString("integTest")); assertThat(currentLicense, endsWith("}")); } } diff --git a/client/rest-high-level/src/test/java/org/elasticsearch/client/documentation/MlClientDocumentationIT.java b/client/rest-high-level/src/test/java/org/elasticsearch/client/documentation/MlClientDocumentationIT.java index fe7d04a4e0a8d..93f196212ca0c 100644 --- a/client/rest-high-level/src/test/java/org/elasticsearch/client/documentation/MlClientDocumentationIT.java +++ b/client/rest-high-level/src/test/java/org/elasticsearch/client/documentation/MlClientDocumentationIT.java @@ -39,6 +39,7 @@ import org.elasticsearch.client.ml.DeleteCalendarEventRequest; import org.elasticsearch.client.ml.DeleteCalendarJobRequest; import org.elasticsearch.client.ml.DeleteCalendarRequest; +import org.elasticsearch.client.ml.DeleteDataFrameAnalyticsRequest; import org.elasticsearch.client.ml.DeleteDatafeedRequest; import org.elasticsearch.client.ml.DeleteExpiredDataRequest; import org.elasticsearch.client.ml.DeleteExpiredDataResponse; @@ -47,6 +48,8 @@ import org.elasticsearch.client.ml.DeleteJobRequest; import org.elasticsearch.client.ml.DeleteJobResponse; import org.elasticsearch.client.ml.DeleteModelSnapshotRequest; +import org.elasticsearch.client.ml.EvaluateDataFrameRequest; +import org.elasticsearch.client.ml.EvaluateDataFrameResponse; import org.elasticsearch.client.ml.FindFileStructureRequest; import org.elasticsearch.client.ml.FindFileStructureResponse; import org.elasticsearch.client.ml.FlushJobRequest; @@ -61,8 +64,10 @@ import org.elasticsearch.client.ml.GetCalendarsResponse; import org.elasticsearch.client.ml.GetCategoriesRequest; import org.elasticsearch.client.ml.GetCategoriesResponse; -import org.elasticsearch.client.ml.GetModelSnapshotsRequest; -import org.elasticsearch.client.ml.GetModelSnapshotsResponse; +import org.elasticsearch.client.ml.GetDataFrameAnalyticsRequest; +import org.elasticsearch.client.ml.GetDataFrameAnalyticsResponse; +import org.elasticsearch.client.ml.GetDataFrameAnalyticsStatsRequest; +import org.elasticsearch.client.ml.GetDataFrameAnalyticsStatsResponse; import org.elasticsearch.client.ml.GetDatafeedRequest; import org.elasticsearch.client.ml.GetDatafeedResponse; import org.elasticsearch.client.ml.GetDatafeedStatsRequest; @@ -75,6 +80,8 @@ import org.elasticsearch.client.ml.GetJobResponse; import org.elasticsearch.client.ml.GetJobStatsRequest; import org.elasticsearch.client.ml.GetJobStatsResponse; +import org.elasticsearch.client.ml.GetModelSnapshotsRequest; +import org.elasticsearch.client.ml.GetModelSnapshotsResponse; import org.elasticsearch.client.ml.GetOverallBucketsRequest; import org.elasticsearch.client.ml.GetOverallBucketsResponse; import org.elasticsearch.client.ml.GetRecordsRequest; @@ -92,6 +99,8 @@ import org.elasticsearch.client.ml.PutCalendarJobRequest; import org.elasticsearch.client.ml.PutCalendarRequest; import org.elasticsearch.client.ml.PutCalendarResponse; +import org.elasticsearch.client.ml.PutDataFrameAnalyticsRequest; +import org.elasticsearch.client.ml.PutDataFrameAnalyticsResponse; import org.elasticsearch.client.ml.PutDatafeedRequest; import org.elasticsearch.client.ml.PutDatafeedResponse; import org.elasticsearch.client.ml.PutFilterRequest; @@ -101,8 +110,11 @@ import org.elasticsearch.client.ml.RevertModelSnapshotRequest; import org.elasticsearch.client.ml.RevertModelSnapshotResponse; import org.elasticsearch.client.ml.SetUpgradeModeRequest; +import org.elasticsearch.client.ml.StartDataFrameAnalyticsRequest; import org.elasticsearch.client.ml.StartDatafeedRequest; import org.elasticsearch.client.ml.StartDatafeedResponse; +import org.elasticsearch.client.ml.StopDataFrameAnalyticsRequest; +import org.elasticsearch.client.ml.StopDataFrameAnalyticsResponse; import org.elasticsearch.client.ml.StopDatafeedRequest; import org.elasticsearch.client.ml.StopDatafeedResponse; import org.elasticsearch.client.ml.UpdateDatafeedRequest; @@ -118,6 +130,21 @@ import org.elasticsearch.client.ml.datafeed.DatafeedStats; import org.elasticsearch.client.ml.datafeed.DatafeedUpdate; import org.elasticsearch.client.ml.datafeed.DelayedDataCheckConfig; +import org.elasticsearch.client.ml.dataframe.DataFrameAnalysis; +import org.elasticsearch.client.ml.dataframe.DataFrameAnalyticsConfig; +import org.elasticsearch.client.ml.dataframe.DataFrameAnalyticsDest; +import org.elasticsearch.client.ml.dataframe.DataFrameAnalyticsSource; +import org.elasticsearch.client.ml.dataframe.DataFrameAnalyticsState; +import org.elasticsearch.client.ml.dataframe.DataFrameAnalyticsStats; +import org.elasticsearch.client.ml.dataframe.OutlierDetection; +import org.elasticsearch.client.ml.dataframe.QueryConfig; +import org.elasticsearch.client.ml.dataframe.evaluation.EvaluationMetric; +import org.elasticsearch.client.ml.dataframe.evaluation.softclassification.AucRocMetric; +import org.elasticsearch.client.ml.dataframe.evaluation.softclassification.BinarySoftClassification; +import org.elasticsearch.client.ml.dataframe.evaluation.softclassification.ConfusionMatrixMetric; +import org.elasticsearch.client.ml.dataframe.evaluation.softclassification.ConfusionMatrixMetric.ConfusionMatrix; +import org.elasticsearch.client.ml.dataframe.evaluation.softclassification.PrecisionMetric; +import org.elasticsearch.client.ml.dataframe.evaluation.softclassification.RecallMetric; import org.elasticsearch.client.ml.filestructurefinder.FileStructure; import org.elasticsearch.client.ml.job.config.AnalysisConfig; import org.elasticsearch.client.ml.job.config.AnalysisLimits; @@ -139,13 +166,18 @@ import org.elasticsearch.client.ml.job.results.OverallBucket; import org.elasticsearch.client.ml.job.stats.JobStats; import org.elasticsearch.common.bytes.BytesReference; +import org.elasticsearch.common.unit.ByteSizeUnit; +import org.elasticsearch.common.unit.ByteSizeValue; import org.elasticsearch.common.unit.TimeValue; import org.elasticsearch.common.xcontent.XContentFactory; import org.elasticsearch.common.xcontent.XContentType; +import org.elasticsearch.index.query.MatchAllQueryBuilder; import org.elasticsearch.index.query.QueryBuilders; import org.elasticsearch.search.aggregations.AggregatorFactories; import org.elasticsearch.search.builder.SearchSourceBuilder; +import org.elasticsearch.search.fetch.subphase.FetchSourceContext; import org.elasticsearch.tasks.TaskId; +import org.hamcrest.CoreMatchers; import org.junit.After; import java.io.IOException; @@ -870,18 +902,7 @@ public void testPreviewDatafeed() throws Exception { client.machineLearning().putJob(new PutJobRequest(job), RequestOptions.DEFAULT); String datafeedId = job.getId() + "-feed"; String indexName = "preview_data_2"; - CreateIndexRequest createIndexRequest = new CreateIndexRequest(indexName); - createIndexRequest.mapping(XContentFactory.jsonBuilder().startObject() - .startObject("properties") - .startObject("timestamp") - .field("type", "date") - .endObject() - .startObject("total") - .field("type", "long") - .endObject() - .endObject() - .endObject()); - highLevelClient().indices().create(createIndexRequest, RequestOptions.DEFAULT); + createIndex(indexName); DatafeedConfig datafeed = DatafeedConfig.builder(datafeedId, job.getId()) .setIndices(indexName) .build(); @@ -938,18 +959,7 @@ public void testStartDatafeed() throws Exception { client.machineLearning().putJob(new PutJobRequest(job), RequestOptions.DEFAULT); String datafeedId = job.getId() + "-feed"; String indexName = "start_data_2"; - CreateIndexRequest createIndexRequest = new CreateIndexRequest(indexName); - createIndexRequest.mapping(XContentFactory.jsonBuilder().startObject() - .startObject("properties") - .startObject("timestamp") - .field("type", "date") - .endObject() - .startObject("total") - .field("type", "long") - .endObject() - .endObject() - .endObject()); - highLevelClient().indices().create(createIndexRequest, RequestOptions.DEFAULT); + createIndex(indexName); DatafeedConfig datafeed = DatafeedConfig.builder(datafeedId, job.getId()) .setIndices(indexName) .build(); @@ -1067,18 +1077,7 @@ public void testGetDatafeedStats() throws Exception { client.machineLearning().putJob(new PutJobRequest(secondJob), RequestOptions.DEFAULT); String datafeedId1 = job.getId() + "-feed"; String indexName = "datafeed_stats_data_2"; - CreateIndexRequest createIndexRequest = new CreateIndexRequest(indexName); - createIndexRequest.mapping(XContentFactory.jsonBuilder().startObject() - .startObject("properties") - .startObject("timestamp") - .field("type", "date") - .endObject() - .startObject("total") - .field("type", "long") - .endObject() - .endObject() - .endObject()); - highLevelClient().indices().create(createIndexRequest, RequestOptions.DEFAULT); + createIndex(indexName); DatafeedConfig datafeed = DatafeedConfig.builder(datafeedId1, job.getId()) .setIndices(indexName) .build(); @@ -2802,6 +2801,466 @@ public void onFailure(Exception e) { } } + public void testGetDataFrameAnalytics() throws Exception { + createIndex(DF_ANALYTICS_CONFIG.getSource().getIndex()[0]); + + RestHighLevelClient client = highLevelClient(); + client.machineLearning().putDataFrameAnalytics(new PutDataFrameAnalyticsRequest(DF_ANALYTICS_CONFIG), RequestOptions.DEFAULT); + { + // tag::get-data-frame-analytics-request + GetDataFrameAnalyticsRequest request = new GetDataFrameAnalyticsRequest("my-analytics-config"); // <1> + // end::get-data-frame-analytics-request + + // tag::get-data-frame-analytics-execute + GetDataFrameAnalyticsResponse response = client.machineLearning().getDataFrameAnalytics(request, RequestOptions.DEFAULT); + // end::get-data-frame-analytics-execute + + // tag::get-data-frame-analytics-response + List configs = response.getAnalytics(); + // end::get-data-frame-analytics-response + + assertThat(configs.size(), equalTo(1)); + } + { + GetDataFrameAnalyticsRequest request = new GetDataFrameAnalyticsRequest("my-analytics-config"); + + // tag::get-data-frame-analytics-execute-listener + ActionListener listener = new ActionListener<>() { + @Override + public void onResponse(GetDataFrameAnalyticsResponse response) { + // <1> + } + + @Override + public void onFailure(Exception e) { + // <2> + } + }; + // end::get-data-frame-analytics-execute-listener + + // Replace the empty listener by a blocking listener in test + CountDownLatch latch = new CountDownLatch(1); + listener = new LatchedActionListener<>(listener, latch); + + // tag::get-data-frame-analytics-execute-async + client.machineLearning().getDataFrameAnalyticsAsync(request, RequestOptions.DEFAULT, listener); // <1> + // end::get-data-frame-analytics-execute-async + + assertTrue(latch.await(30L, TimeUnit.SECONDS)); + } + } + + public void testGetDataFrameAnalyticsStats() throws Exception { + createIndex(DF_ANALYTICS_CONFIG.getSource().getIndex()[0]); + + RestHighLevelClient client = highLevelClient(); + client.machineLearning().putDataFrameAnalytics(new PutDataFrameAnalyticsRequest(DF_ANALYTICS_CONFIG), RequestOptions.DEFAULT); + { + // tag::get-data-frame-analytics-stats-request + GetDataFrameAnalyticsStatsRequest request = new GetDataFrameAnalyticsStatsRequest("my-analytics-config"); // <1> + // end::get-data-frame-analytics-stats-request + + // tag::get-data-frame-analytics-stats-execute + GetDataFrameAnalyticsStatsResponse response = + client.machineLearning().getDataFrameAnalyticsStats(request, RequestOptions.DEFAULT); + // end::get-data-frame-analytics-stats-execute + + // tag::get-data-frame-analytics-stats-response + List stats = response.getAnalyticsStats(); + // end::get-data-frame-analytics-stats-response + + assertThat(stats.size(), equalTo(1)); + } + { + GetDataFrameAnalyticsStatsRequest request = new GetDataFrameAnalyticsStatsRequest("my-analytics-config"); + + // tag::get-data-frame-analytics-stats-execute-listener + ActionListener listener = new ActionListener<>() { + @Override + public void onResponse(GetDataFrameAnalyticsStatsResponse response) { + // <1> + } + + @Override + public void onFailure(Exception e) { + // <2> + } + }; + // end::get-data-frame-analytics-stats-execute-listener + + // Replace the empty listener by a blocking listener in test + CountDownLatch latch = new CountDownLatch(1); + listener = new LatchedActionListener<>(listener, latch); + + // tag::get-data-frame-analytics-stats-execute-async + client.machineLearning().getDataFrameAnalyticsStatsAsync(request, RequestOptions.DEFAULT, listener); // <1> + // end::get-data-frame-analytics-stats-execute-async + + assertTrue(latch.await(30L, TimeUnit.SECONDS)); + } + } + + public void testPutDataFrameAnalytics() throws Exception { + createIndex(DF_ANALYTICS_CONFIG.getSource().getIndex()[0]); + + RestHighLevelClient client = highLevelClient(); + { + // tag::put-data-frame-analytics-query-config + QueryConfig queryConfig = new QueryConfig(new MatchAllQueryBuilder()); + // end::put-data-frame-analytics-query-config + + // tag::put-data-frame-analytics-source-config + DataFrameAnalyticsSource sourceConfig = DataFrameAnalyticsSource.builder() // <1> + .setIndex("put-test-source-index") // <2> + .setQueryConfig(queryConfig) // <3> + .build(); + // end::put-data-frame-analytics-source-config + + // tag::put-data-frame-analytics-dest-config + DataFrameAnalyticsDest destConfig = DataFrameAnalyticsDest.builder() // <1> + .setIndex("put-test-dest-index") // <2> + .build(); + // end::put-data-frame-analytics-dest-config + + // tag::put-data-frame-analytics-analysis-default + DataFrameAnalysis outlierDetection = OutlierDetection.createDefault(); // <1> + // end::put-data-frame-analytics-analysis-default + + // tag::put-data-frame-analytics-analysis-customized + DataFrameAnalysis outlierDetectionCustomized = OutlierDetection.builder() // <1> + .setMethod(OutlierDetection.Method.DISTANCE_KNN) // <2> + .setNNeighbors(5) // <3> + .build(); + // end::put-data-frame-analytics-analysis-customized + + // tag::put-data-frame-analytics-analyzed-fields + FetchSourceContext analyzedFields = + new FetchSourceContext( + true, + new String[] { "included_field_1", "included_field_2" }, + new String[] { "excluded_field" }); + // end::put-data-frame-analytics-analyzed-fields + + // tag::put-data-frame-analytics-config + DataFrameAnalyticsConfig config = DataFrameAnalyticsConfig.builder("my-analytics-config") // <1> + .setSource(sourceConfig) // <2> + .setDest(destConfig) // <3> + .setAnalysis(outlierDetection) // <4> + .setAnalyzedFields(analyzedFields) // <5> + .setModelMemoryLimit(new ByteSizeValue(5, ByteSizeUnit.MB)) // <6> + .build(); + // end::put-data-frame-analytics-config + + // tag::put-data-frame-analytics-request + PutDataFrameAnalyticsRequest request = new PutDataFrameAnalyticsRequest(config); // <1> + // end::put-data-frame-analytics-request + + // tag::put-data-frame-analytics-execute + PutDataFrameAnalyticsResponse response = client.machineLearning().putDataFrameAnalytics(request, RequestOptions.DEFAULT); + // end::put-data-frame-analytics-execute + + // tag::put-data-frame-analytics-response + DataFrameAnalyticsConfig createdConfig = response.getConfig(); + // end::put-data-frame-analytics-response + + assertThat(createdConfig.getId(), equalTo("my-analytics-config")); + } + { + PutDataFrameAnalyticsRequest request = new PutDataFrameAnalyticsRequest(DF_ANALYTICS_CONFIG); + // tag::put-data-frame-analytics-execute-listener + ActionListener listener = new ActionListener<>() { + @Override + public void onResponse(PutDataFrameAnalyticsResponse response) { + // <1> + } + + @Override + public void onFailure(Exception e) { + // <2> + } + }; + // end::put-data-frame-analytics-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-data-frame-analytics-execute-async + client.machineLearning().putDataFrameAnalyticsAsync(request, RequestOptions.DEFAULT, listener); // <1> + // end::put-data-frame-analytics-execute-async + + assertTrue(latch.await(30L, TimeUnit.SECONDS)); + } + } + + public void testDeleteDataFrameAnalytics() throws Exception { + createIndex(DF_ANALYTICS_CONFIG.getSource().getIndex()[0]); + + RestHighLevelClient client = highLevelClient(); + client.machineLearning().putDataFrameAnalytics(new PutDataFrameAnalyticsRequest(DF_ANALYTICS_CONFIG), RequestOptions.DEFAULT); + { + // tag::delete-data-frame-analytics-request + DeleteDataFrameAnalyticsRequest request = new DeleteDataFrameAnalyticsRequest("my-analytics-config"); // <1> + // end::delete-data-frame-analytics-request + + // tag::delete-data-frame-analytics-execute + AcknowledgedResponse response = client.machineLearning().deleteDataFrameAnalytics(request, RequestOptions.DEFAULT); + // end::delete-data-frame-analytics-execute + + // tag::delete-data-frame-analytics-response + boolean acknowledged = response.isAcknowledged(); + // end::delete-data-frame-analytics-response + + assertThat(acknowledged, is(true)); + } + client.machineLearning().putDataFrameAnalytics(new PutDataFrameAnalyticsRequest(DF_ANALYTICS_CONFIG), RequestOptions.DEFAULT); + { + DeleteDataFrameAnalyticsRequest request = new DeleteDataFrameAnalyticsRequest("my-analytics-config"); + + // tag::delete-data-frame-analytics-execute-listener + ActionListener listener = new ActionListener<>() { + @Override + public void onResponse(AcknowledgedResponse response) { + // <1> + } + + @Override + public void onFailure(Exception e) { + // <2> + } + }; + // end::delete-data-frame-analytics-execute-listener + + // Replace the empty listener by a blocking listener in test + CountDownLatch latch = new CountDownLatch(1); + listener = new LatchedActionListener<>(listener, latch); + + // tag::delete-data-frame-analytics-execute-async + client.machineLearning().deleteDataFrameAnalyticsAsync(request, RequestOptions.DEFAULT, listener); // <1> + // end::delete-data-frame-analytics-execute-async + + assertTrue(latch.await(30L, TimeUnit.SECONDS)); + } + } + + public void testStartDataFrameAnalytics() throws Exception { + createIndex(DF_ANALYTICS_CONFIG.getSource().getIndex()[0]); + highLevelClient().index( + new IndexRequest(DF_ANALYTICS_CONFIG.getSource().getIndex()[0]).source(XContentType.JSON, "total", 10000) + .setRefreshPolicy(WriteRequest.RefreshPolicy.IMMEDIATE), RequestOptions.DEFAULT); + RestHighLevelClient client = highLevelClient(); + client.machineLearning().putDataFrameAnalytics(new PutDataFrameAnalyticsRequest(DF_ANALYTICS_CONFIG), RequestOptions.DEFAULT); + { + // tag::start-data-frame-analytics-request + StartDataFrameAnalyticsRequest request = new StartDataFrameAnalyticsRequest("my-analytics-config"); // <1> + // end::start-data-frame-analytics-request + + // tag::start-data-frame-analytics-execute + AcknowledgedResponse response = client.machineLearning().startDataFrameAnalytics(request, RequestOptions.DEFAULT); + // end::start-data-frame-analytics-execute + + // tag::start-data-frame-analytics-response + boolean acknowledged = response.isAcknowledged(); + // end::start-data-frame-analytics-response + + assertThat(acknowledged, is(true)); + } + assertBusy( + () -> assertThat(getAnalyticsState(DF_ANALYTICS_CONFIG.getId()), equalTo(DataFrameAnalyticsState.STOPPED)), + 30, TimeUnit.SECONDS); + { + StartDataFrameAnalyticsRequest request = new StartDataFrameAnalyticsRequest("my-analytics-config"); + + // tag::start-data-frame-analytics-execute-listener + ActionListener listener = new ActionListener<>() { + @Override + public void onResponse(AcknowledgedResponse response) { + // <1> + } + + @Override + public void onFailure(Exception e) { + // <2> + } + }; + // end::start-data-frame-analytics-execute-listener + + // Replace the empty listener by a blocking listener in test + CountDownLatch latch = new CountDownLatch(1); + listener = new LatchedActionListener<>(listener, latch); + + // tag::start-data-frame-analytics-execute-async + client.machineLearning().startDataFrameAnalyticsAsync(request, RequestOptions.DEFAULT, listener); // <1> + // end::start-data-frame-analytics-execute-async + + assertTrue(latch.await(30L, TimeUnit.SECONDS)); + } + assertBusy( + () -> assertThat(getAnalyticsState(DF_ANALYTICS_CONFIG.getId()), equalTo(DataFrameAnalyticsState.STOPPED)), + 30, TimeUnit.SECONDS); + } + + public void testStopDataFrameAnalytics() throws Exception { + createIndex(DF_ANALYTICS_CONFIG.getSource().getIndex()[0]); + highLevelClient().index( + new IndexRequest(DF_ANALYTICS_CONFIG.getSource().getIndex()[0]).source(XContentType.JSON, "total", 10000) + .setRefreshPolicy(WriteRequest.RefreshPolicy.IMMEDIATE), RequestOptions.DEFAULT); + RestHighLevelClient client = highLevelClient(); + client.machineLearning().putDataFrameAnalytics(new PutDataFrameAnalyticsRequest(DF_ANALYTICS_CONFIG), RequestOptions.DEFAULT); + { + // tag::stop-data-frame-analytics-request + StopDataFrameAnalyticsRequest request = new StopDataFrameAnalyticsRequest("my-analytics-config"); // <1> + request.setForce(false); // <2> + // end::stop-data-frame-analytics-request + + // tag::stop-data-frame-analytics-execute + StopDataFrameAnalyticsResponse response = client.machineLearning().stopDataFrameAnalytics(request, RequestOptions.DEFAULT); + // end::stop-data-frame-analytics-execute + + // tag::stop-data-frame-analytics-response + boolean acknowledged = response.isStopped(); + // end::stop-data-frame-analytics-response + + assertThat(acknowledged, is(true)); + } + assertBusy( + () -> assertThat(getAnalyticsState(DF_ANALYTICS_CONFIG.getId()), equalTo(DataFrameAnalyticsState.STOPPED)), + 30, TimeUnit.SECONDS); + { + StopDataFrameAnalyticsRequest request = new StopDataFrameAnalyticsRequest("my-analytics-config"); + + // tag::stop-data-frame-analytics-execute-listener + ActionListener listener = new ActionListener<>() { + @Override + public void onResponse(StopDataFrameAnalyticsResponse response) { + // <1> + } + + @Override + public void onFailure(Exception e) { + // <2> + } + }; + // end::stop-data-frame-analytics-execute-listener + + // Replace the empty listener by a blocking listener in test + CountDownLatch latch = new CountDownLatch(1); + listener = new LatchedActionListener<>(listener, latch); + + // tag::stop-data-frame-analytics-execute-async + client.machineLearning().stopDataFrameAnalyticsAsync(request, RequestOptions.DEFAULT, listener); // <1> + // end::stop-data-frame-analytics-execute-async + + assertTrue(latch.await(30L, TimeUnit.SECONDS)); + } + assertBusy( + () -> assertThat(getAnalyticsState(DF_ANALYTICS_CONFIG.getId()), equalTo(DataFrameAnalyticsState.STOPPED)), + 30, TimeUnit.SECONDS); + } + + public void testEvaluateDataFrame() throws Exception { + String indexName = "evaluate-test-index"; + CreateIndexRequest createIndexRequest = + new CreateIndexRequest(indexName) + .mapping(XContentFactory.jsonBuilder().startObject() + .startObject("properties") + .startObject("label") + .field("type", "keyword") + .endObject() + .startObject("p") + .field("type", "double") + .endObject() + .endObject() + .endObject()); + BulkRequest bulkRequest = + new BulkRequest(indexName) + .setRefreshPolicy(WriteRequest.RefreshPolicy.IMMEDIATE) + .add(new IndexRequest().source(XContentType.JSON, "label", false, "p", 0.1)) // #0 + .add(new IndexRequest().source(XContentType.JSON, "label", false, "p", 0.2)) // #1 + .add(new IndexRequest().source(XContentType.JSON, "label", false, "p", 0.3)) // #2 + .add(new IndexRequest().source(XContentType.JSON, "label", false, "p", 0.4)) // #3 + .add(new IndexRequest().source(XContentType.JSON, "label", false, "p", 0.7)) // #4 + .add(new IndexRequest().source(XContentType.JSON, "label", true, "p", 0.2)) // #5 + .add(new IndexRequest().source(XContentType.JSON, "label", true, "p", 0.3)) // #6 + .add(new IndexRequest().source(XContentType.JSON, "label", true, "p", 0.4)) // #7 + .add(new IndexRequest().source(XContentType.JSON, "label", true, "p", 0.8)) // #8 + .add(new IndexRequest().source(XContentType.JSON, "label", true, "p", 0.9)); // #9 + RestHighLevelClient client = highLevelClient(); + client.indices().create(createIndexRequest, RequestOptions.DEFAULT); + client.bulk(bulkRequest, RequestOptions.DEFAULT); + { + // tag::evaluate-data-frame-request + EvaluateDataFrameRequest request = new EvaluateDataFrameRequest( // <1> + indexName, // <2> + new BinarySoftClassification( // <3> + "label", // <4> + "p", // <5> + // Evaluation metrics // <6> + PrecisionMetric.at(0.4, 0.5, 0.6), // <7> + RecallMetric.at(0.5, 0.7), // <8> + ConfusionMatrixMetric.at(0.5), // <9> + AucRocMetric.withCurve())); // <10> + // end::evaluate-data-frame-request + + // tag::evaluate-data-frame-execute + EvaluateDataFrameResponse response = client.machineLearning().evaluateDataFrame(request, RequestOptions.DEFAULT); + // end::evaluate-data-frame-execute + + // tag::evaluate-data-frame-response + List metrics = response.getMetrics(); // <1> + + PrecisionMetric.Result precisionResult = response.getMetricByName(PrecisionMetric.NAME); // <2> + double precision = precisionResult.getScoreByThreshold("0.4"); // <3> + + ConfusionMatrixMetric.Result confusionMatrixResult = response.getMetricByName(ConfusionMatrixMetric.NAME); // <4> + ConfusionMatrix confusionMatrix = confusionMatrixResult.getScoreByThreshold("0.5"); // <5> + // end::evaluate-data-frame-response + + assertThat( + metrics.stream().map(m -> m.getMetricName()).collect(Collectors.toList()), + containsInAnyOrder(PrecisionMetric.NAME, RecallMetric.NAME, ConfusionMatrixMetric.NAME, AucRocMetric.NAME)); + assertThat(precision, closeTo(0.6, 1e-9)); + assertThat(confusionMatrix.getTruePositives(), CoreMatchers.equalTo(2L)); // docs #8 and #9 + assertThat(confusionMatrix.getFalsePositives(), CoreMatchers.equalTo(1L)); // doc #4 + assertThat(confusionMatrix.getTrueNegatives(), CoreMatchers.equalTo(4L)); // docs #0, #1, #2 and #3 + assertThat(confusionMatrix.getFalseNegatives(), CoreMatchers.equalTo(3L)); // docs #5, #6 and #7 + } + { + EvaluateDataFrameRequest request = new EvaluateDataFrameRequest( + indexName, + new BinarySoftClassification( + "label", + "p", + PrecisionMetric.at(0.4, 0.5, 0.6), + RecallMetric.at(0.5, 0.7), + ConfusionMatrixMetric.at(0.5), + AucRocMetric.withCurve())); + + // tag::evaluate-data-frame-execute-listener + ActionListener listener = new ActionListener<>() { + @Override + public void onResponse(EvaluateDataFrameResponse response) { + // <1> + } + + @Override + public void onFailure(Exception e) { + // <2> + } + }; + // end::evaluate-data-frame-execute-listener + + // Replace the empty listener by a blocking listener in test + CountDownLatch latch = new CountDownLatch(1); + listener = new LatchedActionListener<>(listener, latch); + + // tag::evaluate-data-frame-execute-async + client.machineLearning().evaluateDataFrameAsync(request, RequestOptions.DEFAULT, listener); // <1> + // end::evaluate-data-frame-execute-async + + assertTrue(latch.await(30L, TimeUnit.SECONDS)); + } + } public void testCreateFilter() throws Exception { RestHighLevelClient client = highLevelClient(); @@ -3140,4 +3599,39 @@ private String createFilter(RestHighLevelClient client) throws IOException { assertThat(createdFilter.getId(), equalTo("my_safe_domains")); return createdFilter.getId(); } + + private void createIndex(String indexName) throws IOException { + CreateIndexRequest createIndexRequest = new CreateIndexRequest(indexName); + createIndexRequest.mapping(XContentFactory.jsonBuilder().startObject() + .startObject("properties") + .startObject("timestamp") + .field("type", "date") + .endObject() + .startObject("total") + .field("type", "long") + .endObject() + .endObject() + .endObject()); + highLevelClient().indices().create(createIndexRequest, RequestOptions.DEFAULT); + } + + private DataFrameAnalyticsState getAnalyticsState(String configId) throws IOException { + GetDataFrameAnalyticsStatsResponse statsResponse = + highLevelClient().machineLearning().getDataFrameAnalyticsStats( + new GetDataFrameAnalyticsStatsRequest(configId), RequestOptions.DEFAULT); + assertThat(statsResponse.getAnalyticsStats(), hasSize(1)); + DataFrameAnalyticsStats stats = statsResponse.getAnalyticsStats().get(0); + return stats.getState(); + } + + private static final DataFrameAnalyticsConfig DF_ANALYTICS_CONFIG = + DataFrameAnalyticsConfig.builder("my-analytics-config") + .setSource(DataFrameAnalyticsSource.builder() + .setIndex("put-test-source-index") + .build()) + .setDest(DataFrameAnalyticsDest.builder() + .setIndex("put-test-dest-index") + .build()) + .setAnalysis(OutlierDetection.createDefault()) + .build(); } diff --git a/client/rest-high-level/src/test/java/org/elasticsearch/client/documentation/QueryDSLDocumentationTests.java b/client/rest-high-level/src/test/java/org/elasticsearch/client/documentation/QueryDSLDocumentationTests.java index 51670b29de1b6..3530e63e47e1d 100644 --- a/client/rest-high-level/src/test/java/org/elasticsearch/client/documentation/QueryDSLDocumentationTests.java +++ b/client/rest-high-level/src/test/java/org/elasticsearch/client/documentation/QueryDSLDocumentationTests.java @@ -42,7 +42,6 @@ import static java.util.Collections.singletonMap; import static org.elasticsearch.index.query.QueryBuilders.boolQuery; import static org.elasticsearch.index.query.QueryBuilders.boostingQuery; -import static org.elasticsearch.index.query.QueryBuilders.commonTermsQuery; import static org.elasticsearch.index.query.QueryBuilders.constantScoreQuery; import static org.elasticsearch.index.query.QueryBuilders.disMaxQuery; import static org.elasticsearch.index.query.QueryBuilders.existsQuery; @@ -106,13 +105,6 @@ public void testBoosting() { // end::boosting } - public void testCommonTerms() { - // tag::common_terms - commonTermsQuery("name", // <1> - "kimchy"); // <2> - // end::common_terms - } - public void testConstantScore() { // tag::constant_score constantScoreQuery( diff --git a/client/rest-high-level/src/test/java/org/elasticsearch/client/documentation/SecurityDocumentationIT.java b/client/rest-high-level/src/test/java/org/elasticsearch/client/documentation/SecurityDocumentationIT.java index f9a1c5c6571eb..9bbc3b2ea9072 100644 --- a/client/rest-high-level/src/test/java/org/elasticsearch/client/documentation/SecurityDocumentationIT.java +++ b/client/rest-high-level/src/test/java/org/elasticsearch/client/documentation/SecurityDocumentationIT.java @@ -50,6 +50,7 @@ import org.elasticsearch.client.security.ExpressionRoleMapping; import org.elasticsearch.client.security.GetApiKeyRequest; import org.elasticsearch.client.security.GetApiKeyResponse; +import org.elasticsearch.client.security.GetBuiltinPrivilegesResponse; import org.elasticsearch.client.security.GetPrivilegesRequest; import org.elasticsearch.client.security.GetPrivilegesResponse; import org.elasticsearch.client.security.GetRoleMappingsRequest; @@ -97,7 +98,6 @@ import javax.crypto.SecretKeyFactory; import javax.crypto.spec.PBEKeySpec; - import java.io.IOException; import java.time.Instant; import java.util.ArrayList; @@ -120,6 +120,7 @@ import static org.hamcrest.Matchers.emptyIterable; import static org.hamcrest.Matchers.equalTo; import static org.hamcrest.Matchers.greaterThan; +import static org.hamcrest.Matchers.hasItem; import static org.hamcrest.Matchers.is; import static org.hamcrest.Matchers.isIn; import static org.hamcrest.Matchers.iterableWithSize; @@ -1500,6 +1501,60 @@ public void onFailure(Exception e) { } } + public void testGetBuiltinPrivileges() throws Exception { + final RestHighLevelClient client = highLevelClient(); + { + //tag::get-builtin-privileges-execute + GetBuiltinPrivilegesResponse response = client.security().getBuiltinPrivileges(RequestOptions.DEFAULT); + //end::get-builtin-privileges-execute + + assertNotNull(response); + //tag::get-builtin-privileges-response + final Set cluster = response.getClusterPrivileges(); + final Set index = response.getIndexPrivileges(); + //end::get-builtin-privileges-response + + assertThat(cluster, hasItem("all")); + assertThat(cluster, hasItem("manage")); + assertThat(cluster, hasItem("monitor")); + assertThat(cluster, hasItem("manage_security")); + + assertThat(index, hasItem("all")); + assertThat(index, hasItem("manage")); + assertThat(index, hasItem("monitor")); + assertThat(index, hasItem("read")); + assertThat(index, hasItem("write")); + } + { + // tag::get-builtin-privileges-execute-listener + ActionListener listener = new ActionListener() { + @Override + public void onResponse(GetBuiltinPrivilegesResponse response) { + // <1> + } + + @Override + public void onFailure(Exception e) { + // <2> + } + }; + // end::get-builtin-privileges-execute-listener + + // Replace the empty listener by a blocking listener in test + final PlainActionFuture future = new PlainActionFuture<>(); + listener = future; + + // tag::get-builtin-privileges-execute-async + client.security().getBuiltinPrivilegesAsync(RequestOptions.DEFAULT, listener); // <1> + // end::get-builtin-privileges-execute-async + + final GetBuiltinPrivilegesResponse response = future.get(30, TimeUnit.SECONDS); + assertNotNull(response); + assertThat(response.getClusterPrivileges(), hasItem("manage_security")); + assertThat(response.getIndexPrivileges(), hasItem("read")); + } + } + public void testGetPrivileges() throws Exception { final RestHighLevelClient client = highLevelClient(); final ApplicationPrivilege readTestappPrivilege = @@ -1559,9 +1614,9 @@ public void testGetPrivileges() throws Exception { assertNotNull(response); assertThat(response.getPrivileges().size(), equalTo(3)); - final GetPrivilegesResponse exptectedResponse = + final GetPrivilegesResponse expectedResponse = new GetPrivilegesResponse(Arrays.asList(readTestappPrivilege, writeTestappPrivilege, allTestappPrivilege)); - assertThat(response, equalTo(exptectedResponse)); + assertThat(response, equalTo(expectedResponse)); //tag::get-privileges-response Set privileges = response.getPrivileges(); //end::get-privileges-response diff --git a/client/rest-high-level/src/test/java/org/elasticsearch/client/documentation/SnapshotClientDocumentationIT.java b/client/rest-high-level/src/test/java/org/elasticsearch/client/documentation/SnapshotClientDocumentationIT.java index d80c24be6618a..b8149c99a5473 100644 --- a/client/rest-high-level/src/test/java/org/elasticsearch/client/documentation/SnapshotClientDocumentationIT.java +++ b/client/rest-high-level/src/test/java/org/elasticsearch/client/documentation/SnapshotClientDocumentationIT.java @@ -452,7 +452,7 @@ public void testSnapshotVerifyRepository() throws IOException { List repositoryMetaDataResponse = response.getNodes(); // end::verify-repository-response assertThat(1, equalTo(repositoryMetaDataResponse.size())); - assertThat("node-0", equalTo(repositoryMetaDataResponse.get(0).getName())); + assertThat("integTest-0", equalTo(repositoryMetaDataResponse.get(0).getName())); } public void testSnapshotVerifyRepositoryAsync() throws InterruptedException { @@ -590,7 +590,7 @@ public void testSnapshotGetSnapshots() throws IOException { // end::get-snapshots-request // tag::get-snapshots-request-repositoryName - request.repository(repositoryName); // <1> + request.repositories(repositoryName); // <1> // end::get-snapshots-request-repositoryName // tag::get-snapshots-request-snapshots @@ -616,7 +616,7 @@ public void testSnapshotGetSnapshots() throws IOException { // end::get-snapshots-execute // tag::get-snapshots-response - List snapshotsInfos = response.getSnapshots(); + List snapshotsInfos = response.getSnapshots(repositoryName); SnapshotInfo snapshotInfo = snapshotsInfos.get(0); RestStatus restStatus = snapshotInfo.status(); // <1> SnapshotId snapshotId = snapshotInfo.snapshotId(); // <2> diff --git a/client/rest-high-level/src/test/java/org/elasticsearch/client/indices/AnalyzeGlobalRequestTests.java b/client/rest-high-level/src/test/java/org/elasticsearch/client/indices/AnalyzeGlobalRequestTests.java new file mode 100644 index 0000000000000..a18971d28fe2d --- /dev/null +++ b/client/rest-high-level/src/test/java/org/elasticsearch/client/indices/AnalyzeGlobalRequestTests.java @@ -0,0 +1,58 @@ +/* + * 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.client.indices; + +import org.elasticsearch.action.admin.indices.analyze.AnalyzeAction; +import org.elasticsearch.common.xcontent.XContentParser; + +import java.io.IOException; +import java.util.Map; + +public class AnalyzeGlobalRequestTests extends AnalyzeRequestTests { + + @Override + protected AnalyzeRequest createClientTestInstance() { + int option = random().nextInt(3); + switch (option) { + case 0: + return AnalyzeRequest.withGlobalAnalyzer("my_analyzer", "some text", "some more text"); + case 1: + return AnalyzeRequest.buildCustomAnalyzer("my_tokenizer") + .addCharFilter("my_char_filter") + .addCharFilter(Map.of("type", "html_strip")) + .addTokenFilter("my_token_filter") + .addTokenFilter(Map.of("type", "synonym")) + .build("some text", "some more text"); + case 2: + return AnalyzeRequest.buildCustomNormalizer() + .addCharFilter("my_char_filter") + .addCharFilter(Map.of("type", "html_strip")) + .addTokenFilter("my_token_filter") + .addTokenFilter(Map.of("type", "synonym")) + .build("some text", "some more text"); + } + throw new IllegalStateException("nextInt(3) has returned a value greater than 2"); + } + + @Override + protected AnalyzeAction.Request doParseToServerInstance(XContentParser parser) throws IOException { + return AnalyzeAction.Request.fromXContent(parser, null); + } +} diff --git a/client/rest-high-level/src/test/java/org/elasticsearch/client/indices/AnalyzeIndexRequestTests.java b/client/rest-high-level/src/test/java/org/elasticsearch/client/indices/AnalyzeIndexRequestTests.java new file mode 100644 index 0000000000000..7cf271e89ac48 --- /dev/null +++ b/client/rest-high-level/src/test/java/org/elasticsearch/client/indices/AnalyzeIndexRequestTests.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.client.indices; + +import org.elasticsearch.action.admin.indices.analyze.AnalyzeAction; +import org.elasticsearch.common.xcontent.XContentParser; + +import java.io.IOException; +import java.util.Map; + +public class AnalyzeIndexRequestTests extends AnalyzeRequestTests { + + @Override + protected AnalyzeRequest createClientTestInstance() { + int option = random().nextInt(5); + switch (option) { + case 0: + return AnalyzeRequest.withField("index", "field", "some text", "some more text"); + case 1: + return AnalyzeRequest.withIndexAnalyzer("index", "my_analyzer", "some text", "some more text"); + case 2: + return AnalyzeRequest.withNormalizer("index", "my_normalizer", "text", "more text"); + case 3: + return AnalyzeRequest.buildCustomAnalyzer("index", "my_tokenizer") + .addCharFilter("my_char_filter") + .addCharFilter(Map.of("type", "html_strip")) + .addTokenFilter("my_token_filter") + .addTokenFilter(Map.of("type", "synonym")) + .build("some text", "some more text"); + case 4: + return AnalyzeRequest.buildCustomNormalizer("index") + .addCharFilter("my_char_filter") + .addCharFilter(Map.of("type", "html_strip")) + .addTokenFilter("my_token_filter") + .addTokenFilter(Map.of("type", "synonym")) + .build("some text", "some more text"); + } + throw new IllegalStateException("nextInt(5) has returned a value greater than 4"); + } + + @Override + protected AnalyzeAction.Request doParseToServerInstance(XContentParser parser) throws IOException { + return AnalyzeAction.Request.fromXContent(parser, "index"); + } +} diff --git a/client/rest-high-level/src/test/java/org/elasticsearch/client/indices/AnalyzeRequestTests.java b/client/rest-high-level/src/test/java/org/elasticsearch/client/indices/AnalyzeRequestTests.java new file mode 100644 index 0000000000000..50a339fc8058a --- /dev/null +++ b/client/rest-high-level/src/test/java/org/elasticsearch/client/indices/AnalyzeRequestTests.java @@ -0,0 +1,54 @@ +/* + * 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.client.indices; + +import org.elasticsearch.action.admin.indices.analyze.AnalyzeAction; +import org.elasticsearch.client.AbstractRequestTestCase; + +public abstract class AnalyzeRequestTests extends AbstractRequestTestCase { + + @Override + protected void assertInstances(AnalyzeAction.Request serverInstance, AnalyzeRequest clientTestInstance) { + assertEquals(serverInstance.index(), clientTestInstance.index()); + assertArrayEquals(serverInstance.text(), clientTestInstance.text()); + assertEquals(serverInstance.analyzer(), clientTestInstance.analyzer()); + assertEquals(serverInstance.normalizer(), clientTestInstance.normalizer()); + assertEquals(serverInstance.charFilters().size(), clientTestInstance.charFilters().size()); + for (int i = 0; i < serverInstance.charFilters().size(); i++) { + assertEquals(serverInstance.charFilters().get(i).name, clientTestInstance.charFilters().get(i).name); + assertEquals(serverInstance.charFilters().get(i).definition, clientTestInstance.charFilters().get(i).definition); + } + assertEquals(serverInstance.tokenFilters().size(), clientTestInstance.tokenFilters().size()); + for (int i = 0; i < serverInstance.tokenFilters().size(); i++) { + assertEquals(serverInstance.tokenFilters().get(i).name, clientTestInstance.tokenFilters().get(i).name); + assertEquals(serverInstance.tokenFilters().get(i).definition, clientTestInstance.tokenFilters().get(i).definition); + } + if (serverInstance.tokenizer() != null) { + assertEquals(serverInstance.tokenizer().name, clientTestInstance.tokenizer().name); + assertEquals(serverInstance.tokenizer().definition, clientTestInstance.tokenizer().definition); + } + else { + assertNull(clientTestInstance.tokenizer()); + } + assertEquals(serverInstance.field(), clientTestInstance.field()); + assertEquals(serverInstance.explain(), clientTestInstance.explain()); + assertArrayEquals(serverInstance.attributes(), clientTestInstance.attributes()); + } +} diff --git a/client/rest-high-level/src/test/java/org/elasticsearch/client/indices/AnalyzeResponseTests.java b/client/rest-high-level/src/test/java/org/elasticsearch/client/indices/AnalyzeResponseTests.java new file mode 100644 index 0000000000000..e29fa88d7fe3e --- /dev/null +++ b/client/rest-high-level/src/test/java/org/elasticsearch/client/indices/AnalyzeResponseTests.java @@ -0,0 +1,174 @@ +/* + * 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.client.indices; + +import org.elasticsearch.action.admin.indices.analyze.AnalyzeAction; +import org.elasticsearch.client.AbstractResponseTestCase; +import org.elasticsearch.common.xcontent.XContentParser; + +import java.io.IOException; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.HashMap; +import java.util.List; +import java.util.Map; + +public class AnalyzeResponseTests extends AbstractResponseTestCase { + + @Override + protected AnalyzeAction.Response createServerTestInstance() { + int tokenCount = randomIntBetween(1, 30); + AnalyzeAction.AnalyzeToken[] tokens = new AnalyzeAction.AnalyzeToken[tokenCount]; + for (int i = 0; i < tokenCount; i++) { + tokens[i] = randomToken(); + } + if (randomBoolean()) { + AnalyzeAction.CharFilteredText[] charfilters = null; + AnalyzeAction.AnalyzeTokenList[] tokenfilters = null; + if (randomBoolean()) { + charfilters = new AnalyzeAction.CharFilteredText[]{ + new AnalyzeAction.CharFilteredText("my_charfilter", new String[]{"one two"}) + }; + } + if (randomBoolean()) { + tokenfilters = new AnalyzeAction.AnalyzeTokenList[]{ + new AnalyzeAction.AnalyzeTokenList("my_tokenfilter_1", tokens), + new AnalyzeAction.AnalyzeTokenList("my_tokenfilter_2", tokens) + }; + } + AnalyzeAction.DetailAnalyzeResponse dar = new AnalyzeAction.DetailAnalyzeResponse( + charfilters, + new AnalyzeAction.AnalyzeTokenList("my_tokenizer", tokens), + tokenfilters); + return new AnalyzeAction.Response(null, dar); + } + return new AnalyzeAction.Response(Arrays.asList(tokens), null); + } + + private AnalyzeAction.AnalyzeToken randomToken() { + String token = randomAlphaOfLengthBetween(1, 20); + int position = randomIntBetween(0, 1000); + int startOffset = randomIntBetween(0, 1000); + int endOffset = randomIntBetween(0, 1000); + int posLength = randomIntBetween(1, 5); + String type = randomAlphaOfLengthBetween(1, 20); + Map extras = new HashMap<>(); + if (randomBoolean()) { + int entryCount = randomInt(6); + for (int i = 0; i < entryCount; i++) { + switch (randomInt(6)) { + case 0: + case 1: + case 2: + case 3: + String key = randomAlphaOfLength(5); + String value = randomAlphaOfLength(10); + extras.put(key, value); + break; + case 4: + String objkey = randomAlphaOfLength(5); + Map obj = new HashMap<>(); + obj.put(randomAlphaOfLength(5), randomAlphaOfLength(10)); + extras.put(objkey, obj); + break; + case 5: + String listkey = randomAlphaOfLength(5); + List list = new ArrayList<>(); + list.add(randomAlphaOfLength(4)); + list.add(randomAlphaOfLength(6)); + extras.put(listkey, list); + break; + } + } + } + return new AnalyzeAction.AnalyzeToken(token, position, startOffset, endOffset, posLength, type, extras); + } + + @Override + protected AnalyzeResponse doParseToClientInstance(XContentParser parser) throws IOException { + return AnalyzeResponse.fromXContent(parser); + } + + @Override + protected void assertInstances(AnalyzeAction.Response serverTestInstance, AnalyzeResponse clientInstance) { + if (serverTestInstance.detail() != null) { + assertNotNull(clientInstance.detail()); + assertInstances(serverTestInstance.detail(), clientInstance.detail()); + } + else { + assertEquals(serverTestInstance.getTokens().size(), clientInstance.getTokens().size()); + for (int i = 0; i < serverTestInstance.getTokens().size(); i++) { + assertEqualTokens(serverTestInstance.getTokens().get(0), clientInstance.getTokens().get(0)); + } + } + } + + private static void assertEqualTokens(AnalyzeAction.AnalyzeToken serverToken, AnalyzeResponse.AnalyzeToken clientToken) { + assertEquals(serverToken.getTerm(), clientToken.getTerm()); + assertEquals(serverToken.getPosition(), clientToken.getPosition()); + assertEquals(serverToken.getPositionLength(), clientToken.getPositionLength()); + assertEquals(serverToken.getStartOffset(), clientToken.getStartOffset()); + assertEquals(serverToken.getEndOffset(), clientToken.getEndOffset()); + assertEquals(serverToken.getType(), clientToken.getType()); + assertEquals(serverToken.getAttributes(), clientToken.getAttributes()); + } + + private static void assertInstances(AnalyzeAction.DetailAnalyzeResponse serverResponse, DetailAnalyzeResponse clientResponse) { + assertInstances(serverResponse.analyzer(), clientResponse.analyzer()); + assertInstances(serverResponse.tokenizer(), clientResponse.tokenizer()); + if (serverResponse.tokenfilters() == null) { + assertNull(clientResponse.tokenfilters()); + } + else { + assertEquals(serverResponse.tokenfilters().length, clientResponse.tokenfilters().length); + for (int i = 0; i < serverResponse.tokenfilters().length; i++) { + assertInstances(serverResponse.tokenfilters()[i], clientResponse.tokenfilters()[i]); + } + } + if (serverResponse.charfilters() == null) { + assertNull(clientResponse.charfilters()); + } + else { + assertEquals(serverResponse.charfilters().length, clientResponse.charfilters().length); + for (int i = 0; i < serverResponse.charfilters().length; i++) { + assertInstances(serverResponse.charfilters()[i], clientResponse.charfilters()[i]); + } + } + } + + private static void assertInstances(AnalyzeAction.AnalyzeTokenList serverTokens, + DetailAnalyzeResponse.AnalyzeTokenList clientTokens) { + if (serverTokens == null) { + assertNull(clientTokens); + } + else { + assertEquals(serverTokens.getName(), clientTokens.getName()); + assertEquals(serverTokens.getTokens().length, clientTokens.getTokens().length); + for (int i = 0; i < serverTokens.getTokens().length; i++) { + assertEqualTokens(serverTokens.getTokens()[i], clientTokens.getTokens()[i]); + } + } + } + + private static void assertInstances(AnalyzeAction.CharFilteredText serverText, DetailAnalyzeResponse.CharFilteredText clientText) { + assertEquals(serverText.getName(), clientText.getName()); + assertArrayEquals(serverText.getTexts(), clientText.getTexts()); + } +} diff --git a/client/rest-high-level/src/test/java/org/elasticsearch/client/indices/ReloadAnalyzersResponseTests.java b/client/rest-high-level/src/test/java/org/elasticsearch/client/indices/ReloadAnalyzersResponseTests.java new file mode 100644 index 0000000000000..6719e10808e42 --- /dev/null +++ b/client/rest-high-level/src/test/java/org/elasticsearch/client/indices/ReloadAnalyzersResponseTests.java @@ -0,0 +1,111 @@ +/* + * 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.client.indices; + +import org.elasticsearch.action.support.DefaultShardOperationFailedException; +import org.elasticsearch.client.AbstractResponseTestCase; +import org.elasticsearch.common.xcontent.XContentParser; +import org.elasticsearch.index.seqno.RetentionLeaseNotFoundException; +import org.elasticsearch.xpack.core.action.ReloadAnalyzersResponse.ReloadDetails; + +import java.io.IOException; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.HashMap; +import java.util.HashSet; +import java.util.List; +import java.util.Map; +import java.util.Map.Entry; +import java.util.Set; + +import static org.hamcrest.Matchers.containsString; +import static org.hamcrest.Matchers.equalTo; +import static org.hamcrest.Matchers.hasSize; +import static org.hamcrest.Matchers.isIn; + +public class ReloadAnalyzersResponseTests + extends AbstractResponseTestCase { + + private String index; + private String id; + private Set shardIds; + + @Override + protected org.elasticsearch.xpack.core.action.ReloadAnalyzersResponse createServerTestInstance() { + index = randomAlphaOfLength(8); + id = randomAlphaOfLength(8); + final int total = randomIntBetween(1, 16); + final int successful = total - scaledRandomIntBetween(0, total); + final int failed = scaledRandomIntBetween(0, total - successful); + final List failures = new ArrayList<>(); + shardIds = new HashSet<>(); + for (int i = 0; i < failed; i++) { + final DefaultShardOperationFailedException failure = new DefaultShardOperationFailedException( + index, + randomValueOtherThanMany(shardIds::contains, () -> randomIntBetween(0, total - 1)), + new RetentionLeaseNotFoundException(id)); + failures.add(failure); + shardIds.add(failure.shardId()); + } + Map reloadedDetailsMap = new HashMap<>(); + int randomIndices = randomIntBetween(0, 5); + for (int i = 0; i < randomIndices; i++) { + String indexName = randomAlphaOfLengthBetween(5, 10); + Set randomNodeIds = new HashSet<>(Arrays.asList(generateRandomStringArray(5, 5, false, true))); + Set randomAnalyzers = new HashSet<>(Arrays.asList(generateRandomStringArray(5, 5, false, true))); + + ReloadDetails reloadedDetails = new ReloadDetails(indexName, randomNodeIds, randomAnalyzers); + reloadedDetailsMap.put(indexName, reloadedDetails); + } + return new org.elasticsearch.xpack.core.action.ReloadAnalyzersResponse(total, successful, failed, failures, reloadedDetailsMap); + } + + @Override + protected ReloadAnalyzersResponse doParseToClientInstance(XContentParser parser) throws IOException { + return ReloadAnalyzersResponse.fromXContent(parser); + } + + @Override + protected void assertInstances(org.elasticsearch.xpack.core.action.ReloadAnalyzersResponse serverTestInstance, + ReloadAnalyzersResponse clientInstance) { + assertThat(clientInstance.shards().total(), equalTo(serverTestInstance.getTotalShards())); + assertThat(clientInstance.shards().successful(), equalTo(serverTestInstance.getSuccessfulShards())); + assertThat(clientInstance.shards().skipped(), equalTo(0)); + assertThat(clientInstance.shards().failed(), equalTo(serverTestInstance.getFailedShards())); + assertThat(clientInstance.shards().failures(), hasSize(clientInstance.shards().failed() == 0 ? 0 : 1)); // failures are grouped + if (clientInstance.shards().failed() > 0) { + final DefaultShardOperationFailedException groupedFailure = clientInstance.shards().failures().iterator().next(); + assertThat(groupedFailure.index(), equalTo(index)); + assertThat(groupedFailure.shardId(), isIn(shardIds)); + assertThat(groupedFailure.reason(), containsString("reason=retention lease with ID [" + id + "] not found")); + } + Map serverDetails = serverTestInstance.getReloadDetails(); + assertThat(clientInstance.getReloadedDetails().size(), equalTo(serverDetails.size())); + for (Entry entry : clientInstance + .getReloadedDetails().entrySet()) { + String indexName = entry.getKey(); + assertTrue(serverDetails.keySet().contains(indexName)); + assertEquals(serverDetails.get(indexName).getIndexName(), entry.getValue().getIndexName()); + assertEquals(serverDetails.get(indexName).getReloadedAnalyzers(), entry.getValue().getReloadedAnalyzers()); + assertEquals(serverDetails.get(indexName).getReloadedIndicesNodes(), entry.getValue().getReloadedIndicesNodes()); + } + } + +} diff --git a/client/rest-high-level/src/test/java/org/elasticsearch/client/license/PutLicenseResponseTests.java b/client/rest-high-level/src/test/java/org/elasticsearch/client/license/PutLicenseResponseTests.java index efb7ed98b366f..2422403659199 100644 --- a/client/rest-high-level/src/test/java/org/elasticsearch/client/license/PutLicenseResponseTests.java +++ b/client/rest-high-level/src/test/java/org/elasticsearch/client/license/PutLicenseResponseTests.java @@ -18,8 +18,9 @@ */ package org.elasticsearch.client.license; +import org.elasticsearch.client.AbstractHlrcWriteableXContentTestCase; +import org.elasticsearch.common.io.stream.Writeable; import org.elasticsearch.common.xcontent.XContentParser; -import org.elasticsearch.client.AbstractHlrcStreamableXContentTestCase; import org.elasticsearch.protocol.xpack.license.LicensesStatus; import java.io.IOException; @@ -29,8 +30,8 @@ import java.util.function.Function; import java.util.function.Predicate; -public class PutLicenseResponseTests extends AbstractHlrcStreamableXContentTestCase< - org.elasticsearch.protocol.xpack.license.PutLicenseResponse, org.elasticsearch.client.license.PutLicenseResponse> { +public class PutLicenseResponseTests extends AbstractHlrcWriteableXContentTestCase< + org.elasticsearch.protocol.xpack.license.PutLicenseResponse, PutLicenseResponse> { @Override public org.elasticsearch.client.license.PutLicenseResponse doHlrcParseInstance(XContentParser parser) throws IOException { @@ -96,8 +97,8 @@ private static Map randomAckMessages() { } @Override - protected org.elasticsearch.protocol.xpack.license.PutLicenseResponse createBlankInstance() { - return new org.elasticsearch.protocol.xpack.license.PutLicenseResponse(); + protected Writeable.Reader instanceReader() { + return org.elasticsearch.protocol.xpack.license.PutLicenseResponse::new; } @Override diff --git a/client/rest-high-level/src/test/java/org/elasticsearch/client/license/StartBasicResponseTests.java b/client/rest-high-level/src/test/java/org/elasticsearch/client/license/StartBasicResponseTests.java index 69f831cff188e..a22b40d7e6e63 100644 --- a/client/rest-high-level/src/test/java/org/elasticsearch/client/license/StartBasicResponseTests.java +++ b/client/rest-high-level/src/test/java/org/elasticsearch/client/license/StartBasicResponseTests.java @@ -18,10 +18,11 @@ */ package org.elasticsearch.client.license; +import org.elasticsearch.client.AbstractHlrcWriteableXContentTestCase; import org.elasticsearch.common.Strings; +import org.elasticsearch.common.io.stream.Writeable; import org.elasticsearch.common.xcontent.XContentParser; import org.elasticsearch.license.PostStartBasicResponse; -import org.elasticsearch.client.AbstractHlrcStreamableXContentTestCase; import java.io.IOException; import java.util.Collections; @@ -29,8 +30,8 @@ import java.util.Map; import java.util.function.Predicate; -public class StartBasicResponseTests extends - AbstractHlrcStreamableXContentTestCase { +public class StartBasicResponseTests extends AbstractHlrcWriteableXContentTestCase< + PostStartBasicResponse, StartBasicResponse> { @Override public org.elasticsearch.client.license.StartBasicResponse doHlrcParseInstance(XContentParser parser) throws IOException { @@ -44,8 +45,8 @@ public PostStartBasicResponse convertHlrcToInternal(org.elasticsearch.client.lic } @Override - protected PostStartBasicResponse createBlankInstance() { - return new PostStartBasicResponse(); + protected Writeable.Reader instanceReader() { + return PostStartBasicResponse::new; } @Override diff --git a/server/src/main/java/org/elasticsearch/action/admin/indices/alias/exists/AliasesExistResponse.java b/client/rest-high-level/src/test/java/org/elasticsearch/client/ml/AucRocMetricAucRocPointTests.java similarity index 51% rename from server/src/main/java/org/elasticsearch/action/admin/indices/alias/exists/AliasesExistResponse.java rename to client/rest-high-level/src/test/java/org/elasticsearch/client/ml/AucRocMetricAucRocPointTests.java index 2d430bf9820d2..825adcd2060f8 100644 --- a/server/src/main/java/org/elasticsearch/action/admin/indices/alias/exists/AliasesExistResponse.java +++ b/client/rest-high-level/src/test/java/org/elasticsearch/client/ml/AucRocMetricAucRocPointTests.java @@ -16,43 +16,32 @@ * specific language governing permissions and limitations * under the License. */ +package org.elasticsearch.client.ml; -package org.elasticsearch.action.admin.indices.alias.exists; - -import org.elasticsearch.action.ActionResponse; -import org.elasticsearch.common.io.stream.StreamInput; -import org.elasticsearch.common.io.stream.StreamOutput; +import org.elasticsearch.client.ml.dataframe.evaluation.softclassification.AucRocMetric; +import org.elasticsearch.common.xcontent.XContentParser; +import org.elasticsearch.test.AbstractXContentTestCase; import java.io.IOException; -public class AliasesExistResponse extends ActionResponse { - - private boolean exists; - - public AliasesExistResponse(boolean exists) { - this.exists = exists; - } +public class AucRocMetricAucRocPointTests extends AbstractXContentTestCase { - AliasesExistResponse() { + static AucRocMetric.AucRocPoint randomPoint() { + return new AucRocMetric.AucRocPoint(randomDouble(), randomDouble(), randomDouble()); } - public boolean exists() { - return exists; - } - - public boolean isExists() { - return exists(); + @Override + protected AucRocMetric.AucRocPoint createTestInstance() { + return randomPoint(); } @Override - public void readFrom(StreamInput in) throws IOException { - super.readFrom(in); - exists = in.readBoolean(); + protected AucRocMetric.AucRocPoint doParseInstance(XContentParser parser) throws IOException { + return AucRocMetric.AucRocPoint.fromXContent(parser); } @Override - public void writeTo(StreamOutput out) throws IOException { - super.writeTo(out); - out.writeBoolean(exists); + protected boolean supportsUnknownFields() { + return true; } } diff --git a/client/rest-high-level/src/test/java/org/elasticsearch/client/ml/AucRocMetricResultTests.java b/client/rest-high-level/src/test/java/org/elasticsearch/client/ml/AucRocMetricResultTests.java new file mode 100644 index 0000000000000..9ea7689d60f32 --- /dev/null +++ b/client/rest-high-level/src/test/java/org/elasticsearch/client/ml/AucRocMetricResultTests.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.client.ml; + +import org.elasticsearch.client.ml.dataframe.evaluation.softclassification.AucRocMetric; +import org.elasticsearch.common.xcontent.XContentParser; +import org.elasticsearch.test.AbstractXContentTestCase; + +import java.io.IOException; +import java.util.function.Predicate; +import java.util.stream.Collectors; +import java.util.stream.Stream; + +import static org.elasticsearch.client.ml.AucRocMetricAucRocPointTests.randomPoint; + +public class AucRocMetricResultTests extends AbstractXContentTestCase { + + static AucRocMetric.Result randomResult() { + return new AucRocMetric.Result( + randomDouble(), + Stream + .generate(() -> randomPoint()) + .limit(randomIntBetween(1, 10)) + .collect(Collectors.toList())); + } + + @Override + protected AucRocMetric.Result createTestInstance() { + return randomResult(); + } + + @Override + protected AucRocMetric.Result doParseInstance(XContentParser parser) throws IOException { + return AucRocMetric.Result.fromXContent(parser); + } + + @Override + protected boolean supportsUnknownFields() { + return true; + } + + @Override + protected Predicate getRandomFieldsExcludeFilter() { + // allow unknown fields in the root of the object only + return field -> !field.isEmpty(); + } +} diff --git a/client/rest-high-level/src/test/java/org/elasticsearch/client/ml/ConfusionMatrixMetricConfusionMatrixTests.java b/client/rest-high-level/src/test/java/org/elasticsearch/client/ml/ConfusionMatrixMetricConfusionMatrixTests.java new file mode 100644 index 0000000000000..28eb221b318c6 --- /dev/null +++ b/client/rest-high-level/src/test/java/org/elasticsearch/client/ml/ConfusionMatrixMetricConfusionMatrixTests.java @@ -0,0 +1,47 @@ +/* + * 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.client.ml; + +import org.elasticsearch.client.ml.dataframe.evaluation.softclassification.ConfusionMatrixMetric; +import org.elasticsearch.common.xcontent.XContentParser; +import org.elasticsearch.test.AbstractXContentTestCase; + +import java.io.IOException; + +public class ConfusionMatrixMetricConfusionMatrixTests extends AbstractXContentTestCase { + + static ConfusionMatrixMetric.ConfusionMatrix randomConfusionMatrix() { + return new ConfusionMatrixMetric.ConfusionMatrix(randomInt(), randomInt(), randomInt(), randomInt()); + } + + @Override + protected ConfusionMatrixMetric.ConfusionMatrix createTestInstance() { + return randomConfusionMatrix(); + } + + @Override + protected ConfusionMatrixMetric.ConfusionMatrix doParseInstance(XContentParser parser) throws IOException { + return ConfusionMatrixMetric.ConfusionMatrix.fromXContent(parser); + } + + @Override + protected boolean supportsUnknownFields() { + return true; + } +} diff --git a/client/rest-high-level/src/test/java/org/elasticsearch/client/ml/ConfusionMatrixMetricResultTests.java b/client/rest-high-level/src/test/java/org/elasticsearch/client/ml/ConfusionMatrixMetricResultTests.java new file mode 100644 index 0000000000000..c4b299a96b536 --- /dev/null +++ b/client/rest-high-level/src/test/java/org/elasticsearch/client/ml/ConfusionMatrixMetricResultTests.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.client.ml; + +import org.elasticsearch.client.ml.dataframe.evaluation.softclassification.ConfusionMatrixMetric; +import org.elasticsearch.common.xcontent.XContentParser; +import org.elasticsearch.test.AbstractXContentTestCase; + +import java.io.IOException; +import java.util.function.Predicate; +import java.util.stream.Collectors; +import java.util.stream.Stream; + +import static org.elasticsearch.client.ml.ConfusionMatrixMetricConfusionMatrixTests.randomConfusionMatrix; + +public class ConfusionMatrixMetricResultTests extends AbstractXContentTestCase { + + static ConfusionMatrixMetric.Result randomResult() { + return new ConfusionMatrixMetric.Result( + Stream + .generate(() -> randomConfusionMatrix()) + .limit(randomIntBetween(1, 5)) + .collect(Collectors.toMap(v -> String.valueOf(randomDouble()), v -> v))); + } + + @Override + protected ConfusionMatrixMetric.Result createTestInstance() { + return randomResult(); + } + + @Override + protected ConfusionMatrixMetric.Result doParseInstance(XContentParser parser) throws IOException { + return ConfusionMatrixMetric.Result.fromXContent(parser); + } + + @Override + protected boolean supportsUnknownFields() { + return true; + } + + @Override + protected Predicate getRandomFieldsExcludeFilter() { + // disallow unknown fields in the root of the object as field names must be parsable as numbers + return field -> field.isEmpty(); + } +} diff --git a/client/rest-high-level/src/test/java/org/elasticsearch/client/ml/DeleteDataFrameAnalyticsRequestTests.java b/client/rest-high-level/src/test/java/org/elasticsearch/client/ml/DeleteDataFrameAnalyticsRequestTests.java new file mode 100644 index 0000000000000..bc2ca2d954e76 --- /dev/null +++ b/client/rest-high-level/src/test/java/org/elasticsearch/client/ml/DeleteDataFrameAnalyticsRequestTests.java @@ -0,0 +1,39 @@ +/* + * 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.client.ml; + +import org.elasticsearch.test.ESTestCase; + +import java.util.Optional; + +import static org.hamcrest.Matchers.containsString; + +public class DeleteDataFrameAnalyticsRequestTests extends ESTestCase { + + public void testValidate_Ok() { + assertEquals(Optional.empty(), new DeleteDataFrameAnalyticsRequest("valid-id").validate()); + assertEquals(Optional.empty(), new DeleteDataFrameAnalyticsRequest("").validate()); + } + + public void testValidate_Failure() { + assertThat(new DeleteDataFrameAnalyticsRequest(null).validate().get().getMessage(), + containsString("data frame analytics id must not be null")); + } +} diff --git a/client/rest-high-level/src/test/java/org/elasticsearch/client/ml/EvaluateDataFrameResponseTests.java b/client/rest-high-level/src/test/java/org/elasticsearch/client/ml/EvaluateDataFrameResponseTests.java new file mode 100644 index 0000000000000..b41d113686ccf --- /dev/null +++ b/client/rest-high-level/src/test/java/org/elasticsearch/client/ml/EvaluateDataFrameResponseTests.java @@ -0,0 +1,76 @@ +/* + * 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.client.ml; + +import org.elasticsearch.client.ml.dataframe.evaluation.EvaluationMetric; +import org.elasticsearch.client.ml.dataframe.evaluation.MlEvaluationNamedXContentProvider; +import org.elasticsearch.common.xcontent.NamedXContentRegistry; +import org.elasticsearch.common.xcontent.XContentParser; +import org.elasticsearch.test.AbstractXContentTestCase; + +import java.io.IOException; +import java.util.ArrayList; +import java.util.List; +import java.util.function.Predicate; + +public class EvaluateDataFrameResponseTests extends AbstractXContentTestCase { + + public static EvaluateDataFrameResponse randomResponse() { + List metrics = new ArrayList<>(); + if (randomBoolean()) { + metrics.add(AucRocMetricResultTests.randomResult()); + } + if (randomBoolean()) { + metrics.add(PrecisionMetricResultTests.randomResult()); + } + if (randomBoolean()) { + metrics.add(RecallMetricResultTests.randomResult()); + } + if (randomBoolean()) { + metrics.add(ConfusionMatrixMetricResultTests.randomResult()); + } + return new EvaluateDataFrameResponse(randomAlphaOfLength(5), metrics); + } + + @Override + protected EvaluateDataFrameResponse createTestInstance() { + return randomResponse(); + } + + @Override + protected EvaluateDataFrameResponse doParseInstance(XContentParser parser) throws IOException { + return EvaluateDataFrameResponse.fromXContent(parser); + } + + @Override + protected boolean supportsUnknownFields() { + return true; + } + + @Override + protected Predicate getRandomFieldsExcludeFilter() { + // allow unknown fields in the metrics map (i.e. alongside named metrics like "precision" or "recall") + return field -> field.isEmpty() || field.contains("."); + } + + @Override + protected NamedXContentRegistry xContentRegistry() { + return new NamedXContentRegistry(new MlEvaluationNamedXContentProvider().getNamedXContentParsers()); + } +} diff --git a/client/rest-high-level/src/test/java/org/elasticsearch/client/ml/FindFileStructureRequestTests.java b/client/rest-high-level/src/test/java/org/elasticsearch/client/ml/FindFileStructureRequestTests.java index 4cb8bf0a7c166..752d0593bef95 100644 --- a/client/rest-high-level/src/test/java/org/elasticsearch/client/ml/FindFileStructureRequestTests.java +++ b/client/rest-high-level/src/test/java/org/elasticsearch/client/ml/FindFileStructureRequestTests.java @@ -35,6 +35,7 @@ public class FindFileStructureRequestTests extends AbstractXContentTestCase p.setTimeout(TimeValue.parseTimeValue(c, FindFileStructureRequest.TIMEOUT.getPreferredName())), FindFileStructureRequest.TIMEOUT); PARSER.declareString(FindFileStructureRequest::setCharset, FindFileStructureRequest.CHARSET); @@ -72,6 +73,9 @@ public static FindFileStructureRequest createTestRequestWithoutSample() { if (randomBoolean()) { findFileStructureRequest.setLinesToSample(randomIntBetween(1000, 2000)); } + if (randomBoolean()) { + findFileStructureRequest.setLineMergeSizeLimit(randomIntBetween(10000, 20000)); + } if (randomBoolean()) { findFileStructureRequest.setTimeout(TimeValue.timeValueSeconds(randomIntBetween(10, 20))); } diff --git a/client/rest-high-level/src/test/java/org/elasticsearch/client/ml/GetDataFrameAnalyticsRequestTests.java b/client/rest-high-level/src/test/java/org/elasticsearch/client/ml/GetDataFrameAnalyticsRequestTests.java new file mode 100644 index 0000000000000..56d87ea6bef49 --- /dev/null +++ b/client/rest-high-level/src/test/java/org/elasticsearch/client/ml/GetDataFrameAnalyticsRequestTests.java @@ -0,0 +1,39 @@ +/* + * 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.client.ml; + +import org.elasticsearch.test.ESTestCase; + +import java.util.Optional; + +import static org.hamcrest.Matchers.containsString; + +public class GetDataFrameAnalyticsRequestTests extends ESTestCase { + + public void testValidate_Ok() { + assertEquals(Optional.empty(), new GetDataFrameAnalyticsRequest("valid-id").validate()); + assertEquals(Optional.empty(), new GetDataFrameAnalyticsRequest("").validate()); + } + + public void testValidate_Failure() { + assertThat(new GetDataFrameAnalyticsRequest(new String[0]).validate().get().getMessage(), + containsString("data frame analytics id must not be null")); + } +} diff --git a/client/rest-high-level/src/test/java/org/elasticsearch/client/ml/GetDataFrameAnalyticsStatsRequestTests.java b/client/rest-high-level/src/test/java/org/elasticsearch/client/ml/GetDataFrameAnalyticsStatsRequestTests.java new file mode 100644 index 0000000000000..4e08d99eaa659 --- /dev/null +++ b/client/rest-high-level/src/test/java/org/elasticsearch/client/ml/GetDataFrameAnalyticsStatsRequestTests.java @@ -0,0 +1,39 @@ +/* + * 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.client.ml; + +import org.elasticsearch.test.ESTestCase; + +import java.util.Optional; + +import static org.hamcrest.Matchers.containsString; + +public class GetDataFrameAnalyticsStatsRequestTests extends ESTestCase { + + public void testValidate_Ok() { + assertEquals(Optional.empty(), new GetDataFrameAnalyticsStatsRequest("valid-id").validate()); + assertEquals(Optional.empty(), new GetDataFrameAnalyticsStatsRequest("").validate()); + } + + public void testValidate_Failure() { + assertThat(new GetDataFrameAnalyticsStatsRequest(new String[0]).validate().get().getMessage(), + containsString("data frame analytics id must not be null")); + } +} diff --git a/client/rest-high-level/src/test/java/org/elasticsearch/client/ml/PrecisionMetricResultTests.java b/client/rest-high-level/src/test/java/org/elasticsearch/client/ml/PrecisionMetricResultTests.java new file mode 100644 index 0000000000000..607adacebb827 --- /dev/null +++ b/client/rest-high-level/src/test/java/org/elasticsearch/client/ml/PrecisionMetricResultTests.java @@ -0,0 +1,60 @@ +/* + * 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.client.ml; + +import org.elasticsearch.client.ml.dataframe.evaluation.softclassification.PrecisionMetric; +import org.elasticsearch.common.xcontent.XContentParser; +import org.elasticsearch.test.AbstractXContentTestCase; + +import java.io.IOException; +import java.util.function.Predicate; +import java.util.stream.Collectors; +import java.util.stream.Stream; + +public class PrecisionMetricResultTests extends AbstractXContentTestCase { + + static PrecisionMetric.Result randomResult() { + return new PrecisionMetric.Result( + Stream + .generate(() -> randomDouble()) + .limit(randomIntBetween(1, 5)) + .collect(Collectors.toMap(v -> String.valueOf(randomDouble()), v -> v))); + } + + @Override + protected PrecisionMetric.Result createTestInstance() { + return randomResult(); + } + + @Override + protected PrecisionMetric.Result doParseInstance(XContentParser parser) throws IOException { + return PrecisionMetric.Result.fromXContent(parser); + } + + @Override + protected boolean supportsUnknownFields() { + return true; + } + + @Override + protected Predicate getRandomFieldsExcludeFilter() { + // disallow unknown fields in the root of the object as field names must be parsable as numbers + return field -> field.isEmpty(); + } +} diff --git a/client/rest-high-level/src/test/java/org/elasticsearch/client/ml/PutDataFrameAnalyticsRequestTests.java b/client/rest-high-level/src/test/java/org/elasticsearch/client/ml/PutDataFrameAnalyticsRequestTests.java new file mode 100644 index 0000000000000..7387ba8ddeb65 --- /dev/null +++ b/client/rest-high-level/src/test/java/org/elasticsearch/client/ml/PutDataFrameAnalyticsRequestTests.java @@ -0,0 +1,74 @@ +/* + * 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.client.ml; + +import org.elasticsearch.client.ValidationException; +import org.elasticsearch.client.ml.dataframe.DataFrameAnalyticsConfig; +import org.elasticsearch.client.ml.dataframe.DataFrameAnalyticsConfigTests; +import org.elasticsearch.client.ml.dataframe.MlDataFrameAnalysisNamedXContentProvider; +import org.elasticsearch.common.settings.Settings; +import org.elasticsearch.common.xcontent.NamedXContentRegistry; +import org.elasticsearch.common.xcontent.XContentParser; +import org.elasticsearch.search.SearchModule; +import org.elasticsearch.test.AbstractXContentTestCase; + +import java.io.IOException; +import java.util.ArrayList; +import java.util.Collections; +import java.util.List; +import java.util.Optional; + +import static org.hamcrest.Matchers.containsString; + +public class PutDataFrameAnalyticsRequestTests extends AbstractXContentTestCase { + + public void testValidate_Ok() { + assertFalse(createTestInstance().validate().isPresent()); + } + + public void testValidate_Failure() { + Optional exception = new PutDataFrameAnalyticsRequest(null).validate(); + assertTrue(exception.isPresent()); + assertThat(exception.get().getMessage(), containsString("put requires a non-null data frame analytics config")); + } + + @Override + protected PutDataFrameAnalyticsRequest createTestInstance() { + return new PutDataFrameAnalyticsRequest(DataFrameAnalyticsConfigTests.randomDataFrameAnalyticsConfig()); + } + + @Override + protected PutDataFrameAnalyticsRequest doParseInstance(XContentParser parser) throws IOException { + return new PutDataFrameAnalyticsRequest(DataFrameAnalyticsConfig.fromXContent(parser)); + } + + @Override + protected boolean supportsUnknownFields() { + return false; + } + + @Override + protected NamedXContentRegistry xContentRegistry() { + List namedXContent = new ArrayList<>(); + namedXContent.addAll(new SearchModule(Settings.EMPTY, Collections.emptyList()).getNamedXContents()); + namedXContent.addAll(new MlDataFrameAnalysisNamedXContentProvider().getNamedXContentParsers()); + return new NamedXContentRegistry(namedXContent); + } +} diff --git a/client/rest-high-level/src/test/java/org/elasticsearch/client/ml/RecallMetricResultTests.java b/client/rest-high-level/src/test/java/org/elasticsearch/client/ml/RecallMetricResultTests.java new file mode 100644 index 0000000000000..138875007e30d --- /dev/null +++ b/client/rest-high-level/src/test/java/org/elasticsearch/client/ml/RecallMetricResultTests.java @@ -0,0 +1,60 @@ +/* + * 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.client.ml; + +import org.elasticsearch.client.ml.dataframe.evaluation.softclassification.RecallMetric; +import org.elasticsearch.common.xcontent.XContentParser; +import org.elasticsearch.test.AbstractXContentTestCase; + +import java.io.IOException; +import java.util.function.Predicate; +import java.util.stream.Collectors; +import java.util.stream.Stream; + +public class RecallMetricResultTests extends AbstractXContentTestCase { + + static RecallMetric.Result randomResult() { + return new RecallMetric.Result( + Stream + .generate(() -> randomDouble()) + .limit(randomIntBetween(1, 5)) + .collect(Collectors.toMap(v -> String.valueOf(randomDouble()), v -> v))); + } + + @Override + protected RecallMetric.Result createTestInstance() { + return randomResult(); + } + + @Override + protected RecallMetric.Result doParseInstance(XContentParser parser) throws IOException { + return RecallMetric.Result.fromXContent(parser); + } + + @Override + protected boolean supportsUnknownFields() { + return true; + } + + @Override + protected Predicate getRandomFieldsExcludeFilter() { + // disallow unknown fields in the root of the object as field names must be parsable as numbers + return field -> field.isEmpty(); + } +} diff --git a/client/rest-high-level/src/test/java/org/elasticsearch/client/ml/StartDataFrameAnalyticsRequestTests.java b/client/rest-high-level/src/test/java/org/elasticsearch/client/ml/StartDataFrameAnalyticsRequestTests.java new file mode 100644 index 0000000000000..6e43b50bcd12b --- /dev/null +++ b/client/rest-high-level/src/test/java/org/elasticsearch/client/ml/StartDataFrameAnalyticsRequestTests.java @@ -0,0 +1,43 @@ +/* + * 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.client.ml; + +import org.elasticsearch.common.unit.TimeValue; +import org.elasticsearch.test.ESTestCase; + +import java.util.Optional; + +import static org.hamcrest.Matchers.containsString; + +public class StartDataFrameAnalyticsRequestTests extends ESTestCase { + + public void testValidate_Ok() { + assertEquals(Optional.empty(), new StartDataFrameAnalyticsRequest("foo").validate()); + assertEquals(Optional.empty(), new StartDataFrameAnalyticsRequest("foo").setTimeout(null).validate()); + assertEquals(Optional.empty(), new StartDataFrameAnalyticsRequest("foo").setTimeout(TimeValue.ZERO).validate()); + } + + public void testValidate_Failure() { + assertThat(new StartDataFrameAnalyticsRequest(null).validate().get().getMessage(), + containsString("data frame analytics id must not be null")); + assertThat(new StartDataFrameAnalyticsRequest(null).setTimeout(TimeValue.ZERO).validate().get().getMessage(), + containsString("data frame analytics id must not be null")); + } +} diff --git a/client/rest-high-level/src/test/java/org/elasticsearch/client/ml/StopDataFrameAnalyticsRequestTests.java b/client/rest-high-level/src/test/java/org/elasticsearch/client/ml/StopDataFrameAnalyticsRequestTests.java new file mode 100644 index 0000000000000..57af2083743ae --- /dev/null +++ b/client/rest-high-level/src/test/java/org/elasticsearch/client/ml/StopDataFrameAnalyticsRequestTests.java @@ -0,0 +1,43 @@ +/* + * 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.client.ml; + +import org.elasticsearch.common.unit.TimeValue; +import org.elasticsearch.test.ESTestCase; + +import java.util.Optional; + +import static org.hamcrest.Matchers.containsString; + +public class StopDataFrameAnalyticsRequestTests extends ESTestCase { + + public void testValidate_Ok() { + assertEquals(Optional.empty(), new StopDataFrameAnalyticsRequest("foo").validate()); + assertEquals(Optional.empty(), new StopDataFrameAnalyticsRequest("foo").setTimeout(null).validate()); + assertEquals(Optional.empty(), new StopDataFrameAnalyticsRequest("foo").setTimeout(TimeValue.ZERO).validate()); + } + + public void testValidate_Failure() { + assertThat(new StopDataFrameAnalyticsRequest(null).validate().get().getMessage(), + containsString("data frame analytics id must not be null")); + assertThat(new StopDataFrameAnalyticsRequest(null).setTimeout(TimeValue.ZERO).validate().get().getMessage(), + containsString("data frame analytics id must not be null")); + } +} diff --git a/server/src/main/java/org/elasticsearch/action/admin/indices/exists/indices/IndicesExistsResponse.java b/client/rest-high-level/src/test/java/org/elasticsearch/client/ml/StopDataFrameAnalyticsResponseTests.java similarity index 53% rename from server/src/main/java/org/elasticsearch/action/admin/indices/exists/indices/IndicesExistsResponse.java rename to client/rest-high-level/src/test/java/org/elasticsearch/client/ml/StopDataFrameAnalyticsResponseTests.java index fb2d5c0fa9b9c..55ef1aed7534a 100644 --- a/server/src/main/java/org/elasticsearch/action/admin/indices/exists/indices/IndicesExistsResponse.java +++ b/client/rest-high-level/src/test/java/org/elasticsearch/client/ml/StopDataFrameAnalyticsResponseTests.java @@ -16,39 +16,27 @@ * specific language governing permissions and limitations * under the License. */ +package org.elasticsearch.client.ml; -package org.elasticsearch.action.admin.indices.exists.indices; - -import org.elasticsearch.action.ActionResponse; -import org.elasticsearch.common.io.stream.StreamInput; -import org.elasticsearch.common.io.stream.StreamOutput; +import org.elasticsearch.common.xcontent.XContentParser; +import org.elasticsearch.test.AbstractXContentTestCase; import java.io.IOException; -public class IndicesExistsResponse extends ActionResponse { - - private boolean exists; - - IndicesExistsResponse() { - } +public class StopDataFrameAnalyticsResponseTests extends AbstractXContentTestCase { - public IndicesExistsResponse(boolean exists) { - this.exists = exists; - } - - public boolean isExists() { - return this.exists; + @Override + protected StopDataFrameAnalyticsResponse createTestInstance() { + return new StopDataFrameAnalyticsResponse(randomBoolean()); } @Override - public void readFrom(StreamInput in) throws IOException { - super.readFrom(in); - exists = in.readBoolean(); + protected StopDataFrameAnalyticsResponse doParseInstance(XContentParser parser) throws IOException { + return StopDataFrameAnalyticsResponse.fromXContent(parser); } @Override - public void writeTo(StreamOutput out) throws IOException { - super.writeTo(out); - out.writeBoolean(exists); + protected boolean supportsUnknownFields() { + return true; } } diff --git a/client/rest-high-level/src/test/java/org/elasticsearch/client/ml/dataframe/DataFrameAnalyticsConfigTests.java b/client/rest-high-level/src/test/java/org/elasticsearch/client/ml/dataframe/DataFrameAnalyticsConfigTests.java new file mode 100644 index 0000000000000..957afc69dd0c2 --- /dev/null +++ b/client/rest-high-level/src/test/java/org/elasticsearch/client/ml/dataframe/DataFrameAnalyticsConfigTests.java @@ -0,0 +1,96 @@ +/* + * 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.client.ml.dataframe; + +import org.elasticsearch.Version; +import org.elasticsearch.common.settings.Settings; +import org.elasticsearch.common.unit.ByteSizeUnit; +import org.elasticsearch.common.unit.ByteSizeValue; +import org.elasticsearch.common.xcontent.NamedXContentRegistry; +import org.elasticsearch.common.xcontent.XContentParser; +import org.elasticsearch.search.SearchModule; +import org.elasticsearch.search.fetch.subphase.FetchSourceContext; +import org.elasticsearch.test.AbstractXContentTestCase; + +import java.io.IOException; +import java.time.Instant; +import java.util.ArrayList; +import java.util.Collections; +import java.util.List; +import java.util.function.Predicate; + +import static org.elasticsearch.client.ml.dataframe.DataFrameAnalyticsSourceTests.randomSourceConfig; +import static org.elasticsearch.client.ml.dataframe.DataFrameAnalyticsDestTests.randomDestConfig; +import static org.elasticsearch.client.ml.dataframe.OutlierDetectionTests.randomOutlierDetection; + +public class DataFrameAnalyticsConfigTests extends AbstractXContentTestCase { + + public static DataFrameAnalyticsConfig randomDataFrameAnalyticsConfig() { + DataFrameAnalyticsConfig.Builder builder = + DataFrameAnalyticsConfig.builder(randomAlphaOfLengthBetween(1, 10)) + .setSource(randomSourceConfig()) + .setDest(randomDestConfig()) + .setAnalysis(randomOutlierDetection()); + if (randomBoolean()) { + builder.setAnalyzedFields(new FetchSourceContext(true, + generateRandomStringArray(10, 10, false, false), + generateRandomStringArray(10, 10, false, false))); + } + if (randomBoolean()) { + builder.setModelMemoryLimit(new ByteSizeValue(randomIntBetween(1, 16), randomFrom(ByteSizeUnit.MB, ByteSizeUnit.GB))); + } + if (randomBoolean()) { + builder.setCreateTime(Instant.now()); + } + if (randomBoolean()) { + builder.setVersion(Version.CURRENT); + } + return builder.build(); + } + + @Override + protected DataFrameAnalyticsConfig createTestInstance() { + return randomDataFrameAnalyticsConfig(); + } + + @Override + protected DataFrameAnalyticsConfig doParseInstance(XContentParser parser) throws IOException { + return DataFrameAnalyticsConfig.fromXContent(parser); + } + + @Override + protected boolean supportsUnknownFields() { + return true; + } + + @Override + protected Predicate getRandomFieldsExcludeFilter() { + // allow unknown fields in the root of the object only + return field -> !field.isEmpty(); + } + + @Override + protected NamedXContentRegistry xContentRegistry() { + List namedXContent = new ArrayList<>(); + namedXContent.addAll(new SearchModule(Settings.EMPTY, Collections.emptyList()).getNamedXContents()); + namedXContent.addAll(new MlDataFrameAnalysisNamedXContentProvider().getNamedXContentParsers()); + return new NamedXContentRegistry(namedXContent); + } +} diff --git a/client/rest-high-level/src/test/java/org/elasticsearch/client/ml/dataframe/DataFrameAnalyticsDestTests.java b/client/rest-high-level/src/test/java/org/elasticsearch/client/ml/dataframe/DataFrameAnalyticsDestTests.java new file mode 100644 index 0000000000000..dce7ca5204d57 --- /dev/null +++ b/client/rest-high-level/src/test/java/org/elasticsearch/client/ml/dataframe/DataFrameAnalyticsDestTests.java @@ -0,0 +1,50 @@ +/* + * 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.client.ml.dataframe; + +import org.elasticsearch.common.xcontent.XContentParser; +import org.elasticsearch.test.AbstractXContentTestCase; + +import java.io.IOException; + +public class DataFrameAnalyticsDestTests extends AbstractXContentTestCase { + + public static DataFrameAnalyticsDest randomDestConfig() { + return DataFrameAnalyticsDest.builder() + .setIndex(randomAlphaOfLengthBetween(1, 10)) + .setResultsField(randomBoolean() ? null : randomAlphaOfLengthBetween(1, 10)) + .build(); + } + + @Override + protected DataFrameAnalyticsDest doParseInstance(XContentParser parser) throws IOException { + return DataFrameAnalyticsDest.fromXContent(parser); + } + + @Override + protected boolean supportsUnknownFields() { + return true; + } + + @Override + protected DataFrameAnalyticsDest createTestInstance() { + return randomDestConfig(); + } +} diff --git a/client/rest-high-level/src/test/java/org/elasticsearch/client/ml/dataframe/DataFrameAnalyticsSourceTests.java b/client/rest-high-level/src/test/java/org/elasticsearch/client/ml/dataframe/DataFrameAnalyticsSourceTests.java new file mode 100644 index 0000000000000..d82e1999f3034 --- /dev/null +++ b/client/rest-high-level/src/test/java/org/elasticsearch/client/ml/dataframe/DataFrameAnalyticsSourceTests.java @@ -0,0 +1,70 @@ +/* + * 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.client.ml.dataframe; + +import org.elasticsearch.common.settings.Settings; +import org.elasticsearch.common.xcontent.NamedXContentRegistry; +import org.elasticsearch.common.xcontent.XContentParser; +import org.elasticsearch.search.SearchModule; +import org.elasticsearch.test.AbstractXContentTestCase; + +import java.io.IOException; +import java.util.function.Predicate; + +import static java.util.Collections.emptyList; +import static org.elasticsearch.client.ml.dataframe.QueryConfigTests.randomQueryConfig; + + +public class DataFrameAnalyticsSourceTests extends AbstractXContentTestCase { + + public static DataFrameAnalyticsSource randomSourceConfig() { + return DataFrameAnalyticsSource.builder() + .setIndex(generateRandomStringArray(10, 10, false, false)) + .setQueryConfig(randomBoolean() ? null : randomQueryConfig()) + .build(); + } + + @Override + protected DataFrameAnalyticsSource doParseInstance(XContentParser parser) throws IOException { + return DataFrameAnalyticsSource.fromXContent(parser); + } + + @Override + protected boolean supportsUnknownFields() { + return true; + } + + @Override + protected Predicate getRandomFieldsExcludeFilter() { + // allow unknown fields in the root of the object only as QueryConfig stores a Map + return field -> !field.isEmpty(); + } + + @Override + protected DataFrameAnalyticsSource createTestInstance() { + return randomSourceConfig(); + } + + @Override + protected NamedXContentRegistry xContentRegistry() { + SearchModule searchModule = new SearchModule(Settings.EMPTY, emptyList()); + return new NamedXContentRegistry(searchModule.getNamedXContents()); + } +} diff --git a/client/rest-high-level/src/test/java/org/elasticsearch/client/ml/dataframe/DataFrameAnalyticsStatsTests.java b/client/rest-high-level/src/test/java/org/elasticsearch/client/ml/dataframe/DataFrameAnalyticsStatsTests.java new file mode 100644 index 0000000000000..fad02eac161c7 --- /dev/null +++ b/client/rest-high-level/src/test/java/org/elasticsearch/client/ml/dataframe/DataFrameAnalyticsStatsTests.java @@ -0,0 +1,70 @@ +/* + * 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.client.ml.dataframe; + +import org.elasticsearch.client.ml.NodeAttributesTests; +import org.elasticsearch.common.xcontent.XContentBuilder; +import org.elasticsearch.test.ESTestCase; + +import java.io.IOException; + +import static org.elasticsearch.test.AbstractXContentTestCase.xContentTester; + +public class DataFrameAnalyticsStatsTests extends ESTestCase { + + public void testFromXContent() throws IOException { + xContentTester(this::createParser, + DataFrameAnalyticsStatsTests::randomDataFrameAnalyticsStats, + DataFrameAnalyticsStatsTests::toXContent, + DataFrameAnalyticsStats::fromXContent) + .supportsUnknownFields(true) + .randomFieldsExcludeFilter(field -> field.startsWith("node.attributes")) + .test(); + } + + public static DataFrameAnalyticsStats randomDataFrameAnalyticsStats() { + return new DataFrameAnalyticsStats( + randomAlphaOfLengthBetween(1, 10), + randomFrom(DataFrameAnalyticsState.values()), + randomBoolean() ? null : randomAlphaOfLength(10), + randomBoolean() ? null : randomIntBetween(0, 100), + randomBoolean() ? null : NodeAttributesTests.createRandom(), + randomBoolean() ? null : randomAlphaOfLengthBetween(1, 20)); + } + + public static void toXContent(DataFrameAnalyticsStats stats, XContentBuilder builder) throws IOException { + builder.startObject(); + builder.field(DataFrameAnalyticsStats.ID.getPreferredName(), stats.getId()); + builder.field(DataFrameAnalyticsStats.STATE.getPreferredName(), stats.getState().value()); + if (stats.getFailureReason() != null) { + builder.field(DataFrameAnalyticsStats.FAILURE_REASON.getPreferredName(), stats.getFailureReason()); + } + if (stats.getProgressPercent() != null) { + builder.field(DataFrameAnalyticsStats.PROGRESS_PERCENT.getPreferredName(), stats.getProgressPercent()); + } + if (stats.getNode() != null) { + builder.field(DataFrameAnalyticsStats.NODE.getPreferredName(), stats.getNode()); + } + if (stats.getAssignmentExplanation() != null) { + builder.field(DataFrameAnalyticsStats.ASSIGNMENT_EXPLANATION.getPreferredName(), stats.getAssignmentExplanation()); + } + builder.endObject(); + } +} diff --git a/client/rest-high-level/src/test/java/org/elasticsearch/client/ml/dataframe/OutlierDetectionTests.java b/client/rest-high-level/src/test/java/org/elasticsearch/client/ml/dataframe/OutlierDetectionTests.java new file mode 100644 index 0000000000000..7307999a2bf71 --- /dev/null +++ b/client/rest-high-level/src/test/java/org/elasticsearch/client/ml/dataframe/OutlierDetectionTests.java @@ -0,0 +1,73 @@ +/* + * 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.client.ml.dataframe; + +import org.elasticsearch.common.xcontent.XContentParser; +import org.elasticsearch.test.AbstractXContentTestCase; + +import java.io.IOException; + +import static org.hamcrest.Matchers.closeTo; +import static org.hamcrest.Matchers.equalTo; + +public class OutlierDetectionTests extends AbstractXContentTestCase { + + public static OutlierDetection randomOutlierDetection() { + return OutlierDetection.builder() + .setNNeighbors(randomBoolean() ? null : randomIntBetween(1, 20)) + .setMethod(randomBoolean() ? null : randomFrom(OutlierDetection.Method.values())) + .setFeatureInfluenceThreshold(randomBoolean() ? null : randomDoubleBetween(0.0, 1.0, true)) + .build(); + } + + @Override + protected OutlierDetection doParseInstance(XContentParser parser) throws IOException { + return OutlierDetection.fromXContent(parser); + } + + @Override + protected boolean supportsUnknownFields() { + return true; + } + + @Override + protected OutlierDetection createTestInstance() { + return randomOutlierDetection(); + } + + public void testGetParams_GivenDefaults() { + OutlierDetection outlierDetection = OutlierDetection.createDefault(); + assertNull(outlierDetection.getNNeighbors()); + assertNull(outlierDetection.getMethod()); + assertNull(outlierDetection.getFeatureInfluenceThreshold()); + } + + public void testGetParams_GivenExplicitValues() { + OutlierDetection outlierDetection = + OutlierDetection.builder() + .setNNeighbors(42) + .setMethod(OutlierDetection.Method.LDOF) + .setFeatureInfluenceThreshold(0.5) + .build(); + assertThat(outlierDetection.getNNeighbors(), equalTo(42)); + assertThat(outlierDetection.getMethod(), equalTo(OutlierDetection.Method.LDOF)); + assertThat(outlierDetection.getFeatureInfluenceThreshold(), closeTo(0.5, 1E-9)); + } +} diff --git a/client/rest-high-level/src/test/java/org/elasticsearch/client/ml/dataframe/QueryConfigTests.java b/client/rest-high-level/src/test/java/org/elasticsearch/client/ml/dataframe/QueryConfigTests.java new file mode 100644 index 0000000000000..7413bc936a215 --- /dev/null +++ b/client/rest-high-level/src/test/java/org/elasticsearch/client/ml/dataframe/QueryConfigTests.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.client.ml.dataframe; + +import org.elasticsearch.common.settings.Settings; +import org.elasticsearch.common.xcontent.NamedXContentRegistry; +import org.elasticsearch.common.xcontent.XContentParser; +import org.elasticsearch.index.query.MatchAllQueryBuilder; +import org.elasticsearch.index.query.MatchNoneQueryBuilder; +import org.elasticsearch.index.query.QueryBuilder; +import org.elasticsearch.search.SearchModule; +import org.elasticsearch.test.AbstractXContentTestCase; + +import java.io.IOException; + +import static java.util.Collections.emptyList; + +public class QueryConfigTests extends AbstractXContentTestCase { + + public static QueryConfig randomQueryConfig() { + QueryBuilder queryBuilder = randomBoolean() ? new MatchAllQueryBuilder() : new MatchNoneQueryBuilder(); + return new QueryConfig(queryBuilder); + } + + @Override + protected QueryConfig createTestInstance() { + return randomQueryConfig(); + } + + @Override + protected QueryConfig doParseInstance(XContentParser parser) throws IOException { + return QueryConfig.fromXContent(parser); + } + + @Override + protected boolean supportsUnknownFields() { + return false; + } + + @Override + protected NamedXContentRegistry xContentRegistry() { + SearchModule searchModule = new SearchModule(Settings.EMPTY, emptyList()); + return new NamedXContentRegistry(searchModule.getNamedXContents()); + } +} diff --git a/client/rest-high-level/src/test/java/org/elasticsearch/client/ml/filestructurefinder/FieldStatsTests.java b/client/rest-high-level/src/test/java/org/elasticsearch/client/ml/filestructurefinder/FieldStatsTests.java index daf6c4af90ddc..0d6e5e70290d9 100644 --- a/client/rest-high-level/src/test/java/org/elasticsearch/client/ml/filestructurefinder/FieldStatsTests.java +++ b/client/rest-high-level/src/test/java/org/elasticsearch/client/ml/filestructurefinder/FieldStatsTests.java @@ -43,6 +43,8 @@ static FieldStats createTestFieldStats() { Double maxValue = null; Double meanValue = null; Double medianValue = null; + String earliestTimestamp = null; + String latestTimestamp = null; boolean isMetric = randomBoolean(); if (isMetric) { if (randomBoolean()) { @@ -54,6 +56,12 @@ static FieldStats createTestFieldStats() { } meanValue = randomDouble(); medianValue = randomDouble(); + } else { + boolean isDate = randomBoolean(); + if (isDate) { + earliestTimestamp = randomAlphaOfLength(20); + latestTimestamp = randomAlphaOfLength(20); + } } List> topHits = new ArrayList<>(); @@ -68,7 +76,7 @@ static FieldStats createTestFieldStats() { topHits.add(topHit); } - return new FieldStats(count, cardinality, minValue, maxValue, meanValue, medianValue, topHits); + return new FieldStats(count, cardinality, minValue, maxValue, meanValue, medianValue, earliestTimestamp, latestTimestamp, topHits); } @Override diff --git a/client/rest-high-level/src/test/java/org/elasticsearch/client/ml/job/config/AnalysisConfigTests.java b/client/rest-high-level/src/test/java/org/elasticsearch/client/ml/job/config/AnalysisConfigTests.java index 3914697b768ba..a72de967d86b7 100644 --- a/client/rest-high-level/src/test/java/org/elasticsearch/client/ml/job/config/AnalysisConfigTests.java +++ b/client/rest-high-level/src/test/java/org/elasticsearch/client/ml/job/config/AnalysisConfigTests.java @@ -256,7 +256,7 @@ private static AnalysisConfig.Builder createValidCategorizationConfig() { @Override protected NamedXContentRegistry xContentRegistry() { - SearchModule searchModule = new SearchModule(Settings.EMPTY, false, Collections.emptyList()); + SearchModule searchModule = new SearchModule(Settings.EMPTY, Collections.emptyList()); return new NamedXContentRegistry(searchModule.getNamedXContents()); } } diff --git a/client/rest-high-level/src/test/java/org/elasticsearch/client/ml/job/config/JobTests.java b/client/rest-high-level/src/test/java/org/elasticsearch/client/ml/job/config/JobTests.java index 70e8b4296b0df..8794389066314 100644 --- a/client/rest-high-level/src/test/java/org/elasticsearch/client/ml/job/config/JobTests.java +++ b/client/rest-high-level/src/test/java/org/elasticsearch/client/ml/job/config/JobTests.java @@ -168,7 +168,7 @@ public static Job createRandomizedJob() { @Override protected NamedXContentRegistry xContentRegistry() { - SearchModule searchModule = new SearchModule(Settings.EMPTY, false, Collections.emptyList()); + SearchModule searchModule = new SearchModule(Settings.EMPTY, Collections.emptyList()); return new NamedXContentRegistry(searchModule.getNamedXContents()); } } diff --git a/client/rest-high-level/src/test/java/org/elasticsearch/client/ml/job/process/TimingStatsTests.java b/client/rest-high-level/src/test/java/org/elasticsearch/client/ml/job/process/TimingStatsTests.java new file mode 100644 index 0000000000000..386bc6ac0b3dd --- /dev/null +++ b/client/rest-high-level/src/test/java/org/elasticsearch/client/ml/job/process/TimingStatsTests.java @@ -0,0 +1,97 @@ +/* + * 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.client.ml.job.process; + +import org.elasticsearch.common.xcontent.XContentParser; +import org.elasticsearch.test.AbstractXContentTestCase; + +import static org.hamcrest.Matchers.equalTo; +import static org.hamcrest.Matchers.nullValue; + +public class TimingStatsTests extends AbstractXContentTestCase { + + private static final String JOB_ID = "my-job-id"; + + public static TimingStats createTestInstance(String jobId) { + return new TimingStats( + jobId, + randomLong(), + randomBoolean() ? null : randomDouble(), + randomBoolean() ? null : randomDouble(), + randomBoolean() ? null : randomDouble(), + randomBoolean() ? null : randomDouble()); + } + + @Override + public TimingStats createTestInstance() { + return createTestInstance(randomAlphaOfLength(10)); + } + + @Override + protected TimingStats doParseInstance(XContentParser parser) { + return TimingStats.PARSER.apply(parser, null); + } + + @Override + protected boolean supportsUnknownFields() { + return true; + } + + public void testConstructor() { + TimingStats stats = new TimingStats(JOB_ID, 7, 1.0, 2.0, 1.23, 7.89); + + assertThat(stats.getJobId(), equalTo(JOB_ID)); + assertThat(stats.getBucketCount(), equalTo(7L)); + assertThat(stats.getMinBucketProcessingTimeMs(), equalTo(1.0)); + assertThat(stats.getMaxBucketProcessingTimeMs(), equalTo(2.0)); + assertThat(stats.getAvgBucketProcessingTimeMs(), equalTo(1.23)); + assertThat(stats.getExponentialAvgBucketProcessingTimeMs(), equalTo(7.89)); + } + + public void testConstructor_NullValues() { + TimingStats stats = new TimingStats(JOB_ID, 7, null, null, null, null); + + assertThat(stats.getJobId(), equalTo(JOB_ID)); + assertThat(stats.getBucketCount(), equalTo(7L)); + assertThat(stats.getMinBucketProcessingTimeMs(), nullValue()); + assertThat(stats.getMaxBucketProcessingTimeMs(), nullValue()); + assertThat(stats.getAvgBucketProcessingTimeMs(), nullValue()); + assertThat(stats.getExponentialAvgBucketProcessingTimeMs(), nullValue()); + } + + public void testEquals() { + TimingStats stats1 = new TimingStats(JOB_ID, 7, 1.0, 2.0, 1.23, 7.89); + TimingStats stats2 = new TimingStats(JOB_ID, 7, 1.0, 2.0, 1.23, 7.89); + TimingStats stats3 = new TimingStats(JOB_ID, 7, 1.0, 3.0, 1.23, 7.89); + + assertTrue(stats1.equals(stats1)); + assertTrue(stats1.equals(stats2)); + assertFalse(stats2.equals(stats3)); + } + + public void testHashCode() { + TimingStats stats1 = new TimingStats(JOB_ID, 7, 1.0, 2.0, 1.23, 7.89); + TimingStats stats2 = new TimingStats(JOB_ID, 7, 1.0, 2.0, 1.23, 7.89); + TimingStats stats3 = new TimingStats(JOB_ID, 7, 1.0, 3.0, 1.23, 7.89); + + assertEquals(stats1.hashCode(), stats1.hashCode()); + assertEquals(stats1.hashCode(), stats2.hashCode()); + assertNotEquals(stats2.hashCode(), stats3.hashCode()); + } +} diff --git a/client/rest-high-level/src/test/java/org/elasticsearch/client/ml/job/stats/JobStatsTests.java b/client/rest-high-level/src/test/java/org/elasticsearch/client/ml/job/stats/JobStatsTests.java index 5d00f879140e0..e7aa33f1d7a3b 100644 --- a/client/rest-high-level/src/test/java/org/elasticsearch/client/ml/job/stats/JobStatsTests.java +++ b/client/rest-high-level/src/test/java/org/elasticsearch/client/ml/job/stats/JobStatsTests.java @@ -24,6 +24,8 @@ import org.elasticsearch.client.ml.job.process.DataCountsTests; import org.elasticsearch.client.ml.job.process.ModelSizeStats; import org.elasticsearch.client.ml.job.process.ModelSizeStatsTests; +import org.elasticsearch.client.ml.job.process.TimingStats; +import org.elasticsearch.client.ml.job.process.TimingStatsTests; import org.elasticsearch.common.unit.TimeValue; import org.elasticsearch.common.xcontent.XContentParser; import org.elasticsearch.client.ml.job.config.JobState; @@ -42,12 +44,14 @@ public static JobStats createRandomInstance() { DataCounts dataCounts = DataCountsTests.createTestInstance(jobId); ModelSizeStats modelSizeStats = randomBoolean() ? ModelSizeStatsTests.createRandomized() : null; + TimingStats timingStats = randomBoolean() ? TimingStatsTests.createTestInstance(jobId) : null; ForecastStats forecastStats = randomBoolean() ? ForecastStatsTests.createRandom(1, 22) : null; NodeAttributes nodeAttributes = randomBoolean() ? NodeAttributesTests.createRandom() : null; String assigmentExplanation = randomBoolean() ? randomAlphaOfLength(10) : null; TimeValue openTime = randomBoolean() ? TimeValue.timeValueMillis(randomIntBetween(1, 10000)) : null; - return new JobStats(jobId, dataCounts, state, modelSizeStats, forecastStats, nodeAttributes, assigmentExplanation, openTime); + return new JobStats( + jobId, dataCounts, state, modelSizeStats, timingStats, forecastStats, nodeAttributes, assigmentExplanation, openTime); } @Override diff --git a/client/rest-high-level/src/test/java/org/elasticsearch/client/security/CreateTokenRequestTests.java b/client/rest-high-level/src/test/java/org/elasticsearch/client/security/CreateTokenRequestTests.java index 53f3e1d0f368c..760d5e52cb38f 100644 --- a/client/rest-high-level/src/test/java/org/elasticsearch/client/security/CreateTokenRequestTests.java +++ b/client/rest-high-level/src/test/java/org/elasticsearch/client/security/CreateTokenRequestTests.java @@ -66,31 +66,54 @@ public void testCreateTokenFromClientCredentials() { assertThat(Strings.toString(request), equalTo("{\"grant_type\":\"client_credentials\"}")); } + public void testCreateTokenFromKerberosTicket() { + final CreateTokenRequest request = CreateTokenRequest.kerberosGrant("top secret kerberos ticket".toCharArray()); + assertThat(request.getGrantType(), equalTo("_kerberos")); + assertThat(request.getScope(), nullValue()); + assertThat(request.getUsername(), nullValue()); + assertThat(request.getPassword(), nullValue()); + assertThat(request.getRefreshToken(), nullValue()); + assertThat(new String(request.getKerberosTicket()), equalTo("top secret kerberos ticket")); + assertThat(Strings.toString(request), equalTo("{\"grant_type\":\"_kerberos\"," + + "\"kerberos_ticket\":\"top secret kerberos ticket\"}")); + } + public void testEqualsAndHashCode() { final String grantType = randomAlphaOfLength(8); final String scope = randomBoolean() ? null : randomAlphaOfLength(6); final String username = randomBoolean() ? null : randomAlphaOfLengthBetween(4, 10); final char[] password = randomBoolean() ? null : randomAlphaOfLengthBetween(8, 12).toCharArray(); final String refreshToken = randomBoolean() ? null : randomAlphaOfLengthBetween(12, 24); - final CreateTokenRequest request = new CreateTokenRequest(grantType, scope, username, password, refreshToken); + final char[] kerberosTicket = randomBoolean() ? null : randomAlphaOfLengthBetween(8, 12).toCharArray(); + final CreateTokenRequest request = new CreateTokenRequest(grantType, scope, username, password, refreshToken, kerberosTicket); EqualsHashCodeTestUtils.checkEqualsAndHashCode(request, - r -> new CreateTokenRequest(r.getGrantType(), r.getScope(), r.getUsername(), r.getPassword(), r.getRefreshToken()), + r -> new CreateTokenRequest(r.getGrantType(), r.getScope(), r.getUsername(), r.getPassword(), + r.getRefreshToken(), r.getKerberosTicket()), this::mutate); } private CreateTokenRequest mutate(CreateTokenRequest req) { - switch (randomIntBetween(1, 5)) { - case 1: - return new CreateTokenRequest("g", req.getScope(), req.getUsername(), req.getPassword(), req.getRefreshToken()); - case 2: - return new CreateTokenRequest(req.getGrantType(), "s", req.getUsername(), req.getPassword(), req.getRefreshToken()); - case 3: - return new CreateTokenRequest(req.getGrantType(), req.getScope(), "u", req.getPassword(), req.getRefreshToken()); - case 4: - final char[] password = {'p'}; - return new CreateTokenRequest(req.getGrantType(), req.getScope(), req.getUsername(), password, req.getRefreshToken()); - case 5: - return new CreateTokenRequest(req.getGrantType(), req.getScope(), req.getUsername(), req.getPassword(), "r"); + switch (randomIntBetween(1, 6)) { + case 1: + return new CreateTokenRequest("g", req.getScope(), req.getUsername(), req.getPassword(), req.getRefreshToken(), + req.getKerberosTicket()); + case 2: + return new CreateTokenRequest(req.getGrantType(), "s", req.getUsername(), req.getPassword(), req.getRefreshToken(), + req.getKerberosTicket()); + case 3: + return new CreateTokenRequest(req.getGrantType(), req.getScope(), "u", req.getPassword(), req.getRefreshToken(), + req.getKerberosTicket()); + case 4: + final char[] password = { 'p' }; + return new CreateTokenRequest(req.getGrantType(), req.getScope(), req.getUsername(), password, req.getRefreshToken(), + req.getKerberosTicket()); + case 5: + final char[] kerberosTicket = { 'k' }; + return new CreateTokenRequest(req.getGrantType(), req.getScope(), req.getUsername(), req.getPassword(), req.getRefreshToken(), + kerberosTicket); + case 6: + return new CreateTokenRequest(req.getGrantType(), req.getScope(), req.getUsername(), req.getPassword(), "r", + req.getKerberosTicket()); } throw new IllegalStateException("Bad random number"); } diff --git a/client/rest-high-level/src/test/java/org/elasticsearch/client/security/CreateTokenResponseTests.java b/client/rest-high-level/src/test/java/org/elasticsearch/client/security/CreateTokenResponseTests.java index f99ea668665dd..34a03647f6060 100644 --- a/client/rest-high-level/src/test/java/org/elasticsearch/client/security/CreateTokenResponseTests.java +++ b/client/rest-high-level/src/test/java/org/elasticsearch/client/security/CreateTokenResponseTests.java @@ -37,6 +37,7 @@ public void testFromXContent() throws IOException { final String refreshToken = randomBoolean() ? null : randomAlphaOfLengthBetween(12, 24); final String scope = randomBoolean() ? null : randomAlphaOfLength(4); final String type = randomAlphaOfLength(6); + final String kerberosAuthenticationResponseToken = randomBoolean() ? null : randomAlphaOfLength(7); final XContentType xContentType = randomFrom(XContentType.values()); final XContentBuilder builder = XContentFactory.contentBuilder(xContentType); @@ -50,6 +51,9 @@ public void testFromXContent() throws IOException { if (scope != null || randomBoolean()) { builder.field("scope", scope); } + if (kerberosAuthenticationResponseToken != null) { + builder.field("kerberos_authentication_response_token", kerberosAuthenticationResponseToken); + } builder.endObject(); BytesReference xContent = BytesReference.bytes(builder); @@ -59,5 +63,6 @@ public void testFromXContent() throws IOException { assertThat(response.getScope(), equalTo(scope)); assertThat(response.getType(), equalTo(type)); assertThat(response.getExpiresIn(), equalTo(expiresIn)); + assertThat(response.getKerberosAuthenticationResponseToken(), equalTo(kerberosAuthenticationResponseToken)); } } diff --git a/client/rest-high-level/src/test/java/org/elasticsearch/client/watcher/ExecuteWatchResponseTests.java b/client/rest-high-level/src/test/java/org/elasticsearch/client/watcher/ExecuteWatchResponseTests.java deleted file mode 100644 index 3e0ef4c8a5e5f..0000000000000 --- a/client/rest-high-level/src/test/java/org/elasticsearch/client/watcher/ExecuteWatchResponseTests.java +++ /dev/null @@ -1,115 +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.client.watcher; - -import org.elasticsearch.common.bytes.BytesReference; -import org.elasticsearch.common.xcontent.ObjectPath; -import org.elasticsearch.common.xcontent.XContentBuilder; -import org.elasticsearch.common.xcontent.XContentFactory; -import org.elasticsearch.common.xcontent.XContentParser; -import org.elasticsearch.common.xcontent.json.JsonXContent; -import org.elasticsearch.test.ESTestCase; - -import java.io.IOException; -import java.io.InputStream; -import java.util.List; -import java.util.Map; - -import static org.elasticsearch.test.AbstractXContentTestCase.xContentTester; -import static org.hamcrest.Matchers.is; - -public class ExecuteWatchResponseTests extends ESTestCase { - - public static final String WATCH_ID_VALUE = "my_watch"; - public static final String NODE_VALUE = "my_node"; - public static final String TRIGGER_TYPE_VALUE = "manual"; - public static final String STATE_VALUE = "executed"; - public static final String STATE_KEY = "state"; - public static final String TRIGGER_EVENT_KEY = "trigger_event"; - public static final String TRIGGER_EVENT_TYPE_KEY = "type"; - public static final String MESSAGES_KEY = "messages"; - public static final String NODE_KEY = "node"; - public static final String WATCH_ID_KEY = "watch_id"; - - public void testFromXContent() throws IOException { - xContentTester(this::createParser, - ExecuteWatchResponseTests::createTestInstance, - this::toXContent, - ExecuteWatchResponse::fromXContent) - .supportsUnknownFields(true) - .assertEqualsConsumer(this::assertEqualInstances) - .assertToXContentEquivalence(false) - .test(); - } - - private void assertEqualInstances(ExecuteWatchResponse expected, ExecuteWatchResponse actual) { - assertThat(expected.getRecordId(), is(actual.getRecordId())); - - // This may have extra json, so lets just assume that if all of the original fields from the creation are there, then its equal - // This is the same code that is in createTestInstance in this class. - Map actualMap = actual.getRecordAsMap(); - assertThat(ObjectPath.eval(WATCH_ID_KEY, actualMap), is(WATCH_ID_VALUE)); - assertThat(ObjectPath.eval(NODE_KEY, actualMap), is(NODE_VALUE)); - List messages = ObjectPath.eval(MESSAGES_KEY, actualMap); - assertThat(messages.size(), is(0)); - assertThat(ObjectPath.eval(TRIGGER_EVENT_KEY + "." + TRIGGER_EVENT_TYPE_KEY, actualMap), is(TRIGGER_TYPE_VALUE)); - assertThat(ObjectPath.eval(STATE_KEY, actualMap), is(STATE_VALUE)); - } - - private XContentBuilder toXContent(BytesReference bytes, XContentBuilder builder) throws IOException { - // EMPTY is safe here because we never use namedObject - try (InputStream stream = bytes.streamInput(); - XContentParser parser = createParser(JsonXContent.jsonXContent, stream)) { - parser.nextToken(); - builder.generator().copyCurrentStructure(parser); - return builder; - } - } - - private XContentBuilder toXContent(ExecuteWatchResponse response, XContentBuilder builder) throws IOException { - builder.startObject(); - builder.field("_id", response.getRecordId()); - builder.field("watch_record"); - toXContent(response.getRecord(), builder); - return builder.endObject(); - } - - private static ExecuteWatchResponse createTestInstance() { - String id = "my_watch_0-2015-06-02T23:17:55.124Z"; - try { - XContentBuilder builder = XContentFactory.jsonBuilder(); - builder.startObject(); - builder.field(WATCH_ID_KEY, WATCH_ID_VALUE); - builder.field(NODE_KEY, NODE_VALUE); - builder.startArray(MESSAGES_KEY); - builder.endArray(); - builder.startObject(TRIGGER_EVENT_KEY); - builder.field(TRIGGER_EVENT_TYPE_KEY, TRIGGER_TYPE_VALUE); - builder.endObject(); - builder.field(STATE_KEY, STATE_VALUE); - builder.endObject(); - BytesReference bytes = BytesReference.bytes(builder); - return new ExecuteWatchResponse(id, bytes); - } - catch (IOException e) { - throw new AssertionError(e); - } - } -} diff --git a/client/rest-high-level/src/test/java/org/elasticsearch/client/watcher/GetWatchResponseTests.java b/client/rest-high-level/src/test/java/org/elasticsearch/client/watcher/GetWatchResponseTests.java index 7d919ef5f68c1..b69ea90a49e4a 100644 --- a/client/rest-high-level/src/test/java/org/elasticsearch/client/watcher/GetWatchResponseTests.java +++ b/client/rest-high-level/src/test/java/org/elasticsearch/client/watcher/GetWatchResponseTests.java @@ -19,17 +19,13 @@ package org.elasticsearch.client.watcher; import org.apache.lucene.util.LuceneTestCase; +import org.elasticsearch.client.AbstractResponseTestCase; import org.elasticsearch.common.bytes.BytesReference; import org.elasticsearch.common.time.DateUtils; -import org.elasticsearch.common.xcontent.DeprecationHandler; -import org.elasticsearch.common.xcontent.NamedXContentRegistry; import org.elasticsearch.common.xcontent.ToXContent; -import org.elasticsearch.common.xcontent.XContent; import org.elasticsearch.common.xcontent.XContentBuilder; -import org.elasticsearch.common.xcontent.XContentFactory; import org.elasticsearch.common.xcontent.XContentParser; import org.elasticsearch.common.xcontent.XContentType; -import org.elasticsearch.client.AbstractHlrcStreamableXContentTestCase; import org.elasticsearch.xpack.core.watcher.actions.ActionStatus; import org.elasticsearch.xpack.core.watcher.execution.ExecutionState; import org.elasticsearch.xpack.core.watcher.support.xcontent.XContentSource; @@ -37,70 +33,17 @@ import org.elasticsearch.xpack.core.watcher.watch.WatchStatus; import java.io.IOException; -import java.io.InputStream; import java.time.ZonedDateTime; -import java.util.Collections; import java.util.HashMap; import java.util.Map; -import java.util.function.Predicate; -public class GetWatchResponseTests extends - AbstractHlrcStreamableXContentTestCase { +import static org.hamcrest.Matchers.equalTo; +import static org.hamcrest.Matchers.nullValue; - private static final String[] SHUFFLE_FIELDS_EXCEPTION = new String[] { "watch" }; +public class GetWatchResponseTests extends AbstractResponseTestCase { @Override - protected String[] getShuffleFieldsExceptions() { - return SHUFFLE_FIELDS_EXCEPTION; - } - - @Override - protected ToXContent.Params getToXContentParams() { - return new ToXContent.MapParams(Collections.singletonMap("hide_headers", "false")); - } - - @Override - protected Predicate getRandomFieldsExcludeFilter() { - return f -> f.contains("watch") || f.contains("actions") || f.contains("headers"); - } - - @Override - protected void assertEqualInstances(GetWatchResponse expectedInstance, GetWatchResponse newInstance) { - if (expectedInstance.isFound() && - expectedInstance.getSource().getContentType() != newInstance.getSource().getContentType()) { - /** - * The {@link GetWatchResponse#getContentType()} depends on the content type that - * was used to serialize the main object so we use the same content type than the - * expectedInstance to translate the watch of the newInstance. - */ - XContent from = XContentFactory.xContent(newInstance.getSource().getContentType()); - XContent to = XContentFactory.xContent(expectedInstance.getSource().getContentType()); - final BytesReference newSource; - // It is safe to use EMPTY here because this never uses namedObject - try (InputStream stream = newInstance.getSource().getBytes().streamInput(); - XContentParser parser = XContentFactory.xContent(from.type()).createParser(NamedXContentRegistry.EMPTY, - DeprecationHandler.THROW_UNSUPPORTED_OPERATION, stream)) { - parser.nextToken(); - XContentBuilder builder = XContentFactory.contentBuilder(to.type()); - builder.copyCurrentStructure(parser); - newSource = BytesReference.bytes(builder); - } catch (IOException e) { - throw new AssertionError(e); - } - newInstance = new GetWatchResponse(newInstance.getId(), newInstance.getVersion(), - newInstance.getSeqNo(), newInstance.getPrimaryTerm(), - newInstance.getStatus(), new XContentSource(newSource, expectedInstance.getSource().getContentType())); - } - super.assertEqualInstances(expectedInstance, newInstance); - } - - @Override - protected GetWatchResponse createBlankInstance() { - return new GetWatchResponse(); - } - - @Override - protected GetWatchResponse createTestInstance() { + protected GetWatchResponse createServerTestInstance() { String id = randomAlphaOfLength(10); if (LuceneTestCase.rarely()) { return new GetWatchResponse(id); @@ -113,6 +56,34 @@ protected GetWatchResponse createTestInstance() { return new GetWatchResponse(id, version, seqNo, primaryTerm, status, new XContentSource(source, XContentType.JSON)); } + @Override + protected org.elasticsearch.client.watcher.GetWatchResponse doParseToClientInstance(XContentParser parser) throws IOException { + return org.elasticsearch.client.watcher.GetWatchResponse.fromXContent(parser); + } + + @Override + protected void assertInstances(GetWatchResponse serverTestInstance, org.elasticsearch.client.watcher.GetWatchResponse clientInstance) { + assertThat(clientInstance.getId(), equalTo(serverTestInstance.getId())); + assertThat(clientInstance.getSeqNo(), equalTo(serverTestInstance.getSeqNo())); + assertThat(clientInstance.getPrimaryTerm(), equalTo(serverTestInstance.getPrimaryTerm())); + assertThat(clientInstance.getVersion(), equalTo(serverTestInstance.getVersion())); + if (serverTestInstance.getStatus() != null) { + assertThat(convertWatchStatus(clientInstance.getStatus()), equalTo(serverTestInstance.getStatus())); + } else { + assertThat(clientInstance.getStatus(), nullValue()); + } + if (serverTestInstance.getSource() != null) { + assertThat(clientInstance.getSourceAsMap(), equalTo(serverTestInstance.getSource().getAsMap())); + } else { + assertThat(clientInstance.getSource(), nullValue()); + } + } + + @Override + protected ToXContent.Params getParams() { + return new ToXContent.MapParams(Map.of("hide_headers", "false")); + } + private static BytesReference simpleWatch() { try { XContentBuilder builder = XContentBuilder.builder(XContentType.JSON.xContent()); @@ -179,58 +150,45 @@ private static ActionStatus.Execution randomExecution() { } } - @Override - public org.elasticsearch.client.watcher.GetWatchResponse doHlrcParseInstance(XContentParser parser) throws IOException { - return org.elasticsearch.client.watcher.GetWatchResponse.fromXContent(parser); - } - - @Override - public GetWatchResponse convertHlrcToInternal(org.elasticsearch.client.watcher.GetWatchResponse instance) { - if (instance.isFound()) { - return new GetWatchResponse(instance.getId(), instance.getVersion(), instance.getSeqNo(), instance.getPrimaryTerm(), - convertHlrcToInternal(instance.getStatus()), new XContentSource(instance.getSource(), instance.getContentType())); - } else { - return new GetWatchResponse(instance.getId()); - } - } - - private static WatchStatus convertHlrcToInternal(org.elasticsearch.client.watcher.WatchStatus status) { + private static WatchStatus convertWatchStatus(org.elasticsearch.client.watcher.WatchStatus status) { final Map actions = new HashMap<>(); for (Map.Entry entry : status.getActions().entrySet()) { - actions.put(entry.getKey(), convertHlrcToInternal(entry.getValue())); + actions.put(entry.getKey(), convertActionStatus(entry.getValue())); } return new WatchStatus(status.version(), - convertHlrcToInternal(status.state()), - status.getExecutionState() == null ? null : convertHlrcToInternal(status.getExecutionState()), + convertWatchStatusState(status.state()), + status.getExecutionState() == null ? null : convertWatchStatus(status.getExecutionState()), status.lastChecked(), status.lastMetCondition(), actions, status.getHeaders() ); } - private static ActionStatus convertHlrcToInternal(org.elasticsearch.client.watcher.ActionStatus actionStatus) { - return new ActionStatus(convertHlrcToInternal(actionStatus.ackStatus()), - actionStatus.lastExecution() == null ? null : convertHlrcToInternal(actionStatus.lastExecution()), - actionStatus.lastSuccessfulExecution() == null ? null : convertHlrcToInternal(actionStatus.lastSuccessfulExecution()), - actionStatus.lastThrottle() == null ? null : convertHlrcToInternal(actionStatus.lastThrottle()) + private static ActionStatus convertActionStatus(org.elasticsearch.client.watcher.ActionStatus actionStatus) { + return new ActionStatus(convertAckStatus(actionStatus.ackStatus()), + actionStatus.lastExecution() == null ? null : convertActionStatusExecution(actionStatus.lastExecution()), + actionStatus.lastSuccessfulExecution() == null ? null : convertActionStatusExecution(actionStatus.lastSuccessfulExecution()), + actionStatus.lastThrottle() == null ? null : convertActionStatusThrottle(actionStatus.lastThrottle()) ); } - private static ActionStatus.AckStatus convertHlrcToInternal(org.elasticsearch.client.watcher.ActionStatus.AckStatus ackStatus) { - return new ActionStatus.AckStatus(ackStatus.timestamp(), convertHlrcToInternal(ackStatus.state())); + private static ActionStatus.AckStatus convertAckStatus(org.elasticsearch.client.watcher.ActionStatus.AckStatus ackStatus) { + return new ActionStatus.AckStatus(ackStatus.timestamp(), convertAckStatusState(ackStatus.state())); } - private static ActionStatus.AckStatus.State convertHlrcToInternal(org.elasticsearch.client.watcher.ActionStatus.AckStatus.State state) { + private static ActionStatus.AckStatus.State convertAckStatusState( + org.elasticsearch.client.watcher.ActionStatus.AckStatus.State state) { return ActionStatus.AckStatus.State.valueOf(state.name()); } - private static WatchStatus.State convertHlrcToInternal(org.elasticsearch.client.watcher.WatchStatus.State state) { + private static WatchStatus.State convertWatchStatusState(org.elasticsearch.client.watcher.WatchStatus.State state) { return new WatchStatus.State(state.isActive(), state.getTimestamp()); } - private static ExecutionState convertHlrcToInternal(org.elasticsearch.client.watcher.ExecutionState executionState) { + private static ExecutionState convertWatchStatus(org.elasticsearch.client.watcher.ExecutionState executionState) { return ExecutionState.valueOf(executionState.name()); } - private static ActionStatus.Execution convertHlrcToInternal(org.elasticsearch.client.watcher.ActionStatus.Execution execution) { + private static ActionStatus.Execution convertActionStatusExecution( + org.elasticsearch.client.watcher.ActionStatus.Execution execution) { if (execution.successful()) { return ActionStatus.Execution.successful(execution.timestamp()); } else { @@ -238,7 +196,7 @@ private static ActionStatus.Execution convertHlrcToInternal(org.elasticsearch.cl } } - private static ActionStatus.Throttle convertHlrcToInternal(org.elasticsearch.client.watcher.ActionStatus.Throttle throttle) { + private static ActionStatus.Throttle convertActionStatusThrottle(org.elasticsearch.client.watcher.ActionStatus.Throttle throttle) { return new ActionStatus.Throttle(throttle.timestamp(), throttle.reason()); } } diff --git a/client/rest-high-level/src/test/java/org/elasticsearch/client/watcher/hlrc/DeleteWatchResponseTests.java b/client/rest-high-level/src/test/java/org/elasticsearch/client/watcher/hlrc/DeleteWatchResponseTests.java index eebf2c9cef184..493375c451745 100644 --- a/client/rest-high-level/src/test/java/org/elasticsearch/client/watcher/hlrc/DeleteWatchResponseTests.java +++ b/client/rest-high-level/src/test/java/org/elasticsearch/client/watcher/hlrc/DeleteWatchResponseTests.java @@ -18,17 +18,19 @@ */ package org.elasticsearch.client.watcher.hlrc; +import org.elasticsearch.client.AbstractResponseTestCase; import org.elasticsearch.client.watcher.DeleteWatchResponse; import org.elasticsearch.common.xcontent.XContentParser; -import org.elasticsearch.client.AbstractHlrcXContentTestCase; import java.io.IOException; -public class DeleteWatchResponseTests extends AbstractHlrcXContentTestCase< +import static org.hamcrest.Matchers.equalTo; + +public class DeleteWatchResponseTests extends AbstractResponseTestCase< org.elasticsearch.protocol.xpack.watcher.DeleteWatchResponse, DeleteWatchResponse> { @Override - protected org.elasticsearch.protocol.xpack.watcher.DeleteWatchResponse createTestInstance() { + protected org.elasticsearch.protocol.xpack.watcher.DeleteWatchResponse createServerTestInstance() { String id = randomAlphaOfLength(10); long version = randomLongBetween(1, 10); boolean found = randomBoolean(); @@ -36,23 +38,15 @@ protected org.elasticsearch.protocol.xpack.watcher.DeleteWatchResponse createTes } @Override - protected org.elasticsearch.protocol.xpack.watcher.DeleteWatchResponse doParseInstance(XContentParser parser) throws IOException { - return org.elasticsearch.protocol.xpack.watcher.DeleteWatchResponse.fromXContent(parser); - } - - @Override - public DeleteWatchResponse doHlrcParseInstance(XContentParser parser) throws IOException { + protected DeleteWatchResponse doParseToClientInstance(XContentParser parser) throws IOException { return DeleteWatchResponse.fromXContent(parser); } @Override - public org.elasticsearch.protocol.xpack.watcher.DeleteWatchResponse convertHlrcToInternal(DeleteWatchResponse instance) { - return new org.elasticsearch.protocol.xpack.watcher.DeleteWatchResponse(instance.getId(), instance.getVersion(), - instance.isFound()); - } - - @Override - protected boolean supportsUnknownFields() { - return false; + protected void assertInstances(org.elasticsearch.protocol.xpack.watcher.DeleteWatchResponse serverTestInstance, + DeleteWatchResponse clientInstance) { + assertThat(clientInstance.getId(), equalTo(serverTestInstance.getId())); + assertThat(clientInstance.getVersion(), equalTo(serverTestInstance.getVersion())); + assertThat(clientInstance.isFound(), equalTo(serverTestInstance.isFound())); } } diff --git a/client/rest-high-level/src/test/java/org/elasticsearch/client/watcher/hlrc/ExecuteWatchResponseTests.java b/client/rest-high-level/src/test/java/org/elasticsearch/client/watcher/hlrc/ExecuteWatchResponseTests.java index ace75517a9333..c1492eb53020f 100644 --- a/client/rest-high-level/src/test/java/org/elasticsearch/client/watcher/hlrc/ExecuteWatchResponseTests.java +++ b/client/rest-high-level/src/test/java/org/elasticsearch/client/watcher/hlrc/ExecuteWatchResponseTests.java @@ -19,31 +19,23 @@ package org.elasticsearch.client.watcher.hlrc; +import org.elasticsearch.client.AbstractResponseTestCase; import org.elasticsearch.common.bytes.BytesReference; import org.elasticsearch.common.xcontent.XContentBuilder; import org.elasticsearch.common.xcontent.XContentFactory; import org.elasticsearch.common.xcontent.XContentParser; import org.elasticsearch.common.xcontent.XContentType; -import org.elasticsearch.client.AbstractHlrcXContentTestCase; import org.elasticsearch.xpack.core.watcher.transport.actions.execute.ExecuteWatchResponse; import java.io.IOException; -public class ExecuteWatchResponseTests - extends AbstractHlrcXContentTestCase { +import static org.hamcrest.Matchers.equalTo; - @Override - public org.elasticsearch.client.watcher.ExecuteWatchResponse doHlrcParseInstance(XContentParser parser) throws IOException { - return org.elasticsearch.client.watcher.ExecuteWatchResponse.fromXContent(parser); - } +public class ExecuteWatchResponseTests extends AbstractResponseTestCase< + ExecuteWatchResponse, org.elasticsearch.client.watcher.ExecuteWatchResponse> { @Override - public ExecuteWatchResponse convertHlrcToInternal(org.elasticsearch.client.watcher.ExecuteWatchResponse instance) { - return new ExecuteWatchResponse(instance.getRecordId(), instance.getRecord(), XContentType.JSON); - } - - @Override - protected ExecuteWatchResponse createTestInstance() { + protected ExecuteWatchResponse createServerTestInstance() { String id = "my_watch_0-2015-06-02T23:17:55.124Z"; try { XContentBuilder builder = XContentFactory.jsonBuilder(); @@ -66,12 +58,14 @@ protected ExecuteWatchResponse createTestInstance() { } @Override - protected ExecuteWatchResponse doParseInstance(XContentParser parser) throws IOException { - return ExecuteWatchResponse.fromXContent(parser); + protected org.elasticsearch.client.watcher.ExecuteWatchResponse doParseToClientInstance(XContentParser parser) throws IOException { + return org.elasticsearch.client.watcher.ExecuteWatchResponse.fromXContent(parser); } @Override - protected boolean supportsUnknownFields() { - return false; + protected void assertInstances(ExecuteWatchResponse serverTestInstance, + org.elasticsearch.client.watcher.ExecuteWatchResponse clientInstance) { + assertThat(clientInstance.getRecordId(), equalTo(serverTestInstance.getRecordId())); + assertThat(clientInstance.getRecordAsMap(), equalTo(serverTestInstance.getRecordSource().getAsMap())); } } diff --git a/client/rest-high-level/src/test/java/org/elasticsearch/client/watcher/hlrc/PutWatchResponseTests.java b/client/rest-high-level/src/test/java/org/elasticsearch/client/watcher/hlrc/PutWatchResponseTests.java index 9b65618cafc46..a47de0d15fda6 100644 --- a/client/rest-high-level/src/test/java/org/elasticsearch/client/watcher/hlrc/PutWatchResponseTests.java +++ b/client/rest-high-level/src/test/java/org/elasticsearch/client/watcher/hlrc/PutWatchResponseTests.java @@ -18,17 +18,19 @@ */ package org.elasticsearch.client.watcher.hlrc; +import org.elasticsearch.client.AbstractResponseTestCase; import org.elasticsearch.client.watcher.PutWatchResponse; import org.elasticsearch.common.xcontent.XContentParser; -import org.elasticsearch.client.AbstractHlrcXContentTestCase; import java.io.IOException; -public class PutWatchResponseTests extends AbstractHlrcXContentTestCase< +import static org.hamcrest.Matchers.equalTo; + +public class PutWatchResponseTests extends AbstractResponseTestCase< org.elasticsearch.protocol.xpack.watcher.PutWatchResponse, PutWatchResponse> { @Override - protected org.elasticsearch.protocol.xpack.watcher.PutWatchResponse createTestInstance() { + protected org.elasticsearch.protocol.xpack.watcher.PutWatchResponse createServerTestInstance() { String id = randomAlphaOfLength(10); long seqNo = randomNonNegativeLong(); long primaryTerm = randomLongBetween(1, 20); @@ -38,23 +40,17 @@ protected org.elasticsearch.protocol.xpack.watcher.PutWatchResponse createTestIn } @Override - protected org.elasticsearch.protocol.xpack.watcher.PutWatchResponse doParseInstance(XContentParser parser) throws IOException { - return org.elasticsearch.protocol.xpack.watcher.PutWatchResponse.fromXContent(parser); - } - - @Override - public PutWatchResponse doHlrcParseInstance(XContentParser parser) throws IOException { - return org.elasticsearch.client.watcher.PutWatchResponse.fromXContent(parser); - } - - @Override - public org.elasticsearch.protocol.xpack.watcher.PutWatchResponse convertHlrcToInternal(PutWatchResponse instance) { - return new org.elasticsearch.protocol.xpack.watcher.PutWatchResponse(instance.getId(), instance.getVersion(), - instance.getSeqNo(), instance.getPrimaryTerm(), instance.isCreated()); + protected PutWatchResponse doParseToClientInstance(XContentParser parser) throws IOException { + return PutWatchResponse.fromXContent(parser); } @Override - protected boolean supportsUnknownFields() { - return false; + protected void assertInstances(org.elasticsearch.protocol.xpack.watcher.PutWatchResponse serverTestInstance, + PutWatchResponse clientInstance) { + assertThat(clientInstance.getId(), equalTo(serverTestInstance.getId())); + assertThat(clientInstance.getSeqNo(), equalTo(serverTestInstance.getSeqNo())); + assertThat(clientInstance.getPrimaryTerm(), equalTo(serverTestInstance.getPrimaryTerm())); + assertThat(clientInstance.getVersion(), equalTo(serverTestInstance.getVersion())); + assertThat(clientInstance.isCreated(), equalTo(serverTestInstance.isCreated())); } } diff --git a/client/rest/build.gradle b/client/rest/build.gradle index ee0317457118d..352b15699d081 100644 --- a/client/rest/build.gradle +++ b/client/rest/build.gradle @@ -44,7 +44,7 @@ dependencies { compile "commons-codec:commons-codec:${versions.commonscodec}" compile "commons-logging:commons-logging:${versions.commonslogging}" - testCompile "org.elasticsearch.client:test:${version}" + testCompile project(":client:test") testCompile "com.carrotsearch.randomizedtesting:randomizedtesting-runner:${versions.randomizedrunner}" testCompile "junit:junit:${versions.junit}" testCompile "org.hamcrest:hamcrest:${versions.hamcrest}" @@ -68,7 +68,7 @@ forbiddenApisTest { } // JarHell is part of es server, which we don't want to pull in -// TODO: Not anymore. Now in :libs:core +// TODO: Not anymore. Now in :libs:elasticsearch-core jarHell.enabled=false testingConventions { diff --git a/client/rest/src/main/java/org/elasticsearch/client/Request.java b/client/rest/src/main/java/org/elasticsearch/client/Request.java index 2e4733201b12c..0bf9202486427 100644 --- a/client/rest/src/main/java/org/elasticsearch/client/Request.java +++ b/client/rest/src/main/java/org/elasticsearch/client/Request.java @@ -81,6 +81,10 @@ public void addParameter(String name, String value) { } } + public void addParameters(Map paramSource){ + paramSource.forEach(this::addParameter); + } + /** * Query string parameters. The returned map is an unmodifiable view of the * map in the request so calls to {@link #addParameter(String, String)} diff --git a/client/rest/src/main/java/org/elasticsearch/client/RequestOptions.java b/client/rest/src/main/java/org/elasticsearch/client/RequestOptions.java index 0b2cdce3d52f7..3758b4c489ca2 100644 --- a/client/rest/src/main/java/org/elasticsearch/client/RequestOptions.java +++ b/client/rest/src/main/java/org/elasticsearch/client/RequestOptions.java @@ -170,10 +170,11 @@ public RequestOptions build() { /** * Add the provided header to the request. */ - public void addHeader(String name, String value) { + public Builder addHeader(String name, String value) { Objects.requireNonNull(name, "header name cannot be null"); Objects.requireNonNull(value, "header value cannot be null"); this.headers.add(new ReqHeader(name, value)); + return this; } /** @@ -182,9 +183,10 @@ public void addHeader(String name, String value) { * response body gets streamed from a non-blocking HTTP connection on the * client side. */ - public void setHttpAsyncResponseConsumerFactory(HttpAsyncResponseConsumerFactory httpAsyncResponseConsumerFactory) { + public Builder setHttpAsyncResponseConsumerFactory(HttpAsyncResponseConsumerFactory httpAsyncResponseConsumerFactory) { this.httpAsyncResponseConsumerFactory = Objects.requireNonNull(httpAsyncResponseConsumerFactory, "httpAsyncResponseConsumerFactory cannot be null"); + return this; } /** @@ -204,8 +206,9 @@ public void setHttpAsyncResponseConsumerFactory(HttpAsyncResponseConsumerFactory * fail the request if the warnings returned don't * exactly match some set. */ - public void setWarningsHandler(WarningsHandler warningsHandler) { + public Builder setWarningsHandler(WarningsHandler warningsHandler) { this.warningsHandler = warningsHandler; + return this; } } diff --git a/client/rest/src/test/java/org/elasticsearch/client/RestClientBuilderIntegTests.java b/client/rest/src/test/java/org/elasticsearch/client/RestClientBuilderIntegTests.java index 4aa69d3b3e9c1..8bc8d73fd2b26 100644 --- a/client/rest/src/test/java/org/elasticsearch/client/RestClientBuilderIntegTests.java +++ b/client/rest/src/test/java/org/elasticsearch/client/RestClientBuilderIntegTests.java @@ -134,7 +134,7 @@ private static SSLContext getSslContext() throws Exception { * 12.0.1 so we pin to TLSv1.2 when running on an earlier JDK */ private static String getProtocol() { - String version = AccessController.doPrivileged((PrivilegedAction) () -> System.getProperty("java.version")); + String version = AccessController.doPrivileged((PrivilegedAction) () -> System.getProperty("java.specification.version")); String[] components = version.split("\\."); if (components.length > 0) { final int major = Integer.valueOf(components[0]); diff --git a/client/sniffer/build.gradle b/client/sniffer/build.gradle index 2f9eeca6020c4..8b4d21537a109 100644 --- a/client/sniffer/build.gradle +++ b/client/sniffer/build.gradle @@ -35,14 +35,14 @@ publishing { } dependencies { - compile "org.elasticsearch.client:elasticsearch-rest-client:${version}" + compile project(":client:rest") compile "org.apache.httpcomponents:httpclient:${versions.httpclient}" compile "org.apache.httpcomponents:httpcore:${versions.httpcore}" compile "commons-codec:commons-codec:${versions.commonscodec}" compile "commons-logging:commons-logging:${versions.commonslogging}" compile "com.fasterxml.jackson.core:jackson-core:${versions.jackson}" - testCompile "org.elasticsearch.client:test:${version}" + testCompile project(":client:test") testCompile "com.carrotsearch.randomizedtesting:randomizedtesting-runner:${versions.randomizedrunner}" testCompile "junit:junit:${versions.junit}" testCompile "org.elasticsearch:securemock:${versions.securemock}" @@ -68,7 +68,7 @@ dependencyLicenses { } // JarHell is part of es server, which we don't want to pull in -// TODO: Not anymore. Now in :libs:core +// TODO: Not anymore. Now in :libs:elasticsearch-core jarHell.enabled=false testingConventions { diff --git a/client/test/build.gradle b/client/test/build.gradle index 184606e360791..169172736ecd3 100644 --- a/client/test/build.gradle +++ b/client/test/build.gradle @@ -44,7 +44,7 @@ forbiddenApisTest { } // JarHell is part of es server, which we don't want to pull in -// TODO: Not anymore. Now in :libs:core +// TODO: Not anymore. Now in :libs:elasticsearch-core jarHell.enabled=false // TODO: should we have licenses for our test deps? diff --git a/client/transport/build.gradle b/client/transport/build.gradle deleted file mode 100644 index c1e4503445bb7..0000000000000 --- a/client/transport/build.gradle +++ /dev/null @@ -1,57 +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. - */ -apply plugin: 'elasticsearch.build' -apply plugin: 'nebula.maven-base-publish' -apply plugin: 'nebula.maven-scm' - -group = 'org.elasticsearch.client' - -dependencies { - compile "org.elasticsearch:elasticsearch:${version}" - compile "org.elasticsearch.plugin:transport-netty4-client:${version}" - compile "org.elasticsearch.plugin:reindex-client:${version}" - compile "org.elasticsearch.plugin:lang-mustache-client:${version}" - compile "org.elasticsearch.plugin:percolator-client:${version}" - compile "org.elasticsearch.plugin:parent-join-client:${version}" - compile "org.elasticsearch.plugin:rank-eval-client:${version}" - testCompile "com.carrotsearch.randomizedtesting:randomizedtesting-runner:${versions.randomizedrunner}" - testCompile "junit:junit:${versions.junit}" - testCompile "org.hamcrest:hamcrest:${versions.hamcrest}" -} - -dependencyLicenses { - dependencies = project.configurations.runtime.fileCollection { - it.group.startsWith('org.elasticsearch') == false - } -} - -forbiddenApisTest { - // we don't use the core test-framework, no lucene classes present so we don't want the es-test-signatures to - // be pulled in - replaceSignatureFiles 'jdk-signatures', 'es-all-signatures' -} - -testingConventions { - naming.clear() - naming { - Tests { - baseClass 'com.carrotsearch.randomizedtesting.RandomizedTest' - } - } -} diff --git a/client/transport/src/main/java/org/elasticsearch/transport/client/PreBuiltTransportClient.java b/client/transport/src/main/java/org/elasticsearch/transport/client/PreBuiltTransportClient.java deleted file mode 100644 index 4a78c38d06cc6..0000000000000 --- a/client/transport/src/main/java/org/elasticsearch/transport/client/PreBuiltTransportClient.java +++ /dev/null @@ -1,144 +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.transport.client; - -import io.netty.util.ThreadDeathWatcher; -import io.netty.util.concurrent.GlobalEventExecutor; -import org.elasticsearch.client.transport.TransportClient; -import org.elasticsearch.common.SuppressForbidden; -import org.elasticsearch.common.network.NetworkModule; -import org.elasticsearch.common.settings.Settings; -import org.elasticsearch.index.reindex.ReindexPlugin; -import org.elasticsearch.join.ParentJoinPlugin; -import org.elasticsearch.percolator.PercolatorPlugin; -import org.elasticsearch.plugins.Plugin; -import org.elasticsearch.script.mustache.MustachePlugin; -import org.elasticsearch.transport.Netty4Plugin; - -import java.util.Arrays; -import java.util.Collection; -import java.util.List; -import java.util.concurrent.TimeUnit; - -/** - * A builder to create an instance of {@link TransportClient}. This class pre-installs the - * {@link Netty4Plugin}, - * {@link ReindexPlugin}, - * {@link PercolatorPlugin}, - * {@link MustachePlugin}, - * {@link ParentJoinPlugin} - * plugins for the client. These plugins are all the required modules for Elasticsearch. - * - * @deprecated {@link TransportClient} is deprecated in favour of the High Level REST client and will - * be removed in Elasticsearch 8.0. - */ -@SuppressWarnings({"unchecked","varargs"}) -@Deprecated -public class PreBuiltTransportClient extends TransportClient { - - static { - // initialize Netty system properties before triggering any Netty class loads - initializeNetty(); - } - - /** - * Netty wants to do some unwelcome things like use unsafe and replace a private field, or use a poorly considered buffer recycler. This - * method disables these things by default, but can be overridden by setting the corresponding system properties. - */ - private static void initializeNetty() { - /* - * We disable three pieces of Netty functionality here: - * - we disable Netty from being unsafe - * - we disable Netty from replacing the selector key set - * - we disable Netty from using the recycler - * - * While permissions are needed to read and set these, the permissions needed here are innocuous and thus should simply be granted - * rather than us handling a security exception here. - */ - setSystemPropertyIfUnset("io.netty.noUnsafe", Boolean.toString(true)); - setSystemPropertyIfUnset("io.netty.noKeySetOptimization", Boolean.toString(true)); - setSystemPropertyIfUnset("io.netty.recycler.maxCapacityPerThread", Integer.toString(0)); - } - - @SuppressForbidden(reason = "set system properties to configure Netty") - private static void setSystemPropertyIfUnset(final String key, final String value) { - final String currentValue = System.getProperty(key); - if (currentValue == null) { - System.setProperty(key, value); - } - } - - private static final Collection> PRE_INSTALLED_PLUGINS = - List.of(Netty4Plugin.class, ReindexPlugin.class, PercolatorPlugin.class, MustachePlugin.class, ParentJoinPlugin.class); - - /** - * Creates a new transport client with pre-installed plugins. - * - * @param settings the settings passed to this transport client - * @param plugins an optional array of additional plugins to run with this client - */ - @SafeVarargs - public PreBuiltTransportClient(Settings settings, Class... plugins) { - this(settings, Arrays.asList(plugins)); - } - - /** - * Creates a new transport client with pre-installed plugins. - * - * @param settings the settings passed to this transport client - * @param plugins a collection of additional plugins to run with this client - */ - public PreBuiltTransportClient(Settings settings, Collection> plugins) { - this(settings, plugins, null); - } - - /** - * Creates a new transport client with pre-installed plugins. - * - * @param settings the settings passed to this transport client - * @param plugins a collection of additional plugins to run with this client - * @param hostFailureListener a failure listener that is invoked if a node is disconnected; this can be null - */ - public PreBuiltTransportClient( - Settings settings, - Collection> plugins, - HostFailureListener hostFailureListener) { - super(settings, Settings.EMPTY, addPlugins(plugins, PRE_INSTALLED_PLUGINS), hostFailureListener); - } - - @Override - public void close() { - super.close(); - if (NetworkModule.TRANSPORT_TYPE_SETTING.exists(settings) == false - || NetworkModule.TRANSPORT_TYPE_SETTING.get(settings).equals(Netty4Plugin.NETTY_TRANSPORT_NAME)) { - try { - GlobalEventExecutor.INSTANCE.awaitInactivity(5, TimeUnit.SECONDS); - } catch (InterruptedException e) { - Thread.currentThread().interrupt(); - } - try { - ThreadDeathWatcher.awaitInactivity(5, TimeUnit.SECONDS); - } catch (InterruptedException e) { - Thread.currentThread().interrupt(); - } - } - } - -} diff --git a/client/transport/src/test/java/org/elasticsearch/transport/client/PreBuiltTransportClientTests.java b/client/transport/src/test/java/org/elasticsearch/transport/client/PreBuiltTransportClientTests.java deleted file mode 100644 index dbcfa8715f5f4..0000000000000 --- a/client/transport/src/test/java/org/elasticsearch/transport/client/PreBuiltTransportClientTests.java +++ /dev/null @@ -1,66 +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.transport.client; - -import com.carrotsearch.randomizedtesting.RandomizedTest; -import org.elasticsearch.client.transport.TransportClient; -import org.elasticsearch.common.network.NetworkModule; -import org.elasticsearch.common.settings.Settings; -import org.elasticsearch.index.reindex.ReindexPlugin; -import org.elasticsearch.join.ParentJoinPlugin; -import org.elasticsearch.percolator.PercolatorPlugin; -import org.elasticsearch.plugins.Plugin; -import org.elasticsearch.script.mustache.MustachePlugin; -import org.elasticsearch.transport.Netty4Plugin; -import org.junit.Test; - -import java.util.Arrays; - -import static org.junit.Assert.assertEquals; -import static org.junit.Assert.assertTrue; -import static org.junit.Assert.fail; - -public class PreBuiltTransportClientTests extends RandomizedTest { - - @Test - public void testPluginInstalled() { - try (TransportClient client = new PreBuiltTransportClient(Settings.EMPTY)) { - Settings settings = client.settings(); - assertEquals(Netty4Plugin.NETTY_TRANSPORT_NAME, NetworkModule.HTTP_DEFAULT_TYPE_SETTING.get(settings)); - assertEquals(Netty4Plugin.NETTY_TRANSPORT_NAME, NetworkModule.TRANSPORT_DEFAULT_TYPE_SETTING.get(settings)); - } - } - - @Test - public void testInstallPluginTwice() { - for (Class plugin : - Arrays.asList(ParentJoinPlugin.class, ReindexPlugin.class, PercolatorPlugin.class, - MustachePlugin.class)) { - try { - new PreBuiltTransportClient(Settings.EMPTY, plugin); - fail("exception expected"); - } catch (IllegalArgumentException ex) { - assertTrue("Expected message to start with [plugin already exists: ] but was instead [" + ex.getMessage() + "]", - ex.getMessage().startsWith("plugin already exists: ")); - } - } - } - -} diff --git a/distribution/archives/build.gradle b/distribution/archives/build.gradle index 14a7e566587c4..ad4c77aaa5e5b 100644 --- a/distribution/archives/build.gradle +++ b/distribution/archives/build.gradle @@ -36,13 +36,13 @@ apply plugin: 'base' // parent to copy to the root of the distribution ext.logsDir = new File(buildDir, 'logs-hack/logs') task createLogsDir(type: EmptyDirTask) { - dir "${logsDir}" - dirMode 0755 + dir = "${logsDir}" + dirMode = 0755 } ext.pluginsDir= new File(buildDir, 'plugins-hack/plugins') task createPluginsDir(type: EmptyDirTask) { - dir "${pluginsDir}" - dirMode 0755 + dir = "${pluginsDir}" + dirMode = 0755 } CopySpec archiveFiles(CopySpec modulesFiles, String distributionType, String platform, boolean oss, boolean jdk) { diff --git a/distribution/build.gradle b/distribution/build.gradle index 9c98101da5be3..9606604036101 100644 --- a/distribution/build.gradle +++ b/distribution/build.gradle @@ -117,7 +117,7 @@ task buildTransportModules { void copyModule(Sync copyTask, Project module) { copyTask.configure { - dependsOn { module.bundlePlugin } + dependsOn "${module.path}:bundlePlugin" from({ zipTree(module.bundlePlugin.outputs.files.singleFile) }) { includeEmptyDirs false @@ -167,6 +167,7 @@ buildDefaultLog4jConfig.doLast(writeLog4jProperties) // copy log4j2.properties from modules that have it void copyLog4jProperties(Task buildTask, Project module) { + buildTask.dependsOn "${module.path}:bundlePlugin" buildTask.doFirst { FileTree tree = zipTree(module.bundlePlugin.outputs.files.singleFile) FileTree filtered = tree.matching { @@ -241,7 +242,7 @@ configure(subprojects.findAll { ['archives', 'packages'].contains(it.name) }) { // delay by using closures, since they have not yet been configured, so no jar task exists yet from { project(':server').jar } from { project(':server').configurations.runtime } - from { project(':libs:plugin-classloader').jar } + from { project(':libs:elasticsearch-plugin-classloader').jar } from { project(':distribution:tools:java-version-checker').jar } from { project(':distribution:tools:launchers').jar } into('tools/plugin-cli') { @@ -381,6 +382,9 @@ configure(subprojects.findAll { ['archives', 'packages'].contains(it.name) }) { if (details.relativePath.segments[-2] == 'bin' || details.relativePath.segments[-1] == 'jspawnhelper') { details.mode = 0755 } + if (details.name == 'src.zip') { + details.exclude() + } } } } @@ -397,11 +401,11 @@ task run(type: RunTask) { setting 'xpack.graph.enabled', 'true' setting 'xpack.watcher.enabled', 'true' setting 'xpack.license.self_generated.type', 'trial' - setupCommand 'setupTestAdmin', - 'bin/elasticsearch-users', 'useradd', 'elastic-admin', '-p', 'elastic-password', '-r', 'superuser' } else if (licenseType != 'basic') { throw new IllegalArgumentException("Unsupported self-generated license type: [" + licenseType + "[basic] or [trial].") } + setupCommand 'setupTestAdmin', + 'bin/elasticsearch-users', 'useradd', 'elastic-admin', '-p', 'elastic-password', '-r', 'superuser' setting 'xpack.security.enabled', 'true' setting 'xpack.monitoring.enabled', 'true' setting 'xpack.sql.enabled', 'true' diff --git a/distribution/docker/build.gradle b/distribution/docker/build.gradle index ab55c737783e5..46f456c8c0493 100644 --- a/distribution/docker/build.gradle +++ b/distribution/docker/build.gradle @@ -2,6 +2,7 @@ import org.elasticsearch.gradle.BuildPlugin import org.elasticsearch.gradle.LoggedExec import org.elasticsearch.gradle.MavenFilteringHack import org.elasticsearch.gradle.VersionProperties +import org.elasticsearch.gradle.testfixtures.TestFixturesPlugin apply plugin: 'base' apply plugin: 'elasticsearch.test.fixtures' @@ -44,7 +45,12 @@ project.ext { } into('config') { + /* + * Oss and default distribution can have different configuration, therefore we want to allow overriding the default configuration + * by creating config files in oss or default build-context sub-modules. + */ from project.projectDir.toPath().resolve("src/docker/config") + from project.projectDir.toPath().resolve(oss ? "oss-docker-build-context" : "docker-build-context").resolve("src/docker/config") } from(project.projectDir.toPath().resolve("src/docker/Dockerfile")) { @@ -72,7 +78,10 @@ void addCopyDockerContextTask(final boolean oss) { } preProcessFixture { - dependsOn assemble + // don't add the tasks to build the docker images if we have no way of testing them + if (TestFixturesPlugin.dockerComposeSupported()) { + dependsOn assemble + } } postProcessFixture.doLast { diff --git a/distribution/docker/docker-build-context/src/docker/config/log4j2.properties b/distribution/docker/docker-build-context/src/docker/config/log4j2.properties new file mode 100644 index 0000000000000..40be55d2e1c3a --- /dev/null +++ b/distribution/docker/docker-build-context/src/docker/config/log4j2.properties @@ -0,0 +1,121 @@ +status = error + +# log action execution errors for easier debugging +logger.action.name = org.elasticsearch.action +logger.action.level = debug + +appender.rolling.type = Console +appender.rolling.name = rolling +appender.rolling.layout.type = ESJsonLayout +appender.rolling.layout.type_name = server + +rootLogger.level = info +rootLogger.appenderRef.rolling.ref = rolling + +appender.deprecation_rolling.type = Console +appender.deprecation_rolling.name = deprecation_rolling +appender.deprecation_rolling.layout.type = ESJsonLayout +appender.deprecation_rolling.layout.type_name = deprecation + +logger.deprecation.name = org.elasticsearch.deprecation +logger.deprecation.level = warn +logger.deprecation.appenderRef.deprecation_rolling.ref = deprecation_rolling +logger.deprecation.additivity = false + +appender.index_search_slowlog_rolling.type = Console +appender.index_search_slowlog_rolling.name = index_search_slowlog_rolling +appender.index_search_slowlog_rolling.layout.type = ESJsonLayout +appender.index_search_slowlog_rolling.layout.type_name = index_search_slowlog + +logger.index_search_slowlog_rolling.name = index.search.slowlog +logger.index_search_slowlog_rolling.level = trace +logger.index_search_slowlog_rolling.appenderRef.index_search_slowlog_rolling.ref = index_search_slowlog_rolling +logger.index_search_slowlog_rolling.additivity = false + +appender.index_indexing_slowlog_rolling.type = Console +appender.index_indexing_slowlog_rolling.name = index_indexing_slowlog_rolling +appender.index_indexing_slowlog_rolling.layout.type = ESJsonLayout +appender.index_indexing_slowlog_rolling.layout.type_name = index_indexing_slowlog + +logger.index_indexing_slowlog.name = index.indexing.slowlog.index +logger.index_indexing_slowlog.level = trace +logger.index_indexing_slowlog.appenderRef.index_indexing_slowlog_rolling.ref = index_indexing_slowlog_rolling +logger.index_indexing_slowlog.additivity = false + +appender.audit_rolling.type = Console +appender.audit_rolling.name = audit_rolling +appender.audit_rolling.layout.type = PatternLayout +appender.audit_rolling.layout.pattern = {\ +"type": "audit", \ +"timestamp":"%d{yyyy-MM-dd'T'HH:mm:ss,SSSZ}"\ +%varsNotEmpty{, "node.name":"%enc{%map{node.name}}{JSON}"}\ +%varsNotEmpty{, "node.id":"%enc{%map{node.id}}{JSON}"}\ +%varsNotEmpty{, "host.name":"%enc{%map{host.name}}{JSON}"}\ +%varsNotEmpty{, "host.ip":"%enc{%map{host.ip}}{JSON}"}\ +%varsNotEmpty{, "event.type":"%enc{%map{event.type}}{JSON}"}\ +%varsNotEmpty{, "event.action":"%enc{%map{event.action}}{JSON}"}\ +%varsNotEmpty{, "user.name":"%enc{%map{user.name}}{JSON}"}\ +%varsNotEmpty{, "user.run_by.name":"%enc{%map{user.run_by.name}}{JSON}"}\ +%varsNotEmpty{, "user.run_as.name":"%enc{%map{user.run_as.name}}{JSON}"}\ +%varsNotEmpty{, "user.realm":"%enc{%map{user.realm}}{JSON}"}\ +%varsNotEmpty{, "user.run_by.realm":"%enc{%map{user.run_by.realm}}{JSON}"}\ +%varsNotEmpty{, "user.run_as.realm":"%enc{%map{user.run_as.realm}}{JSON}"}\ +%varsNotEmpty{, "user.roles":%map{user.roles}}\ +%varsNotEmpty{, "origin.type":"%enc{%map{origin.type}}{JSON}"}\ +%varsNotEmpty{, "origin.address":"%enc{%map{origin.address}}{JSON}"}\ +%varsNotEmpty{, "realm":"%enc{%map{realm}}{JSON}"}\ +%varsNotEmpty{, "url.path":"%enc{%map{url.path}}{JSON}"}\ +%varsNotEmpty{, "url.query":"%enc{%map{url.query}}{JSON}"}\ +%varsNotEmpty{, "request.method":"%enc{%map{request.method}}{JSON}"}\ +%varsNotEmpty{, "request.body":"%enc{%map{request.body}}{JSON}"}\ +%varsNotEmpty{, "request.id":"%enc{%map{request.id}}{JSON}"}\ +%varsNotEmpty{, "action":"%enc{%map{action}}{JSON}"}\ +%varsNotEmpty{, "request.name":"%enc{%map{request.name}}{JSON}"}\ +%varsNotEmpty{, "indices":%map{indices}}\ +%varsNotEmpty{, "opaque_id":"%enc{%map{opaque_id}}{JSON}"}\ +%varsNotEmpty{, "x_forwarded_for":"%enc{%map{x_forwarded_for}}{JSON}"}\ +%varsNotEmpty{, "transport.profile":"%enc{%map{transport.profile}}{JSON}"}\ +%varsNotEmpty{, "rule":"%enc{%map{rule}}{JSON}"}\ +%varsNotEmpty{, "event.category":"%enc{%map{event.category}}{JSON}"}\ +}%n +# "node.name" node name from the `elasticsearch.yml` settings +# "node.id" node id which should not change between cluster restarts +# "host.name" unresolved hostname of the local node +# "host.ip" the local bound ip (i.e. the ip listening for connections) +# "event.type" a received REST request is translated into one or more transport requests. This indicates which processing layer generated the event "rest" or "transport" (internal) +# "event.action" the name of the audited event, eg. "authentication_failed", "access_granted", "run_as_granted", etc. +# "user.name" the subject name as authenticated by a realm +# "user.run_by.name" the original authenticated subject name that is impersonating another one. +# "user.run_as.name" if this "event.action" is of a run_as type, this is the subject name to be impersonated as. +# "user.realm" the name of the realm that authenticated "user.name" +# "user.run_by.realm" the realm name of the impersonating subject ("user.run_by.name") +# "user.run_as.realm" if this "event.action" is of a run_as type, this is the realm name the impersonated user is looked up from +# "user.roles" the roles array of the user; these are the roles that are granting privileges +# "origin.type" it is "rest" if the event is originating (is in relation to) a REST request; possible other values are "transport" and "ip_filter" +# "origin.address" the remote address and port of the first network hop, i.e. a REST proxy or another cluster node +# "realm" name of a realm that has generated an "authentication_failed" or an "authentication_successful"; the subject is not yet authenticated +# "url.path" the URI component between the port and the query string; it is percent (URL) encoded +# "url.query" the URI component after the path and before the fragment; it is percent (URL) encoded +# "request.method" the method of the HTTP request, i.e. one of GET, POST, PUT, DELETE, OPTIONS, HEAD, PATCH, TRACE, CONNECT +# "request.body" the content of the request body entity, JSON escaped +# "request.id" a synthentic identifier for the incoming request, this is unique per incoming request, and consistent across all audit events generated by that request +# "action" an action is the most granular operation that is authorized and this identifies it in a namespaced way (internal) +# "request.name" if the event is in connection to a transport message this is the name of the request class, similar to how rest requests are identified by the url path (internal) +# "indices" the array of indices that the "action" is acting upon +# "opaque_id" opaque value conveyed by the "X-Opaque-Id" request header +# "x_forwarded_for" the addresses from the "X-Forwarded-For" request header, as a verbatim string value (not an array) +# "transport.profile" name of the transport profile in case this is a "connection_granted" or "connection_denied" event +# "rule" name of the applied rulee if the "origin.type" is "ip_filter" +# "event.category" fixed value "elasticsearch-audit" + +logger.xpack_security_audit_logfile.name = org.elasticsearch.xpack.security.audit.logfile.LoggingAuditTrail +logger.xpack_security_audit_logfile.level = info +logger.xpack_security_audit_logfile.appenderRef.audit_rolling.ref = audit_rolling +logger.xpack_security_audit_logfile.additivity = false + +logger.xmlsig.name = org.apache.xml.security.signature.XMLSignature +logger.xmlsig.level = error +logger.samlxml_decrypt.name = org.opensaml.xmlsec.encryption.support.Decrypter +logger.samlxml_decrypt.level = fatal +logger.saml2_decrypt.name = org.opensaml.saml.saml2.encryption.Decrypter +logger.saml2_decrypt.level = fatal diff --git a/distribution/docker/src/docker/config/log4j2.properties b/distribution/docker/oss-docker-build-context/src/docker/config/log4j2.properties similarity index 100% rename from distribution/docker/src/docker/config/log4j2.properties rename to distribution/docker/oss-docker-build-context/src/docker/config/log4j2.properties diff --git a/distribution/src/bin/elasticsearch.bat b/distribution/src/bin/elasticsearch.bat index 8ef77ac4c7fe9..03dc48728d52c 100644 --- a/distribution/src/bin/elasticsearch.bat +++ b/distribution/src/bin/elasticsearch.bat @@ -54,7 +54,7 @@ if "%MAYBE_JVM_OPTIONS_PARSER_FAILED%" == "jvm_options_parser_failed" ( exit /b 1 ) -%JAVA% %ES_JAVA_OPTS% -Delasticsearch -Des.path.home="%ES_HOME%" -Des.path.conf="%ES_PATH_CONF%" -Des.distribution.flavor="%ES_DISTRIBUTION_FLAVOR%" -Des.distribution.type="%ES_DISTRIBUTION_TYPE%" -Des.bundled_jd="%ES_BUNDLED_JDK%" -cp "%ES_CLASSPATH%" "org.elasticsearch.bootstrap.Elasticsearch" !newparams! +%JAVA% %ES_JAVA_OPTS% -Delasticsearch -Des.path.home="%ES_HOME%" -Des.path.conf="%ES_PATH_CONF%" -Des.distribution.flavor="%ES_DISTRIBUTION_FLAVOR%" -Des.distribution.type="%ES_DISTRIBUTION_TYPE%" -Des.bundled_jdk="%ES_BUNDLED_JDK%" -cp "%ES_CLASSPATH%" "org.elasticsearch.bootstrap.Elasticsearch" !newparams! endlocal endlocal diff --git a/distribution/tools/keystore-cli/build.gradle b/distribution/tools/keystore-cli/build.gradle index 5d57ef2e05597..de09e78e37493 100644 --- a/distribution/tools/keystore-cli/build.gradle +++ b/distribution/tools/keystore-cli/build.gradle @@ -20,9 +20,9 @@ apply plugin: 'elasticsearch.build' dependencies { - compileOnly "org.elasticsearch:elasticsearch:${version}" - compileOnly "org.elasticsearch:elasticsearch-cli:${version}" - testCompile "org.elasticsearch.test:framework:${version}" + compileOnly project(":server") + compileOnly project(":libs:elasticsearch-cli") + testCompile project(":test:framework") testCompile 'com.google.jimfs:jimfs:1.1' testCompile 'com.google.guava:guava:18.0' } diff --git a/distribution/tools/keystore-cli/src/test/java/org/elasticsearch/common/settings/KeyStoreWrapperTests.java b/distribution/tools/keystore-cli/src/test/java/org/elasticsearch/common/settings/KeyStoreWrapperTests.java index f68a731edf8f7..568ddfe97df16 100644 --- a/distribution/tools/keystore-cli/src/test/java/org/elasticsearch/common/settings/KeyStoreWrapperTests.java +++ b/distribution/tools/keystore-cli/src/test/java/org/elasticsearch/common/settings/KeyStoreWrapperTests.java @@ -51,10 +51,12 @@ import java.nio.file.Path; import java.security.GeneralSecurityException; import java.security.KeyStore; +import java.security.MessageDigest; import java.security.SecureRandom; import java.util.ArrayList; import java.util.Base64; import java.util.List; +import java.util.Locale; import java.util.Set; import static org.hamcrest.Matchers.containsString; @@ -126,6 +128,27 @@ public void testCannotReadStringFromClosedKeystore() throws Exception { assertThat(exception.getMessage(), containsString("closed")); } + public void testValueSHA256Digest() throws Exception { + final KeyStoreWrapper keystore = KeyStoreWrapper.create(); + final String stringSettingKeyName = randomAlphaOfLength(5).toLowerCase(Locale.ROOT) + "1"; + final String stringSettingValue = randomAlphaOfLength(32); + keystore.setString(stringSettingKeyName, stringSettingValue.toCharArray()); + final String fileSettingKeyName = randomAlphaOfLength(5).toLowerCase(Locale.ROOT) + "2"; + final byte[] fileSettingValue = randomByteArrayOfLength(32); + keystore.setFile(fileSettingKeyName, fileSettingValue); + + final byte[] stringSettingHash = MessageDigest.getInstance("SHA-256").digest(stringSettingValue.getBytes(StandardCharsets.UTF_8)); + assertThat(keystore.getSHA256Digest(stringSettingKeyName), equalTo(stringSettingHash)); + final byte[] fileSettingHash = MessageDigest.getInstance("SHA-256").digest(fileSettingValue); + assertThat(keystore.getSHA256Digest(fileSettingKeyName), equalTo(fileSettingHash)); + + keystore.close(); + + // value hashes accessible even when the keystore is closed + assertThat(keystore.getSHA256Digest(stringSettingKeyName), equalTo(stringSettingHash)); + assertThat(keystore.getSHA256Digest(fileSettingKeyName), equalTo(fileSettingHash)); + } + public void testUpgradeNoop() throws Exception { KeyStoreWrapper keystore = KeyStoreWrapper.create(); SecureString seed = keystore.getString(KeyStoreWrapper.SEED_SETTING.getKey()); diff --git a/distribution/tools/plugin-cli/build.gradle b/distribution/tools/plugin-cli/build.gradle index 48bc899cd29b4..3db958c6ec41e 100644 --- a/distribution/tools/plugin-cli/build.gradle +++ b/distribution/tools/plugin-cli/build.gradle @@ -22,11 +22,11 @@ apply plugin: 'elasticsearch.build' archivesBaseName = 'elasticsearch-plugin-cli' dependencies { - compileOnly "org.elasticsearch:elasticsearch:${version}" - compileOnly "org.elasticsearch:elasticsearch-cli:${version}" + compileOnly project(":server") + compileOnly project(":libs:elasticsearch-cli") compile "org.bouncycastle:bcpg-jdk15on:${versions.bouncycastle}" compile "org.bouncycastle:bcprov-jdk15on:${versions.bouncycastle}" - testCompile "org.elasticsearch.test:framework:${version}" + testCompile project(":test:framework") testCompile 'com.google.jimfs:jimfs:1.1' testCompile 'com.google.guava:guava:18.0' } diff --git a/docs/README.asciidoc b/docs/README.asciidoc index 0a2840b02e7e4..b7fad903f54b0 100644 --- a/docs/README.asciidoc +++ b/docs/README.asciidoc @@ -62,9 +62,9 @@ for its modifiers: them" which looks like `// TESTRESPONSE[s/\d+/$body.$_path/]`. * You can't use `// TESTRESPONSE` immediately after `// TESTSETUP`. Instead, consider using `// TEST[continued]` or rearrange your snippets. - * `// TESTRESPONSE[_cat]`: Add substitutions for testing `_cat` responses. Use - this after all other substitutions so it doesn't make other substitutions - difficult. + * `// TESTRESPONSE[non_json]`: Add substitutions for testing responses in a + format other than JSON. Use this after all other substitutions so it doesn't + make other substitutions difficult. * `// TESTRESPONSE[skip:reason]`: Skip the assertions specified by this response. * `// TESTSETUP`: Marks this snippet as the "setup" for all other snippets in diff --git a/docs/build.gradle b/docs/build.gradle index feda444301ec7..08cb2de971320 100644 --- a/docs/build.gradle +++ b/docs/build.gradle @@ -1137,3 +1137,12 @@ buildRestTests.setups['seats'] = ''' {"theatre": "Graye", "cost": 33, "row": 2, "number": 6, "sold": false} {"index":{"_id": "4"}} {"theatre": "Skyline", "cost": 20, "row": 5, "number": 2, "sold": false}''' +buildRestTests.setups['kibana_sample_data_ecommerce'] = ''' + - do: + indices.create: + index: kibana_sample_data_ecommerce + body: + settings: + number_of_shards: 1 + number_of_replicas: 0 +''' diff --git a/docs/groovy-api/anatomy.asciidoc b/docs/groovy-api/anatomy.asciidoc deleted file mode 100644 index ba7cf83bb00cf..0000000000000 --- a/docs/groovy-api/anatomy.asciidoc +++ /dev/null @@ -1,102 +0,0 @@ -[[anatomy]] -== API Anatomy - -Once a <> has been -obtained, all of Elasticsearch APIs can be executed on it. Each Groovy -API is exposed using three different mechanisms. - - -[[closure]] -=== Closure Request - -The first type is to simply provide the request as a Closure, which -automatically gets resolved into the respective request instance (for -the index API, its the `IndexRequest` class). The API returns a special -future, called `GActionFuture`. This is a groovier version of -Elasticsearch Java `ActionFuture` (in turn a nicer extension to Java own -`Future`) which allows to register listeners (closures) on it for -success and failures, as well as blocking for the response. For example: - -[source,groovy] --------------------------------------------------- -def indexR = client.index { - index "test" - type "_doc" - id "1" - source { - test = "value" - complex { - value1 = "value1" - value2 = "value2" - } - } -} - -println "Indexed $indexR.response.id into $indexR.response.index/$indexR.response.type" --------------------------------------------------- - -In the above example, calling `indexR.response` will simply block for -the response. We can also block for the response for a specific timeout: - -[source,groovy] --------------------------------------------------- -IndexResponse response = indexR.response "5s" // block for 5 seconds, same as: -response = indexR.response 5, TimeValue.SECONDS // --------------------------------------------------- - -We can also register closures that will be called on success and on -failure: - -[source,groovy] --------------------------------------------------- -indexR.success = {IndexResponse response -> - println "Indexed $response.id into $response.index/$response.type" -} -indexR.failure = {Throwable t -> - println "Failed to index: $t.message" -} --------------------------------------------------- - - -[[request]] -=== Request - -This option allows to pass the actual instance of the request (instead -of a closure) as a parameter. The rest is similar to the closure as a -parameter option (the `GActionFuture` handling). For example: - -[source,groovy] --------------------------------------------------- -def indexR = client.index (new IndexRequest( - index: "test", - type: "_doc", - id: "1", - source: { - test = "value" - complex { - value1 = "value1" - value2 = "value2" - } - })) - -println "Indexed $indexR.response.id into $indexR.response.index/$indexR.response.type" --------------------------------------------------- - - -[[java-like]] -=== Java Like - -The last option is to provide an actual instance of the API request, and -an `ActionListener` for the callback. This is exactly like the Java API -with the added `gexecute` which returns the `GActionFuture`: - -[source,groovy] --------------------------------------------------- -def indexR = node.client.prepareIndex("test", "_doc", "1").setSource({ - test = "value" - complex { - value1 = "value1" - value2 = "value2" - } -}).gexecute() --------------------------------------------------- diff --git a/docs/groovy-api/client.asciidoc b/docs/groovy-api/client.asciidoc deleted file mode 100644 index c3c89e71bc535..0000000000000 --- a/docs/groovy-api/client.asciidoc +++ /dev/null @@ -1,59 +0,0 @@ -[[client]] -== Client - -Obtaining an Elasticsearch Groovy `GClient` (a `GClient` is a simple -wrapper on top of the Java `Client`) is simple. The most common way to -get a client is by starting an embedded `Node` which acts as a node -within the cluster. - - -[[node-client]] -=== Node Client - -A Node based client is the simplest form to get a `GClient` to start -executing operations against Elasticsearch. - -[source,groovy] --------------------------------------------------- -import org.elasticsearch.groovy.client.GClient -import org.elasticsearch.groovy.node.GNode -import static org.elasticsearch.groovy.node.GNodeBuilder.nodeBuilder - -// on startup - -GNode node = nodeBuilder().node(); -GClient client = node.client(); - -// on shutdown - -node.close(); --------------------------------------------------- - -Since Elasticsearch allows to configure it using JSON based settings, -the configuration itself can be done using a closure that represent the -JSON: - -[source,groovy] --------------------------------------------------- -import org.elasticsearch.groovy.node.GNode -import org.elasticsearch.groovy.node.GNodeBuilder -import static org.elasticsearch.groovy.node.GNodeBuilder.* - -// on startup - -GNodeBuilder nodeBuilder = nodeBuilder(); -nodeBuilder.settings { - node { - client = true - } - cluster { - name = "test" - } -} - -GNode node = nodeBuilder.node() - -// on shutdown - -node.stop().close() --------------------------------------------------- diff --git a/docs/groovy-api/delete.asciidoc b/docs/groovy-api/delete.asciidoc deleted file mode 100644 index 3d6547820041a..0000000000000 --- a/docs/groovy-api/delete.asciidoc +++ /dev/null @@ -1,16 +0,0 @@ -[[delete]] -== Delete API - -The delete API is very similar to the -// {javaclient}/java-docs-delete.html[] -Java delete API, here is an -example: - -[source,groovy] --------------------------------------------------- -def deleteF = node.client.delete { - index "test" - type "_doc" - id "1" -} --------------------------------------------------- diff --git a/docs/groovy-api/get.asciidoc b/docs/groovy-api/get.asciidoc deleted file mode 100644 index 2cac8429c3e79..0000000000000 --- a/docs/groovy-api/get.asciidoc +++ /dev/null @@ -1,19 +0,0 @@ -[[get]] -== Get API - -The get API is very similar to the -// {javaclient}/java-docs-get.html[] -Java get API. The main benefit -of using groovy is handling the source content. It can be automatically -converted to a `Map` which means using Groovy to navigate it is simple: - -[source,groovy] --------------------------------------------------- -def getF = node.client.get { - index "test" - type "_doc" - id "1" -} - -println "Result of field2: $getF.response.source.complex.field2" --------------------------------------------------- diff --git a/docs/groovy-api/index.asciidoc b/docs/groovy-api/index.asciidoc deleted file mode 100644 index e1bb81856f15a..0000000000000 --- a/docs/groovy-api/index.asciidoc +++ /dev/null @@ -1,48 +0,0 @@ -= Groovy API - -include::../Versions.asciidoc[] - -[preface] -== Preface - -This section describes the http://groovy-lang.org/[Groovy] API -Elasticsearch provides. All Elasticsearch APIs are executed using a -<>, and are completely -asynchronous in nature (they either accept a listener, or return a -future). - -The Groovy API is a wrapper on top of the -{javaclient}[Java API] exposing it in a groovier -manner. The execution options for each API follow a similar manner and -covered in <>. - - -[[maven]] -=== Maven Repository - -The Groovy API is hosted on -http://search.maven.org/#search%7Cga%7C1%7Ca%3A%22elasticsearch-groovy%22[Maven -Central]. - -For example, you can define the latest version in your `pom.xml` file: - -["source","xml",subs="attributes"] --------------------------------------------------- - - org.elasticsearch - elasticsearch-groovy - {version} - --------------------------------------------------- - -include::anatomy.asciidoc[] - -include::client.asciidoc[] - -include::index_.asciidoc[] - -include::get.asciidoc[] - -include::delete.asciidoc[] - -include::search.asciidoc[] diff --git a/docs/groovy-api/index_.asciidoc b/docs/groovy-api/index_.asciidoc deleted file mode 100644 index deefb30e031a3..0000000000000 --- a/docs/groovy-api/index_.asciidoc +++ /dev/null @@ -1,32 +0,0 @@ -[[index_]] -== Index API - -The index API is very similar to the -// {javaclient}/java-docs-index.html[] -Java index API. The Groovy -extension to it is the ability to provide the indexed source using a -closure. For example: - -[source,groovy] --------------------------------------------------- -def indexR = client.index { - index "test" - type "_doc" - id "1" - source { - test = "value" - complex { - value1 = "value1" - value2 = "value2" - } - } -} --------------------------------------------------- - -In the above example, the source closure itself gets transformed into an -XContent (defaults to JSON). In order to change how the source closure -is serialized, a global (static) setting can be set on the `GClient` by -changing the `indexContentType` field. - -Note also that the `source` can be set using the typical Java based -APIs, the `Closure` option is a Groovy extension. diff --git a/docs/groovy-api/search.asciidoc b/docs/groovy-api/search.asciidoc deleted file mode 100644 index 7834e45abc81a..0000000000000 --- a/docs/groovy-api/search.asciidoc +++ /dev/null @@ -1,116 +0,0 @@ -[[search]] -== Search API - -The search API is very similar to the -// {javaclient}/java-search.html[] -Java search API. The Groovy -extension allows to provide the search source to execute as a `Closure` -including the query itself (similar to GORM criteria builder): - -[source,groovy] --------------------------------------------------- -def search = node.client.search { - indices "test" - types "_doc" - source { - query { - term(test: "value") - } - } -} - -search.response.hits.each {SearchHit hit -> - println "Got hit $hit.id from $hit.index/$hit.type" -} --------------------------------------------------- - -It can also be executed using the "Java API" while still using a closure -for the query: - -[source,groovy] --------------------------------------------------- -def search = node.client.prepareSearch("test").setQuery({ - term(test: "value") -}).gexecute(); - -search.response.hits.each {SearchHit hit -> - println "Got hit $hit.id from $hit.index/$hit.type" -} --------------------------------------------------- - -The format of the search `Closure` follows the same JSON syntax as the -{ref}/search-search.html[Search API] request. - - -[[more-examples]] -=== More examples - -Term query where multiple values are provided (see -{ref}/query-dsl-terms-query.html[terms]): - -[source,groovy] --------------------------------------------------- -def search = node.client.search { - indices "test" - types "_doc" - source { - query { - terms(test: ["value1", "value2"]) - } - } -} --------------------------------------------------- - -Query string (see -{ref}/query-dsl-query-string-query.html[query string]): - -[source,groovy] --------------------------------------------------- -def search = node.client.search { - indices "test" - types "_doc" - source { - query { - query_string( - fields: ["test"], - query: "value1 value2") - } - } -} --------------------------------------------------- - -Pagination (see -{ref}/search-request-from-size.html[from/size]): - -[source,groovy] --------------------------------------------------- -def search = node.client.search { - indices "test" - types "_doc" - source { - from = 0 - size = 10 - query { - term(test: "value") - } - } -} --------------------------------------------------- - -Sorting (see {ref}/search-request-sort.html[sort]): - -[source,groovy] --------------------------------------------------- -def search = node.client.search { - indices "test" - types "_doc" - source { - query { - term(test: "value") - } - sort = [ - date : [ order: "desc"] - ] - } -} --------------------------------------------------- diff --git a/docs/java-api/admin/cluster/health.asciidoc b/docs/java-api/admin/cluster/health.asciidoc deleted file mode 100644 index 615a011cf72c9..0000000000000 --- a/docs/java-api/admin/cluster/health.asciidoc +++ /dev/null @@ -1,76 +0,0 @@ -[[java-admin-cluster-health]] -==== Cluster Health - -[[java-admin-cluster-health-health]] -===== Health - -The cluster health API allows to get a very simple status on the health of the cluster and also can give you -some technical information about the cluster status per index: - -[source,java] --------------------------------------------------- -ClusterHealthResponse healths = client.admin().cluster().prepareHealth().get(); <1> -String clusterName = healths.getClusterName(); <2> -int numberOfDataNodes = healths.getNumberOfDataNodes(); <3> -int numberOfNodes = healths.getNumberOfNodes(); <4> - -for (ClusterIndexHealth health : healths.getIndices().values()) { <5> - String index = health.getIndex(); <6> - int numberOfShards = health.getNumberOfShards(); <7> - int numberOfReplicas = health.getNumberOfReplicas(); <8> - ClusterHealthStatus status = health.getStatus(); <9> -} --------------------------------------------------- -<1> Get information for all indices -<2> Access the cluster name -<3> Get the total number of data nodes -<4> Get the total number of nodes -<5> Iterate over all indices -<6> Index name -<7> Number of shards -<8> Number of replicas -<9> Index status - -[[java-admin-cluster-health-wait-status]] -===== Wait for status - -You can use the cluster health API to wait for a specific status for the whole cluster or for a given index: - -[source,java] --------------------------------------------------- -client.admin().cluster().prepareHealth() <1> - .setWaitForYellowStatus() <2> - .get(); -client.admin().cluster().prepareHealth("company") <3> - .setWaitForGreenStatus() <4> - .get(); - -client.admin().cluster().prepareHealth("employee") <5> - .setWaitForGreenStatus() <6> - .setTimeout(TimeValue.timeValueSeconds(2)) <7> - .get(); --------------------------------------------------- -<1> Prepare a health request -<2> Wait for the cluster being yellow -<3> Prepare the health request for index `company` -<4> Wait for the index being green -<5> Prepare the health request for index `employee` -<6> Wait for the index being green -<7> Wait at most for 2 seconds - -If the index does not have the expected status and you want to fail in that case, you need -to explicitly interpret the result: - -[source,java] --------------------------------------------------- -ClusterHealthResponse response = client.admin().cluster().prepareHealth("company") - .setWaitForGreenStatus() <1> - .get(); - -ClusterHealthStatus status = response.getIndices().get("company").getStatus(); -if (!status.equals(ClusterHealthStatus.GREEN)) { - throw new RuntimeException("Index is in " + status + " state"); <2> -} --------------------------------------------------- -<1> Wait for the index being green -<2> Throw an exception if not `GREEN` diff --git a/docs/java-api/admin/cluster/index.asciidoc b/docs/java-api/admin/cluster/index.asciidoc deleted file mode 100644 index 4e1850a34fe47..0000000000000 --- a/docs/java-api/admin/cluster/index.asciidoc +++ /dev/null @@ -1,16 +0,0 @@ -[[java-admin-cluster]] -=== Cluster Administration - -To access cluster Java API, you need to call `cluster()` method from an <>: - -[source,java] --------------------------------------------------- -ClusterAdminClient clusterAdminClient = client.admin().cluster(); --------------------------------------------------- - -[NOTE] -In the rest of this guide, we will use `client.admin().cluster()`. - -include::health.asciidoc[] - -include::stored-scripts.asciidoc[] diff --git a/docs/java-api/admin/cluster/stored-scripts.asciidoc b/docs/java-api/admin/cluster/stored-scripts.asciidoc deleted file mode 100644 index 5ebf89e92be55..0000000000000 --- a/docs/java-api/admin/cluster/stored-scripts.asciidoc +++ /dev/null @@ -1,29 +0,0 @@ -[[stored-scripts]] -==== Stored Scripts API - -The stored script API allows one to interact with scripts and templates -stored in Elasticsearch. It can be used to create, update, get, -and delete stored scripts and templates. - -[source,java] --------------------------------------------------- -PutStoredScriptResponse response = client.admin().cluster().preparePutStoredScript() - .setId("script1") - .setContent(new BytesArray("{\"script\": {\"lang\": \"painless\", \"source\": \"_score * doc['my_numeric_field'].value\"} }"), XContentType.JSON) - .get(); - -GetStoredScriptResponse response = client().admin().cluster().prepareGetStoredScript() - .setId("script1") - .get(); - -DeleteStoredScriptResponse response = client().admin().cluster().prepareDeleteStoredScript() - .setId("script1") - .get(); --------------------------------------------------- - -To store templates simply use "mustache" for the scriptLang. - -===== Script Language - -The put stored script API allows one to set the language of the stored script. -If one is not provided the default scripting language will be used. diff --git a/docs/java-api/admin/index.asciidoc b/docs/java-api/admin/index.asciidoc deleted file mode 100644 index 41599a82c7b3a..0000000000000 --- a/docs/java-api/admin/index.asciidoc +++ /dev/null @@ -1,18 +0,0 @@ -[[java-admin]] -== Java API Administration - -Elasticsearch provides a full Java API to deal with administration tasks. - -To access them, you need to call `admin()` method from a client to get an `AdminClient`: - -[source,java] --------------------------------------------------- -AdminClient adminClient = client.admin(); --------------------------------------------------- - -[NOTE] -In the rest of this guide, we will use `client.admin()`. - -include::indices/index.asciidoc[] - -include::cluster/index.asciidoc[] diff --git a/docs/java-api/admin/indices/create-index.asciidoc b/docs/java-api/admin/indices/create-index.asciidoc deleted file mode 100644 index 34b776bd04e23..0000000000000 --- a/docs/java-api/admin/indices/create-index.asciidoc +++ /dev/null @@ -1,28 +0,0 @@ -[[java-admin-indices-create-index]] -==== Create Index - -Using an <>, you can create an index with all default settings and no mapping: - -[source,java] --------------------------------------------------- -client.admin().indices().prepareCreate("twitter").get(); --------------------------------------------------- - -[float] -[[java-admin-indices-create-index-settings]] -===== Index Settings - -Each index created can have specific settings associated with it. - -[source,java] --------------------------------------------------- -client.admin().indices().prepareCreate("twitter") - .setSettings(Settings.builder() <1> - .put("index.number_of_shards", 3) - .put("index.number_of_replicas", 2) - ) - .get(); <2> --------------------------------------------------- -<1> Settings for this index -<2> Execute the action and wait for the result - diff --git a/docs/java-api/admin/indices/get-settings.asciidoc b/docs/java-api/admin/indices/get-settings.asciidoc deleted file mode 100644 index 844aaf65ec9b5..0000000000000 --- a/docs/java-api/admin/indices/get-settings.asciidoc +++ /dev/null @@ -1,22 +0,0 @@ -[[java-admin-indices-get-settings]] -==== Get Settings - -The get settings API allows to retrieve settings of index/indices: - -[source,java] --------------------------------------------------- -GetSettingsResponse response = client.admin().indices() - .prepareGetSettings("company", "employee").get(); <1> -for (ObjectObjectCursor cursor : response.getIndexToSettings()) { <2> - String index = cursor.key; <3> - Settings settings = cursor.value; <4> - Integer shards = settings.getAsInt("index.number_of_shards", null); <5> - Integer replicas = settings.getAsInt("index.number_of_replicas", null); <6> -} --------------------------------------------------- -<1> Get settings for indices `company` and `employee` -<2> Iterate over results -<3> Index name -<4> Settings for the given index -<5> Number of shards for this index -<6> Number of replicas for this index diff --git a/docs/java-api/admin/indices/index.asciidoc b/docs/java-api/admin/indices/index.asciidoc deleted file mode 100644 index bbd365076c72e..0000000000000 --- a/docs/java-api/admin/indices/index.asciidoc +++ /dev/null @@ -1,21 +0,0 @@ -[[java-admin-indices]] -=== Indices Administration - -To access indices Java API, you need to call `indices()` method from an <>: - -[source,java] --------------------------------------------------- -IndicesAdminClient indicesAdminClient = client.admin().indices(); --------------------------------------------------- - -[NOTE] -In the rest of this guide, we will use `client.admin().indices()`. - -include::create-index.asciidoc[] - -include::put-mapping.asciidoc[] - -include::refresh.asciidoc[] - -include::get-settings.asciidoc[] -include::update-settings.asciidoc[] diff --git a/docs/java-api/admin/indices/put-mapping.asciidoc b/docs/java-api/admin/indices/put-mapping.asciidoc deleted file mode 100644 index d63a498d994d2..0000000000000 --- a/docs/java-api/admin/indices/put-mapping.asciidoc +++ /dev/null @@ -1,30 +0,0 @@ -[[java-admin-indices-put-mapping]] - -==== Put Mapping - -You can add mappings at index creation time: - -["source","java",subs="attributes,callouts,macros"] --------------------------------------------------- -include-tagged::{client-tests}/IndicesDocumentationIT.java[index-with-mapping] --------------------------------------------------- -<1> <> called `twitter` -<2> Add a `_doc` type with a field called `message` that has the datatype `text`. - -There are several variants of the above `addMapping` method, some taking an -`XContentBuilder` or a `Map` with the mapping definition as arguments. Make sure -to check the javadocs to pick the simplest one for your use case. - -The PUT mapping API also allows for updating the mapping after index -creation. In this case you can provide the mapping as a String similar -to the REST API syntax: - -["source","java",subs="attributes,callouts,macros"] --------------------------------------------------- -include-tagged::{client-tests}/IndicesDocumentationIT.java[putMapping-request-source] --------------------------------------------------- -<1> Puts a mapping on existing index called `twitter` -<2> Adds a new field `name` to the mapping -<3> The type can be also provided within the source - -:base-dir!: diff --git a/docs/java-api/admin/indices/refresh.asciidoc b/docs/java-api/admin/indices/refresh.asciidoc deleted file mode 100644 index 856c270daf368..0000000000000 --- a/docs/java-api/admin/indices/refresh.asciidoc +++ /dev/null @@ -1,19 +0,0 @@ -[[java-admin-indices-refresh]] -==== Refresh - -The refresh API allows to explicitly refresh one or more index: - -[source,java] --------------------------------------------------- -client.admin().indices().prepareRefresh().get(); <1> -client.admin().indices() - .prepareRefresh("twitter") <2> - .get(); -client.admin().indices() - .prepareRefresh("twitter", "company") <3> - .get(); --------------------------------------------------- -<1> Refresh all indices -<2> Refresh one index -<3> Refresh many indices - diff --git a/docs/java-api/admin/indices/update-settings.asciidoc b/docs/java-api/admin/indices/update-settings.asciidoc deleted file mode 100644 index 9c2cba2adf03b..0000000000000 --- a/docs/java-api/admin/indices/update-settings.asciidoc +++ /dev/null @@ -1,16 +0,0 @@ -[[java-admin-indices-update-settings]] -==== Update Indices Settings - -You can change index settings by calling: - -[source,java] --------------------------------------------------- -client.admin().indices().prepareUpdateSettings("twitter") <1> - .setSettings(Settings.builder() <2> - .put("index.number_of_replicas", 0) - ) - .get(); --------------------------------------------------- -<1> Index to update -<2> Settings - diff --git a/docs/java-api/aggregations/bucket.asciidoc b/docs/java-api/aggregations/bucket.asciidoc deleted file mode 100644 index fe2e0ea9be309..0000000000000 --- a/docs/java-api/aggregations/bucket.asciidoc +++ /dev/null @@ -1,33 +0,0 @@ -[[java-aggregations-bucket]] - -include::bucket/global-aggregation.asciidoc[] - -include::bucket/filter-aggregation.asciidoc[] - -include::bucket/filters-aggregation.asciidoc[] - -include::bucket/missing-aggregation.asciidoc[] - -include::bucket/nested-aggregation.asciidoc[] - -include::bucket/reverse-nested-aggregation.asciidoc[] - -include::bucket/children-aggregation.asciidoc[] - -include::bucket/terms-aggregation.asciidoc[] - -include::bucket/significantterms-aggregation.asciidoc[] - -include::bucket/range-aggregation.asciidoc[] - -include::bucket/daterange-aggregation.asciidoc[] - -include::bucket/iprange-aggregation.asciidoc[] - -include::bucket/histogram-aggregation.asciidoc[] - -include::bucket/datehistogram-aggregation.asciidoc[] - -include::bucket/geodistance-aggregation.asciidoc[] - -include::bucket/geohashgrid-aggregation.asciidoc[] diff --git a/docs/java-api/aggregations/bucket/children-aggregation.asciidoc b/docs/java-api/aggregations/bucket/children-aggregation.asciidoc deleted file mode 100644 index f6a23fdafe976..0000000000000 --- a/docs/java-api/aggregations/bucket/children-aggregation.asciidoc +++ /dev/null @@ -1,35 +0,0 @@ -[[java-aggs-bucket-children]] -==== Children Aggregation - -Here is how you can use -{ref}/search-aggregations-bucket-children-aggregation.html[Children Aggregation] -with Java API. - - -===== Prepare aggregation request - -Here is an example on how to create the aggregation request: - -[source,java] --------------------------------------------------- -AggregationBuilder aggregation = - AggregationBuilders - .children("agg", "reseller"); <1> --------------------------------------------------- -1. `"agg"` is the name of the aggregation and `"reseller"` is the child type - -===== Use aggregation response - -Import Aggregation definition classes: - -[source,java] --------------------------------------------------- -import org.elasticsearch.join.aggregations.Children; --------------------------------------------------- - -[source,java] --------------------------------------------------- -// sr is here your SearchResponse object -Children agg = sr.getAggregations().get("agg"); -agg.getDocCount(); // Doc count --------------------------------------------------- diff --git a/docs/java-api/aggregations/bucket/datehistogram-aggregation.asciidoc b/docs/java-api/aggregations/bucket/datehistogram-aggregation.asciidoc deleted file mode 100644 index 610262b046c21..0000000000000 --- a/docs/java-api/aggregations/bucket/datehistogram-aggregation.asciidoc +++ /dev/null @@ -1,73 +0,0 @@ -[[java-aggs-bucket-datehistogram]] -==== Date Histogram Aggregation - -Here is how you can use -{ref}/search-aggregations-bucket-datehistogram-aggregation.html[Date Histogram Aggregation] -with Java API. - - -===== Prepare aggregation request - -Here is an example on how to create the aggregation request: - -[source,java] --------------------------------------------------- -AggregationBuilder aggregation = - AggregationBuilders - .dateHistogram("agg") - .field("dateOfBirth") - .calendarInterval(DateHistogramInterval.YEAR); --------------------------------------------------- - -Or if you want to set an interval of 10 days: - -[source,java] --------------------------------------------------- -AggregationBuilder aggregation = - AggregationBuilders - .dateHistogram("agg") - .field("dateOfBirth") - .fixedInterval(DateHistogramInterval.days(10)); --------------------------------------------------- - - -===== Use aggregation response - -Import Aggregation definition classes: - -[source,java] --------------------------------------------------- -import org.elasticsearch.search.aggregations.bucket.histogram.Histogram; --------------------------------------------------- - -[source,java] --------------------------------------------------- -// sr is here your SearchResponse object -Histogram agg = sr.getAggregations().get("agg"); - -// For each entry -for (Histogram.Bucket entry : agg.getBuckets()) { - DateTime key = (DateTime) entry.getKey(); // Key - String keyAsString = entry.getKeyAsString(); // Key as String - long docCount = entry.getDocCount(); // Doc count - - logger.info("key [{}], date [{}], doc_count [{}]", keyAsString, key.getYear(), docCount); -} --------------------------------------------------- - -This will basically produce for the first example: - -[source,text] --------------------------------------------------- -key [1942-01-01T00:00:00.000Z], date [1942], doc_count [1] -key [1945-01-01T00:00:00.000Z], date [1945], doc_count [1] -key [1946-01-01T00:00:00.000Z], date [1946], doc_count [1] -... -key [2005-01-01T00:00:00.000Z], date [2005], doc_count [1] -key [2007-01-01T00:00:00.000Z], date [2007], doc_count [2] -key [2008-01-01T00:00:00.000Z], date [2008], doc_count [3] --------------------------------------------------- - -===== Order - -Supports the same order functionality as the <>. diff --git a/docs/java-api/aggregations/bucket/daterange-aggregation.asciidoc b/docs/java-api/aggregations/bucket/daterange-aggregation.asciidoc deleted file mode 100644 index fa8f31e8cd0b7..0000000000000 --- a/docs/java-api/aggregations/bucket/daterange-aggregation.asciidoc +++ /dev/null @@ -1,59 +0,0 @@ -[[java-aggs-bucket-daterange]] -==== Date Range Aggregation - -Here is how you can use -{ref}/search-aggregations-bucket-daterange-aggregation.html[Date Range Aggregation] -with Java API. - - -===== Prepare aggregation request - -Here is an example on how to create the aggregation request: - -[source,java] --------------------------------------------------- -AggregationBuilder aggregation = - AggregationBuilders - .dateRange("agg") - .field("dateOfBirth") - .format("yyyy") - .addUnboundedTo("1950") // from -infinity to 1950 (excluded) - .addRange("1950", "1960") // from 1950 to 1960 (excluded) - .addUnboundedFrom("1960"); // from 1960 to +infinity --------------------------------------------------- - - -===== Use aggregation response - -Import Aggregation definition classes: - -[source,java] --------------------------------------------------- -import org.elasticsearch.search.aggregations.bucket.range.Range; --------------------------------------------------- - -[source,java] --------------------------------------------------- -// sr is here your SearchResponse object -Range agg = sr.getAggregations().get("agg"); - -// For each entry -for (Range.Bucket entry : agg.getBuckets()) { - String key = entry.getKeyAsString(); // Date range as key - DateTime fromAsDate = (DateTime) entry.getFrom(); // Date bucket from as a Date - DateTime toAsDate = (DateTime) entry.getTo(); // Date bucket to as a Date - long docCount = entry.getDocCount(); // Doc count - - logger.info("key [{}], from [{}], to [{}], doc_count [{}]", key, fromAsDate, toAsDate, docCount); -} --------------------------------------------------- - -This will basically produce: - -[source,text] --------------------------------------------------- -key [*-1950], from [null], to [1950-01-01T00:00:00.000Z], doc_count [8] -key [1950-1960], from [1950-01-01T00:00:00.000Z], to [1960-01-01T00:00:00.000Z], doc_count [5] -key [1960-*], from [1960-01-01T00:00:00.000Z], to [null], doc_count [37] --------------------------------------------------- - diff --git a/docs/java-api/aggregations/bucket/filter-aggregation.asciidoc b/docs/java-api/aggregations/bucket/filter-aggregation.asciidoc deleted file mode 100644 index 3ffb05202bbef..0000000000000 --- a/docs/java-api/aggregations/bucket/filter-aggregation.asciidoc +++ /dev/null @@ -1,34 +0,0 @@ -[[java-aggs-bucket-filter]] -==== Filter Aggregation - -Here is how you can use -{ref}/search-aggregations-bucket-filter-aggregation.html[Filter Aggregation] -with Java API. - - -===== Prepare aggregation request - -Here is an example on how to create the aggregation request: - -[source,java] --------------------------------------------------- -AggregationBuilders - .filter("agg", QueryBuilders.termQuery("gender", "male")); --------------------------------------------------- - - -===== Use aggregation response - -Import Aggregation definition classes: - -[source,java] --------------------------------------------------- -import org.elasticsearch.search.aggregations.bucket.filter.Filter; --------------------------------------------------- - -[source,java] --------------------------------------------------- -// sr is here your SearchResponse object -Filter agg = sr.getAggregations().get("agg"); -agg.getDocCount(); // Doc count --------------------------------------------------- diff --git a/docs/java-api/aggregations/bucket/filters-aggregation.asciidoc b/docs/java-api/aggregations/bucket/filters-aggregation.asciidoc deleted file mode 100644 index 0b782304dacc0..0000000000000 --- a/docs/java-api/aggregations/bucket/filters-aggregation.asciidoc +++ /dev/null @@ -1,51 +0,0 @@ -[[java-aggs-bucket-filters]] -==== Filters Aggregation - -Here is how you can use -{ref}/search-aggregations-bucket-filters-aggregation.html[Filters Aggregation] -with Java API. - - -===== Prepare aggregation request - -Here is an example on how to create the aggregation request: - -[source,java] --------------------------------------------------- -AggregationBuilder aggregation = - AggregationBuilders - .filters("agg", - new FiltersAggregator.KeyedFilter("men", QueryBuilders.termQuery("gender", "male")), - new FiltersAggregator.KeyedFilter("women", QueryBuilders.termQuery("gender", "female"))); --------------------------------------------------- - - -===== Use aggregation response - -Import Aggregation definition classes: - -[source,java] --------------------------------------------------- -import org.elasticsearch.search.aggregations.bucket.filters.Filters; --------------------------------------------------- - -[source,java] --------------------------------------------------- -// sr is here your SearchResponse object -Filters agg = sr.getAggregations().get("agg"); - -// For each entry -for (Filters.Bucket entry : agg.getBuckets()) { - String key = entry.getKeyAsString(); // bucket key - long docCount = entry.getDocCount(); // Doc count - logger.info("key [{}], doc_count [{}]", key, docCount); -} --------------------------------------------------- - -This will basically produce: - -[source,text] --------------------------------------------------- -key [men], doc_count [4982] -key [women], doc_count [5018] --------------------------------------------------- diff --git a/docs/java-api/aggregations/bucket/geodistance-aggregation.asciidoc b/docs/java-api/aggregations/bucket/geodistance-aggregation.asciidoc deleted file mode 100644 index 472c3ac59bf48..0000000000000 --- a/docs/java-api/aggregations/bucket/geodistance-aggregation.asciidoc +++ /dev/null @@ -1,58 +0,0 @@ -[[java-aggs-bucket-geodistance]] -==== Geo Distance Aggregation - -Here is how you can use -{ref}/search-aggregations-bucket-geodistance-aggregation.html[Geo Distance Aggregation] -with Java API. - - -===== Prepare aggregation request - -Here is an example on how to create the aggregation request: - -[source,java] --------------------------------------------------- -AggregationBuilder aggregation = - AggregationBuilders - .geoDistance("agg", new GeoPoint(48.84237171118314,2.33320027692004)) - .field("address.location") - .unit(DistanceUnit.KILOMETERS) - .addUnboundedTo(3.0) - .addRange(3.0, 10.0) - .addRange(10.0, 500.0); --------------------------------------------------- - - -===== Use aggregation response - -Import Aggregation definition classes: - -[source,java] --------------------------------------------------- -import org.elasticsearch.search.aggregations.bucket.range.Range; --------------------------------------------------- - -[source,java] --------------------------------------------------- -// sr is here your SearchResponse object -Range agg = sr.getAggregations().get("agg"); - -// For each entry -for (Range.Bucket entry : agg.getBuckets()) { - String key = entry.getKeyAsString(); // key as String - Number from = (Number) entry.getFrom(); // bucket from value - Number to = (Number) entry.getTo(); // bucket to value - long docCount = entry.getDocCount(); // Doc count - - logger.info("key [{}], from [{}], to [{}], doc_count [{}]", key, from, to, docCount); -} --------------------------------------------------- - -This will basically produce: - -[source,text] --------------------------------------------------- -key [*-3.0], from [0.0], to [3.0], doc_count [161] -key [3.0-10.0], from [3.0], to [10.0], doc_count [460] -key [10.0-500.0], from [10.0], to [500.0], doc_count [4925] --------------------------------------------------- diff --git a/docs/java-api/aggregations/bucket/geohashgrid-aggregation.asciidoc b/docs/java-api/aggregations/bucket/geohashgrid-aggregation.asciidoc deleted file mode 100644 index 19e3f03349397..0000000000000 --- a/docs/java-api/aggregations/bucket/geohashgrid-aggregation.asciidoc +++ /dev/null @@ -1,57 +0,0 @@ -[[java-aggs-bucket-geohashgrid]] -==== Geo Hash Grid Aggregation - -Here is how you can use -{ref}/search-aggregations-bucket-geohashgrid-aggregation.html[Geo Hash Grid Aggregation] -with Java API. - - -===== Prepare aggregation request - -Here is an example on how to create the aggregation request: - -[source,java] --------------------------------------------------- -AggregationBuilder aggregation = - AggregationBuilders - .geohashGrid("agg") - .field("address.location") - .precision(4); --------------------------------------------------- - - -===== Use aggregation response - -Import Aggregation definition classes: - -[source,java] --------------------------------------------------- -import org.elasticsearch.search.aggregations.bucket.geogrid.GeoHashGrid; --------------------------------------------------- - -[source,java] --------------------------------------------------- -// sr is here your SearchResponse object -GeoHashGrid agg = sr.getAggregations().get("agg"); - -// For each entry -for (GeoHashGrid.Bucket entry : agg.getBuckets()) { - String keyAsString = entry.getKeyAsString(); // key as String - GeoPoint key = (GeoPoint) entry.getKey(); // key as geo point - long docCount = entry.getDocCount(); // Doc count - - logger.info("key [{}], point {}, doc_count [{}]", keyAsString, key, docCount); -} --------------------------------------------------- - -This will basically produce: - -[source,text] --------------------------------------------------- -key [gbqu], point [47.197265625, -1.58203125], doc_count [1282] -key [gbvn], point [50.361328125, -4.04296875], doc_count [1248] -key [u1j0], point [50.712890625, 7.20703125], doc_count [1156] -key [u0j2], point [45.087890625, 7.55859375], doc_count [1138] -... --------------------------------------------------- - diff --git a/docs/java-api/aggregations/bucket/global-aggregation.asciidoc b/docs/java-api/aggregations/bucket/global-aggregation.asciidoc deleted file mode 100644 index e0a731159adf5..0000000000000 --- a/docs/java-api/aggregations/bucket/global-aggregation.asciidoc +++ /dev/null @@ -1,35 +0,0 @@ -[[java-aggs-bucket-global]] -==== Global Aggregation - -Here is how you can use -{ref}/search-aggregations-bucket-global-aggregation.html[Global Aggregation] -with Java API. - - -===== Prepare aggregation request - -Here is an example on how to create the aggregation request: - -[source,java] --------------------------------------------------- -AggregationBuilders - .global("agg") - .subAggregation(AggregationBuilders.terms("genders").field("gender")); --------------------------------------------------- - - -===== Use aggregation response - -Import Aggregation definition classes: - -[source,java] --------------------------------------------------- -import org.elasticsearch.search.aggregations.bucket.global.Global; --------------------------------------------------- - -[source,java] --------------------------------------------------- -// sr is here your SearchResponse object -Global agg = sr.getAggregations().get("agg"); -agg.getDocCount(); // Doc count --------------------------------------------------- diff --git a/docs/java-api/aggregations/bucket/histogram-aggregation.asciidoc b/docs/java-api/aggregations/bucket/histogram-aggregation.asciidoc deleted file mode 100644 index 59bb555401c5b..0000000000000 --- a/docs/java-api/aggregations/bucket/histogram-aggregation.asciidoc +++ /dev/null @@ -1,48 +0,0 @@ -[[java-aggs-bucket-histogram]] -==== Histogram Aggregation - -Here is how you can use -{ref}/search-aggregations-bucket-histogram-aggregation.html[Histogram Aggregation] -with Java API. - - -===== Prepare aggregation request - -Here is an example on how to create the aggregation request: - -[source,java] --------------------------------------------------- -AggregationBuilder aggregation = - AggregationBuilders - .histogram("agg") - .field("height") - .interval(1); --------------------------------------------------- - - -===== Use aggregation response - -Import Aggregation definition classes: - -[source,java] --------------------------------------------------- -import org.elasticsearch.search.aggregations.bucket.histogram.Histogram; --------------------------------------------------- - -[source,java] --------------------------------------------------- -// sr is here your SearchResponse object -Histogram agg = sr.getAggregations().get("agg"); - -// For each entry -for (Histogram.Bucket entry : agg.getBuckets()) { - Number key = (Number) entry.getKey(); // Key - long docCount = entry.getDocCount(); // Doc count - - logger.info("key [{}], doc_count [{}]", key, docCount); -} --------------------------------------------------- - -===== Order - -Supports the same order functionality as the <>. diff --git a/docs/java-api/aggregations/bucket/iprange-aggregation.asciidoc b/docs/java-api/aggregations/bucket/iprange-aggregation.asciidoc deleted file mode 100644 index a2c07df1b26e7..0000000000000 --- a/docs/java-api/aggregations/bucket/iprange-aggregation.asciidoc +++ /dev/null @@ -1,79 +0,0 @@ -[[java-aggs-bucket-iprange]] -==== Ip Range Aggregation - -Here is how you can use -{ref}/search-aggregations-bucket-iprange-aggregation.html[Ip Range Aggregation] -with Java API. - - -===== Prepare aggregation request - -Here is an example on how to create the aggregation request: - -[source,java] --------------------------------------------------- -AggregationBuilder aggregation = - AggregationBuilders - .ipRange("agg") - .field("ip") - .addUnboundedTo("192.168.1.0") // from -infinity to 192.168.1.0 (excluded) - .addRange("192.168.1.0", "192.168.2.0") // from 192.168.1.0 to 192.168.2.0 (excluded) - .addUnboundedFrom("192.168.2.0"); // from 192.168.2.0 to +infinity --------------------------------------------------- - -Note that you could also use ip masks as ranges: - -[source,java] --------------------------------------------------- -AggregationBuilder aggregation = - AggregationBuilders - .ipRange("agg") - .field("ip") - .addMaskRange("192.168.0.0/32") - .addMaskRange("192.168.0.0/24") - .addMaskRange("192.168.0.0/16"); --------------------------------------------------- - -===== Use aggregation response - -Import Aggregation definition classes: - -[source,java] --------------------------------------------------- -import org.elasticsearch.search.aggregations.bucket.range.Range; --------------------------------------------------- - -[source,java] --------------------------------------------------- -// sr is here your SearchResponse object -Range agg = sr.getAggregations().get("agg"); - -// For each entry -for (Range.Bucket entry : agg.getBuckets()) { - String key = entry.getKeyAsString(); // Ip range as key - String fromAsString = entry.getFromAsString(); // Ip bucket from as a String - String toAsString = entry.getToAsString(); // Ip bucket to as a String - long docCount = entry.getDocCount(); // Doc count - - logger.info("key [{}], from [{}], to [{}], doc_count [{}]", key, fromAsString, toAsString, docCount); -} --------------------------------------------------- - -This will basically produce for the first example: - -[source,text] --------------------------------------------------- -key [*-192.168.1.0], from [null], to [192.168.1.0], doc_count [13] -key [192.168.1.0-192.168.2.0], from [192.168.1.0], to [192.168.2.0], doc_count [14] -key [192.168.2.0-*], from [192.168.2.0], to [null], doc_count [23] --------------------------------------------------- - -And for the second one (using Ip masks): - -[source,text] --------------------------------------------------- -key [192.168.0.0/32], from [192.168.0.0], to [192.168.0.1], doc_count [0] -key [192.168.0.0/24], from [192.168.0.0], to [192.168.1.0], doc_count [13] -key [192.168.0.0/16], from [192.168.0.0], to [192.169.0.0], doc_count [50] --------------------------------------------------- - diff --git a/docs/java-api/aggregations/bucket/missing-aggregation.asciidoc b/docs/java-api/aggregations/bucket/missing-aggregation.asciidoc deleted file mode 100644 index 31d21604dc57a..0000000000000 --- a/docs/java-api/aggregations/bucket/missing-aggregation.asciidoc +++ /dev/null @@ -1,34 +0,0 @@ -[[java-aggs-bucket-missing]] -==== Missing Aggregation - -Here is how you can use -{ref}/search-aggregations-bucket-missing-aggregation.html[Missing Aggregation] -with Java API. - - -===== Prepare aggregation request - -Here is an example on how to create the aggregation request: - -[source,java] --------------------------------------------------- -AggregationBuilders.missing("agg").field("gender"); --------------------------------------------------- - - -===== Use aggregation response - -Import Aggregation definition classes: - -[source,java] --------------------------------------------------- -import org.elasticsearch.search.aggregations.bucket.missing.Missing; --------------------------------------------------- - -[source,java] --------------------------------------------------- -// sr is here your SearchResponse object -Missing agg = sr.getAggregations().get("agg"); -agg.getDocCount(); // Doc count --------------------------------------------------- - diff --git a/docs/java-api/aggregations/bucket/nested-aggregation.asciidoc b/docs/java-api/aggregations/bucket/nested-aggregation.asciidoc deleted file mode 100644 index b1ebad7a63bfa..0000000000000 --- a/docs/java-api/aggregations/bucket/nested-aggregation.asciidoc +++ /dev/null @@ -1,34 +0,0 @@ -[[java-aggs-bucket-nested]] -==== Nested Aggregation - -Here is how you can use -{ref}/search-aggregations-bucket-nested-aggregation.html[Nested Aggregation] -with Java API. - - -===== Prepare aggregation request - -Here is an example on how to create the aggregation request: - -[source,java] --------------------------------------------------- -AggregationBuilders - .nested("agg", "resellers"); --------------------------------------------------- - - -===== Use aggregation response - -Import Aggregation definition classes: - -[source,java] --------------------------------------------------- -import org.elasticsearch.search.aggregations.bucket.nested.Nested; --------------------------------------------------- - -[source,java] --------------------------------------------------- -// sr is here your SearchResponse object -Nested agg = sr.getAggregations().get("agg"); -agg.getDocCount(); // Doc count --------------------------------------------------- diff --git a/docs/java-api/aggregations/bucket/range-aggregation.asciidoc b/docs/java-api/aggregations/bucket/range-aggregation.asciidoc deleted file mode 100644 index b30c856ebeada..0000000000000 --- a/docs/java-api/aggregations/bucket/range-aggregation.asciidoc +++ /dev/null @@ -1,58 +0,0 @@ -[[java-aggs-bucket-range]] -==== Range Aggregation - -Here is how you can use -{ref}/search-aggregations-bucket-range-aggregation.html[Range Aggregation] -with Java API. - - -===== Prepare aggregation request - -Here is an example on how to create the aggregation request: - -[source,java] --------------------------------------------------- -AggregationBuilder aggregation = - AggregationBuilders - .range("agg") - .field("height") - .addUnboundedTo(1.0f) // from -infinity to 1.0 (excluded) - .addRange(1.0f, 1.5f) // from 1.0 to 1.5 (excluded) - .addUnboundedFrom(1.5f); // from 1.5 to +infinity --------------------------------------------------- - - -===== Use aggregation response - -Import Aggregation definition classes: - -[source,java] --------------------------------------------------- -import org.elasticsearch.search.aggregations.bucket.range.Range; --------------------------------------------------- - -[source,java] --------------------------------------------------- -// sr is here your SearchResponse object -Range agg = sr.getAggregations().get("agg"); - -// For each entry -for (Range.Bucket entry : agg.getBuckets()) { - String key = entry.getKeyAsString(); // Range as key - Number from = (Number) entry.getFrom(); // Bucket from - Number to = (Number) entry.getTo(); // Bucket to - long docCount = entry.getDocCount(); // Doc count - - logger.info("key [{}], from [{}], to [{}], doc_count [{}]", key, from, to, docCount); -} --------------------------------------------------- - -This will basically produce for the first example: - -[source,text] --------------------------------------------------- -key [*-1.0], from [-Infinity], to [1.0], doc_count [9] -key [1.0-1.5], from [1.0], to [1.5], doc_count [21] -key [1.5-*], from [1.5], to [Infinity], doc_count [20] --------------------------------------------------- - diff --git a/docs/java-api/aggregations/bucket/reverse-nested-aggregation.asciidoc b/docs/java-api/aggregations/bucket/reverse-nested-aggregation.asciidoc deleted file mode 100644 index 635b0e8cf77ee..0000000000000 --- a/docs/java-api/aggregations/bucket/reverse-nested-aggregation.asciidoc +++ /dev/null @@ -1,50 +0,0 @@ -[[java-aggs-bucket-reverse-nested]] -==== Reverse Nested Aggregation - -Here is how you can use -{ref}/search-aggregations-bucket-reverse-nested-aggregation.html[Reverse Nested Aggregation] -with Java API. - - -===== Prepare aggregation request - -Here is an example on how to create the aggregation request: - -[source,java] --------------------------------------------------- -AggregationBuilder aggregation = - AggregationBuilders - .nested("agg", "resellers") - .subAggregation( - AggregationBuilders - .terms("name").field("resellers.name") - .subAggregation( - AggregationBuilders - .reverseNested("reseller_to_product") - ) - ); --------------------------------------------------- - - -===== Use aggregation response - -Import Aggregation definition classes: - -[source,java] --------------------------------------------------- -import org.elasticsearch.search.aggregations.bucket.nested.Nested; -import org.elasticsearch.search.aggregations.bucket.nested.ReverseNested; -import org.elasticsearch.search.aggregations.bucket.terms.Terms; --------------------------------------------------- - -[source,java] --------------------------------------------------- -// sr is here your SearchResponse object -Nested agg = sr.getAggregations().get("agg"); -Terms name = agg.getAggregations().get("name"); -for (Terms.Bucket bucket : name.getBuckets()) { - ReverseNested resellerToProduct = bucket.getAggregations().get("reseller_to_product"); - resellerToProduct.getDocCount(); // Doc count -} --------------------------------------------------- - diff --git a/docs/java-api/aggregations/bucket/significantterms-aggregation.asciidoc b/docs/java-api/aggregations/bucket/significantterms-aggregation.asciidoc deleted file mode 100644 index 4450c324c8209..0000000000000 --- a/docs/java-api/aggregations/bucket/significantterms-aggregation.asciidoc +++ /dev/null @@ -1,47 +0,0 @@ -[[java-aggs-bucket-significantterms]] -==== Significant Terms Aggregation - -Here is how you can use -{ref}/search-aggregations-bucket-significantterms-aggregation.html[Significant Terms Aggregation] -with Java API. - - -===== Prepare aggregation request - -Here is an example on how to create the aggregation request: - -[source,java] --------------------------------------------------- -AggregationBuilder aggregation = - AggregationBuilders - .significantTerms("significant_countries") - .field("address.country"); - -// Let say you search for men only -SearchResponse sr = client.prepareSearch() - .setQuery(QueryBuilders.termQuery("gender", "male")) - .addAggregation(aggregation) - .get(); --------------------------------------------------- - - -===== Use aggregation response - -Import Aggregation definition classes: - -[source,java] --------------------------------------------------- -import org.elasticsearch.search.aggregations.bucket.significant.SignificantTerms; --------------------------------------------------- - -[source,java] --------------------------------------------------- -// sr is here your SearchResponse object -SignificantTerms agg = sr.getAggregations().get("significant_countries"); - -// For each entry -for (SignificantTerms.Bucket entry : agg.getBuckets()) { - entry.getKey(); // Term - entry.getDocCount(); // Doc count -} --------------------------------------------------- diff --git a/docs/java-api/aggregations/bucket/terms-aggregation.asciidoc b/docs/java-api/aggregations/bucket/terms-aggregation.asciidoc deleted file mode 100644 index db584fd4cedd2..0000000000000 --- a/docs/java-api/aggregations/bucket/terms-aggregation.asciidoc +++ /dev/null @@ -1,97 +0,0 @@ -[[java-aggs-bucket-terms]] -==== Terms Aggregation - -Here is how you can use -{ref}/search-aggregations-bucket-terms-aggregation.html[Terms Aggregation] -with Java API. - - -===== Prepare aggregation request - -Here is an example on how to create the aggregation request: - -[source,java] --------------------------------------------------- -AggregationBuilders - .terms("genders") - .field("gender"); --------------------------------------------------- - - -===== Use aggregation response - -Import Aggregation definition classes: - -[source,java] --------------------------------------------------- -import org.elasticsearch.search.aggregations.bucket.terms.Terms; --------------------------------------------------- - -[source,java] --------------------------------------------------- -// sr is here your SearchResponse object -Terms genders = sr.getAggregations().get("genders"); - -// For each entry -for (Terms.Bucket entry : genders.getBuckets()) { - entry.getKey(); // Term - entry.getDocCount(); // Doc count -} --------------------------------------------------- - -===== Order - -Import bucket ordering strategy classes: - -[source,java] --------------------------------------------------- -import org.elasticsearch.search.aggregations.BucketOrder; --------------------------------------------------- - -Ordering the buckets by their `doc_count` in an ascending manner: - -[source,java] --------------------------------------------------- -AggregationBuilders - .terms("genders") - .field("gender") - .order(BucketOrder.count(true)) --------------------------------------------------- - -Ordering the buckets alphabetically by their terms in an ascending manner: - -[source,java] --------------------------------------------------- -AggregationBuilders - .terms("genders") - .field("gender") - .order(BucketOrder.key(true)) --------------------------------------------------- - -Ordering the buckets by single value metrics sub-aggregation (identified by the aggregation name): - -[source,java] --------------------------------------------------- -AggregationBuilders - .terms("genders") - .field("gender") - .order(BucketOrder.aggregation("avg_height", false)) - .subAggregation( - AggregationBuilders.avg("avg_height").field("height") - ) --------------------------------------------------- - -Ordering the buckets by multiple criteria: - -[source,java] --------------------------------------------------- -AggregationBuilders - .terms("genders") - .field("gender") - .order(BucketOrder.compound( // in order of priority: - BucketOrder.aggregation("avg_height", false), // sort by sub-aggregation first - BucketOrder.count(true))) // then bucket count as a tie-breaker - .subAggregation( - AggregationBuilders.avg("avg_height").field("height") - ) --------------------------------------------------- diff --git a/docs/java-api/aggregations/metrics.asciidoc b/docs/java-api/aggregations/metrics.asciidoc deleted file mode 100644 index c9afb4c39d484..0000000000000 --- a/docs/java-api/aggregations/metrics.asciidoc +++ /dev/null @@ -1,27 +0,0 @@ -[[java-aggregations-metrics]] - -include::metrics/min-aggregation.asciidoc[] - -include::metrics/max-aggregation.asciidoc[] - -include::metrics/sum-aggregation.asciidoc[] - -include::metrics/avg-aggregation.asciidoc[] - -include::metrics/stats-aggregation.asciidoc[] - -include::metrics/extendedstats-aggregation.asciidoc[] - -include::metrics/valuecount-aggregation.asciidoc[] - -include::metrics/percentile-aggregation.asciidoc[] - -include::metrics/percentile-rank-aggregation.asciidoc[] - -include::metrics/cardinality-aggregation.asciidoc[] - -include::metrics/geobounds-aggregation.asciidoc[] - -include::metrics/tophits-aggregation.asciidoc[] - -include::metrics/scripted-metric-aggregation.asciidoc[] diff --git a/docs/java-api/aggregations/metrics/avg-aggregation.asciidoc b/docs/java-api/aggregations/metrics/avg-aggregation.asciidoc deleted file mode 100644 index 511cbabf5c848..0000000000000 --- a/docs/java-api/aggregations/metrics/avg-aggregation.asciidoc +++ /dev/null @@ -1,37 +0,0 @@ -[[java-aggs-metrics-avg]] -==== Avg Aggregation - -Here is how you can use -{ref}/search-aggregations-metrics-avg-aggregation.html[Avg Aggregation] -with Java API. - - -===== Prepare aggregation request - -Here is an example on how to create the aggregation request: - -[source,java] --------------------------------------------------- -AvgAggregationBuilder aggregation = - AggregationBuilders - .avg("agg") - .field("height"); --------------------------------------------------- - - -===== Use aggregation response - -Import Aggregation definition classes: - -[source,java] --------------------------------------------------- -import org.elasticsearch.search.aggregations.metrics.avg.Avg; --------------------------------------------------- - -[source,java] --------------------------------------------------- -// sr is here your SearchResponse object -Avg agg = sr.getAggregations().get("agg"); -double value = agg.getValue(); --------------------------------------------------- - diff --git a/docs/java-api/aggregations/metrics/cardinality-aggregation.asciidoc b/docs/java-api/aggregations/metrics/cardinality-aggregation.asciidoc deleted file mode 100644 index 8a854e553f4a3..0000000000000 --- a/docs/java-api/aggregations/metrics/cardinality-aggregation.asciidoc +++ /dev/null @@ -1,38 +0,0 @@ -[[java-aggs-metrics-cardinality]] -==== Cardinality Aggregation - -Here is how you can use -{ref}/search-aggregations-metrics-cardinality-aggregation.html[Cardinality Aggregation] -with Java API. - - -===== Prepare aggregation request - -Here is an example on how to create the aggregation request: - -[source,java] --------------------------------------------------- -CardinalityAggregationBuilder aggregation = - AggregationBuilders - .cardinality("agg") - .field("tags"); --------------------------------------------------- - - -===== Use aggregation response - -Import Aggregation definition classes: - -[source,java] --------------------------------------------------- -import org.elasticsearch.search.aggregations.metrics.cardinality.Cardinality; --------------------------------------------------- - -[source,java] --------------------------------------------------- -// sr is here your SearchResponse object -Cardinality agg = sr.getAggregations().get("agg"); -long value = agg.getValue(); --------------------------------------------------- - - diff --git a/docs/java-api/aggregations/metrics/extendedstats-aggregation.asciidoc b/docs/java-api/aggregations/metrics/extendedstats-aggregation.asciidoc deleted file mode 100644 index 8f2f12ede6849..0000000000000 --- a/docs/java-api/aggregations/metrics/extendedstats-aggregation.asciidoc +++ /dev/null @@ -1,44 +0,0 @@ -[[java-aggs-metrics-extendedstats]] -==== Extended Stats Aggregation - -Here is how you can use -{ref}/search-aggregations-metrics-extendedstats-aggregation.html[Extended Stats Aggregation] -with Java API. - - -===== Prepare aggregation request - -Here is an example on how to create the aggregation request: - -[source,java] --------------------------------------------------- -ExtendedStatsAggregationBuilder aggregation = - AggregationBuilders - .extendedStats("agg") - .field("height"); --------------------------------------------------- - - -===== Use aggregation response - -Import Aggregation definition classes: - -[source,java] --------------------------------------------------- -import org.elasticsearch.search.aggregations.metrics.stats.extended.ExtendedStats; --------------------------------------------------- - -[source,java] --------------------------------------------------- -// sr is here your SearchResponse object -ExtendedStats agg = sr.getAggregations().get("agg"); -double min = agg.getMin(); -double max = agg.getMax(); -double avg = agg.getAvg(); -double sum = agg.getSum(); -long count = agg.getCount(); -double stdDeviation = agg.getStdDeviation(); -double sumOfSquares = agg.getSumOfSquares(); -double variance = agg.getVariance(); --------------------------------------------------- - diff --git a/docs/java-api/aggregations/metrics/geobounds-aggregation.asciidoc b/docs/java-api/aggregations/metrics/geobounds-aggregation.asciidoc deleted file mode 100644 index 571a61f12e7cc..0000000000000 --- a/docs/java-api/aggregations/metrics/geobounds-aggregation.asciidoc +++ /dev/null @@ -1,46 +0,0 @@ -[[java-aggs-metrics-geobounds]] -==== Geo Bounds Aggregation - -Here is how you can use -{ref}/search-aggregations-metrics-geobounds-aggregation.html[Geo Bounds Aggregation] -with Java API. - - -===== Prepare aggregation request - -Here is an example on how to create the aggregation request: - -[source,java] --------------------------------------------------- -GeoBoundsAggregationBuilder aggregation = - GeoBoundsAggregationBuilder - .geoBounds("agg") - .field("address.location") - .wrapLongitude(true); --------------------------------------------------- - - -===== Use aggregation response - -Import Aggregation definition classes: - -[source,java] --------------------------------------------------- -import org.elasticsearch.search.aggregations.metrics.geobounds.GeoBounds; --------------------------------------------------- - -[source,java] --------------------------------------------------- -// sr is here your SearchResponse object -GeoBounds agg = sr.getAggregations().get("agg"); -GeoPoint bottomRight = agg.bottomRight(); -GeoPoint topLeft = agg.topLeft(); -logger.info("bottomRight {}, topLeft {}", bottomRight, topLeft); --------------------------------------------------- - -This will basically produce: - -[source,text] --------------------------------------------------- -bottomRight [40.70500764381921, 13.952946866893775], topLeft [53.49603022435221, -4.190029308156676] --------------------------------------------------- diff --git a/docs/java-api/aggregations/metrics/max-aggregation.asciidoc b/docs/java-api/aggregations/metrics/max-aggregation.asciidoc deleted file mode 100644 index 9bd393698429b..0000000000000 --- a/docs/java-api/aggregations/metrics/max-aggregation.asciidoc +++ /dev/null @@ -1,37 +0,0 @@ -[[java-aggs-metrics-max]] -==== Max Aggregation - -Here is how you can use -{ref}/search-aggregations-metrics-max-aggregation.html[Max Aggregation] -with Java API. - - -===== Prepare aggregation request - -Here is an example on how to create the aggregation request: - -[source,java] --------------------------------------------------- -MaxAggregationBuilder aggregation = - AggregationBuilders - .max("agg") - .field("height"); --------------------------------------------------- - - -===== Use aggregation response - -Import Aggregation definition classes: - -[source,java] --------------------------------------------------- -import org.elasticsearch.search.aggregations.metrics.max.Max; --------------------------------------------------- - -[source,java] --------------------------------------------------- -// sr is here your SearchResponse object -Max agg = sr.getAggregations().get("agg"); -double value = agg.getValue(); --------------------------------------------------- - diff --git a/docs/java-api/aggregations/metrics/min-aggregation.asciidoc b/docs/java-api/aggregations/metrics/min-aggregation.asciidoc deleted file mode 100644 index 0205cae44d8f8..0000000000000 --- a/docs/java-api/aggregations/metrics/min-aggregation.asciidoc +++ /dev/null @@ -1,37 +0,0 @@ -[[java-aggs-metrics-min]] -==== Min Aggregation - -Here is how you can use -{ref}/search-aggregations-metrics-min-aggregation.html[Min Aggregation] -with Java API. - - -===== Prepare aggregation request - -Here is an example on how to create the aggregation request: - -[source,java] --------------------------------------------------- -MinAggregationBuilder aggregation = - AggregationBuilders - .min("agg") - .field("height"); --------------------------------------------------- - - -===== Use aggregation response - -Import Aggregation definition classes: - -[source,java] --------------------------------------------------- -import org.elasticsearch.search.aggregations.metrics.min.Min; --------------------------------------------------- - -[source,java] --------------------------------------------------- -// sr is here your SearchResponse object -Min agg = sr.getAggregations().get("agg"); -double value = agg.getValue(); --------------------------------------------------- - diff --git a/docs/java-api/aggregations/metrics/percentile-aggregation.asciidoc b/docs/java-api/aggregations/metrics/percentile-aggregation.asciidoc deleted file mode 100644 index ad54fbf5a46be..0000000000000 --- a/docs/java-api/aggregations/metrics/percentile-aggregation.asciidoc +++ /dev/null @@ -1,68 +0,0 @@ -[[java-aggs-metrics-percentile]] -==== Percentile Aggregation - -Here is how you can use -{ref}/search-aggregations-metrics-percentile-aggregation.html[Percentile Aggregation] -with Java API. - - -===== Prepare aggregation request - -Here is an example on how to create the aggregation request: - -[source,java] --------------------------------------------------- -PercentilesAggregationBuilder aggregation = - AggregationBuilders - .percentiles("agg") - .field("height"); --------------------------------------------------- - -You can provide your own percentiles instead of using defaults: - -[source,java] --------------------------------------------------- -PercentilesAggregationBuilder aggregation = - AggregationBuilders - .percentiles("agg") - .field("height") - .percentiles(1.0, 5.0, 10.0, 20.0, 30.0, 75.0, 95.0, 99.0); --------------------------------------------------- - -===== Use aggregation response - -Import Aggregation definition classes: - -[source,java] --------------------------------------------------- -import org.elasticsearch.search.aggregations.metrics.percentiles.Percentile; -import org.elasticsearch.search.aggregations.metrics.percentiles.Percentiles; --------------------------------------------------- - -[source,java] --------------------------------------------------- -// sr is here your SearchResponse object -Percentiles agg = sr.getAggregations().get("agg"); -// For each entry -for (Percentile entry : agg) { - double percent = entry.getPercent(); // Percent - double value = entry.getValue(); // Value - - logger.info("percent [{}], value [{}]", percent, value); -} --------------------------------------------------- - - -This will basically produce for the first example: - -[source,text] --------------------------------------------------- -percent [1.0], value [0.814338896154595] -percent [5.0], value [0.8761912455821302] -percent [25.0], value [1.173346540141847] -percent [50.0], value [1.5432023318692198] -percent [75.0], value [1.923915462033674] -percent [95.0], value [2.2273644908535335] -percent [99.0], value [2.284989339108279] --------------------------------------------------- - diff --git a/docs/java-api/aggregations/metrics/percentile-rank-aggregation.asciidoc b/docs/java-api/aggregations/metrics/percentile-rank-aggregation.asciidoc deleted file mode 100644 index a846d59f82029..0000000000000 --- a/docs/java-api/aggregations/metrics/percentile-rank-aggregation.asciidoc +++ /dev/null @@ -1,55 +0,0 @@ -[[java-aggs-metrics-percentile-rank]] -==== Percentile Ranks Aggregation - -Here is how you can use -{ref}/search-aggregations-metrics-percentile-rank-aggregation.html[Percentile Ranks Aggregation] -with Java API. - - -===== Prepare aggregation request - -Here is an example on how to create the aggregation request: - -[source,java] --------------------------------------------------- -PercentileRanksAggregationBuilder aggregation = - AggregationBuilders - .percentileRanks("agg") - .field("height") - .values(1.24, 1.91, 2.22); --------------------------------------------------- - - -===== Use aggregation response - -Import Aggregation definition classes: - -[source,java] --------------------------------------------------- -import org.elasticsearch.search.aggregations.metrics.percentiles.Percentile; -import org.elasticsearch.search.aggregations.metrics.percentiles.PercentileRanks; --------------------------------------------------- - -[source,java] --------------------------------------------------- -// sr is here your SearchResponse object -PercentileRanks agg = sr.getAggregations().get("agg"); -// For each entry -for (Percentile entry : agg) { - double percent = entry.getPercent(); // Percent - double value = entry.getValue(); // Value - - logger.info("percent [{}], value [{}]", percent, value); -} --------------------------------------------------- - - -This will basically produce: - -[source,text] --------------------------------------------------- -percent [29.664353095090945], value [1.24] -percent [73.9335313461868], value [1.91] -percent [94.40095147327283], value [2.22] --------------------------------------------------- - diff --git a/docs/java-api/aggregations/metrics/scripted-metric-aggregation.asciidoc b/docs/java-api/aggregations/metrics/scripted-metric-aggregation.asciidoc deleted file mode 100644 index 5b68fa7be451f..0000000000000 --- a/docs/java-api/aggregations/metrics/scripted-metric-aggregation.asciidoc +++ /dev/null @@ -1,100 +0,0 @@ -[[java-aggs-metrics-scripted-metric]] -==== Scripted Metric Aggregation - -Here is how you can use -{ref}/search-aggregations-metrics-scripted-metric-aggregation.html[Scripted Metric Aggregation] -with Java API. - -===== Prepare aggregation request - -Here is an example on how to create the aggregation request: - -[source,java] --------------------------------------------------- -ScriptedMetricAggregationBuilder aggregation = AggregationBuilders - .scriptedMetric("agg") - .initScript(new Script("state.heights = []")) - .mapScript(new Script("state.heights.add(doc.gender.value == 'male' ? doc.height.value : -1.0 * doc.height.value)")); --------------------------------------------------- - -You can also specify a `combine` script which will be executed on each shard: - -[source,java] --------------------------------------------------- -ScriptedMetricAggregationBuilder aggregation = AggregationBuilders - .scriptedMetric("agg") - .initScript(new Script("state.heights = []")) - .mapScript(new Script("state.heights.add(doc.gender.value == 'male' ? doc.height.value : -1.0 * doc.height.value)")) - .combineScript(new Script("double heights_sum = 0.0; for (t in state.heights) { heights_sum += t } return heights_sum")); --------------------------------------------------- - -You can also specify a `reduce` script which will be executed on the node which gets the request: - -[source,java] --------------------------------------------------- -ScriptedMetricAggregationBuilder aggregation = AggregationBuilders - .scriptedMetric("agg") - .initScript(new Script("state.heights = []")) - .mapScript(new Script("state.heights.add(doc.gender.value == 'male' ? doc.height.value : -1.0 * doc.height.value)")) - .combineScript(new Script("double heights_sum = 0.0; for (t in state.heights) { heights_sum += t } return heights_sum")) - .reduceScript(new Script("double heights_sum = 0.0; for (a in states) { heights_sum += a } return heights_sum")); --------------------------------------------------- - - -===== Use aggregation response - -Import Aggregation definition classes: - -[source,java] --------------------------------------------------- -import org.elasticsearch.search.aggregations.bucket.terms.Terms; -import org.elasticsearch.search.aggregations.metrics.tophits.TopHits; --------------------------------------------------- - -[source,java] --------------------------------------------------- -// sr is here your SearchResponse object -ScriptedMetric agg = sr.getAggregations().get("agg"); -Object scriptedResult = agg.aggregation(); -logger.info("scriptedResult [{}]", scriptedResult); --------------------------------------------------- - -Note that the result depends on the script you built. -For the first example, this will basically produce: - -[source,text] --------------------------------------------------- -scriptedResult object [ArrayList] -scriptedResult [ { -"heights" : [ 1.122218480146643, -1.8148918111233887, -1.7626731575142909, ... ] -}, { -"heights" : [ -0.8046067304119863, -2.0785486707864553, -1.9183567430207953, ... ] -}, { -"heights" : [ 2.092635728868694, 1.5697545960886536, 1.8826954461968808, ... ] -}, { -"heights" : [ -2.1863201099468403, 1.6328549117346856, -1.7078288405893842, ... ] -}, { -"heights" : [ 1.6043904836424177, -2.0736538674414025, 0.9898266674373053, ... ] -} ] --------------------------------------------------- - -The second example will produce: - -[source,text] --------------------------------------------------- -scriptedResult object [ArrayList] -scriptedResult [-41.279615707402876, - -60.88007362339038, - 38.823270659734256, - 14.840192739445632, - 11.300902755741326] --------------------------------------------------- - -The last example will produce: - -[source,text] --------------------------------------------------- -scriptedResult object [Double] -scriptedResult [2.171917696507009] --------------------------------------------------- - diff --git a/docs/java-api/aggregations/metrics/stats-aggregation.asciidoc b/docs/java-api/aggregations/metrics/stats-aggregation.asciidoc deleted file mode 100644 index 260d9c01cb944..0000000000000 --- a/docs/java-api/aggregations/metrics/stats-aggregation.asciidoc +++ /dev/null @@ -1,41 +0,0 @@ -[[java-aggs-metrics-stats]] -==== Stats Aggregation - -Here is how you can use -{ref}/search-aggregations-metrics-stats-aggregation.html[Stats Aggregation] -with Java API. - - -===== Prepare aggregation request - -Here is an example on how to create the aggregation request: - -[source,java] --------------------------------------------------- -StatsAggregationBuilder aggregation = - AggregationBuilders - .stats("agg") - .field("height"); --------------------------------------------------- - - -===== Use aggregation response - -Import Aggregation definition classes: - -[source,java] --------------------------------------------------- -import org.elasticsearch.search.aggregations.metrics.stats.Stats; --------------------------------------------------- - -[source,java] --------------------------------------------------- -// sr is here your SearchResponse object -Stats agg = sr.getAggregations().get("agg"); -double min = agg.getMin(); -double max = agg.getMax(); -double avg = agg.getAvg(); -double sum = agg.getSum(); -long count = agg.getCount(); --------------------------------------------------- - diff --git a/docs/java-api/aggregations/metrics/sum-aggregation.asciidoc b/docs/java-api/aggregations/metrics/sum-aggregation.asciidoc deleted file mode 100644 index 453616916d755..0000000000000 --- a/docs/java-api/aggregations/metrics/sum-aggregation.asciidoc +++ /dev/null @@ -1,37 +0,0 @@ -[[java-aggs-metrics-sum]] -==== Sum Aggregation - -Here is how you can use -{ref}/search-aggregations-metrics-sum-aggregation.html[Sum Aggregation] -with Java API. - - -===== Prepare aggregation request - -Here is an example on how to create the aggregation request: - -[source,java] --------------------------------------------------- -SumAggregationBuilder aggregation = - AggregationBuilders - .sum("agg") - .field("height"); --------------------------------------------------- - - -===== Use aggregation response - -Import Aggregation definition classes: - -[source,java] --------------------------------------------------- -import org.elasticsearch.search.aggregations.metrics.sum.Sum; --------------------------------------------------- - -[source,java] --------------------------------------------------- -// sr is here your SearchResponse object -Sum agg = sr.getAggregations().get("agg"); -double value = agg.getValue(); --------------------------------------------------- - diff --git a/docs/java-api/aggregations/metrics/tophits-aggregation.asciidoc b/docs/java-api/aggregations/metrics/tophits-aggregation.asciidoc deleted file mode 100644 index 2473b4b89d77b..0000000000000 --- a/docs/java-api/aggregations/metrics/tophits-aggregation.asciidoc +++ /dev/null @@ -1,79 +0,0 @@ -[[java-aggs-metrics-tophits]] -==== Top Hits Aggregation - -Here is how you can use -{ref}/search-aggregations-metrics-top-hits-aggregation.html[Top Hits Aggregation] -with Java API. - - -===== Prepare aggregation request - -Here is an example on how to create the aggregation request: - -[source,java] --------------------------------------------------- -AggregationBuilder aggregation = - AggregationBuilders - .terms("agg").field("gender") - .subAggregation( - AggregationBuilders.topHits("top") - ); --------------------------------------------------- - -You can use most of the options available for standard search such as `from`, `size`, `sort`, `highlight`, `explain`... - -[source,java] --------------------------------------------------- -AggregationBuilder aggregation = - AggregationBuilders - .terms("agg").field("gender") - .subAggregation( - AggregationBuilders.topHits("top") - .explain(true) - .size(1) - .from(10) - ); --------------------------------------------------- - -===== Use aggregation response - -Import Aggregation definition classes: - -[source,java] --------------------------------------------------- -import org.elasticsearch.search.aggregations.bucket.terms.Terms; -import org.elasticsearch.search.aggregations.metrics.tophits.TopHits; --------------------------------------------------- - -[source,java] --------------------------------------------------- -// sr is here your SearchResponse object -Terms agg = sr.getAggregations().get("agg"); - -// For each entry -for (Terms.Bucket entry : agg.getBuckets()) { - String key = entry.getKey(); // bucket key - long docCount = entry.getDocCount(); // Doc count - logger.info("key [{}], doc_count [{}]", key, docCount); - - // We ask for top_hits for each bucket - TopHits topHits = entry.getAggregations().get("top"); - for (SearchHit hit : topHits.getHits().getHits()) { - logger.info(" -> id [{}], _source [{}]", hit.getId(), hit.getSourceAsString()); - } -} --------------------------------------------------- - -This will basically produce for the first example: - -[source,text] --------------------------------------------------- -key [male], doc_count [5107] - -> id [AUnzSZze9k7PKXtq04x2], _source [{"gender":"male",...}] - -> id [AUnzSZzj9k7PKXtq04x4], _source [{"gender":"male",...}] - -> id [AUnzSZzl9k7PKXtq04x5], _source [{"gender":"male",...}] -key [female], doc_count [4893] - -> id [AUnzSZzM9k7PKXtq04xy], _source [{"gender":"female",...}] - -> id [AUnzSZzp9k7PKXtq04x8], _source [{"gender":"female",...}] - -> id [AUnzSZ0W9k7PKXtq04yS], _source [{"gender":"female",...}] --------------------------------------------------- diff --git a/docs/java-api/aggregations/metrics/valuecount-aggregation.asciidoc b/docs/java-api/aggregations/metrics/valuecount-aggregation.asciidoc deleted file mode 100644 index b180d22af33cd..0000000000000 --- a/docs/java-api/aggregations/metrics/valuecount-aggregation.asciidoc +++ /dev/null @@ -1,37 +0,0 @@ -[[java-aggs-metrics-valuecount]] -==== Value Count Aggregation - -Here is how you can use -{ref}/search-aggregations-metrics-valuecount-aggregation.html[Value Count Aggregation] -with Java API. - - -===== Prepare aggregation request - -Here is an example on how to create the aggregation request: - -[source,java] --------------------------------------------------- -ValueCountAggregationBuilder aggregation = - AggregationBuilders - .count("agg") - .field("height"); --------------------------------------------------- - - -===== Use aggregation response - -Import Aggregation definition classes: - -[source,java] --------------------------------------------------- -import org.elasticsearch.search.aggregations.metrics.valuecount.ValueCount; --------------------------------------------------- - -[source,java] --------------------------------------------------- -// sr is here your SearchResponse object -ValueCount agg = sr.getAggregations().get("agg"); -long value = agg.getValue(); --------------------------------------------------- - diff --git a/docs/java-api/aggs.asciidoc b/docs/java-api/aggs.asciidoc deleted file mode 100644 index c2e09b4901e87..0000000000000 --- a/docs/java-api/aggs.asciidoc +++ /dev/null @@ -1,63 +0,0 @@ -[[java-aggs]] -== Aggregations - -Elasticsearch provides a full Java API to play with aggregations. See the -{ref}/search-aggregations.html[Aggregations guide]. - -Use the factory for aggregation builders (`AggregationBuilders`) and add each aggregation -you want to compute when querying and add it to your search request: - -[source,java] --------------------------------------------------- -SearchResponse sr = node.client().prepareSearch() - .setQuery( /* your query */ ) - .addAggregation( /* add an aggregation */ ) - .execute().actionGet(); --------------------------------------------------- - -Note that you can add more than one aggregation. See -{ref}/search-search.html[Search Java API] for details. - -To build aggregation requests, use `AggregationBuilders` helpers. Just import them -in your class: - -[source,java] --------------------------------------------------- -import org.elasticsearch.search.aggregations.AggregationBuilders; --------------------------------------------------- - -=== Structuring aggregations - -As explained in the -{ref}/search-aggregations.html[Aggregations guide], you can define -sub aggregations inside an aggregation. - -An aggregation could be a metrics aggregation or a bucket aggregation. - -For example, here is a 3 levels aggregation composed of: - -* Terms aggregation (bucket) -* Date Histogram aggregation (bucket) -* Average aggregation (metric) - -[source,java] --------------------------------------------------- -SearchResponse sr = node.client().prepareSearch() - .addAggregation( - AggregationBuilders.terms("by_country").field("country") - .subAggregation(AggregationBuilders.dateHistogram("by_year") - .field("dateOfBirth") - .calendarInterval(DateHistogramInterval.YEAR) - .subAggregation(AggregationBuilders.avg("avg_children").field("children")) - ) - ) - .execute().actionGet(); --------------------------------------------------- - -=== Metrics aggregations - -include::aggregations/metrics.asciidoc[] - -=== Bucket aggregations - -include::aggregations/bucket.asciidoc[] diff --git a/docs/java-api/client.asciidoc b/docs/java-api/client.asciidoc deleted file mode 100644 index 811d7c398d940..0000000000000 --- a/docs/java-api/client.asciidoc +++ /dev/null @@ -1,110 +0,0 @@ -[[client]] -== Client - -You can use the *Java client* in multiple ways: - -* Perform standard <>, <>, - <> and <> operations on an - existing cluster -* Perform administrative tasks on a running cluster - -Obtaining an Elasticsearch `Client` is simple. The most common way to -get a client is by creating a <> -that connects to a cluster. - -[IMPORTANT] -============================== - -The client must have the same major version (e.g. `2.x`, or `5.x`) as the -nodes in the cluster. Clients may connect to clusters which have a different -minor version (e.g. `2.3.x`) but it is possible that new functionality may not -be supported. Ideally, the client should have the same version as the -cluster. - -============================== - -[[transport-client]] -=== Transport Client - -deprecated[7.0.0, The `TransportClient` is deprecated in favour of the {java-rest}/java-rest-high.html[Java High Level REST Client] and will be removed in Elasticsearch 8.0. The {java-rest}/java-rest-high-level-migration.html[migration guide] describes all the steps needed to migrate.] - -The `TransportClient` connects remotely to an Elasticsearch cluster -using the transport module. It does not join the cluster, but simply -gets one or more initial transport addresses and communicates with them -in round robin fashion on each action (though most actions will probably -be "two hop" operations). - -[source,java] --------------------------------------------------- -// on startup - -TransportClient client = new PreBuiltTransportClient(Settings.EMPTY) - .addTransportAddress(new TransportAddress(InetAddress.getByName("host1"), 9300)) - .addTransportAddress(new TransportAddress(InetAddress.getByName("host2"), 9300)); - -// on shutdown - -client.close(); --------------------------------------------------- - -Note that you have to set the cluster name if you use one different than -"elasticsearch": - -[source,java] --------------------------------------------------- -Settings settings = Settings.builder() - .put("cluster.name", "myClusterName").build(); -TransportClient client = new PreBuiltTransportClient(settings); -//Add transport addresses and do something with the client... --------------------------------------------------- - -The Transport client comes with a cluster sniffing feature which -allows it to dynamically add new hosts and remove old ones. -When sniffing is enabled, the transport client will connect to the nodes in its -internal node list, which is built via calls to `addTransportAddress`. -After this, the client will call the internal cluster state API on those nodes -to discover available data nodes. The internal node list of the client will -be replaced with those data nodes only. This list is refreshed every five seconds by default. -Note that the IP addresses the sniffer connects to are the ones declared as the 'publish' -address in those node's Elasticsearch config. - -Keep in mind that the list might possibly not include the original node it connected to -if that node is not a data node. If, for instance, you initially connect to a -master node, after sniffing, no further requests will go to that master node, -but rather to any data nodes instead. The reason the transport client excludes non-data -nodes is to avoid sending search traffic to master only nodes. - -In order to enable sniffing, set `client.transport.sniff` to `true`: - -[source,java] --------------------------------------------------- -Settings settings = Settings.builder() - .put("client.transport.sniff", true).build(); -TransportClient client = new PreBuiltTransportClient(settings); --------------------------------------------------- - -Other transport client level settings include: - -[cols="<,<",options="header",] -|======================================================================= -|Parameter |Description -|`client.transport.ignore_cluster_name` |Set to `true` to ignore cluster -name validation of connected nodes. (since 0.19.4) - -|`client.transport.ping_timeout` |The time to wait for a ping response -from a node. Defaults to `5s`. - -|`client.transport.nodes_sampler_interval` |How often to sample / ping -the nodes listed and connected. Defaults to `5s`. -|======================================================================= - - -[[client-connected-to-client-node]] -=== Connecting a Client to a Coordinating Only Node - -You can start locally a {ref}/modules-node.html#coordinating-only-node[Coordinating Only Node] -and then simply create a <> in your -application which connects to this Coordinating Only Node. - -This way, the coordinating only node will be able to load whatever plugin you -need (think about discovery plugins for example). diff --git a/docs/java-api/docs.asciidoc b/docs/java-api/docs.asciidoc deleted file mode 100644 index 181c5d8e0bd99..0000000000000 --- a/docs/java-api/docs.asciidoc +++ /dev/null @@ -1,36 +0,0 @@ -[[java-docs]] -== Document APIs - -This section describes the following CRUD APIs: - -.Single document APIs -* <> -* <> -* <> -* <> - -.Multi-document APIs -* <> -* <> -* <> -* <> -* <> - -NOTE: All CRUD APIs are single-index APIs. The `index` parameter accepts a single -index name, or an `alias` which points to a single index. - -include::docs/index_.asciidoc[] - -include::docs/get.asciidoc[] - -include::docs/delete.asciidoc[] - -include::docs/update.asciidoc[] - -include::docs/multi-get.asciidoc[] - -include::docs/bulk.asciidoc[] - -include::docs/update-by-query.asciidoc[] - -include::docs/reindex.asciidoc[] \ No newline at end of file diff --git a/docs/java-api/docs/bulk.asciidoc b/docs/java-api/docs/bulk.asciidoc deleted file mode 100644 index 1c2882d9c07e7..0000000000000 --- a/docs/java-api/docs/bulk.asciidoc +++ /dev/null @@ -1,190 +0,0 @@ -[[java-docs-bulk]] -=== Bulk API - -The bulk API allows one to index and delete several documents in a -single request. Here is a sample usage: - -[source,java] --------------------------------------------------- -import static org.elasticsearch.common.xcontent.XContentFactory.*; - -BulkRequestBuilder bulkRequest = client.prepareBulk(); - -// either use client#prepare, or use Requests# to directly build index/delete requests -bulkRequest.add(client.prepareIndex("twitter", "_doc", "1") - .setSource(jsonBuilder() - .startObject() - .field("user", "kimchy") - .field("postDate", new Date()) - .field("message", "trying out Elasticsearch") - .endObject() - ) - ); - -bulkRequest.add(client.prepareIndex("twitter", "_doc", "2") - .setSource(jsonBuilder() - .startObject() - .field("user", "kimchy") - .field("postDate", new Date()) - .field("message", "another post") - .endObject() - ) - ); - -BulkResponse bulkResponse = bulkRequest.get(); -if (bulkResponse.hasFailures()) { - // process failures by iterating through each bulk response item -} --------------------------------------------------- - -[[java-docs-bulk-processor]] -=== Using Bulk Processor - -The `BulkProcessor` class offers a simple interface to flush bulk operations automatically based on the number or size -of requests, or after a given period. - -To use it, first create a `BulkProcessor` instance: - -[source,java] --------------------------------------------------- -import org.elasticsearch.action.bulk.BackoffPolicy; -import org.elasticsearch.action.bulk.BulkProcessor; -import org.elasticsearch.common.unit.ByteSizeUnit; -import org.elasticsearch.common.unit.ByteSizeValue; -import org.elasticsearch.common.unit.TimeValue; - -BulkProcessor bulkProcessor = BulkProcessor.builder( - client, <1> - new BulkProcessor.Listener() { - @Override - public void beforeBulk(long executionId, - BulkRequest request) { ... } <2> - - @Override - public void afterBulk(long executionId, - BulkRequest request, - BulkResponse response) { ... } <3> - - @Override - public void afterBulk(long executionId, - BulkRequest request, - Throwable failure) { ... } <4> - }) - .setBulkActions(10000) <5> - .setBulkSize(new ByteSizeValue(5, ByteSizeUnit.MB)) <6> - .setFlushInterval(TimeValue.timeValueSeconds(5)) <7> - .setConcurrentRequests(1) <8> - .setBackoffPolicy( - BackoffPolicy.exponentialBackoff(TimeValue.timeValueMillis(100), 3)) <9> - .build(); --------------------------------------------------- -<1> Add your Elasticsearch client -<2> This method is called just before bulk is executed. You can for example see the numberOfActions with - `request.numberOfActions()` -<3> This method is called after bulk execution. You can for example check if there was some failing requests - with `response.hasFailures()` -<4> This method is called when the bulk failed and raised a `Throwable` -<5> We want to execute the bulk every 10 000 requests -<6> We want to flush the bulk every 5mb -<7> We want to flush the bulk every 5 seconds whatever the number of requests -<8> Set the number of concurrent requests. A value of 0 means that only a single request will be allowed to be - executed. A value of 1 means 1 concurrent request is allowed to be executed while accumulating new bulk requests. -<9> Set a custom backoff policy which will initially wait for 100ms, increase exponentially and retries up to three - times. A retry is attempted whenever one or more bulk item requests have failed with an `EsRejectedExecutionException` - which indicates that there were too little compute resources available for processing the request. To disable backoff, - pass `BackoffPolicy.noBackoff()`. - -By default, `BulkProcessor`: - -* sets bulkActions to `1000` -* sets bulkSize to `5mb` -* does not set flushInterval -* sets concurrentRequests to 1, which means an asynchronous execution of the flush operation. -* sets backoffPolicy to an exponential backoff with 8 retries and a start delay of 50ms. The total wait time is roughly 5.1 seconds. - -[[java-docs-bulk-processor-requests]] -==== Add requests - -Then you can simply add your requests to the `BulkProcessor`: - -[source,java] --------------------------------------------------- -bulkProcessor.add(new IndexRequest("twitter", "_doc", "1").source(/* your doc here */)); -bulkProcessor.add(new DeleteRequest("twitter", "_doc", "2")); --------------------------------------------------- - -[[java-docs-bulk-processor-close]] -==== Closing the Bulk Processor - -When all documents are loaded to the `BulkProcessor` it can be closed by using `awaitClose` or `close` methods: - -[source,java] --------------------------------------------------- -bulkProcessor.awaitClose(10, TimeUnit.MINUTES); --------------------------------------------------- - -or - -[source,java] --------------------------------------------------- -bulkProcessor.close(); --------------------------------------------------- - -Both methods flush any remaining documents and disable all other scheduled flushes if they were scheduled by setting -`flushInterval`. If concurrent requests were enabled the `awaitClose` method waits for up to the specified timeout for -all bulk requests to complete then returns `true`, if the specified waiting time elapses before all bulk requests complete, -`false` is returned. The `close` method doesn't wait for any remaining bulk requests to complete and exits immediately. - -[[java-docs-bulk-processor-tests]] -==== Using Bulk Processor in tests - -If you are running tests with Elasticsearch and are using the `BulkProcessor` to populate your dataset -you should better set the number of concurrent requests to `0` so the flush operation of the bulk will be executed -in a synchronous manner: - -[source,java] --------------------------------------------------- -BulkProcessor bulkProcessor = BulkProcessor.builder(client, new BulkProcessor.Listener() { /* Listener methods */ }) - .setBulkActions(10000) - .setConcurrentRequests(0) - .build(); - -// Add your requests -bulkProcessor.add(/* Your requests */); - -// Flush any remaining requests -bulkProcessor.flush(); - -// Or close the bulkProcessor if you don't need it anymore -bulkProcessor.close(); - -// Refresh your indices -client.admin().indices().prepareRefresh().get(); - -// Now you can start searching! -client.prepareSearch().get(); --------------------------------------------------- - - -[[java-docs-bulk-global-parameters]] -==== Global Parameters - -Global parameters can be specified on the BulkRequest as well as BulkProcessor, similar to the REST API. These global - parameters serve as defaults and can be overridden by local parameters specified on each sub request. Some parameters - have to be set before any sub request is added - index, type - and you have to specify them during BulkRequest or - BulkProcessor creation. Some are optional - pipeline, routing - and can be specified at any point before the bulk is sent. - -["source","java",subs="attributes,callouts,macros"] --------------------------------------------------- -include-tagged::{hlrc-tests}/BulkProcessorIT.java[bulk-processor-mix-parameters] --------------------------------------------------- -<1> global parameters from the BulkRequest will be applied on a sub request -<2> local pipeline parameter on a sub request will override global parameters from BulkRequest - - -["source","java",subs="attributes,callouts,macros"] --------------------------------------------------- -include-tagged::{hlrc-tests}/BulkRequestWithGlobalParametersIT.java[bulk-request-mix-pipeline] --------------------------------------------------- -<1> local pipeline parameter on a sub request will override global pipeline from the BulkRequest -<2> global parameter from the BulkRequest will be applied on a sub request diff --git a/docs/java-api/docs/delete.asciidoc b/docs/java-api/docs/delete.asciidoc deleted file mode 100644 index 004edc84b3d65..0000000000000 --- a/docs/java-api/docs/delete.asciidoc +++ /dev/null @@ -1,42 +0,0 @@ -[[java-docs-delete]] -=== Delete API - -The delete API allows one to delete a typed JSON document from a specific -index based on its id. The following example deletes the JSON document -from an index called twitter, under a type called `_doc`, with id valued -1: - -[source,java] --------------------------------------------------- -DeleteResponse response = client.prepareDelete("twitter", "_doc", "1").get(); --------------------------------------------------- - -For more information on the delete operation, check out the -{ref}/docs-delete.html[delete API] docs. - -[[java-docs-delete-by-query]] -=== Delete By Query API - -The delete by query API allows one to delete a given set of documents based on -the result of a query: - -["source","java",subs="attributes,callouts,macros"] --------------------------------------------------- -include-tagged::{client-reindex-tests}/ReindexDocumentationIT.java[delete-by-query-sync] --------------------------------------------------- -<1> query -<2> index -<3> execute the operation -<4> number of deleted documents - -As it can be a long running operation, if you wish to do it asynchronously, you can call `execute` instead of `get` -and provide a listener like: - -["source","java",subs="attributes,callouts,macros"] --------------------------------------------------- -include-tagged::{client-reindex-tests}/ReindexDocumentationIT.java[delete-by-query-async] --------------------------------------------------- -<1> query -<2> index -<3> listener -<4> number of deleted documents diff --git a/docs/java-api/docs/get.asciidoc b/docs/java-api/docs/get.asciidoc deleted file mode 100644 index ae03eb971004f..0000000000000 --- a/docs/java-api/docs/get.asciidoc +++ /dev/null @@ -1,14 +0,0 @@ -[[java-docs-get]] -=== Get API - -The get API allows to get a typed JSON document from the index based on -its id. The following example gets a JSON document from an index called -twitter, under a type called `_doc``, with id valued 1: - -[source,java] --------------------------------------------------- -GetResponse response = client.prepareGet("twitter", "_doc", "1").get(); --------------------------------------------------- - -For more information on the get operation, check out the REST -{ref}/docs-get.html[get] docs. diff --git a/docs/java-api/docs/index_.asciidoc b/docs/java-api/docs/index_.asciidoc deleted file mode 100644 index 80d187d5a77f3..0000000000000 --- a/docs/java-api/docs/index_.asciidoc +++ /dev/null @@ -1,167 +0,0 @@ -[[java-docs-index]] -=== Index API - -The index API allows one to index a typed JSON document into a specific -index and make it searchable. - - -[[java-docs-index-generate]] -==== Generate JSON document - -There are several different ways of generating a JSON document: - -* Manually (aka do it yourself) using native `byte[]` or as a `String` - -* Using a `Map` that will be automatically converted to its JSON -equivalent - -* Using a third party library to serialize your beans such as -https://github.com/FasterXML/jackson[Jackson] - -* Using built-in helpers XContentFactory.jsonBuilder() - -Internally, each type is converted to `byte[]` (so a String is converted -to a `byte[]`). Therefore, if the object is in this form already, then -use it. The `jsonBuilder` is highly optimized JSON generator that -directly constructs a `byte[]`. - - -[[java-docs-index-generate-diy]] -===== Do It Yourself - -Nothing really difficult here but note that you will have to encode -dates according to the -{ref}/mapping-date-format.html[Date Format]. - -[source,java] --------------------------------------------------- -String json = "{" + - "\"user\":\"kimchy\"," + - "\"postDate\":\"2013-01-30\"," + - "\"message\":\"trying out Elasticsearch\"" + - "}"; --------------------------------------------------- - - -[[java-docs-index-generate-using-map]] -===== Using Map - -Map is a key:values pair collection. It represents a JSON structure: - -[source,java] --------------------------------------------------- -Map json = new HashMap(); -json.put("user","kimchy"); -json.put("postDate",new Date()); -json.put("message","trying out Elasticsearch"); --------------------------------------------------- - - -[[java-docs-index-generate-beans]] -===== Serialize your beans - -You can use https://github.com/FasterXML/jackson[Jackson] to serialize -your beans to JSON. Please add http://search.maven.org/#search%7Cga%7C1%7Cjackson-databind[Jackson Databind] - to your project. Then you can use `ObjectMapper` to serialize your beans: - -[source,java] --------------------------------------------------- -import com.fasterxml.jackson.databind.*; - -// instance a json mapper -ObjectMapper mapper = new ObjectMapper(); // create once, reuse - -// generate json -byte[] json = mapper.writeValueAsBytes(yourbeaninstance); --------------------------------------------------- - - -[[java-docs-index-generate-helpers]] -===== Use Elasticsearch helpers - -Elasticsearch provides built-in helpers to generate JSON content. - -[source,java] --------------------------------------------------- -import static org.elasticsearch.common.xcontent.XContentFactory.*; - -XContentBuilder builder = jsonBuilder() - .startObject() - .field("user", "kimchy") - .field("postDate", new Date()) - .field("message", "trying out Elasticsearch") - .endObject() --------------------------------------------------- - -Note that you can also add arrays with `startArray(String)` and -`endArray()` methods. By the way, the `field` method + - accepts many object types. You can directly pass numbers, dates and even -other XContentBuilder objects. - -If you need to see the generated JSON content, you can use the -`Strings.toString()` method. - -[source,java] --------------------------------------------------- -import org.elasticsearch.common.Strings; - -String json = Strings.toString(builder); --------------------------------------------------- - - -[[java-docs-index-doc]] -==== Index document - -The following example indexes a JSON document into an index called -twitter, under a type called `_doc``, with id valued 1: - -[source,java] --------------------------------------------------- -import static org.elasticsearch.common.xcontent.XContentFactory.*; - -IndexResponse response = client.prepareIndex("twitter", "_doc", "1") - .setSource(jsonBuilder() - .startObject() - .field("user", "kimchy") - .field("postDate", new Date()) - .field("message", "trying out Elasticsearch") - .endObject() - ) - .get(); --------------------------------------------------- - -Note that you can also index your documents as JSON String and that you -don't have to give an ID: - -[source,java] --------------------------------------------------- -String json = "{" + - "\"user\":\"kimchy\"," + - "\"postDate\":\"2013-01-30\"," + - "\"message\":\"trying out Elasticsearch\"" + - "}"; - -IndexResponse response = client.prepareIndex("twitter", "_doc") -       .setSource(json, XContentType.JSON) - .get(); --------------------------------------------------- - -`IndexResponse` object will give you a report: - -[source,java] --------------------------------------------------- -// Index name -String _index = response.getIndex(); -// Type name -String _type = response.getType(); -// Document ID (generated or not) -String _id = response.getId(); -// Version (if it's the first time you index this document, you will get: 1) -long _version = response.getVersion(); -// status has stored current instance statement. -RestStatus status = response.status(); --------------------------------------------------- - -For more information on the index operation, check out the REST -{ref}/docs-index_.html[index] docs. - diff --git a/docs/java-api/docs/multi-get.asciidoc b/docs/java-api/docs/multi-get.asciidoc deleted file mode 100644 index 8ed2bede2927c..0000000000000 --- a/docs/java-api/docs/multi-get.asciidoc +++ /dev/null @@ -1,30 +0,0 @@ -[[java-docs-multi-get]] -=== Multi Get API - -The multi get API allows to get a list of documents based on their `index` and `id`: - -[source,java] --------------------------------------------------- -MultiGetResponse multiGetItemResponses = client.prepareMultiGet() - .add("twitter", "_doc", "1") <1> - .add("twitter", "_doc", "2", "3", "4") <2> - .add("another", "_doc", "foo") <3> - .get(); - -for (MultiGetItemResponse itemResponse : multiGetItemResponses) { <4> - GetResponse response = itemResponse.getResponse(); - if (response.isExists()) { <5> - String json = response.getSourceAsString(); <6> - } -} --------------------------------------------------- -<1> get by a single id -<2> or by a list of ids for the same index -<3> you can also get from another index -<4> iterate over the result set -<5> you can check if the document exists -<6> access to the `_source` field - -For more information on the multi get operation, check out the REST -{ref}/docs-multi-get.html[multi get] docs. - diff --git a/docs/java-api/docs/reindex.asciidoc b/docs/java-api/docs/reindex.asciidoc deleted file mode 100644 index 842e763f74d71..0000000000000 --- a/docs/java-api/docs/reindex.asciidoc +++ /dev/null @@ -1,11 +0,0 @@ -[[java-docs-reindex]] -=== Reindex API - -See {ref}/docs-reindex.html[reindex API]. - -["source","java",subs="attributes,callouts,macros"] --------------------------------------------------- -include-tagged::{client-reindex-tests}/ReindexDocumentationIT.java[reindex1] --------------------------------------------------- -<1> Optionally a query can provided to filter what documents should be - re-indexed from the source to the target index. diff --git a/docs/java-api/docs/update-by-query.asciidoc b/docs/java-api/docs/update-by-query.asciidoc deleted file mode 100644 index ef58d3754276e..0000000000000 --- a/docs/java-api/docs/update-by-query.asciidoc +++ /dev/null @@ -1,166 +0,0 @@ -[[java-docs-update-by-query]] -=== Update By Query API - -The simplest usage of `updateByQuery` updates each -document in an index without changing the source. This usage enables -picking up a new property or another online mapping change. - -["source","java",subs="attributes,callouts,macros"] --------------------------------------------------- -include-tagged::{client-reindex-tests}/ReindexDocumentationIT.java[update-by-query] --------------------------------------------------- - -Calls to the `updateByQuery` API start by getting a snapshot of the index, indexing -any documents found using the `internal` versioning. - -NOTE: Version conflicts happen when a document changes between the time of the -snapshot and the time the index request processes. - -When the versions match, `updateByQuery` updates the document -and increments the version number. - -All update and query failures cause `updateByQuery` to abort. These failures are -available from the `BulkByScrollResponse#getIndexingFailures` method. Any -successful updates remain and are not rolled back. While the first failure -causes the abort, the response contains all of the failures generated by the -failed bulk request. - -To prevent version conflicts from causing `updateByQuery` to abort, set -`abortOnVersionConflict(false)`. The first example does this because it is -trying to pick up an online mapping change and a version conflict means that -the conflicting document was updated between the start of the `updateByQuery` -and the time when it attempted to update the document. This is fine because -that update will have picked up the online mapping update. - -The `UpdateByQueryRequestBuilder` API supports filtering the updated documents, -limiting the total number of documents to update, and updating documents -with a script: - - -["source","java",subs="attributes,callouts,macros"] --------------------------------------------------- -include-tagged::{client-reindex-tests}/ReindexDocumentationIT.java[update-by-query-filter] --------------------------------------------------- - -`UpdateByQueryRequestBuilder` also enables direct access to the query used -to select the documents. You can use this access to change the default scroll size or -otherwise modify the request for matching documents. - -["source","java",subs="attributes,callouts,macros"] --------------------------------------------------- -include-tagged::{client-reindex-tests}/ReindexDocumentationIT.java[update-by-query-size] --------------------------------------------------- - -You can also combine `size` with sorting to limit the documents updated: - -["source","java",subs="attributes,callouts,macros"] --------------------------------------------------- -include-tagged::{client-reindex-tests}/ReindexDocumentationIT.java[update-by-query-sort] --------------------------------------------------- - -In addition to changing the `_source` field for the document, you can use a -script to change the action, similar to the Update API: - -["source","java",subs="attributes,callouts,macros"] --------------------------------------------------- -include-tagged::{client-reindex-tests}/ReindexDocumentationIT.java[update-by-query-script] --------------------------------------------------- - -As in the <>, you can set the value of `ctx.op` to change the -operation that executes: - -`noop`:: - -Set `ctx.op = "noop"` if your script doesn't make any -changes. The `updateByQuery` operation then omits that document from the updates. -This behavior increments the `noop` counter in the response body. - -`delete`:: - -Set `ctx.op = "delete"` if your script decides that the document must be -deleted. The deletion will be reported in the `deleted` counter in the -response body. - -Setting `ctx.op` to any other value generates an error. Setting any -other field in `ctx` generates an error. - -This API doesn't allow you to move the documents it touches, just modify their -source. This is intentional! We've made no provisions for removing the document -from its original location. - -You can also perform these operations on multiple indices at once, similar to the search API: - -["source","java",subs="attributes,callouts,macros"] --------------------------------------------------- -include-tagged::{client-reindex-tests}/ReindexDocumentationIT.java[update-by-query-multi-index] --------------------------------------------------- - -If you provide a `routing` value then the process copies the routing value to the scroll query, -limiting the process to the shards that match that routing value: - -["source","java",subs="attributes,callouts,macros"] --------------------------------------------------- -include-tagged::{client-reindex-tests}/ReindexDocumentationIT.java[update-by-query-routing] --------------------------------------------------- - -`updateByQuery` can also use the ingest node by -specifying a `pipeline` like this: - -["source","java",subs="attributes,callouts,macros"] --------------------------------------------------- -include-tagged::{client-reindex-tests}/ReindexDocumentationIT.java[update-by-query-pipeline] --------------------------------------------------- - -[float] -[[java-docs-update-by-query-task-api]] -=== Works with the Task API - -You can fetch the status of all running update-by-query requests with the Task API: - -["source","java",subs="attributes,callouts,macros"] --------------------------------------------------- -include-tagged::{client-reindex-tests}/ReindexDocumentationIT.java[update-by-query-list-tasks] --------------------------------------------------- - -With the `TaskId` shown above you can look up the task directly: - -// provide API Example -["source","java",subs="attributes,callouts,macros"] --------------------------------------------------- -include-tagged::{client-reindex-tests}/ReindexDocumentationIT.java[update-by-query-get-task] --------------------------------------------------- - -[float] -[[java-docs-update-by-query-cancel-task-api]] -=== Works with the Cancel Task API - -Any Update By Query can be canceled using the Task Cancel API: - -["source","java",subs="attributes,callouts,macros"] --------------------------------------------------- -include-tagged::{client-reindex-tests}/ReindexDocumentationIT.java[update-by-query-cancel-task] --------------------------------------------------- - -Use the `list tasks` API to find the value of `taskId`. - -Cancelling a request is typically a very fast process but can take up to a few seconds. -The task status API continues to list the task until the cancellation is complete. - -[float] -[[java-docs-update-by-query-rethrottle]] -=== Rethrottling - -Use the `_rethrottle` API to change the value of `requests_per_second` on a running update: - -["source","java",subs="attributes,callouts,macros"] --------------------------------------------------- -include-tagged::{client-reindex-tests}/ReindexDocumentationIT.java[update-by-query-rethrottle] --------------------------------------------------- - -Use the `list tasks` API to find the value of `taskId`. - -As with the `updateByQuery` API, the value of `requests_per_second` -can be any positive float value to set the level of the throttle, or `Float.POSITIVE_INFINITY` to disable throttling. -A value of `requests_per_second` that speeds up the process takes -effect immediately. `requests_per_second` values that slow the query take effect -after completing the current batch in order to prevent scroll timeouts. diff --git a/docs/java-api/docs/update.asciidoc b/docs/java-api/docs/update.asciidoc deleted file mode 100644 index 0935c9f11eca4..0000000000000 --- a/docs/java-api/docs/update.asciidoc +++ /dev/null @@ -1,118 +0,0 @@ -[[java-docs-update]] -=== Update API - - -You can either create an `UpdateRequest` and send it to the client: - -[source,java] --------------------------------------------------- -UpdateRequest updateRequest = new UpdateRequest(); -updateRequest.index("index"); -updateRequest.type("_doc"); -updateRequest.id("1"); -updateRequest.doc(jsonBuilder() - .startObject() - .field("gender", "male") - .endObject()); -client.update(updateRequest).get(); --------------------------------------------------- - -Or you can use `prepareUpdate()` method: - -[source,java] --------------------------------------------------- -client.prepareUpdate("ttl", "doc", "1") - .setScript(new Script( - "ctx._source.gender = \"male\"", <1> - ScriptService.ScriptType.INLINE, null, null)) - .get(); - -client.prepareUpdate("ttl", "doc", "1") - .setDoc(jsonBuilder() <2> - .startObject() - .field("gender", "male") - .endObject()) - .get(); --------------------------------------------------- -<1> Your script. It could also be a locally stored script name. -In that case, you'll need to use `ScriptService.ScriptType.FILE` -<2> Document which will be merged to the existing one. - -Note that you can't provide both `script` and `doc`. - -[[java-docs-update-api-script]] -==== Update by script - -The update API allows to update a document based on a script provided: - -[source,java] --------------------------------------------------- -UpdateRequest updateRequest = new UpdateRequest("ttl", "doc", "1") - .script(new Script("ctx._source.gender = \"male\"")); -client.update(updateRequest).get(); --------------------------------------------------- - - -[[java-docs-update-api-merge-docs]] -==== Update by merging documents - -The update API also support passing a partial document, which will be merged into the existing document (simple -recursive merge, inner merging of objects, replacing core "keys/values" and arrays). For example: - -[source,java] --------------------------------------------------- -UpdateRequest updateRequest = new UpdateRequest("index", "type", "1") - .doc(jsonBuilder() - .startObject() - .field("gender", "male") - .endObject()); -client.update(updateRequest).get(); --------------------------------------------------- - - -[[java-docs-update-api-upsert]] -==== Upsert - -There is also support for `upsert`. If the document does not exist, the content of the `upsert` -element will be used to index the fresh doc: - -[source,java] --------------------------------------------------- -IndexRequest indexRequest = new IndexRequest("index", "type", "1") - .source(jsonBuilder() - .startObject() - .field("name", "Joe Smith") - .field("gender", "male") - .endObject()); -UpdateRequest updateRequest = new UpdateRequest("index", "type", "1") - .doc(jsonBuilder() - .startObject() - .field("gender", "male") - .endObject()) - .upsert(indexRequest); <1> -client.update(updateRequest).get(); --------------------------------------------------- -<1> If the document does not exist, the one in `indexRequest` will be added - -If the document `index/_doc/1` already exists, we will have after this operation a document like: - -[source,js] --------------------------------------------------- -{ - "name" : "Joe Dalton", - "gender": "male" <1> -} --------------------------------------------------- -// NOTCONSOLE -<1> This field is added by the update request - -If it does not exist, we will have a new document: - -[source,js] --------------------------------------------------- -{ - "name" : "Joe Smith", - "gender": "male" -} --------------------------------------------------- -// NOTCONSOLE diff --git a/docs/java-api/index.asciidoc b/docs/java-api/index.asciidoc deleted file mode 100644 index 4a7fd7482d26e..0000000000000 --- a/docs/java-api/index.asciidoc +++ /dev/null @@ -1,149 +0,0 @@ -= Java API - -include::../Versions.asciidoc[] - -[[java-api]] -[preface] -== Preface - -deprecated[7.0.0, The `TransportClient` is deprecated in favour of the {java-rest}/java-rest-high.html[Java High Level REST Client] and will be removed in Elasticsearch 8.0. The {java-rest}/java-rest-high-level-migration.html[migration guide] describes all the steps needed to migrate.] - -This section describes the Java API that Elasticsearch provides. All -Elasticsearch operations are executed using a -<> object. All -operations are completely asynchronous in nature (either accepts a -listener, or returns a future). - -Additionally, operations on a client may be accumulated and executed in -<>. - -Note, all the APIs are exposed through the -Java API (actually, the Java API is used internally to execute them). - -== Javadoc - -The javadoc for the transport client can be found at {transport-client-javadoc}/index.html. - -== Maven Repository - -Elasticsearch is hosted on -http://search.maven.org/#search%7Cga%7C1%7Cg%3A%22org.elasticsearch.client%22[Maven -Central]. - -For example, you can define the latest version in your `pom.xml` file: - -["source","xml",subs="attributes"] --------------------------------------------------- - - org.elasticsearch.client - transport - {version} - --------------------------------------------------- - -[[java-transport-usage-maven-lucene]] -=== Lucene Snapshot repository - -The very first releases of any major version (like a beta), might have been built on top of a Lucene Snapshot version. -In such a case you will be unable to resolve the Lucene dependencies of the client. - -For example, if you want to use the `6.0.0-beta1` version which depends on Lucene `7.0.0-snapshot-00142c9`, you must -define the following repository. - -For Maven: - -["source","xml",subs="attributes"] --------------------------------------------------- - - elastic-lucene-snapshots - Elastic Lucene Snapshots - https://s3.amazonaws.com/download.elasticsearch.org/lucenesnapshots/00142c9 - true - false - --------------------------------------------------- - -For Gradle: - -["source","groovy",subs="attributes"] --------------------------------------------------- -maven { - name "lucene-snapshots" - url 'https://s3.amazonaws.com/download.elasticsearch.org/lucenesnapshots/00142c9' -} --------------------------------------------------- - -=== Log4j 2 Logger - -You need to also include Log4j 2 dependencies: - -["source","xml",subs="attributes"] --------------------------------------------------- - - org.apache.logging.log4j - log4j-core - 2.11.1 - --------------------------------------------------- - -And also provide a Log4j 2 configuration file in your classpath. -For example, you can add in your `src/main/resources` project dir a `log4j2.properties` file like: - - -["source","properties",subs="attributes"] --------------------------------------------------- -appender.console.type = Console -appender.console.name = console -appender.console.layout.type = PatternLayout -appender.console.layout.pattern = [%d{ISO8601}][%-5p][%-25c] %marker%m%n - -rootLogger.level = info -rootLogger.appenderRef.console.ref = console --------------------------------------------------- - -=== Using another Logger - -If you want to use another logger than Log4j 2, you can use http://www.slf4j.org/[SLF4J] bridge to do that: - -["source","xml",subs="attributes"] --------------------------------------------------- - - org.apache.logging.log4j - log4j-to-slf4j - 2.11.1 - - - org.slf4j - slf4j-api - 1.7.24 - --------------------------------------------------- - -http://www.slf4j.org/manual.html[This page] lists implementations you can use. Pick your favorite logger -and add it as a dependency. As an example, we will use the `slf4j-simple` logger: - -["source","xml",subs="attributes"] --------------------------------------------------- - - org.slf4j - slf4j-simple - 1.7.21 - --------------------------------------------------- - -:client-tests: {docdir}/../../server/src/test/java/org/elasticsearch/client/documentation -:hlrc-tests: {docdir}/../../client/rest-high-level/src/test/java/org/elasticsearch/client - -:client-reindex-tests: {docdir}/../../modules/reindex/src/test/java/org/elasticsearch/client/documentation - -include::client.asciidoc[] - -include::docs.asciidoc[] - -include::search.asciidoc[] - -include::aggs.asciidoc[] - -include::query-dsl.asciidoc[] - -include::admin/index.asciidoc[] diff --git a/docs/java-api/query-dsl.asciidoc b/docs/java-api/query-dsl.asciidoc deleted file mode 100644 index f4823fa08ab51..0000000000000 --- a/docs/java-api/query-dsl.asciidoc +++ /dev/null @@ -1,40 +0,0 @@ -[[java-query-dsl]] -== Query DSL - -Elasticsearch provides a full Java query dsl in a similar manner to the -REST {ref}/query-dsl.html[Query DSL]. The factory for query -builders is `QueryBuilders`. Once your query is ready, you can use the -<>. - -To use `QueryBuilders` just import them in your class: - -[source,java] --------------------------------------------------- -import static org.elasticsearch.index.query.QueryBuilders.*; --------------------------------------------------- - -Note that you can easily print (aka debug) JSON generated queries using -`toString()` method on `QueryBuilder` object. - -The `QueryBuilder` can then be used with any API that accepts a query, -such as `count` and `search`. - -:query-dsl-test: {docdir}/../../client/rest-high-level/src/test/java/org/elasticsearch/client/documentation/QueryDSLDocumentationTests.java - -include::query-dsl/match-all-query.asciidoc[] - -include::query-dsl/full-text-queries.asciidoc[] - -include::query-dsl/term-level-queries.asciidoc[] - -include::query-dsl/compound-queries.asciidoc[] - -include::query-dsl/joining-queries.asciidoc[] - -include::query-dsl/geo-queries.asciidoc[] - -include::query-dsl/special-queries.asciidoc[] - -include::query-dsl/span-queries.asciidoc[] - -:query-dsl-test!: diff --git a/docs/java-api/query-dsl/bool-query.asciidoc b/docs/java-api/query-dsl/bool-query.asciidoc deleted file mode 100644 index da9ca0ad0cc8c..0000000000000 --- a/docs/java-api/query-dsl/bool-query.asciidoc +++ /dev/null @@ -1,13 +0,0 @@ -[[java-query-dsl-bool-query]] -==== Bool Query - -See {ref}/query-dsl-bool-query.html[Bool Query] - -["source","java",subs="attributes,callouts,macros"] --------------------------------------------------- -include-tagged::{query-dsl-test}[bool] --------------------------------------------------- -<1> must query -<2> must not query -<3> should query -<4> a query that must appear in the matching documents but doesn't contribute to scoring. diff --git a/docs/java-api/query-dsl/boosting-query.asciidoc b/docs/java-api/query-dsl/boosting-query.asciidoc deleted file mode 100644 index 2a3c4437d1f89..0000000000000 --- a/docs/java-api/query-dsl/boosting-query.asciidoc +++ /dev/null @@ -1,12 +0,0 @@ -[[java-query-dsl-boosting-query]] -==== Boosting Query - -See {ref}/query-dsl-boosting-query.html[Boosting Query] - -["source","java",subs="attributes,callouts,macros"] --------------------------------------------------- -include-tagged::{query-dsl-test}[boosting] --------------------------------------------------- -<1> query that will promote documents -<2> query that will demote documents -<3> negative boost diff --git a/docs/java-api/query-dsl/common-terms-query.asciidoc b/docs/java-api/query-dsl/common-terms-query.asciidoc deleted file mode 100644 index 2c8dfc7a88cfe..0000000000000 --- a/docs/java-api/query-dsl/common-terms-query.asciidoc +++ /dev/null @@ -1,11 +0,0 @@ -[[java-query-dsl-common-terms-query]] -==== Common Terms Query - -See {ref}/query-dsl-common-terms-query.html[Common Terms Query] - -["source","java",subs="attributes,callouts,macros"] --------------------------------------------------- -include-tagged::{query-dsl-test}[common_terms] --------------------------------------------------- -<1> field -<2> value diff --git a/docs/java-api/query-dsl/compound-queries.asciidoc b/docs/java-api/query-dsl/compound-queries.asciidoc deleted file mode 100644 index b93e3b694a5ef..0000000000000 --- a/docs/java-api/query-dsl/compound-queries.asciidoc +++ /dev/null @@ -1,45 +0,0 @@ -[[java-compound-queries]] -=== Compound queries - -Compound queries wrap other compound or leaf queries, either to combine their -results and scores, to change their behaviour, or to switch from query to -filter context. - -The queries in this group are: - -<>:: - -A query which wraps another query, but executes it in filter context. All -matching documents are given the same ``constant'' `_score`. - -<>:: - -The default query for combining multiple leaf or compound query clauses, as -`must`, `should`, `must_not`, or `filter` clauses. The `must` and `should` -clauses have their scores combined -- the more matching clauses, the better -- -while the `must_not` and `filter` clauses are executed in filter context. - -<>:: - -A query which accepts multiple queries, and returns any documents which match -any of the query clauses. While the `bool` query combines the scores from all -matching queries, the `dis_max` query uses the score of the single best- -matching query clause. - -<>:: - -Modify the scores returned by the main query with functions to take into -account factors like popularity, recency, distance, or custom algorithms -implemented with scripting. - -<>:: - -Return documents which match a `positive` query, but reduce the score of -documents which also match a `negative` query. - - -include::constant-score-query.asciidoc[] -include::bool-query.asciidoc[] -include::dis-max-query.asciidoc[] -include::function-score-query.asciidoc[] -include::boosting-query.asciidoc[] diff --git a/docs/java-api/query-dsl/constant-score-query.asciidoc b/docs/java-api/query-dsl/constant-score-query.asciidoc deleted file mode 100644 index 49c5adbee6a73..0000000000000 --- a/docs/java-api/query-dsl/constant-score-query.asciidoc +++ /dev/null @@ -1,11 +0,0 @@ -[[java-query-dsl-constant-score-query]] -==== Constant Score Query - -See {ref}/query-dsl-constant-score-query.html[Constant Score Query] - -["source","java",subs="attributes,callouts,macros"] --------------------------------------------------- -include-tagged::{query-dsl-test}[constant_score] --------------------------------------------------- -<1> your query -<2> query score diff --git a/docs/java-api/query-dsl/dis-max-query.asciidoc b/docs/java-api/query-dsl/dis-max-query.asciidoc deleted file mode 100644 index 8c91bcb99011a..0000000000000 --- a/docs/java-api/query-dsl/dis-max-query.asciidoc +++ /dev/null @@ -1,13 +0,0 @@ -[[java-query-dsl-dis-max-query]] -==== Dis Max Query - -See {ref}/query-dsl-dis-max-query.html[Dis Max Query] - -["source","java",subs="attributes,callouts,macros"] --------------------------------------------------- -include-tagged::{query-dsl-test}[dis_max] --------------------------------------------------- -<1> add your queries -<2> add your queries -<3> boost factor -<4> tie breaker diff --git a/docs/java-api/query-dsl/exists-query.asciidoc b/docs/java-api/query-dsl/exists-query.asciidoc deleted file mode 100644 index 6fa5ba6a6f257..0000000000000 --- a/docs/java-api/query-dsl/exists-query.asciidoc +++ /dev/null @@ -1,10 +0,0 @@ -[[java-query-dsl-exists-query]] -==== Exists Query - -See {ref}/query-dsl-exists-query.html[Exists Query]. - -["source","java",subs="attributes,callouts,macros"] --------------------------------------------------- -include-tagged::{query-dsl-test}[exists] --------------------------------------------------- -<1> field diff --git a/docs/java-api/query-dsl/full-text-queries.asciidoc b/docs/java-api/query-dsl/full-text-queries.asciidoc deleted file mode 100644 index 27ce4bee1ba64..0000000000000 --- a/docs/java-api/query-dsl/full-text-queries.asciidoc +++ /dev/null @@ -1,44 +0,0 @@ -[[java-full-text-queries]] -=== Full text queries - -The high-level full text queries are usually used for running full text -queries on full text fields like the body of an email. They understand how the -field being queried is analyzed and will apply each field's -`analyzer` (or `search_analyzer`) to the query string before executing. - -The queries in this group are: - -<>:: - - The standard query for performing full text queries, including fuzzy matching - and phrase or proximity queries. - -<>:: - - The multi-field version of the `match` query. - -<>:: - - A more specialized query which gives more preference to uncommon words. - -<>:: - - Supports the compact Lucene query string syntax, - allowing you to specify AND|OR|NOT conditions and multi-field search - within a single query string. For expert users only. - -<>:: - - A simpler, more robust version of the `query_string` syntax suitable - for exposing directly to users. - -include::match-query.asciidoc[] - -include::multi-match-query.asciidoc[] - -include::common-terms-query.asciidoc[] - -include::query-string-query.asciidoc[] - -include::simple-query-string-query.asciidoc[] - diff --git a/docs/java-api/query-dsl/function-score-query.asciidoc b/docs/java-api/query-dsl/function-score-query.asciidoc deleted file mode 100644 index fcd5f2dc473f5..0000000000000 --- a/docs/java-api/query-dsl/function-score-query.asciidoc +++ /dev/null @@ -1,19 +0,0 @@ -[[java-query-dsl-function-score-query]] -==== Function Score Query - -See {ref}/query-dsl-function-score-query.html[Function Score Query]. - -To use `ScoreFunctionBuilders` just import them in your class: - -[source,java] --------------------------------------------------- -import static org.elasticsearch.index.query.functionscore.ScoreFunctionBuilders.*; --------------------------------------------------- - -["source","java",subs="attributes,callouts,macros"] --------------------------------------------------- -include-tagged::{query-dsl-test}[function_score] --------------------------------------------------- -<1> Add a first function based on a query -<2> And randomize the score based on a given seed -<3> Add another function based on the age field diff --git a/docs/java-api/query-dsl/fuzzy-query.asciidoc b/docs/java-api/query-dsl/fuzzy-query.asciidoc deleted file mode 100644 index 4a7bde82cdfb7..0000000000000 --- a/docs/java-api/query-dsl/fuzzy-query.asciidoc +++ /dev/null @@ -1,11 +0,0 @@ -[[java-query-dsl-fuzzy-query]] -==== Fuzzy Query - -See {ref}/query-dsl-fuzzy-query.html[Fuzzy Query] - -["source","java",subs="attributes,callouts,macros"] --------------------------------------------------- -include-tagged::{query-dsl-test}[fuzzy] --------------------------------------------------- -<1> field -<2> text diff --git a/docs/java-api/query-dsl/geo-bounding-box-query.asciidoc b/docs/java-api/query-dsl/geo-bounding-box-query.asciidoc deleted file mode 100644 index 4983a21213376..0000000000000 --- a/docs/java-api/query-dsl/geo-bounding-box-query.asciidoc +++ /dev/null @@ -1,12 +0,0 @@ -[[java-query-dsl-geo-bounding-box-query]] -==== Geo Bounding Box Query - -See {ref}/query-dsl-geo-bounding-box-query.html[Geo Bounding Box Query] - -["source","java",subs="attributes,callouts,macros"] --------------------------------------------------- -include-tagged::{query-dsl-test}[geo_bounding_box] --------------------------------------------------- -<1> field -<2> bounding box top left point -<3> bounding box bottom right point diff --git a/docs/java-api/query-dsl/geo-distance-query.asciidoc b/docs/java-api/query-dsl/geo-distance-query.asciidoc deleted file mode 100644 index cc8c89ca61eea..0000000000000 --- a/docs/java-api/query-dsl/geo-distance-query.asciidoc +++ /dev/null @@ -1,12 +0,0 @@ -[[java-query-dsl-geo-distance-query]] -==== Geo Distance Query - -See {ref}/query-dsl-geo-distance-query.html[Geo Distance Query] - -["source","java",subs="attributes,callouts,macros"] --------------------------------------------------- -include-tagged::{query-dsl-test}[geo_distance] --------------------------------------------------- -<1> field -<2> center point -<3> distance from center point diff --git a/docs/java-api/query-dsl/geo-polygon-query.asciidoc b/docs/java-api/query-dsl/geo-polygon-query.asciidoc deleted file mode 100644 index 7dbf49b8d1afd..0000000000000 --- a/docs/java-api/query-dsl/geo-polygon-query.asciidoc +++ /dev/null @@ -1,11 +0,0 @@ -[[java-query-dsl-geo-polygon-query]] -==== Geo Polygon Query - -See {ref}/query-dsl-geo-polygon-query.html[Geo Polygon Query] - -["source","java",subs="attributes,callouts,macros"] --------------------------------------------------- -include-tagged::{query-dsl-test}[geo_polygon] --------------------------------------------------- -<1> add your polygon of points a document should fall within -<2> initialise the query with field and points diff --git a/docs/java-api/query-dsl/geo-queries.asciidoc b/docs/java-api/query-dsl/geo-queries.asciidoc deleted file mode 100644 index 10df4ff5e8716..0000000000000 --- a/docs/java-api/query-dsl/geo-queries.asciidoc +++ /dev/null @@ -1,34 +0,0 @@ -[[java-geo-queries]] -=== Geo queries - -Elasticsearch supports two types of geo data: -`geo_point` fields which support lat/lon pairs, and -`geo_shape` fields, which support points, lines, circles, polygons, multi-polygons etc. - -The queries in this group are: - -<> query:: - - Find document with geo-shapes which either intersect, are contained by, or - do not intersect with the specified geo-shape. - -<> query:: - - Finds documents with geo-points that fall into the specified rectangle. - -<> query:: - - Finds document with geo-points within the specified distance of a central - point. - -<> query:: - - Find documents with geo-points within the specified polygon. - -include::geo-shape-query.asciidoc[] - -include::geo-bounding-box-query.asciidoc[] - -include::geo-distance-query.asciidoc[] - -include::geo-polygon-query.asciidoc[] diff --git a/docs/java-api/query-dsl/geo-shape-query.asciidoc b/docs/java-api/query-dsl/geo-shape-query.asciidoc deleted file mode 100644 index c2cd4c14e3adc..0000000000000 --- a/docs/java-api/query-dsl/geo-shape-query.asciidoc +++ /dev/null @@ -1,56 +0,0 @@ -[[java-query-dsl-geo-shape-query]] -==== GeoShape Query - -See {ref}/query-dsl-geo-shape-query.html[Geo Shape Query] - -Note: the `geo_shape` type uses `Spatial4J` and `JTS`, both of which are -optional dependencies. Consequently you must add `Spatial4J` and `JTS` -to your classpath in order to use this type: - -[source,xml] ------------------------------------------------ - - org.locationtech.spatial4j - spatial4j - 0.7 <1> - - - - org.locationtech.jts - jts-core - 1.15.0 <2> - - - xerces - xercesImpl - - - ------------------------------------------------ -<1> check for updates in http://search.maven.org/#search%7Cga%7C1%7Cg%3A%22org.locationtech.spatial4j%22%20AND%20a%3A%22spatial4j%22[Maven Central] -<2> check for updates in http://search.maven.org/#search%7Cga%7C1%7Cg%3A%22org.locationtech.jts%22%20AND%20a%3A%22jts-core%22[Maven Central] - -[source,java] --------------------------------------------------- -// Import ShapeRelation and ShapeBuilder -import org.elasticsearch.common.geo.ShapeRelation; -import org.elasticsearch.common.geo.builders.ShapeBuilder; --------------------------------------------------- - -["source","java",subs="attributes,callouts,macros"] --------------------------------------------------- -include-tagged::{query-dsl-test}[geo_shape] --------------------------------------------------- -<1> field -<2> shape -<3> relation can be `ShapeRelation.CONTAINS`, `ShapeRelation.WITHIN`, `ShapeRelation.INTERSECTS` or `ShapeRelation.DISJOINT` - -["source","java",subs="attributes,callouts,macros"] --------------------------------------------------- -include-tagged::{query-dsl-test}[indexed_geo_shape] --------------------------------------------------- -<1> field -<2> The ID of the document that containing the pre-indexed shape. -<3> relation -<4> Name of the index where the pre-indexed shape is. Defaults to 'shapes'. -<5> The field specified as path containing the pre-indexed shape. Defaults to 'shape'. diff --git a/docs/java-api/query-dsl/has-child-query.asciidoc b/docs/java-api/query-dsl/has-child-query.asciidoc deleted file mode 100644 index f47f3af487dfe..0000000000000 --- a/docs/java-api/query-dsl/has-child-query.asciidoc +++ /dev/null @@ -1,23 +0,0 @@ -[[java-query-dsl-has-child-query]] -==== Has Child Query - -See {ref}/query-dsl-has-child-query.html[Has Child Query] - -When using the `has_child` query it is important to use the `PreBuiltTransportClient` instead of the regular client: - -[source,java] --------------------------------------------------- -Settings settings = Settings.builder().put("cluster.name", "elasticsearch").build(); -TransportClient client = new PreBuiltTransportClient(settings); -client.addTransportAddress(new TransportAddress(new InetSocketAddress(InetAddresses.forString("127.0.0.1"), 9300))); --------------------------------------------------- - -Otherwise the parent-join module doesn't get loaded and the `has_child` query can't be used from the transport client. - -["source","java",subs="attributes,callouts,macros"] --------------------------------------------------- -include-tagged::{query-dsl-test}[has_child] --------------------------------------------------- -<1> child type to query against -<2> query -<3> score mode can be `ScoreMode.Avg`, `ScoreMode.Max`, `ScoreMode.Min`, `ScoreMode.None` or `ScoreMode.Total` diff --git a/docs/java-api/query-dsl/has-parent-query.asciidoc b/docs/java-api/query-dsl/has-parent-query.asciidoc deleted file mode 100644 index 6a83fe2b0698f..0000000000000 --- a/docs/java-api/query-dsl/has-parent-query.asciidoc +++ /dev/null @@ -1,23 +0,0 @@ -[[java-query-dsl-has-parent-query]] -==== Has Parent Query - -See {ref}/query-dsl-has-parent-query.html[Has Parent] - -When using the `has_parent` query it is important to use the `PreBuiltTransportClient` instead of the regular client: - -[source,java] --------------------------------------------------- -Settings settings = Settings.builder().put("cluster.name", "elasticsearch").build(); -TransportClient client = new PreBuiltTransportClient(settings); -client.addTransportAddress(new TransportAddress(new InetSocketAddress(InetAddresses.forString("127.0.0.1"), 9300))); --------------------------------------------------- - -Otherwise the parent-join module doesn't get loaded and the `has_parent` query can't be used from the transport client. - -["source","java",subs="attributes,callouts,macros"] --------------------------------------------------- -include-tagged::{query-dsl-test}[has_parent] --------------------------------------------------- -<1> parent type to query against -<2> query -<3> whether the score from the parent hit should propagate to the child hit diff --git a/docs/java-api/query-dsl/ids-query.asciidoc b/docs/java-api/query-dsl/ids-query.asciidoc deleted file mode 100644 index ba12a5df38b0e..0000000000000 --- a/docs/java-api/query-dsl/ids-query.asciidoc +++ /dev/null @@ -1,10 +0,0 @@ -[[java-query-dsl-ids-query]] -==== Ids Query - - -See {ref}/query-dsl-ids-query.html[Ids Query] - -["source","java",subs="attributes,callouts,macros"] --------------------------------------------------- -include-tagged::{query-dsl-test}[ids] --------------------------------------------------- diff --git a/docs/java-api/query-dsl/joining-queries.asciidoc b/docs/java-api/query-dsl/joining-queries.asciidoc deleted file mode 100644 index fcefef5f6245b..0000000000000 --- a/docs/java-api/query-dsl/joining-queries.asciidoc +++ /dev/null @@ -1,28 +0,0 @@ -[[java-joining-queries]] -=== Joining queries - -Performing full SQL-style joins in a distributed system like Elasticsearch is -prohibitively expensive. Instead, Elasticsearch offers two forms of join -which are designed to scale horizontally. - -<>:: - -Documents may contains fields of type `nested`. These -fields are used to index arrays of objects, where each object can be queried -(with the `nested` query) as an independent document. - -<> and <> queries:: - -A parent-child relationship can exist between two -document types within a single index. The `has_child` query returns parent -documents whose child documents match the specified query, while the -`has_parent` query returns child documents whose parent document matches the -specified query. - -include::nested-query.asciidoc[] - -include::has-child-query.asciidoc[] - -include::has-parent-query.asciidoc[] - - diff --git a/docs/java-api/query-dsl/match-all-query.asciidoc b/docs/java-api/query-dsl/match-all-query.asciidoc deleted file mode 100644 index 85d847528f5b8..0000000000000 --- a/docs/java-api/query-dsl/match-all-query.asciidoc +++ /dev/null @@ -1,9 +0,0 @@ -[[java-query-dsl-match-all-query]] -=== Match All Query - -See {ref}/query-dsl-match-all-query.html[Match All Query] - -["source","java",subs="attributes,callouts,macros"] --------------------------------------------------- -include-tagged::{query-dsl-test}[match_all] --------------------------------------------------- diff --git a/docs/java-api/query-dsl/match-query.asciidoc b/docs/java-api/query-dsl/match-query.asciidoc deleted file mode 100644 index 6884deb5f1f24..0000000000000 --- a/docs/java-api/query-dsl/match-query.asciidoc +++ /dev/null @@ -1,11 +0,0 @@ -[[java-query-dsl-match-query]] -==== Match Query - -See {ref}/query-dsl-match-query.html[Match Query] - -["source","java",subs="attributes,callouts,macros"] --------------------------------------------------- -include-tagged::{query-dsl-test}[match] --------------------------------------------------- -<1> field -<2> text diff --git a/docs/java-api/query-dsl/mlt-query.asciidoc b/docs/java-api/query-dsl/mlt-query.asciidoc deleted file mode 100644 index 11e5c7ef40482..0000000000000 --- a/docs/java-api/query-dsl/mlt-query.asciidoc +++ /dev/null @@ -1,13 +0,0 @@ -[[java-query-dsl-mlt-query]] -==== More Like This Query - -See {ref}/query-dsl-mlt-query.html[More Like This Query] - -["source","java",subs="attributes,callouts,macros"] --------------------------------------------------- -include-tagged::{query-dsl-test}[more_like_this] --------------------------------------------------- -<1> fields -<2> text -<3> ignore threshold -<4> max num of Terms in generated queries diff --git a/docs/java-api/query-dsl/multi-match-query.asciidoc b/docs/java-api/query-dsl/multi-match-query.asciidoc deleted file mode 100644 index 86b384d44d3c0..0000000000000 --- a/docs/java-api/query-dsl/multi-match-query.asciidoc +++ /dev/null @@ -1,11 +0,0 @@ -[[java-query-dsl-multi-match-query]] -==== Multi Match Query - -See {ref}/query-dsl-multi-match-query.html[Multi Match Query] - -["source","java",subs="attributes,callouts,macros"] --------------------------------------------------- -include-tagged::{query-dsl-test}[multi_match] --------------------------------------------------- -<1> text -<2> fields diff --git a/docs/java-api/query-dsl/nested-query.asciidoc b/docs/java-api/query-dsl/nested-query.asciidoc deleted file mode 100644 index 9b675ea72acfd..0000000000000 --- a/docs/java-api/query-dsl/nested-query.asciidoc +++ /dev/null @@ -1,12 +0,0 @@ -[[java-query-dsl-nested-query]] -==== Nested Query - -See {ref}/query-dsl-nested-query.html[Nested Query] - -["source","java",subs="attributes,callouts,macros"] --------------------------------------------------- -include-tagged::{query-dsl-test}[nested] --------------------------------------------------- -<1> path to nested document -<2> your query. Any fields referenced inside the query must use the complete path (fully qualified). -<3> score mode could be `ScoreMode.Max`, `ScoreMode.Min`, `ScoreMode.Total`, `ScoreMode.Avg` or `ScoreMode.None` diff --git a/docs/java-api/query-dsl/percolate-query.asciidoc b/docs/java-api/query-dsl/percolate-query.asciidoc deleted file mode 100644 index 18cdd4a14e5a9..0000000000000 --- a/docs/java-api/query-dsl/percolate-query.asciidoc +++ /dev/null @@ -1,61 +0,0 @@ -[[java-query-percolate-query]] -==== Percolate Query - -See: - * {ref}/query-dsl-percolate-query.html[Percolate Query] - - -[source,java] --------------------------------------------------- -Settings settings = Settings.builder().put("cluster.name", "elasticsearch").build(); -TransportClient client = new PreBuiltTransportClient(settings); -client.addTransportAddress(new TransportAddress(new InetSocketAddress(InetAddresses.forString("127.0.0.1"), 9300))); --------------------------------------------------- - -Before the `percolate` query can be used an `percolator` mapping should be added and -a document containing a percolator query should be indexed: - -[source,java] --------------------------------------------------- -// create an index with a percolator field with the name 'query': -client.admin().indices().prepareCreate("myIndexName") - .addMapping("_doc", "query", "type=percolator", "content", "type=text") - .get(); - -//This is the query we're registering in the percolator -QueryBuilder qb = termQuery("content", "amazing"); - -//Index the query = register it in the percolator -client.prepareIndex("myIndexName", "_doc", "myDesignatedQueryName") - .setSource(jsonBuilder() - .startObject() - .field("query", qb) // Register the query - .endObject()) - .setRefreshPolicy(RefreshPolicy.IMMEDIATE) // Needed when the query shall be available immediately - .get(); --------------------------------------------------- - -This indexes the above term query under the name -*myDesignatedQueryName*. - -In order to check a document against the registered queries, use this -code: - -[source,java] --------------------------------------------------- -//Build a document to check against the percolator -XContentBuilder docBuilder = XContentFactory.jsonBuilder().startObject(); -docBuilder.field("content", "This is amazing!"); -docBuilder.endObject(); //End of the JSON root object - -PercolateQueryBuilder percolateQuery = new PercolateQueryBuilder("query", "_doc", BytesReference.bytes(docBuilder)); - -// Percolate, by executing the percolator query in the query dsl: -SearchResponse response = client().prepareSearch("myIndexName") - .setQuery(percolateQuery)) - .get(); -//Iterate over the results -for(SearchHit hit : response.getHits()) { - // Percolator queries as hit -} --------------------------------------------------- diff --git a/docs/java-api/query-dsl/prefix-query.asciidoc b/docs/java-api/query-dsl/prefix-query.asciidoc deleted file mode 100644 index eb15c4426f633..0000000000000 --- a/docs/java-api/query-dsl/prefix-query.asciidoc +++ /dev/null @@ -1,11 +0,0 @@ -[[java-query-dsl-prefix-query]] -==== Prefix Query - -See {ref}/query-dsl-prefix-query.html[Prefix Query] - -["source","java",subs="attributes,callouts,macros"] --------------------------------------------------- -include-tagged::{query-dsl-test}[prefix] --------------------------------------------------- -<1> field -<2> prefix diff --git a/docs/java-api/query-dsl/query-string-query.asciidoc b/docs/java-api/query-dsl/query-string-query.asciidoc deleted file mode 100644 index 7d8bead2e340a..0000000000000 --- a/docs/java-api/query-dsl/query-string-query.asciidoc +++ /dev/null @@ -1,9 +0,0 @@ -[[java-query-dsl-query-string-query]] -==== Query String Query - -See {ref}/query-dsl-query-string-query.html[Query String Query] - -["source","java",subs="attributes,callouts,macros"] --------------------------------------------------- -include-tagged::{query-dsl-test}[query_string] --------------------------------------------------- diff --git a/docs/java-api/query-dsl/range-query.asciidoc b/docs/java-api/query-dsl/range-query.asciidoc deleted file mode 100644 index 2d58fbd3a34ef..0000000000000 --- a/docs/java-api/query-dsl/range-query.asciidoc +++ /dev/null @@ -1,22 +0,0 @@ -[[java-query-dsl-range-query]] -==== Range Query - -See {ref}/query-dsl-range-query.html[Range Query] - -["source","java",subs="attributes,callouts,macros"] --------------------------------------------------- -include-tagged::{query-dsl-test}[range] --------------------------------------------------- -<1> field -<2> from -<3> to -<4> include lower value means that `from` is `gt` when `false` or `gte` when `true` -<5> include upper value means that `to` is `lt` when `false` or `lte` when `true` - -["source","java",subs="attributes,callouts,macros"] --------------------------------------------------- -include-tagged::{query-dsl-test}[range_simplified] --------------------------------------------------- -<1> field -<2> set `from` to 10 and `includeLower` to `true` -<3> set `to` to 20 and `includeUpper` to `false` diff --git a/docs/java-api/query-dsl/regexp-query.asciidoc b/docs/java-api/query-dsl/regexp-query.asciidoc deleted file mode 100644 index f9cd8cd72d9d5..0000000000000 --- a/docs/java-api/query-dsl/regexp-query.asciidoc +++ /dev/null @@ -1,11 +0,0 @@ -[[java-query-dsl-regexp-query]] -==== Regexp Query - -See {ref}/query-dsl-regexp-query.html[Regexp Query] - -["source","java",subs="attributes,callouts,macros"] --------------------------------------------------- -include-tagged::{query-dsl-test}[regexp] --------------------------------------------------- -<1> field -<2> regexp diff --git a/docs/java-api/query-dsl/script-query.asciidoc b/docs/java-api/query-dsl/script-query.asciidoc deleted file mode 100644 index a8c60f1d8eb0d..0000000000000 --- a/docs/java-api/query-dsl/script-query.asciidoc +++ /dev/null @@ -1,29 +0,0 @@ -[[java-query-dsl-script-query]] -==== Script Query - -See {ref}/query-dsl-script-query.html[Script Query] - -["source","java",subs="attributes,callouts,macros"] --------------------------------------------------- -include-tagged::{query-dsl-test}[script_inline] --------------------------------------------------- -<1> inlined script - - -If you have stored on each data node a script named `myscript.painless` with: - -[source,painless] --------------------------------------------------- -doc['num1'].value > params.param1 --------------------------------------------------- - -You can use it then with: - -["source","java",subs="attributes,callouts,macros"] --------------------------------------------------- -include-tagged::{query-dsl-test}[script_file] --------------------------------------------------- -<1> Script type: either `ScriptType.FILE`, `ScriptType.INLINE` or `ScriptType.INDEXED` -<2> Scripting engine -<3> Script name -<4> Parameters as a `Map` diff --git a/docs/java-api/query-dsl/simple-query-string-query.asciidoc b/docs/java-api/query-dsl/simple-query-string-query.asciidoc deleted file mode 100644 index c3b32ecd1cbb2..0000000000000 --- a/docs/java-api/query-dsl/simple-query-string-query.asciidoc +++ /dev/null @@ -1,9 +0,0 @@ -[[java-query-dsl-simple-query-string-query]] -==== Simple Query String Query - -See {ref}/query-dsl-simple-query-string-query.html[Simple Query String Query] - -["source","java",subs="attributes,callouts,macros"] --------------------------------------------------- -include-tagged::{query-dsl-test}[simple_query_string] --------------------------------------------------- diff --git a/docs/java-api/query-dsl/span-containing-query.asciidoc b/docs/java-api/query-dsl/span-containing-query.asciidoc deleted file mode 100644 index 173e26952c265..0000000000000 --- a/docs/java-api/query-dsl/span-containing-query.asciidoc +++ /dev/null @@ -1,11 +0,0 @@ -[[java-query-dsl-span-containing-query]] -==== Span Containing Query - -See {ref}/query-dsl-span-containing-query.html[Span Containing Query] - -["source","java",subs="attributes,callouts,macros"] --------------------------------------------------- -include-tagged::{query-dsl-test}[span_containing] --------------------------------------------------- -<1> `big` part -<2> `little` part diff --git a/docs/java-api/query-dsl/span-first-query.asciidoc b/docs/java-api/query-dsl/span-first-query.asciidoc deleted file mode 100644 index d02c164754c53..0000000000000 --- a/docs/java-api/query-dsl/span-first-query.asciidoc +++ /dev/null @@ -1,11 +0,0 @@ -[[java-query-dsl-span-first-query]] -==== Span First Query - -See {ref}/query-dsl-span-first-query.html[Span First Query] - -["source","java",subs="attributes,callouts,macros"] --------------------------------------------------- -include-tagged::{query-dsl-test}[span_first] --------------------------------------------------- -<1> query -<2> max end position diff --git a/docs/java-api/query-dsl/span-multi-term-query.asciidoc b/docs/java-api/query-dsl/span-multi-term-query.asciidoc deleted file mode 100644 index eea00f61fe7e1..0000000000000 --- a/docs/java-api/query-dsl/span-multi-term-query.asciidoc +++ /dev/null @@ -1,11 +0,0 @@ -[[java-query-dsl-span-multi-term-query]] -==== Span Multi Term Query - -See {ref}/query-dsl-span-multi-term-query.html[Span Multi Term Query] - -["source","java",subs="attributes,callouts,macros"] --------------------------------------------------- -include-tagged::{query-dsl-test}[span_multi] --------------------------------------------------- -<1> Can be any builder extending the `MultiTermQueryBuilder` class. For example: `FuzzyQueryBuilder`, -`PrefixQueryBuilder`, `RangeQueryBuilder`, `RegexpQueryBuilder` or `WildcardQueryBuilder`. diff --git a/docs/java-api/query-dsl/span-near-query.asciidoc b/docs/java-api/query-dsl/span-near-query.asciidoc deleted file mode 100644 index 6f4661e34c9d1..0000000000000 --- a/docs/java-api/query-dsl/span-near-query.asciidoc +++ /dev/null @@ -1,12 +0,0 @@ -[[java-query-dsl-span-near-query]] -==== Span Near Query - -See {ref}/query-dsl-span-near-query.html[Span Near Query] - -["source","java",subs="attributes,callouts,macros"] --------------------------------------------------- -include-tagged::{query-dsl-test}[span_near] --------------------------------------------------- -<1> span term queries -<2> slop factor: the maximum number of intervening unmatched positions -<3> whether matches are required to be in-order diff --git a/docs/java-api/query-dsl/span-not-query.asciidoc b/docs/java-api/query-dsl/span-not-query.asciidoc deleted file mode 100644 index 001c2ca025e6d..0000000000000 --- a/docs/java-api/query-dsl/span-not-query.asciidoc +++ /dev/null @@ -1,11 +0,0 @@ -[[java-query-dsl-span-not-query]] -==== Span Not Query - -See {ref}/query-dsl-span-not-query.html[Span Not Query] - -["source","java",subs="attributes,callouts,macros"] --------------------------------------------------- -include-tagged::{query-dsl-test}[span_not] --------------------------------------------------- -<1> span query whose matches are filtered -<2> span query whose matches must not overlap those returned diff --git a/docs/java-api/query-dsl/span-or-query.asciidoc b/docs/java-api/query-dsl/span-or-query.asciidoc deleted file mode 100644 index 787628b59342f..0000000000000 --- a/docs/java-api/query-dsl/span-or-query.asciidoc +++ /dev/null @@ -1,10 +0,0 @@ -[[java-query-dsl-span-or-query]] -==== Span Or Query - -See {ref}/query-dsl-span-or-query.html[Span Or Query] - -["source","java",subs="attributes,callouts,macros"] --------------------------------------------------- -include-tagged::{query-dsl-test}[span_or] --------------------------------------------------- -<1> span term queries diff --git a/docs/java-api/query-dsl/span-queries.asciidoc b/docs/java-api/query-dsl/span-queries.asciidoc deleted file mode 100644 index 0ccbe30638c6a..0000000000000 --- a/docs/java-api/query-dsl/span-queries.asciidoc +++ /dev/null @@ -1,65 +0,0 @@ -[[java-span-queries]] -=== Span queries - -Span queries are low-level positional queries which provide expert control -over the order and proximity of the specified terms. These are typically used -to implement very specific queries on legal documents or patents. - -Span queries cannot be mixed with non-span queries (with the exception of the `span_multi` query). - -The queries in this group are: - -<>:: - -The equivalent of the <> but for use with -other span queries. - -<>:: - -Wraps a <>, <>, -<>, <>, -<>, or <> query. - -<>:: - -Accepts another span query whose matches must appear within the first N -positions of the field. - -<>:: - -Accepts multiple span queries whose matches must be within the specified distance of each other, and possibly in the same order. - -<>:: - -Combines multiple span queries -- returns documents which match any of the -specified queries. - -<>:: - -Wraps another span query, and excludes any documents which match that query. - -<>:: - -Accepts a list of span queries, but only returns those spans which also match a second span query. - -<>:: - -The result from a single span query is returned as long is its span falls -within the spans returned by a list of other span queries. - - -include::span-term-query.asciidoc[] - -include::span-multi-term-query.asciidoc[] - -include::span-first-query.asciidoc[] - -include::span-near-query.asciidoc[] - -include::span-or-query.asciidoc[] - -include::span-not-query.asciidoc[] - -include::span-containing-query.asciidoc[] - -include::span-within-query.asciidoc[] diff --git a/docs/java-api/query-dsl/span-term-query.asciidoc b/docs/java-api/query-dsl/span-term-query.asciidoc deleted file mode 100644 index 2bdf9276515dc..0000000000000 --- a/docs/java-api/query-dsl/span-term-query.asciidoc +++ /dev/null @@ -1,11 +0,0 @@ -[[java-query-dsl-span-term-query]] -==== Span Term Query - -See {ref}/query-dsl-span-term-query.html[Span Term Query] - -["source","java",subs="attributes,callouts,macros"] --------------------------------------------------- -include-tagged::{query-dsl-test}[span_term] --------------------------------------------------- -<1> field -<2> value diff --git a/docs/java-api/query-dsl/span-within-query.asciidoc b/docs/java-api/query-dsl/span-within-query.asciidoc deleted file mode 100644 index afa527c0b67fb..0000000000000 --- a/docs/java-api/query-dsl/span-within-query.asciidoc +++ /dev/null @@ -1,11 +0,0 @@ -[[java-query-dsl-span-within-query]] -==== Span Within Query - -See {ref}/query-dsl-span-within-query.html[Span Within Query] - -["source","java",subs="attributes,callouts,macros"] --------------------------------------------------- -include-tagged::{query-dsl-test}[span_within] --------------------------------------------------- -<1> `big` part -<2> `little` part diff --git a/docs/java-api/query-dsl/special-queries.asciidoc b/docs/java-api/query-dsl/special-queries.asciidoc deleted file mode 100644 index bca3bde3b3f62..0000000000000 --- a/docs/java-api/query-dsl/special-queries.asciidoc +++ /dev/null @@ -1,31 +0,0 @@ -[[java-specialized-queries]] - -=== Specialized queries - -This group contains queries which do not fit into the other groups: - -<>:: - -This query finds documents which are similar to the specified text, document, -or collection of documents. - -<>:: - -This query allows a script to act as a filter. Also see the -<>. - -<>:: - -This query finds percolator queries based on documents. - -<>:: - -A query that accepts other queries as json or yaml string. - -include::mlt-query.asciidoc[] - -include::script-query.asciidoc[] - -include::percolate-query.asciidoc[] - -include::wrapper-query.asciidoc[] diff --git a/docs/java-api/query-dsl/term-level-queries.asciidoc b/docs/java-api/query-dsl/term-level-queries.asciidoc deleted file mode 100644 index 7d3649e372bbd..0000000000000 --- a/docs/java-api/query-dsl/term-level-queries.asciidoc +++ /dev/null @@ -1,77 +0,0 @@ -[[java-term-level-queries]] -=== Term level queries - -While the <> will analyze the query -string before executing, the _term-level queries_ operate on the exact terms -that are stored in the inverted index. - -These queries are usually used for structured data like numbers, dates, and -enums, rather than full text fields. Alternatively, they allow you to craft -low-level queries, foregoing the analysis process. - -The queries in this group are: - -<>:: - - Find documents which contain the exact term specified in the field - specified. - -<>:: - - Find documents which contain any of the exact terms specified in the field - specified. - -<>:: - - Find documents where the field specified contains values (dates, numbers, - or strings) in the range specified. - -<>:: - - Find documents where the field specified contains any non-null value. - -<>:: - - Find documents where the field specified contains terms which being with - the exact prefix specified. - -<>:: - - Find documents where the field specified contains terms which match the - pattern specified, where the pattern supports single character wildcards - (`?`) and multi-character wildcards (`*`) - -<>:: - - Find documents where the field specified contains terms which match the - regular expression specified. - -<>:: - - Find documents where the field specified contains terms which are fuzzily - similar to the specified term. Fuzziness is measured as a - http://en.wikipedia.org/wiki/Damerau%E2%80%93Levenshtein_distance[Levenshtein edit distance] - of 1 or 2. - -<>:: - - Find documents with the specified type and IDs. - - -include::term-query.asciidoc[] - -include::terms-query.asciidoc[] - -include::range-query.asciidoc[] - -include::exists-query.asciidoc[] - -include::prefix-query.asciidoc[] - -include::wildcard-query.asciidoc[] - -include::regexp-query.asciidoc[] - -include::fuzzy-query.asciidoc[] - -include::ids-query.asciidoc[] diff --git a/docs/java-api/query-dsl/term-query.asciidoc b/docs/java-api/query-dsl/term-query.asciidoc deleted file mode 100644 index 7c8549dbed403..0000000000000 --- a/docs/java-api/query-dsl/term-query.asciidoc +++ /dev/null @@ -1,11 +0,0 @@ -[[java-query-dsl-term-query]] -==== Term Query - -See {ref}/query-dsl-term-query.html[Term Query] - -["source","java",subs="attributes,callouts,macros"] --------------------------------------------------- -include-tagged::{query-dsl-test}[term] --------------------------------------------------- -<1> field -<2> text diff --git a/docs/java-api/query-dsl/terms-query.asciidoc b/docs/java-api/query-dsl/terms-query.asciidoc deleted file mode 100644 index 587968ba18e77..0000000000000 --- a/docs/java-api/query-dsl/terms-query.asciidoc +++ /dev/null @@ -1,11 +0,0 @@ -[[java-query-dsl-terms-query]] -==== Terms Query - -See {ref}/query-dsl-terms-query.html[Terms Query] - -["source","java",subs="attributes,callouts,macros"] --------------------------------------------------- -include-tagged::{query-dsl-test}[terms] --------------------------------------------------- -<1> field -<2> values diff --git a/docs/java-api/query-dsl/wildcard-query.asciidoc b/docs/java-api/query-dsl/wildcard-query.asciidoc deleted file mode 100644 index f9ace822aac9d..0000000000000 --- a/docs/java-api/query-dsl/wildcard-query.asciidoc +++ /dev/null @@ -1,11 +0,0 @@ -[[java-query-dsl-wildcard-query]] -==== Wildcard Query - -See {ref}/query-dsl-wildcard-query.html[Wildcard Query] - -["source","java",subs="attributes,callouts,macros"] --------------------------------------------------- -include-tagged::{query-dsl-test}[wildcard] --------------------------------------------------- -<1> field -<2> wildcard expression diff --git a/docs/java-api/query-dsl/wrapper-query.asciidoc b/docs/java-api/query-dsl/wrapper-query.asciidoc deleted file mode 100644 index 3bdf3cc69d30a..0000000000000 --- a/docs/java-api/query-dsl/wrapper-query.asciidoc +++ /dev/null @@ -1,11 +0,0 @@ -[[java-query-dsl-wrapper-query]] -==== Wrapper Query - -See {ref}/query-dsl-wrapper-query.html[Wrapper Query] - -["source","java",subs="attributes,callouts,macros"] --------------------------------------------------- -include-tagged::{query-dsl-test}[wrapper] --------------------------------------------------- - -<1> query defined as query builder diff --git a/docs/java-api/search.asciidoc b/docs/java-api/search.asciidoc deleted file mode 100644 index ecf8415f4dcbe..0000000000000 --- a/docs/java-api/search.asciidoc +++ /dev/null @@ -1,250 +0,0 @@ -[[java-search]] -== Search API - -The search API allows one to execute a search query and get back search hits -that match the query. It can be executed across one or more indices and -across one or more types. The query can be provided using the <>. -The body of the search request is built using the `SearchSourceBuilder`. Here is an example: - -[source,java] --------------------------------------------------- -import org.elasticsearch.action.search.SearchResponse; -import org.elasticsearch.action.search.SearchType; -import org.elasticsearch.index.query.QueryBuilders.*; --------------------------------------------------- - -[source,java] --------------------------------------------------- -SearchResponse response = client.prepareSearch("index1", "index2") - .setSearchType(SearchType.DFS_QUERY_THEN_FETCH) - .setQuery(QueryBuilders.termQuery("multi", "test")) // Query - .setPostFilter(QueryBuilders.rangeQuery("age").from(12).to(18)) // Filter - .setFrom(0).setSize(60).setExplain(true) - .get(); --------------------------------------------------- - -Note that all parameters are optional. Here is the smallest search call -you can write: - -[source,java] --------------------------------------------------- -// MatchAll on the whole cluster with all default options -SearchResponse response = client.prepareSearch().get(); --------------------------------------------------- - -NOTE: Although the Java API defines the additional search types QUERY_AND_FETCH and - DFS_QUERY_AND_FETCH, these modes are internal optimizations and should not - be specified explicitly by users of the API. - -For more information on the search operation, check out the REST -{ref}/search.html[search] docs. - - -[[java-search-scrolling]] -=== Using scrolls in Java - -Read the {ref}/search-request-scroll.html[scroll documentation] -first! - -[source,java] --------------------------------------------------- -import static org.elasticsearch.index.query.QueryBuilders.*; - -QueryBuilder qb = termQuery("multi", "test"); - -SearchResponse scrollResp = client.prepareSearch(test) - .addSort(FieldSortBuilder.DOC_FIELD_NAME, SortOrder.ASC) - .setScroll(new TimeValue(60000)) - .setQuery(qb) - .setSize(100).get(); //max of 100 hits will be returned for each scroll -//Scroll until no hits are returned -do { - for (SearchHit hit : scrollResp.getHits().getHits()) { - //Handle the hit... - } - - scrollResp = client.prepareSearchScroll(scrollResp.getScrollId()).setScroll(new TimeValue(60000)).execute().actionGet(); -} while(scrollResp.getHits().getHits().length != 0); // Zero hits mark the end of the scroll and the while loop. --------------------------------------------------- - -[[java-search-msearch]] -=== MultiSearch API - -See {ref}/search-multi-search.html[MultiSearch API Query] -documentation - -[source,java] --------------------------------------------------- -SearchRequestBuilder srb1 = client - .prepareSearch().setQuery(QueryBuilders.queryStringQuery("elasticsearch")).setSize(1); -SearchRequestBuilder srb2 = client - .prepareSearch().setQuery(QueryBuilders.matchQuery("name", "kimchy")).setSize(1); - -MultiSearchResponse sr = client.prepareMultiSearch() - .add(srb1) - .add(srb2) - .get(); - -// You will get all individual responses from MultiSearchResponse#getResponses() -long nbHits = 0; -for (MultiSearchResponse.Item item : sr.getResponses()) { - SearchResponse response = item.getResponse(); - nbHits += response.getHits().getTotalHits().value; -} --------------------------------------------------- - - -[[java-search-aggs]] -=== Using Aggregations - -The following code shows how to add two aggregations within your search: - -[source,java] --------------------------------------------------- -SearchResponse sr = client.prepareSearch() - .setQuery(QueryBuilders.matchAllQuery()) - .addAggregation( - AggregationBuilders.terms("agg1").field("field") - ) - .addAggregation( - AggregationBuilders.dateHistogram("agg2") - .field("birth") - .calendarInterval(DateHistogramInterval.YEAR) - ) - .get(); - -// Get your facet results -Terms agg1 = sr.getAggregations().get("agg1"); -Histogram agg2 = sr.getAggregations().get("agg2"); --------------------------------------------------- - -See <> -documentation for details. - - -[[java-search-terminate-after]] -=== Terminate After - -The maximum number of documents to collect for each shard, upon reaching which the query execution will terminate early. -If set, you will be able to check if the operation terminated early by asking for `isTerminatedEarly()` in the -`SearchResponse` object: - -[source,java] --------------------------------------------------- -SearchResponse sr = client.prepareSearch(INDEX) - .setTerminateAfter(1000) <1> - .get(); - -if (sr.isTerminatedEarly()) { - // We finished early -} --------------------------------------------------- -<1> Finish after 1000 docs - -[[java-search-template]] -=== Search Template - -See {ref}/search-template.html[Search Template] documentation - -Define your template parameters as a `Map`: - -[source,java] --------------------------------------------------- -Map template_params = new HashMap<>(); -template_params.put("param_gender", "male"); --------------------------------------------------- - -You can use your stored search templates in `config/scripts`. -For example, if you have a file named `config/scripts/template_gender.mustache` containing: - -[source,js] --------------------------------------------------- -{ - "query" : { - "match" : { - "gender" : "{{param_gender}}" - } - } -} --------------------------------------------------- -// NOTCONSOLE - -Create your search template request: - -[source,java] --------------------------------------------------- -SearchResponse sr = new SearchTemplateRequestBuilder(client) - .setScript("template_gender") <1> - .setScriptType(ScriptService.ScriptType.FILE) <2> - .setScriptParams(template_params) <3> - .setRequest(new SearchRequest()) <4> - .get() <5> - .getResponse(); <6> --------------------------------------------------- -<1> template name -<2> template stored on disk in `gender_template.mustache` -<3> parameters -<4> set the execution context (ie. define the index name here) -<5> execute and get the template response -<6> get from the template response the search response itself - -You can also store your template in the cluster state: - -[source,java] --------------------------------------------------- -client.admin().cluster().preparePutStoredScript() - .setScriptLang("mustache") - .setId("template_gender") - .setSource(new BytesArray( - "{\n" + - " \"query\" : {\n" + - " \"match\" : {\n" + - " \"gender\" : \"{{param_gender}}\"\n" + - " }\n" + - " }\n" + - "}")).get(); --------------------------------------------------- - -To execute a stored templates, use `ScriptService.ScriptType.STORED`: - -[source,java] --------------------------------------------------- -SearchResponse sr = new SearchTemplateRequestBuilder(client) - .setScript("template_gender") <1> - .setScriptType(ScriptType.STORED) <2> - .setScriptParams(template_params) <3> - .setRequest(new SearchRequest()) <4> - .get() <5> - .getResponse(); <6> --------------------------------------------------- -<1> template name -<2> template stored in the cluster state -<3> parameters -<4> set the execution context (ie. define the index name here) -<5> execute and get the template response -<6> get from the template response the search response itself - -You can also execute inline templates: - -[source,java] --------------------------------------------------- -sr = new SearchTemplateRequestBuilder(client) - .setScript("{\n" + <1> - " \"query\" : {\n" + - " \"match\" : {\n" + - " \"gender\" : \"{{param_gender}}\"\n" + - " }\n" + - " }\n" + - "}") - .setScriptType(ScriptType.INLINE) <2> - .setScriptParams(template_params) <3> - .setRequest(new SearchRequest()) <4> - .get() <5> - .getResponse(); <6> --------------------------------------------------- -<1> template's body -<2> template is passed inline -<3> parameters -<4> set the execution context (ie. define the index name here) -<5> execute and get the template response -<6> get from the template response the search response itself diff --git a/docs/java-rest/high-level/dataframe/get_data_frame.asciidoc b/docs/java-rest/high-level/dataframe/get_data_frame.asciidoc index ec2253b2c25f4..995d9d2c08963 100644 --- a/docs/java-rest/high-level/dataframe/get_data_frame.asciidoc +++ b/docs/java-rest/high-level/dataframe/get_data_frame.asciidoc @@ -32,6 +32,7 @@ include-tagged::{doc-tests-file}[{api}-request-options] <1> The page parameters `from` and `size`. `from` specifies the number of {dataframe-transforms} to skip. `size` specifies the maximum number of {dataframe-transforms} to get. Defaults to `0` and `100` respectively. +<2> Whether to ignore if a wildcard expression matches no transforms. include::../execution.asciidoc[] diff --git a/docs/java-rest/high-level/dataframe/get_data_frame_stats.asciidoc b/docs/java-rest/high-level/dataframe/get_data_frame_stats.asciidoc index cdc6254a4e443..8a3e1a96acb14 100644 --- a/docs/java-rest/high-level/dataframe/get_data_frame_stats.asciidoc +++ b/docs/java-rest/high-level/dataframe/get_data_frame_stats.asciidoc @@ -22,6 +22,19 @@ include-tagged::{doc-tests-file}[{api}-request] -------------------------------------------------- <1> Constructing a new GET Stats request referencing an existing {dataframe-transform} +==== Optional Arguments + +The following arguments are optional. + +["source","java",subs="attributes,callouts,macros"] +-------------------------------------------------- +include-tagged::{doc-tests-file}[{api}-request-options] +-------------------------------------------------- +<1> The page parameters `from` and `size`. `from` specifies the number of data frame transform stats to skip. +`size` specifies the maximum number of data frame transform stats to get. +Defaults to `0` and `100` respectively. +<2> Whether to ignore if a wildcard expression matches no transforms. + include::../execution.asciidoc[] @@ -39,4 +52,5 @@ include-tagged::{doc-tests-file}[{api}-response] <3> The running state of the transform indexer e.g `started`, `indexing`, etc. <4> The overall transform statistics recording the number of documents indexed etc. <5> The progress of the current run in the transform. Supplies the number of docs left until the next checkpoint -and the total number of docs expected. \ No newline at end of file +and the total number of docs expected. +<6> The assigned node information if the task is currently assigned to a node and running. \ No newline at end of file diff --git a/docs/java-rest/high-level/dataframe/put_data_frame.asciidoc b/docs/java-rest/high-level/dataframe/put_data_frame.asciidoc index 567449c9c25b1..19c7fe443dbcd 100644 --- a/docs/java-rest/high-level/dataframe/put_data_frame.asciidoc +++ b/docs/java-rest/high-level/dataframe/put_data_frame.asciidoc @@ -33,7 +33,7 @@ include-tagged::{doc-tests-file}[{api}-config] -------------------------------------------------- <1> The {dataframe-transform} ID <2> The source indices and query from which to gather data -<3> The destination index +<3> The destination index and optional pipeline <4> The PivotConfig <5> Optional free text description of the transform @@ -49,6 +49,16 @@ If query is not set, a `match_all` query is used by default. include-tagged::{doc-tests-file}[{api}-source-config] -------------------------------------------------- +==== DestConfig + +The index where to write the data and the optional pipeline +through which the docs should be indexed + +["source","java",subs="attributes,callouts,macros"] +-------------------------------------------------- +include-tagged::{doc-tests-file}[{api}-dest-config] +-------------------------------------------------- + ===== QueryConfig The query with which to select data from the source. diff --git a/docs/java-rest/high-level/dataframe/stop_data_frame.asciidoc b/docs/java-rest/high-level/dataframe/stop_data_frame.asciidoc index 9b05687c00875..1de4af5c5d592 100644 --- a/docs/java-rest/high-level/dataframe/stop_data_frame.asciidoc +++ b/docs/java-rest/high-level/dataframe/stop_data_frame.asciidoc @@ -32,6 +32,7 @@ include-tagged::{doc-tests-file}[{api}-request-options] -------------------------------------------------- <1> If true wait for the data frame task to stop before responding <2> Controls the amount of time to wait until the {dataframe-job} stops. +<3> Whether to ignore if a wildcard expression matches no transforms. include::../execution.asciidoc[] diff --git a/docs/java-rest/high-level/document/delete-by-query.asciidoc b/docs/java-rest/high-level/document/delete-by-query.asciidoc index 37be8e2f3f4c8..f4ef87741e639 100644 --- a/docs/java-rest/high-level/document/delete-by-query.asciidoc +++ b/docs/java-rest/high-level/document/delete-by-query.asciidoc @@ -39,11 +39,11 @@ include-tagged::{doc-tests-file}[{api}-request-query] -------------------------------------------------- <1> Only copy documents which have field `user` set to `kimchy` -It’s also possible to limit the number of processed documents by setting size. +It’s also possible to limit the number of processed documents by setting `maxDocs`. ["source","java",subs="attributes,callouts,macros"] -------------------------------------------------- -include-tagged::{doc-tests-file}[{api}-request-size] +include-tagged::{doc-tests-file}[{api}-request-maxDocs] -------------------------------------------------- <1> Only copy 10 documents diff --git a/docs/java-rest/high-level/document/reindex.asciidoc b/docs/java-rest/high-level/document/reindex.asciidoc index d78851f3d6a86..ccfb0c9db1a56 100644 --- a/docs/java-rest/high-level/document/reindex.asciidoc +++ b/docs/java-rest/high-level/document/reindex.asciidoc @@ -65,11 +65,11 @@ include-tagged::{doc-tests-file}[{api}-request-query] -------------------------------------------------- <1> Only copy documents which have field `user` set to `kimchy` -It’s also possible to limit the number of processed documents by setting size. +It’s also possible to limit the number of processed documents by setting `maxDocs`. ["source","java",subs="attributes,callouts,macros"] -------------------------------------------------- -include-tagged::{doc-tests-file}[{api}-request-size] +include-tagged::{doc-tests-file}[{api}-request-maxDocs] -------------------------------------------------- <1> Only copy 10 documents @@ -90,7 +90,7 @@ include-tagged::{doc-tests-file}[{api}-request-pipeline] <1> set pipeline to `my_pipeline` If you want a particular set of documents from the source index you’ll need to use sort. If possible, prefer a more -selective query to size and sort. +selective query to maxDocs and sort. ["source","java",subs="attributes,callouts,macros"] -------------------------------------------------- diff --git a/docs/java-rest/high-level/document/term-vectors.asciidoc b/docs/java-rest/high-level/document/term-vectors.asciidoc index e8d4a25a2cac0..36c7553d4885e 100644 --- a/docs/java-rest/high-level/document/term-vectors.asciidoc +++ b/docs/java-rest/high-level/document/term-vectors.asciidoc @@ -15,7 +15,7 @@ artificially provided by the user. [id="{upid}-{api}-request"] ==== Term Vectors Request -A +{request}+ expects an `index`, a `type` and an `id` to specify +A +{request}+ expects an `index` and an `id` to specify a certain document, and fields for which the information is retrieved. ["source","java",subs="attributes,callouts,macros"] @@ -71,9 +71,8 @@ include::../execution.asciidoc[] include-tagged::{doc-tests-file}[{api}-response] -------------------------------------------------- <1> The index name of the document. -<2> The type name of the document. -<3> The id of the document. -<4> Indicates whether or not the document found. +<2> The id of the document. +<3> Indicates whether or not the document found. ===== Inspecting Term Vectors diff --git a/docs/java-rest/high-level/document/update-by-query.asciidoc b/docs/java-rest/high-level/document/update-by-query.asciidoc index be1692c4e9f6a..26a6bc362b191 100644 --- a/docs/java-rest/high-level/document/update-by-query.asciidoc +++ b/docs/java-rest/high-level/document/update-by-query.asciidoc @@ -40,11 +40,11 @@ include-tagged::{doc-tests-file}[{api}-request-query] -------------------------------------------------- <1> Only copy documents which have field `user` set to `kimchy` -It’s also possible to limit the number of processed documents by setting size. +It’s also possible to limit the number of processed documents by setting `maxDocs`. ["source","java",subs="attributes,callouts,macros"] -------------------------------------------------- -include-tagged::{doc-tests-file}[{api}-request-size] +include-tagged::{doc-tests-file}[{api}-request-maxDocs] -------------------------------------------------- <1> Only copy 10 documents diff --git a/docs/java-rest/high-level/indices/analyze.asciidoc b/docs/java-rest/high-level/indices/analyze.asciidoc index 4978c9ebcca64..9464394fd1eb9 100644 --- a/docs/java-rest/high-level/indices/analyze.asciidoc +++ b/docs/java-rest/high-level/indices/analyze.asciidoc @@ -19,18 +19,18 @@ The simplest version uses a built-in analyzer: --------------------------------------------------- include-tagged::{doc-tests-file}[{api}-builtin-request] --------------------------------------------------- -<1> The text to include. Multiple strings are treated as a multi-valued field -<2> A built-in analyzer +<1> A built-in analyzer +<2> The text to include. Multiple strings are treated as a multi-valued field You can configure a custom analyzer: ["source","java",subs="attributes,callouts,macros"] --------------------------------------------------- include-tagged::{doc-tests-file}[{api}-custom-request] --------------------------------------------------- -<1> Configure char filters +<1> Configuration for a custom tokenfilter <2> Configure the tokenizer -<3> Add a built-in tokenfilter -<4> Configuration for a custom tokenfilter +<3> Configure char filters +<4> Add a built-in tokenfilter <5> Add the custom tokenfilter You can also build a custom normalizer, by including only charfilters and diff --git a/docs/java-rest/high-level/indices/reload_analyzers.asciidoc b/docs/java-rest/high-level/indices/reload_analyzers.asciidoc new file mode 100644 index 0000000000000..29db206bf1402 --- /dev/null +++ b/docs/java-rest/high-level/indices/reload_analyzers.asciidoc @@ -0,0 +1,50 @@ +-- +:api: reload-analyzers +:request: ReloadAnalyzersRequest +:response: ReloadAnalyzersResponse +-- + +[id="{upid}-{api}"] +=== Reload Search Analyzers API + +[id="{upid}-{api}-request"] +==== Reload Search Analyzers Request + +An +{request}+ requires an `index` argument: + +["source","java",subs="attributes,callouts,macros"] +-------------------------------------------------- +include-tagged::{doc-tests-file}[{api}-request] +-------------------------------------------------- +<1> The index to reload + +==== Optional arguments +The following arguments can optionally be provided: + +["source","java",subs="attributes,callouts,macros"] +-------------------------------------------------- +include-tagged::{doc-tests-file}[{api}-request-indicesOptions] +-------------------------------------------------- +<1> Setting `IndicesOptions` controls how unavailable indices are resolved and +how wildcard expressions are expanded + +include::../execution.asciidoc[] + +[id="{upid}-{api}-response"] +==== Reload Search Analyzers Response + +The returned +{response}+ allows to retrieve information about the +executed operation as follows: + +["source","java",subs="attributes,callouts,macros"] +-------------------------------------------------- +include-tagged::{doc-tests-file}[{api}-response] +-------------------------------------------------- +<1> Shard statistics. Note that reloading does not happen on each shard of an +index, but once on each node the index has shards on. The reported shard count +can therefore differ from the number of index shards +<2> Reloading details of all indices the request was executed on +<3> Details can be retrieved by index name +<4> The reloaded index name +<5> The nodes the index was reloaded on +<6> The analyzer names that were reloaded diff --git a/docs/java-rest/high-level/ml/delete-data-frame-analytics.asciidoc b/docs/java-rest/high-level/ml/delete-data-frame-analytics.asciidoc new file mode 100644 index 0000000000000..2e5ade37107cf --- /dev/null +++ b/docs/java-rest/high-level/ml/delete-data-frame-analytics.asciidoc @@ -0,0 +1,28 @@ +-- +:api: delete-data-frame-analytics +:request: DeleteDataFrameAnalyticsRequest +:response: AcknowledgedResponse +-- +[id="{upid}-{api}"] +=== Delete Data Frame Analytics API + +The Delete Data Frame Analytics API is used to delete an existing {dataframe-analytics-config}. +The API accepts a +{request}+ object as a request and returns a +{response}+. + +[id="{upid}-{api}-request"] +==== Delete Data Frame Analytics Request + +A +{request}+ object requires a {dataframe-analytics-config} id. + +["source","java",subs="attributes,callouts,macros"] +--------------------------------------------------- +include-tagged::{doc-tests-file}[{api}-request] +--------------------------------------------------- +<1> Constructing a new request referencing an existing {dataframe-analytics-config} + +include::../execution.asciidoc[] + +[id="{upid}-{api}-response"] +==== Response + +The returned +{response}+ object acknowledges the {dataframe-analytics-config} deletion. diff --git a/docs/java-rest/high-level/ml/evaluate-data-frame.asciidoc b/docs/java-rest/high-level/ml/evaluate-data-frame.asciidoc new file mode 100644 index 0000000000000..660603d2e38e7 --- /dev/null +++ b/docs/java-rest/high-level/ml/evaluate-data-frame.asciidoc @@ -0,0 +1,45 @@ +-- +:api: evaluate-data-frame +:request: EvaluateDataFrameRequest +:response: EvaluateDataFrameResponse +-- +[id="{upid}-{api}"] +=== Evaluate Data Frame API + +The Evaluate Data Frame API is used to evaluate an ML algorithm that ran on a {dataframe}. +The API accepts an +{request}+ object and returns an +{response}+. + +[id="{upid}-{api}-request"] +==== Evaluate Data Frame Request + +["source","java",subs="attributes,callouts,macros"] +-------------------------------------------------- +include-tagged::{doc-tests-file}[{api}-request] +-------------------------------------------------- +<1> Constructing a new evaluation request +<2> Reference to an existing index +<3> Kind of evaluation to perform +<4> Name of the field in the index. Its value denotes the actual (i.e. ground truth) label for an example. Must be either true or false +<5> Name of the field in the index. Its value denotes the probability (as per some ML algorithm) of the example being classified as positive +<6> The remaining parameters are the metrics to be calculated based on the two fields described above. +<7> https://en.wikipedia.org/wiki/Precision_and_recall[Precision] calculated at thresholds: 0.4, 0.5 and 0.6 +<8> https://en.wikipedia.org/wiki/Precision_and_recall[Recall] calculated at thresholds: 0.5 and 0.7 +<9> https://en.wikipedia.org/wiki/Confusion_matrix[Confusion matrix] calculated at threshold 0.5 +<10> https://en.wikipedia.org/wiki/Receiver_operating_characteristic#Area_under_the_curve[AuC ROC] calculated and the curve points returned + +include::../execution.asciidoc[] + +[id="{upid}-{api}-response"] +==== Response + +The returned +{response}+ contains the requested evaluation metrics. + +["source","java",subs="attributes,callouts,macros"] +-------------------------------------------------- +include-tagged::{doc-tests-file}[{api}-response] +-------------------------------------------------- +<1> Fetching all the calculated metrics results +<2> Fetching precision metric by name +<3> Fetching precision at a given (0.4) threshold +<4> Fetching confusion matrix metric by name +<5> Fetching confusion matrix at a given (0.5) threshold \ No newline at end of file diff --git a/docs/java-rest/high-level/ml/get-data-frame-analytics-stats.asciidoc b/docs/java-rest/high-level/ml/get-data-frame-analytics-stats.asciidoc new file mode 100644 index 0000000000000..e1047e9b3e002 --- /dev/null +++ b/docs/java-rest/high-level/ml/get-data-frame-analytics-stats.asciidoc @@ -0,0 +1,34 @@ +-- +:api: get-data-frame-analytics-stats +:request: GetDataFrameAnalyticsStatsRequest +:response: GetDataFrameAnalyticsStatsResponse +-- +[id="{upid}-{api}"] +=== Get Data Frame Analytics Stats API + +The Get Data Frame Analytics Stats API is used to read the operational statistics of one or more {dataframe-analytics-config}s. +The API accepts a +{request}+ object and returns a +{response}+. + +[id="{upid}-{api}-request"] +==== Get Data Frame Analytics Stats Request + +A +{request}+ requires either a {dataframe-analytics-config} id, a comma separated list of ids or +the special wildcard `_all` to get the statistics for all {dataframe-analytics-config}s + +["source","java",subs="attributes,callouts,macros"] +-------------------------------------------------- +include-tagged::{doc-tests-file}[{api}-request] +-------------------------------------------------- +<1> Constructing a new GET Stats request referencing an existing {dataframe-analytics-config} + +include::../execution.asciidoc[] + +[id="{upid}-{api}-response"] +==== Response + +The returned +{response}+ contains the requested {dataframe-analytics-config} statistics. + +["source","java",subs="attributes,callouts,macros"] +-------------------------------------------------- +include-tagged::{doc-tests-file}[{api}-response] +-------------------------------------------------- \ No newline at end of file diff --git a/docs/java-rest/high-level/ml/get-data-frame-analytics.asciidoc b/docs/java-rest/high-level/ml/get-data-frame-analytics.asciidoc new file mode 100644 index 0000000000000..c6d368efbcae9 --- /dev/null +++ b/docs/java-rest/high-level/ml/get-data-frame-analytics.asciidoc @@ -0,0 +1,34 @@ +-- +:api: get-data-frame-analytics +:request: GetDataFrameAnalyticsRequest +:response: GetDataFrameAnalyticsResponse +-- +[id="{upid}-{api}"] +=== Get Data Frame Analytics API + +The Get Data Frame Analytics API is used to get one or more {dataframe-analytics-config}s. +The API accepts a +{request}+ object and returns a +{response}+. + +[id="{upid}-{api}-request"] +==== Get Data Frame Analytics Request + +A +{request}+ requires either a {dataframe-analytics-config} id, a comma separated list of ids or +the special wildcard `_all` to get all {dataframe-analytics-config}s. + +["source","java",subs="attributes,callouts,macros"] +-------------------------------------------------- +include-tagged::{doc-tests-file}[{api}-request] +-------------------------------------------------- +<1> Constructing a new GET request referencing an existing {dataframe-analytics-config} + +include::../execution.asciidoc[] + +[id="{upid}-{api}-response"] +==== Response + +The returned +{response}+ contains the requested {dataframe-analytics-config}s. + +["source","java",subs="attributes,callouts,macros"] +-------------------------------------------------- +include-tagged::{doc-tests-file}[{api}-response] +-------------------------------------------------- diff --git a/docs/java-rest/high-level/ml/put-data-frame-analytics.asciidoc b/docs/java-rest/high-level/ml/put-data-frame-analytics.asciidoc new file mode 100644 index 0000000000000..05fbd5bc3922a --- /dev/null +++ b/docs/java-rest/high-level/ml/put-data-frame-analytics.asciidoc @@ -0,0 +1,115 @@ +-- +:api: put-data-frame-analytics +:request: PutDataFrameAnalyticsRequest +:response: PutDataFrameAnalyticsResponse +-- +[id="{upid}-{api}"] +=== Put Data Frame Analytics API + +The Put Data Frame Analytics API is used to create a new {dataframe-analytics-config}. +The API accepts a +{request}+ object as a request and returns a +{response}+. + +[id="{upid}-{api}-request"] +==== Put Data Frame Analytics Request + +A +{request}+ requires the following argument: + +["source","java",subs="attributes,callouts,macros"] +-------------------------------------------------- +include-tagged::{doc-tests-file}[{api}-request] +-------------------------------------------------- +<1> The configuration of the {dataframe-job} to create + +[id="{upid}-{api}-config"] +==== Data Frame Analytics Configuration + +The `DataFrameAnalyticsConfig` object contains all the details about the {dataframe-job} +configuration and contains the following arguments: + +["source","java",subs="attributes,callouts,macros"] +-------------------------------------------------- +include-tagged::{doc-tests-file}[{api}-config] +-------------------------------------------------- +<1> The {dataframe-analytics-config} id +<2> The source index and query from which to gather data +<3> The destination index +<4> The analysis to be performed +<5> The fields to be included in / excluded from the analysis +<6> The memory limit for the model created as part of the analysis process + +[id="{upid}-{api}-query-config"] + +==== SourceConfig + +The index and the query from which to collect data. + +["source","java",subs="attributes,callouts,macros"] +-------------------------------------------------- +include-tagged::{doc-tests-file}[{api}-source-config] +-------------------------------------------------- +<1> Constructing a new DataFrameAnalyticsSource +<2> The source index +<3> The query from which to gather the data. If query is not set, a `match_all` query is used by default. + +===== QueryConfig + +The query with which to select data from the source. + +["source","java",subs="attributes,callouts,macros"] +-------------------------------------------------- +include-tagged::{doc-tests-file}[{api}-query-config] +-------------------------------------------------- + +==== DestinationConfig + +The index to which data should be written by the {dataframe-job}. + +["source","java",subs="attributes,callouts,macros"] +-------------------------------------------------- +include-tagged::{doc-tests-file}[{api}-dest-config] +-------------------------------------------------- +<1> Constructing a new DataFrameAnalyticsDest +<2> The destination index + +==== Analysis + +The analysis to be performed. +Currently, only one analysis is supported: +OutlierDetection+. + ++OutlierDetection+ analysis can be created in one of two ways: + +["source","java",subs="attributes,callouts,macros"] +-------------------------------------------------- +include-tagged::{doc-tests-file}[{api}-analysis-default] +-------------------------------------------------- +<1> Constructing a new OutlierDetection object with default strategy to determine outliers + +or +["source","java",subs="attributes,callouts,macros"] +-------------------------------------------------- +include-tagged::{doc-tests-file}[{api}-analysis-customized] +-------------------------------------------------- +<1> Constructing a new OutlierDetection object +<2> The method used to perform the analysis +<3> Number of neighbors taken into account during analysis + +==== Analyzed fields + +FetchContext object containing fields to be included in / excluded from the analysis + +["source","java",subs="attributes,callouts,macros"] +-------------------------------------------------- +include-tagged::{doc-tests-file}[{api}-analyzed-fields] +-------------------------------------------------- + +include::../execution.asciidoc[] + +[id="{upid}-{api}-response"] +==== Response + +The returned +{response}+ contains the newly created {dataframe-analytics-config}. + +["source","java",subs="attributes,callouts,macros"] +-------------------------------------------------- +include-tagged::{doc-tests-file}[{api}-response] +-------------------------------------------------- \ No newline at end of file diff --git a/docs/java-rest/high-level/ml/start-data-frame-analytics.asciidoc b/docs/java-rest/high-level/ml/start-data-frame-analytics.asciidoc new file mode 100644 index 0000000000000..610607daba1f8 --- /dev/null +++ b/docs/java-rest/high-level/ml/start-data-frame-analytics.asciidoc @@ -0,0 +1,28 @@ +-- +:api: start-data-frame-analytics +:request: StartDataFrameAnalyticsRequest +:response: AcknowledgedResponse +-- +[id="{upid}-{api}"] +=== Start Data Frame Analytics API + +The Start Data Frame Analytics API is used to start an existing {dataframe-analytics-config}. +It accepts a +{request}+ object and responds with a +{response}+ object. + +[id="{upid}-{api}-request"] +==== Start Data Frame Analytics Request + +A +{request}+ object requires a {dataframe-analytics-config} id. + +["source","java",subs="attributes,callouts,macros"] +--------------------------------------------------- +include-tagged::{doc-tests-file}[{api}-request] +--------------------------------------------------- +<1> Constructing a new start request referencing an existing {dataframe-analytics-config} + +include::../execution.asciidoc[] + +[id="{upid}-{api}-response"] +==== Response + +The returned +{response}+ object acknowledges the {dataframe-job} has started. \ No newline at end of file diff --git a/docs/java-rest/high-level/ml/stop-data-frame-analytics.asciidoc b/docs/java-rest/high-level/ml/stop-data-frame-analytics.asciidoc new file mode 100644 index 0000000000000..3a06f268836f6 --- /dev/null +++ b/docs/java-rest/high-level/ml/stop-data-frame-analytics.asciidoc @@ -0,0 +1,29 @@ +-- +:api: stop-data-frame-analytics +:request: StopDataFrameAnalyticsRequest +:response: StopDataFrameAnalyticsResponse +-- +[id="{upid}-{api}"] +=== Stop Data Frame Analytics API + +The Stop Data Frame Analytics API is used to stop a running {dataframe-analytics-config}. +It accepts a +{request}+ object and responds with a +{response}+ object. + +[id="{upid}-{api}-request"] +==== Stop Data Frame Analytics Request + +A +{request}+ object requires a {dataframe-analytics-config} id. + +["source","java",subs="attributes,callouts,macros"] +--------------------------------------------------- +include-tagged::{doc-tests-file}[{api}-request] +--------------------------------------------------- +<1> Constructing a new stop request referencing an existing {dataframe-analytics-config} +<2> Optionally used to stop a failed task + +include::../execution.asciidoc[] + +[id="{upid}-{api}-response"] +==== Response + +The returned +{response}+ object acknowledges the {dataframe-job} has stopped. \ No newline at end of file diff --git a/docs/java-rest/high-level/query-builders.asciidoc b/docs/java-rest/high-level/query-builders.asciidoc index 53d9b9af97d12..f07e81c2c413c 100644 --- a/docs/java-rest/high-level/query-builders.asciidoc +++ b/docs/java-rest/high-level/query-builders.asciidoc @@ -23,12 +23,11 @@ This page lists all the available search queries with their corresponding `Query | {ref}/query-dsl-match-query-phrase.html[Match Phrase] | {query-ref}/MatchPhraseQueryBuilder.html[MatchPhraseQueryBuilder] | {query-ref}/QueryBuilders.html#matchPhraseQuery-java.lang.String-java.lang.Object-[QueryBuilders.matchPhraseQuery()] | {ref}/query-dsl-match-query-phrase-prefix.html[Match Phrase Prefix] | {query-ref}/MatchPhrasePrefixQueryBuilder.html[MatchPhrasePrefixQueryBuilder] | {query-ref}/QueryBuilders.html#matchPhrasePrefixQuery-java.lang.String-java.lang.Object-[QueryBuilders.matchPhrasePrefixQuery()] | {ref}/query-dsl-multi-match-query.html[Multi Match] | {query-ref}/MultiMatchQueryBuilder.html[MultiMatchQueryBuilder] | {query-ref}/QueryBuilders.html#multiMatchQuery-java.lang.Object-java.lang.String\…-[QueryBuilders.multiMatchQuery()] -| {ref}/query-dsl-common-terms-query.html[Common Terms] | {query-ref}/CommonTermsQueryBuilder.html[CommonTermsQueryBuilder] | {query-ref}/QueryBuilders.html#commonTermsQuery-java.lang.String-java.lang.Object-[QueryBuilders.commonTermsQuery()] | {ref}/query-dsl-query-string-query.html[Query String] | {query-ref}/QueryStringQueryBuilder.html[QueryStringQueryBuilder] | {query-ref}/QueryBuilders.html#queryStringQuery-java.lang.String-[QueryBuilders.queryStringQuery()] | {ref}/query-dsl-simple-query-string-query.html[Simple Query String] | {query-ref}/SimpleQueryStringBuilder.html[SimpleQueryStringBuilder] | {query-ref}/QueryBuilders.html#simpleQueryStringQuery-java.lang.String-[QueryBuilders.simpleQueryStringQuery()] |====== -==== Term level queries +==== Term-level queries [options="header"] |====== | Search Query | QueryBuilder Class | Method in QueryBuilders diff --git a/docs/java-rest/high-level/security/get-builtin-privileges.asciidoc b/docs/java-rest/high-level/security/get-builtin-privileges.asciidoc new file mode 100644 index 0000000000000..e8eeb7b3c9417 --- /dev/null +++ b/docs/java-rest/high-level/security/get-builtin-privileges.asciidoc @@ -0,0 +1,27 @@ +-- +:api: get-builtin-privileges +:request: GetBuiltinPrivilegesRequest +:response: GetBuiltinPrivilegesResponse +-- + +[id="{upid}-{api}"] +=== Get Builtin Privileges API + +include::../execution-no-req.asciidoc[] + +[id="{upid}-{api}-response"] +==== Get Builtin Privileges Response + +The returned +{response}+ contains the following properties + +`clusterPrivileges`:: +A `Set` of all _cluster_ privileges that are understood by this node. + +`indexPrivileges`:: +A `Set` of all _index_ privileges that are understood by this node. + +["source","java",subs="attributes,callouts,macros"] +-------------------------------------------------- +include-tagged::{doc-tests-file}[{api}-response] +-------------------------------------------------- + diff --git a/docs/java-rest/high-level/supported-apis.asciidoc b/docs/java-rest/high-level/supported-apis.asciidoc index 4e28efc2941db..27f5f38136d2e 100644 --- a/docs/java-rest/high-level/supported-apis.asciidoc +++ b/docs/java-rest/high-level/supported-apis.asciidoc @@ -155,6 +155,7 @@ include::indices/get_index.asciidoc[] include::indices/freeze_index.asciidoc[] include::indices/unfreeze_index.asciidoc[] include::indices/delete_template.asciidoc[] +include::indices/reload_analyzers.asciidoc[] == Cluster APIs @@ -285,6 +286,13 @@ The Java High Level REST Client supports the following Machine Learning APIs: * <<{upid}-put-calendar-job>> * <<{upid}-delete-calendar-job>> * <<{upid}-delete-calendar>> +* <<{upid}-get-data-frame-analytics>> +* <<{upid}-get-data-frame-analytics-stats>> +* <<{upid}-put-data-frame-analytics>> +* <<{upid}-delete-data-frame-analytics>> +* <<{upid}-start-data-frame-analytics>> +* <<{upid}-stop-data-frame-analytics>> +* <<{upid}-evaluate-data-frame>> * <<{upid}-put-filter>> * <<{upid}-get-filters>> * <<{upid}-update-filter>> @@ -329,6 +337,13 @@ include::ml/delete-calendar-event.asciidoc[] include::ml/put-calendar-job.asciidoc[] include::ml/delete-calendar-job.asciidoc[] include::ml/delete-calendar.asciidoc[] +include::ml/get-data-frame-analytics.asciidoc[] +include::ml/get-data-frame-analytics-stats.asciidoc[] +include::ml/put-data-frame-analytics.asciidoc[] +include::ml/delete-data-frame-analytics.asciidoc[] +include::ml/start-data-frame-analytics.asciidoc[] +include::ml/stop-data-frame-analytics.asciidoc[] +include::ml/evaluate-data-frame.asciidoc[] include::ml/put-filter.asciidoc[] include::ml/get-filters.asciidoc[] include::ml/update-filter.asciidoc[] @@ -404,6 +419,7 @@ The Java High Level REST Client supports the following Security APIs: * <> * <> * <<{upid}-invalidate-token>> +* <<{upid}-get-builtin-privileges>> * <<{upid}-get-privileges>> * <<{upid}-put-privileges>> * <<{upid}-delete-privileges>> @@ -421,6 +437,7 @@ include::security/put-role.asciidoc[] include::security/get-roles.asciidoc[] include::security/delete-role.asciidoc[] include::security/delete-privileges.asciidoc[] +include::security/get-builtin-privileges.asciidoc[] include::security/get-privileges.asciidoc[] include::security/clear-roles-cache.asciidoc[] include::security/clear-realm-cache.asciidoc[] @@ -568,4 +585,4 @@ include::dataframe/put_data_frame.asciidoc[] include::dataframe/delete_data_frame.asciidoc[] include::dataframe/preview_data_frame.asciidoc[] include::dataframe/start_data_frame.asciidoc[] -include::dataframe/stop_data_frame.asciidoc[] \ No newline at end of file +include::dataframe/stop_data_frame.asciidoc[] diff --git a/docs/painless/painless-api-reference/painless-api-reference-score/packages.asciidoc b/docs/painless/painless-api-reference/painless-api-reference-score/packages.asciidoc index 10f0f1b6daeab..a1beaeb5bc520 100644 --- a/docs/painless/painless-api-reference/painless-api-reference-score/packages.asciidoc +++ b/docs/painless/painless-api-reference/painless-api-reference-score/packages.asciidoc @@ -30,6 +30,8 @@ See the <> for a high-level overview of * List findResults(Function) * void {java11-javadoc}/java.base/java/lang/Iterable.html#forEach(java.util.function.Consumer)[forEach](Consumer) * def {java11-javadoc}/java.base/java/util/List.html#get(int)[get](int) +* Object getByPath(String) +* Object getByPath(String, Object) * int getLength() * Map groupBy(Function) * int {java11-javadoc}/java.base/java/util/List.html#hashCode()[hashCode]() @@ -84,6 +86,8 @@ See the <> for a high-level overview of * List findResults(Function) * void {java11-javadoc}/java.base/java/lang/Iterable.html#forEach(java.util.function.Consumer)[forEach](Consumer) * def {java11-javadoc}/java.base/java/util/List.html#get(int)[get](int) +* Object getByPath(String) +* Object getByPath(String, Object) * int getLength() * Map groupBy(Function) * int {java11-javadoc}/java.base/java/util/List.html#hashCode()[hashCode]() @@ -138,6 +142,8 @@ See the <> for a high-level overview of * List findResults(Function) * void {java11-javadoc}/java.base/java/lang/Iterable.html#forEach(java.util.function.Consumer)[forEach](Consumer) * def {java11-javadoc}/java.base/java/util/List.html#get(int)[get](int) +* Object getByPath(String) +* Object getByPath(String, Object) * int getLength() * Map groupBy(Function) * int {java11-javadoc}/java.base/java/util/List.html#hashCode()[hashCode]() diff --git a/docs/painless/painless-api-reference/painless-api-reference-shared/packages.asciidoc b/docs/painless/painless-api-reference/painless-api-reference-shared/packages.asciidoc index 75ad21ddc93f2..dd8141c2e96d0 100644 --- a/docs/painless/painless-api-reference/painless-api-reference-shared/packages.asciidoc +++ b/docs/painless/painless-api-reference/painless-api-reference-shared/packages.asciidoc @@ -4335,6 +4335,8 @@ See the <> for a high-level overview * List findResults(Function) * void {java11-javadoc}/java.base/java/lang/Iterable.html#forEach(java.util.function.Consumer)[forEach](Consumer) * def {java11-javadoc}/java.base/java/util/List.html#get(int)[get](int) +* Object getByPath(String) +* Object getByPath(String, Object) * int getLength() * Map groupBy(Function) * int {java11-javadoc}/java.base/java/util/List.html#hashCode()[hashCode]() @@ -4386,6 +4388,8 @@ See the <> for a high-level overview * List findResults(BiFunction) * void {java11-javadoc}/java.base/java/util/Map.html#forEach(java.util.function.BiConsumer)[forEach](BiConsumer) * def {java11-javadoc}/java.base/java/util/Map.html#get(java.lang.Object)[get](def) +* Object getByPath(String) +* Object getByPath(String, Object) * def {java11-javadoc}/java.base/java/util/Map.html#getOrDefault(java.lang.Object,java.lang.Object)[getOrDefault](def, def) * Map groupBy(BiFunction) * int {java11-javadoc}/java.base/java/lang/Object.html#hashCode()[hashCode]() @@ -4500,6 +4504,8 @@ See the <> for a high-level overview * List findResults(Function) * void {java11-javadoc}/java.base/java/lang/Iterable.html#forEach(java.util.function.Consumer)[forEach](Consumer) * def {java11-javadoc}/java.base/java/util/List.html#get(int)[get](int) +* Object getByPath(String) +* Object getByPath(String, Object) * int getLength() * Map groupBy(Function) * int {java11-javadoc}/java.base/java/util/List.html#hashCode()[hashCode]() @@ -4666,6 +4672,8 @@ See the <> for a high-level overview * List findResults(Function) * void {java11-javadoc}/java.base/java/lang/Iterable.html#forEach(java.util.function.Consumer)[forEach](Consumer) * def {java11-javadoc}/java.base/java/util/List.html#get(int)[get](int) +* Object getByPath(String) +* Object getByPath(String, Object) * int getLength() * Map groupBy(Function) * int {java11-javadoc}/java.base/java/util/List.html#hashCode()[hashCode]() @@ -5367,6 +5375,8 @@ See the <> for a high-level overview * List findResults(BiFunction) * void {java11-javadoc}/java.base/java/util/Map.html#forEach(java.util.function.BiConsumer)[forEach](BiConsumer) * def {java11-javadoc}/java.base/java/util/Map.html#get(java.lang.Object)[get](def) +* Object getByPath(String) +* Object getByPath(String, Object) * def {java11-javadoc}/java.base/java/util/Map.html#getOrDefault(java.lang.Object,java.lang.Object)[getOrDefault](def, def) * Map groupBy(BiFunction) * int {java11-javadoc}/java.base/java/lang/Object.html#hashCode()[hashCode]() @@ -5457,6 +5467,8 @@ See the <> for a high-level overview * List findResults(BiFunction) * void {java11-javadoc}/java.base/java/util/Map.html#forEach(java.util.function.BiConsumer)[forEach](BiConsumer) * def {java11-javadoc}/java.base/java/util/Map.html#get(java.lang.Object)[get](def) +* Object getByPath(String) +* Object getByPath(String, Object) * def {java11-javadoc}/java.base/java/util/Map.html#getOrDefault(java.lang.Object,java.lang.Object)[getOrDefault](def, def) * Map groupBy(BiFunction) * int {java11-javadoc}/java.base/java/lang/Object.html#hashCode()[hashCode]() @@ -5502,6 +5514,8 @@ See the <> for a high-level overview * List findResults(BiFunction) * void {java11-javadoc}/java.base/java/util/Map.html#forEach(java.util.function.BiConsumer)[forEach](BiConsumer) * def {java11-javadoc}/java.base/java/util/Map.html#get(java.lang.Object)[get](def) +* Object getByPath(String) +* Object getByPath(String, Object) * def {java11-javadoc}/java.base/java/util/Map.html#getOrDefault(java.lang.Object,java.lang.Object)[getOrDefault](def, def) * Map groupBy(BiFunction) * int {java11-javadoc}/java.base/java/lang/Object.html#hashCode()[hashCode]() @@ -5668,6 +5682,8 @@ See the <> for a high-level overview * List findResults(BiFunction) * void {java11-javadoc}/java.base/java/util/Map.html#forEach(java.util.function.BiConsumer)[forEach](BiConsumer) * def {java11-javadoc}/java.base/java/util/Map.html#get(java.lang.Object)[get](def) +* Object getByPath(String) +* Object getByPath(String, Object) * def {java11-javadoc}/java.base/java/util/Map.html#getOrDefault(java.lang.Object,java.lang.Object)[getOrDefault](def, def) * Map groupBy(BiFunction) * int {java11-javadoc}/java.base/java/lang/Object.html#hashCode()[hashCode]() @@ -5764,6 +5780,8 @@ See the <> for a high-level overview * List findResults(Function) * void {java11-javadoc}/java.base/java/lang/Iterable.html#forEach(java.util.function.Consumer)[forEach](Consumer) * def {java11-javadoc}/java.base/java/util/List.html#get(int)[get](int) +* Object getByPath(String) +* Object getByPath(String, Object) * def {java11-javadoc}/java.base/java/util/Deque.html#getFirst()[getFirst]() * def {java11-javadoc}/java.base/java/util/Deque.html#getLast()[getLast]() * int getLength() @@ -5836,6 +5854,8 @@ See the <> for a high-level overview * List findResults(Function) * void {java11-javadoc}/java.base/java/lang/Iterable.html#forEach(java.util.function.Consumer)[forEach](Consumer) * def {java11-javadoc}/java.base/java/util/List.html#get(int)[get](int) +* Object getByPath(String) +* Object getByPath(String, Object) * int getLength() * Map groupBy(Function) * int {java11-javadoc}/java.base/java/util/List.html#hashCode()[hashCode]() @@ -6056,6 +6076,8 @@ See the <> for a high-level overview * List findResults(BiFunction) * void {java11-javadoc}/java.base/java/util/Map.html#forEach(java.util.function.BiConsumer)[forEach](BiConsumer) * def {java11-javadoc}/java.base/java/util/Map.html#get(java.lang.Object)[get](def) +* Object getByPath(String) +* Object getByPath(String, Object) * def {java11-javadoc}/java.base/java/util/Map.html#getOrDefault(java.lang.Object,java.lang.Object)[getOrDefault](def, def) * Map groupBy(BiFunction) * int {java11-javadoc}/java.base/java/lang/Object.html#hashCode()[hashCode]() @@ -6157,6 +6179,8 @@ See the <> for a high-level overview * def {java11-javadoc}/java.base/java/util/NavigableMap.html#floorKey(java.lang.Object)[floorKey](def) * void {java11-javadoc}/java.base/java/util/Map.html#forEach(java.util.function.BiConsumer)[forEach](BiConsumer) * def {java11-javadoc}/java.base/java/util/Map.html#get(java.lang.Object)[get](def) +* Object getByPath(String) +* Object getByPath(String, Object) * def {java11-javadoc}/java.base/java/util/Map.html#getOrDefault(java.lang.Object,java.lang.Object)[getOrDefault](def, def) * Map groupBy(BiFunction) * int {java11-javadoc}/java.base/java/lang/Object.html#hashCode()[hashCode]() @@ -6642,6 +6666,8 @@ See the <> for a high-level overview * def {java11-javadoc}/java.base/java/util/SortedMap.html#firstKey()[firstKey]() * void {java11-javadoc}/java.base/java/util/Map.html#forEach(java.util.function.BiConsumer)[forEach](BiConsumer) * def {java11-javadoc}/java.base/java/util/Map.html#get(java.lang.Object)[get](def) +* Object getByPath(String) +* Object getByPath(String, Object) * def {java11-javadoc}/java.base/java/util/Map.html#getOrDefault(java.lang.Object,java.lang.Object)[getOrDefault](def, def) * Map groupBy(BiFunction) * int {java11-javadoc}/java.base/java/lang/Object.html#hashCode()[hashCode]() @@ -6844,6 +6870,8 @@ See the <> for a high-level overview * def {java11-javadoc}/java.base/java/util/Vector.html#firstElement()[firstElement]() * void {java11-javadoc}/java.base/java/lang/Iterable.html#forEach(java.util.function.Consumer)[forEach](Consumer) * def {java11-javadoc}/java.base/java/util/List.html#get(int)[get](int) +* Object getByPath(String) +* Object getByPath(String, Object) * int getLength() * Map groupBy(Function) * int {java11-javadoc}/java.base/java/util/List.html#hashCode()[hashCode]() @@ -6988,6 +7016,8 @@ See the <> for a high-level overview * def {java11-javadoc}/java.base/java/util/NavigableMap.html#floorKey(java.lang.Object)[floorKey](def) * void {java11-javadoc}/java.base/java/util/Map.html#forEach(java.util.function.BiConsumer)[forEach](BiConsumer) * def {java11-javadoc}/java.base/java/util/Map.html#get(java.lang.Object)[get](def) +* Object getByPath(String) +* Object getByPath(String, Object) * def {java11-javadoc}/java.base/java/util/Map.html#getOrDefault(java.lang.Object,java.lang.Object)[getOrDefault](def, def) * Map groupBy(BiFunction) * int {java11-javadoc}/java.base/java/lang/Object.html#hashCode()[hashCode]() @@ -7158,6 +7188,8 @@ See the <> for a high-level overview * def {java11-javadoc}/java.base/java/util/Vector.html#firstElement()[firstElement]() * void {java11-javadoc}/java.base/java/lang/Iterable.html#forEach(java.util.function.Consumer)[forEach](Consumer) * def {java11-javadoc}/java.base/java/util/List.html#get(int)[get](int) +* Object getByPath(String) +* Object getByPath(String, Object) * int getLength() * Map groupBy(Function) * int {java11-javadoc}/java.base/java/util/List.html#hashCode()[hashCode]() @@ -8016,6 +8048,8 @@ See the <> for a high-level overview * List findResults(Function) * void {java11-javadoc}/java.base/java/lang/Iterable.html#forEach(java.util.function.Consumer)[forEach](Consumer) * Boolean get(int) +* Object getByPath(String) +* Object getByPath(String, Object) * int getLength() * boolean getValue() * Map groupBy(Function) @@ -8071,6 +8105,8 @@ See the <> for a high-level overview * List findResults(Function) * void {java11-javadoc}/java.base/java/lang/Iterable.html#forEach(java.util.function.Consumer)[forEach](Consumer) * BytesRef get(int) +* Object getByPath(String) +* Object getByPath(String, Object) * int getLength() * BytesRef getValue() * Map groupBy(Function) @@ -8126,6 +8162,8 @@ See the <> for a high-level overview * List findResults(Function) * void {java11-javadoc}/java.base/java/lang/Iterable.html#forEach(java.util.function.Consumer)[forEach](Consumer) * JodaCompatibleZonedDateTime get(int) +* Object getByPath(String) +* Object getByPath(String, Object) * int getLength() * JodaCompatibleZonedDateTime getValue() * Map groupBy(Function) @@ -8181,6 +8219,8 @@ See the <> for a high-level overview * List findResults(Function) * void {java11-javadoc}/java.base/java/lang/Iterable.html#forEach(java.util.function.Consumer)[forEach](Consumer) * Double get(int) +* Object getByPath(String) +* Object getByPath(String, Object) * int getLength() * double getValue() * Map groupBy(Function) @@ -8240,6 +8280,8 @@ See the <> for a high-level overview * double geohashDistance(String) * double geohashDistanceWithDefault(String, double) * GeoPoint get(int) +* Object getByPath(String) +* Object getByPath(String, Object) * double getLat() * double[] getLats() * int getLength() @@ -8301,6 +8343,8 @@ See the <> for a high-level overview * List findResults(Function) * void {java11-javadoc}/java.base/java/lang/Iterable.html#forEach(java.util.function.Consumer)[forEach](Consumer) * Long get(int) +* Object getByPath(String) +* Object getByPath(String, Object) * int getLength() * long getValue() * Map groupBy(Function) @@ -8356,6 +8400,8 @@ See the <> for a high-level overview * List findResults(Function) * void {java11-javadoc}/java.base/java/lang/Iterable.html#forEach(java.util.function.Consumer)[forEach](Consumer) * String get(int) +* Object getByPath(String) +* Object getByPath(String, Object) * int getLength() * String getValue() * Map groupBy(Function) @@ -8415,6 +8461,8 @@ See the <> for a high-level overview * List findResults(Function) * void {java11-javadoc}/java.base/java/lang/Iterable.html#forEach(java.util.function.Consumer)[forEach](Consumer) * String get(int) +* Object getByPath(String) +* Object getByPath(String, Object) * int getLength() * String getValue() * Map groupBy(Function) diff --git a/docs/painless/painless-guide/index.asciidoc b/docs/painless/painless-guide/index.asciidoc index b45406a4e7273..f7e5de693df79 100644 --- a/docs/painless/painless-guide/index.asciidoc +++ b/docs/painless/painless-guide/index.asciidoc @@ -1,7 +1,11 @@ include::painless-walkthrough.asciidoc[] +include::painless-datetime.asciidoc[] + include::painless-method-dispatch.asciidoc[] include::painless-debugging.asciidoc[] include::painless-execute-script.asciidoc[] + +include::../redirects.asciidoc[] diff --git a/docs/painless/painless-guide/painless-datetime.asciidoc b/docs/painless/painless-guide/painless-datetime.asciidoc new file mode 100644 index 0000000000000..45eb34b75726d --- /dev/null +++ b/docs/painless/painless-guide/painless-datetime.asciidoc @@ -0,0 +1,720 @@ +[[painless-datetime]] +=== Using Datetime in Painless + +==== Datetime API + +Datetimes in Painless use the standard Java libraries and are available through +the Painless <>. Most of the classes +from the following Java packages are available to use in Painless scripts: + +* <> +* <> +* <> +* <> +* <> + +==== Datetime Representation + +Datetimes in Painless are most commonly represented as a numeric value, a +string value, or a complex value. + +numeric:: a datetime representation as a number from a starting offset called +an epoch; in Painless this is typically a <> as +milliseconds since an epoch of 1970-01-01 00:00:00 Zulu Time +string:: a datetime representation as a sequence of characters defined by +a standard format or a custom format; in Painless this is typically a +<> of the standard format +https://en.wikipedia.org/wiki/ISO_8601[ISO 8601] +complex:: a datetime representation as a complex type +(<>) that abstracts away internal details of how the +datetime is stored and often provides utilities for modification and +comparison; in Painless this is typically a +<> + +Switching between different representations of datetimes is often necessary to +achieve a script's objective(s). A typical pattern in a script is to switch a +numeric or string datetime to a complex datetime, modify or compare the complex +datetime, and then switch it back to a numeric or string datetime for storage +or to return a result. + +==== Datetime Parsing and Formatting + +Datetime parsing is a switch from a string datetime to a complex datetime, and +datetime formatting is a switch from a complex datetime to a string datetime. + +A <> is a +complex type (<>) that defines the allowed sequence +of characters for a string datetime. Datetime parsing and formatting often +requires a DateTimeFormatter. For more information about how to use a +DateTimeFormatter see the +{java11-javadoc}/java.base/java/time/format/DateTimeFormatter.html[Java documentation]. + +===== Datetime Parsing Examples + +* parse from milliseconds ++ +[source,Painless] +---- +String milliSinceEpochString = "434931330000"; +long milliSinceEpoch = Long.parseLong(milliSinceEpochString); +Instant instant = Instant.ofEpochMilli(milliSinceEpoch); +ZonedDateTime zdt = ZonedDateTime.ofInstant(instant, ZoneId.of('Z')); +---- ++ +* parse from ISO 8601 ++ +[source,Painless] +---- +String datetime = '1983-10-13T22:15:30Z'; +ZonedDateTime zdt = ZonedDateTime.parse(datetime); <1> +---- +<1> Note the parse method uses ISO 8601 by default. ++ +* parse from RFC 1123 ++ +[source,Painless] +---- +String datetime = 'Thu, 13 Oct 1983 22:15:30 GMT'; +ZonedDateTime zdt = ZonedDateTime.parse(datetime, + DateTimeFormatter.RFC_1123_DATE_TIME); <1> +---- +<1> Note the use of a built-in DateTimeFormatter. ++ +* parse from a custom format ++ +[source,Painless] +---- +String datetime = 'custom y 1983 m 10 d 13 22:15:30 Z'; +DateTimeFormatter dtf = DateTimeFormatter.ofPattern( + "'custom' 'y' yyyy 'm' MM 'd' dd HH:mm:ss VV"); +ZonedDateTime zdt = ZonedDateTime.parse(datetime, dtf); <1> +---- +<1> Note the use of a custom DateTimeFormatter. + +===== Datetime Formatting Examples + +* format to ISO 8601 ++ +[source,Painless] +---- +ZonedDateTime zdt = + ZonedDateTime.of(1983, 10, 13, 22, 15, 30, 0, ZoneId.of('Z')); +String datetime = zdt.format(DateTimeFormatter.ISO_INSTANT); <1> +---- +<1> Note the use of a built-in DateTimeFormatter. ++ +* format to a custom format ++ +[source,Painless] +---- +ZonedDateTime zdt = + ZonedDateTime.of(1983, 10, 13, 22, 15, 30, 0, ZoneId.of('Z')); +DateTimeFormatter dtf = DateTimeFormatter.ofPattern( + "'date:' yyyy/MM/dd 'time:' HH:mm:ss"); +String datetime = zdt.format(dtf); <1> +---- +<1> Note the use of a custom DateTimeFormatter. + +==== Datetime Conversion + +Datetime conversion is a switch from a numeric datetime to a complex datetime +and vice versa. + +===== Datetime Conversion Examples + +* convert from milliseconds ++ +[source,Painless] +---- +long milliSinceEpoch = 434931330000L; +Instant instant = Instant.ofEpochMilli(milliSinceEpoch); +ZonedDateTime zdt = ZonedDateTime.ofInstant(instant, ZoneId.of('Z')); +---- ++ +* convert to milliseconds ++ +[source,Painless] +----- +ZonedDateTime zdt = + ZonedDateTime.of(1983, 10, 13, 22, 15, 30, 0, ZoneId.of('Z')); +long milliSinceEpoch = zdt.toInstant().toEpochMilli(); +----- + +==== Datetime Pieces + +Datetime representations often contain the data to extract individual datetime +pieces such as year, hour, timezone, etc. Use individual pieces of a datetime +to create a complex datetime, and use a complex datetime to extract individual +pieces. + +===== Datetime Pieces Examples + +* create a complex datetime from pieces ++ +[source,Painless] +---- +int year = 1983; +int month = 10; +int day = 13; +int hour = 22; +int minutes = 15; +int seconds = 30; +int nanos = 0; +ZonedDateTime zdt = ZonedDateTime.of( + year, month, day, hour, minutes, seconds, nanos, ZoneId.of('Z')); +---- ++ +* extract pieces from a complex datetime ++ +[source,Painless] +---- +ZonedDateTime zdt = + ZonedDateTime.of(1983, 10, 13, 22, 15, 30, 100, ZoneId.of(tz)); +int year = zdt.getYear(); +int month = zdt.getMonthValue(); +int day = zdt.getDayOfMonth(); +int hour = zdt.getHour(); +int minutes = zdt.getMinute(); +int seconds = zdt.getSecond(); +int nanos = zdt.getNano(); +---- + +==== Datetime Modification + +Use either a numeric datetime or a complex datetime to do modification such as +adding several seconds to a datetime or subtracting several days from a +datetime. Use standard <> to +modify a numeric datetime. Use +<> (or fields) to modify +a complex datetime. Note many complex datetimes are immutable so upon +modification a new complex datetime is created that requires +<> or immediate use. + +===== Datetime Modification Examples + +* Subtract three seconds from a numeric datetime in milliseconds ++ +[source,Painless] +---- +long milliSinceEpoch = 434931330000L; +milliSinceEpoch = milliSinceEpoch - 1000L*3L; +---- ++ +* Add three days to a complex datetime ++ +[source,Painless] +---- +ZonedDateTime zdt = + ZonedDateTime.of(1983, 10, 13, 22, 15, 30, 0, ZoneId.of('Z')); +ZonedDateTime updatedZdt = zdt.plusDays(3); +---- ++ +* Subtract 125 minutes from a complex datetime ++ +[source,Painless] +---- +ZonedDateTime zdt = + ZonedDateTime.of(1983, 10, 13, 22, 15, 30, 0, ZoneId.of('Z')); +ZonedDateTime updatedZdt = zdt.minusMinutes(125); +---- ++ +* Set the year on a complex datetime ++ +[source,Painless] +---- +ZonedDateTime zdt = + ZonedDateTime.of(1983, 10, 13, 22, 15, 30, 0, ZoneId.of('Z')); +ZonedDateTime updatedZdt = zdt.withYear(1976); +---- + +==== Datetime Difference (Elapsed Time) + +Use either two numeric datetimes or two complex datetimes to calculate the +difference (elapsed time) between two different datetimes. Use +<> to calculate the difference between +between two numeric datetimes of the same time unit such as milliseconds. For +complex datetimes there is often a method or another complex type +(<>) available to calculate the difference. Use +<> +to calculate the difference between two complex datetimes if supported. + +===== Datetime Difference Examples + +* Difference in milliseconds between two numeric datetimes ++ +[source,Painless] +---- +long startTimestamp = 434931327000L; +long endTimestamp = 434931330000L; +long differenceInMillis = endTimestamp - startTimestamp; +---- ++ +* Difference in milliseconds between two complex datetimes ++ +[source,Painless] +---- +ZonedDateTime zdt1 = + ZonedDateTime.of(1983, 10, 13, 22, 15, 30, 11000000, ZoneId.of('Z')); +ZonedDateTime zdt2 = + ZonedDateTime.of(1983, 10, 13, 22, 15, 35, 0, ZoneId.of('Z')); +long differenceInMillis = ChronoUnit.MILLIS.between(zdt1, zdt2); +---- ++ +* Difference in days between two complex datetimes ++ +[source,Painless] +---- +ZonedDateTime zdt1 = + ZonedDateTime.of(1983, 10, 13, 22, 15, 30, 11000000, ZoneId.of('Z')); +ZonedDateTime zdt2 = + ZonedDateTime.of(1983, 10, 17, 22, 15, 35, 0, ZoneId.of('Z')); +long differenceInDays = ChronoUnit.DAYS.between(zdt1, zdt2); +---- + +==== Datetime Comparison + +Use either two numeric datetimes or two complex datetimes to do a datetime +comparison. Use standard <> +to compare two numeric datetimes of the same time unit such as milliseconds. +For complex datetimes there is often a method or another complex type +(<>) available to do the comparison. + +===== Datetime Comparison Examples + +* Greater than comparison of two numeric datetimes in milliseconds ++ +[source,Painless] +---- +long timestamp1 = 434931327000L; +long timestamp2 = 434931330000L; + +if (timestamp1 > timestamp2) { + // handle condition +} +---- ++ +* Equality comparision of two complex datetimes ++ +[source,Painless] +---- +ZonedDateTime zdt1 = + ZonedDateTime.of(1983, 10, 13, 22, 15, 30, 0, ZoneId.of('Z')); +ZonedDateTime zdt2 = + ZonedDateTime.of(1983, 10, 13, 22, 15, 30, 0, ZoneId.of('Z')); + +if (zdt1.equals(zdt2)) { + // handle condition +} +---- ++ +* Less than comparision of two complex datetimes ++ +[source,Painless] +---- +ZonedDateTime zdt1 = + ZonedDateTime.of(1983, 10, 13, 22, 15, 30, 0, ZoneId.of('Z')); +ZonedDateTime zdt2 = + ZonedDateTime.of(1983, 10, 17, 22, 15, 35, 0, ZoneId.of('Z')); + +if (zdt1.isBefore(zdt2)) { + // handle condition +} +---- ++ +* Greater than comparision of two complex datetimes ++ +[source,Painless] +---- +ZonedDateTime zdt1 = + ZonedDateTime.of(1983, 10, 13, 22, 15, 30, 0, ZoneId.of('Z')); +ZonedDateTime zdt2 = + ZonedDateTime.of(1983, 10, 17, 22, 15, 35, 0, ZoneId.of('Z')); + +if (zdt1.isAfter(zdt2)) { + // handle condition +} +---- + +==== Datetime Zone + +Both string datetimes and complex datetimes have a timezone with a default of +`UTC`. Numeric datetimes do not have enough explicit information to +have a timezone, so `UTC` is always assumed. Use +<> (or fields) in +conjunction with a <> to change +the timezone for a complex datetime. Parse a string datetime into a complex +datetime to change the timezone, and then format the complex datetime back into +a desired string datetime. Note many complex datetimes are immutable so upon +modification a new complex datetime is created that requires +<> or immediate use. + +===== Datetime Zone Examples + +* Modify the timezone for a complex datetime ++ +[source,Painless] +---- +ZonedDateTime utc = + ZonedDateTime.of(1983, 10, 13, 22, 15, 30, 0, ZoneId.of('Z')); +ZonedDateTime pst = utc.withZoneSameInstant(ZoneId.of('America/Los_Angeles')); +---- ++ +* Modify the timezone for a string datetime ++ +[source,Painless] +---- +String gmtString = 'Thu, 13 Oct 1983 22:15:30 GMT'; +ZonedDateTime gmtZdt = ZonedDateTime.parse(gmtString, + DateTimeFormatter.RFC_1123_DATE_TIME); <1> +ZonedDateTime pstZdt = + gmtZdt.withZoneSameInstant(ZoneId.of('America/Los_Angeles')); +String pstString = pstZdt.format(DateTimeFormatter.RFC_1123_DATE_TIME); +---- +<1> Note the use of a built-in DateTimeFormatter. + +==== Datetime Input + +There are several common ways datetimes are used as input for a script +determined by the <>. Typically, datetime +input will be accessed from parameters specified by the user, from an original +source document, or from an indexed document. + +===== Datetime Input From User Parameters + +Use the {ref}/modules-scripting-using.html#_script_parameters[params section] +during script specification to pass in a numeric datetime or string datetime as +a script input. Access to user-defined parameters within a script is dependent +on the Painless context, though, the parameters are most commonly accessible +through an input called `params`. + +*Examples* + +* Parse a numeric datetime from user parameters to a complex datetime ++ +** Input: ++ +[source,JSON] +---- +... +"script": { + ... + "params": { + "input_datetime": 434931327000 + } +} +... +---- ++ +** Script: ++ +[source,Painless] +---- +long inputDateTime = params['input_datetime']; +Instant instant = Instant.ofEpochMilli(inputDateTime); +ZonedDateTime zdt = ZonedDateTime.ofInstant(instant, ZoneId.of('Z')); +---- ++ +* Parse a string datetime from user parameters to a complex datetime ++ +** Input: ++ +[source,JSON] +---- +... +"script": { + ... + "params": { + "input_datetime": "custom y 1983 m 10 d 13 22:15:30 Z" + } +} +... +---- ++ +** Script: ++ +[source,Painless] +---- +String datetime = params['input_datetime']; +DateTimeFormatter dtf = DateTimeFormatter.ofPattern( + "'custom' 'y' yyyy 'm' MM 'd' dd HH:mm:ss VV"); +ZonedDateTime zdt = ZonedDateTime.parse(datetime, dtf); <1> +---- +<1> Note the use of a custom DateTimeFormatter. + +===== Datetime Input From a Source Document + +Use an original {ref}/mapping-source-field.html[source] document as a script +input to access a numeric datetime or string datetime for a specific field +within that document. Access to an original source document within a script is +dependent on the Painless context and is not always available. An original +source document is most commonly accessible through an input called +`ctx['_source']` or `params['_source']`. + +*Examples* + +* Parse a numeric datetime from a sourced document to a complex datetime ++ +** Input: ++ +[source,JSON] +---- +{ + ... + "input_datetime": 434931327000 + ... +} +---- ++ +** Script: ++ +[source,Painless] +---- +long inputDateTime = ctx['_source']['input_datetime']; <1> +Instant instant = Instant.ofEpochMilli(inputDateTime); +ZonedDateTime zdt = ZonedDateTime.ofInstant(instant, ZoneId.of('Z')); +---- +<1> Note access to `_source` is dependent on the Painless context. ++ +* Parse a string datetime from a sourced document to a complex datetime ++ +** Input: ++ +[source,JSON] +---- +{ + ... + "input_datetime": "1983-10-13T22:15:30Z" + ... +} +---- ++ +** Script: ++ +[source,Painless] +---- +String datetime = params['_source']['input_datetime']; <1> +ZonedDateTime zdt = ZonedDateTime.parse(datetime); <2> +---- +<1> Note access to `_source` is dependent on the Painless context. +<2> Note the parse method uses ISO 8601 by default. + +===== Datetime Input From an Indexed Document + +Use an indexed document as a script input to access a complex datetime for a +specific field within that document where the field is mapped as a +{ref}/date.html[standard date] or a {ref}/date_nanos.html[nanosecond date]. +Numeric datetime fields mapped as {ref}/number.html[numeric] and string +datetime fields mapped as {ref}/keyword.html[keyword] are accessible through an +indexed document as well. Access to an indexed document within a script is +dependent on the Painless context and is not always available. An indexed +document is most commonly accessible through an input called `doc`. + +*Examples* + +* Format a complex datetime from an indexed document to a string datetime ++ +** Assumptions: ++ +*** The field `input_datetime` exists in all indexes as part of the query +*** All indexed documents contain the field `input_datetime` ++ +** Mappings: ++ +[source,JSON] +---- +{ + "mappings": { + ... + "properties": { + ... + "input_datetime": { + "type": "date" + } + ... + } + ... + } +} +---- ++ +** Script: ++ +[source,Painless] +---- +def input = doc['input_datetime'].value; +String output = input.format(DateTimeFormatter.ISO_INSTANT); <1> +---- +<1> Note the use of a built-in DateTimeFormatter. ++ +* Find the difference between two complex datetimes from an indexed document ++ +** Assumptions: ++ +*** The fields `start` and `end` may *not* exist in all indexes as part of the +query +*** The fields `start` and `end` may *not* have values in all indexed documents ++ +** Mappings: ++ +[source,JSON] +---- +{ + "mappings": { + ... + "properties": { + ... + "start": { + "type": "date" + }, + "end": { + "type": "date" + } + ... + } + ... + } +} +---- ++ +** Script: ++ +[source,Painless] +---- +if (doc.containsKey('start') && doc.containsKey('end')) { <1> + + if (doc['start'].size() > 0 && doc['end'].size() > 0) { <2> + + def start = doc['start'].value; + def end = doc['end'].value; + long differenceInMillis = ChronoUnit.MILLIS.between(start, end); + + // handle difference in times + } else { + // handle fields without values + } +} else { + // handle index with missing fields +} +---- +<1> When a query's results span multiple indexes, some indexes may not +contain a specific field. Use the `containsKey` method call on the `doc` input +to ensure a field exists as part of the index for the current document. +<2> Some fields within a document may have no values. Use the `size` method +call on a field within the `doc` input to ensure that field has at least one +value for the current document. + +==== Datetime Now + +Under most Painless contexts the current datetime, `now`, is not supported. +There are two primary reasons for this. The first is scripts are often run once +per document, so each time the script is run a different `now` is returned. The +second is scripts are often run in a distributed fashion without a way to +appropriately synchronize `now`. Instead, pass in a user-defined parameter with +either a string datetime or numeric datetime for `now`. A numeric datetime is +preferred as there is no need to parse it for comparision. + +===== Datetime Now Examples + +* Use a numeric datetime as `now` ++ +** Assumptions: ++ +*** The field `input_datetime` exists in all indexes as part of the query +*** All indexed documents contain the field `input_datetime` ++ +** Mappings: ++ +[source,JSON] +---- +{ + "mappings": { + ... + "properties": { + ... + "input_datetime": { + "type": "date" + } + ... + } + ... + } +} +---- ++ +** Input: ++ +[source,JSON] +---- +... +"script": { + ... + "params": { + "now": + } +} +... +---- ++ +** Script: ++ +[source,Painless] +---- +long now = params['now']; +def inputDateTime = doc['input_datetime']; +long millisDateTime = zdt.toInstant().toEpochMilli(); +long elapsedTime = now - millisDateTime; +---- ++ +* Use a string datetime as `now` ++ +** Assumptions: ++ +*** The field `input_datetime` exists in all indexes as part of the query +*** All indexed documents contain the field `input_datetime` ++ +** Mappings: ++ +[source,JSON] +---- +{ + "mappings": { + ... + "properties": { + ... + "input_datetime": { + "type": "date" + } + ... + } + ... + } +} +---- ++ +** Input: ++ +[source,JSON] +---- +... +"script": { + ... + "params": { + "now": "" + } +} +... +---- ++ +** Script: ++ +[source,Painless] +---- +String nowString = params['now']; +ZonedDateTime nowZdt = ZonedDateTime.parse(datetime); <1> +long now = ZonedDateTime.toInstant().toEpochMilli(); +def inputDateTime = doc['input_datetime']; +long millisDateTime = zdt.toInstant().toEpochMilli(); +long elapsedTime = now - millisDateTime; +---- +<1> Note this parses the same string datetime every time the script runs. Use a +numeric datetime to avoid a significant performance hit. diff --git a/docs/painless/redirects.asciidoc b/docs/painless/redirects.asciidoc new file mode 100644 index 0000000000000..94dd5524e9acd --- /dev/null +++ b/docs/painless/redirects.asciidoc @@ -0,0 +1,9 @@ +["appendix",role="exclude",id="redirects"] += Deleted pages + +The following pages have moved or been deleted. + +[role="exclude",id="painless-examples"] +=== Painless examples + +See <>. \ No newline at end of file diff --git a/docs/plugins/analysis-phonetic.asciidoc b/docs/plugins/analysis-phonetic.asciidoc index e22f819e1eb3e..3627751670a32 100644 --- a/docs/plugins/analysis-phonetic.asciidoc +++ b/docs/plugins/analysis-phonetic.asciidoc @@ -65,6 +65,14 @@ GET phonetic_sample/_analyze <1> Returns: `J`, `joe`, `BLKS`, `bloggs` +It is important to note that `"replace": false` can lead to unexpected behavior since +the original and the phonetically analyzed version are both kept at the same token position. +Some queries handle these stacked tokens in special ways. For example, the fuzzy `match` +query does not apply {ref}/common-options.html#fuzziness[fuzziness] to stacked synonym tokens. +This can lead to issues that are difficult to diagnose and reason about. For this reason, it +is often beneficial to use separate fields for analysis with and without phonetic filtering. +That way searches can be run against both fields with differing boosts and trade-offs (e.g. +only run a fuzzy `match` query on the original text field, but not on the phonetic version). [float] ===== Double metaphone settings diff --git a/docs/plugins/analysis-smartcn.asciidoc b/docs/plugins/analysis-smartcn.asciidoc index 4c019ebf2732c..3786cbc5e2b8b 100644 --- a/docs/plugins/analysis-smartcn.asciidoc +++ b/docs/plugins/analysis-smartcn.asciidoc @@ -17,7 +17,415 @@ include::install_remove.asciidoc[] [float] ==== `smartcn` tokenizer and token filter -The plugin provides the `smartcn` analyzer and `smartcn_tokenizer` tokenizer, -which are not configurable. +The plugin provides the `smartcn` analyzer, `smartcn_tokenizer` tokenizer, and +`smartcn_stop` token filter which are not configurable. NOTE: The `smartcn_word` token filter and `smartcn_sentence` have been deprecated. + +==== Reimplementing and extending the analyzers + +The `smartcn` analyzer could be reimplemented as a `custom` analyzer that can +then be extended and configured as follows: + +[source,js] +---------------------------------------------------- +PUT smartcn_example +{ + "settings": { + "analysis": { + "analyzer": { + "rebuilt_smartcn": { + "tokenizer": "smartcn_tokenizer", + "filter": [ + "porter_stem", + "smartcn_stop" + ] + } + } + } + } +} +---------------------------------------------------- +// CONSOLE +// TEST[s/\n$/\nstartyaml\n - compare_analyzers: {index: smartcn_example, first: smartcn, second: rebuilt_smartcn}\nendyaml\n/] + +[[analysis-smartcn_stop]] +==== `smartcn_stop` token filter + +The `smartcn_stop` token filter filters out stopwords defined by `smartcn` +analyzer (`_smartcn_`), and any other custom stopwords specified by the user. +This filter only supports the predefined `_smartcn_` stopwords list. +If you want to use a different predefined list, then use the +{ref}/analysis-stop-tokenfilter.html[`stop` token filter] instead. + +[source,js] +-------------------------------------------------- +PUT smartcn_example +{ + "settings": { + "index": { + "analysis": { + "analyzer": { + "smartcn_with_stop": { + "tokenizer": "smartcn_tokenizer", + "filter": [ + "porter_stem", + "my_smartcn_stop" + ] + } + }, + "filter": { + "my_smartcn_stop": { + "type": "smartcn_stop", + "stopwords": [ + "_smartcn_", + "stack", + "的" + ] + } + } + } + } + } +} + +GET smartcn_example/_analyze +{ + "analyzer": "smartcn_with_stop", + "text": "哈喽,我们是 Elastic 我们是 Elastic Stack(Elasticsearch、Kibana、Beats 和 Logstash)的开发公司。从股票行情到 Twitter 消息流,从 Apache 日志到 WordPress 博文,我们可以帮助人们体验搜索的强大力量,帮助他们以截然不同的方式探索和分析数据" +} +-------------------------------------------------- +// CONSOLE + +The above request returns: + +[source,js] +-------------------------------------------------- +{ + "tokens": [ + { + "token": "哈", + "start_offset": 0, + "end_offset": 1, + "type": "word", + "position": 0 + }, + { + "token": "喽", + "start_offset": 1, + "end_offset": 2, + "type": "word", + "position": 1 + }, + { + "token": "我们", + "start_offset": 3, + "end_offset": 5, + "type": "word", + "position": 3 + }, + { + "token": "是", + "start_offset": 5, + "end_offset": 6, + "type": "word", + "position": 4 + }, + { + "token": "elast", + "start_offset": 7, + "end_offset": 14, + "type": "word", + "position": 5 + }, + { + "token": "我们", + "start_offset": 17, + "end_offset": 19, + "type": "word", + "position": 6 + }, + { + "token": "是", + "start_offset": 19, + "end_offset": 20, + "type": "word", + "position": 7 + }, + { + "token": "elast", + "start_offset": 21, + "end_offset": 28, + "type": "word", + "position": 8 + }, + { + "token": "elasticsearch", + "start_offset": 35, + "end_offset": 48, + "type": "word", + "position": 11 + }, + { + "token": "kibana", + "start_offset": 49, + "end_offset": 55, + "type": "word", + "position": 13 + }, + { + "token": "beat", + "start_offset": 56, + "end_offset": 61, + "type": "word", + "position": 15 + }, + { + "token": "和", + "start_offset": 62, + "end_offset": 63, + "type": "word", + "position": 16 + }, + { + "token": "logstash", + "start_offset": 64, + "end_offset": 72, + "type": "word", + "position": 17 + }, + { + "token": "开发", + "start_offset": 74, + "end_offset": 76, + "type": "word", + "position": 20 + }, + { + "token": "公司", + "start_offset": 76, + "end_offset": 78, + "type": "word", + "position": 21 + }, + { + "token": "从", + "start_offset": 79, + "end_offset": 80, + "type": "word", + "position": 23 + }, + { + "token": "股票", + "start_offset": 80, + "end_offset": 82, + "type": "word", + "position": 24 + }, + { + "token": "行情", + "start_offset": 82, + "end_offset": 84, + "type": "word", + "position": 25 + }, + { + "token": "到", + "start_offset": 84, + "end_offset": 85, + "type": "word", + "position": 26 + }, + { + "token": "twitter", + "start_offset": 86, + "end_offset": 93, + "type": "word", + "position": 27 + }, + { + "token": "消息", + "start_offset": 94, + "end_offset": 96, + "type": "word", + "position": 28 + }, + { + "token": "流", + "start_offset": 96, + "end_offset": 97, + "type": "word", + "position": 29 + }, + { + "token": "从", + "start_offset": 98, + "end_offset": 99, + "type": "word", + "position": 31 + }, + { + "token": "apach", + "start_offset": 100, + "end_offset": 106, + "type": "word", + "position": 32 + }, + { + "token": "日志", + "start_offset": 107, + "end_offset": 109, + "type": "word", + "position": 33 + }, + { + "token": "到", + "start_offset": 109, + "end_offset": 110, + "type": "word", + "position": 34 + }, + { + "token": "wordpress", + "start_offset": 111, + "end_offset": 120, + "type": "word", + "position": 35 + }, + { + "token": "博", + "start_offset": 121, + "end_offset": 122, + "type": "word", + "position": 36 + }, + { + "token": "文", + "start_offset": 122, + "end_offset": 123, + "type": "word", + "position": 37 + }, + { + "token": "我们", + "start_offset": 124, + "end_offset": 126, + "type": "word", + "position": 39 + }, + { + "token": "可以", + "start_offset": 126, + "end_offset": 128, + "type": "word", + "position": 40 + }, + { + "token": "帮助", + "start_offset": 128, + "end_offset": 130, + "type": "word", + "position": 41 + }, + { + "token": "人们", + "start_offset": 130, + "end_offset": 132, + "type": "word", + "position": 42 + }, + { + "token": "体验", + "start_offset": 132, + "end_offset": 134, + "type": "word", + "position": 43 + }, + { + "token": "搜索", + "start_offset": 134, + "end_offset": 136, + "type": "word", + "position": 44 + }, + { + "token": "强大", + "start_offset": 137, + "end_offset": 139, + "type": "word", + "position": 46 + }, + { + "token": "力量", + "start_offset": 139, + "end_offset": 141, + "type": "word", + "position": 47 + }, + { + "token": "帮助", + "start_offset": 142, + "end_offset": 144, + "type": "word", + "position": 49 + }, + { + "token": "他们", + "start_offset": 144, + "end_offset": 146, + "type": "word", + "position": 50 + }, + { + "token": "以", + "start_offset": 146, + "end_offset": 147, + "type": "word", + "position": 51 + }, + { + "token": "截然不同", + "start_offset": 147, + "end_offset": 151, + "type": "word", + "position": 52 + }, + { + "token": "方式", + "start_offset": 152, + "end_offset": 154, + "type": "word", + "position": 54 + }, + { + "token": "探索", + "start_offset": 154, + "end_offset": 156, + "type": "word", + "position": 55 + }, + { + "token": "和", + "start_offset": 156, + "end_offset": 157, + "type": "word", + "position": 56 + }, + { + "token": "分析", + "start_offset": 157, + "end_offset": 159, + "type": "word", + "position": 57 + }, + { + "token": "数据", + "start_offset": 159, + "end_offset": 161, + "type": "word", + "position": 58 + } + ] +} +-------------------------------------------------- +// TESTRESPONSE \ No newline at end of file diff --git a/docs/plugins/analysis-stempel.asciidoc b/docs/plugins/analysis-stempel.asciidoc index a5526129a97d3..cd234c8d42756 100644 --- a/docs/plugins/analysis-stempel.asciidoc +++ b/docs/plugins/analysis-stempel.asciidoc @@ -12,7 +12,107 @@ include::install_remove.asciidoc[] [[analysis-stempel-tokenizer]] [float] -==== `stempel` tokenizer and token filter +==== `stempel` tokenizer and token filters -The plugin provides the `polish` analyzer and `polish_stem` token filter, +The plugin provides the `polish` analyzer and the `polish_stem` and `polish_stop` token filters, which are not configurable. + +==== Reimplementing and extending the analyzers + +The `polish` analyzer could be reimplemented as a `custom` analyzer that can +then be extended and configured differently as follows: + +[source,js] +---------------------------------------------------- +PUT /stempel_example +{ + "settings": { + "analysis": { + "analyzer": { + "rebuilt_stempel": { + "tokenizer": "standard", + "filter": [ + "lowercase", + "polish_stop", + "polish_stem" + ] + } + } + } + } +} +---------------------------------------------------- +// CONSOLE +// TEST[s/\n$/\nstartyaml\n - compare_analyzers: {index: stempel_example, first: polish, second: rebuilt_stempel}\nendyaml\n/] + +[[analysis-polish-stop]] +==== `polish_stop` token filter + +The `polish_stop` token filter filters out Polish stopwords (`_polish_`), and +any other custom stopwords specified by the user. This filter only supports +the predefined `_polish_` stopwords list. If you want to use a different +predefined list, then use the +{ref}/analysis-stop-tokenfilter.html[`stop` token filter] instead. + +[source,js] +-------------------------------------------------- +PUT /polish_stop_example +{ + "settings": { + "index": { + "analysis": { + "analyzer": { + "analyzer_with_stop": { + "tokenizer": "standard", + "filter": [ + "lowercase", + "polish_stop" + ] + } + }, + "filter": { + "polish_stop": { + "type": "polish_stop", + "stopwords": [ + "_polish_", + "jeść" + ] + } + } + } + } + } +} + +GET polish_stop_example/_analyze +{ + "analyzer": "analyzer_with_stop", + "text": "Gdzie kucharek sześć, tam nie ma co jeść." +} +-------------------------------------------------- +// CONSOLE + +The above request returns: + +[source,js] +-------------------------------------------------- +{ + "tokens" : [ + { + "token" : "kucharek", + "start_offset" : 6, + "end_offset" : 14, + "type" : "", + "position" : 1 + }, + { + "token" : "sześć", + "start_offset" : 15, + "end_offset" : 20, + "type" : "", + "position" : 2 + } + ] +} +-------------------------------------------------- +// TESTRESPONSE diff --git a/docs/plugins/api.asciidoc b/docs/plugins/api.asciidoc index 74fbba25810d8..7eeba28b22266 100644 --- a/docs/plugins/api.asciidoc +++ b/docs/plugins/api.asciidoc @@ -23,9 +23,6 @@ A number of plugins have been contributed by our community: * https://github.com/zentity-io/zentity[Entity Resolution Plugin] (https://zentity.io[zentity]): Real-time entity resolution with pure Elasticsearch (by Dave Moore) -* https://github.com/NLPchina/elasticsearch-sql/[SQL language Plugin]: - Allows Elasticsearch to be queried with SQL (by nlpcn) - * https://github.com/ritesh-kapoor/elasticsearch-pql[PQL language Plugin]: Allows Elasticsearch to be queried with simple pipeline query syntax. diff --git a/docs/plugins/repository-azure.asciidoc b/docs/plugins/repository-azure.asciidoc index 61dcadd6e10d6..4269cd4f5a524 100644 --- a/docs/plugins/repository-azure.asciidoc +++ b/docs/plugins/repository-azure.asciidoc @@ -19,7 +19,11 @@ bin/elasticsearch-keystore add azure.client.default.account bin/elasticsearch-keystore add azure.client.default.key ---------------------------------------------------------------- -Where `account` is the azure account name and `key` the azure secret key. +Where `account` is the azure account name and `key` the azure secret key. Instead of an azure secret key under `key`, you can alternatively +define a shared access signatures (SAS) token under `sas_token` to use for authentication instead. When using an SAS token instead of an +account key, the SAS token must have read (r), write (w), list (l), and delete (d) permissions for the repository base path and +all its contents. These permissions need to be granted for the blob service (b) and apply to resource types service (s), container (c), and +object (o). These settings are used by the repository's internal azure client. Note that you can also define more than one account: @@ -29,14 +33,14 @@ Note that you can also define more than one account: bin/elasticsearch-keystore add azure.client.default.account bin/elasticsearch-keystore add azure.client.default.key bin/elasticsearch-keystore add azure.client.secondary.account -bin/elasticsearch-keystore add azure.client.secondary.key +bin/elasticsearch-keystore add azure.client.secondary.sas_token ---------------------------------------------------------------- `default` is the default account name which will be used by a repository, unless you set an explicit one in the <>. -Both `account` and `key` storage settings are +The `account`, `key`, and `sas_token` storage settings are {ref}/secure-settings.html#reloadable-secure-settings[reloadable]. After you reload the settings, the internal azure clients, which are used to transfer the snapshot, will utilize the latest settings from the keystore. diff --git a/docs/reference/administering.asciidoc b/docs/reference/administering.asciidoc new file mode 100644 index 0000000000000..7170185c6ef77 --- /dev/null +++ b/docs/reference/administering.asciidoc @@ -0,0 +1,25 @@ +[[administer-elasticsearch]] += Administering {es} + +[partintro] +-- +Elasticsearch is a complex piece of software, with many moving parts. There are +many APIs and features that are designed to help you manage your Elasticsearch +cluster. + +-- + +[[backup-cluster]] +== Back up a cluster + +As with any software that stores data, it is important to routinely back up your +data. {es} replicas provide high availability during runtime; they enable you to +tolerate sporadic node loss without an interruption of service. + +Replicas do not provide protection from catastrophic failure, however. For that, +you need a real backup of your cluster—a complete copy in case something goes +wrong. + +To back up your cluster, you can use the <>. + +include::{es-repo-dir}/modules/snapshots.asciidoc[tag=snapshot-intro] diff --git a/docs/reference/aggregations/bucket.asciidoc b/docs/reference/aggregations/bucket.asciidoc index b9fbddc65c125..9f186ef1ffbaa 100644 --- a/docs/reference/aggregations/bucket.asciidoc +++ b/docs/reference/aggregations/bucket.asciidoc @@ -55,6 +55,8 @@ include::bucket/parent-aggregation.asciidoc[] include::bucket/range-aggregation.asciidoc[] +include::bucket/rare-terms-aggregation.asciidoc[] + include::bucket/reverse-nested-aggregation.asciidoc[] include::bucket/sampler-aggregation.asciidoc[] @@ -64,3 +66,4 @@ include::bucket/significantterms-aggregation.asciidoc[] include::bucket/significanttext-aggregation.asciidoc[] include::bucket/terms-aggregation.asciidoc[] + diff --git a/docs/reference/aggregations/bucket/autodatehistogram-aggregation.asciidoc b/docs/reference/aggregations/bucket/autodatehistogram-aggregation.asciidoc index dfc4f62a91b09..c09d1a67bf253 100644 --- a/docs/reference/aggregations/bucket/autodatehistogram-aggregation.asciidoc +++ b/docs/reference/aggregations/bucket/autodatehistogram-aggregation.asciidoc @@ -258,6 +258,39 @@ Like with the normal <>, +actually ordering a `terms` agg by count ascending has unbounded error. Instead, you should use the `rare_terms` +aggregation + +////////////////////////// + +[source,js] +-------------------------------------------------- +PUT /products +{ + "mappings": { + "properties": { + "genre": { + "type": "keyword" + }, + "product": { + "type": "keyword" + } + } + } +} + +POST /products/_doc/_bulk?refresh +{"index":{"_id":0}} +{"genre": "rock", "product": "Product A"} +{"index":{"_id":1}} +{"genre": "rock"} +{"index":{"_id":2}} +{"genre": "rock"} +{"index":{"_id":3}} +{"genre": "jazz", "product": "Product Z"} +{"index":{"_id":4}} +{"genre": "jazz"} +{"index":{"_id":5}} +{"genre": "electronic"} +{"index":{"_id":6}} +{"genre": "electronic"} +{"index":{"_id":7}} +{"genre": "electronic"} +{"index":{"_id":8}} +{"genre": "electronic"} +{"index":{"_id":9}} +{"genre": "electronic"} +{"index":{"_id":10}} +{"genre": "swing"} + +------------------------------------------------- +// NOTCONSOLE +// TESTSETUP + +////////////////////////// + +==== Syntax + +A `rare_terms` aggregation looks like this in isolation: + +[source,js] +-------------------------------------------------- +{ + "rare_terms": { + "field": "the_field", + "max_doc_count": 1 + } +} +-------------------------------------------------- +// NOTCONSOLE + +.`rare_terms` Parameters +|=== +|Parameter Name |Description |Required |Default Value +|`field` |The field we wish to find rare terms in |Required | +|`max_doc_count` |The maximum number of documents a term should appear in. |Optional |`1` +|`precision` |The precision of the internal CuckooFilters. Smaller precision leads to +better approximation, but higher memory usage. Cannot be smaller than `0.00001` |Optional |`0.01` +|`include` |Terms that should be included in the aggregation|Optional | +|`exclude` |Terms that should be excluded from the aggregation|Optional | +|`missing` |The value that should be used if a document does not have the field being aggregated|Optional | +|=== + + +Example: + +[source,js] +-------------------------------------------------- +GET /_search +{ + "aggs" : { + "genres" : { + "rare_terms" : { + "field" : "genre" + } + } + } +} +-------------------------------------------------- +// CONSOLE +// TEST[s/_search/_search\?filter_path=aggregations/] + +Response: + +[source,js] +-------------------------------------------------- +{ + ... + "aggregations" : { + "genres" : { + "buckets" : [ + { + "key" : "swing", + "doc_count" : 1 + } + ] + } + } +} +-------------------------------------------------- +// TESTRESPONSE[s/\.\.\.//] + +In this example, the only bucket that we see is the "swing" bucket, because it is the only term that appears in +one document. If we increase the `max_doc_count` to `2`, we'll see some more buckets: + +[source,js] +-------------------------------------------------- +GET /_search +{ + "aggs" : { + "genres" : { + "rare_terms" : { + "field" : "genre", + "max_doc_count": 2 + } + } + } +} +-------------------------------------------------- +// CONSOLE +// TEST[s/_search/_search\?filter_path=aggregations/] + +This now shows the "jazz" term which has a `doc_count` of 2": + +[source,js] +-------------------------------------------------- +{ + ... + "aggregations" : { + "genres" : { + "buckets" : [ + { + "key" : "swing", + "doc_count" : 1 + }, + { + "key" : "jazz", + "doc_count" : 2 + } + ] + } + } +} +-------------------------------------------------- +// TESTRESPONSE[s/\.\.\.//] + +[[search-aggregations-bucket-rare-terms-aggregation-max-doc-count]] +==== Maximum document count + +The `max_doc_count` parameter is used to control the upper bound of document counts that a term can have. There +is not a size limitation on the `rare_terms` agg like `terms` agg has. This means that terms +which match the `max_doc_count` criteria will be returned. The aggregation functions in this manner to avoid +the order-by-ascending issues that afflict the `terms` aggregation. + +This does, however, mean that a large number of results can be returned if chosen incorrectly. +To limit the danger of this setting, the maximum `max_doc_count` is 100. + +[[search-aggregations-bucket-rare-terms-aggregation-max-buckets]] +==== Max Bucket Limit + +The Rare Terms aggregation is more liable to trip the `search.max_buckets` soft limit than other aggregations due +to how it works. The `max_bucket` soft-limit is evaluated on a per-shard basis while the aggregation is collecting +results. It is possible for a term to be "rare" on a shard but become "not rare" once all the shard results are +merged together. This means that individual shards tend to collect more buckets than are truly rare, because +they only have their own local view. This list is ultimately pruned to the correct, smaller list of rare +terms on the coordinating node... but a shard may have already tripped the `max_buckets` soft limit and aborted +the request. + +When aggregating on fields that have potentially many "rare" terms, you may need to increase the `max_buckets` soft +limit. Alternatively, you might need to find a way to filter the results to return fewer rare values (smaller time +span, filter by category, etc), or re-evaluate your definition of "rare" (e.g. if something +appears 100,000 times, is it truly "rare"?) + +[[search-aggregations-bucket-rare-terms-aggregation-approximate-counts]] +==== Document counts are approximate + +The naive way to determine the "rare" terms in a dataset is to place all the values in a map, incrementing counts +as each document is visited, then return the bottom `n` rows. This does not scale beyond even modestly sized data +sets. A sharded approach where only the "top n" values are retained from each shard (ala the `terms` aggregation) +fails because the long-tail nature of the problem means it is impossible to find the "top n" bottom values without +simply collecting all the values from all shards. + +Instead, the Rare Terms aggregation uses a different approximate algorithm: + +1. Values are placed in a map the first time they are seen. +2. Each addition occurrence of the term increments a counter in the map +3. If the counter > the `max_doc_count` threshold, the term is removed from the map and placed in a +https://www.cs.cmu.edu/~dga/papers/cuckoo-conext2014.pdf[CuckooFilter] +4. The CuckooFilter is consulted on each term. If the value is inside the filter, it is known to be above the +threshold already and skipped. + +After execution, the map of values is the map of "rare" terms under the `max_doc_count` threshold. This map and CuckooFilter +are then merged with all other shards. If there are terms that are greater than the threshold (or appear in +a different shard's CuckooFilter) the term is removed from the merged list. The final map of values is returned +to the user as the "rare" terms. + +CuckooFilters have the possibility of returning false positives (they can say a value exists in their collection when +it actually does not). Since the CuckooFilter is being used to see if a term is over threshold, this means a false positive +from the CuckooFilter will mistakenly say a value is common when it is not (and thus exclude it from it final list of buckets). +Practically, this means the aggregations exhibits false-negative behavior since the filter is being used "in reverse" +of how people generally think of approximate set membership sketches. + +CuckooFilters are described in more detail in the paper: + +https://www.cs.cmu.edu/~dga/papers/cuckoo-conext2014.pdf[Fan, Bin, et al. "Cuckoo filter: Practically better than bloom."] +Proceedings of the 10th ACM International on Conference on emerging Networking Experiments and Technologies. ACM, 2014. + +==== Precision + +Although the internal CuckooFilter is approximate in nature, the false-negative rate can be controlled with a +`precision` parameter. This allows the user to trade more runtime memory for more accurate results. + +The default precision is `0.001`, and the smallest (e.g. most accurate and largest memory overhead) is `0.00001`. +Below are some charts which demonstrate how the accuracy of the aggregation is affected by precision and number +of distinct terms. + +The X-axis shows the number of distinct values the aggregation has seen, and the Y-axis shows the percent error. +Each line series represents one "rarity" condition (ranging from one rare item to 100,000 rare items). For example, +the orange "10" line means ten of the values were "rare" (`doc_count == 1`), out of 1-20m distinct values (where the +rest of the values had `doc_count > 1`) + +This first chart shows precision `0.01`: + +image:images/rare_terms/accuracy_01.png[] + +And precision `0.001` (the default): + +image:images/rare_terms/accuracy_001.png[] + +And finally `precision 0.0001`: + +image:images/rare_terms/accuracy_0001.png[] + +The default precision of `0.001` maintains an accuracy of < 2.5% for the tested conditions, and accuracy slowly +degrades in a controlled, linear fashion as the number of distinct values increases. + +The default precision of `0.001` has a memory profile of `1.748⁻⁶ * n` bytes, where `n` is the number +of distinct values the aggregation has seen (it can also be roughly eyeballed, e.g. 20 million unique values is about +30mb of memory). The memory usage is linear to the number of distinct values regardless of which precision is chosen, +the precision only affects the slope of the memory profile as seen in this chart: + +image:images/rare_terms/memory.png[] + +For comparison, an equivalent terms aggregation at 20 million buckets would be roughly +`20m * 69b == ~1.38gb` (with 69 bytes being a very optimistic estimate of an empty bucket cost, far lower than what +the circuit breaker accounts for). So although the `rare_terms` agg is relatively heavy, it is still orders of +magnitude smaller than the equivalent terms aggregation + +==== Filtering Values + +It is possible to filter the values for which buckets will be created. This can be done using the `include` and +`exclude` parameters which are based on regular expression strings or arrays of exact values. Additionally, +`include` clauses can filter using `partition` expressions. + +===== Filtering Values with regular expressions + +[source,js] +-------------------------------------------------- +GET /_search +{ + "aggs" : { + "genres" : { + "rare_terms" : { + "field" : "genre", + "include" : "swi*", + "exclude" : "electro*" + } + } + } +} +-------------------------------------------------- +// CONSOLE + +In the above example, buckets will be created for all the tags that starts with `swi`, except those starting +with `electro` (so the tag `swing` will be aggregated but not `electro_swing`). The `include` regular expression will determine what +values are "allowed" to be aggregated, while the `exclude` determines the values that should not be aggregated. When +both are defined, the `exclude` has precedence, meaning, the `include` is evaluated first and only then the `exclude`. + +The syntax is the same as <>. + +===== Filtering Values with exact values + +For matching based on exact values the `include` and `exclude` parameters can simply take an array of +strings that represent the terms as they are found in the index: + +[source,js] +-------------------------------------------------- +GET /_search +{ + "aggs" : { + "genres" : { + "rare_terms" : { + "field" : "genre", + "include" : ["swing", "rock"], + "exclude" : ["jazz"] + } + } + } +} +-------------------------------------------------- +// CONSOLE + + +==== Missing value + +The `missing` parameter defines how documents that are missing a value should be treated. +By default they will be ignored but it is also possible to treat them as if they +had a value. + +[source,js] +-------------------------------------------------- +GET /_search +{ + "aggs" : { + "genres" : { + "rare_terms" : { + "field" : "genre", + "missing": "N/A" <1> + } + } + } +} +-------------------------------------------------- +// CONSOLE + +<1> Documents without a value in the `tags` field will fall into the same bucket as documents that have the value `N/A`. + +==== Nested, RareTerms, and scoring sub-aggregations + +The RareTerms aggregation has to operate in `breadth_first` mode, since it needs to prune terms as doc count thresholds +are breached. This requirement means the RareTerms aggregation is incompatible with certain combinations of aggregations +that require `depth_first`. In particular, scoring sub-aggregations that are inside a `nested` force the entire aggregation tree to run +in `depth_first` mode. This will throw an exception since RareTerms is unable to process `depth_first`. + +As a concrete example, if `rare_terms` aggregation is the child of a `nested` aggregation, and one of the child aggregations of `rare_terms` +needs document scores (like a `top_hits` aggregation), this will throw an exception. \ No newline at end of file diff --git a/docs/reference/analysis/tokenfilters/elision-tokenfilter.asciidoc b/docs/reference/analysis/tokenfilters/elision-tokenfilter.asciidoc index 924903b9f65a8..34646a0413e36 100644 --- a/docs/reference/analysis/tokenfilters/elision-tokenfilter.asciidoc +++ b/docs/reference/analysis/tokenfilters/elision-tokenfilter.asciidoc @@ -4,8 +4,12 @@ A token filter which removes elisions. For example, "l'avion" (the plane) will tokenized as "avion" (plane). -Accepts `articles` setting which is a set of stop words articles. For -example: +Requires either an `articles` parameter which is a set of stop word articles, or +`articles_path` which points to a text file containing the stop set. Also optionally +accepts `articles_case`, which indicates whether the filter treats those articles as +case sensitive. + +For example: [source,js] -------------------------------------------------- @@ -22,6 +26,7 @@ PUT /elision_example "filter" : { "elision" : { "type" : "elision", + "articles_case": true, "articles" : ["l", "m", "t", "qu", "n", "s", "j"] } } diff --git a/docs/reference/analysis/tokenfilters/synonym-tokenfilter.asciidoc b/docs/reference/analysis/tokenfilters/synonym-tokenfilter.asciidoc index 139f7c3ab0ad0..f47e97d27ea7f 100644 --- a/docs/reference/analysis/tokenfilters/synonym-tokenfilter.asciidoc +++ b/docs/reference/analysis/tokenfilters/synonym-tokenfilter.asciidoc @@ -43,6 +43,8 @@ Additional settings are: * `expand` (defaults to `true`). * `lenient` (defaults to `false`). If `true` ignores exceptions while parsing the synonym configuration. It is important to note that only those synonym rules which cannot get parsed are ignored. For instance consider the following request: + + [source,js] -------------------------------------------------- diff --git a/docs/reference/analysis/tokenfilters/word-delimiter-tokenfilter.asciidoc b/docs/reference/analysis/tokenfilters/word-delimiter-tokenfilter.asciidoc index 009b027b9ef2d..9b1b0b0ce098f 100644 --- a/docs/reference/analysis/tokenfilters/word-delimiter-tokenfilter.asciidoc +++ b/docs/reference/analysis/tokenfilters/word-delimiter-tokenfilter.asciidoc @@ -18,7 +18,7 @@ Parameters include: `generate_word_parts`:: If `true` causes parts of words to be - generated: "PowerShot" => "Power" "Shot". Defaults to `true`. + generated: "Power-Shot", "(Power,Shot)" => "Power" "Shot". Defaults to `true`. `generate_number_parts`:: If `true` causes number subwords to be diff --git a/docs/reference/analysis/tokenizers.asciidoc b/docs/reference/analysis/tokenizers.asciidoc index d6f15ded05fab..628afebfdcbb8 100644 --- a/docs/reference/analysis/tokenizers.asciidoc +++ b/docs/reference/analysis/tokenizers.asciidoc @@ -155,3 +155,7 @@ include::tokenizers/simplepattern-tokenizer.asciidoc[] include::tokenizers/simplepatternsplit-tokenizer.asciidoc[] include::tokenizers/pathhierarchy-tokenizer.asciidoc[] + +include::tokenizers/pathhierarchy-tokenizer-examples.asciidoc[] + + diff --git a/docs/reference/analysis/tokenizers/pathhierarchy-tokenizer-examples.asciidoc b/docs/reference/analysis/tokenizers/pathhierarchy-tokenizer-examples.asciidoc new file mode 100644 index 0000000000000..ee02d66e4034b --- /dev/null +++ b/docs/reference/analysis/tokenizers/pathhierarchy-tokenizer-examples.asciidoc @@ -0,0 +1,191 @@ +[[analysis-pathhierarchy-tokenizer-examples]] +=== Path Hierarchy Tokenizer Examples + +A common use-case for the `path_hierarchy` tokenizer is filtering results by +file paths. If indexing a file path along with the data, the use of the +`path_hierarchy` tokenizer to analyze the path allows filtering the results +by different parts of the file path string. + + +This example configures an index to have two custom analyzers and applies +those analyzers to multifields of the `file_path` text field that will +store filenames. One of the two analyzers uses reverse tokenization. +Some sample documents are then indexed to represent some file paths +for photos inside photo folders of two different users. + + +[source,js] +-------------------------------------------------- +PUT file-path-test +{ + "settings": { + "analysis": { + "analyzer": { + "custom_path_tree": { + "tokenizer": "custom_hierarchy" + }, + "custom_path_tree_reversed": { + "tokenizer": "custom_hierarchy_reversed" + } + }, + "tokenizer": { + "custom_hierarchy": { + "type": "path_hierarchy", + "delimiter": "/" + }, + "custom_hierarchy_reversed": { + "type": "path_hierarchy", + "delimiter": "/", + "reverse": "true" + } + } + } + }, + "mappings": { + "properties": { + "file_path": { + "type": "text", + "fields": { + "tree": { + "type": "text", + "analyzer": "custom_path_tree" + }, + "tree_reversed": { + "type": "text", + "analyzer": "custom_path_tree_reversed" + } + } + } + } + } +} + +POST file-path-test/_doc/1 +{ + "file_path": "/User/alice/photos/2017/05/16/my_photo1.jpg" +} + +POST file-path-test/_doc/2 +{ + "file_path": "/User/alice/photos/2017/05/16/my_photo2.jpg" +} + +POST file-path-test/_doc/3 +{ + "file_path": "/User/alice/photos/2017/05/16/my_photo3.jpg" +} + +POST file-path-test/_doc/4 +{ + "file_path": "/User/alice/photos/2017/05/15/my_photo1.jpg" +} + +POST file-path-test/_doc/5 +{ + "file_path": "/User/bob/photos/2017/05/16/my_photo1.jpg" +} +-------------------------------------------------- +// CONSOLE +// TESTSETUP + + +A search for a particular file path string against the text field matches all +the example documents, with Bob's documents ranking highest due to `bob` also +being one of the terms created by the standard analyzer boosting relevance for +Bob's documents. + +[source,js] +-------------------------------------------------- +GET file-path-test/_search +{ + "query": { + "match": { + "file_path": "/User/bob/photos/2017/05" + } + } +} +-------------------------------------------------- +// CONSOLE + + +It's simple to match or filter documents with file paths that exist within a +particular directory using the `file_path.tree` field. + +[source,js] +-------------------------------------------------- +GET file-path-test/_search +{ + "query": { + "term": { + "file_path.tree": "/User/alice/photos/2017/05/16" + } + } +} +-------------------------------------------------- +// CONSOLE + +With the reverse parameter for this tokenizer, it's also possible to match +from the other end of the file path, such as individual file names or a deep +level subdirectory. The following example shows a search for all files named +`my_photo1.jpg` within any directory via the `file_path.tree_reversed` field +configured to use the reverse parameter in the mapping. + + +[source,js] +-------------------------------------------------- +GET file-path-test/_search +{ + "query": { + "term": { + "file_path.tree_reversed": { + "value": "my_photo1.jpg" + } + } + } +} +-------------------------------------------------- +// CONSOLE + + +Viewing the tokens generated with both forward and reverse is instructive +in showing the tokens created for the same file path value. + + +[source,js] +-------------------------------------------------- +POST file-path-test/_analyze +{ + "analyzer": "custom_path_tree", + "text": "/User/alice/photos/2017/05/16/my_photo1.jpg" +} + +POST file-path-test/_analyze +{ + "analyzer": "custom_path_tree_reversed", + "text": "/User/alice/photos/2017/05/16/my_photo1.jpg" +} +-------------------------------------------------- +// CONSOLE + + +It's also useful to be able to filter with file paths when combined with other +types of searches, such as this example looking for any files paths with `16` +that also must be in Alice's photo directory. + +[source,js] +-------------------------------------------------- +GET file-path-test/_search +{ + "query": { + "bool" : { + "must" : { + "match" : { "file_path" : "16" } + }, + "filter": { + "term" : { "file_path.tree" : "/User/alice" } + } + } + } +} +-------------------------------------------------- +// CONSOLE diff --git a/docs/reference/analysis/tokenizers/pathhierarchy-tokenizer.asciidoc b/docs/reference/analysis/tokenizers/pathhierarchy-tokenizer.asciidoc index 55aa7d66da343..8d425197a2a6d 100644 --- a/docs/reference/analysis/tokenizers/pathhierarchy-tokenizer.asciidoc +++ b/docs/reference/analysis/tokenizers/pathhierarchy-tokenizer.asciidoc @@ -170,3 +170,7 @@ If we were to set `reverse` to `true`, it would produce the following: --------------------------- [ one/two/three/, two/three/, three/ ] --------------------------- + +[float] +=== Detailed Examples +See <>. diff --git a/docs/reference/api-conventions.asciidoc b/docs/reference/api-conventions.asciidoc index 0b7d9ce403466..b46bd21a32bb1 100644 --- a/docs/reference/api-conventions.asciidoc +++ b/docs/reference/api-conventions.asciidoc @@ -86,6 +86,10 @@ Where: `date_format`:: is the optional format in which the computed date should be rendered. Defaults to `yyyy.MM.dd`. Format should be compatible with java-time https://docs.oracle.com/javase/8/docs/api/java/time/format/DateTimeFormatter.html `time_zone`:: is the optional time zone. Defaults to `utc`. +NOTE: Pay attention to the usage of small vs capital letters used in the `date_format`. For example: +`mm` denotes minute of hour, while `MM` denotes month of year. Similarly `hh` denotes the hour in the +`1-12` range in combination with `AM/PM`, while `HH` denotes the hour in the `0-23` 24-hour range. + Date math expressions are resolved locale-independent. Consequently, it is not possible to use any other calendars than the Gregorian calendar. diff --git a/docs/reference/cat.asciidoc b/docs/reference/cat.asciidoc index 0da752aba42d2..743bae7b85679 100644 --- a/docs/reference/cat.asciidoc +++ b/docs/reference/cat.asciidoc @@ -40,7 +40,7 @@ Might respond with: id host ip node u_n93zwxThWHi1PDBJAGAg 127.0.0.1 127.0.0.1 u_n93zw -------------------------------------------------- -// TESTRESPONSE[s/u_n93zw(xThWHi1PDBJAGAg)?/.+/ _cat] +// TESTRESPONSE[s/u_n93zw(xThWHi1PDBJAGAg)?/.+/ non_json] [float] [[help]] @@ -64,7 +64,7 @@ host | h | host name ip | | ip address node | n | node name -------------------------------------------------- -// TESTRESPONSE[s/[|]/[|]/ _cat] +// TESTRESPONSE[s/[|]/[|]/ non_json] NOTE: `help` is not supported if any optional url parameter is used. For example `GET _cat/shards/twitter?help` or `GET _cat/indices/twi*?help` @@ -90,7 +90,7 @@ Responds with: -------------------------------------------------- 127.0.0.1 9300 27 sLBaIGK -------------------------------------------------- -// TESTRESPONSE[s/9300 27 sLBaIGK/\\d+ \\d+ .+/ _cat] +// TESTRESPONSE[s/9300 27 sLBaIGK/\\d+ \\d+ .+/ non_json] You can also request multiple columns using simple wildcards like `/_cat/thread_pool?h=ip,queue*` to get all headers (or aliases) starting diff --git a/docs/reference/cat/alias.asciidoc b/docs/reference/cat/alias.asciidoc index 394231e448dc0..a46a4c658d4b0 100644 --- a/docs/reference/cat/alias.asciidoc +++ b/docs/reference/cat/alias.asciidoc @@ -49,7 +49,7 @@ alias2 test1 * - - alias3 test1 - 1 1 alias4 test1 - 2 1,2 -------------------------------------------------- -// TESTRESPONSE[s/[*]/[*]/ _cat] +// TESTRESPONSE[s/[*]/[*]/ non_json] The output shows that `alias2` has configured a filter, and specific routing configurations in `alias3` and `alias4`. diff --git a/docs/reference/cat/allocation.asciidoc b/docs/reference/cat/allocation.asciidoc index 0ff745c3e5cb3..8d2c931665ba1 100644 --- a/docs/reference/cat/allocation.asciidoc +++ b/docs/reference/cat/allocation.asciidoc @@ -19,7 +19,7 @@ shards disk.indices disk.used disk.avail disk.total disk.percent host ip 1 260b 47.3gb 43.4gb 100.7gb 46 127.0.0.1 127.0.0.1 CSUXak2 -------------------------------------------------- // TESTRESPONSE[s/\d+(\.\d+)?[tgmk]?b/\\d+(\\.\\d+)?[tgmk]?b/ s/46/\\d+/] -// TESTRESPONSE[s/CSUXak2/.+/ _cat] +// TESTRESPONSE[s/CSUXak2/.+/ non_json] Here we can see that the single shard created has been allocated to the single node available. diff --git a/docs/reference/cat/count.asciidoc b/docs/reference/cat/count.asciidoc index 28dc39adc8dff..01fec6df73180 100644 --- a/docs/reference/cat/count.asciidoc +++ b/docs/reference/cat/count.asciidoc @@ -19,7 +19,7 @@ Looks like: epoch timestamp count 1475868259 15:24:19 121 -------------------------------------------------- -// TESTRESPONSE[s/1475868259 15:24:19/\\d+ \\d+:\\d+:\\d+/ _cat] +// TESTRESPONSE[s/1475868259 15:24:19/\\d+ \\d+:\\d+:\\d+/ non_json] Or for a single index: @@ -35,7 +35,7 @@ GET /_cat/count/twitter?v epoch timestamp count 1475868259 15:24:20 120 -------------------------------------------------- -// TESTRESPONSE[s/1475868259 15:24:20/\\d+ \\d+:\\d+:\\d+/ _cat] +// TESTRESPONSE[s/1475868259 15:24:20/\\d+ \\d+:\\d+:\\d+/ non_json] NOTE: The document count indicates the number of live documents and does not include deleted documents which have not yet been cleaned up by the merge process. diff --git a/docs/reference/cat/fielddata.asciidoc b/docs/reference/cat/fielddata.asciidoc index ff71728a4f50b..f94bcd6fe5d78 100644 --- a/docs/reference/cat/fielddata.asciidoc +++ b/docs/reference/cat/fielddata.asciidoc @@ -52,7 +52,7 @@ Nqk-6inXQq-OxUfOUI8jNQ 127.0.0.1 127.0.0.1 Nqk-6in body 544b Nqk-6inXQq-OxUfOUI8jNQ 127.0.0.1 127.0.0.1 Nqk-6in soul 480b -------------------------------------------------- // TESTRESPONSE[s/544b|480b/\\d+(\\.\\d+)?[tgmk]?b/] -// TESTRESPONSE[s/Nqk-6in[^ ]*/.+/ s/soul|body/\\w+/ _cat] +// TESTRESPONSE[s/Nqk-6in[^ ]*/.+/ s/soul|body/\\w+/ non_json] Fields can be specified either as a query parameter, or in the URL path: @@ -71,7 +71,7 @@ id host ip node field size Nqk-6inXQq-OxUfOUI8jNQ 127.0.0.1 127.0.0.1 Nqk-6in body 544b -------------------------------------------------- // TESTRESPONSE[s/544b|480b/\\d+(\\.\\d+)?[tgmk]?b/] -// TESTRESPONSE[s/Nqk-6in[^ ]*/.+/ _cat] +// TESTRESPONSE[s/Nqk-6in[^ ]*/.+/ non_json] And it accepts a comma delimited list: @@ -91,6 +91,6 @@ Nqk-6inXQq-OxUfOUI8jNQ 127.0.0.1 127.0.0.1 Nqk-6in body 544b Nqk-6inXQq-OxUfOUI8jNQ 127.0.0.1 127.0.0.1 Nqk-6in soul 480b -------------------------------------------------- // TESTRESPONSE[s/544b|480b/\\d+(\\.\\d+)?[tgmk]?b/] -// TESTRESPONSE[s/Nqk-6in[^ ]*/.+/ s/soul|body/\\w+/ _cat] +// TESTRESPONSE[s/Nqk-6in[^ ]*/.+/ s/soul|body/\\w+/ non_json] The output shows the individual fielddata for the `body` and `soul` fields, one row per field per node. diff --git a/docs/reference/cat/health.asciidoc b/docs/reference/cat/health.asciidoc index 5f053edf30866..883119925fd5f 100644 --- a/docs/reference/cat/health.asciidoc +++ b/docs/reference/cat/health.asciidoc @@ -17,7 +17,7 @@ epoch timestamp cluster status node.total node.data shards pri relo i 1475871424 16:17:04 elasticsearch green 1 1 1 1 0 0 0 0 - 100.0% -------------------------------------------------- // TESTRESPONSE[s/1475871424 16:17:04/\\d+ \\d+:\\d+:\\d+/] -// TESTRESPONSE[s/elasticsearch/[^ ]+/ s/0 -/\\d+ (-|\\d+(\\.\\d+)?[ms]+)/ _cat] +// TESTRESPONSE[s/elasticsearch/[^ ]+/ s/0 -/\\d+ (-|\\d+(\\.\\d+)?[ms]+)/ non_json] It has one option `ts` to disable the timestamping: @@ -35,7 +35,7 @@ which looks like: cluster status node.total node.data shards pri relo init unassign pending_tasks max_task_wait_time active_shards_percent elasticsearch green 1 1 1 1 0 0 0 0 - 100.0% -------------------------------------------------- -// TESTRESPONSE[s/elasticsearch/[^ ]+/ s/0 -/\\d+ (-|\\d+(\\.\\d+)?[ms]+)/ _cat] +// TESTRESPONSE[s/elasticsearch/[^ ]+/ s/0 -/\\d+ (-|\\d+(\\.\\d+)?[ms]+)/ non_json] A common use of this command is to verify the health is consistent across nodes: diff --git a/docs/reference/cat/indices.asciidoc b/docs/reference/cat/indices.asciidoc index 2a5b865fefa47..653889dac8315 100644 --- a/docs/reference/cat/indices.asciidoc +++ b/docs/reference/cat/indices.asciidoc @@ -21,7 +21,7 @@ yellow open twitter u8FNjxh8Rfy_awN11oDKYQ 1 1 1200 0 green open twitter2 nYFWZEO7TUiOjLQXBaYJpA 1 0 0 0 260b 260b -------------------------------------------------- // TESTRESPONSE[s/\d+(\.\d+)?[tgmk]?b/\\d+(\\.\\d+)?[tgmk]?b/] -// TESTRESPONSE[s/u8FNjxh8Rfy_awN11oDKYQ|nYFWZEO7TUiOjLQXBaYJpA/.+/ _cat] +// TESTRESPONSE[s/u8FNjxh8Rfy_awN11oDKYQ|nYFWZEO7TUiOjLQXBaYJpA/.+/ non_json] We can tell quickly how many shards make up an index, the number of docs, deleted docs, primary store size, and total store size (all shards including replicas). @@ -64,7 +64,7 @@ health status index uuid pri rep docs.count docs.deleted st yellow open twitter u8FNjxh8Rfy_awN11oDKYQ 1 1 1200 0 88.1kb 88.1kb -------------------------------------------------- // TESTRESPONSE[s/\d+(\.\d+)?[tgmk]?b/\\d+(\\.\\d+)?[tgmk]?b/] -// TESTRESPONSE[s/u8FNjxh8Rfy_awN11oDKYQ/.+/ _cat] +// TESTRESPONSE[s/u8FNjxh8Rfy_awN11oDKYQ/.+/ non_json] Which index has the largest number of documents? @@ -84,7 +84,7 @@ yellow open twitter u8FNjxh8Rfy_awN11oDKYQ 1 1 1200 0 green open twitter2 nYFWZEO7TUiOjLQXBaYJpA 1 0 0 0 260b 260b -------------------------------------------------- // TESTRESPONSE[s/\d+(\.\d+)?[tgmk]?b/\\d+(\\.\\d+)?[tgmk]?b/] -// TESTRESPONSE[s/u8FNjxh8Rfy_awN11oDKYQ|nYFWZEO7TUiOjLQXBaYJpA/.+/ _cat] +// TESTRESPONSE[s/u8FNjxh8Rfy_awN11oDKYQ|nYFWZEO7TUiOjLQXBaYJpA/.+/ non_json] How many merge operations have the shards for the `twitter` completed? @@ -102,7 +102,7 @@ Might look like: health index pri rep docs.count mt pri.mt yellow twitter 1 1 1200 16 16 -------------------------------------------------- -// TESTRESPONSE[s/16/\\d+/ _cat] +// TESTRESPONSE[s/16/\\d+/ non_json] How much memory is used per index? @@ -122,4 +122,4 @@ twitter 8.1gb twitter2 30.5kb -------------------------------------------------- // TESTRESPONSE[s/\d+(\.\d+)?[tgmk]?b/\\d+(\\.\\d+)?[tgmk]?b/] -// TESTRESPONSE[_cat] +// TESTRESPONSE[non_json] diff --git a/docs/reference/cat/master.asciidoc b/docs/reference/cat/master.asciidoc index dfa10d6e3a4e6..a38baa455caee 100644 --- a/docs/reference/cat/master.asciidoc +++ b/docs/reference/cat/master.asciidoc @@ -17,7 +17,7 @@ might respond: id host ip node YzWoH_2BT-6UjVGDyPdqYg 127.0.0.1 127.0.0.1 YzWoH_2 -------------------------------------------------- -// TESTRESPONSE[s/YzWoH_2.+/.+/ _cat] +// TESTRESPONSE[s/YzWoH_2.+/.+/ non_json] This information is also available via the `nodes` command, but this is slightly shorter when all you want to do, for example, is verify diff --git a/docs/reference/cat/nodeattrs.asciidoc b/docs/reference/cat/nodeattrs.asciidoc index 2b893a4c79b11..e5c335f7c375f 100644 --- a/docs/reference/cat/nodeattrs.asciidoc +++ b/docs/reference/cat/nodeattrs.asciidoc @@ -22,7 +22,7 @@ node-0 127.0.0.1 127.0.0.1 testattr test ... -------------------------------------------------- // TESTRESPONSE[s/\.\.\.\n$/\n(.+ xpack\\.installed true\n)?\n/] -// TESTRESPONSE[s/\.\.\.\n/(.+ ml\\..+\n)*/ _cat] +// TESTRESPONSE[s/\.\.\.\n/(.+ ml\\..+\n)*/ non_json] // If xpack is not installed then neither ... with match anything // If xpack is installed then the first ... contains ml attributes // and the second contains xpack.installed=true @@ -68,7 +68,7 @@ node-0 19566 testattr test -------------------------------------------------- // TESTRESPONSE[s/19566/\\d*/] // TESTRESPONSE[s/\.\.\.\n$/\n(.+ xpack\\.installed true\n)?\n/] -// TESTRESPONSE[s/\.\.\.\n/(.+ ml\\..+\n)*/ _cat] +// TESTRESPONSE[s/\.\.\.\n/(.+ ml\\..+\n)*/ non_json] // If xpack is not installed then neither ... with match anything // If xpack is installed then the first ... contains ml attributes // and the second contains xpack.installed=true diff --git a/docs/reference/cat/nodes.asciidoc b/docs/reference/cat/nodes.asciidoc index 151ce80196b50..a882f5f9fa398 100644 --- a/docs/reference/cat/nodes.asciidoc +++ b/docs/reference/cat/nodes.asciidoc @@ -14,11 +14,11 @@ Might look like: [source,txt] -------------------------------------------------- ip heap.percent ram.percent cpu load_1m load_5m load_15m node.role master name -127.0.0.1 65 99 42 3.07 mdi * mJw06l1 +127.0.0.1 65 99 42 3.07 dim * mJw06l1 -------------------------------------------------- // TESTRESPONSE[s/3.07/(\\d+\\.\\d+( \\d+\\.\\d+ (\\d+\\.\\d+)?)?)?/] // TESTRESPONSE[s/65 99 42/\\d+ \\d+ \\d+/] -// TESTRESPONSE[s/[*]/[*]/ s/mJw06l1/.+/ _cat] +// TESTRESPONSE[s/[*]/[*]/ s/mJw06l1/.+/ non_json] The first few columns (`ip`, `heap.percent`, `ram.percent`, `cpu`, `load_*`) tell you where your nodes live and give a quick picture of performance stats. @@ -63,7 +63,7 @@ Might look like: id ip port v m veJR 127.0.0.1 59938 {version} * -------------------------------------------------- -// TESTRESPONSE[s/veJR/.+/ s/59938/\\d+/ s/[*]/[*]/ _cat] +// TESTRESPONSE[s/veJR/.+/ s/59938/\\d+/ s/[*]/[*]/ non_json] [cols="<,<,<,<,<",options="header",subs="normal"] |======================================================================= diff --git a/docs/reference/cat/pending_tasks.asciidoc b/docs/reference/cat/pending_tasks.asciidoc index d5216c1eb0023..ec923f270c571 100644 --- a/docs/reference/cat/pending_tasks.asciidoc +++ b/docs/reference/cat/pending_tasks.asciidoc @@ -24,6 +24,6 @@ insertOrder timeInQueue priority source 1690 787ms HIGH update-mapping [foo][t] 1691 773ms HIGH update-mapping [foo][t] -------------------------------------------------- -// TESTRESPONSE[s/(\n.+)+/(\\n.+)*/ _cat] +// TESTRESPONSE[s/(\n.+)+/(\\n.+)*/ non_json] // We can't assert anything about the tasks in progress here because we don't // know what might be in progress.... diff --git a/docs/reference/cat/plugins.asciidoc b/docs/reference/cat/plugins.asciidoc index 5e516fee72cf4..9800b4fef0a64 100644 --- a/docs/reference/cat/plugins.asciidoc +++ b/docs/reference/cat/plugins.asciidoc @@ -31,6 +31,6 @@ U7321H6 mapper-size {version_qualified} The Mapper Size plugin allow U7321H6 store-smb {version_qualified} The Store SMB plugin adds support for SMB stores. U7321H6 transport-nio {version_qualified} The nio transport. ------------------------------------------------------------------------------ -// TESTRESPONSE[s/([.()])/\\$1/ s/U7321H6/.+/ _cat] +// TESTRESPONSE[s/([.()])/\\$1/ s/U7321H6/.+/ non_json] We can tell quickly how many plugins per node we have and which versions. diff --git a/docs/reference/cat/recovery.asciidoc b/docs/reference/cat/recovery.asciidoc index c4288f882e21e..1477dfb676f9f 100644 --- a/docs/reference/cat/recovery.asciidoc +++ b/docs/reference/cat/recovery.asciidoc @@ -30,7 +30,7 @@ twitter 0 13ms store done n/a n/a 127.0.0.1 node-0 n // TESTRESPONSE[s/100%/0.0%/] // TESTRESPONSE[s/9928/0/] // TESTRESPONSE[s/13ms/\\d+m?s/] -// TESTRESPONSE[s/13/\\d+/ _cat] +// TESTRESPONSE[s/13/\\d+/ non_json] In the above case, the source and target nodes are the same because the recovery type was store, i.e. they were read from local storage on node start. @@ -57,7 +57,7 @@ twitter 0 1252ms peer done 192.168.1.1 192.168.1.2 0 100.0% 0 100.0% // TESTRESPONSE[s/192.168.1.2/127.0.0.1/] // TESTRESPONSE[s/192.168.1.1/n\/a/] // TESTRESPONSE[s/100.0%/0.0%/] -// TESTRESPONSE[s/1252/\\d+/ _cat] +// TESTRESPONSE[s/1252/\\d+/ non_json] We can see in the above listing that our thw twitter shard was recovered from another node. Notice that the recovery type is shown as `peer`. The files and bytes copied are @@ -81,4 +81,4 @@ This will show a recovery of type snapshot in the response i s t ty st rep snap f fp b bp twitter 0 1978ms snapshot done twitter snap_1 79 8.0% 12086 9.0% -------------------------------------------------------------------------------- -// TESTRESPONSE[_cat] +// TESTRESPONSE[non_json] diff --git a/docs/reference/cat/repositories.asciidoc b/docs/reference/cat/repositories.asciidoc index 8caf3c5fd6f73..89daf7748a5d4 100644 --- a/docs/reference/cat/repositories.asciidoc +++ b/docs/reference/cat/repositories.asciidoc @@ -19,6 +19,6 @@ id type repo1 fs repo2 s3 -------------------------------------------------- -// TESTRESPONSE[s/\nrepo2 s3// _cat] +// TESTRESPONSE[s/\nrepo2 s3// non_json] We can quickly see which repositories are registered and their type. diff --git a/docs/reference/cat/segments.asciidoc b/docs/reference/cat/segments.asciidoc index a4c2c54d8eefd..5fa2f66e384a0 100644 --- a/docs/reference/cat/segments.asciidoc +++ b/docs/reference/cat/segments.asciidoc @@ -20,7 +20,7 @@ index shard prirep ip segment generation docs.count docs.deleted size siz test 0 p 127.0.0.1 _0 0 1 0 3kb 2042 false true {lucene_version} true test1 0 p 127.0.0.1 _0 0 1 0 3kb 2042 false true {lucene_version} true -------------------------------------------------- -// TESTRESPONSE[s/3kb/\\d+(\\.\\d+)?[mk]?b/ s/2042/\\d+/ _cat] +// TESTRESPONSE[s/3kb/\\d+(\\.\\d+)?[mk]?b/ s/2042/\\d+/ non_json] The output shows information about index names and shard numbers in the first two columns. diff --git a/docs/reference/cat/shards.asciidoc b/docs/reference/cat/shards.asciidoc index f32c553ba72f6..345e493375400 100644 --- a/docs/reference/cat/shards.asciidoc +++ b/docs/reference/cat/shards.asciidoc @@ -23,7 +23,7 @@ twitter 0 p STARTED 3014 31.1mb 192.168.56.10 H5dfFeA // TESTRESPONSE[s/3014/\\d+/] // TESTRESPONSE[s/31.1mb/\\d+(\.\\d+)?[kmg]?b/] // TESTRESPONSE[s/192.168.56.10/.*/] -// TESTRESPONSE[s/H5dfFeA/node-0/ _cat] +// TESTRESPONSE[s/H5dfFeA/node-0/ non_json] [float] [[index-pattern]] @@ -49,7 +49,7 @@ twitter 0 p STARTED 3014 31.1mb 192.168.56.10 H5dfFeA // TESTRESPONSE[s/3014/\\d+/] // TESTRESPONSE[s/31.1mb/\\d+(\.\\d+)?[kmg]?b/] // TESTRESPONSE[s/192.168.56.10/.*/] -// TESTRESPONSE[s/H5dfFeA/node-0/ _cat] +// TESTRESPONSE[s/H5dfFeA/node-0/ non_json] [float] @@ -72,7 +72,7 @@ A relocating shard will be shown as follows --------------------------------------------------------------------------- twitter 0 p RELOCATING 3014 31.1mb 192.168.56.10 H5dfFeA -> -> 192.168.56.30 bGG90GE --------------------------------------------------------------------------- -// TESTRESPONSE[_cat] +// TESTRESPONSE[non_json] [float] [[states]] @@ -95,7 +95,7 @@ You can get the initializing state in the response like this twitter 0 p STARTED 3014 31.1mb 192.168.56.10 H5dfFeA twitter 0 r INITIALIZING 0 14.3mb 192.168.56.30 bGG90GE --------------------------------------------------------------------------- -// TESTRESPONSE[_cat] +// TESTRESPONSE[non_json] If a shard cannot be assigned, for example you've overallocated the number of replicas for the number of nodes in the cluster, the shard @@ -119,7 +119,7 @@ twitter 0 r STARTED 3014 31.1mb 192.168.56.30 bGG90GE twitter 0 r STARTED 3014 31.1mb 192.168.56.20 I8hydUG twitter 0 r UNASSIGNED ALLOCATION_FAILED --------------------------------------------------------------------------- -// TESTRESPONSE[_cat] +// TESTRESPONSE[non_json] [float] [[reason-unassigned]] diff --git a/docs/reference/cat/snapshots.asciidoc b/docs/reference/cat/snapshots.asciidoc index 5677a0f2a7cd4..37d57292e0488 100644 --- a/docs/reference/cat/snapshots.asciidoc +++ b/docs/reference/cat/snapshots.asciidoc @@ -1,7 +1,8 @@ [[cat-snapshots]] == cat snapshots -The `snapshots` command shows all snapshots that belong to a specific repository. +The `snapshots` command shows all snapshots that belong to a specific repository +or multiple repositories. To find a list of available repositories to query, the command `/_cat/repositories` can be used. Querying the snapshots of a repository named `repo1` then looks as follows. @@ -18,17 +19,31 @@ Which looks like: [source,txt] -------------------------------------------------- -id status start_epoch start_time end_epoch end_time duration indices successful_shards failed_shards total_shards -snap1 FAILED 1445616705 18:11:45 1445616978 18:16:18 4.6m 1 4 1 5 -snap2 SUCCESS 1445634298 23:04:58 1445634672 23:11:12 6.2m 2 10 0 10 +id repository status start_epoch start_time end_epoch end_time duration indices successful_shards failed_shards total_shards +snap1 repo1 FAILED 1445616705 18:11:45 1445616978 18:16:18 4.6m 1 4 1 5 +snap2 repo1 SUCCESS 1445634298 23:04:58 1445634672 23:11:12 6.2m 2 10 0 10 -------------------------------------------------- // TESTRESPONSE[s/FAILED/SUCCESS/ s/14456\d+/\\d+/ s/\d+(\.\d+)?(m|s|ms)/\\d+(\\.\\d+)?(m|s|ms)/] // TESTRESPONSE[s/\d+:\d+:\d+/\\d+:\\d+:\\d+/] // TESTRESPONSE[s/1 4 1 5/\\d+ \\d+ \\d+ \\d+/] // TESTRESPONSE[s/2 10 0 10/\\d+ \\d+ \\d+ \\d+/] -// TESTRESPONSE[_cat] +// TESTRESPONSE[non_json] Each snapshot contains information about when it was started and stopped. Start and stop timestamps are available in two formats. The `HH:MM:SS` output is simply for quick human consumption. The epoch time retains more information, including date, and is machine sortable if the snapshot process spans days. + +It is also possible to get the list of snapshots from multiple repositories. +Here are some examples: + +[source,js] +-------------------------------------------------- +GET /_cat/snapshots/_all +GET /_cat/snapshots/repo1,repo2 +GET /_cat/snapshots/repo* +-------------------------------------------------- +// CONSOLE +// TEST[skip:no repo2] + +Please note that if one of the repositories fails during the request you will get an exception instead of the table. \ No newline at end of file diff --git a/docs/reference/cat/templates.asciidoc b/docs/reference/cat/templates.asciidoc index 304459ba96edb..41d27cfbf0306 100644 --- a/docs/reference/cat/templates.asciidoc +++ b/docs/reference/cat/templates.asciidoc @@ -28,7 +28,7 @@ template0 [te*] 0 template1 [tea*] 1 template2 [teak*] 2 7 -------------------------------------------------- -// TESTRESPONSE[s/\*/\\*/ s/\[/\\[/ s/\]/\\]/ _cat] +// TESTRESPONSE[s/\*/\\*/ s/\[/\\[/ s/\]/\\]/ non_json] The output shows that there are three existing templates, with template2 having a version value. diff --git a/docs/reference/cat/thread_pool.asciidoc b/docs/reference/cat/thread_pool.asciidoc index 03854fae2f61f..f63abc8903650 100644 --- a/docs/reference/cat/thread_pool.asciidoc +++ b/docs/reference/cat/thread_pool.asciidoc @@ -23,7 +23,7 @@ node-0 flush 0 0 0 node-0 write 0 0 0 -------------------------------------------------- // TESTRESPONSE[s/\.\.\./(node-0 \\S+ 0 0 0\n)*/] -// TESTRESPONSE[s/\d+/\\d+/ _cat] +// TESTRESPONSE[s/\d+/\\d+/ non_json] // The substitutions do two things: // 1. Expect any number of extra thread pools. This allows us to only list a // few thread pools. The list would be super long otherwise. In addition, @@ -107,7 +107,7 @@ which looks like: id name active rejected completed 0EWUhXeBQtaVGlexUeVwMg generic 0 0 70 -------------------------------------------------- -// TESTRESPONSE[s/0EWUhXeBQtaVGlexUeVwMg/[\\w-]+/ s/\d+/\\d+/ _cat] +// TESTRESPONSE[s/0EWUhXeBQtaVGlexUeVwMg/[\\w-]+/ s/\d+/\\d+/ non_json] Here the host columns and the active, rejected and completed suggest thread pool statistics are displayed. diff --git a/docs/reference/ccr/apis/follow-request-body.asciidoc b/docs/reference/ccr/apis/follow-request-body.asciidoc index e7e6ae2e26a05..d8fb725f02b14 100644 --- a/docs/reference/ccr/apis/follow-request-body.asciidoc +++ b/docs/reference/ccr/apis/follow-request-body.asciidoc @@ -1,4 +1,3 @@ -[role="xpack"] [testenv="platinum"] `max_read_request_operation_count`:: (integer) the maximum number of operations to pull per read from the remote diff --git a/docs/reference/ccr/apis/follow/get-follow-stats.asciidoc b/docs/reference/ccr/apis/follow/get-follow-stats.asciidoc index 73bdd9494d1c8..db4bf910a05c6 100644 --- a/docs/reference/ccr/apis/follow/get-follow-stats.asciidoc +++ b/docs/reference/ccr/apis/follow/get-follow-stats.asciidoc @@ -114,6 +114,9 @@ The `shards` array consists of objects containing the following fields: `indices[].shards[].follower_settings_version`:: (long) the index settings version the follower is synced up to +`indices[].shards[].follower_aliases_version`:: + (long) the index aliases version the follower is synced up to + `indices[].shards[].total_read_time_millis`:: (long) the total time reads were outstanding, measured from the time a read was sent to the leader to the time a reply was returned to the follower @@ -217,6 +220,7 @@ The API returns the following results: "write_buffer_operation_count" : 64, "follower_mapping_version" : 4, "follower_settings_version" : 2, + "follower_aliases_version" : 8, "total_read_time_millis" : 32768, "total_read_remote_exec_time_millis" : 16384, "successful_read_requests" : 32, @@ -246,6 +250,7 @@ The API returns the following results: // TESTRESPONSE[s/"write_buffer_operation_count" : 64/"write_buffer_operation_count" : $body.indices.0.shards.0.write_buffer_operation_count/] // TESTRESPONSE[s/"follower_mapping_version" : 4/"follower_mapping_version" : $body.indices.0.shards.0.follower_mapping_version/] // TESTRESPONSE[s/"follower_settings_version" : 2/"follower_settings_version" : $body.indices.0.shards.0.follower_settings_version/] +// TESTRESPONSE[s/"follower_aliases_version" : 8/"follower_aliases_version" : $body.indices.0.shards.0.follower_aliases_version/] // TESTRESPONSE[s/"total_read_time_millis" : 32768/"total_read_time_millis" : $body.indices.0.shards.0.total_read_time_millis/] // TESTRESPONSE[s/"total_read_remote_exec_time_millis" : 16384/"total_read_remote_exec_time_millis" : $body.indices.0.shards.0.total_read_remote_exec_time_millis/] // TESTRESPONSE[s/"successful_read_requests" : 32/"successful_read_requests" : $body.indices.0.shards.0.successful_read_requests/] diff --git a/docs/reference/ccr/apis/get-ccr-stats.asciidoc b/docs/reference/ccr/apis/get-ccr-stats.asciidoc index 03f2f3eee0c0b..d1acbb0064a9f 100644 --- a/docs/reference/ccr/apis/get-ccr-stats.asciidoc +++ b/docs/reference/ccr/apis/get-ccr-stats.asciidoc @@ -126,6 +126,7 @@ The API returns the following results: "write_buffer_operation_count" : 64, "follower_mapping_version" : 4, "follower_settings_version" : 2, + "follower_aliases_version" : 8, "total_read_time_millis" : 32768, "total_read_remote_exec_time_millis" : 16384, "successful_read_requests" : 32, @@ -161,6 +162,7 @@ The API returns the following results: // TESTRESPONSE[s/"write_buffer_operation_count" : 64/"write_buffer_operation_count" : $body.follow_stats.indices.0.shards.0.write_buffer_operation_count/] // TESTRESPONSE[s/"follower_mapping_version" : 4/"follower_mapping_version" : $body.follow_stats.indices.0.shards.0.follower_mapping_version/] // TESTRESPONSE[s/"follower_settings_version" : 2/"follower_settings_version" : $body.follow_stats.indices.0.shards.0.follower_settings_version/] +// TESTRESPONSE[s/"follower_aliases_version" : 8/"follower_aliases_version" : $body.follow_stats.indices.0.shards.0.follower_aliases_version/] // TESTRESPONSE[s/"total_read_time_millis" : 32768/"total_read_time_millis" : $body.follow_stats.indices.0.shards.0.total_read_time_millis/] // TESTRESPONSE[s/"total_read_remote_exec_time_millis" : 16384/"total_read_remote_exec_time_millis" : $body.follow_stats.indices.0.shards.0.total_read_remote_exec_time_millis/] // TESTRESPONSE[s/"successful_read_requests" : 32/"successful_read_requests" : $body.follow_stats.indices.0.shards.0.successful_read_requests/] diff --git a/docs/reference/ccr/overview.asciidoc b/docs/reference/ccr/overview.asciidoc index def19d80729de..be009b4aefaba 100644 --- a/docs/reference/ccr/overview.asciidoc +++ b/docs/reference/ccr/overview.asciidoc @@ -112,13 +112,18 @@ governed by settings that you can configure when you create the follower index in {kib} or by using the {ref}/ccr-put-follow.html[create follower API]. Mapping updates applied to the leader index are automatically retrieved -as-needed by the follower index. +as-needed by the follower index. It is not possible to manually modify the +mapping of a follower index. Settings updates applied to the leader index that are needed by the follower index are automatically retried as-needed by the follower index. Not all settings updates are needed by the follower index. For example, changing the number of replicas on the leader index is not replicated by the follower index. +Alias updates applied to the leader index are automatically retrieved by the +follower index. It is not possible to manually modify an alias of a follower +index. + NOTE: If you apply a non-dynamic settings change to the leader index that is needed by the follower index, the follower index will go through a cycle of closing itself, applying the settings update, and then re-opening itself. The diff --git a/docs/reference/cluster.asciidoc b/docs/reference/cluster.asciidoc index cfa2d5a6488d7..da5ce2b410281 100644 --- a/docs/reference/cluster.asciidoc +++ b/docs/reference/cluster.asciidoc @@ -22,12 +22,14 @@ one of the following: * an IP address or hostname, to add all matching nodes to the subset. * a pattern, using `*` wildcards, which adds all nodes to the subset whose name, address or hostname matches the pattern. -* `master:true`, `data:true`, `ingest:true` or `coordinating_only:true`, which - respectively add to the subset all master-eligible nodes, all data nodes, - all ingest nodes, and all coordinating-only nodes. -* `master:false`, `data:false`, `ingest:false` or `coordinating_only:false`, - which respectively remove from the subset all master-eligible nodes, all data - nodes, all ingest nodes, and all coordinating-only nodes. +* `master:true`, `data:true`, `ingest:true`, `voting_only:true` or + `coordinating_only:true`, which respectively add to the subset all + master-eligible nodes, all data nodes, all ingest nodes, all voting-only + nodes, and all coordinating-only nodes. +* `master:false`, `data:false`, `ingest:false`, `voting_only:true`, or + `coordinating_only:false`, which respectively remove from the subset all + master-eligible nodes, all data nodes, all ingest nodes, all voting-only + nodes and all coordinating-only nodes. * a pair of patterns, using `*` wildcards, of the form `attrname:attrvalue`, which adds to the subset all nodes with a custom node attribute whose name and value match the respective patterns. Custom node attributes are @@ -46,6 +48,9 @@ means that filters such as `master:false` which remove nodes from the chosen subset are only useful if they come after some other filters. When used on its own, `master:false` selects no nodes. +NOTE: The `voting_only` role requires the {default-dist} of Elasticsearch and +is not supported in the {oss-dist}. + Here are some examples of the use of node filters with the <> APIs. @@ -69,6 +74,7 @@ GET /_nodes/10.0.0.* GET /_nodes/_all,master:false GET /_nodes/data:true,ingest:true GET /_nodes/coordinating_only:true +GET /_nodes/master:true,voting_only:false # Select nodes by custom attribute (e.g. with something like `node.attr.rack: 2` in the configuration file) GET /_nodes/rack:2 GET /_nodes/ra*:2 diff --git a/docs/reference/cluster/stats.asciidoc b/docs/reference/cluster/stats.asciidoc index 791fc2414f331..259d8ae2bba11 100644 --- a/docs/reference/cluster/stats.asciidoc +++ b/docs/reference/cluster/stats.asciidoc @@ -109,7 +109,8 @@ Will return, for example: "data": 1, "coordinating_only": 0, "master": 1, - "ingest": 1 + "ingest": 1, + "voting_only": 0 }, "versions": [ "{version}" @@ -207,6 +208,7 @@ Will return, for example: // TESTRESPONSE[s/"plugins": \[[^\]]*\]/"plugins": $body.$_path/] // TESTRESPONSE[s/"network_types": \{[^\}]*\}/"network_types": $body.$_path/] // TESTRESPONSE[s/"discovery_types": \{[^\}]*\}/"discovery_types": $body.$_path/] +// TESTRESPONSE[s/"count": \{[^\}]*\}/"count": $body.$_path/] // TESTRESPONSE[s/"packaging_types": \[[^\]]*\]/"packaging_types": $body.$_path/] // TESTRESPONSE[s/: true|false/: $body.$_path/] // TESTRESPONSE[s/: (\-)?[0-9]+/: $body.$_path/] @@ -217,7 +219,10 @@ Will return, for example: // see an exhaustive list anyway. // 2. Similarly, ignore the contents of `network_types`, `discovery_types`, and // `packaging_types`. -// 3. All of the numbers and strings on the right hand side of *every* field in +// 3. Ignore the contents of the (nodes) count object, as what's shown here +// depends on the license. Voting-only nodes are e.g. only shown when this +// test runs with a basic license. +// 4. All of the numbers and strings on the right hand side of *every* field in // the response are ignored. So we're really only asserting things about the // the shape of this response, not the values in it. diff --git a/docs/reference/commands/certutil.asciidoc b/docs/reference/commands/certutil.asciidoc index 6f4d3224d7aeb..07a3f96738dac 100644 --- a/docs/reference/commands/certutil.asciidoc +++ b/docs/reference/commands/certutil.asciidoc @@ -103,7 +103,7 @@ which prompts you for details about each instance. Alternatively, you can use the `--in` parameter to specify a YAML file that contains details about the instances. -The `cert` mode produces a single zip file which contains the CSRs and the +The `csr` mode produces a single zip file which contains the CSRs and the private keys for each instance. Each CSR is provided as a standard PEM encoding of a PKCS#10 CSR. Each key is provided as a PEM encoding of an RSA private key. diff --git a/docs/reference/commands/shard-tool.asciidoc b/docs/reference/commands/shard-tool.asciidoc index 6fca1355a27be..c13c8d3db6a36 100644 --- a/docs/reference/commands/shard-tool.asciidoc +++ b/docs/reference/commands/shard-tool.asciidoc @@ -51,14 +51,14 @@ $ bin/elasticsearch-shard remove-corrupted-data --index twitter --shard-id 0 Please make a complete backup of your index before using this tool. -Opening Lucene index at /var/lib/elasticsearchdata/nodes/0/indices/P45vf_YQRhqjfwLMUvSqDw/0/index/ +Opening Lucene index at /var/lib/elasticsearchdata/indices/P45vf_YQRhqjfwLMUvSqDw/0/index/ - >> Lucene index is corrupted at /var/lib/elasticsearchdata/nodes/0/indices/P45vf_YQRhqjfwLMUvSqDw/0/index/ + >> Lucene index is corrupted at /var/lib/elasticsearchdata/indices/P45vf_YQRhqjfwLMUvSqDw/0/index/ -Opening translog at /var/lib/elasticsearchdata/nodes/0/indices/P45vf_YQRhqjfwLMUvSqDw/0/translog/ +Opening translog at /var/lib/elasticsearchdata/indices/P45vf_YQRhqjfwLMUvSqDw/0/translog/ - >> Translog is clean at /var/lib/elasticsearchdata/nodes/0/indices/P45vf_YQRhqjfwLMUvSqDw/0/translog/ + >> Translog is clean at /var/lib/elasticsearchdata/indices/P45vf_YQRhqjfwLMUvSqDw/0/translog/ Corrupted Lucene index segments found - 32 documents will be lost. @@ -93,7 +93,7 @@ POST /_cluster/reroute You must accept the possibility of data loss by changing parameter `accept_data_loss` to `true`. -Deleted corrupt marker corrupted_FzTSBSuxT7i3Tls_TgwEag from /var/lib/elasticsearchdata/nodes/0/indices/P45vf_YQRhqjfwLMUvSqDw/0/index/ +Deleted corrupt marker corrupted_FzTSBSuxT7i3Tls_TgwEag from /var/lib/elasticsearchdata/indices/P45vf_YQRhqjfwLMUvSqDw/0/index/ -------------------------------------------------- diff --git a/docs/reference/data-frames/apis/delete-transform.asciidoc b/docs/reference/data-frames/apis/delete-transform.asciidoc index 9fb3d8558c0ea..d772bc3c15d89 100644 --- a/docs/reference/data-frames/apis/delete-transform.asciidoc +++ b/docs/reference/data-frames/apis/delete-transform.asciidoc @@ -8,25 +8,42 @@ Delete {dataframe-transforms} ++++ -Deletes an existing {dataframe-transform}. +beta[] +Deletes an existing {dataframe-transform}. -==== Request +[discrete] +[[delete-data-frame-transform-request]] +==== {api-request-title} `DELETE _data_frame/transforms/` -//==== Description +[discrete] +[[delete-data-frame-transform-prereqs]] +==== {api-prereq-title} -==== Path Parameters +* If the {es} {security-features} are enabled, you must have +`manage_data_frame_transforms` cluster privileges to use this API. The built-in +`data_frame_transforms_admin` role has these privileges. For more information, +see {stack-ov}/security-privileges.html[Security privileges] and +{stack-ov}/built-in-roles.html[Built-in roles]. -`data_frame_transform_id` (required):: - (string) Identifier for the {dataframe-transform}. +[discrete] +[[delete-data-frame-transform-desc]] +==== {api-description-title} -//===== Authorization +NOTE: Before you can delete the {dataframe-transform}, you must stop it. -==== Examples +[discrete] +[[delete-data-frame-transform-path-parms]] +==== {api-path-parms-title} + +`` (Required):: + (string) Identifier for the {dataframe-transform}. -The following example deletes the `ecommerce_transform` {dataframe-transform}: +[discrete] +[[delete-data-frame-transform-examples]] +==== {api-examples-title} [source,js] -------------------------------------------------- diff --git a/docs/reference/data-frames/apis/get-transform-stats.asciidoc b/docs/reference/data-frames/apis/get-transform-stats.asciidoc index 09c383f249488..889a109b8a376 100644 --- a/docs/reference/data-frames/apis/get-transform-stats.asciidoc +++ b/docs/reference/data-frames/apis/get-transform-stats.asciidoc @@ -8,10 +8,13 @@ Get {dataframe-transform} statistics ++++ -Retrieves usage information for {dataframe-transforms}. +beta[] +Retrieves usage information for {dataframe-transforms}. -==== Request +[discrete] +[[get-data-frame-transform-stats-request]] +==== {api-request-title} `GET _data_frame/transforms//_stats` @@ -24,27 +27,90 @@ Retrieves usage information for {dataframe-transforms}. `GET _data_frame/transforms/*/_stats` + -//===== Description +[discrete] +[[get-data-frame-transform-stats-prereqs]] +==== {api-prereq-title} + +* If the {es} {security-features} are enabled, you must have +`monitor_data_frame_transforms` cluster privileges to use this API. The built-in +`data_frame_transforms_user` role has these privileges. For more information, +see {stack-ov}/security-privileges.html[Security privileges] and +{stack-ov}/built-in-roles.html[Built-in roles]. + +[discrete] +[[get-data-frame-transform-stats-desc]] +==== {api-description-title} -==== Path Parameters +You can get statistics for multiple {dataframe-transforms} in a single API +request by using a comma-separated list of identifiers or a wildcard expression. +You can get statistics for all {dataframe-transforms} by using `_all`, by +specifying `*` as the ``, or by omitting the +``. -`data_frame_transform_id`:: +[discrete] +[[get-data-frame-transform-stats-path-parms]] +==== {api-path-parms-title} + +`` (Optional):: (string) Identifier for the {dataframe-transform}. It can be a {dataframe-transform} identifier or a wildcard expression. If you do not specify one of these options, the API returns information for all {dataframe-transforms}. - - -==== Results - -The API returns the following information: + +[discrete] +[[get-data-frame-transform-stats-query-parms]] +==== {api-query-parms-title} + +`allow_no_match` (Optional):: + (boolean) Specifies what to do when the request: ++ +-- +* Contains wildcard expressions and there are no {dataframe-transforms} that match. +* Contains the `_all` string or no identifiers and there are no matches. +* Contains wildcard expressions and there are only partial matches. + +The default value is `true`, which returns an empty `transforms` array when +there are no matches and the subset of results when there are partial matches. +If this parameter is `false`, the request returns a `404` status code when there +are no matches or only partial matches. +-- + +`from` (Optional):: + (integer) Skips the specified number of {dataframe-transforms}. The + default value is `0`. + +`size` (Optional):: + (integer) Specifies the maximum number of {dataframe-transforms} to obtain. + The default value is `100`. + +[discrete] +[[get-data-frame-transform-stats-response]] +==== {api-response-body-title} `transforms`:: - (array) An array of statistics objects for {dataframe-transforms}. + (array) An array of statistics objects for {dataframe-transforms}, which are + sorted by the `id` value in ascending order. + +[[get-data-frame-transform-stats-response-codes]] +==== {api-response-codes-title} + +`404` (Missing resources):: + If `allow_no_match` is `false`, this code indicates that there are no + resources that match the request or only partial matches for the request. + +[discrete] +[[get-data-frame-transform-stats-example]] +==== Examples -//==== Authorization +The following example skips for the first five {dataframe-transforms} and +gets usage information for a maximum of ten results: -==== Examples +[source,js] +-------------------------------------------------- +GET _data_frame/transforms/_stats?from=5&size=10 +-------------------------------------------------- +// CONSOLE +// TEST[skip:todo] The following example gets usage information for the `ecommerce_transform` {dataframe-transform}: diff --git a/docs/reference/data-frames/apis/get-transform.asciidoc b/docs/reference/data-frames/apis/get-transform.asciidoc index e2b5c5eccb7da..bf7901c191e33 100644 --- a/docs/reference/data-frames/apis/get-transform.asciidoc +++ b/docs/reference/data-frames/apis/get-transform.asciidoc @@ -8,10 +8,13 @@ Get {dataframe-transforms} ++++ -Retrieves configuration information for {dataframe-transforms}. +beta[] +Retrieves configuration information for {dataframe-transforms}. -==== Request +[discrete] +[[get-data-frame-transform-request]] +==== {api-request-title} `GET _data_frame/transforms/` + @@ -23,39 +26,89 @@ Retrieves configuration information for {dataframe-transforms}. `GET _data_frame/transforms/*` -//===== Description +[discrete] +[[get-data-frame-transform-prereqs]] +==== {api-prereq-title} + +* If the {es} {security-features} are enabled, you must have +`monitor_data_frame_transforms` cluster privileges to use this API. The built-in +`data_frame_transforms_user` role has these privileges. For more information, +see {stack-ov}/security-privileges.html[Security privileges] and +{stack-ov}/built-in-roles.html[Built-in roles]. + +[discrete] +[[get-data-frame-transform-desc]] +==== {api-description-title} -==== Path Parameters +You can get information for multiple {dataframe-transforms} in a single API +request by using a comma-separated list of identifiers or a wildcard expression. +You can get information for all {dataframe-transforms} by using `_all`, by +specifying `*` as the ``, or by omitting the +``. -`data_frame_transform_id`:: +[discrete] +[[get-data-frame-transform-path-parms]] +==== {api-path-parms-title} + +`` (Optional):: (string) Identifier for the {dataframe-transform}. It can be a {dataframe-transform} identifier or a wildcard expression. If you do not specify one of these options, the API returns information for all {dataframe-transforms}. + +[discrete] +[[get-data-frame-transform-query-parms]] +==== {api-query-parms-title} + +`allow_no_match` (Optional):: +(boolean) Specifies what to do when the request: ++ +-- +* Contains wildcard expressions and there are no {dataframe-transforms} that match. +* Contains the `_all` string or no identifiers and there are no matches. +* Contains wildcard expressions and there are only partial matches. + +The default value is `true`, which returns an empty `transforms` array when +there are no matches and the subset of results when there are partial matches. +If this parameter is `false`, the request returns a `404` status code when there +are no matches or only partial matches. +-- + +`from` (Optional):: + (integer) Skips the specified number of {dataframe-transforms}. The + default value is `0`. + +`size` (Optional):: + (integer) Specifies the maximum number of {dataframe-transforms} to obtain. + The default value is `100`. + +[discrete] +[[get-data-frame-transform-response]] +==== {api-response-body-title} -//// -==== Request Body - - `page`:: - `from`::: - (integer) Skips the specified number of {dataframe-transforms}. The - default value is `0`. - - `size`::: - (integer) Specifies the maximum number of {dataframe-transforms} to obtain. - The default value is `100`. -//// - -==== Results +`transforms`:: + (array) An array of transform resources, which are sorted by the `id` value in + ascending order. + +[[get-data-frame-transform-response-codes]] +==== {api-response-codes-title} -The API returns the following information: +`404` (Missing resources):: + If `allow_no_match` is `false`, this code indicates that there are no + resources that match the request or only partial matches for the request. -`transforms`:: - (array) An array of transform resources. +[discrete] +[[get-data-frame-transform-example]] +==== {api-examples-title} -//==== Authorization +The following example retrieves information about a maximum of ten transforms: -==== Examples +[source,js] +-------------------------------------------------- +GET _data_frame/transforms?size=10 +-------------------------------------------------- +// CONSOLE +// TEST[skip:setup kibana sample data] The following example gets configuration information for the `ecommerce_transform` {dataframe-transform}: diff --git a/docs/reference/data-frames/apis/index.asciidoc b/docs/reference/data-frames/apis/index.asciidoc index a75f44e995158..b3fa17e3c48be 100644 --- a/docs/reference/data-frames/apis/index.asciidoc +++ b/docs/reference/data-frames/apis/index.asciidoc @@ -8,11 +8,11 @@ === {dataframe-transforms-cap} * <> -* <>, +* <> or <> -* <>, -<> -* <>, <> +* <> or +<> +* <> or <> //CREATE include::put-transform.asciidoc[] diff --git a/docs/reference/data-frames/apis/pivotresource.asciidoc b/docs/reference/data-frames/apis/pivotresource.asciidoc new file mode 100644 index 0000000000000..64c49af03ea77 --- /dev/null +++ b/docs/reference/data-frames/apis/pivotresource.asciidoc @@ -0,0 +1,26 @@ +[role="xpack"] +[testenv="basic"] +[[data-frame-transform-pivot]] +=== Pivot resources + +A pivot configuration object has the following properties: + +`group_by` (required):: (object) Defines how to group the data. More than one grouping can be defined per pivot. The following groupings are supported: +* {ref}/search-aggregations-bucket-composite-aggregation.html#_terms[Terms] +* {ref}/search-aggregations-bucket-composite-aggregation.html#_histogram[Histogram] +* {ref}/search-aggregations-bucket-composite-aggregation.html#_date_histogram[Date Histogram] + +`aggregations` (required):: (object) Defines how to aggregate the grouped data. +The following aggregations are supported: +* {ref}/search-aggregations-metrics-avg-aggregation.html[Average] +* {ref}/search-aggregations-metrics-weight-avg-aggregation.html[Weighted Average] +* {ref}/search-aggregations-metrics-cardinality-aggregation.html[Cardinality] +* {ref}/search-aggregations-metrics-geocentroid-aggregation.html[Geo Centroid] +* {ref}/search-aggregations-metrics-max-aggregation.html[Max] +* {ref}/search-aggregations-metrics-min-aggregation.html[Min] +* {ref}/search-aggregations-metrics-scripted-metric-aggregation.html[Scripted Metric] +* {ref}/search-aggregations-metrics-sum-aggregation.html[Sum] +* {ref}/search-aggregations-metrics-valuecount-aggregation.html[Value Count] +* {ref}/search-aggregations-pipeline-bucket-script-aggregation.html[Bucket Script] + +//For more information, see {stack-ov}/ml-dataframes.html[dataframes-cap}]. \ No newline at end of file diff --git a/docs/reference/data-frames/apis/preview-transform.asciidoc b/docs/reference/data-frames/apis/preview-transform.asciidoc index cf3f3cb96f9a7..a4338d0ef60b3 100644 --- a/docs/reference/data-frames/apis/preview-transform.asciidoc +++ b/docs/reference/data-frames/apis/preview-transform.asciidoc @@ -8,29 +8,41 @@ Preview {dataframe-transforms} ++++ -Previews a {dataframe-transform}. +beta[] +Previews a {dataframe-transform}. -==== Request +[discrete] +[[preview-data-frame-transform-request]] +==== {api-request-title} `POST _data_frame/transforms/_preview` -//==== Description -//==== Path Parameters -//==== Authorization +[discrete] +[[preview-data-frame-transform-prereq]] +==== {api-prereq-title} -==== Request Body +* If the {es} {security-features} are enabled, you must have +`manage_data_frame_transforms` cluster privileges to use this API. The built-in +`data_frame_transforms_admin` role has these privileges. You must also have +`read` and `view_index_metadata` privileges on the source index for the +{dataframe-transform}. For more information, see +{stack-ov}/security-privileges.html[Security privileges] and +{stack-ov}/built-in-roles.html[Built-in roles]. -`source`:: The source index or index pattern. +[discrete] +[[preview-data-frame-transform-request-body]] +==== {api-request-body-title} -`pivot`:: Defines the pivot function `group by` fields and the aggregation to -reduce the data. +`source` (Required):: + (object) The source index or index pattern. +`pivot` (Required):: + (object) Defines the pivot function `group by` fields and the aggregation to + reduce the data. See <>. -==== Examples - -The following example obtains a preview of a {dataframe-transform} on the {kib} -eCommerce sample data: +[discrete] +==== {api-examples-title} [source,js] -------------------------------------------------- @@ -78,7 +90,17 @@ The data that is returned for this example is as follows: "customer_id" : "12" } ... - ] + ], + "mappings": { + "properties": { + "max_price": { + "type": "double" + }, + "customer_id": { + "type": "keyword" + } + } + } } ---- // NOTCONSOLE diff --git a/docs/reference/data-frames/apis/put-transform.asciidoc b/docs/reference/data-frames/apis/put-transform.asciidoc index f452c38ab4c94..6910cb85a25f5 100644 --- a/docs/reference/data-frames/apis/put-transform.asciidoc +++ b/docs/reference/data-frames/apis/put-transform.asciidoc @@ -8,14 +8,31 @@ Create {dataframe-transforms} ++++ -Instantiates a {dataframe-transform}. +beta[] +Instantiates a {dataframe-transform}. -==== Request +[discrete] +[[put-data-frame-transform-request]] +==== {api-request-title} `PUT _data_frame/transforms/` -===== Description +[discrete] +[[put-data-frame-transform-prereqs]] +==== {api-prereq-title} + +* If the {es} {security-features} are enabled, you must have +`manage_data_frame_transforms` cluster privileges to use this API. The built-in +`data_frame_transforms_admin` role has these privileges. You must also +have `read` and `view_index_metadata` privileges on the source index and `read`, +`create_index`, and `index` privileges on the destination index. For more +information, see {stack-ov}/security-privileges.html[Security privileges] and +{stack-ov}/built-in-roles.html[Built-in roles]. + +[discrete] +[[put-data-frame-transform-desc]] +==== {api-description-title} IMPORTANT: You must use {kib} or this API to create a {dataframe-transform}. Do not put a {dataframe-transform} directly into any @@ -23,33 +40,37 @@ IMPORTANT: You must use {kib} or this API to create a {dataframe-transform}. If {es} {security-features} are enabled, do not give users any privileges on `.data-frame-internal*` indices. -==== Path Parameters +[discrete] +[[put-data-frame-transform-path-parms]] +==== {api-path-parms-title} -`data_frame_transform_id` (required):: +`` (Required):: (string) Identifier for the {dataframe-transform}. This identifier can contain lowercase alphanumeric characters (a-z and 0-9), hyphens, and underscores. It must start and end with alphanumeric characters. +[discrete] +[[put-data-frame-transform-request-body]] +==== {api-request-body-title} -==== Request Body - -`source` (required):: (object) The source configuration, consisting of `index` and optionally -a `query`. - -`dest` (required):: (object) The destination configuration, consisting of `index`. - -`pivot`:: (object) Defines the pivot function `group by` fields and the aggregation to -reduce the data. - -`description`:: Optional free text description of the data frame transform +`description` (Optional):: + (string) Free text description of the {dataframe-transform}. +`dest` (Required):: + (object) The destination configuration, which consists of `index` and + optionally a `pipeline` id. -//==== Authorization +`pivot` (Optional):: + (object) Defines the pivot function `group by` fields and the aggregation to + reduce the data. See <>. -==== Examples +`source` (Required):: + (object) The source configuration, which consists of `index` and optionally + a `query`. -The following example creates a {dataframe-transform} for the {kib} eCommerce -sample data: +[discrete] +[[put-data-frame-transform-example]] +==== {api-examples-title} [source,js] -------------------------------------------------- @@ -66,7 +87,8 @@ PUT _data_frame/transforms/ecommerce_transform } }, "dest": { - "index": "kibana_sample_data_ecommerce_transform" + "index": "kibana_sample_data_ecommerce_transform", + "pipeline": "add_timestamp_pipeline" }, "pivot": { "group_by": { @@ -88,7 +110,7 @@ PUT _data_frame/transforms/ecommerce_transform } -------------------------------------------------- // CONSOLE -// TEST[skip:add sample kibana data] +// TEST[skip: https://github.com/elastic/elasticsearch/issues/43271] When the transform is created, you receive the following results: [source,js] @@ -97,4 +119,4 @@ When the transform is created, you receive the following results: "acknowledged" : true } ---- -// NOTCONSOLE +// TESTRESPONSE diff --git a/docs/reference/data-frames/apis/start-transform.asciidoc b/docs/reference/data-frames/apis/start-transform.asciidoc index d200ef34f587d..b76bcb0dd4796 100644 --- a/docs/reference/data-frames/apis/start-transform.asciidoc +++ b/docs/reference/data-frames/apis/start-transform.asciidoc @@ -8,27 +8,39 @@ Start {dataframe-transforms} ++++ +beta[] + Starts one or more {dataframe-transforms}. -==== Request +[discrete] +[[start-data-frame-transform-request]] +==== {api-request-title} `POST _data_frame/transforms//_start` -//==== Description +[discrete] +[[start-data-frame-transform-prereqs]] +==== {api-prereq-title} + +* If the {es} {security-features} are enabled, you must have +`manage_data_frame_transforms` cluster privileges to use this API. You must also +have `view_index_metadata` privileges on the source index for the +{dataframe-transform}. For more information, see +{stack-ov}/security-privileges.html[Security privileges] and +{stack-ov}/built-in-roles.html[Built-in roles]. -==== Path Parameters +[discrete] +[[start-data-frame-transform-path-parms]] +==== {api-path-parms-title} -`data_frame_transform_id` (required):: +`` (Required):: (string) Identifier for the {dataframe-transform}. This identifier can contain lowercase alphanumeric characters (a-z and 0-9), hyphens, and underscores. It must start and end with alphanumeric characters. -//==== Request Body -//==== Authorization - -==== Examples - -The following example starts the `ecommerce_transform` {dataframe-transform}: +[discrete] +[[start-data-frame-transform-example]] +==== {api-examples-title} [source,js] -------------------------------------------------- @@ -41,7 +53,7 @@ When the {dataframe-transform} starts, you receive the following results: [source,js] ---- { - "started" : true + "acknowledged" : true } ---- // TESTRESPONSE diff --git a/docs/reference/data-frames/apis/stop-transform.asciidoc b/docs/reference/data-frames/apis/stop-transform.asciidoc index 4e89579b52ec1..80c2654babe0d 100644 --- a/docs/reference/data-frames/apis/stop-transform.asciidoc +++ b/docs/reference/data-frames/apis/stop-transform.asciidoc @@ -8,9 +8,13 @@ Stop {dataframe-transforms} ++++ +beta[] + Stops one or more {dataframe-transforms}. -==== Request +[discrete] +[[stop-data-frame-transform-request]] +==== {api-request-title} `POST _data_frame/transforms//_stop` + @@ -18,37 +22,81 @@ Stops one or more {dataframe-transforms}. `POST _data_frame/transforms/_all/_stop` +[discrete] +[[stop-data-frame-transform-prereq]] +==== {api-prereq-title} + +* If the {es} {security-features} are enabled, you must have +`manage_data_frame_transforms` cluster privileges to use this API. The built-in +`data_frame_transforms_admin` role has these privileges. For more information, +see {stack-ov}/security-privileges.html[Security privileges] and +{stack-ov}/built-in-roles.html[Built-in roles]. + +[discrete] +[[stop-data-frame-transform-desc]] +==== {api-description-title} -==== Description You can stop multiple {dataframe-transforms} in a single API request by using a comma-separated list of {dataframe-transforms} or a wildcard expression. -All {dataframe-transforms} can be stopped by using `_all` or `*` as the ``. +All {dataframe-transforms} can be stopped by using `_all` or `*` as the +``. -==== Path Parameters +[discrete] +[[stop-data-frame-transform-path-parms]] +==== {api-path-parms-title} -`data_frame_transform_id` (required):: +`` (Required):: (string) Identifier for the {dataframe-transform}. This identifier can contain lowercase alphanumeric characters (a-z and 0-9), hyphens, and underscores. It must start and end with alphanumeric characters. -==== Query Parameters - -`wait_for_completion`:: - (boolean) If set to true, causes the API to block until the indexer state completely stops. If set to false, the API returns immediately and the indexer will be stopped asynchronously in the background. Defaults to `false`. - - `timeout`:: - (time value) If `wait_for_completion=true`, the API blocks for (at maximum) - the specified duration while waiting for the job to stop. If more than - `timeout` time has passed, the API throws a timeout exception. Even if a - timeout exception is thrown, the stop request is still processing and - eventually moves the job to `STOPPED`. The timeout simply means the API call itself timed out while waiting for the status change. Defaults to `30s` - -//==== Request Body -//==== Authorization - -==== Examples - -The following example stops the `ecommerce_transform` {dataframe-transform}: +[discrete] +[[stop-data-frame-transform-query-parms]] +==== {api-query-parms-title} + +`allow_no_match` (Optional):: +(boolean) Specifies what to do when the request: ++ +-- +* Contains wildcard expressions and there are no {dataframe-transforms} that match. +* Contains the `_all` string or no identifiers and there are no matches. +* Contains wildcard expressions and there are only partial matches. + +The default value is `true`, which returns a successful acknowledgement message +when there are no matches. When there are only partial matches, the API stops +the appropriate {dataframe-transforms}. For example, if the request contains +`test-id1*,test-id2*` as the identifiers and there are no {dataframe-transforms} +that match `test-id2*`, the API nonetheless stops the {dataframe-transforms} +that match `test-id1*`. + +If this parameter is `false`, the request returns a `404` status code when there +are no matches or only partial matches. +-- + +`timeout` (Optional):: + (time value) If `wait_for_completion=true`, the API blocks for (at maximum) + the specified duration while waiting for the transform to stop. If more than + `timeout` time has passed, the API throws a timeout exception. Even if a + timeout exception is thrown, the stop request is still processing and + eventually moves the transform to `STOPPED`. The timeout simply means the API + call itself timed out while waiting for the status change. Defaults to `30s` + +`wait_for_completion` (Optional):: + (boolean) If set to `true`, causes the API to block until the indexer state + completely stops. If set to `false`, the API returns immediately and the + indexer will be stopped asynchronously in the background. Defaults to `false`. + +[discrete] +[[stop-data-frame-transform-response-codes]] +==== {api-response-codes-title} + +`404` (Missing resources):: + If `allow_no_match` is `false`, this code indicates that there are no + resources that match the request or only partial matches for the request. + +[discrete] +[[stop-data-frame-transform-example]] +==== {api-examples-title} [source,js] -------------------------------------------------- @@ -61,7 +109,7 @@ When the {dataframe-transform} stops, you receive the following results: [source,js] ---- { - "stopped" : true + "acknowledged" : true } ---- // TESTRESPONSE diff --git a/docs/reference/docs/data-replication.asciidoc b/docs/reference/docs/data-replication.asciidoc index b83858cecfd8d..28389fb05ba94 100644 --- a/docs/reference/docs/data-replication.asciidoc +++ b/docs/reference/docs/data-replication.asciidoc @@ -5,12 +5,12 @@ [float] === Introduction -Each index in Elasticsearch is <> +Each index in Elasticsearch is <> and each shard can have multiple copies. These copies are known as a _replication group_ and must be kept in sync when documents are added or removed. If we fail to do so, reading from one copy will result in very different results than reading from another. The process of keeping the shard copies in sync and serving reads from them is what we call the _data replication model_. -Elasticsearch’s data replication model is based on the _primary-backup model_ and is described very well in the +Elasticsearch’s data replication model is based on the _primary-backup model_ and is described very well in the https://www.microsoft.com/en-us/research/publication/pacifica-replication-in-log-based-distributed-storage-systems/[PacificA paper] of Microsoft Research. That model is based on having a single copy from the replication group that acts as the primary shard. The other copies are called _replica shards_. The primary serves as the main entry point for all indexing operations. It is in charge of @@ -23,7 +23,7 @@ it has for various interactions between write and read operations. [float] === Basic write model -Every indexing operation in Elasticsearch is first resolved to a replication group using <>, +Every indexing operation in Elasticsearch is first resolved to a replication group using <>, typically based on the document ID. Once the replication group has been determined, the operation is forwarded internally to the current _primary shard_ of the group. The primary shard is responsible for validating the operation and forwarding it to the other replicas. Since replicas can be offline, the primary @@ -50,7 +50,7 @@ configuration mistake could cause an operation to fail on a replica despite it b are infrequent but the primary has to respond to them. In the case that the primary itself fails, the node hosting the primary will send a message to the master about it. The indexing -operation will wait (up to 1 minute, by <>) for the master to promote one of the replicas to be a +operation will wait (up to 1 minute, by <>) for the master to promote one of the replicas to be a new primary. The operation will then be forwarded to the new primary for processing. Note that the master also monitors the health of the nodes and may decide to proactively demote a primary. This typically happens when the node holding the primary is isolated from the cluster by a networking issue. See <> for more details. @@ -60,8 +60,8 @@ when executing it on the replica shards. This may be caused by an actual failure issue preventing the operation from reaching the replica (or preventing the replica from responding). All of these share the same end result: a replica which is part of the in-sync replica set misses an operation that is about to be acknowledged. In order to avoid violating the invariant, the primary sends a message to the master requesting -that the problematic shard be removed from the in-sync replica set. Only once removal of the shard has been acknowledged -by the master does the primary acknowledge the operation. Note that the master will also instruct another node to start +that the problematic shard be removed from the in-sync replica set. Only once removal of the shard has been acknowledged +by the master does the primary acknowledge the operation. Note that the master will also instruct another node to start building a new shard copy in order to restore the system to a healthy state. [[demoted-primary]] @@ -72,13 +72,13 @@ will be rejected by the replicas. When the primary receives a response from the it is no longer the primary then it will reach out to the master and will learn that it has been replaced. The operation is then routed to the new primary. -.What happens if there are no replicas? +.What happens if there are no replicas? ************ This is a valid scenario that can happen due to index configuration or simply because all the replicas have failed. In that case the primary is processing operations without any external validation, which may seem problematic. On the other hand, the primary cannot fail other shards on its own but request the master to do -so on its behalf. This means that the master knows that the primary is the only single good copy. We are therefore guaranteed -that the master will not promote any other (out-of-date) shard copy to be a new primary and that any operation indexed +so on its behalf. This means that the master knows that the primary is the only single good copy. We are therefore guaranteed +that the master will not promote any other (out-of-date) shard copy to be a new primary and that any operation indexed into the primary will not be lost. Of course, since at that point we are running with only single copy of the data, physical hardware issues can cause data loss. See <> for some mitigation options. ************ @@ -91,7 +91,7 @@ take non-trivial CPU power. One of the beauties of the primary-backup model is t (with the exception of in-flight operations). As such, a single in-sync copy is sufficient to serve read requests. When a read request is received by a node, that node is responsible for forwarding it to the nodes that hold the relevant shards, -collating the responses, and responding to the client. We call that node the _coordinating node_ for that request. The basic flow +collating the responses, and responding to the client. We call that node the _coordinating node_ for that request. The basic flow is as follows: . Resolve the read requests to the relevant shards. Note that since most searches will be sent to one or more indices, @@ -153,8 +153,8 @@ Dirty reads:: An isolated primary can expose writes that will not be acknowledge [float] === The Tip of the Iceberg -This document provides a high level overview of how Elasticsearch deals with data. Of course, there is much much more -going on under the hood. Things like primary terms, cluster state publishing, and master election all play a role in +This document provides a high level overview of how Elasticsearch deals with data. Of course, there is much much more +going on under the hood. Things like primary terms, cluster state publishing, and master election all play a role in keeping this system behaving correctly. This document also doesn't cover known and important bugs (both closed and open). We recognize that https://github.com/elastic/elasticsearch/issues?q=label%3Aresiliency[GitHub is hard to keep up with]. To help people stay on top of those, we maintain a dedicated https://www.elastic.co/guide/en/elasticsearch/resiliency/current/index.html[resiliency page] diff --git a/docs/reference/docs/delete-by-query.asciidoc b/docs/reference/docs/delete-by-query.asciidoc index f8cb84ab790fa..4dbdd18ca02d3 100644 --- a/docs/reference/docs/delete-by-query.asciidoc +++ b/docs/reference/docs/delete-by-query.asciidoc @@ -571,11 +571,11 @@ sub-request proportionally. * Due to the nature of `slices` each sub-request won't get a perfectly even portion of the documents. All documents will be addressed, but some slices may be larger than others. Expect larger slices to have a more even distribution. -* Parameters like `requests_per_second` and `size` on a request with `slices` -are distributed proportionally to each sub-request. Combine that with the point -above about distribution being uneven and you should conclude that the using -`size` with `slices` might not result in exactly `size` documents being -deleted. +* Parameters like `requests_per_second` and `max_docs` on a request with +slices` are distributed proportionally to each sub-request. Combine that with +the point above about distribution being uneven and you should conclude that +using `max_docs` with `slices` might not result in exactly `max_docs` documents +being deleted. * Each sub-request gets a slightly different snapshot of the source index though these are all taken at approximately the same time. diff --git a/docs/reference/docs/reindex.asciidoc b/docs/reference/docs/reindex.asciidoc index e96c262d67bb4..7717506a2d23e 100644 --- a/docs/reference/docs/reindex.asciidoc +++ b/docs/reference/docs/reindex.asciidoc @@ -190,14 +190,14 @@ not a good idea to rely on this behavior. Instead, make sure that IDs are unique using a script. It's also possible to limit the number of processed documents by setting -`size`. This will only copy a single document from `twitter` to +`max_docs`. This will only copy a single document from `twitter` to `new_twitter`: [source,js] -------------------------------------------------- POST _reindex { - "size": 1, + "max_docs": 1, "source": { "index": "twitter" }, @@ -211,14 +211,14 @@ POST _reindex If you want a particular set of documents from the `twitter` index you'll need to use `sort`. Sorting makes the scroll less efficient but in some contexts -it's worth it. If possible, prefer a more selective query to `size` and `sort`. +it's worth it. If possible, prefer a more selective query to `max_docs` and `sort`. This will copy 10000 documents from `twitter` into `new_twitter`: [source,js] -------------------------------------------------- POST _reindex { - "size": 10000, + "max_docs": 10000, "source": { "index": "twitter", "sort": { "date": "desc" } @@ -1115,11 +1115,11 @@ sub-request proportionally. * Due to the nature of `slices` each sub-request won't get a perfectly even portion of the documents. All documents will be addressed, but some slices may be larger than others. Expect larger slices to have a more even distribution. -* Parameters like `requests_per_second` and `size` on a request with `slices` -are distributed proportionally to each sub-request. Combine that with the point -above about distribution being uneven and you should conclude that the using -`size` with `slices` might not result in exactly `size` documents being -reindexed. +* Parameters like `requests_per_second` and `max_docs` on a request with +`slices` are distributed proportionally to each sub-request. Combine that with +the point above about distribution being uneven and you should conclude that +using `max_docs` with `slices` might not result in exactly `max_docs` documents +being reindexed. * Each sub-request gets a slightly different snapshot of the source index, though these are all taken at approximately the same time. @@ -1236,7 +1236,7 @@ to load only the existing data into the new index and rename any fields if neede ---------------------------------------------------------------- POST _reindex { - "size": 10, + "max_docs": 10, "source": { "index": "twitter", "query": { diff --git a/docs/reference/docs/termvectors.asciidoc b/docs/reference/docs/termvectors.asciidoc index 1e2748916005c..8ec4019fe8861 100644 --- a/docs/reference/docs/termvectors.asciidoc +++ b/docs/reference/docs/termvectors.asciidoc @@ -207,7 +207,6 @@ Response: { "_id": "1", "_index": "twitter", - "_type": "_doc", "_version": 1, "found": true, "took": 6, @@ -344,7 +343,6 @@ Response: -------------------------------------------------- { "_index": "twitter", - "_type": "_doc", "_version": 0, "found": true, "took": 6, @@ -415,7 +413,6 @@ Response: -------------------------------------------------- { "_index": "imdb", - "_type": "_doc", "_version": 0, "found": true, "term_vectors": { diff --git a/docs/reference/docs/update-by-query.asciidoc b/docs/reference/docs/update-by-query.asciidoc index 883f6ad2a29e3..163ce17194268 100644 --- a/docs/reference/docs/update-by-query.asciidoc +++ b/docs/reference/docs/update-by-query.asciidoc @@ -602,11 +602,11 @@ sub-request proportionally. * Due to the nature of `slices` each sub-request won't get a perfectly even portion of the documents. All documents will be addressed, but some slices may be larger than others. Expect larger slices to have a more even distribution. -* Parameters like `requests_per_second` and `size` on a request with `slices` -are distributed proportionally to each sub-request. Combine that with the point -above about distribution being uneven and you should conclude that the using -`size` with `slices` might not result in exactly `size` documents being -updated. +* Parameters like `requests_per_second` and `max_docs` on a request with +`slices` are distributed proportionally to each sub-request. Combine that with +the point above about distribution being uneven and you should conclude that +using `max_docs` with `slices` might not result in exactly `max_docs` documents +being updated. * Each sub-request gets a slightly different snapshot of the source index though these are all taken at approximately the same time. diff --git a/docs/reference/frozen-indices.asciidoc b/docs/reference/frozen-indices.asciidoc index b37c3426b1701..60f07b58bcb7d 100644 --- a/docs/reference/frozen-indices.asciidoc +++ b/docs/reference/frozen-indices.asciidoc @@ -118,5 +118,5 @@ The response looks like: i sth twitter true -------------------------------------------------- -// TESTRESPONSE[_cat] +// TESTRESPONSE[non_json] diff --git a/docs/reference/getting-started.asciidoc b/docs/reference/getting-started.asciidoc index 7df9bdfe7aa6c..77b380dd1dfed 100755 --- a/docs/reference/getting-started.asciidoc +++ b/docs/reference/getting-started.asciidoc @@ -1,106 +1,33 @@ [[getting-started]] -= Getting started += Getting started with {es} [partintro] -- +Ready to take {es} for a test drive and see for yourself how you can use the +REST APIs to store, search, and analyze data? -Elasticsearch is a highly scalable open-source full-text search and analytics engine. It allows you to store, search, and analyze big volumes of data quickly and in near real time. It is generally used as the underlying engine/technology that powers applications that have complex search features and requirements. +Follow this getting started tutorial to: -Here are a few sample use-cases that Elasticsearch could be used for: +. Get an {es} instance up and running +. Index some sample documents +. Search for documents using the {es} query language +. Analyze the results using bucket and metrics aggregations -* You run an online web store where you allow your customers to search for products that you sell. In this case, you can use Elasticsearch to store your entire product catalog and inventory and provide search and autocomplete suggestions for them. -* You want to collect log or transaction data and you want to analyze and mine this data to look for trends, statistics, summarizations, or anomalies. In this case, you can use Logstash (part of the Elasticsearch/Logstash/Kibana stack) to collect, aggregate, and parse your data, and then have Logstash feed this data into Elasticsearch. Once the data is in Elasticsearch, you can run searches and aggregations to mine any information that is of interest to you. -* You run a price alerting platform which allows price-savvy customers to specify a rule like "I am interested in buying a specific electronic gadget and I want to be notified if the price of gadget falls below $X from any vendor within the next month". In this case you can scrape vendor prices, push them into Elasticsearch and use its reverse-search (Percolator) capability to match price movements against customer queries and eventually push the alerts out to the customer once matches are found. -* You have analytics/business-intelligence needs and want to quickly investigate, analyze, visualize, and ask ad-hoc questions on a lot of data (think millions or billions of records). In this case, you can use Elasticsearch to store your data and then use Kibana (part of the Elasticsearch/Logstash/Kibana stack) to build custom dashboards that can visualize aspects of your data that are important to you. Additionally, you can use the Elasticsearch aggregations functionality to perform complex business intelligence queries against your data. -For the rest of this tutorial, you will be guided through the process of getting Elasticsearch up and running, taking a peek inside it, and performing basic operations like indexing, searching, and modifying your data. At the end of this tutorial, you should have a good idea of what Elasticsearch is, how it works, and hopefully be inspired to see how you can use it to either build sophisticated search applications or to mine intelligence from your data. --- - -[[getting-started-concepts]] -== Basic Concepts - -There are a few concepts that are core to Elasticsearch. Understanding these concepts from the outset will tremendously help ease the learning process. - -[float] -=== Near Realtime (NRT) - -Elasticsearch is a near-realtime search platform. What this means is there is a slight latency (normally one second) from the time you index a document until the time it becomes searchable. - -[float] -=== Cluster - -A cluster is a collection of one or more nodes (servers) that together holds your entire data and provides federated indexing and search capabilities across all nodes. A cluster is identified by a unique name which by default is "elasticsearch". This name is important because a node can only be part of a cluster if the node is set up to join the cluster by its name. - -Make sure that you don't reuse the same cluster names in different -environments, otherwise you might end up with nodes joining the wrong cluster. -For instance you could use `logging-dev`, `logging-stage`, and `logging-prod` -for the development, staging, and production clusters. - -Note that it is valid and perfectly fine to have a cluster with only a single node in it. Furthermore, you may also have multiple independent clusters each with its own unique cluster name. - -[float] -=== Node - -A node is a single server that is part of your cluster, stores your data, and participates in the cluster's indexing and search -capabilities. Just like a cluster, a node is identified by a name which by default is a random Universally Unique IDentifier (UUID) that is assigned to the node at startup. You can define any node name you want if you do not want the default. This name is important for administration purposes where you want to identify which servers in your network correspond to which nodes in your Elasticsearch cluster. - -A node can be configured to join a specific cluster by the cluster name. By default, each node is set up to join a cluster named `elasticsearch` which means that if you start up a number of nodes on your network and--assuming they can discover each other--they will all automatically form and join a single cluster named `elasticsearch`. - -In a single cluster, you can have as many nodes as you want. Furthermore, if there are no other Elasticsearch nodes currently running on your network, starting a single node will by default form a new single-node cluster named `elasticsearch`. - -[float] -=== Index - -An index is a collection of documents that have somewhat similar characteristics. For example, you can have an index for customer data, another index for a product catalog, and yet another index for order data. An index is identified by a name (that must be all lowercase) and this name is used to refer to the index when performing indexing, search, update, and delete operations against the documents in it. - -In a single cluster, you can define as many indexes as you want. - -[float] -=== Type +Need more context? -deprecated[6.0.0,See <>] +Check out the <> to learn the lingo and understand the basics of +how {es} works. If you're already familiar with {es} and want to see how it works +with the rest of the stack, you might want to jump to the +{stack-gs}/get-started-elastic-stack.html[Elastic Stack +Tutorial] to see how to set up a system monitoring solution with {es}, {kib}, +{beats}, and {ls}. -A type used to be a logical category/partition of your index to allow you to store different types of documents in the same index, e.g. one type for users, another type for blog posts. It is no longer possible to create multiple types in an index, and the whole concept of types will be removed in a later version. See <> for more. - -[float] -=== Document - -A document is a basic unit of information that can be indexed. For example, you can have a document for a single customer, another document for a single product, and yet another for a single order. This document is expressed in http://json.org/[JSON] (JavaScript Object Notation) which is a ubiquitous internet data interchange format. Within an index, you can store as many documents as you want. - -[[getting-started-shards-and-replicas]] -[float] -=== Shards & Replicas - -An index can potentially store a large amount of data that can exceed the hardware limits of a single node. For example, a single index of a billion documents taking up 1TB of disk space may not fit on the disk of a single node or may be too slow to serve search requests from a single node alone. - -To solve this problem, Elasticsearch provides the ability to subdivide your index into multiple pieces called shards. When you create an index, you can simply define the number of shards that you want. Each shard is in itself a fully-functional and independent "index" that can be hosted on any node in the cluster. - -Sharding is important for two primary reasons: - -* It allows you to horizontally split/scale your content volume -* It allows you to distribute and parallelize operations across shards (potentially on multiple nodes) thus increasing performance/throughput - - -The mechanics of how a shard is distributed and also how its documents are aggregated back into search requests are completely managed by Elasticsearch and is transparent to you as the user. - -In a network/cloud environment where failures can be expected anytime, it is very useful and highly recommended to have a failover mechanism in case a shard/node somehow goes offline or disappears for whatever reason. To this end, Elasticsearch allows you to make one or more copies of your index's shards into what are called replica shards, or replicas for short. - -Replication is important for two primary reasons: - -* It provides high availability in case a shard/node fails. For this reason, it is important to note that a replica shard is never allocated on the same node as the original/primary shard that it was copied from. -* It allows you to scale out your search volume/throughput since searches can be executed on all replicas in parallel. - - -To summarize, each index can be split into multiple shards. An index can also be replicated zero (meaning no replicas) or more times. Once replicated, each index will have primary shards (the original shards that were replicated from) and replica shards (the copies of the primary shards). - -The number of shards and replicas can be defined per index at the time the index is created. After the index is created, you may also change the number of replicas dynamically anytime. You can change the number of shards for an existing index using the {ref}/indices-shrink-index.html[`_shrink`] and {ref}/indices-split-index.html[`_split`] APIs, however this is not a trivial task and pre-planning for the correct number of shards is the optimal approach. - -By default, each index in Elasticsearch is allocated one primary shard and one replica which means that if you have at least two nodes in your cluster, your index will have one primary shard and another replica shard (one complete replica) for a total of two shards per index. - -NOTE: Each Elasticsearch shard is a Lucene index. There is a maximum number of documents you can have in a single Lucene index. As of https://issues.apache.org/jira/browse/LUCENE-5843[`LUCENE-5843`], the limit is `2,147,483,519` (= Integer.MAX_VALUE - 128) documents. -You can monitor shard sizes using the {ref}/cat-shards.html[`_cat/shards`] API. - -With that out of the way, let's get started with the fun part... +TIP: The fastest way to get started with {es} is to +https://www.elastic.co/cloud/elasticsearch-service/signup[start a free 14-day +trial of Elasticsearch Service] in the cloud. +-- [[getting-started-install]] == Installation @@ -118,10 +45,11 @@ NOTE: Elasticsearch includes a bundled version of http://openjdk.java.net[OpenJD from the JDK maintainers (GPLv2+CE). To use your own version of Java, see the <> -The binaries are available from http://www.elastic.co/downloads[`www.elastic.co/downloads`] -along with all the releases that have been made in the past. For each release, platform -dependent archive versions are available for Windows, Linux and MacOS, as well as `DEB` and `RPM` -packages for Linux, and `MSI` installation packages for Windows. +The binaries are available from http://www.elastic.co/downloads[`www.elastic.co/downloads`]. +Platform dependent archives are available for Windows, Linux and macOS. In addition, +`DEB` and `RPM` packages are available for Linux, and an `MSI` installation package +is available for Windows. You can also use the Elastic Homebrew tap to <> on macOS. [float] === Installation example on Linux @@ -303,7 +231,7 @@ epoch timestamp cluster status node.total node.data shards pri relo i -------------------------------------------------- // TESTRESPONSE[s/1475247709 17:01:49 elasticsearch/\\d+ \\d+:\\d+:\\d+ integTest/] // TESTRESPONSE[s/0 0 -/0 \\d+ -/] -// TESTRESPONSE[_cat] +// TESTRESPONSE[non_json] We can see that our cluster named "elasticsearch" is up with a green status. @@ -330,10 +258,10 @@ And the response: [source,txt] -------------------------------------------------- ip heap.percent ram.percent cpu load_1m load_5m load_15m node.role master name -127.0.0.1 10 5 5 4.46 mdi * PB2SGZY +127.0.0.1 10 5 5 4.46 dim * PB2SGZY -------------------------------------------------- // TESTRESPONSE[s/10 5 5 4.46/\\d+ \\d+ \\d+ (\\d+\\.\\d+)? (\\d+\\.\\d+)? (\\d+\.\\d+)?/] -// TESTRESPONSE[s/[*]/[*]/ s/PB2SGZY/.+/ _cat] +// TESTRESPONSE[s/[*]/[*]/ s/PB2SGZY/.+/ non_json] Here, we can see our one node named "PB2SGZY", which is the single node that is currently in our cluster. @@ -354,7 +282,7 @@ And the response: -------------------------------------------------- health status index uuid pri rep docs.count docs.deleted store.size pri.store.size -------------------------------------------------- -// TESTRESPONSE[_cat] +// TESTRESPONSE[non_json] Which simply means we have no indices yet in the cluster. @@ -379,7 +307,7 @@ And the response: health status index uuid pri rep docs.count docs.deleted store.size pri.store.size yellow open customer 95SQ4TSUT7mWBT7VNHH67A 1 1 0 0 260b 260b -------------------------------------------------- -// TESTRESPONSE[s/95SQ4TSUT7mWBT7VNHH67A/.+/ s/260b/\\d+\\.?\\d?k?b/ _cat] +// TESTRESPONSE[s/95SQ4TSUT7mWBT7VNHH67A/.+/ s/260b/\\d+\\.?\\d?k?b/ non_json] The results of the second command tells us that we now have one index named customer and it has one primary shard and one replica (the defaults) and it contains zero documents in it. @@ -471,7 +399,7 @@ And the response: -------------------------------------------------- health status index uuid pri rep docs.count docs.deleted store.size pri.store.size -------------------------------------------------- -// TESTRESPONSE[_cat] +// TESTRESPONSE[non_json] Which means that the index was deleted successfully and we are now back to where we started with nothing in our cluster. @@ -715,7 +643,7 @@ health status index uuid pri rep docs.count docs.deleted store yellow open bank l7sSYV2cQXmu6_4rJWVIww 5 1 1000 0 128.6kb 128.6kb -------------------------------------------------- // TESTRESPONSE[s/128.6kb/\\d+(\\.\\d+)?[mk]?b/] -// TESTRESPONSE[s/l7sSYV2cQXmu6_4rJWVIww/.+/ _cat] +// TESTRESPONSE[s/l7sSYV2cQXmu6_4rJWVIww/.+/ non_json] Which means that we just successfully bulk indexed 1000 documents into the bank index. diff --git a/docs/reference/how-to/recipes/stemming.asciidoc b/docs/reference/how-to/recipes/stemming.asciidoc index d7ddda116327e..d435ce1c94d4b 100644 --- a/docs/reference/how-to/recipes/stemming.asciidoc +++ b/docs/reference/how-to/recipes/stemming.asciidoc @@ -228,3 +228,6 @@ In the above case, since `ski` was in-between quotes, it was searched on the `body.exact` field due to the `quote_field_suffix` parameter, so only document `1` matched. This allows users to mix exact search with stemmed search as they like. + +NOTE: If the choice of field passed in `quote_field_suffix` does not exist +the search will fall back to using the default field for the query string. diff --git a/docs/reference/ilm/policy-definitions.asciidoc b/docs/reference/ilm/policy-definitions.asciidoc index 00578ce8c050f..4949c43e6ce65 100644 --- a/docs/reference/ilm/policy-definitions.asciidoc +++ b/docs/reference/ilm/policy-definitions.asciidoc @@ -120,9 +120,9 @@ configuring allocation rules is optional. When configuring allocation rules, setting number of replicas is optional. Although this action can be treated as two separate index settings updates, both can be configured at once. -Read more about index replicas <>. -Read more about shard allocation filtering in -the <>. +For more information about how {es} uses replicas for scaling, see +<>. See <> for more information about +controlling where Elasticsearch allocates shards of a particular index. [[ilm-allocate-options]] .Allocate Options diff --git a/docs/reference/images/rare_terms/accuracy_0001.png b/docs/reference/images/rare_terms/accuracy_0001.png new file mode 100644 index 0000000000000000000000000000000000000000..0c13a3938cde2e2ca119f098aabb1ee8c4d1ce8c GIT binary patch literal 25315 zcmd432T)Y)wk_JAD7KOWL_wk?K|zrQ$q0xjIcE?RB!lFfBr6#~o1AmE!NEkaq_)|qP(ER-mOaEZM-s}npDN!|w?##J1tZY)LUGqBcB$+MrdPqq@9hr_A4SI4GKT-wXP zbw*BbM>g|6KkR*BJn8W{8k^FKVh}6yxkY2_Rf;h0N&mfhxVy5pi!rW_x$(#hSLryH z5yDWGXynnZRibB_^(d;UgQx!1REUPk)+R&G(a|G`!x-F>vM-lzmC2WL^}hlxmhB;7 zDnE&3U|xCFBV~L2X(u}kbMS|jd^;0ZA54|)gr4QZQh(kWZhy*^XqZd0P-bG^6WjQb z#?UE7vrwbp#YlmA-uRZ~U}5coXD|ufypQN%!bybS)ev(aBd$i_I* zyq-C)3^`*SC?(%FVf5N~P+*!7$slH#=5ttJ=+Mrlg)*qkM_*gV`r_-wj#kU}zTb!% zrgt4TiIO;}QAG0JoT(BHK69Q_Cq>w~SSkp9(=IMw4#N@SncV(7+Pd-lvx#yrFTE*N zoD)apPF?587@QUinvM@z1Hc7_yHdqgkmCe?j1-};v%$z8-n&J{&Yi&;pVaTG_^B}O z7UZQm5UDN=6vT0l8#@nbuD>cb>nv7I6X$`69{i50dit5Lel9_O_2-9sF5P1L!_D}k z=CRV}ZR4|hYhz>Gz9)`H#|si^1Ep+bX!Ew-LQU?YQB2c3yxcM~XS-%5oZEWt$2(N* z5Sh=$8P`;<9v;1l@2JK-z3OHfBe&1rBI&KrjeAJX;tl(LMZsFLkgx&v;~AP}`|fLe zFG6wM_~R&Co9}VR1ep9BDNK7Dhs}7+t96|yMv3*v%^`ie+q<-3*8TaaH(Vp@7#Ic> zoY%)k28oxOWb4Jl#%JBy(rH`CB#wIo>?RGCX^+=ko6A>F^{C%h`(nO(GEIuDjFji$ zrpbKmD>H>d&NF)Te2ZelcBx~YEZmy(l=Xj7nG4tPXhMqlU_GZC#}cnt{a!zUGkwCwFixtoEHM8vD+~{&xP!Z5%6%aP6&3@JF$RybR#H7#=6kZ> zTSaHr(0c+Vtlp+#4T-QSel`0I1sg9jm9IWtugAI&_2Q4a@fCJH+exT%Pl)WP`86++ z7X@<|upEWETwlJC({j=&IJKIZ=eRgjk~Creg~T?g`D7Q3y7G%|>Q8V{`$(DTn~Uf0 zTfF$0F3lqin{cG&lF;r7ruwFO?pIMG>LKdTb0oAcG>fz<$8766=eyIeQaaelM)|7# zUBept3=whaY-67T)B2Ocm6B(z;cS&|Vb)o+WQ}H=)Q6P{(*i@@PkuABiJS|%VK}92 z&BYhywT^T)kU#`)&QXQRQqTD~waiE0bU& zr_pmi!J{}=jqBsen5}om+R{{WbCITszcQ*}>xp)a_GcBvAdPg5Y3y3{l#4f?hCP*c z7RqjGkml=AuE7KOsZT{NvP5$|J_@C*b!fS^>E5~{Z1YqQ;UK%y+UVe*9_QKoE1yKP z2VtC&4clEJD2tSUIOLNscw3=!0eGhI=UVEuu2bF^YUD$VawU0T=LTU+*Gg#{gCl~} z`)JR(Lqh~%HFRsN{H1cs7%SK4ZkTHnwZ{zoYpDRFUg5d+i!$&J9SOG`z4KTQ4;5rt zu>NRY+{!Dg?%HUjVMz?~C}__xTqZ+Dao_2L&vJm1=3AA`*gTTT(*E)w&oV-0S{vaH z_OZR*0vr|;45Fs%4$0nJqOs59R=4-M<6d0a@mFuk^u-Fv4=+qhgW(n^!~oD);eyQA#Kb^jjEkT#@=By zipcGHj8GF!*s#j{C=b$%AUH)$SxWQZMx5@&qb$4F9>MVp6S)DI$A~i}`qeH(GT!%4 zcGF*i1bw%wHj&SQ`UJPb2FkKQ`03nek@iE%1d-LGn`Y*k)yXbLWs0-Eq2YB2GFX#7 z12fkxp4Fx-n!=y#qjF6*etl-^kT_htK_jbKlrgW*rSnL*O67k%fzr-Z4bLpe5Oy1O{J(eG8AU6rWR6 zDll{6;w&cWYfbP_k~qQ^3AAvR2wr~h|J#zGD)9m%=$qYTHW>Ieh3 z=^!?Q5QAZo+vVD}FoxIp3FcozDb{K8yJHDm*jcw2dN^GR#>+MG_mufzkD7gOPZEl- zS7Coh{D~!*@;@Gwr{R7^ROI>-QAH07JnoE6;4xg5-Rf)Lp$!^!uvJjXDcA+ue7EL)A_1N^^a}8qQONUaoyv_^^-D-IA+tCF!Sm2m^^K+FytY z-n^5;Tf3izgnLNapYcm0hw`xOl9~id#JAi46nuos-jcP%7=WO4Iy}~EK?K!?SZaZ`fSs2DiIzX= zL#Ra*8a)C(yPYD~V81E-iT%*|j1P;OsFocmEL#FEpnCF|vk_@;n^b8!C$hx3Jc?Uh zvU0H&GL;Zk%WCykjXtXI5nr~uT+;oR<)}~YMU0Fds>S;Lp!H%~Ruh7@bhUAR(lM*} zrf4A9S8^rA?=z|rS8;(1Z{U5)#@QHR>`!zwg+j=!I-TZpV!Z=UqCo6cdUX(J+ z{{5KQo&2TLmUwnWMXfrlzs&jv%bEekCs`LPui9b_q#xF46}n$+Of3{rkBAP)49FJD zf4%5T@QD||K|*Dst&NN~n#8;CADj7-%e%F5+U)!9h^ZYMRci*bGdH`qCk5&_3K*<1 z_wP^MEjFwl0=arBepx%-Vsr9L*!aDa5TCvbUP69!9P-B!;$eoWbT79kfuAv0cUY+O z>pS_z1BIGJ<6jFt&?;m^TihWckCMz*+_7QS#l#QSr zxe+BggK`#Nb!@&oB0jNuc?I{*u6;bpdaB;be6*sVGf9w_dm(;pt zanHdOq#^8wT$i7`nPu2zJV&HtPE>yjJ9U*6xb(8zfT_O$Ktp;e0RUo6Ck=ckI6G+ zqQBr1hx^6=X?5$PO}_+_l>35%=^AD1y{l<8)znCGjqfXUz!z|CJcm$*BHHTY%fH*> zT=rIVi!=(dZ{E zxwWMh%b-j%FCV-O-)SOLdH=_TDBRETDTB`+lOs^9S@Jgh$>e5w%OEy7*vI)^6o0&} znkM43s8Ws7ZrY-z6SPeRF>I;9#>jIG;kn%@xK_K6%B{N>#Ow809DmSVKW{%xvwvs83OZ3r>JywDieP4a{%F}fB*C!?p#XR?h z0n%UID`_zS1*`SR@j;H|NAcqgX{gKqJG5!Q`BI)#TI4~S#7U{x;bImKY#?9t3%kPm zr<5E@PGb$bL;82S4`StcN{KFBsX@Ne*;iSpwjAU-!eM=}Rlc(kf|--u8td8(Ynt}* zFB^?`hmN@4SN*praSDEyx&{6`4e4!*D$1GUwV&*mHB8Vt#lz@&GivmGVH zlEG{vX0~y@&Us~ZKFQ7{jSQ-)J5$B#H9pGyPOd=fmYa>>Pq6qp?^~c&YfNNUPPI<+ z+6sdz$4aopt9jf!Fh!uG<>3s0O2hFc)=BTTuB#bMdzENi0CFd`#YXW6jKOTvi=N$Z z*V|(7dEqNWo+(MmLXO`i80~Z$o=5FN3r;K}Npat4G1@9Kn=r)k1r$h!tDAOj*DvKJ zt&Z0^?tP_HmhoN(RrMmA093!>brJAO)La@LA;_DP0F|q@hWDuh&XF;!Hd$!;s}r$M zNAfoAsEsxIHX8!)VOxvzZ3OHD$}gKR9!|;ydsO(Dd{A;xXUZ4vtu3i#6S$Z0T!mS7 zA#?9!G%-?}Xpzpg<}-D&#{J&yB5kzodD83lN2+T+@YCXxioI9YK3Mzvwu<-F8J($#}l?S`Z4E0e*^XqLa{ zsnkf_dv52bKr(-~u)PMiH)crwPNw^@zb5lyI5*1v!DEc}*y1667wzl3u=n#`_wXnG zGx8{H5GriOs%+R6a(S(G8bC4g?$#S;M&9P*t@mvV>vbzAM*^9)<-zYzxxbFDcUp?= zsoYttTNK?v_9nCnyELL_8__T-c0vjjR*l6CWZn{41H2pHbNiqW9fVAn%Jd^eiI$}C z{PgSq<@-cTHbsmlZWMj0;#wW!W|2SDH^cs6T3KsAMcAO&yzk zeXddx+T@JePQx3Nk$KOS*nO^y@?5WN10>SAWo83AFIV!<6cLQ~SP47Vp_ySf*qpig z=$3GXB9&S1FU2c!2*qtwmXV0sXU%Ta&+XjVh4xtwu2B*1Mz}RNlG3a(ELZO8aeZau zBY*IZ@TL5)*s4g9>I8~ji&M?MQv1fF45Et(1=rr^Uy(x76 z9z9^StZ#_E0P#`Q%L4`Lbmr8Snj4gc=e<|3jb@80As9=Uy2$0%6cKQr3=PVRn1Rb; z3vTF?A7TpsNJa(WE;eEr*hDuh8SnendVJ=*&mp^UB{={d_Cyv$riey(;-a_G{$w!T z_NxcF%~O$tfra+Cg;E&)B6A@d$*c@09z$XkVp78^Yu9%tRpituC4=%eFp(~dy9}Yu ztJ4}CIiq?+>=Ta~$@qVmn3uS-VE8CDBXQ>?{yb>Ga>IgxB+pWXnWt13`{H)rds8!b zu9nu&Hw^kp(!bY~Bwn*GW@po*iP}%zz+Awa@oT*_L)&MHD|4B<59~{>?F@c>f)Gg& zajm*`&rAm@s6V4Y{Xo%kH#-wyP(a{y(g?vh3;yzdlq9 zQ!8mFMz8fY$S)^FhdC-~{&VMRR#?$J2G5JJW9vOai!)np<;-gxsyd1xu3hAj&OAYDFBjOw+q?sETP5jf+TW$4GlOO-|Mv7fVAO^k%^EPY* z6x~$2zs2f!dr(xljeIoZdy$R&?_svyeQLiJDJH`36Sto?Fb5EwM3)kvpW7WzNAG|)-I^mimU}M0nF%|4Vk3?|A!F3J=5ERQ+8MsA* zE4)UlqtiucuMZi_dkD~-P4FjN!}+k)h+8>b9kg=_of-qG##!=9`W6TiRg))KQaMjj z1_zs^f)sXDN9vr5xPQHvNtRyH_jDfEI|hWaQ_x-KWU|EXIJFR`2}i?jx3MBeH{lhw zp`#+eAwM=8lbdOj(6|`Ah|_lDa{h|35-{)iL~ZW8cB3;XJ`4nABiYN?MuwH>=- zOLmRt8B%qTQ$+>sJD!5^hxmMaWQ(7yqlWzzBFjdfhNG+4;Tyf!$jP)iS-AVM_V35@ zddr9b1r_5abMgy~RRJk9xwG{3hjwg-|JWjufS0P&IB!nP@bJcci+mmL*M%+7NXdey z>Ak5Me>6?*Qy}BjiI|sDqPn`0ZRV!nF#%4O~wgS4}pp`5T9F33m(I5iujt=MwZI!~(;&vwI4ta5aO~>#RF`=expm5LVN^R^ z__I;))xtI6H4O%Avh)$-S5KQkYff@={PK0QkiJIE8}>;YoHy_k*B87MpjjD}turc} zWX$xx@N5F#bF}$qHmY8ngR5bs^2=-A#j1jaYU#w*`ow9FXBru>2CO|7z)HB~S_sRvAiew?v3KZ3r# zvLS#40g~+huS%5ab8pt8k|*!DC@}QmUc)8a`c>AEQIJq5nM=e%ej6ZL2zAGCC|6P3 zkd>LRZ4IXu0Nf@`Xe6VAEu8;PQd!3$(P^}P@t`X`at0Mw@Y+F4%ze~Ettx(9N@wD&}eZJ({`7tdkb1x|F)8p*zlaF@^$N!472ZDge7j!zej2 zZCy8hJ)pbu)bF-#*;hb<3ws`|Rj(nfP^AEC=(I)BPt>_@&yN5IXLGc-HdLU_v0n}l zl~rGMSm%d(ZNX%Ww%ppNiYD))I&ZE!H&rNt!JYhCejiB=pp7L~`f}a?I-Yyp#gZ^@ zls<^rVjw@p(0%S5&(#!B_dd&k{OOkH3hR-+pw!kkH1!LByZ(6_(=6h;?u3`{J@K*{ z{+O}5I&$yL%k$8R(KnIFz@@|T0s6YAoN8F_GLmH1WHC|iTKzKBJ(qRIZO089%OOBn z$09ipH|=l_cOeuaLxV5nE}|{|$%^FJGS{ zDFebr+P9}3(`@mFK+CL#&>K)8tk=hD*Gms63ziJQK9FDI=;Mu(y4?H%TsJ6;w*nU=Nj1|- zW9)Lyc09B%?$! zE2E4$l>dCpJjqt@&%Jz!=yP&C2C~v2q|V*ij?68jOxD;BSvu@ z#3-ODKQrw}P_(26gY%oY94c91-pi77$0R^8RdfUp5fsk#Byfo_;AYa@ z=fM%zh4|*{p0`Dg{QCTQ4N$^bwGO8GK+1c^iytwaD1R9RF+)Sey3gNnY5r)hg^r(G zGWJaGpUqhtKwD^4+Zu|znvoe;gCgIVbC*8;&vB=}!lFNarOJBb8Bn83g&gOHH>SVX z0L(vBZD(AqOq2^k<=^l_IQKt?pOvAKa$1CC&8Hi;^{bpdn{Cdt4%Ip?Z; ztv#8jb<8`$A8aq^gF>|gL{wc-5ims9(uOya+SMmtBk4t+1VlWFs>qJ*wkH`6z5S#tHqMe{ zl3N*qS+JS;8ism2u{GCW^ErS>04Q)tLXNrMdD}-TES>`>Rt^B_2mr>8?oVKTOzLvP z&+Zd1yPuJ}=D&z9(yJ_lSlR(TD9Y&lbcQOYGYHOO&Z`B=s^CnQxMC*-%zj=?5^+U& z7N5ufnWq92jR~VsQW~ti31?t%{WRGF@c>`qA5*a#Rt)jm=c2Ej&BUX@QV4kNt*Uhl zyhQ=c_AVw&HnzVT=%vMvq8O@N*7T~67kjhV&TuUnN667Ign}y6ak6yrwP>FMOp;<) zfPLzb3d(=Ix9>BYT7$@7^IYw1NFpp zXTyAuKF8v3su!DeBG*f0^B|)0U&GK@1)d~3>L++M7$-{;sSMC6UUg2$IG!(p2!)`r{Kw?4qGwm2Rkii}8^Xlf>XV<>b z;O6%}+!;?5g?4`p2yq~>E?Bx`j(|{G&^A%$>=;+DKGi%Gaf@cO;r5Vl@5bceg`AA0 z!9b-4?McPduwT!o5&X65<7mu9v0$c4cgPF0ZV@k#Fg@qY2~iD&B2wQ<=Kn=VdLh-c zzy$Z6NdNhe%7Qer$Js~B4tDJ!6Lg@+b%1OM{`vo(zwFcGbR-XIYMNSL33utl`Aow2 zBLBbQqwK7nd8VyD?mU?~nDK7#%-sLC0XMr)g~N3B&#Txz8B{G}HytMvNF8MtDA|Vr zscZv8yaVy%3P^K+!B}4$|Jj6Oi6Y>PNCEUQ5Gp86rz73(-NEwPhKBXpsR{4NG9TQb zZX%CiF<1%r`MoJpf#KdU{p1v}^4#2(o9wqs8FXJ8wpQfaTh&*lla$JWmK2-~GgLOf%kR>|@S9$+@;ln64$ z%(EZ`?Tk{Rc$@*2`4?C>9`Vm0I#UHD_X)>tF_-Beno(c|d0IX0@J8_iAKSbSh(czs z&y!V^zwFink%l zdLIq}s|*A+4R@bb;PRIYfZJ!>0ax$1ptO(QUg)j{UM4;8OpXpHa8+3B!ir=C$StuS zXL@Hr+)b%2kx~M9bPdusATJAm2&+anO`rHP*6=k-# zH)dBY&)dj9?^EZBwS_2h#brPZ+FrbJ>!WIx>F%Uz0u*G^@|*ZYsL8I|4TUUTH&Bt&Pg8bh72$maVSYPKO1`6F*ilh9(<^bI@BLa|MBO+5)Q040<${CDUq_>E=~w;id^phTi;p#hKSI{ab%TxK(bdfe zR{uL@LqCdv0>N$5leHw^f0fsF3x&sC(IpJa*)x$?Kv^J$^4NZ4jVX!$dXp5M{=r zbe&fQixPXPXl+nRDZ&=mwyYNhx0bGgPluZRNA$97AGnvf&bF+_XE!M-=$cpe-f4ND z9T(V**EJ{n8|<=`Z)fsfF4vO^(yO*jS>Z+BVO@^x12{;(!R_wf@I~t}Z}Aj`2}5c%8vpWw7lGkX4U;T+&4v}K)F@h|{0AWE+~ z1BGzbtYHw?skW0vmkhR_iQA=zn%j1dl)p^W%QbTX@B7hD!S*+6P?ZsTyTc>qu>*3B z(3Z&Rw1}?-B~HD|kQ12cfKwBoGH_N5XaTX#Ont_|fMubSte1<*x5ci!P2>-zg3 ze-;$>(A)r9XWED!hEu%*RZPe(+D695gCmqSm zaA#5$<-1+&abZ?QZ$XB7`Cq>oK=bGS66@J3d5j${ho*XMJ@&-o_92q0)=dZu52jo8 z?*Lh~%nM*h0qfyAA`ZVpfnfB*?B{!v8;>I2*7Hn(OXyWuTR<@m5_y(EHS;DB3Z2m= zvu#n;pqQ=a!-3HCB!N-FXZ+lyt4|@rD}Y*~!01?_o~N{)2XL+`rqXsoIO^s0hvc)F zz-KiA4&iE92T!>?PwiJOjeM0dTi~Xvy$@X&W&stv&P#o*V8?1&9)Qxn80gbH8=+h> zhkI*Qz?f9kl_K)UkmkMQK$Ewpe%Xt#cf87pY%s|fu03!JTlqPZ+0pKIQv}I*j$N6x zS&kz)FFh=K-bM-}N!YM%atU-!vq~#WSF1ea zIR4JJOHv53Qh_P0=!4gsU-h(!9kfidl(fmj|D+%JI%litw*Xge3y|C27fC~dk0ydl zf&jU_)5~ouBo3=Yk^g2TZ~Qa!W-mnI__Jn?;j30^NM1hGCS?X!+sC3D~9KwWpuO&GGt{o>F?uy*0DOz$dy2vR@S#!KX@YXu`-JQ0%y_j8+!W zSt!k(GnWyHV6FDpwM-IrE=1Ul-(RkiijI>6n@#Kpm~>x)$9K#c@1-{I6!0ufV-_YOXtIE=1PP_Z%>Bw1Ro+S31O8`Y2Mpn!)RsLG{;JA z=sMvHu?t7QEB8yu;DT!0y@^+C^pG#t;5rmx{P^?7Q4U`G0ECu98ZU4s8(#bf06cre z>AnK4B0<{<$mBTbHcKK{_%-`+Tro(^wQlavc!bh>vWHCCQ5|_;&j34c-y~2U=|o(t zd5lP5em}Q5uFIr80}+=F({L3OC|zJk76PN7A{rwR9DfRudS|OVGz>`tp$jq~`IbWj z4DcM7B^o${X#$<`#RLeuj(;ECJ{8;q?AUsfy&s9)BcdOn5s+&uT@DW(&m}u$BBZ1l z{{dG2{pU5uKN}{3SaaYwxL3uK zKPWSoS;FM|uQFd8Y|ggXeDWgzkV90v36kKQq3FzSGs>Ui44i>o?LuJYfF+1XN2dK;#F`lDRUUOYIt;AqJ02tN7|>&6egnh~xUL5A0{fAny-> zGDxihQ=dF&2q8#?QONF$$M;YY;Q0&t-o2sUMV zqu|0zI>Ds$mJpir^y;`>Jq$`7gY72sBULwAQMZpNVxB`>+9>ObG(5MKxGP_FGtdE> zbBS*GH()W3uXF^bz|WT?)z{s~$hFr7n_R76!Wj$3ssm#4liU3e1k(ivN=+}38|($R z1h-bPy?;wquA^PkK`sDXB35X+qGcYn8I4aCfGI37sO>|n)bEX28J1`P0H3-366o{m zpoITfI{hW2et~Py;|Rb2S`+BtH03@yqe&X5R>W0_dg5$|0>|BZmTu}J$T#)9^BPu} zdha2N{W`E&6bp#G(R2m{CxlUeEj*d-oIx8vmo&AxKAB;rS!<8RJPNe%fL80rHP0KW zMd$!bS?=Nk?jp&Y|L^7Uj}-?t4u3ht??!IZMEby{(FnZ9>t`9=ei76!7Z#Nr)|D2n zw-n^&f%;(&7$Bav9XqKK?Oc{E>>lW9vDQ% zB)}bi2?EQHD>YMwMzHN*p(b>{msvjx7`rgdC(iqE++fwbcX}V85*C$x594qhO>?W#A&;0p>bH6CZ2Y2cf8EoAjo4s?Bc zqU20Xb}KQ*GRw4Crd-@s_I$goi5i$WDJ8-fOEo8gK{AekH2m>P%}F@l=mle z=<_h>#}DQKUVOEm^(&ik@QwxNt41N2?p+9pPTEgq{F7`Df?#jt!}m-T%C@rpoiXA$ zU8TxdcMsCC`8hBRux`4CZ28Q3>V5eF91xVK-6Y(@PGM!EF5ceHt>d8qiFqftH+l$= z?mW7=CWUr&OYDL*QdPo=t%Y@Q|Kuum(1eiz-TTdWZ%m*qh)_^#$enc`QQ>Bw!RmA( z5fje``9QeupyfAzng{w9c*&ud>63YiPTrzAs*2CEAgGTz?R;jDZwhFCW+ATzMQ=0- zeaP1PprdL37t6}=heQ*0KaUcduNvGQ*nfmeo6(%RDSGqwE_SN<#fO%dU|Q`Tz^>~t z)I|yHWy>;#P%0i&^NEM&{)FoqJ`V!I|HcsoQL}YdC6pSOoWXfV<}FN} ztW8;i;>b4LUj<1FsSB_zW)AKloq+u>Nx_$7(C?_i zL3TK8M5te{>xdY30Y`RjhK&FwryO`-@AY_r+5>fn_e!ziLAx`j2>Lbrc+4~k)YGKm z3YWD}m!+J<1X|rlaT#zaKVOKnbdC$-|EMD{l*7&8pAX{X-ne78sduOibhDA110jUq z!aoQDwhlq-;a~H=ME?VtP0zD-4%I2NKW6Ill0Fo2T~AJG<7h}YB>|@CBzzjyCL6~9 zP+O6W-MjuDpv>|4dF!dB31R2u-#&mKM{b@*|6!py>#Cj*MxS9II9M|J9+!G=wR1qt zG&)(b5##NrfY#dK1G~HXM8SefT6HmS3=ZYU(}2oAyI}4gL}h&Z8lcB5z>`!0VMHHz zr0+GM772TE6_dvO?Is&WY?}_us#m|!*;N3YW*D>}VcA8Az{9ExQw6h(ls*purByK? z0w;k?34#sx>)(m5)}j&a|+M`+(Cr==*TG(G5KXr_sM4GjNZNA9V^~oesYVHkkl^!X>M7 zZKS*nN<+X46$=p>y4QrsV15;6O#udH{@ETEWn~1IF;=G2Kn3_U?MbqnzxBl5OK}-D zH8gItT&x1<%M;{H!$Qc1yjoa<`qA2P3MWsC&zSjyTe>r{!Tq83Tt`+amER)hUV0g_ z&soRULLEs&Sy*qZ=lUj~5uw(uD!KqVFnUQ7C^lJrV@Fb&|Al7})aex`=d^wIDnDfLE=U56?j zN-)>JaPJ)|Rq$g4YWa%0eF|j7!3ip`7?&m!`)wJ>|Dg|%qBc;40fa&f`V?z|!)GC5 zg9EGsRyN2D`18ik%0r@J5xPm3hWGT$Q2FK#=@2r; zDu6e1Kp5R_;-L^KDs%rc9byCAL#ON$)Ug&Nx)loW&-Zhr*Z)}$|cz*8< z*|(g!C<|vm7MVSYyj}{bqY=>SXARm7+;tP*f*1qP@5oH+8yiqVxd3Zsy{X`&Th{Jk z?-wXm3Lf18gY;_=@HuuHYxJz9<6Hv?maPLcb^TL-sD(%U526eh?Hub+00%!#QCA}D z1yj4+_xzvmE9Lv=*XK(>QD1u>V&2jh$D40Tpgrx5wiAIm*j?we^bD*qSEUyK?^+-b z#10wLn$`hNFG?*NAUYvK?P!0w>6bwELp|yE7a#>qt)$n3l-J+mqJ-Sx)5v=?6w!Br zGUQx<;0N|7ccsEVC5azPbuS>y&NuWIDebL}JO_1CgQ6c9$lj^USWc-y3gp3o5LFOaz-Up6A9Uz(-IM#x) zuuci;jfEX&0N%_tyOvEpp{93=e>q7n#9}lBTzkREu&j(W6ciE*q}D*AL>(?KNZ@Os z!wGuE(mE5Nm=|qw$|wxVY@g(3N^etG@LQ1hmK=ofYxyM<93X$-fXr?`w%}pN2QI}g zdqC6tr!}_93uEM2f;+@Ccg>%cHQxz4=?nEfn1k|oUN3MBqWB55OP{qil*r3!or#Hk zzUaS%uY?xY56{UDf;F0Hh>dH-I}~c7^xR)}#^V7SIRaV(hi9Qqa?J?$OC6d$f*C&75b7RnsP8%xG zf8;$0SUd~JfJ?{BJL&UZ$Z-3j|0cr$))vqdMXFiB7w`6g$U>t9YNG21axq|{2F?cd zdSEo3xHG-E2X;JIQ^$^7EMOug@ZD(=Q?vrs=Ag>`xo1HLBR7A&fPcO-`mnk&uI)y9 zNqVYav;|-Uq3)~;+ze2b)Bn0gMlbvV94)}hC%mKO5*_jZjl)(PO3C&e8gUx+E;eJ; zc4||4Gnh)t?&yQ6EsJ;|c~X zfko^l%Vpj^4|`#hCY!^!eJvSh72Ap9zV5Zs_ejmdwzPr9r_p!7y`xXaj_Es~BO&p? z(pzfj6%ZV|#`b{Yu2OKys~;2|o7sE)8PexzGO2v1-VWsD17>Ha-qnuop4r<@U{n>; z^34R{U+>gK0HM%@##C{y8_=!WtM2A7Y+bV+X=cAR>T6zJs<4XInY`GX>g&Vg(1GXW zR8^1n4x){Zp#{_~IKqiD(C(u+rg$Y(=wn`Rxsr++Am|W6|=sS7&VUWu23M}sGIjOT>u1% zQ^5kDByd%(SR+qR2u+ATTWh=Lv16XZXBrOmfs5IKwuyhi)-Gr6fy21E?U;r%uY8uV zg7a9H6!;?y1UIwZ%!dNNq7SLoR`UX&BhsojGe{r(*P75b5@pVG?%x>UN-5BY(B1VI zY&Dh500J0m3lIgMNWd;P0ZiNg-4toWRf75roCDC>khI~G4Pe_k2ShmV*_CO5 z6G`V&d^3!S&^dCt1Lw9bS#>ru-UwwN1Y(7e(O#B&_znOq(ei;TOn{>W;PsR7<&vA5 zBD@=CH^==IpjP~Vh{w7&I_6n(gc{}_V`8Z`eiv^Nk|{3u4M5tIWN%gv8IvYNc&cbT z8;t)`nZT+?`B|>x*zH78;FQ4iic~o(^N0zAjfb}z(t$@JR>mqs3Ul`pVf1~7_x0=k zbDN5AMCSAzH+qp|5|~(WOTV8`r*pFE>#MCISldZ+$-~l;NC`gxpCt^)V9&^cC99ypNUA=`mc-`K zUM9R0O@BJ*4ZdrZ;TbNh{mhYiEKH|Fw4@~Ub55i|ZkVE2wa8Qm0V6aGewlF!%uU0V zkNEDsmtt+5^ip}ixr4v(@H-Xs44mez2BEzCtyf6d_=yQdB*AUZQaX~evBx=X{oX`! z7>cM#I#N(O=(OjWKUj=tQ`X%c<5rOmeEav(u?2oAp@Eh=PsB7(u#O6_nK8xo_Ys5BjFD?E2t%EaPplD2pYfmSO z=%%(}of--3!zJkti;R>@ZZkpI28a~-dA+6g&*>>mh(E|eb1J7+`ZI#kCpm^tO(3a@ zjV`bO6B&Fb&>E61144vCW8=bv-1+P~?`d9Dx(|~P!YYJh$W=;P@+6Xkoz1pxP~y)( zE0aq#o$jH}H{&UKS{`yQ=26~aNwMNOl4r|z@3yd|?z(7JFmO5NGPrs;bajY)_xsfT zV(NnerYxHl#zOz7u2MHFI8)Cz zwe3co*NAM&z`#3!QqU5?h;metI)Xa1@JZI;G|?vn>Af*YX8s~TCt8!Hr^o5TUU*H3of1f#>*9tGH*yn?ORL?tA_*jTq*!}TdD+`(9tESYo(+J|G_@kVX6 zx5L<(Ug+kZTdvC#JbuZO$%=B%*kwd|IbTbv+Z~sxtzJvmlg74{t9KM1)Z+g${ z=SSc$lpS~Z&s>PDby(WokccBugkd64Shse zAdSjSM5SUtLq)a0UnUWMe6U>t?3hEK$~8#gfiW@6Gd)q}a-_&{>x`ohx)l@PQ$avA z(^=q%FRfmk{DHquhHFlseywmW9qiB2s`KIg;Qo-Is3rW=P~?-t&!> zlmByGh;eeH4RM~FXRvqf^6ga)M65yJ3Ezi&wr~xR_@#`za@C_axHv}D8sG~mnE^CM z{5hGPPsF9Gfa3s6+H*3oooGl>xx1hBi68i}Fxj>TTXWF&Z~UX>u%BHK`2x_}rGSYU z0pz4Lxc4=QM=!JlWT{#tWVm+hJN=(NsT49O3p%kTDN0#=b!ifLA`X85*8M>A(cLX) zrow+`UI!Me$7^~?VmE%hCaWW$m|4pMWsAVGuf#O}6}VwRG(zfM0ghAV;bj{kz6H$D zqlX44vO_z=L%GPPZ}?0@yJ_SN?L&Sp-23j)-)x>FU#gW^iGwi72HPhRMmIj_R~2A~ z^H)YCj^gs$P~BNNX}W3jk36vSjk2Xp;Nxk}#5v9;q0gv-%FzZ?QaPYA#}HTuSZwh? z-l{_;=cRi5AbB6X=tGsf)M9Lb^GXQG}drdopb2n$9Y= zabtzDqT4PYtr@@?XYdUXyDPolUs?9$_D+LJZS8RPdtZ*c;cze01CsDfbNnN- ztRH8qUS@8>nZoYC8PRR=Ck#{qmZm^ne*u+{z;ILZzXtHHJPDq22PdszlXuC;P$#eI|;d%<%3=AS^!pX|%QQdWK~=jr4Y z_EV9}umWP4n~gw?1asNgu=;45YUz9s^Q53NRZoB7>Y}uPV>WlpQzrH5gp)q2twGZh zZ1V?_OC#}%?%zCp8n)lk*vEc|<8Ttm+r^6s{x)H^e9EeSqdmw+oW$p#;x?S#LnKD% zeda2qq{#(e|pQchQgq!#Tw) zsP0cD=|T}oZe9u>88BewQFm0c?~Dn3(GsA|q{Q}A#zUzJKKubL*`ACoAjJ=}PuNpA zlLu}uJcznEiFM=JXMoKxlQWa6tqnXO5NlFU?rB3jVr@ScywUQ}v!Ucwe2Kb5q+HOt z%;7*_aE!)7_}xjX5S0Ay$_ASmqO90oaW8tpSbIm~G+Lgqe^DuX&3)%Rp^7N6%toVA z(k&IqC?5UtcaELY?Pd{@fA&5UK7^A|e;}5K*-Kj8j=dmzfHHhs`R&$J8|WVf&mh-+ zC4YAJ)df8QG5rUeeLgkwpW~x)RX-5R#g>ewDXiU#neEtFbP6-~@gz#}^7OHKo!S1C zeB8^7AP97f8+SOx1)2L&y&l+jS1Oe0?gq)Dr=zm5ul#WT3#Th15C{Sxw_j~qoW5S=9g?^fmXjSjDjk}oC7 zP-*q(?ANqrmU~8&VSN^OZ@?b?jM>s9=E+6Ac}6Oo5x#`X;~FH_kn`p*qxWJNI#wQgOL3>ubU4dvWb4roMF}!UybH++Q*0YJ-doJz93iL5-k?N9`{cnDq0j!J zJ<}Q{8JPJyB!JBK{_~D+*F{PL9-(C5rjvZwKd~1cJRp_K$n<8fnJWKCdh+1U1ow_m zEqwYaU%TDHnX}-FPnte6X18d+r3^DeN-0+$xo+EeJMZk@onG+rp}%?8(C4ZJ1Y7!{ofr+^d%j5QeieI_K=!AJFbM&4D~r&cy;L1}Uo z$t{-F7eX@px*x>vs!&<*WNu(KXRnKu&OaqoaUz!C^l17stM_S(+AExSAp6VUNkx=| z_GK(~TlHRivdsMNnK}YuK0=D{bBc3i+%%s#7@6~~B2#)RZoayEURhG1p&P~o{v3UN z-|h#_Ds?%!9+_%=KJ6K?P8#NYx{6XW)|`cPAaQ4m8#duJkm-v)`1KX z=l-8s&O93G{r}^VN>p@hxss()5%mozm1RN^HKaz?ZX4Mq$rdwHqLe}^iewF4%S6^N zA~odNx|SNtWNXx5FfnGv@_oJCZ|B^5&$shE_jk_kpE<`l4)a;&Gw=87`FK1N&J2`2 zx6KsZ|L6^Wg@75H{7+Im1Cfuy`?xD~d#zUL8H(vVyVXHo?-F{f8tXfzxA+Q45&q6# zf27Z^RT)v$7O?&kOI9JJ5W*GG4v&Q-57!U!ZwD?-!op42JkesUVKufT1}jnD>OHrv zupu(ItKepp#KlduEKLD%`0!^%FpzS8fp#EnV} zRi+6SEvm7<)V9N05l)JNx1#lJnRd(7l{@5vg|#bq)CSB}f0c^P-d$}(l9$8jTZ#_`A+nM8W z?_;FX8MUF1aO*t-p-LEWafg4kNw2!R7LPx(diprQTeq*F;b`w**V5b8!Qx87hSn>RH)s43 zwmU3enZ`F~uMF+@sr2lUF-pDaXJx^ls*bBez|KNrh5%3;PoVpxxd>191jOQ^J_PfJ z{W~FzGjS;Xu?_|4Lyk)|R{6bbg}Rn`cPHiB%Y&qo@pMO}UV=pMmFiW-E1rB(Xq8+v zCif$3)-u}zY^b$932}`-lA~}L@vDE&mA_k&K-WwiAhGt~2aB8NbF>=2eIr?#eRv@u zObGqyJgd%_JG1X*kp0I!>0>bhK_+L~xg2(#F$4dA&auxObF7c`t}KHQj6mX;k0LGa z^Z`?w@u6`fQ_+e5W&q^Oqq7~#1AAkh#XN(>Kb#Cb-01?Sg+OGK+c_Ozqq1NscD+d) zrqc$xf8Aizxy$`C!=WkAVc(``mwFg=8=P4FsGn(IUFdoY6>k7>hTz1sqC17T|7)l4 zw=t-EII(KIdehBB_l71W%p^7_;m>Ydf17*fFDw1?_OQ~ArtR$cGynP=#TA1<@TkjT z4d+2-A2YCanfSO)hite#au@4lZ~3+LVT`Sh{hkX=miKOh?Je+8@a2uN$+&{^mr4)? zN!*uz>5G+&uhv0Kwg=3g*Lv$e`5%e{%=(aNWv%uC*d{q(BrP6puhRDZPLJi_{tz6- z;cOg+Y~b@}<*(2uP`4v=eS<@qZ88wGjHv;X7t{c7ya5Re5@`zpgR~6o3<}1Un14>j1ZFh~2bgpeY5>5q@U+56PrrOE)fc*(v#58L|RRyvUH%3TZLOgjfp z%Tg=Er1kn|Hno>78P{yU3mYvt18JM5atDx(xD@`*)muH^8mUXF>^O zKB|4tC?CV{VKdql;MRAp^1V@eT+QW)fjZcdhD8q&=Wd7lHWKH9!hL(zRT3%2d+?Q{ zxA3Y@di#baKgD*oZ59s`2_e?~S9lZhL#|$Vc8h{o0w;Rop<#fBUKG=bolr#gvrcl3 z%IlTEdsyYa>v0DpIKMFcc>7z@OeN5CV~?{`w3gun#9!PX1>5!@h)R7?SEb;42=zS> zXn@4ZsR!hsH?wr!hzNi(7OAfg$TJ=41s{0XJoC}@Sw8CO;X z$B`noDeF6s1@Ul4Hz!=IVP*E0oc!2!t0u*jI&fcHoGGj*C z04Wh3%8SU|5TlQFo|00W(XB5PF~2*_)zqNv=FhU#X#wGoPo*Rj)Ajq|HBrC+)a)vK z_HUY$S#wckLCdRH{!xPeb*p9jd|E|Cj=fNnvu^uZj=6Y|{i6fyOT;5k_; zBYu8dmceQ&8Kd_mkDPApjzYuQc;2OI|x^Y`Ghca zf}?(qV!Yy(s7A>v$lRRu=f)!)G(+Gk?PXw2V+m1X>yTYPqGSnf{y=$_MileXkA;S! zW;nQiA1|M9UYm%odgc6>q~EVRyJ{I%vmp{LObWOKj61JVRa6|~?ezo4-^g?qyAbD7 zSAV^^>B)0Q-(N$nH?)ofJdiKov3^Z$UDc7T_YFv!eV+6*r@>)!jyW?AI$r*=(Qj0B z^Fd$?h7URMqAO-5>M2{cZcxrl^a`RZ>1SI^-)1x5=78fSrgNp>xS^?_?M`*tU2`qh znXFjsQ4QwpY9MKBEpv=;R-A$^2HG8jZ#JQ0fE97>M&}_g(UqkXC6wsoDLP$ zWx%?UhHy0+s)~i%(W=Jd{q9lBZh#k}LAATG9RP$pxV#`^bffj=({|Jn=i3VpT?vHB zd;o&_Q;dS{B?Om-!UXbCe9(KZBjD453n1$gZygZ`%!-E4*}lCMgpDuP3S2RVY{x(d zt!ELJ-WI3j+4Vsk1=tl}q8!gVWz4bBHWe|+3&DGXfNPg`Dzp;^1Z_4TFrHlKem(;~ zIFy%Pp8^4jwo=} zvOesOWl}arK{Q;pP=f|{E$*u}0B+831jS_HWY(w`V89C%GTMX=2fqZ-Abk1wb%(%_ z!ciZIiXs7ld00yr2&r~uy*-UYSb(dG%`FKv~{d1{K)$ zS1V!Ii2(G@P#MREW0V_^*&txT&a?Cl@T%B}wQY|xP0j+oFq-FjbF^}PjvcXw#&mO{ za~u8KzYVoay;CFKcUNHAqv^A27^<8rrT$uH)zGF>$!&-xgo-%i(v;gS%tE#Fw%rGo z!~%N)o3TNM5+p>m<=Az|zi>_Z+?e17@rBQ#Bl?iK`!yZgb=2uO&^YR&g-Zmc5i^D)B7!tlJNwl zU9PE33O}M<2*pJj=Xa{OI!c=sbl`FeY zWHkHmQsiF1_`S#@^81H;noqYyZM^vXLagyD8`;SRY2-W#Y3%S7+BpP7;=oGNal*!; zz}hZlZU^9JUfODRT`G8enV-n%&cBWXpvqM5JfkW3J3P#@<8}R^F-A`MS`S0QeKo!= zPE(8)k-ri5a?7g_s}f76Oc5#7n!VQ#Qt#J(KL@9Vz}j*1O;>??tHI38$(5L)Vh6~a z?-DPmS8W+jVF!ku+yEq?48Y&gDHghoX@KfD93?kn0<#GRQi*$^tNA$-m$pa@wb!dm z@_y~MBq?ZbGrzy|GWOp7J1!Yj@~wDNa?Uw?4+V2AJWlm;y-x%n#EHy92&4Z99mRe! z5u?yW9pa?l;*E+{F2EC9aj8^HTeexUVWHfY=1e!VqFF4P$&Fbmh;@qbZ@upU@X%EA zelfT?EUc*WmpD)gv1koE9Y^Bg&e!Y=y<_C$WYGsE>}c=l4`ZPFoUdNTW`jeOqa3Y*#cDVC(fvHsgd zew1Uzo&Cp0IA>Ac8z|bI#AW{lQ4UN)` zG)TSYkNbZ5d7t%u|9r95cdd0TWcbAy``mjU`#AQ5s4B}5z$xG`7>qzcUPc`TyZRIc z!+MN|3qIi`d?N#XTz@0~%mD_w*$V#ucIB;1gg*>s;iDiUsp*=sF@@itY1VkLO`E|= zE-gvguZkzhZU4dl#`}5$(oj-t#`YVJUPxpQr4P(!;ZXQJ$s39`PTkAsm&BK4=d_KB zq?VEzk|2D+YVK>_e10NU?OgG?ujrYBUd5{)OJ~Rp^6yDin4%tDkr}tNr5fc&=Y-I> zVDc;e>RZr1U@~_zuR}jQh?1ZmE!X~E`>PN9&zjN2IEA3CwLzUD+ddj5F-FgSC7;z{ z@q1a0V!XDVcA++x)j)Pgl@+G9X)lLo?C$H7Tis2Q4t=6%`sA+J_V?1GlwYR^1SgGY zFOE7kGJ|N?|Lrl}e+Sdo3-Z#+DeW4{EXTIzI&3<^sUN=lcH`urrrAdP;<&F^C0$H* zCVo))@sD1ylXX-FhYq@u+{okSlL#7vJRH~v?#A+?aezyfp?F14~*3BG#xWm_a<~l_3XF!o>%=2(6-uYyun#@$(v@cEc zV6%SfIoSD0x5#(b2*)39HE7s8%kn%q7<1ohe4Ur>RX1Fq!Lnh+jBnq&H~!dVv+_Zo z$e^;w_SY25b@pL5wjr-qYh$m|ok~&>k!$NN2I@9?7(r_4nF+h*jqojh63d4K&Ku)q zU|k&APGA|tw~}DL()l^JqSH^aDVwr!v#|W70$aboj1XOQYxf_ZjSk1Cz%wZdGw}bV}iW-sNHhn zW_fSzQrcSh)1@!Fz*@$b;`L{dq)wuPsqs?lc}T_!U;txvPHU*tcSh$s@y=sr@md?w z(7>Qn%e#kJpMSP)Plz zgQ}9+Ol3=duf0#SRXRqV!(u06edEJAMmFB3bF@>!&t1mK%|=jK92nir{>*oJ=V5jp zZ33njYt839Vu!VTwP~CJBg3j%5m81eYUOkjh#K0YiX7uN8&#OPnL)n2)!`m=-70%c zFPIxSX7Ep9pT{@brsG$7&0dZ8x4bPtR*(i{XtK{~uCv!&8pf4TG;FY74zX>RX12{lwU&F) z?`IB6y-qXvYsrp%u{wq;NVXy4nBCsTedy+MB&KvIl1V^**wq6p#pLNN0TJ1m@xmI3 z-RK6pIkIniT+K+gp?u{OoduTVYW0mExxT5l9n{!x{2Ec)8u_01z-DcvSgRw& zA~j>+(!g2+0v%8+-S|)884=>=pTnphKU{btXE9z`utq^2Dfow! zM7eCnE4w~J(?&}04{^B#&o*D2H0zuhNcY_@jtexUNBkjdPypb~GJMo^v*u$BB*nwqPEkmaA zZ^_JLvEVZQRAt-ecU&Fn65!q6sBc)$5=f=f>$i=tT+-uD$GrHrIp2ht+DHd*RDQR+c#!%)pi^T*Ba3d$^d4}9U%f(+fuz;L z_56;@s)8cQqLRS2do5|-=`VSp71>wdM$4iZLPk*yKfY!Vz+(OSu`?K$ydIP^TX9EF zq*9bGOr2^-9}MM>G$!-=cj1#I^<*70hBPdRtlo{Z7|hiwjEsM8T^-v%d#w8M_+Y}$ zMVul$mzURV$5b$TO-}r;TKCL#``(5ppV<$N8=Z5V2Gs1Hme231=bw@8&lX7dA(Qzr z@`uAag+2+&{c9;bPF(@y`3IX*dN#GlU$Be;5YK|}7cCv>G=G)$NJ1+jQf`ScT52OtuJI|b7<-3g{vgEf z$9j_J3ika4djtDSA;YXzx~tc27Ck?IZ*-cKQF0Zw;M0!{J2)pQqCNd}4egf_O3x}| z;Tjt#&Go#pQ;TO%^sQ#n7>7!+r4_qds;xWw>tY+r{5W=Wfk%~XM&X8g*sqKe?R*=Y z9O7|yWJVgfmYqgP+qW=EgUlNxZ=~2I`x2$TsCey==#=X%ogOcxyEHPPZmBA~iZ~ za55RLg9vWVR76tPQb1pF{CFjQTo0`NiEP0izi`=hmZ6DH_s2p&%nCo)^1e9qE(RWM zv&>LB%!5o$-eqo7YAcVG(d`g1AOM%jslQnAKJOr7`f0_1hf~`tiB?3}EJ+=54puvg z4V|DcKdT+R@Tcmp!_=ODGp5GB)Mhswv>3#g^yqDtm1EQPH_FYX$A3o*9qsP@Nd5Uz zz)(Z&o<^9hK(AJ--kK%l$|AJ|jZ$4LuVi)3s#QVubM&`2_xEF#J3a}CqXrHZlkMu* z?co$%?f29nWA;?Nsvo;nxiO!G4QEIkv6yLUEZoUucV##H6+1dI^8k25U_@=$WS<=$ zjMEJ{!Z1gTU5=qrmrl9)3s0K+2#iMJ1x#m#81StSRQ<}e|Q{r@wm`9JZ4 zW`)hf!;?1JbK~^m@2}Uhqm@wylLs3UHeFFn2h;9bV~|^`Kbn0%XuDK}F{}WI_D@6a zGpCdEi(^`ltnP+3uDPv^mVNR(nV6~ey?QNh2f38yjz0SnB<{FWcc;A8P;!K@T(tR6 zrNGRg>Aj~7BJ*kT!-ubXQ?jTu%KCKp4jU6DaEWL^2Bqb-6P-S))3if2QDqfxMJH%A zQnXq-bGj(_(6|M5QsaFhh>@t@NPWO@5u~q^%ni84qID zMTMyQf&T%rW*S*oWj7<%UkBwpAkV?z5>SwupIMja(e`7Pf@#7;w0GTF@~X$0r+w>nfeQ|$D%;#c^!(o~6?JN>+(Y|ZjAltj zZ}ef4^6mKqQr1;WiBVOU=1Jma<2uWr)A!1)F}OYVJNVAsm{m`l43BLdir_2z^OFF4 z9Gq==&!l-CJi~O}dF))YwxQ;FykFv-df);M$L7ezR~9i75@w}R0NqA(m-@WVD;jsY zIS%HdRkXQII_1P((4PMZn(D8ge{z&3erADrkh#HaP&ecS4CVPdTmtL8<)76a$M%>O zD5+e_`8*rSF}aoEwwQ?iat5Wt<5gBmUMyq4;!Hq0TBm>2OQ^>NK}mC>2=1HrY4 znv!Du+VXKXrQE2KsT ztcBjr!43Dih|y3kOMUE8w~-$&v!rmP?~~1{M}ZE@wg~E0oIC*oP7%n<#%UZ6 zTN7|BvNX1^jhW$V26zFT8Mi{go?yNljr9cs5LqRL4FjDXx`|!)3&tMD%^@P?eXmZ( zvj+x7TOL-A;;`OHWh|DR0<(7sb0px?xGMP8;Oqpf>lxZ$=Y%;{vm0!>!I}egxf26I62p(g?V$=^Ige|-Q?#ocS8aiQSG+Q6K1`o z9!lJ$RJlfH)VC5ZxiFrR%tG8KxB@HxYVx5{M+)bZz>w8i)ezAQb$$9v_hnr}?mttj zVUF+;H?2F@F?2J_SmUs3KC9NtrGK3@eWTG9HGSjED{^hwW#hiW{`8?c4+re1aS>p5 zeU%PO^SKi_@!qGv`c|2T9of9U$llbR?(}RxWgt-Vm}^WE*SVj?%wVN@Gvn4=cBwl;SIP5l*^3id~RVkZ>P%9E2`90C_alBQ+ z#(sCDR})4&)p4me`UtYe#y@c@e?tRrqvRwS?@rY@IfU5+m=WwEZtn>%x^G0yoOnUL zqri_j%a1zk>uRiIgo3D28ghTNHX|b=9@yoq+1c7xkAvKGhN@>16yiT&1s@k5aV$ zySFk@X`z8}T^}xt*Hf8+v79F=SI@v%O17w9_Ppp0Vf6$mK7{m3JmL#ZV z0jV@bH5bFCF?q`JBM3o-6~7|f1WcjnBrO!lrq!j!)Hgi$35(y5ju6Q`sNf-PHhyS5 ztpn>O%)A~fbheasRBTgDM=u#hRD5_cbJ2t<%lvJdbP;*PWiLEk)A2V~duGR-zp^7) zA~i(t!;_hCtLgW01ndHfUfb`DJ|EvrBG0l*mutlNl`8MWv^B>hGX#WvEhZ|Kx)0_eFJYhny3A1g+lBUhPjyre}DE%5U42H zPR7j&qYWF!4;xbyK+R?{o&Ce_fNMM-Z`D<8?ov$XMKk4jaOdNnQF%6=8@&QiEYV-3hnk%)iVOhxy0#P8qFC{txXZ-wX+KJu6Tl z`$_4ajtNkkId-Z7+D2rR{MfHMBkAwfOw~HP#zYR+pr;vav_7fddpa|Yn^{Xg9v*J? z#Sh1J+Go+s9PJNha&@>dG^#!)A7);X6~}B^SeG35J^#oDWw|n8N06uXPHDK*KIi&T z5@A*#YH_IHPb)t9qh|0+_Ckf4HBEjIiGdWFlkepM?&|t4--E(@v~y4*8$a5fGgv5+qOOn&d$hlr zPhSvNfXG^-+;t?_R>COe0v=JYbmSgvNxKr!uS3C0=s1PNpeN~&JQtRgPoRH!A$VYMvo#&QWb%7J+ zX?_Qihhp1JI3rEV6ZTrj!Gp%kM-)=IAv1=u96!qXYnuNy&eKHd_kIc_Dwr&#I9t_H zNMTK1Ng!ELG)3C%XRvCTpCtV$ai7ljO;FBVi=B+@sr1TSawy1Ja@kZ*C2U^fVbA^L z;z`nX_>NF)d9fwmw*;en^CDzzacBCee%#7)rVFpg%gp#_=zMjOT_#`fi7eb+l=;WR zPZfrPOq@r=94TbstWTvsd6)3CjwtZ#|%N@ zxa1DdhmcFbk%9Mhq~~zwFfS^@zA;>~M5~+*n03SP(-qRy#>DqZ7(S4x-Ai^TAc|S> z{9Us()>HzExdtj1G4y1D5^leqUy(Sz2Y_@qf35gn&4PTG!!;Ny3xHFdYSnGse|Sh{C5O^_y0Y=<|p)50t8lQ)a*4qF;?+9VnP|zCh<#kpkZWjA7rjT zVH*_w>-!ylwOpYDI0K+Vu7$eS>7Ld2vLPdA11kW#(twBUe)8dtiTmMZRa+3@#NNuF z4TK-OFT5y)9rLLlJAYptE_@B)ywo?XCwWewT>H>^lvl6XIx*E_r%P5`8+u~^`C9)T zDBMkCNWQcF-5LO~0%KMsjjzZJT{HJr20K6y)v}p_=nVinevb(4et%FHm+m;Az-?{u z7$U8{yt_65Dpti{Ae*nx{_c;|Iawu(x>tu$2@S+?K6`N$_r{y%Le(6fC&pOdh21(u z($Rta1xs?i3LuHn$5>?>$5>;~2Ae84x$}_~Apf@-%KwP3Gyqa|@(;H;b$=Ot0%dU9 z+H;TNh4`tNrQTF4F1_kXko(+TS%&7|Pd}1b@OJLo!#BTLP#wvFcGVz*YDXS!8oV?J zU?wPZU(O7&24#{>7r8Ks1!V3+n{iFD4f$BCh>h8WEa zqi5ivLK~JTapnNF^Gq z*;vuD3RH0ky*vDQ7F}*S1VI=`D5hCZqf0*`2GUWL@l==OtBfAB!NY9d%{%o`EtTd! z8OVb*M>4X}3Zrnv8t{j< zqlP=YkF{15i-uT$={vOTljb&1PD6Ii%2PQX%{s|AJ^4@#fb=4;@y~2jzHsn)K!yZ` z3-AN>N75X+7zP!4{{LV9pIr?JP9^m4_?giDQ5^Bs_1QpB*@qFyQ9eIg+e$U$n(`t| zB#()Tzm=jY4{esuXAPfFwKoW|7@rpmQ1JNr^aYQ`Z2A40%44`p$mD-q1(Q)niz57) zNGUEA2_Gd@rAPsr1%u5p4)-^FU1E476OrMc526(eM(2pl6hi{VBMBxp6FU;g>kFZm zV<7jE-bMIl_&>S~QeP91u~IiRa&918iUjJbtfrH%pe}>iSFJh|S6Qm8cpGDO&4{R0 z_1T&0%B+l+52T85H8*@{QqV8GMSPV+^~OJDpeP_yBGLF%vQ3?Dk@Yy>a@1rbNub#I z8c>>qm78gz?vr(p5Mfma%(xRTQ_3*6MYHI|O-W=No)T*|a2k1O$nI|uv|Cbxpwwv% z3VXo2R>rKM|NHRN)VDFyJOc_Nmf-YMfY>s2FGA!6OUj>f3&<@&lyV%G-Yb@L4}0^D z8!#BVQe>tZ4kzBNY?lH}-;h%O7r{>E$dip48$dKyzxgdyzh2(64wYf?c^diw(0bX3 z+(%WTv0_J3AG{7Hpb;4bbgMaVkuKt3TnZ0WS`CTto$oPYE~Eg zON9Cuvq03y8pK4;o2mPPI*H$3SyUXZWWQOvH14*Py40#sPR60F3+iwBtWh_qg+KqP zF#dv74{+XqSRTfFqPkcQEp7xjs>8!tho$F$eJZw_X`-iM1KaAyaO1~IRvPwaAc_F5 zZ1bzdS1Ur$ZrZw}VJCEdbE^JZUGhU46kMD}bh^~!oq)vvSO~GutR{!fGpxo0+3p`}L_1KNt9+P0h0%Sa%VsMmhP_7IJ_+{s3 zCljEmXS0;zpd7YDX3fvS_7N_Y|L5I|1Kb!Y@U0Z#%TW(u9SQgbdw;X*anc63x z={Z8y>bktT_^_sbH1gt8fq$u{jQR6MWEIEA8;pTVZR?E!e)IkoP+~K%Nu&|99_>G@ z1#WgMU+t5=gu3ehRi1{jP&;??sXe8jZORl7zHF8)3Q`L@cCL;T>)AYp+ZQaiu}lFm z5=`sihr;vEgM}l_8n@TRtVfET z4aZq@8Upk4OeewNJ6oTu9VAQCZSttqLx*}A-oiU8K)827rUVhJoMqIw8w z(}HFgjKk(%qgYlxX``Y)Q_TDSCYEV3>t`y6O8&dJHT9=H@xkloATOEPIud1psP%fC zL7)P7R;6?Wap@+I6-7bfz|Kq)Eu{Tum`rEd< zvs}8?xOUGOMKUG+F=V>;za4~dnA81;K5iLAX)XmBvXJ6k+6Hqg_W@Bz>3mR&Kh{5< zk~t4N)pez3r2fI0#)zsuY-UoT%S%`9V<*1Y<8`ww#&mCvUX;Ly3rZQE-O$6fm%Usz zUagMgzf!A9p=#^DN02M&5&Fi6AVX%-JC}E74aAaQ`WvKhbHBPNu4<`osb_TtG6gQs zGhR*{ZwG5|h&&wQiq`dID`wTse5p$#C3E>)qUPk`+v*vmnzmWD*NG@EFAv5eP1w1C z>)RUDCz{OQYvP7?Nzc(?0jt)O-rZ5VEm-iw-^7je%4OIHA#;~EjGWG)@MU}N{xFuR zQHk~bKmKqOz@)QIFH|{CbQ3;auenN(ywM{aaw)$<(5Ph^5Wp(m8}`<@NyC|0R|Ta!;{o*RD5<*mcx8Y zy>4fEVEsH;=9tU<%z)hwVCrJxohrMam86Gj zZMSmbeF2ZUak&SClC4qLaHVH~QBjASWG{nh+IEN}FV~Zbl%LbAVBUnydhdSb*UP6r z!+XG4o9U*2phyTcAo4SD3ue7s0a*JDMfROHR-zXs7G#)uY?*%;Qdowhf<1v^q6{}D zSs@*Eqf#*M<>gVV{!BQ5dKJ_5A@h%AB4!DLA3>kDh{v*Xno%U_d<3)iht^x7ox@o- zN}^G5AuAGx{!U!beIU|se?}_No?tSs7Q>}8&++}V8rzg ztQ^oRRQVXl;$~TsFZb{+WX(RK?-9CDAAAdl{iJIT(3vlwoJFDtm^Tx?;sOl~Tu7#b zxRs&%HnconwR2lYL(+0Sck#b4foh34dR=`dP>1eDPvITG zDaUAtbxsnndUcJEh7<76mmB@{NnJ}!p=wW1yn|1tCK1C|!>UXNar500Z_!H3`_Sj< zDTYxP(eoa=Azc6vCDwj^3IfSZ3AVc?)#X6(18ykE5-PsEo3z9-NYscGD1(zT-w}H! zB}>Ye%@!JW#yS~0uhw3e#rqujwwvl6wV=<5n#_DEf{W;+; z1^FqT@>n}#c7=~Yh&$l1Jw6Fm5$ZYMh&jAheRO$m+|G3+&959$N;vdbxXgaUS(oW% zw=&w7Ey{%Lrt~hHF}_qrH8c(&Z-^V5oFSwYv+jxKaXGXaXiNfRn8s{l4>hzvH_orj!W`BysEtD0-xVZ%Br}{iQoJ%e>&pdk8^g}+49tlPS|pnldm-Vh6%rOBk>Tdmu{X&K@touXqJg%pufTXh9XFD z??HAPoFavtEAOAf(8bKTK%x#O>jrmR`f?(+L@bT5J?`T!6LZ7rE4f6H0i{w5=vzCGp>pq|v85op|M-g>8pi_2740AXC)29(7)EJ5 zeDM`;qTZ#1TFhfKS0&vREHL-)G~CM-VbW3aWozpfX(jHSdMY?&3gu3Kf*b=ij>0%v zz)1tT-1I*eJ@Pd2l$j>yzT-W8#$JP#fvPwL#-Xyh@JIY0w{O_D(M-Y^RM&sY0tqst zk}LuNcQ~#@x2%;yz;Yfd+Htx63rInYfs%`kUskei^NZt=|q2~^adc# zwMz^|-RD2bt^1Qqivu~AZ53eAfOxWJ9T3gjt9)26)$kY830QQT^x>ol>SKTrs{(*~ z{63UaO5drEg|NQk)VnUgds^BxYzH<1zSVg#@wGOW_)2!PM?EoM#O=U?t3i2c0!Vrx zBbuND@ck+ca+zKaq-g@SlcNL+^`L(4Sx;Q=xS~G%E{CQd_i6Gtz%2%SgM%vK zHeg4b3XuQIjQwaqy0XA5#GCs0n>cT*f$mPe4qK1?95&NDXZvJUz6A!I`Xfo8aQuE% z#bf7n-7HZ!D8sR_;?VrC7$6cQr~oqbFiNi)2V~ms)FX>(!pSj9yH> z9i|~PaA4f1GkP=^w5f8wC6}_lJ~miYOU|voJl>nMQx2ww=^}FS&-GSs1$?u2ENr;6 z4$^AVc>IyX6vm)A%76i&PRq5G4)H{o8n|I7VIL&}zrs==(Up@y!+~rwXG5*;0WrwiZ4J zHsFoThP?%)j-&bWO7oBSGVJRa$8rERq>5gg?xKC#s%<9Lf%c;UaQ~0r=*)rxVFRSR zT3-r5nG}@QUJ0&m0h>i90Dlq7nA}3TUrL%C{LTm@qF1&wN-d`w+zMRoag^PBnpk&@ z?0;K4s=pR(x{CO(g(BiTDBIrrauud3RnQuh4OddTsrXE*UJ<=4~g@M@oU$J+ZfWs_QGGQx=XJa>E!$dM_wQWT8gpwrqj9!N8 zhFC>oc6oV(j4>8F78Dcal>~D{C%HV0*`0K|Sr|j?wsEmYRQWEWFb5nlOCI+XUnytx zSvM0e4*HE8xD5{7hCK_ipdpaX2`Nfm)lfETxvnHJeXu_E5=2ZDp~IQ2+JB_XMv`q& zN}_4!xm70H_)2aw93d^(*jW?%%i5wSrQ(S*SMB2G3$|SqE?b5!iWw_|c@xV&KRpoB zvCQ<(m810>nntd1QHpzc+~EPmPBP*KVlD@*#2djkah`qvg&|BBp+&HS%i#U1s-!C#M3b5OTPl1I6YdR_#xyj*e!pwMjv@S^h0-Z8tTJOi%4j9@ z>XVgZWm~KqRCW*#x=CFQYL{&kpqLBnnP4vG3QQu(QL-!A#TfoK%!TH6!Lo@0o|1&5 zz`r4A4$31@a-2-3vwZt+1};+z=V;42N$dOl(u8B}E7Gm; zUy$&7KUJy0ias%`=HnlBlfLPotnIu$%6Bqc;^Io+Y&G1iK~N;BgCmowKOx*uku|_$B?~*DfNqO@!$FM*VGbHn&P_2!-i`Q5S(I8p zi<4dsq*Rz`w?CLY2&X@O{=DCbzefE>*eX=AK%ZICfxUdAYw7&cl1o}?Op zWXxN2Ae)lti+#?Txl-Q4U2*rpt5G?zom=s@ofw=tW|Hli#!_`*u*l{Nla&$lt7xl* z{`wd@T||vx;70G(ftLf5kjSkBpPlT3W?`*zPjrmb^{0rCLYc1?9HbLF-yC~{kSk%aj_5aS78|p zrX+D+OI#Ezv2$MpyGx5Pngn8ZPOEuRh8YAs(jEE5H+DQ~u(`wJF_iC`GlzFT0y==|;nu!`9*U6Qp02){2%6ly=%3+qyp{h<5fKn`&{?iEicvidDznHA4z7Q z0K&cqCQ4)xa6?gzgE|cXJY-e|G)B^Y$raxY+FyFC(-z;W56%1OlK-0$WY$3)M2U7F^U*+J-HHqh;U%&LIh7DdX%%d~^9g3U}v z3dHA?G@?kLVLC%`8I`>V5C9@Q99p%GZl5bxt$`fDxr-@&eovoMN~64H#%lu9CrK9- zv0y}O1+D7SZCfI-RB=%Y_pJ$8UpASYyk{M5HUXgrRBTN_f>>Ow6ak8H#*o+m43f7C z59UKe%D@RK&BQ-T8JVstg)AFjF-Y%z1jM3_O=75U{%2`)UqApz%_VLz1?BkdvnwP9 zGF@lly%HP)GUZWF(JbV;Oy(9(^>ssOl7M*BNK0Ekx*0eZwf!RyyW73oU@!#3OlizR zRt0}30>H8d&<2ln@VGw)t~&uAw6xT!zSqCEn|?8Q<{HOs@O&NEa)#;7-M}Tv82?gi zx-k2r!{Lh8DwvQgy;}toX{cXpluAHx^t{S4cF@6u&C+}zM2*@B)ZEeeCJKJoxx~FQ z^t@LhLE>Nc+cAr(6i6;znwh5x6`0zzwA?R~oWRi^DMEweHFjGgefyA6@B`%Vo^l)u z_JRvjO3KQr@U<-}reXA|PiMYHUicarA`dw9YNs`AK{Gk6ZBVC7K6Egl`w9HRJN>N+ zO*Y7B!_?I-^AE{%j%rn@=x=Hb#ypb`hxxG5bDrV*UYE#WN17vnm}xzf16>M58%Xq~#RW9I z)v3&sCVj^l59<{8YW>lgl{~shbPC z{>hkn(?(DYGUSOGJEN+94Vu3=W=a{GK;0f2X2ucIzgLld_We>T4f=pAA&Oy+=?V;w zQA0P%FZR|U$;VPgY-U(KH@FfKT7z&8!5fnXDj4i;07cv>G7(P^#vW;_kljlj5JFWgi(Nfup={8k~w;Eih?dgu&Rz7&sNC8x?^k01;1B9@31)RS|6k#3(MdMuMR7;Q)lmq22#tSqojstb7 zUM&0yj5Wr(b-*;;X?CQ75qcEa521giE|VEpH``s&ksz#DVbD$nrXX-u0Oi04S3M95 zLrtd6?PNN-VA`#t1TYDlPSONWi|cyKD_a^5oMuK0Na~F1UAEA`?EvmtH)aXW1~wkF zS=OWO0*bxpITVs=L-ZgWWGWCro2=u)=)NJuNO@ZE6eUh$+g|JZr#$BUjy^qDVA9Am zg^RK}q&~O1#75K<wSKcS?HRFa8ZoscG7SK zE<@|jKgI#|4>Cz;{hWYkDO$0~H1|!RdY}OfwcllfNtgwHS8c!pkb*S;awGH(S?p98Y(Ok#VS!F1ra2Nuwf6RRmmabnzGC45Xs<#B*s-_5+g1 zOAdVP$yFF#Y&o7H(~~V?7^x0xrh@bz{yesi%odINBYGo%$KiFp0#gkHxF*AdZ74(2 z&k&UAVRTfP{>%hz^`Jw+pidoS7=X+2tcFgaxK~5+5{M#MnkFHJIjg``5~ZmUzh~Di z(ytxR>VpL1MNsD?=qs!L<%i$u>IRcYWSuA_wdWye+lJABV|$4!2--FJqBh6Clgj5? zG;f=*nF5=YumERT*m2nyfb*(CbX_;7Am084qGGqqSWr$RG7&^IpMmR4{1CPF5U_Ee zaUw3&>+gy)&?;+-)_~9mO^1lDHZVsXJBR3JJvLqVKS5CqkmSdo*3yAP06HS70`SSz z_pZQxx%w8YBV(h;YA-Ea|3|v`43N?14gC@Z`kz6Yn-%CAsQxEBQryvh zD5by$GLEDZA+OWVpw!bAOIHFqYwmznoZ!e&{~RJ-Dng0cx|Q5yYrreoKJWdX>rwzf zWQt*cj-@D^trDI7zdPq^H$hfoS9U{DcGWD)<+=p-sV%mlQjK$Qtu-JhM?T3Bf^w-* z)dGih(VIi_{qYNwJFjsIOs~|~*TBAuDN~HRtH08D<9Ut4|Lg%e_tIS-N-8kW#)Sq? zGr`_ww8V4+5H~5RN3yaF3O+?ZC6g%@Ukq~6-9GWHZ}+RM$I2}eL#6V2)&c0Q23_6{ zK1>0*5936srn6*0%cXP6$4=78KB;0>e~U|Ifp=1C{jUx{@OsZb9e{w%I6s76{Eh9o zkg?^m;ApY^NAGjzufrIeiGL-q8fsh9+N%Z~6F^l}0~&XFZEF|eQhLtGH28YRup}l; zDPyvW+SISOC*BFamZ7TtgDW%%$YW}iBITtIgq_#_`KW@SBQW5Xz^_z6M?I$=v<40K zieH@a8j9I;wVBq&a7;FPHxp0`4}sXeSa(6=HvbVdunG?bEi)BB8$Z^?l1|lKh9#l9 ztkq2o&Lq1UNO%o!M5jXKwKg<=C?89-&jN=*gF&VK#?b z0w;zz80=mRa2pfrV-+^@ogb@ByNGpSrAbj>?SUkVh>{G{k&0lKix~{{vDcZr%?b}( z=!viI#dp%r=PK^K5hgr%pCC3_$RQVW4^0=-9ypA#BF)v`xPlY1zFyJiEXpJNmi>@C z&XI{Qdf_=1RPsYn{oMufBWLgQ$1Z3NkDoPmGYmA;KIKC-Kby~+tX=-@Gx!_kw6Z=k zMtA^}mDsR^&wyBJB-R>m=aSMS(@VXz0yTA;VNKO*4R1h>uY9RoZK%J}^TZj0b6M$O z+&Vwma#>OKW#C*v&@lH@VZQ)!Ocb(o4_D0upSlK_mge=>0&|x4Z#u5U4A|Eo3awjb z5uy_rjNBqg20ZzaKtlwy|AujYIX~0z&R(TO)-AzeYR86YpZuOdayKG& zO44!$XrPgu{s?KE#@T zR_B(bZYX6eqX~^??K5%f@?|5I-fs13=Bruq1mr-s_D|_SAZD0U$s+XnV|3w51tP;} zePrB$qOW`awLO<_f;l8HG3zF^f?6($Q0_^6ImWK@owsz9up|QBOU&_M44G8g{d2H_ zWsDJCeX&`;(HN!42t)#gG+_5;hH)rG5~&PAh>Ay}OFDoi_PUO_x{9jU0-VlebM<%BCv5z0rD4KYW zGuDvtDhVvZ7~HLzi>D|XWd>z8uvv&MF$*jurH1lpNbSQK%emxA6ekradwBsn^bIMk zksV*(0B{u}YaIH5&kwF0d4Q%Ug~AR2NS?c;S`2f`K}h!T1I(7hSE9JA z_bu44*bwlFTLAUniVLL+I)NsUVGsDojF?Ru>x0;4Ezy+mCF<@l_PmBmB&rX*92fE&Nx-5_BEA!ALOuWZ_&o}!Dc&JqBZ){cm8;q1>jP2Ic5#BMR( zSPeeEM`Z6<^T6IgDrY=aAx;H*WA(&8B$(`(C(2A;uPbM=UJj4~4A3-^ol^nTGS5=t z&fU)RXyHx8%!?g>tN^hi4(^t}XP_HpeN{s;#^FtwVtXmta(+?^rv)sI0(qOTUAn z&$V1$8G%PQ1d6hwnEB+vzaa+DSprSN60-QDap86aw{WAbYM7g_sm1tdTyIr>NDtbH zp)d8(%m!7q@Tx`11O#y$ajfv804l(sB0zQrgH#l%e16zW5w^;OEsj==kykai9Yhef zQl#LD{){jQQgHv*=OFVyA>%ncjl1u}A^7-2zW)Nn+=ZiY|7@yQ2R=W0Zd|qAC29$6 zaFSlIOk^+`g7OP*ta{lyja#k6U>K@UFZ2ug_zcD_69u4_kPpg{X~$E>fhz{EPD-d^ z!D2w+U1F@8h1HF?EsF6PbAYd&12UGFUGojQgl@MW>#Zn&s-f@Caki)Siih?H)uKrp zkTCaAxE%rc2271RTmLX?czv#sB0gqwu8DFC@4U|EBUUR+~R0d=P# zlm*Upk5XJiw-4X};h` zX)dt4&=Y}TMN#&;qWYq!5?p?F_sCO<7--~yNLQ7pc8{Bki&Nn)Zmvhr%U*X~XB+yO~eC^a5Lr8~4`I?#&i*aEUg<-8jZ9QKee{912kd zLP10$piJ?gwKWDLeSd3|3@@)4W!xQm0B{|__)RsJK?x<7(@#Anhyx(7E5K$wWUE~i z815w8&y)jW-T^#!q;@~$a3D=w_E+DowWkQqYnyQ_F;`V&bfvy%O7Z!z9-y^b-L6Ff zafbwfq7EAp}d~p~>Z{H-J@t1SzDz(Ua^BAV-j1 zzMUSt9SbKn3X}x-c;DW84vJyG*J_r6*r#F4ZmAm;iKQ_j{U_m0cqvHwxDkzovA_

    H4Suen#sD0@?h)EO`^i9|_0@LP)HGz{!YS9v1`1O>WdeTTD3#AYFYQt+OuxgRHfIY2T@vu)?b5fRRconvY|0uYzll^ zR;v9J;7le0vSdI7qzXWJo(*S!sqKKu9_Z}Cp8+lE`bRmMG7@Fii%Oty90n-o6+mtk z(Dz|2cZ7D|=I4~n@^BS?hRckM1`3-yKxTN8>X9)Je3T10EROyM!V3U7%QaB_9h!EX zBX9>__OlK;-HM)j4Jfc2+x`!o7ZCcvA_-+qHq6ObXk$Y8hD3q_9|AN@ShR0C4IRt4>GL7Sk(?is&!9DQ@h+44t)O`~#jKW3LQg5O@=Tn; zmn6Jx+X6C)jr!F8OLV=gsNq)~CV!^4zHInR_z0yNa-*(&vA9drE#jah;2(?|`us{> z2hb@OmDvIf>Bg|mg+P#0+p3|*wk-5DVFo4YbsLUrBR}@#+346rEW!uDcPRa@=FU8v z%Jtpj+9fPo=B-VpiVQ^=ijWMMWggmwD3u5aiDigT*}F)FP-JCg3QguQLn~B<%q*)# znP-+|cJ7Dv-oM{)+~^jEmp#BY>jIEigXU01iA;7h%JHNb};B6 zOrVV~Zj_|8kxH4UT?ddy)jPS@Li1%c(cggP-WN*JDDtmxFNIX&6zhB z&8(iMT*qkDnbNDO0KudP9)9-*DHl!7+4(mfL#M|;5`d@g03fDW3H*3vv&)k5GL)(0 z)d#C5{12yLcg@7==0@#c(47sKGPWmK#SiiUD^4Q)J{nm9XNa(Mx4hQeh-tHBS;@8; zi)9~8Tl%gUwy~yG@W~$=cZH-P*Tu(~eTP8l;FEOJpX_WxpeNC*qYIt%F>+VnD8Yvv zKZ0CC(yjz)pfD2u$^midEA3)hQMmT)j=1o)RQA)B^#(^ne|^oMV=HOmpk~}^`14OR zHw{tFhWDlQ)up2lAw)dcm~6t2^QuyA)3YUAj6U~i?`o!Fg;mU#9=nH)$rghy3{mM( zjhn*k;y^?RR@TPndyH8F$5T(Y0~85jz{&)fPqAJj2h5%$ZlY6(H5b2R%{Pkq%qd*3 zWQJg92u#*g@cK^*BW&bq^#t2(0{E+Qx{EPKA09Bs2XP#N**g>Y8G)V$;LnB3`MoqmdAO}M!iae=^%{!zP|Ubp$= z5qcULRapJQ_b0#HZ@(ZxoB5ljB?1K@$$-OBK2&IiNl)cg6iX~rM-OiQ=|dV@Qd-mZ z>bX%H2m-yYxf*hh+FKDKzdr^{I))U64{GvZLfSl!z$Z^bBMWQuo)FI1{n33wRmMKJ z-LS-@fW;oQhr4CREx&5xrlN6m~5=$GG#Wv)9XfgA-n^y zhd-J^>Lbrv>I{aad6IFvYkApNV-2_03T*Ae{fF!v)0=O73w@^7v!>Z8ay>WdKvtzX zY6X$qs_gxuzc)o@g0YSZ4oR~bizmJ!Df^-Jps9z=cqR%^3Pa9DMy_A;vCd`GWP@D> zfOHyjLd9!qH(STrw{}ldJ#3vj;ol9vzBGMSS*?7V>_ZmeRm%Mu4H1=Ksst-4@S29B zs!zk~+d^N~Vx@?Y?PJ;y;hQXCiZwA5y)ngRpCPq5s&RW|QWQDm4r-sXt2gZx)ViTl zuHq^jgax=C3Q2))OTQiVK9Jal9~URnKAMXbRNO2ccdsdAmZj)rbibGM#DH=#fAOH! z*N$sJclR>me^m8i?QT@nueX{)9K}^F*(48h&hzPR6QB4rwXm+NP%#bG9yms!W#m=a zaiWC2n5SjBkSTK1Y&z<87d+AvPjcO%0|y1IIBvm!bEMb56T3H5$jDC2!LnY4tBz4k zrP{hl?JNnI8 zB*T(m=$lccmQcx(>aA^G$Eas3A{JIDC<=829?;qdtrR!`nUuFb?F{p@AHmZxTVSoz z?Y%O-@kX@>g564^4v9K=c7elYp-<`5I+J&tV$_0`7EuThj@B@?yT&=#&XF4@!du1K zg3N^XSP5+CW;?Rbq%e?mIUj|G7{bWE=?9OT1IEZ{uGXJVskrJf*8H+L2!Y|PdZ=dS zM8&gbtp|YnseFtLjR%yG6SB|FPUmNBKJ+LwI(M5w`WueG=^8FrXhVcKELc&@>6E7I zA()5Ole6&zf)5=FAm+7DAoy)XXu?&XAVc_9&7ke@uW%z?)lrp9ORuMLdtkF5Lnrvw zJa#i62L%5^+2<}Tio&7b1n`YMrrPBS7n!C+^imUYcz1$MsAiUFCg=d zKZrrZ4GlfyuSf-g3GSfR*V?DHwGtqyD?^inN_B`(fKdXe1Z}iG>xBr_VA!i!kM?=g z4jhY!fOQN`T&ds>J{NqB1(uJ{rVuAU^fQUS<(lPfEet1xpq033pVj3##VcuT{!BsD|JI9p6HThAUY*``=-I!1HyB$N+F=yPG4<^KV$2KYMx zhB5rX-23ZNjEj91)r+j1M2K76L*ozx-^FazD)pQyf^1%ZM2a~?yh__YT~t8#*Ps1o z%al?)lGO9JfB7P0q`uMQ^Qk#4M_xndfjNZS9zm4t>;-KtG<3oWpbTplsIDnqfkWEN5UBwglw}j9*Bt(b&T;x@C+fXWHUT*WB@?Q{t4u zXE#00azCbqd#%QiL?{vh8nCBxn9DSZJP4Odf24MFY+0$}f92?%rbF}YX*!g%6=ATL z3hO{P>&3hrk_Y6ty)k;GQS?ua+D~o}r;0@Squtjz-Xe!F6gJX)UsVvy8N=1_njLNIKHL7?6Gi!-YS{5m3>M)*7J8t zQ@x5PfhJDoVC-&!tq;LAgK%h%0>M^Wj8ce?ir__-U~1o!rMaP=X>zqof6ZN2E#i+% z+)1A;4{ZHgXq*4R2txxZ7!v=Y5F?uSaS|e9X76n`KM_9=%)SG7fsC7SXFL<-y z@7reL<$s*0@GZ<+4i)Ue_$%~-re3+d0HQVWDu@vv&CZ4)&@AnROeu*(xWqRVq*Hat z){it5K(MGw_Hb8m43C6(UQr?$$e2I5gM#K~nsL4bsxu&z+yzO;M7qylE&iEFX*^|~ z?n#s^Up!Yb-=gaR(OP`p6C4&YAc;)RHb7FYkbv0OlFOdpjn*l>R7>qVAJK)Bi1`;Q z>BgUU_qh{t672~wuDf!0ZW;W@vB)cBJ2BV7ixu)4#4B7kuvUf(@6C&a!ob0$HEd2u zW04M2mws&@53rY?;-v{Ynfb0jKHgiXdPy=cyD`4haGi|@!<_&tJI2{ukKe?|LHV?p zXZzUms+fM~vnvT@V9-yh6|fB}-$Nm&C?VZm_bhTM&*?yb41n3^Y(pYsC4&jh*3~1y zj1n&)irNKfmY0$jYYw~sb{OAR7F_Zp;0X^et#D)7u5DIl@uRNyc+7Ns++-(v?TmEe zRN=rnx`59i>eQLDn3^?@oG6xrz2|-$aj6kHCpQqMP;KF_DB4XuI5~*Scj@z~CKWYeT zYCDwe)Udg*33Wf4mVtTZ-|35D(SK%!dMa?@Ad9sJv;)Rx2>b_J39D(?ftuR zP{{%4@KR3#AGJ}jBCydwaYlVyc}R>QMRc!ByBR<3u8h4QJDvF5;suI|FO}%sBR}-m zWR%EIi$tgcla_yxaAmsVxxP-VC~>n51l{l+YQ=^>76!u=(9p>TCiK`9!Ot>*WN6dt z10*;!O_bkTZHX&)fTNg3hW8;;zrnJIM#U3ziywcF_!ijc_qmo4ejJw2ujT%_sp!Cc zu$eE^fFCIsS_!ypP1qG?j;LjNPz6X zn7KNv%(j-6vZ1$50%0%L;qbv@ejqD|rr48A%@GoHEVsfHX{*_&IYDB=Bu_m&F<-LJ z=EwKXQzQ?ez|Xa;(B@zeICuwcCswOaOWaim|d>x6%I7LXH(6|5c=Hc0rp-V~ud!wYL!Nem7#}#bm%L z8;x+@6C|1BQ$jZeR^x}f#~~g!)L5$5p_C)V>-|`HPKpsVJcjxE zAV`h?fyw*in3H%p*>;}6Ohz-zTG|Xq|8*gr+`?Z=?WkBSBEh^PxYw*A=3W8&K=5x! zIQ@62yjYx^|KNq=lX8LZxqv>Ms%w@!a#~=m(BM~uolTPBN|S;dF*sRxCnFA-wAbv&pz^ z>O#pz?eRV&#|^?W3J`R4&-;!hII;xI2+=zI6tKWirkB>W`C%iZCJB}%xHzrMCyJ0V zD($9*KjMlJifmPmrMhAv)yQd8$^D6OO90xCU|lxM(<5|&owc$$r0j-FRhX?rd0@#S~0s#q*rO@X!V{lrL0OL9`ShWUt zivjL$NQkT<-;FdGd$uY!<-*++_=H8w+j= zHS7kKU}R3zi|4)=r8EbuqU9BNbRn*txgcs(?4k?__y`jRM`$wK)<6*vTJEEkUqYE% zD=Qmoq>V0sA13e5Sdo^^@2wf+@prFRGb@ktodsmZZ0yo;;fX1Dl>@O{G&LXW*6w_y z-x;6@vM>Wxoj|*|B29!q8;n1*(tIT1hm-<94t*S$khc|U6K?HzT`r_S>->11%@+iz zNPrMRbVQzhX#XbA2`cEFkh%X2I}Tal5Oli9(K>Ch&mRmSfPG~EpzmoNwurdrc%u)A z%W_GzXj^MC%^LXivtNfryBkT|BDi5}#3=VO>`hmI8$lotm*V=I^2V0{cmqID2ND}y zSxa;+u9NLtp^RNZ8uz`|6$5(AZx}5aJ^H{g+nV^SFVQ)t=sk4Jnn3|SW;e=jKwvV6 zcz@5Ntlh6;G7asE&2LAP+!^vEfl$8q=MI5%bqDsy- z8iDf7k-^>6VT;Xnjx>mus%0`;sWS-Z=bo^2tdI3o;h+aepA%He6cwgbE7zjhPol=m z55iJJ>C@BH-<<+-?rYT_VV&^y2|`X`KW3$p?gYz?9vCMcO0qD~Y~bAl{ACLunqZS_ zit}7aaR~sy?EF{f3EfEd6+P}{aCqmWjnFAC24P<$9ifEyPkJP0b(>2u@vJsS1ekIr z%zN|34R?YT@Q81kdYKU;m&6O!gjL66&yOGk7}zFnD#8tUGf$o2cq|FFYHG^ir(b6p z&aSt}E+@)5n?F0OeiTv*@J=4#t@K=!oDcBw5VEZ%+DBDG(z(>t+TSY2+4^Q6ZSa>M zEdDU4;K{^cDT;>ZbSoGx-Lp__+Z4|h-iHfRU=y=`GuU=&H6LF|LGP!a*ohXV-wE~I zP+!TkJu(pi^7Y>gI>dPZ!63&aGz=QuX~_?^20|FTA2gM>eZ>Oc%?>4*E=YQIt~-`{ zfo|8Hbl(^Dwxcf1rB6UOwfJdNcv8n>0ogv;QvEVawao{+`V1&pSXVY5uGCw3 zC}(X%Nb@mVCaPnw%Vl~U;6wpkSDNXR6g%$z=NiRff#22C)=WRx!AhAEl=Hq|uo2Fe zC^0GfZAAe(0NKf;T?7^!z1gG_y*|Q;>3W3=9bWJ*ylg|V*ZMhu4|{MrAU#fNE@FN~ zB||GTOg{7V6O4)901xRZ?Oqv<&pqaumH3yqj`o24VS71WHm!=#3eyTQ;q`ugz!!S< zi8Jj{+GCta)5Cj4JKIdWMl;g7fw;8bdWznaL0i0&=H?;x+!|>~M`-hFTtpWg`Al)lV29^;YX|8BQbl?{C&)H%UNamvVx{|} zZQ>Rg>m#$@@RX=$IV9qdF?z#!WdkNB+vQE&F+>j?mDZ`wZ8R~+O`+YIl^Rws1ju&w zLTSe{+_hFG%ip;-GS-s{@26)z@s%)bKHc_J5!~C?JiLsR+DpYKSJLHfRq4+%*-E=u zMg4yK1|dGfTf`kS$p-+jdN@{QBpEmPCVJ$N`rN~}6{9XXy`&l!{4A2Bd@uEh+@gOw z&n-a{U&T9W)Dkc{b+_YWt)i)uwV-7+cbVP(;-y@Sk>l>9g*tIJzIko{{}HPqX!1-! zzI6wyL%K~)&u4Gt`QoCF$;W4E*qz7Sa(Y^(gnkKZ%Ky@QxZIQDbqnW%4;Ksf(Fptp mPtDc=@Gr;B|K;Vt#`f-`U&0wbxGf=rMElff^&C~Z;Qs=?Q~QVj literal 0 HcmV?d00001 diff --git a/docs/reference/images/rare_terms/accuracy_01.png b/docs/reference/images/rare_terms/accuracy_01.png new file mode 100644 index 0000000000000000000000000000000000000000..7182b7d3c537e2b376a27d9c68a6117dd2d3ed3d GIT binary patch literal 25075 zcmd432UJtr+AbVGEK!OAq98?4s(^(eNC!bcIs#HHAiYlMeo=fE@oy9i`xBX60gl78!c%0j=i-(8GNqLmil~} zwIEW*YIv~Xb->A&eiz_nN%Qy-00cTcVF!glcY^ei->l5FJ^F zD@?ovt_CKdK0UDu{SPcDn^}Sgyn3q~V~75}90((YUd7HPK`*sz#MIF1mW($s^qP1Q z+!_q_i;4)kU6|*U|DO-OH`iyHSgc!Y@o1pLdJVx2Q+-aE1f%wSu{PB=*PX=@9mS>H zVu8b0H^&N=E_qJhb!opxJ2{~<)e?_~V;Zg-VMR{JMKI6{+7w;9{k%6y+i>>a-bbky zWP3G-gv0qJqk{s`ote}}2u9e+bHvo%Da$c6d*8IOb)J+CyIY#|^%a_0k6;l3j#CeI zS3O2c`#hGar{PuLtihE_t|JIsi};~umVUW|Szkj0 z+t4Z)kIE6k8G3=A`)-=2m%r27_DWNTQoA0mY)($!nP^Lv-dw3?(i^I99&!#FuMZvA zU8-TqzdrbmMQ%PqMJ)d`Z&e$oW+o*;D0P2-$*VLY5!uh^xhz?^P%<$8PL45OK7#ez zpjLgJ*CDcUYb+?|>9;rLw&U8_`0;5*)71-C^>Saz5aze16J4k2lnG*t@oO(hP7Deq zIx07e)yXj-j7?kO?3G|dN$;U&D9S&yJEvVUhq{TD$h?tUn=G|Jr6Mv4$8?}I*l3Sq z%EsV?vPXr_doom*XfH5&?b++)J$^al)T!jMvu<9w)^c}2{9s&s9*pp!{qE+%0&UHf zFXCV=QDwGwZ=P$|zR04x5xZHOTV9uodOIt^Q#pT2uiPQcQfxDoY5zxj2D~=tJa<7> zF1CMRseC%2F#lP7zh!Rq6eE|_pxqZT%*Bt1{cns8R$~@j{mxu`dbnG2XnDAucxdiA ztK4p76v>xaPa1^=UVgG)=|?~**8C#NCt|`HNNt14J{!}ar%x*GZz%Tv|2LviQ@YTGxkR*yzGiG!v4F9m95`H znU-2Ka0=7chb?7^9uybAjdu|IU-k|NHA^M!&Rx|8r8WjLsQn2x@q8YGiZWcgHxY?~ zGZv;g1D5GBI{vO}#q;Nemk&1wY+aQSG-vk4Xlu-DvD@j0wb+p&d~VI~!QQlZrFGE) zir+HZe*8P8cCf%iv2@R0d@q@+>;UJrAA`W_jX3b@bLM3|{-o1MV+^*cCEujiMK8&uOn7Y#1GO$7gM#`|0@CI+nU} zjYiTRdn9t2N<&)=Q#l5hnt*bo<Bt588M+7~=iec!Qx*q)5ZF{pG&ezw(Yi+r{**Ij5> z?Y@o0tz7jW%arBp6V=F5n^EKLzuhV_UJ~fMzm9sb+Q4S;t7Akaiu0k>&-XW7>KR;n zYhRKW6n+`~;$Ml<^|kZO%}8)>G$2`E(^8^=9V1(Eg1u=>5cRmim^B*>-`Wu{7)kNi zAuKIfaBE6LHu2UB`V!Gx?RJtURi!yU;*RIWWyp{!wF_>3;!rg@zZC06dK>X2x1Wh7 z`^)Z=Yh$iP`5|9Yok0Z%J$?35&P;!CRu+w#nJV`ySInm;s=wTJ7fKB@#7VjNhtAQY zKg`ldl@0z_pPlZpRJ9huWgLV_v=GwN8zW_p$9vio+JG)0$Vjq*{l>2r!>G&IcOS$z*%2-3_-rg&YaVJ;P(NY>Gp%7915h-AoeR*Z58z27AOxAv4+z@{4 zY^z4$b0v@2v=~n&Th84#V&tJHV_({_htJZGyoPS8(P&nw<}aG;+LWq}H@a_SUPM_K zq6Dnx@b9~qyvZGi>&fEi&rQ< zbM?9L%{4mqX3?N)!xD7vh|H1qGE28b!bl|AK-;E{TDG0s@Y>i{yR6Y%UezGeovlJ@ z*&Ub35kY&SYx*AILEQ_y$S0aG^Um~pZwsPBG&gAwm(r5!R@P@bheA=GI4B<^i1hI< zba+#)WPV7n;Hmn|F2bpwyn48>c!h z-4hOR>)KP=_vU*qnVmdF!|5dIOxEox%Tj4xFtog;9vpUDylua&oNYk1~DJTbSx~P(|OM z;2GkF+pRgD+AWDLJ4co6^|;npnLhA+}A1u|jDpE@_zDWBsksx@dK$EmMFBh`dq zyl{V(mLkcS%yj8jw==(A>8_1vKx`+!q`e=AGX~d-l~v4PUX#sL;_X zv-EA3*CRtDshz4`#5m>NYG}2rZ58cQ9Y0(XKO8^4+E)Jk6UVyrTAI00h3u#N?lm)` zim4Bnv!vcq0HUf=cTrqbGoswMcuMd|-7)PSU15W$R=JVhpebc_CVEk=C;aDWI3F%+ z5gD+wn(HmF7>ZxphzZ1*es9I{sq0eS;@2wlXZe-`w+VmoW7kzMMfi{ z#JDR|PJFaJ#iw*fZdNGe1g|Q)e2Ef35raSM=#*4mU2>Cl!XGBM;p$sOHyz^``LAhi zbzCh)b1pzPp+Sb)aWpTwM`pPgZ&{{;Q{eX)#cA+6ezI53(_CIZo=+*t3p>UDkoA}< zk(fKX8I5kWkMk|gT6cu30$0TDsY-@og{`unn0AGkX0HCcEDSu}`KGJZ!#J036ca7{#>!0b^d|N>C5zH)Xpo@HwUPJ$kmqr6b6JgK4jW zdY+oSRVyZUbTI&%$h~nUy&B}c!>VU69e=?ICeJnlyw&|l-JZfO)*K=S${jNXrmw~5hi7T9M?nn5Wu`4TPISb;b;yKU zw`jR|q0aYoj(VDcz~fq&(LjcK&7M`_%!b-Ya6xD*J*^qVpfCOurY0kOSi?S%O{+zE ziM+E(-dF6`XVkQBt71L3BautP9!t)?PW!vIhnu$7dMx|$o_v{y@b}<2s7~MdOKokX z{m$6?E}AW2cQIf_wsaf6h9GjhkmGkwZOo;$J4MO^U-gz2q{X|lv_s1ey3iVnY{Oz9 z40oq6p~Uh<4wH@LUK-UQ^`T6mdMs2+asCucjE=iHt@~f8RIcVo!ei$46-p2VIJql< z7^Upp`1ynHm0tOn48s%CaT=>EH+o7M3}eF=M`FGT+VcWi?a6M;Szbv+T_RHTeva0{UX}!Yehb96+f)v z3VX{a+TTsg@k3;HzR=#w6^yIRZm#Dfmw?US9SFK8Sgirh&$K{5Wg|ut_=(>3b z5e9!o_aCGo0QdL5K1F3U_BGHZR&d1g_Nb7N+o*5J1Q_JWni;uDbQ*nxfCj50X{+<1C3->m%# zLHE7Livq;nO8sVcF0vcBJ1|m_G-r_8lcQJpm|-D3*2ZBs+?a5)FW-N+ABTMoFsf6{ zQu{$}#T+Lg%?G1hU^%N6hRdG`g2l(ii6Q!jm~0;vJX6`MAf~*2jGeiboKa*3rH}Sb zrAw@5^!N$2&;c^R!N;bp3B`b_@+~eIdhKsu+S<_?8D#^R-s<^J@9hpwi_zxqz2U;x z;Zx9U{BFyS2}*hJGEQ&w4J6PV`ml)I`Kq-R$ENN}kN{vudW)u4Q&sKdT`jV}Oh1|W z9k7VXiOVe%izpag!Igw*H(cAWPF0+FMu1XclG5wKkTrllmH>iX%G8aS5$2eGQKVwP z+r9bPNRFnr4!!&_kKekVPp`rW&81`EA?n}q(D&v>nu1fyQq3WjtLs{&Km4YJSlY53 zpbqubDzn49;sb2%2pL5eYfbhL=bDJ25o1DYWr9UP+H#UaCX$y4xs}Y7mvs>ZuN)ni zTVm-$C^*@s7}4hcUPLmjXsr9ruDW-Zdv_#|S4QkVKio@*AjU{_n#34P`9xuIOWHRM z0fm@rSwLqb6adEWhtdAxM`qm=&7JJn8FPE58Za^6=xY_BF^2BubvmObVwd0Z4ob&J}2p zw5R$+QRz`iEG7DJx{nqgDXVoBlP`O>gw=3WA0)3F2_vP!sAIEw+ar5Zf*6SmpAh(y zk)58>r;M*G4Ayg&p(WZTwjNxCO~Z*TR;?-PVLi*UW>-ncE5aI-m_G)?7 z9?QG!mJgpWw>vTlIehK*9<;A#;JGm)=j%oqq}d|>xt4w~&2?$GL&0UHO87&#GDP-FFbpq4V*NV{0QB8RknjV%d@i8pBpRpZD&01|Y0V>|R z`QD9+ja{%1&VYsRtIm>Q{UwGn6?^4FlFu7dg-quy@qJjA$OFyPv&!{VmN-n?xJ-Ok zHDVA!7&_aEwvDf5wPsUEC;|(k!uH3*04+%kCk9QfQBl*aH91_x*NTw2SLZ_Es8LOg zly`G`362HR%F~SL?llS?UgLqT)*_>XyM6l`KKcc_Y!=8&6mhqM(@fiEnkt&N-PG4n zBfFaY9#s4N%ImsHn{jmWGJL)MY@U&dHPataY$|}O>zk>Xkpnh;X`^2?QzJsg7c(;* z;>xr^ci(BgT{9@XU!U~)kX$u6AKch2ry|jER<7XLVS6ZYV%M3J`NBz6iP~@Yu7e$X zF57(8%JLTqwgt-E1FE^@9AbI56W09i+AEB!MgoxuGqyHV&Ig-t>AAu*pLjV0Z)ul!eYtGwk-B~A3}>EMrTG4=O0rKN>ML6O-d1Vrira`Vp5CyV+^tLG2R<(T zEsk1l#~e2zz5if6r#N#qDFcW@i$*lRs#qWF2pB54bn$Txp`vH{ zuQq$6VK08TGwt;E+kGnU;#G|lL@p)n6lI=5p}c10noo4U=B`MJ4c#pQCwdK<^BgXB z>7YA_&M<7*C(w8|j_-Y7-J2cadn#&RAly5TeZ9M>)3vU~A}$8x5e%!slg5a~VogU$ z25C1Y_FRG1*8HH6zuo4jJmO4v`;3MKd>ql4TZh5EdzgXPm{#*T6?`S#*06iih{YLw zu{BBZRrq41Qmg=`Ez8!jn*0~th61TDVgT>IgEA8rj)4d|^5oofGM1Lk_%psR$>#%H zozu>P#T8vxD*H~OkeKWCI8m9sNbwKJE6k%$U(B+t+# zb+#hx0cq30SDcKGvbl9%{RDEz(Lm*>6n9|^?xtB%DHiNz=^n2ks5DpdEqf;vO1?Ud zfn{6LS`L4DFqk1jxDpLPm@Zxgr3KBkWV*97bIyB)N~-Bb9;01e^82CGjC{IjJ|*nl z`0a?HdgFS=Aj4gE#2g(5^(Ui9wkS*Vv})l!L|=;J?e#&pkKvZG%lw?z6yf(0fSsG` zFSeBNx2Ms%$6{LwC&g@HGcnTtD&KpjPtGpw^WNOE}03 zsDL1*G6$rN%3y}OJ;7~JXvfXW+Khf?8sUcl-8b+%>7s_oqJ8@v%RQV{?xVten+=9c zXv0{Yb3;Ze9*0H2m{1FN5pTD9=C4b(e2pE$8ODSw{XnIk{FvJ;d|qzbMgZHD2r=+3NhI3hc9(}qhy*7 zqpv?Yx1!@~;C#?S*+obGsQBC{f@kkiuJGrqhL9c6nG1Y`H-v@!kb5HqmNG&MBbF_x z&&}pE7lY%{#OQZ=bZTe~9K&0jHE&-fv(NTU>VLniyw%VmRCRc-U|Op=h&W&~_wzBn z=@59t%GFAvc3-UVK*nu4qZCUjyeHL=QM<&Mz(u324bO>pa{7rEX*M?o+a<+42b>IJ zN1MWjf@VetYf$NgrwAJw=#j#^JIuUaIkn2~d#+O{*DRf?F>GM%R{A|CaI^fKn(gvv z!#n4Ktuh~{p){?tYSNZ<*Fg`qj;O}Kj-4igOUPy>?xBV5gWy+|0iwO)Qdt&;FS z*?pEQnPG_bsy=JWqpq%&RNH1MntoCAaBj!&h#eB6bUkn}cdOcqbNLfG%4l|&t%h&~ zm58&rW$2aZc59;(U+%GAy?rP6>INALxlC$fV0VyTw2+AfIe2hoacLQ(u=wmffCf5 zGm!V>6&WyV$eF~fpneMZQ{d#!7RB33R(cpj-MfxcFj@i2B%kx)r(VF|7UCSmDkmyk zmr5>6`}IPKJ)lgFu)@$6-?~gmWU;rkQeuhY$jf$7cRb zz+C$tY8#k{y07N}%2MIHHq~4R5L-Wh)j7anTFh|9_nh6;UVc+*-I=a5jv1(WQ3wd< z*QuW$o&kzj3fwkbV22Hf(d1E2oKyi{EYWv10M?yrB#!S$3j(q`*VA%T2f9aYRGyCC z!Z^kVS$Q2fG&@GUY zUXj>NWee3x`B3P)|)?9VaW;z~f@v4EM>wdEtqTAK1z7olOE{apt&V86d`kwPfmmhu#Iu z;j4?gezp6?xtc_?0U5lo)6B2qMXEaLVd{yQVB~bb6|I{V38AM*fzKf4=9 z@V}o2oLKhjoQY>s-bnu<=~`s zk;i#IoKGqd)oZ?91xR}(Qog`xR-HD4k6k9k8hEO4cH)d=iDD*=)Hq&6KKH~OV}7%? zTIG1*$>T@QO|`et(_0v^Q)DU2hI%E>FM^^_P@3i|dp)JM>PJq;&M+kv8w$&yd> zi4rX50S9~6X{}##CCzi+n2T~B(A*qv7g5`R+;CL^YQ%5DyXOC};r$;8R^ST%AM@$| zD2_lS1jX?oml4)e0bfwe1TGY2qVkxyE9G90Ro&}T0xwpL>2E%jG28YdqqpkKd*ZUS z{N(P7)sM~ZZ+?*tx%9!wpvY;{5$I0zB5rw*bo5CR{q!^~PmyEG_Ix;46a$bI3kuT8N z1N?ch0u(&CdGVorh%y30<*DP;&wSwe4S=)e0Ikg?5v~IYAd3hz8BFdo6KjHMk*yfR z)03s08!!np{6eUpJw7UEL{LHNIVxl+B$^ewY8sH3Q9ilj3S8;M!?o$wQgFArtqG!q zu4<|llUQMv^~1uN-6xEp$6%__%iBc9^q7ew>ol%|jKgWt3!#j*eFfh@sDZ)NAbqnF z)HK!L_ZIncUFc^(=W*s&2hwc;#J-+;jeid=1b#(^=kN~(#D&$%Ct>$#5}72Hk6n@Y zmTH%Rl^f;z@#?t8{URQX-C0mBmE!KS^RGZ$ms=;V7Dx>yw@q7$fUo~3R=_$NEO-I4 zHfA7UA_Awom%c`rF%-PAB!}HU0nV9sOjrW4YJqTaA^$-r7>5Kq=<<@KPY5RY)`j~- z?#Og!s(o=MELuPN{x%uB^xeiUj64 zx|Yg*HZn*qi9{1LBkRrizOzkTm!Q&h3TNUVPG_znzDrmtWfXPq1toT9l|?S(_3^N_ z8oGR5;iK#c`}LRPs!*vygaTkThH5u|eN?Nka=@GqsZ@^UF_^1+cifk~87Yz~7h1xh zk*))A`e(tzrH<3emTgHdL5FB0lYt&}QeMiZbDB&TU3W%z918lk(u#d_0G)bD$k&8&1#)b zk#gBO7(lb!H-}we5&j6YSj86xCI3*W?j4mX?+5>5l85qI&n3oD{QD*nSQw|6xc9H4 zBa9_dcz?l;o%r9xQ4;TGghZ<3DZa$qzIX-3Ob6!5%KvyJ&>j7YOwpFVo1>fCp(zgq zOyUzXSpHBNNfjQ5^(LVdf$WI?FO~0xmvq~$m=iE&22UjHv()c`{yV{B{`8yyb2(@> zeUSROI?*8T^lN}8lQQ~u?8*4s*2=gw4r8DN5*Bxndu~_?HkRL=pdiJl2HBIQNQ(y@ z5leCH+0ok)-ZFyR%~3i^Iy4DBe*Zc`^>5Lm>K*lqKG6-7M6lUkYb51gEx9U*nUbKc zew1_ps$wBw#B0Q+VwN=)r~!JVAUi}aVD%nYtG&tzBIdWB|I7nHp6`PyDc0jTciq=B z$rwd?TjGSCJ^ZAZ?|YiIUufFXr2*(i)z(b)0PDhH z@sN`x*Pybe`9xZI(o}P7%yg=vQ4&cPa5EkY4%)q{-dk;QnT*ozhXOqizC%oM8RfUc z-yHzecCeZLj$M*9(5Ho+=O&Q4r8W%tiJ%gH8!7>2vpvrGe=fJoVIYI_tFc{@Y6e#9 zx@1VO1?6tAW;)Re%Wu*3V;H0=Vgf(|n3W1Gvg+&bjRmYd=ALF*GfeT1(4FHg!pE=i= z5h{AHHU8h@NtFGg*-8L!7PlX=01m(zEA0Hqa#Ij^0a;*D__abwKZ$fwVF@UeW>dK! zK!4XI+j_99y;}qUwA1_`o{Ar&v>S@>3?gPeN|_=E-r%Y|Tx0Lpr&x}adTu*F5hvgs zH9l{NGml_ZDiE;l*8yutcc{1JD&FA`U9hAsL|J_f)#t(%*jmwmJSTPE^Y)lZs7Q z2ZU>`Y^Ju+sH8Zv$7|+t6Nq8VV99*D+(tA9>DbM))CnI!+G{h77tvtoxpEKp(H5%I z16k+wfgM3?0jz z+ANH$)g@>f3fM_jF>y*i34IY_h9th#1leyMK(}7#P~>&ySGC~GNd5-yeQxI(_oITZ z{P~?O;I3Tu!;OLc&T|RP^zcs%n$VsqS2@~g3D^OnCX>=%Oo)H{W_0aim4S5ec}i zSMsx&w7bUZu9j(_&iFU zdi{)3Sdi+QwuGLfi8y03RT>1Tl61`1ErOpKXf=KV>;k+2oA9rw%*<*6jC1H@cLE?! zWjx%d^Y@?801ELXSlIx|gMmKc7e}hm}BSO28F|B&2dW8T**!6Zp9bh;142SMK&WbcBjq`Vvekt53>T3(Qt0q18|hO6Y84Qn>z z=#)cCt@^@)zHrhIQ@_zNmsNcV)pTJ{)5TJeDxf9>>ZHHC%9kXQ40+N3#GwH_E8o`m zGHVZ7+?tcj!O@+m*IrU(ce}ZBg$4y`(Ewx@MWY`Xfs(NL=SN@@nxIe*-SkCiRMTEc(M-|YfuPOtJ1|9f->ZxJc!?|=tXka!(78^PbOJ~--rDr7n{s( z^7N!GtMoD0$5*WsNmjpa^N*Nq;(Hld*gy4Kp;M}%;s<3QYT+T^GT*B&0&!FCTVwx# zsaLPD5q$#jEnx0>LXkTN@0PnMfKS3mAGfF669B1)Sm2xmU$}Xt$P?Hw0#>~@?s{(J z#f!Y?c>eSK6M&x{RPc;AoJ-nV9J0LYx>%&hQ~s5wYFR2vC;zG05Fp(S_9vn9IavKM zS+oNk{xd{9y<#O65drJQB0M0;A>NVP02Ai*;gV!vxd`m2uRSVGvqvVBZp2&)`x|%hHuxwKu#9;T-A-S1rrk3 zEd>BZL#fGU!2b2a9}@ua14}@ScoL49T?jfwhK6m$3E$J~0-yyTJdOtLkr(%}n@YiG z)-!nx0?Shrv_UJL1o8kbdIs+g5cnhq;ImaKx?=ZZ1y3TFi+tPNd)sRZ6?lweAted8 zOo>O45l8JG3O5)yG+c8T^`+JQ_J&3p4;oKIkTi5J-s(;A-WH&b)88A9b94}}15V1c(>xd-iuv)gAqmtvTA|%g8t8e#mSIA>CJN67dC5Wd0 zNNSV6rxtY{=mw9#bIt*w?2CFaN?<~{|9W}C3J|f)b~&a#aCcU~T~C5w$#L>a@3W*O z-Zf2~RCwSAXc(YO?8pC$5(p3*$W8e50jjwGinYMA?`PKw%b z`mM61K@woF9dBs3^dH8`DQ$xS_5`BabBxNOq!?z%Z15yj7EtlxUJj?DLzh=Bt2C$&48??~d)xN#ryikX} z7NhHwR>`9c=BXxrb zm=uwO%8_l%fnw@0UE9$ZNb8Z`&?7o=*ce^bA`W#^5o*fsq zSfSt0$L?uyi?EUFHMBqR#=;x8Wwu@*U9e_iJ>$RHPhve^Go zJ|Y`T)E@8zTpxgN3GYhPPQ2T>oLAy~SieD*BWUN{frm29k99yd;Nr@nf7>|o>9H9Q zUv=>gpphv&83Tz^3|g-Y3bA0KS@l{(3HpLBmQ;|dRJb_h{U|=WFVLQV9oFp}Fotyb z5SIO}_(Wb(f&VF3c6r0q9s}=hKC@aJEMLsuk&eG(Ul)At0!AfK>}IQ?ZT3pfx(ta9gP7Bx7`jWE^Hl+;=^t%wTtRPB_L+?JU5-GIg+fMbTI@8LR&uQa zi1tK&I#d8NyEdJv`?dKnSGAo?{VxawS3l_iEx?<(ZC&yc_nRYB=v@vLP`S;`QIk<8 z+@w?wNu-4~qsRPB?Jp^jGisS?TLCr)(enK@K5vOzD3Hq}0CfYEVRM}6@XnLsEOmkL1wE23eiOe2XtGi-YE5bbNV0(T~cEhh~_M^5ra$!nkM?wFw1Y`V&D}@awtu>7Y|z zZ4^4k#rGRj%7r)oL5Yoy2(R>8J@7hQI_&!%Om8rfXopJWpyk1CjlFNuKp!+Y@uygf z7KpyP0q)29XM?=AI|%SEER>Fv93Ctkx=4FMH+s}JHX7Ixp(OuM-Y|Fzrm8i0RpMyJ z!`;QQFJ!}ZM=h1(WthGTtOxrgQROh>%S{;bMbMK9SlEc>K#g!-V}k5`o{E{2fOiCk z?h}y3Brz)FG!urOqO!E|w#w$p0SylL6$79*Mr>Seg8Do4d8O^tStP4Uf)z+^o#Y+? z7Ef%?^z!Hzm(>Y*C^Qf&Eue?L1 zAmlRtJ~qo4=$AIxum4)tZ(}jQrnz`GBTh+Ty_STctifD-XNF~WYdmz}^9iORXah54(NN-1I^Pf8fx7b)B<=I_H5%Q)+Jww8$UB(1y6_LdX7$h zo%76UCdUXcrUrj_f{()9$N`c)4+8sVo7T$!ReJO&Mjougk*)87RsaRq@L`84T?YU@ zKH6;sz~OB!>C+I!XV(YY!CbeQ<4_wAP3OYj;sDPz&aovQggaZbat*C>YYr+DUjnF| zYheG4rrgb)I^WX?_WYL25l4|o|2h|>X;uNqr|cL6W$A@fSf)6*%s~1$nBLSAT(cE5 z7Q%|8Xy9z`F0O#p2jxPYrGcgigPn}-%uuO(?wlg@0n|tQ)T@r(lXMpPYtm>FR5Tj> zH=|?MD!VBXOrx5Eja;bbd91g~NjF83@2<~2LfPhF}B>DtI(NOG-d&K7w{=4dzWIfVvDJ+=YF@h%4I(aA;B)yDipPVy*tpMzbMh}CdmMr4e$qJ$lF7rA#-Xn9h(I@UB zZLz3+4YHb^bR0|&J1Tu8utxw>wc4mJA_~3#(1dsA_|ZGF7GIHbmb6=>jskz z+S>!tsLnu68`2lK7KYoz|C}WHEi8`$5Z7Pnn!7Mq-pNxwajtT??liQuEZG_OSPkDU zLnoI&gGo|sJ2KpS@~8J%=$*!bxT`2=3*LU5s7H^IMcR|*rYNqMpbTi!rD`oOXN`e| zT)aBjG>wR7`k$d2IO_fzp(O90xRcL}igO*H* z+sae0qbp*_Z9Lef51;~D$Xz{JH|HT)fLRA@C6JOq|<8D7TDm1~J zm~XBAb=3jFJhWI%FsvMV!{xBD4n_zDzyJ4jt|N3Acl5Qz_T#bxHC{EA9$NqgiPEZ^ z0-sEh;I|zfELJ_8@f-nAOyhQirirM8xmhi=PYY}&`-NJ9+PHH+R0YCbE@wyz`N{pMmJg<&AhvL0K!(x zp{ounk-G5u&4lEk;BalUdRkSE;yo>r$nfV|FJ}*Rq1g*NQu(<8 zoRKEg6kX6_1R~YqA(X2I0O!iOPM6>xi!%E}W}0G5MWP*)teiOnkEwa{_j-{v=9e&c zB16qz#=+71!V#B=VI7~RPpFoG>jWOxG;2FljJ=tehoCR~%jVw=^j}y{TE9TlKKSQ4 z_4^1jB5F*^DOC{4E`g#04W7`Xh_>#{`wR;GRF8RH(4PT@1LS`rLHq5zHb0m9m+(S) z`4JUYNHZhu0*%~;$!jkk>=ohTTK!kfT@ero#$2ha!D{O)!wc9BTPdOV^1%j`UeI8z z0lx1kwKXbLA~6UJoAv8#HZh89f3ktULQrr!S*R5>rnb7`T zh^K{$tlkfF21M7fSY!04(Q<&mgMmLegT9-_f15~9j=S>&$OCVB0oX=7O^6v58PDNz z7f0}qJ^8GkAtlNKs}-ji^Kib*NcHr};CT=&$OMi-*Rb0-8E)jh*!Bd~T__k3P5_bj z8+>$$a!n64XmS9HwJ%jFQs(POKyEIL9Mw{QEGj1He%8@TNF7>y#m>N>shb4(k)qoZ z5obfe=3T~tLeo&u^)_jYN;vY!2K;+uA!P~)m!qRj0m`3!DG=lB>-ODbLm90#(jVjl z-c?>tS8WTVrk*58lDp1xk@Nvj?T~nK+R0l%rdSN3@X6BtP)J_CuoBFTN06@K{yU)% zWd))eG9)@?|JC>_Wut(>u27FL%fR3oYNVM!tm7K92kme0bqS=Bq|W*ZCPRH=2|UvX zK(Wgc?SKO0ptM7sUDe}Ox4eTW@^t_YxIT6t;x(nZC#>Ib)5+QS?xzN z?~#`W$tiWx-q3~rM?3W2=w1#TgC{M6&3NF?U-tM*F-Q7N(f%o5NIU@nZ191kG(q9*^_<{_yMTIXv- zjsMh;;Uxbgp8lJ2vkRNpQ)|O~U;ft%%q|$*^ZCJq2zA=V3(doA|4-HFWLhXr68ZNC~(BKWcg#mmI2D?XqoG{h%N11#MRnG`EmHxLE zZ$sNaZW1EQ2Hfx=jG`7mODq9iybf4WR-<*kE-gZd2CERf4ZAFD1#q@jxx?h3J^&P| z>2mg@VP3%WK0$sdkBH=WZ}>IEd6Xq4AN*oL6WGnliEmH!2T8N`90*~MUAog&Vl!m% zr?4N=8|tH7Wet30I@ci}spkOmU*He@BS*iy1KL8oiS|xomV-19SYrqj2};w*39XCZ z13DMj?&LuG5nB5|1aeB2(=D-@t!p0p7NmGA6Oa<%^MO6@_PE*YSboc?X#5E1bCrj! ziQ*%40#;^_;?dxIGKJaGa4!%641&9WkP0btn7)xR`|)hBpe_nqLQkdd z+zgDy1~>%J?+8>vKh03k#N+t+-hOzUIqW3av9sLmC4oI0ng3W*|4lSvN(Hc8n1n3N zJ@5W9`&9a$quAX=M-sumftZ0K3qx@av_Vj=+mc@aF}}|C8Y#y{;3u_Izm3JAf$wiG zJ_7^g9NNvKpakav@>@QWF9xV5T(XkojFLhWFm^?s>c<10(1xH%D9p38mo#jq)}|-!?-D9Mb3vRV+EP=(7Itbs>+Eg{&=r0@ZI3ms za=teX1uw1_;#kd&!=K6V1=#8c z5K@@0q2iX*d+hT4dj!H2 znagrV=8}mLyU5$?mQ@Xbg_zV>J@B1DqV?Mekyk?!aKm<3DWft2T zd(6WQlaR8Py4Rn`gnXsBCD3<<)4WH3EWr!i7Zl@np1Wdl;S3`+B|QiRe@x(KVagSH zpE%BbUVmxqEFw4P;mxY=j_$eQ2n?KRT->VwAHT#L_0*1TZw&1vc9KFl6k+wsKOqL4 zN_-BC;8unX(~hpz8PWvLU+%ONb)Dj*`}~Gxg#M+C7LzbeC9^k@qa5o?+_Mbl!r)+s zWb4(PHVXHbY)oat-%{J)UB1kjxhqoDIJExK^)oLq&s~CqKN?OvI3M@*w2B0Pb+E17 zHVTVBKCgNjeBKOvz6O`bXjGvz819$obquZZQm!Ue!EKcaCAAv@(i@uiJ9t5K5**FDDHMI`QcRfXGqJFVVw zPE8xqvc!9k8;;v(3IsiT&-CHlzJIJfMt;y|Z(`aa7EvKaqhTaU?_qR!QYE=|LOO+V zGDjt9ggI*NfI1wdH=jh!ZDUu^irMaZd+A~rc$Vz4$kT3taFK8Nln)-7=a!aIU|#Hu zZA|`H*B5-9@wLM<*dv)eVVM|_U56-Nc{e&cKkqL-G~LP){Vi)YK#p`9E&4Rqag-wp z@$HoInACR`j)ld<8hSPvd2`tuE~PMduKXH3N~EEVDvawGd*ms}1d_&x!tx|_?q8EL zdgHZ?T^pO5-!?XRyI+_n(5J(%i(EU+fL7pt{8jrdQnFW0JpMs9^7WG3OJ>6b;H>$ZcIG4pZhNLJ`t{ib!S?O#Qz|*Rd($8DLwVoqrR0wV9iF*D@Z534eeKOv^Ud;jE{m02 zvPfJPP9q`^&tNOAztnr{Y+U$)pw|1wS{U*6 z8QJY~aQT&;SrOGN&E3$LTtYCbO|f1)DZyU0bph9yM?0zD8Im zzdL6iNE!J{^qX(igQ|>)0+q{4FMojT)09ph_NGE8qx92jIMPm;JVpDZ>yTDyZk`oe zQi2mIE|92SIXjyCB|~!7gDG_V4 z(UDOEYKwT)S}-5OOvXt9Q17-1HeRaOadSkqMa=81XtI)TTrI<4Bn)6L8}OCm8D0v* zqLgehfzkf{nHLDRAFbt?tL>Xf?=HWn+4sx%xGUdvb*1x}ZkKS{P+D%=h{j13b~~`K zEVE#r=<7+uyH)4v>S>;6%LFOAis>0~cmr?R>aoM?aU&0}wK#s`n$dHYmd?1+E&_()JM)PM>_iMBuKeQT?FOKCzFp!0_|DIlv_J4>cz2pVkg%$51(v_Fk<=x%MCrqB>NrQoZ2Uq#Ljbt*$7 z^f7oJl}PB3s$$!=idF)HCG9QsnafY!Nz_~=9beifUE=p5THilY$6-UsmX744v58o$ z%;iSCM3c1m`tPksh03Ll`+Qc;6<&IePAmE9u=D2VhhCVTt}ZHCN_|wJa`D64YRX(e zRc|n79>vU)un$q}I*Co-_Qpqa>LZoemLD9)-q}?s4@*?23DCcPvyAob$b;jeM$-Hl z8Uv9m*e?+uZ-*NR`&%EKUf(Hx5@B@9Zu1c?iihvoyb9vxe%HEa!eZ9EsgadcUJ-|ZmZhzpWJ zo_cXo1%1q4=CVRl(**xvz2$9F$8on?kIEFaYIziW=RBKalf|?N71xKSt$f+*sEEB` zq|{-pel^wn-wW=iEwnKTU3q_)oUs2^>pYKP@~~d?tdQe5(hzl;^>N<}Y99^Z)UW)d zUgXxIpk7VX*fohC1glBd<3w=25l$9Hs|Ia9pPO!aChAcdJA5s(IzREsA@{bZ6w|AF z%*N#mET=C11cxA%M z)zfMbiCGICMyD*F+62~cV%$gx1qUyU%&3h0np}0>X!R^P=PMAP6H(ykPhOTl%>5;d zf`%eVKeA;mvkU)<)C%;DT-Y}Ce7kp^d*n06^9)bl*gH%gH9V?}beKL$ZrAcoP(Eb$ zaR2}z-Ee%+(o4O1cn@(o>5Z_&f5VM9Z0T)>fZJvbd{LLFpBJ;#)826Fm$kJBO&2O+ z!LJ+9-!cx!c~l7fxIkQ?V@piKp|K7=SJ*H8$OQ-mbdMed&^cH6no?B>Zx6v(_o>IE~c&7D0iEmx7 z+)>!8qXZ~5A(a^L9*E%Emp&-g0TFoJ1yV z#dcN8nyE8(GRzI^oihlCU>_%=>jk-?|5eJl$3vO#aeS;$ zO3GzxNo;M1q-_bgmr7eB*FlK2T|{G)dkoSR66NT!$RN3kT*`G!TS+Im=2B)dg&Map zuJbU{aK1l#wr$VudF|`G&Ogt8^SqvUe$Vgn{eIq`55ok}F*tFjHc{11-j>7V&wz&W zlvY0I(=-Mu;UeTd@+-w-(E`U5g92<9z>hM-h<_j$ zZ5a{!Q%Uw0up`tgr}ns>DYndrgzi>x2R*w~@ydso_CxuXN?cBVODU7-4_?!+x18l- z0Pi7dZA;DC7Drvu5|=iyMYVK1_D}5`^lv_6%8$K(zFi?ug(u^B?QbZ6r9M?Bk^^ah z-fFnGQ;r9N;?}VE@)$K}y!T6knC81{nJt%=3FTh~SIYU6AnY^IHg28Xc4pLd|KEIk zI(f)k0(T(YKB_Y9tKNE6L^gB;>SGhO-#RuRNn zH;p|AVkrz@56r@8reVg9jaR2X#|ciaT)XWd^l?rIi>)mGq!I%SJw*<3%#>6}#s0HB zq$5r+#}Yb~ zk)4o8byAT$o!}BS_nv#bIJ6!@MUps=8~^OQQKHf6^;zk(ih6yHk7-Ew0l#L`Fj8z_ zFI*2yV+EQbk-V^r=1X$7yFnHu0Yc!g22}v}${{|Z2U zPQ_D7bx-*-BPU#{{Yn6V*t11)lJ8XOyOI~;R6E68G9l3D)e{p?HF|@^=BOHNOiqln z$`|~9KOdkq02`G7ms1O0zk!04H}shrsd^y@vCt0~y$5J0i<5rlEr`@RJUdiD0!Y$u zsn%jfB}zm0?E!dm=!y%o`$E90TL{Wn(dKAONY<9;13c*``n-meWjDUG?-W<sQd)>Q0bi3K0IgKQl7h?8LMLT0EW+2F`kR@THgjSNP)$j z%k-XISV7h&H)>Q=l2kN70PcS0z#PoKIspNhI?Sk>()Azb2?seu>&qNcZG{L%)#zf9 zzNMwf%70<|RcmeVlvJh+v?jVA=su%_&j({fmsaCb23L%DxzvfL@`(M%W>Jb0lH~b! zm6c4hlvna1I;+m^uItLR+PF>OML(j&7*z!kUqkn{T$4O>fD3G#X!g~FgC%t|S$ar{ zOg~?qL;NL_6pLX_g`5-h(KtI%KXy-NoNd->SFP&Uz5Z4+3%@eQj#>FWaIB1rw79dS0Ho z2WU9GqdZqu0Vp~mV2m_q3QEP7K!nl@%seQB#usP_%h zuQ|IL)tbug!68W}60hrTLiz551tnhm^=+%T>2p(%c~R3{d4$8esQFB5fc++r$)4{t zw@^DcEF1*9I1Kvb_6m+?&jvMfB&Jet=4oEO$d~BjJKgpr*kq^IRcxy1HoZ1_>pZXb z#%L|N7M_xNf_-<*EJAIgmQM7%8*Bk5;4lzSmL!&XQOJ%y;hw$v@}&z#Zbw4c+7M%l zF;w??PFk%PtL$H?qY^*=;QZNMSv6Z9=JTg$wL8jM*MU7a%2L>>${~R@K9@uMXR|)K zK36skHQv1+{s-+!!h4tkY6(6+v7mAQ9Jg@I6V)Qi6FWsk^Kc=u#3j0Zi1 zREI1)suK%{1<4>zo&-IU>$))Gy5snZwfy#*buzvw?dt+FZsq)x1`<#b)WZ==0Och@ zQyqlS>eqncZ0LG(cI58X%j=B87>MXbfH-3kl!YGO7DM~Kg?IFU#P|>}J9U*FYpkT- z@nrxF?q|A5nl;2OD1X!w-^#NWoaV)CJ`~qI z+x)f;(endjMjS^q&HbY~b#?b4?3)Z5eec2Y-IM z9kcOP3fBc*chQF@%PsS)i7~q#?Upng zSg&rLJ08wT4Ll5Lm2*=YEY72m%eVSyt0CtY-5Z?TXn(3Y9t4wpk&%IekO7mw9!fbR zs3Uh97&1)<15x}891AIkW;Y~gbQod`o4(3^LmGDmwuNq_-)e#s<-s>&m7bmPv=^S4 zN?TwU67Wf$PXTBoUEs=E&~irDBDp94YAndV2pj@u?%SP#@(~GU-!I8E^V^c2z^~T~ zTU~uz+#F7HGyY9Nxcc*Ot~63rQvK;8q+t4*ro-IYBzoPvUY=CDx=n1qh@TZZ)*1H3 zvEVyUSZsBO_B_y0i*N8nnP(L=0dXI!M}54WCxgrXYdq~VbdR_X0^pN1mh zP=4DiM4}yot@n(`Tsxq{`RJpbcECTjmCu|v0*Ns|Tc^Z>%9F}-E6(=G$5#X9=HmOE zF+YAExym-Ne_~fWlO-v+zO64cmUkm>DBU$*@DUmX}u5zn(;U~4MDWYYTGiIZs$ZEXvzMY3n&`USjcip37HJku#2V@;Yf#!if3_}n$_WLO3t-AZcS?x{% z?ueY+b#FK@dA}9aF0fe|ZRaJ0$B&@89#%nGNMVnfgokAhSsi!sVGK&3`#_RAXYs=$ z&Qw9n5fI~Cu3F(HAJYlsxanK@n{8YR$H|*lF#XM%V}Sg{^1JqRRvC85q8 z7sQrzw*NHc=`5pbJNTtYq{D(FbFbD;il?sDc8ZQxh3XTJ>1u~7ostK%mE_Ji<9loi zqz=@QV|iRN8~l~MRW`({&5rnP$nk(_ByG3=vF2;+YcdaOHuYZ{bspGf%uCR>FeJ^M zzQ|Ee7WH46V{|0B)uG|eJi)UoCs|n`ZC){~Nqao%qNY8LV6#v2n;6drT0s|*hld>e z6s)U0;3E3aj$~kc&S2-;*@RrZBPY2EMw+(yEyb(@9W81UK!9)PHrrxZ?8jp9tw#M; zHnbp_%03B`cP#jjBCq^iHHY@u&)ijiizWP_6#PGaJa3$y?7Xo&+Di%DWd9xuv!Y#Y Gk^cfCuV!ul literal 0 HcmV?d00001 diff --git a/docs/reference/images/rare_terms/memory.png b/docs/reference/images/rare_terms/memory.png new file mode 100644 index 0000000000000000000000000000000000000000..e0de5c2163913181dba940847b04564fea2f9b8e GIT binary patch literal 20465 zcmc({XINBOw>7!}5d;ZJumK51l3*ezu~f2>K@&TjJ5r>G?W?WIOz}s zVNg|3&_NI?Ed-%xr`-*I5li{n2|xDSRk{8EK^Q;7zgh}f5%+TlqM4+sa6!-Y4Uw?t zHETx|>9>5~%v1AF^*zIs^@)jJR8Hm7+Jt{#rdB*n!CLxA`a;mYxuuv2h{^uTa}H}Ck+u1cOhi)6cGV`)(dWU3jnGSsc_Sh614_^TxW)hG?!{iy zlVLtJ3jXm;3WM8QBw}%&gK$AeL5~u$=8I52dEe()V}G)8{!!K1uNVoNc7aUYG>s_x z*(#EW$7D*R9eIV=Iz$NiL_}nmU zU)h7SAMx^oqGhA|Nl|NrIq%5U>8){}t+cA&U$|08lZ&K{#iqWAmLvj^iiWW}RXxUT z_>;ebk=Jbd=^qA@1Ah8>uD@>|=sh#^LYXIT=%b%e!JQv7kLX3;UClP^NsITkdGPCd z9(wH(ZVRKy{C5@}lGYBMzi(a^B78#4f?QhJB!{ztMS196|Gl~%@?6c_)(hTKH!=NY zToQJ7W4q0DwO3{d(~a*WZQASAudAjeCx3FD>AodmSokhl+_s!7MwySD?kpwW_S#(T zT3H*9H~RRLfw1)2d})0up)=Eb?b_Fub(X>VUDWKS0YCJwjDXfEAh(#a!HpXMT@?PR(y9S&Dv|qZ#vzO zo~>Nes8}5iwEzD4bZ54ar1aWon6RCh8&R!pu&L>Ea=!i2+8YUbL*HE87P(jE@m{~l z+)XR@7%_L46L6tgG-Kb-HWkwFdcM1Y)1lF{{Lv9*T;gCnxk)C?xp3EPujg*(=T@&b z5U%p8TE4}#;VVC(*e0{2n7=ddWc5U9#^ja!0ZX^u zLhF&9@X|J{hGIOhK3{1U>sz?JKv-hy(-s^4g<<@coJ(g9x>Vr|@wy!*D z5*bIACfkUw&4TZhbo>ewEw5)6G1OaMb@X23njWaC=wn*vTJ5PBOSc=Gtn)D5GfSG7Y787OdLx z4IX*&)z~51+^XWG&FLbs1s1ZTS~9LJ)8Hh@r0`_*a<=1I?>47(%b;4U88%IcP(wO9KZ5zI;W!1eq{V?MriV&$Hwov zZ%@tYtLa z3RjyJTwfthD>WFFI1r_>rcMa<3-V2_iE$=(t&<109R|q*B(=WvH2KZ1A`^3MX_~EG z2?Zag-ha~ZQzoUGrFbP5iA*t1VX?=rwK*@DFv>1zkvnp|$Gpear*mD7xS1THsIuG) zUK!fnXxc8d7aN--xC~+s>BQz(#(Q-Z8Rd=zQT}x&f5gf5VZ+5mbRU)`S}W5Py2o93 zyt}M1ziZ#CPudR~dNVajq((XQU1!V2`?oTm)lTupd)^-T5+q#J)KuLep(*qG3AsPL zAXoe4Vxx4Ey1M$ovHjTYWtWaZDPL-A_2#S((O|QLGF+;9-DlH2vz%$OkkBNj6298g zNz$1fXwTb*InDCkTzBr!H|SxI?El$mVa;h`5w9(qR!y`1JmLVOB8iFP%u#FUqbt&{ z8vnk7LUUDa|GI+|^^QCD&cEf6%jU-%Mqj0;3V{P`_m@J-UE(L67d1xPaK4nhozk3A zkjr^q>*nvyLVQ6=aFP_IRA$CLrda!{G|fixcemEYD>SO4w53=wU%b0~vZ@{XMqy|i z?st+0L(mz5`*ilK8yixXis8m+k84hgIKAqm*P^p(l39Fi>^2rLropmwIo3`6f*!Jz zYLrfuadtw;{b!4Zw&j&rjTW)gOT(nE1N~$$DgrKYamO$l6v92 z65I7vPr2+cr?FAl{S>3R#GfgQDN;p{+RLX4J-97vh#k*jFJ2m+wu#@EP?o>Ho!yj} zDr7x{r(>C@S|yZBncKEs;vRviEVF05Sr})txxQ>M5N~q%E#}a0uFt0JB&)RioHFiG zQg7dtp^x;J*BzGZgG3wGN$y~E?{ zeNC`>`q!_JH&^-`OY+R>u9j8h-B#YaOoDq{Vz;sS+rd3eX)QV5EA#9|bU;L@Y+R1d>gBKE-mXEZ&eXbqvDub4Bxs(6&s>hM&UJ~af ze&+0yy7ZOe-S31pW~nsuVfoHo5E^6)I`KK~sDH{n9ml2fZ0Vks`$PVkHVzTmu0$7g zUu|WrHY=xBVlR|#e~M9Kz04ie-KaDyzcJ-{O!U!;l$li0jI4{BZhK*UuZK*Jgj7-{ zDM6cGJ*IVuBZyPKz*Ls?<|C#3?JwFr+E|Yf#;YvVzqa%c8)Ky_G%^~0)fM+h zhsAau3S}67VN^eN? z`qiFl{c+O`CkgQgwVF)k7w5`b>N`BF^Ma-8&XB=PLBs3>lT9WMw`h6_ayzI=VVhP%lyNlE2ea zJ*O2J=b%ecu1(punf(iuE_6lycKsmy3M%DK<0P%OD>x*PbcomZJ>TOAf<&xQ=jB0L~dllrY;Zc zPZOA|t#F^4Qpj8!S1(Z`jbCU}(_g=SM8e%Mf@qZ6mQFI{kR7b`r6;h82HmwLydjaj zS86(IREGyo|1R2`y%aR7*Xb-DpX)Tg$f57L5Fl^WE^yuA9QWlyCBf^@f;WRx$a4lQ zB)*Q3D~i+nnTO%3-RjOGYk`0mJ_gwBRI@?1>eXR-=_!0t ze|dg)hBc4+5f);}6)&E`8b!;Wn=?j5mlRJII-e}&<}A|w*-|JOqLwuMO@)P_7@7J3gWDd zqojO=^6*d>a#B?TUK4KT(!?t zqSez^Pl}0N|D~?pDiD-WLPc?;M8Pv&eYbexsT<`(c~J`VFgwOu4hsD~1tX_)YEWeVd-ug-jWjlWs?N zDJhb#_()dC8<$-670?sWh(BM&q(9$2uH6TddRm_Rz6lU2i7$4LO}>FVV>$H)9bd`5 z1RQfKC*!7BPH>=sah8nBpjU|zk}W7E)Sqy6M5yp*?7kdEeL^dxV{92I6*Kufo9?33 z3m1!aij|xuR-@0^hYe|SG%5^OG1EFrfxDS!bDOhUe;OK%j&EA^rb?}>SNJPAEJ;m;db~Xz;dKpWZuT)6iEO?H z*lug}>2o(H;gSQ~IrpsSt#B_EFg557{54$>4dJAorqR=^m|3(jO3=7@*!YjLFFI{M z>7=Q}a$_g-hK*uPuu`|v*wb6}4r9OXPajF*4RP70AzRbfZ8H@jMEWc=S7v6N5ekUkc^v+uH3j8!!^8QAi6`222Y;@#0tSs5+l}z5CXOo+gj?>w{{l~_oe_8UbuAvov zwdc#+_}M&l|B`Ry8G~h5R5d_elp;gfQ7V$yH@8p9OhvLs3h(-o6E%N{JMvTGt;5)( z!hK4y?>{a4a(Bor$YjjPAtqXDn{ZfRImOo_U5#TiW(PB7E6m%=mU4LYUjVwKvs_Ql zXjrkInNA*gFbQ>}{XJ6+`CWc7nLyL=qQSB~$=W7D|HjfHci3OaW7Ammc#oxgSe+J9 z=7&)8rCFV>VxMk;-rD$0tLE=#d{(!r=_p+Fne?As>`G~gnssu1^e|)iEFBUBOiie+ z*rw8fp?2zrX2MBhoDmA0%MaqbR<28OYtgS@bQq16X#UMns%)$l zn<$^bZw_kQ(vkl)QYYXFt`|52EU~wv@!ZJX7#X+xid_B7t59IK zACO9z;J_=m@~!o0`?_H6V%siJ7mw|&&Ek6_N=79PvCvmnI#rZor+0TKpcK@3(ft%t zW09uLM=nD@U(h5+{!S?IB0H0|=`@$8I_TeF(N8TXPEhIHyO$TlDc%t0_KPD%+QngV zlTNG1wyT)q?453?j@bzb?@?j}@Xwx{JoKRZh7(lnMO`M-kF?V?VuvZ)X9*PqP6721 zE;-Lq5##c97{#5-uTJE#Xq-Fro*gCD>h<3;YAycyQP7cT&?X&t^Tf3|#G^qVv{x7|n*|Jvm!|C2LX*BpDYRa;?kQ=4v3W2fdF0^i& zfv&OO?IpinBgd*S%2%`T&?2ixiwP!lqHJG17KX_-*;kTl?7cRYW2FFOj2#SH!hOk# zOUz2ESslHY(4@m`2@X}uouz#){rCKWxFj2k9SAn+F%m_)20fSd%s|F1EKe0sZOR|%zkXHRY(O?buk{J-#3o7FM)71>5sz}j%&I&~AL9xK(coAIEPut83t1EIVu9CS8d@np?L>&o2VTiHgZ)_(ry2MOnyYm#1$ zKudN)U?ju&!KEjFGUp%kIs z|A}i4m4M6d-Z{g?fKyb-yl8as?ZQ8yF{k+-Q1uH3sgL4jc24r&ha!e23AZcM7##vm zOm**Bq?Z2-g$>Zuoe38nC&Zwk%opM-hd~LKFTvclKpqzmDqQ-oUB;Fr1MZm~S$}8q z&3|QcHaG|qOpnC@PZZ0*apu$3LS#+1VA`A8=#pX4lxcz6V*#9N5#*L~`)I)S>(W5d zGQ0z=gwJ%Wc1gBVo+9eN6p|w70qN<%kg8a|Q@gfuS+mpk({-g});OT5515~&Krkx% ztSDP6?rjha0v=n_T!d_zalPP9XN8L++;bT0RXsg(qhh4-ic%4+p^Il?T+4yA)9Ph8pH+zPCIz7sJm+5_wS>$DVtmg^=t z8^>tx4>7|;B<`bAwTyF1t?&Ylun)4lEj{~*cPM!Ukxux#M!>`)T5(!50_sg|Dq(72 zCAM9SC>x$zwGwUY++!i(v*pPp<%EaiZq!+hwM8T9N3{4iSoN$<1u&g)W}7U`CQv10 zJyr^19IJb>?WIOTFYTnABU%PG?)i44>(+_ZY}_{xt2JT~|GRLYGw62oKtm)|J$m{w zPzQ)8sYjvOw#+Csk1NqEn#mXZri!{T?#tj`^FPg;R1hrHaQ{r zN-c`tV=qjm{*^&`<}RO$>Be6OAPme8yYuaf%Vf2b`mOvi^(psSx* z$e;QsttQg?UnDPPQ3%7`J7M^4eepPs7aK zS{TYh>`@4s`E8pME|i5tgol^b*>q+_ja@~9yPp3olNMBuVxW_HtG&o5WA9ABcJ*YJ z3{3GDRl4S(0qi{6Z|y;vE!x9!yu1Fz5h{n%a47Bu+Z0mdK#)2R`X2+FKi26b!yl*K zr)tF7fH+hbA#4B%fzYox-WdDXK?-i%d@5`5mNZOGJ>+?7!+}b-#T%|Ki|zY*wUxQY z9z`g{{+B8@u9St&PjWEje*N(D!wh5D#)tcn4)!UeBkTIvX+lLwOf`xDg@uIjMU0BO zIV|7YI&0Zj2te$!eQuxd)ht?k5qj?*W!+VFrEER?AEI422-A^kD(@g2bS*t$7mU#) zQ*BHJsfKlEeD?$O>^QMx(7A1i@!+|e0J#c>hgS>a%CkcwRE`*Y)m!=lLA1Zrys%(9 z-rv|RCuFC)BDckZaepF(Nw@mcTVzXwa_m$RBERJ-P3nJO|KSm}1F>!NYuHFJTAr~} z?djhaS=o>C%x)>tB$|g%Ak5++vgd?mCAO7bjSQ^DaP%n)=LFOc+_Ks7KS|$x(tVc} zi|Yl_pI!|L)Pw1me+GMJk}{J)r)T4v^EsUH8S{eI&^kGBhYAUG5#W1q>2)=YxgMTM zuPIKJhzd9^wrh63Zhv*6DVqwRl}q_LWBiF0^So_U0Rfy!v@dRo@y6AW=D zX(cfO|HP#i1C}+BKMU_gzL#yJg$^{f5sdgwKJiw&u^le_;pp#mjVu?+i=e3@7~0cb zOvstXXoUYIhu1&I@}c~*#)c|Ai8}bAlu=Ls{&W%AI|Ng&aC=iXhdmTwcK=6t$2?QG zy(DOQ%8QBi7#g-53JH6Vu3MC$7p^mC(J&r-2k-{U*N-fvAkH37YXmtXnupspBzOsO z8Xx95y_Rn3mAv00b$u(uKD148-rgh`0c^4R`Tk*#&5oe2clj!1JCtHO(6Ea=*Y1;h zn!JC_q7oqy3R?+UF>E?AB7*Lq7lJ7=KZZez0a^;7|G-Xq`EfhH=Jr}@0mR8}3lS+1 z4QZk_80PED31JGgTazH6mK?hDG<+Z}-4m_np!wcO+Jk-@C3JPS892pI;dM75T z&ka;b+^XHh1zK<`(X#Q4wOL(o(Y=u`Wl?rzM};XeAgEWSGIM($QPaU>KR9h$5`Bq11^hU6gA=y; zt9r#7%JS^b0D}@sP%!h5>oImugOKyHxJ!QKJ z)MLN{#{DDZRkuF}-1jjU5{$GshFP9y`KaG7J)KoBlN_dTzw4p<`eYg*SQ-lG%oi?4 zGzLq>b)xbiqnF$SGe@#O2UdyAmlJuBgw=#0hE!;7-e(K1SH8YS>txl^%h(5v=Xbmg z18KLU8NQ(~J12Z?I=3e71(Gu5gUYM)TG2kB1R3-~a+~F`?a0uNl7EwPy#q3FVF?ui zfIyV*!i(KVNBg^5mxM)wthGHaS4!il1HeI zv3@VSX%E>)^nJEbNn^Rw9eUvjgjYQWv9H91 z4Tvd2Q)s*?ZM2uov!Acomk{pwkoyMvCxSDqtv;oLki0VF$6J2J_gmyiP4aYx!a6|8 zq1rHLf4@{!Uu?^(7mG2C&y4+8kBW@9-~x7wig<3(I*qEkMmB4b_vh-A1=3aNvdP9x}BCHkjwe%yi6`%*tJmqqzZ`e@o zczN=~i9!&Ndf<8;=Bh|>MpP-*p6icM5~3r~G~e7m_HON-do?Jlu;eD}z87JZjTEbp z&)v2cxKbKBdVL!A5k2O46O zW!$~b2Nk51KDvOzrHPxt4bzir?V17t2PgL{t%(X# zOhLVPH@LM@;;^cINGXCOhMff|tO(2y0!g{qMOQi3AcMT(y0u{Cs`b$?x_n%lQzMI+ z5~*i;oAKm8_|(;(08??a^=dd&`0Dp{!sgwe=KtZ-PI)FaLg%6Xl^{&|uS%siJnj>3 zOXrU+jyIJ+>sMY3V5L{WuKVZ3SE;LLfMeY9_a{^@#?Ez1&sC0W8C`M#-_e63@^l(8 zl8(W`N8&WfcZdHnoQl@XgY;qnCWL;L*9)|*bhdwI9!{T$j=w+G7} zSdxD}X3DZ|OXct&Jfx!Og?}&Qyg~2jF1VpkUr^2F_K(FJTSEJ>e0E zev%RcDarO^)k1?XYTaqAs|J&dofyo9Iei-mIeP6c=xBUL<(u2_E3a=JQ4Y>(Ee1U1 z)D@|Jx^&+WCG%;K@CG$pt4f&nukTT2&=6wT=HfhN&-dk91Xj3;03JB~gb%UK2cE=* zp&QNO7BY~c^!6@y;>3Wec;JLO(V;bZT!=7 zAH#}-Jlz-9f4^}VaO-imMlkG0cXQ|+W;v8nnc?>QC`Aj~Xz#bzX8jrZnOQLWdrZ^C zyAAg17iZV%HpASCttSr^D;RDsff#>;OC4taoPX>eO0I~)v(7Up%$${D^V53qY|4%+ zJZl$TSL6KWJKtYxyBoEUel^8A0s#Ap4xCy2xg143#b0=nMY)`S*G{srM~@L;3s5=1 z-hv{%Ys27{5ekEwh7VB5^-=5k{D6Y*M{kKEuYs?Tn0_vpz&d&*1g;xVMW}q34AYd$ zgAoIG;%m~br$Fv&1z$PP`QATcI{k2o&`f&%02EeKQl9-?_F%mHXR`aA2>Td%N+YUe z004c~X|YNi;BGwgaMA<0uTbT_++P&40~H681>OFFiludI!B3fZXDRg*`=+Avqlfv+ z<=Rm}Qlgwaydlo=GM{aNlQ!YeBT8F{JG5xyewqUv0n2B!G)~r|7X=L90CYf(6|>ha zjD7R6H_E&!A}PgI_`VYZ;8<-38seVmAU3E0u;fSa&fjr+Z>{qn=6eAnYKZ`>KonG! zsM;a?`fumbacRZur9m*gr`C5N91$%`i4A`M59dr@^HOA0r=AkwW8L2+S$3+e^bHwU zPN5`#?btxBS z(2S7~L05`70?~|Uhjq{~4tg#IdHw!;U{=|HDYy#9jwDQ>4I)y82d^8X=$p_=XLS~* z%V?HE8p$@U=!Cp?$Yb<{@(h@w@=!-e&_xs_zxc4ukL)W3L!JTCEoj0V_$>ZnD8oy# zXWZ`m8)Wr7+jr#5f(cv|c*-mzjh&;=wq3OJCnf2r5p*i(wJWIS>1$n&3 zQAmKdlIz^vgzOt-t-z4~@|JR$o$=^{sVJ`<2eSP+YOUWkB-_RXcEwzXB8m zs0W<;>@A9WN*qm($3V!x%*ygf%-}{=BVBLv$K2L9{5RWqG2|a?rzlH;Q11#5;MwzT z>!dkH;G|0Ayw=T^vMt0g=_(4x+f0Q3B98r1eZ^mYJ{$?%^Frhov$^FV_9RV23lOk= zC$K@KWG9c$0c?WyeaWg}bn(hVf?ROTz+&D4e%5P`Zxq;uS|Ov$cXK&bzW92)d$CF0 z5k*WM>I{2K05Um~i6s?vI#Gdpkg@xb4UU;rotUl&;Mle%>X;C7ci4b9OV%@)HX8R=D3)WvXq+mH%^INqmC&@x4c|d(o zL*};vdDf-IXzgTL%gbYz`wKsK%$C)OmQNgZF%^0sAk2CCTc;H#5l_yoBb@W%Y^!pWml8Q|_dXoPNv%QED(OT( z0*mY$L$~>*FI^aG)+xvxF7mwIt8eLb&ffWcS9dI&){lYfRZAC00$s#f`jbx6@AUX; z!x|2%1>J~H-adyiVfjOOG$T}c_z+dnm%$T@CI&b3e~%#dFSAqX<#5v^df!$)lm^m3 z#{jx6o@8AeYU_9`{(WWIbi0?$R_k?xIj&VqHt3w}QuC#IL#jeCuukibK$D7?t6rrF zPXo2V`RlRB?#~BI-?VhKh$o1BN}F>38|*mxE($|E59JVVe8cR-f)o1YifgS87QK;) zI|C-l)>Df><9DY-noM=_TLh?u^PaoQH#x%gBNydORt~!UPECkGR$sOtDRh58HpJS7 zqLx4C=#r}d>#ht3p6q?Hhs6#Wu_#YB(?@Za_8|J=6CYL|$^0qam{S0urFBH+P`o+B zw7_GbSsI9aN`K)*8@h({_ct=& z@Y5~Fo;$52LaU2r;fxJ53p?c*it`|$BN;|x!Sp?pc*@i{#g8CM+W(bzU7`*d|H6Y# zI-B-jys^4Oc6}l#bLq9orH-1etm|f|85&lp-t;C+WCvdRqfHm%s9_bFV3Mh?AChpE z_>!m?m<6X4?N$L=flzb{x-8@482v(GdRm-eO5GACID%a|-jkOmb%`XUa#$9$WjEeb z*G=d;O|&Wzdy4`qG)oM!uDxh22E8%8)huy7TojlO{L|(fA4(uW{xA>hJo*>rans84 z(<{!NGMjwWfusa@#(S@)dz=NHz3H08UI1BnTYKuIpYWUOg0!1mdd}NE8QtUI= zuc0*#`otyx77jq9l^U_6)b0liSb+8u&L5}`nD3AgFTw+18E}`K5l)1qha%Kc@}D3` z(B7z{u2lrO5J+6uE^kz%Ks~cLPH0o?03u(0{KhE1bMplnizE&Nx1d)72?$}+vT)Gv ziaBc;oEe@*=Z3QHK*UtkVZR=s=4125TlF0YK9{H#l41l19QBe9JOh`wRJ@SIK~PbP zftIuddo#+WSbv5xy!LF;qrMnTYPP3#lA52Y?)1Nt+Nf1=G^L(J-_@}^nD`1o@5HFP zJ1ypfp_Y(GTm~S&2>QzPsZ1iIm6HsYp%pFx990WN1M?JCa9JYUz`uLG%-06cPq)^E z({DH!Mbts@8&gVaJ(fhFX0YY8++jd?7N&C(I1A(7wAucxT$fLX(<1o9PZt?4$R40R z%1rTOe}WCGc{oi*g}pxS<;dr|@i7kNU!nVnUAo7R21|iY8aO#UQza@r;G$?%fwig^ z&h`M#3c-lMinD%r71$iY#z*r-M*SB|2y>`y@#n!f9j)c@+v_}s`Qp>&POtAuRtAs; zy9BJk>|5*wd{GYWQ|O^fGq4$=2|^PyzyI5#ah8kBp=TcLVHSZh1+iV6n0S~Bv=$L< zAA!9#1Xyneh|77vZTvcrk7w5wkWjWOgH(xu^Scx@uQt5S-2Vl2>v|7(E~&@sJ0b)iH5}=l7rm`Og)FZo%UsRmlF{ju1e>i6FuM* zH{e%*zbW$Laj)lb8{H@U>$TZT>*bT!BrRC?FG9cGeChh*c;_jK>mQe7`J(>=mO|Ne zbiYB^$vwyiP!c}i8JX!|us^@L;@pWvT{rTcp;-V3kgIplEeRI+K(p~N?YoAIW<1|5 zz~+x)spF=Um?zp8N{klV0|f)P2cTSKITN!w(;>dyf?N2JvJ?9!??8vOql5>u%r@ zIXFM`(SEWu1>N>w^fwpM$BwGiaiCTw{)7Vl$l(#os7u~E^)=Fg?3m~(rL2)_Pd2*d zT;^Vth4@%fj7Hb*#sm22=`GrpGwTD{&;uZp{rb|R^PheVfFrXHCKBRiPV))zc$|`? z)Vn$OnJ#tio+qNCr-2*xcyem!3Fe%F1Sb8N0ys@#%EzsN?XG~%{M|P{oguiLp@E0F z63~9`k{@FcXr|xLP5~;*J79Wy0Czre;KZ?xeOgcZMCX}%_snhtcZv>+YZXAVAHipi zP|?K}*u0#ecj7ka1I}w>krj#i(1Zv165V-Y54ew_gtOj}gZ^w47r)QUZ#o{T(8xc$ zC_^(rsYjzdK=J5~&opov3jW{f+xOekxDDlFGJNu5>r1k=EVvsWq>TBgMb(+896)da zhYJy${0k;6B-9x+u((4q6gVGP*lU6k$$DC9GMS~XMHVnQnuAoh8_M8NBJZbH86Bo& z8rlPl8|<~h?}ueH^nOVeQDJ~aEF-Xme{>J9`RqR|qU)JSQbLob19$1I1k z?5%w9a0VnJ7@D!cHo z=cczi&kVxy9+3FYTFD1an%G@>x;HOY+NJj&CKLorD9=9ceCHl79OqyCm|>ySEsvEM z!73ln{UmO_M`EO~84L%{HDArSVGVk~19|uw#&@4_K4)u8E9_4D>FJHTm_fRMrdk2K zRp8Ld24S!>O*6ifd_cs|9N1vzAyp~Z{Q~K?pr^o+C4pJvPs#}uUr((`?A%L?D4iEh z$5p85Z{~zFxe~x8oZoWd38pCEeDdXTrBLSUPe3BkVOd8ao_EUr1sz^$UGP_cUPLuM zEwPSS>u-3d$ZScphAgkJ%z&i)^o8ffwSf!S2dS9h_4Mk5SzCX;U#>=@fk z*vsefy83_{WFKG)4)Z49^wz4kbdNt)YhYUq6#+U4VRnGMC)^OUry3p~{5%1`q2m&c z61Vs8I}knqxg~Jk@vd?1nRL<_@QSKks(SMQ)(%n|g7{WOl(M6GPrVo=tCpg}(jGh@ zZoq@Q?l=tAxIkoUCH&D$l|&oq{%-W|&vrxDIy{Mf&X<5{#u(g1lLar4)GJ@_yr3OD zhQ=$i1{%D7?l$EcAM0C^g5YT?Fe3EhPmVJ{ZlAtTb)G&mbEixCG{e zP!D&>9H+1-e-{~(wK+&-a4?J?uI*o6n2Sa}vI{~#nSMIHHFziFy5Laq{zv{U`v_5k zO%FpH$BRJQSED@+hv|n=$!hT_-KOwAIwZ(HYVF!!iOwT(;Rxof+8H+U{h8CpJYvrz zK>{0uEDLccMahb*e2>l{8tr_;J72Q}US?X9h(a*Sa5i?dr;*iMa`oKje?Q*k^S<9D z*8w0QxD$skhb;Vkxm(%egwo&pO$jl`p*ALvv-(qK_lwUrF)GS|(_&?bRq_#$R+SjK%NjxO7|^Lo&+-{cDO z8x}Q>xz>KjcthqSN;Dy%ig1?y7h7bx^?4DHAs1NC+X>4!pTY*Rg#$x)>Tk67ekE#b z11By(cgODGhfZ-Zar|#U5G>S?HtF#MRzEHy&<+O$EG*lYJU40jw=XhPGv0wja53#K z{yV7es}5B?Zd;H!Tea;fBcf>_qBxxzo&AllDG!nT*x_Q?!q{2BUprM*Pa(0iv|F99 zC}!RIi4ci@_wHT)xR$b5kJdx;1}hkSq#eUK~cI1FHc`6eI(Fj+oB@1(hQhDD_*7UOlS%Gl^6}bWY z+C!&6S8nZ^G9$o;P9Qy0`B9MR-l3QczAziem#d)VCd7l4>j+2yJ>hdf()A?Pi*xV&Wh8cqpGDl0Xr9v$UjMQbb0sAIqa_uG zc!gxZ;uMB{#ppW~8iDjbjUf4IHQ#QltUUDFd@ncue-C!(KYK6#!vhI20@T#hvS3r_ zpCPy(52R|@CgKV0?yL@yVaK8G$=fh|ZSAC|#LuB?9NBZ16z3vF%(f0=Qmp>+@S>aF ziGaTU60i1_U%$Sy+a08CBIDi)m$_o$+{#vmn$4Aj7&clj_Ht*UjpdtrPu1gO&QJU@ zC!kxmsncPDeoaLmK!5CklpO6OrS~G=;!xlZ28i2(>zyU25a__WF=bFg2Pb$-|8n~F zC~Q=`ZwgA>G6Kn3a@hTnx8o5-5wlrO97|$up0Tx=Ft28 zA(j1DLj>9rk=j7HNr;IQUsuMEZ-Ef++<1Px8>sm#*hcOQrhvjoB!IKQ^!C2W@Y;Pn@ENshi znH#tuc`^xHy=I_8p` zg3ddumczFFrFrD{mw|lBhQ%!bm{i4ULcD$B6N_f)<435p&%Fzadln7MN)K8C|1$lb zKqnf^$VAll4gy(e(`R&#eTjVHXq%=yc6$&I5^1}aW?e&Jh}`@Sac_1D&n+BA^b+>OGACmaowBYJ^X|Md!E4q;#Wt9ig`7kL7;QOAvVt8T)Tw_G&UNPcO_Utb@JrNxM>fW^sb2@&yVX8CR7IiocZv2vT zXs89khA2mGC~z=B<6+|oQ}@Y~*_okRe%!y+on|LeKAh?yA!Cwd*Y|@grvEm=koga!q7i?5T|FE_6vYhi2vsL52lO z`z!tO;twE}8h5>tY%sFp2#xN!yhG{^jW)1+Ri9U zZhX7-uyQIR^O1w0(&Ux$)Wl~}7tx-iPB9y+?KXfvC@kE|6dlJ3fol=(i^to^D-2z` z4wD+!4&m*|&!Bu{flX!BW>`C^H75F7kdLnZTRfx_cy2>Mll*qS zK2K}mb9~v*Q?7n-2d5j)8szYh?WF^wuR!2MKor8+`e^cMJ+rd@<9RYDx%mu>?Kqsy z;@-3Zz)gr@7SMkkur81>7Uhv=lU(cz5Q~mP^Iq&_@XP}^a zia<`7O2`97cc|XhCCV`*Y}`2+QOqB1QrKD4!Sidalw7a_rMHFVP@u( zH+auK<_)h<$M(hw&*|PC1d!N)KC6cpc1bv*s2LT;q44Y{bp!ch{0FqyA>8-t524g| zDLF7OkO=ijWN|RBZr}QZWa4vq)2J4K2?j4z|f^7cvg}_su{ZjPu!YBi#W1${W-HPuK?0s2hxiGpi#Z zCFlbhIK^$uQQHX^ySij^oGVv)ZC6nG6eLZP-qdSf<2<%D)=)bry7|R^u$lno-W%)E zV6{PMT(AJ7~0kWgiY#HpFBFxpt zZ*N$V(e8SMq1GU7PdatF(g0U6|O$q7WpIFcb?^vw(7yAH>OMLU7QIJ$zn@{;0 zIwc}l>+(_kh^^WZ!4v|c^Pbl(MGOUt}W(0xbb~frJ^%J?+p0yBkpXk+K=I zu9-*er|S-mm}n9!mRc{5tZc)s3Bsny&48+Gulw*6mmbt*ohokPLcpTb4@qug<^h6P zg&2CzkInH}DTwc3@>#uvKBL0|IN(BM9=A_HZ(J#a^f^VeRbETjM^|L?y=~k^p$t6y z=oFvr>{>K9+6q1fb z52UWVK_4Q8s;A(cngEciVyVC@3N1wUM9nI(pIpLbBs{eFlrdiIi zBqYBgNcu99b3ahgpF8_22e4aJX2gQt_lwgxOTEz&~C$z0}*05|r&0?}^kZp{~Y;o=0>bcJu zLV2i7iAKH>zLIR2&j^+^8)GL4w4EpdB#+B)Pu8{rx}CHA*SK}~4cp4vRSjq=rSu|Z zdu?;EzFcD28ix`pwZjym4JF}mUNK_UFDtTJgxJLJvk!!k^q&e~lp!oh>L=*Fo+qzI zCw5hPc^oij#cj+NcNiY2N>yqoWEWm3V|`la(2z9c3Q&6IAz=1L=!(PmHJugt+V8O8 znxn;o_D{-%q|#@E`mQv+_2ZeMv!4*JBBk8T@=ATgUj8cLB7V?|5uud_Tli{1OKLaY zTkYs;1J42}{iFzq7_9-HI)@_}!D;&#^}j5tJqWz`JfkbZ#c2AS7-izik?7V`?TCe= z9u6(NaSEK1+7Z%0yC-FBr+T3P8>JH1sPsv8j~lbNVYLqp0DZ*gpNG`0PsA z{&f7M6WPC8se?L>XP2}V$_0s~d#%rPd*#&f3Yq30S?R$hD*J=%(p`O1r_x1wnbOfT zQ3`2-j|P7!?vJ}0cq=IrY@GB;xmJ30wMlLw_UhMzPR$oAdZVTuSd+VJ_qjKki*=mj rpw2O;7T8(XuK%lr`oDn)ZQE3A0&+)klAX0tB%!LPp^$U&=A-`u_Why_ literal 0 HcmV?d00001 diff --git a/docs/reference/index-modules.asciidoc b/docs/reference/index-modules.asciidoc index ef7c40b11d048..e34a4483ab6e9 100644 --- a/docs/reference/index-modules.asciidoc +++ b/docs/reference/index-modules.asciidoc @@ -36,7 +36,7 @@ specific index module: `index.number_of_shards`:: - The number of primary shards that an index should have. Defaults to 5. + The number of primary shards that an index should have. Defaults to 1. This setting can only be set at index creation time. It cannot be changed on a closed index. Note: the number of shards are limited to `1024` per index. This limitation is a safety limit to prevent accidental creation of indices @@ -199,6 +199,7 @@ specific index module: This setting is only applicable when highlighting is requested on a text that was indexed without offsets or term vectors. Defaults to `1000000`. +[[index-max-terms-count]] `index.max_terms_count`:: The maximum number of terms that can be used in Terms Query. diff --git a/docs/reference/index-modules/slowlog.asciidoc b/docs/reference/index-modules/slowlog.asciidoc index 235256bdce7c0..a96c8fe995b75 100644 --- a/docs/reference/index-modules/slowlog.asciidoc +++ b/docs/reference/index-modules/slowlog.asciidoc @@ -26,7 +26,26 @@ index.search.slowlog.threshold.fetch.trace: 200ms index.search.slowlog.level: info -------------------------------------------------- -All of the above settings are _dynamic_ and are set per-index. +All of the above settings are _dynamic_ and can be set for each index using the +<> API. For example: + +[source,js] +-------------------------------------------------- +PUT /twitter/_settings +{ + "index.search.slowlog.threshold.query.warn": "10s", + "index.search.slowlog.threshold.query.info": "5s", + "index.search.slowlog.threshold.query.debug": "2s", + "index.search.slowlog.threshold.query.trace": "500ms", + "index.search.slowlog.threshold.fetch.warn": "1s", + "index.search.slowlog.threshold.fetch.info": "800ms", + "index.search.slowlog.threshold.fetch.debug": "500ms", + "index.search.slowlog.threshold.fetch.trace": "200ms", + "index.search.slowlog.level": "info" +} +-------------------------------------------------- +// CONSOLE +// TEST[setup:twitter] By default, none are enabled (set to `-1`). Levels (`warn`, `info`, `debug`, `trace`) allow to control under which logging level the log @@ -83,7 +102,23 @@ index.indexing.slowlog.level: info index.indexing.slowlog.source: 1000 -------------------------------------------------- -All of the above settings are _dynamic_ and are set per-index. +All of the above settings are _dynamic_ and can be set for each index using the +<> API. For example: + +[source,js] +-------------------------------------------------- +PUT /twitter/_settings +{ + "index.indexing.slowlog.threshold.index.warn": "10s", + "index.indexing.slowlog.threshold.index.info": "5s", + "index.indexing.slowlog.threshold.index.debug": "2s", + "index.indexing.slowlog.threshold.index.trace": "500ms", + "index.indexing.slowlog.level": "info", + "index.indexing.slowlog.source": "1000" +} +-------------------------------------------------- +// CONSOLE +// TEST[setup:twitter] By default Elasticsearch will log the first 1000 characters of the _source in the slowlog. You can change that with `index.indexing.slowlog.source`. Setting diff --git a/docs/reference/index.asciidoc b/docs/reference/index.asciidoc index 8e5fea810fb7b..f9d4b5a4e0528 100644 --- a/docs/reference/index.asciidoc +++ b/docs/reference/index.asciidoc @@ -10,6 +10,8 @@ include::../Versions.asciidoc[] +include::intro.asciidoc[] + include::getting-started.asciidoc[] include::setup.asciidoc[] @@ -39,6 +41,8 @@ include::cluster.asciidoc[] include::query-dsl.asciidoc[] +include::scripting.asciidoc[] + include::mapping.asciidoc[] include::analysis.asciidoc[] @@ -59,7 +63,7 @@ include::rollup/index.asciidoc[] include::frozen-indices.asciidoc[] -include::rest-api/index.asciidoc[] +include::administering.asciidoc[] include::commands/index.asciidoc[] @@ -70,6 +74,8 @@ include::testing.asciidoc[] include::glossary.asciidoc[] +include::rest-api/index.asciidoc[] + include::release-notes/highlights.asciidoc[] include::migration/index.asciidoc[] diff --git a/docs/reference/indices/apis/reload-analyzers.asciidoc b/docs/reference/indices/apis/reload-analyzers.asciidoc new file mode 100644 index 0000000000000..657f6556df4b2 --- /dev/null +++ b/docs/reference/indices/apis/reload-analyzers.asciidoc @@ -0,0 +1,105 @@ +[role="xpack"] +[testenv="basic"] +[[indices-reload-analyzers]] +== Reload Search Analyzers + +experimental[] + +Reloads search analyzers and its resources. + +Synonym filters (both `synonym` and `synonym_graph`) can be declared as +updateable if they are only used in <> +with the `updateable` flag: + +[source,js] +-------------------------------------------------- +PUT /my_index +{ + "settings": { + "index" : { + "analysis" : { + "analyzer" : { + "my_synonyms" : { + "tokenizer" : "whitespace", + "filter" : ["synonym"] + } + }, + "filter" : { + "synonym" : { + "type" : "synonym", + "synonyms_path" : "analysis/synonym.txt", + "updateable" : true <1> + } + } + } + } + }, + "mappings": { + "properties": { + "text": { + "type": "text", + "analyzer" : "standard", + "search_analyzer": "my_synonyms" <2> + } + } + } +} +-------------------------------------------------- +// CONSOLE + +<1> Mark the synonym filter as updateable. +<2> Synonym analyzer is usable as a search_analyzer. + +NOTE: Trying to use the above analyzer as an index analyzer will result in an error. + +Using the <>, you can trigger reloading of the +synonym definition. The contents of the configured synonyms file will be reloaded and the +synonyms definition the filter uses will be updated. + +The `_reload_search_analyzers` API can be run on one or more indices and will trigger +reloading of the synonyms from the configured file. + +NOTE: Reloading will happen on every node the index has shards, so its important +to update the synonym file contents on every data node (even the ones that don't currently +hold shard copies; shards might be relocated there in the future) before calling +reload to ensure the new state of the file is reflected everywhere in the cluster. + +[source,js] +-------------------------------------------------- +POST /my_index/_reload_search_analyzers +-------------------------------------------------- +// CONSOLE +// TEST[continued] + +The reload request returns information about the nodes it was executed on and the +analyzers that were reloaded: + +[source,js] +-------------------------------------------------- +{ + "_shards" : { + "total" : 2, + "successful" : 2, + "failed" : 0 + }, + "reload_details" : [ + { + "index" : "my_index", + "reloaded_analyzers" : [ + "my_synonyms" + ], + "reloaded_node_ids" : [ + "mfdqTXn_T7SGr2Ho2KT8uw" + ] + } + ] +} +-------------------------------------------------- +// TEST[continued] +// TESTRESPONSE[s/"total" : 2/"total" : $body._shards.total/] +// TESTRESPONSE[s/"successful" : 2/"successful" : $body._shards.successful/] +// TESTRESPONSE[s/mfdqTXn_T7SGr2Ho2KT8uw/$body.reload_details.0.reloaded_node_ids.0/] + +NOTE: Reloading does not happen on each shard of an index, but once on each node +the index has shards on. The total shard count can therefore differ from the number +of index shards. diff --git a/docs/reference/indices/create-index.asciidoc b/docs/reference/indices/create-index.asciidoc index 54efcd9005ac5..3038265891458 100644 --- a/docs/reference/indices/create-index.asciidoc +++ b/docs/reference/indices/create-index.asciidoc @@ -12,7 +12,7 @@ PUT twitter -------------------------------------------------- // CONSOLE -This create an index named `twitter` with all default setting. +This creates an index named `twitter` with all default setting. [NOTE] .Index name limitations diff --git a/docs/reference/indices/open-close.asciidoc b/docs/reference/indices/open-close.asciidoc index 6d0866d303b88..4ba434ecbbb6e 100644 --- a/docs/reference/indices/open-close.asciidoc +++ b/docs/reference/indices/open-close.asciidoc @@ -2,22 +2,68 @@ == Open / Close Index API The open and close index APIs allow to close an index, and later on -opening it. A closed index has almost no overhead on the cluster (except -for maintaining its metadata), and is blocked for read/write operations. -A closed index can be opened which will then go through the normal -recovery process. +opening it. -The REST endpoint is `/{index}/_close` and `/{index}/_open`. For -example: +A closed index is blocked for read/write operations and does not allow +all operations that opened indices allow. It is not possible to index +documents or to search for documents in a closed index. This allows +closed indices to not have to maintain internal data structures for +indexing or searching documents, resulting in a smaller overhead on +the cluster. + +When opening or closing an index, the master is responsible for +restarting the index shards to reflect the new state of the index. +The shards will then go through the normal recovery process. The +data of opened/closed indices is automatically replicated by the +cluster to ensure that enough shard copies are safely kept around +at all times. + +The REST endpoint is `/{index}/_close` and `/{index}/_open`. + +The following example shows how to close an index: [source,js] -------------------------------------------------- POST /my_index/_close +-------------------------------------------------- +// CONSOLE +// TEST[s/^/PUT my_index\n/] + +This will return the following response: + +[source,js] +-------------------------------------------------- +{ + "acknowledged" : true, + "shards_acknowledged" : true, + "indices" : { + "my_index" : { + "closed" : true + } + } +} +-------------------------------------------------- +// TESTRESPONSE + +A closed index can be reopened like this: +[source,js] +-------------------------------------------------- POST /my_index/_open -------------------------------------------------- // CONSOLE -// TEST[s/^/PUT my_index\n/] +// TEST[s/^/PUT my_index\nPOST my_index\/_close\n/] + +which will yield the following response: + +[source,js] +-------------------------------------------------- +{ + "acknowledged" : true, + "shards_acknowledged" : true +} +-------------------------------------------------- +// TESTRESPONSE It is possible to open and close multiple indices. An error will be thrown if the request explicitly refers to a missing index. This behaviour can be @@ -36,6 +82,6 @@ API by setting `cluster.indices.close.enable` to `false`. The default is `true`. [float] === Wait For Active Shards -Because opening an index allocates its shards, the +Because opening or closing an index allocates its shards, the <> setting on -index creation applies to the index opening action as well. +index creation applies to the `_open` and `_close` index actions as well. diff --git a/docs/reference/indices/recovery.asciidoc b/docs/reference/indices/recovery.asciidoc index 0929b36e7742d..6e03ddd16b71b 100644 --- a/docs/reference/indices/recovery.asciidoc +++ b/docs/reference/indices/recovery.asciidoc @@ -249,7 +249,7 @@ Response: } -------------------------------------------------- // TESTRESPONSE[s/"source" : \{[^}]*\}/"source" : $body.$_path/] -// TESTRESPONSE[s/"details" : \[[^\]]*\]//] +// TESTRESPONSE[s/"details" : \[[^\]]*\]/"details" : $body.$_path/] // TESTRESPONSE[s/: (\-)?[0-9]+/: $body.$_path/] // TESTRESPONSE[s/: "[^"]*"/: $body.$_path/] //// diff --git a/docs/reference/indices/shrink-index.asciidoc b/docs/reference/indices/shrink-index.asciidoc index 24a67208f7289..aa73d3d1350bb 100644 --- a/docs/reference/indices/shrink-index.asciidoc +++ b/docs/reference/indices/shrink-index.asciidoc @@ -17,7 +17,10 @@ Shrinking works as follows: * Then it hard-links segments from the source index into the target index. (If the file system doesn't support hard-linking, then all segments are copied - into the new index, which is a much more time consuming process.) + into the new index, which is a much more time consuming process. Also if using + multiple data paths, shards on different data paths require a full copy of + segment files if they are not on the same disk since hardlinks don’t work across + disks) * Finally, it recovers the target index as though it were a closed index which had just been re-opened. diff --git a/docs/reference/ingest.asciidoc b/docs/reference/ingest.asciidoc index 6fa2e8c796df6..a945a56c1942b 100644 --- a/docs/reference/ingest.asciidoc +++ b/docs/reference/ingest.asciidoc @@ -23,7 +23,30 @@ another processor that renames a field. The <> then the configured pipelines. To use a pipeline, simply specify the `pipeline` parameter on an index or bulk request. This -way, the ingest node knows which pipeline to use. For example: +way, the ingest node knows which pipeline to use. + +For example: +Create a pipeline + +[source,js] +-------------------------------------------------- +PUT _ingest/pipeline/my_pipeline_id +{ + "description" : "describe pipeline", + "processors" : [ + { + "set" : { + "field": "foo", + "value": "new" + } + } + ] +} +-------------------------------------------------- +// CONSOLE +// TEST + +Index with defined pipeline [source,js] -------------------------------------------------- @@ -33,7 +56,27 @@ PUT my-index/_doc/my-id?pipeline=my_pipeline_id } -------------------------------------------------- // CONSOLE -// TEST[catch:bad_request] +// TEST[continued] + +Response: +[source,js] +-------------------------------------------------- +{ + "_index" : "my-index", + "_type" : "_doc", + "_id" : "my-id", + "_version" : 1, + "result" : "created", + "_shards" : { + "total" : 2, + "successful" : 2, + "failed" : 0 + }, + "_seq_no" : 0, + "_primary_term" : 1 +} +-------------------------------------------------- +// TESTRESPONSE[s/"successful" : 2/"successful" : 1/] An index may also declare a <> that will be used in the absence of the `pipeline` parameter. diff --git a/docs/reference/ingest/processors/date-index-name.asciidoc b/docs/reference/ingest/processors/date-index-name.asciidoc index fa749fd342056..783ecc9b2b1b0 100644 --- a/docs/reference/ingest/processors/date-index-name.asciidoc +++ b/docs/reference/ingest/processors/date-index-name.asciidoc @@ -140,6 +140,6 @@ understands this to mean `2016-04-01` as is explained in the <>. +| `index_name_format` | no | yyyy-MM-dd | The format to be used when printing the parsed date into the index name. A valid java time pattern is expected here. Supports <>. include::common-options.asciidoc[] |====== diff --git a/docs/reference/ingest/processors/geoip.asciidoc b/docs/reference/ingest/processors/geoip.asciidoc index 4e866624309a7..7ce7b430ef610 100644 --- a/docs/reference/ingest/processors/geoip.asciidoc +++ b/docs/reference/ingest/processors/geoip.asciidoc @@ -10,7 +10,7 @@ under the CCA-ShareAlike 4.0 license. For more details see, http://dev.maxmind.c The `geoip` processor can run with other GeoIP2 databases from Maxmind. The files must be copied into the `ingest-geoip` config directory, and the `database_file` option should be used to specify the filename of the custom database. Custom database files must be stored -uncompressed. The `ingest-geoip` config directory is located at `$ES_HOME/config/ingest-geoip`. +uncompressed. The `ingest-geoip` config directory is located at `$ES_CONFIG/ingest-geoip`. [[using-ingest-geoip]] ==== Using the `geoip` Processor in a Pipeline diff --git a/docs/reference/ingest/processors/set.asciidoc b/docs/reference/ingest/processors/set.asciidoc index c964182ef6573..499e49c29b482 100644 --- a/docs/reference/ingest/processors/set.asciidoc +++ b/docs/reference/ingest/processors/set.asciidoc @@ -17,7 +17,7 @@ include::common-options.asciidoc[] [source,js] -------------------------------------------------- { - "description" : "sets the value of count to 1" + "description" : "sets the value of count to 1", "set": { "field": "count", "value": 1 diff --git a/docs/reference/intro.asciidoc b/docs/reference/intro.asciidoc new file mode 100644 index 0000000000000..c16f1bd78a1d5 --- /dev/null +++ b/docs/reference/intro.asciidoc @@ -0,0 +1,270 @@ +[[elasticsearch-intro]] += Elasticsearch introduction +[partintro] +-- +_**You know, for search (and analysis)**_ + +{es} is the distributed search and analytics engine at the heart of +the {stack}. {ls} and {beats} facilitate collecting, aggregating, and +enriching your data and storing it in {es}. {kib} enables you to +interactively explore, visualize, and share insights into your data and manage +and monitor the stack. {es} is where the indexing, search, and analysis +magic happen. + +{es} provides real-time search and analytics for all types of data. Whether you +have structured or unstructured text, numerical data, or geospatial data, +{es} can efficiently store and index it in a way that supports fast searches. +You can go far beyond simple data retrieval and aggregate information to discover +trends and patterns in your data. And as your data and query volume grows, the +distributed nature of {es} enables your deployment to grow seamlessly right +along with it. + +While not _every_ problem is a search problem, {es} offers speed and flexibility +to handle data in a wide variety of use cases: + +* Add a search box to an app or website +* Store and analyze logs, metrics, and security event data +* Use machine learning to automatically model the behavior of your data in real + time +* Automate business workflows using {es} as a storage engine +* Manage, integrate, and analyze spatial information using {es} as a geographic + information system (GIS) +* Store and process genetic data using {es} as a bioinformatics research tool + +We’re continually amazed by the novel ways people use search. But whether +your use case is similar to one of these, or you're using {es} to tackle a new +problem, the way you work with your data, documents, and indices in {es} is +the same. +-- + +[[documents-indices]] +== Data in: documents and indices + +{es} is a distributed document store. Instead of storing information as rows of +columnar data, {es} stores complex data structures that have been serialized +as JSON documents. When you have multiple {es} nodes in a cluster, stored +documents are distributed across the cluster and can be accessed immediately +from any node. + +When a document is stored, it is indexed and fully searchable in near +real-time--within 1 second. {es} uses a data structure called an +inverted index that supports very fast full-text searches. An inverted index +lists every unique word that appears in any document and identifies all of the +documents each word occurs in. + +An index can be thought of as an optimized collection of documents and each +document is a collection of fields, which are the key-value pairs that contain +your data. By default, {es} indexes all data in every field and each indexed +field has a dedicated, optimized data structure. For example, text fields are +stored in inverted indices, and numeric and geo fields are stored in BKD trees. +The ability to use the per-field data structures to assemble and return search +results is what makes {es} so fast. + +{es} also has the ability to be schema-less, which means that documents can be +indexed without explicitly specifying how to handle each of the different fields +that might occur in a document. When dynamic mapping is enabled, {es} +automatically detects and adds new fields to the index. This default +behavior makes it easy to index and explore your data--just start +indexing documents and {es} will detect and map booleans, floating point and +integer values, dates, and strings to the appropriate {es} datatypes. + +Ultimately, however, you know more about your data and how you want to use it +than {es} can. You can define rules to control dynamic mapping and explicitly +define mappings to take full control of how fields are stored and indexed. + +Defining your own mappings enables you to: + +* Distinguish between full-text string fields and exact value string fields +* Perform language-specific text analysis +* Optimize fields for partial matching +* Use custom date formats +* Use data types such as `geo_point` and `geo_shape` that cannot be automatically +detected + +It’s often useful to index the same field in different ways for different +purposes. For example, you might want to index a string field as both a text +field for full-text search and as a keyword field for sorting or aggregating +your data. Or, you might choose to use more than one language analyzer to +process the contents of a string field that contains user input. + +The analysis chain that is applied to a full-text field during indexing is also +used at search time. When you query a full-text field, the query text undergoes +the same analysis before the terms are looked up in the index. + +[[search-analyze]] +== Information out: search and analyze + +While you can use {es} as a document store and retrieve documents and their +metadata, the real power comes from being able to easily access the full suite +of search capabilities built on the Apache Lucene search engine library. + +{es} provides a simple, coherent REST API for managing your cluster and indexing +and searching your data. For testing purposes, you can easily submit requests +directly from the command line or through the Developer Console in {kib}. From +your applications, you can use the +https://www.elastic.co/guide/en/elasticsearch/client/index.html[{es} client] +for your language of choice: Java, JavaScript, Go, .NET, PHP, Perl, Python +or Ruby. + +[float] +[[search-data]] +=== Searching your data + +The {es} REST APIs support structured queries, full text queries, and complex +queries that combine the two. Structured queries are +similar to the types of queries you can construct in SQL. For example, you +could search the `gender` and `age` fields in your `employee` index and sort the +matches by the `hire_date` field. Full-text queries find all documents that +match the query string and return them sorted by _relevance_—how good a +match they are for your search terms. + +In addition to searching for individual terms, you can perform phrase searches, +similarity searches, and prefix searches, and get autocomplete suggestions. + +Have geospatial or other numerical data that you want to search? {es} indexes +non-textual data in optimized data structures that support +high-performance geo and numerical queries. + +You can access all of these search capabilities using {es}'s +comprehensive JSON-style query language (<>). You can also +construct <> to search and aggregate data +natively inside {es}, and JDBC and ODBC drivers enable a broad range of +third-party applications to interact with {es} via SQL. + +[float] +[[analyze-data]] +=== Analyzing your data + +{es} aggregations enable you to build complex summaries of your data and gain +insight into key metrics, patterns, and trends. Instead of just finding the +proverbial “needle in a haystack”, aggregations enable you to answer questions +like: + +* How many needles are in the haystack? +* What is the average length of the needles? +* What is the median length of the needles, broken down by manufacturer? +* How many needles were added to the haystack in each of the last six months? + +You can also use aggregations to answer more subtle questions, such as: + +* What are your most popular needle manufacturers? +* Are there any unusual or anomalous clumps of needles? + +Because aggregations leverage the same data-structures used for search, they are +also very fast. This enables you to analyze and visualize your data in real time. +Your reports and dashboards update as your data changes so you can take action +based on the latest information. + +What’s more, aggregations operate alongside search requests. You can search +documents, filter results, and perform analytics at the same time, on the same +data, in a single request. And because aggregations are calculated in the +context of a particular search, you’re not just displaying a count of all +size 70 needles, you’re displaying a count of the size 70 needles +that match your users' search criteria--for example, all size 70 _non-stick +embroidery_ needles. + +[float] +[[more-features]] +==== But wait, there’s more + +Want to automate the analysis of your time-series data? You can use +{stack-ov}/ml-overview.html[machine learning] features to create accurate +baselines of normal behavior in your data and identify anomalous patterns. With +machine learning, you can detect: + +* Anomalies related to temporal deviations in values, counts, or frequencies +* Statistical rarity +* Unusual behaviors for a member of a population + +And the best part? You can do this without having to specify algorithms, models, +or other data science-related configurations. + +[[scalability]] +== Scalability and resilience: clusters, nodes, and shards +++++ +Scalability and resilience +++++ + +{es} is built to be always available and to scale with your needs. It does this +by being distributed by nature. You can add servers (nodes) to a cluster to +increase capacity and {es} automatically distributes your data and query load +across all of the available nodes. No need to overhaul your application, {es} +knows how to balance multi-node clusters to provide scale and high availability. +The more nodes, the merrier. + +How does this work? Under the covers, an {es} index is really just a logical +grouping of one or more physical shards, where each shard is actually a +self-contained index. By distributing the documents in an index across multiple +shards, and distributing those shards across multiple nodes, {es} can ensure +redundancy, which both protects against hardware failures and increases +query capacity as nodes are added to a cluster. As the cluster grows (or shrinks), +{es} automatically migrates shards to rebalance the cluster. + +There are two types of shards: primaries and replicas. Each document in an index +belongs to one primary shard. A replica shard is a copy of a primary shard. +Replicas provide redundant copies of your data to protect against hardware +failure and increase capacity to serve read requests +like searching or retrieving a document. + +The number of primary shards in an index is fixed at the time that an index is +created, but the number of replica shards can be changed at any time, without +interrupting indexing or query operations. + +[float] +[[it-depends]] +=== It depends... + +There are a number of performance considerations and trade offs with respect +to shard size and the number of primary shards configured for an index. The more +shards, the more overhead there is simply in maintaining those indices. The +larger the shard size, the longer it takes to move shards around when {es} +needs to rebalance a cluster. + +Querying lots of small shards makes the processing per shard faster, but more +queries means more overhead, so querying a smaller +number of larger shards might be faster. In short...it depends. + +As a starting point: + +* Aim to keep the average shard size between a few GB and a few tens of GB. For + use cases with time-based data, it is common to see shards in the 20GB to 40GB + range. + +* Avoid the gazillion shards problem. The number of shards a node can hold is + proportional to the available heap space. As a general rule, the number of + shards per GB of heap space should be less than 20. + +The best way to determine the optimal configuration for your use case is +through https://www.elastic.co/elasticon/conf/2016/sf/quantitative-cluster-sizing[ +testing with your own data and queries]. + +[float] +[[disaster-ccr]] +=== In case of disaster + +For performance reasons, the nodes within a cluster need to be on the same +network. Balancing shards in a cluster across nodes in different data centers +simply takes too long. But high-availability architectures demand that you avoid +putting all of your eggs in one basket. In the event of a major outage in one +location, servers in another location need to be able to take over. Seamlessly. +The answer? {ccr-cap} (CCR). + +CCR provides a way to automatically synchronize indices from your primary cluster +to a secondary remote cluster that can serve as a hot backup. If the primary +cluster fails, the secondary cluster can take over. You can also use CCR to +create secondary clusters to serve read requests in geo-proximity to your users. + +{ccr-cap} is active-passive. The index on the primary cluster is +the active leader index and handles all write requests. Indices replicated to +secondary clusters are read-only followers. + +[float] +[[admin]] +=== Care and feeding + +As with any enterprise system, you need tools to secure, manage, and +monitor your {es} clusters. Security, monitoring, and administrative features +that are integrated into {es} enable you to use {kibana-ref}/introduction.html[{kib}] +as a control center for managing a cluster. Features like <> and <> +help you intelligently manage your data over time. diff --git a/docs/reference/mapping.asciidoc b/docs/reference/mapping.asciidoc index 2e09a0a8ca24a..d0a3c6e06cd66 100644 --- a/docs/reference/mapping.asciidoc +++ b/docs/reference/mapping.asciidoc @@ -87,15 +87,11 @@ causing a mapping explosion: `2`, etc. The default is `20`. `index.mapping.nested_fields.limit`:: - The maximum number of `nested` fields in an index, defaults to `50`. - Indexing 1 document with 100 nested fields actually indexes 101 documents - as each nested document is indexed as a separate hidden document. + The maximum number of distinct `nested` mappings in an index, defaults to `50`. `index.mapping.nested_objects.limit`:: - The maximum number of `nested` json objects within a single document across - all nested fields, defaults to 10000. Indexing one document with an array of - 100 objects within a nested field, will actually create 101 documents, as - each nested object will be indexed as a separate hidden document. + The maximum number of `nested` JSON objects within a single document across + all nested types, defaults to 10000. `index.mapping.field_name_length.limit`:: Setting for the maximum length of a field name. The default value is diff --git a/docs/reference/mapping/params/coerce.asciidoc b/docs/reference/mapping/params/coerce.asciidoc index 55f31262351fd..be5b2a648c600 100644 --- a/docs/reference/mapping/params/coerce.asciidoc +++ b/docs/reference/mapping/params/coerce.asciidoc @@ -47,8 +47,7 @@ PUT my_index/_doc/2 <1> The `number_one` field will contain the integer `10`. <2> This document will be rejected because coercion is disabled. -TIP: The `coerce` setting is allowed to have different settings for fields of -the same name in the same index. Its value can be updated on existing fields +TIP: The `coerce` setting value can be updated on existing fields using the <>. [[coerce-setting]] diff --git a/docs/reference/mapping/params/enabled.asciidoc b/docs/reference/mapping/params/enabled.asciidoc index 7193c6aa9f6e3..edbfb1f77d71e 100644 --- a/docs/reference/mapping/params/enabled.asciidoc +++ b/docs/reference/mapping/params/enabled.asciidoc @@ -89,8 +89,8 @@ GET my_index/_mapping <3> <2> The document can be retrieved. <3> Checking the mapping reveals that no fields have been added. -TIP: The `enabled` setting can be updated on existing fields -using the <>. +The `enabled` setting for existing fields and the top-level mapping +definition cannot be updated. Note that because Elasticsearch completely skips parsing the field contents, it is possible to add non-object data to a disabled field: diff --git a/docs/reference/mapping/params/ignore-above.asciidoc b/docs/reference/mapping/params/ignore-above.asciidoc index daf5c92bcf34d..33c0eaf339f24 100644 --- a/docs/reference/mapping/params/ignore-above.asciidoc +++ b/docs/reference/mapping/params/ignore-above.asciidoc @@ -30,7 +30,7 @@ PUT my_index/_doc/2 <3> "message": "Syntax error with some long stacktrace" } -GET _search <4> +GET my_index/_search <4> { "aggs": { "messages": { diff --git a/docs/reference/mapping/params/ignore-malformed.asciidoc b/docs/reference/mapping/params/ignore-malformed.asciidoc index 8c91bb48ee7cd..d84a7290eb7c9 100644 --- a/docs/reference/mapping/params/ignore-malformed.asciidoc +++ b/docs/reference/mapping/params/ignore-malformed.asciidoc @@ -46,8 +46,7 @@ PUT my_index/_doc/2 <1> This document will have the `text` field indexed, but not the `number_one` field. <2> This document will be rejected because `number_two` does not allow malformed values. -TIP: The `ignore_malformed` setting is allowed to have different settings for -fields of the same name in the same index. Its value can be updated on +TIP: The `ignore_malformed` setting value can be updated on existing fields using the <>. diff --git a/docs/reference/mapping/params/multi-fields.asciidoc b/docs/reference/mapping/params/multi-fields.asciidoc index ee1bc02c7fd8d..448f7fd2e81ec 100644 --- a/docs/reference/mapping/params/multi-fields.asciidoc +++ b/docs/reference/mapping/params/multi-fields.asciidoc @@ -60,8 +60,7 @@ GET my_index/_search NOTE: Multi-fields do not change the original `_source` field. -TIP: The `fields` setting is allowed to have different settings for fields of -the same name in the same index. New multi-fields can be added to existing +TIP: New multi-fields can be added to existing fields using the <>. ==== Multi-fields with multiple analyzers diff --git a/docs/reference/mapping/params/normalizer.asciidoc b/docs/reference/mapping/params/normalizer.asciidoc index da0298abda228..0f8c09552f4c6 100644 --- a/docs/reference/mapping/params/normalizer.asciidoc +++ b/docs/reference/mapping/params/normalizer.asciidoc @@ -7,7 +7,7 @@ produces a single token. The `normalizer` is applied prior to indexing the keyword, as well as at search-time when the `keyword` field is searched via a query parser such as -the <> query or via a term level query +the <> query or via a term-level query such as the <> query. [source,js] diff --git a/docs/reference/mapping/params/norms.asciidoc b/docs/reference/mapping/params/norms.asciidoc index 8a7be4baef8c5..6a250d296a2dc 100644 --- a/docs/reference/mapping/params/norms.asciidoc +++ b/docs/reference/mapping/params/norms.asciidoc @@ -11,11 +11,10 @@ don't need scoring on a specific field, you should disable norms on that field. In particular, this is the case for fields that are used solely for filtering or aggregations. -TIP: The `norms` setting must have the same setting for fields of the -same name in the same index. Norms can be disabled on existing fields using +TIP: Norms can be disabled on existing fields using the <>. -Norms can be disabled (but not reenabled) after the fact, using the +Norms can be disabled (but not reenabled after the fact), using the <> like so: [source,js] diff --git a/docs/reference/mapping/removal_of_types.asciidoc b/docs/reference/mapping/removal_of_types.asciidoc index 47952d61c4954..4aa178d443f55 100644 --- a/docs/reference/mapping/removal_of_types.asciidoc +++ b/docs/reference/mapping/removal_of_types.asciidoc @@ -258,7 +258,7 @@ Elasticsearch 6.x:: * The `_default_` mapping type is deprecated. -* In 6.7, the index creation, index template, and mapping APIs support a query +* In 6.8, the index creation, index template, and mapping APIs support a query string parameter (`include_type_name`) which indicates whether requests and responses should include a type name. It defaults to `true`, and should be set to an explicit value to prepare to upgrade to 7.0. Not setting `include_type_name` @@ -442,12 +442,12 @@ documents to it using typeless `index` calls, and load documents with typeless Index creation, index template, and mapping APIs support a new `include_type_name` URL parameter that specifies whether mapping definitions in requests and responses -should contain the type name. The parameter defaults to `true` in version 6.7 to +should contain the type name. The parameter defaults to `true` in version 6.8 to match the pre-7.0 behavior of using type names in mappings. It defaults to `false` in version 7.0 and will be removed in version 8.0. -It should be set explicitly in 6.7 to prepare to upgrade to 7.0. To avoid deprecation -warnings in 6.7, the parameter can be set to either `true` or `false`. In 7.0, setting +It should be set explicitly in 6.8 to prepare to upgrade to 7.0. To avoid deprecation +warnings in 6.8, the parameter can be set to either `true` or `false`. In 7.0, setting `include_type_name` at all will result in a deprecation warning. See some examples of interactions with Elasticsearch with this option set to `false`: @@ -717,12 +717,12 @@ indices. [float] ==== Mixed-version clusters -In a cluster composed of both 6.7 and 7.0 nodes, the parameter +In a cluster composed of both 6.8 and 7.0 nodes, the parameter `include_type_name` should be specified in indices APIs like index creation. This is because the parameter has a different default between -6.7 and 7.0, so the same mapping definition will not be valid for both +6.8 and 7.0, so the same mapping definition will not be valid for both node versions. Typeless document APIs such as `bulk` and `update` are only available as of -7.0, and will not work with 6.7 nodes. This also holds true for the typeless +7.0, and will not work with 6.8 nodes. This also holds true for the typeless versions of queries that perform document lookups, such as `terms`. diff --git a/docs/reference/mapping/types.asciidoc b/docs/reference/mapping/types.asciidoc index c0db156dc3a1c..bdfcf1128a061 100644 --- a/docs/reference/mapping/types.asciidoc +++ b/docs/reference/mapping/types.asciidoc @@ -42,8 +42,6 @@ string:: <> and <> <>:: Defines parent/child relation for documents within the same index -<>:: Defines an alias to an existing field. - <>:: Record numeric feature to boost hits at query time. <>:: Record numeric features to boost hits at query time. @@ -54,6 +52,11 @@ string:: <> and <> <>:: A text-like field optimized for queries to implement as-you-type completion +<>:: Defines an alias to an existing field. + +<>:: Allows an entire JSON object to be indexed as a single field. + + [float] === Multi-fields @@ -82,6 +85,8 @@ include::types/date.asciidoc[] include::types/date_nanos.asciidoc[] +include::types/flattened.asciidoc[] + include::types/geo-point.asciidoc[] include::types/geo-shape.asciidoc[] diff --git a/docs/reference/mapping/types/alias.asciidoc b/docs/reference/mapping/types/alias.asciidoc index a7bba54df56e4..c70d96a7e572d 100644 --- a/docs/reference/mapping/types/alias.asciidoc +++ b/docs/reference/mapping/types/alias.asciidoc @@ -16,7 +16,7 @@ PUT trips }, "route_length_miles": { "type": "alias", - "path": "distance" // <1> + "path": "distance" <1> }, "transit_mode": { "type": "keyword" diff --git a/docs/reference/mapping/types/dense-vector.asciidoc b/docs/reference/mapping/types/dense-vector.asciidoc index 335c8f16ba9f1..7ea04b42330c2 100644 --- a/docs/reference/mapping/types/dense-vector.asciidoc +++ b/docs/reference/mapping/types/dense-vector.asciidoc @@ -1,3 +1,5 @@ +[role="xpack"] +[testenv="basic"] [[dense-vector]] === Dense vector datatype @@ -5,9 +7,7 @@ experimental[] A `dense_vector` field stores dense vectors of float values. The maximum number of dimensions that can be in a vector should -not exceed 1024. The number of dimensions can be -different across documents. A `dense_vector` field is -a single-valued field. +not exceed 1024. A `dense_vector` field is a single-valued field. These vectors can be used for <>. For example, a document score can represent a distance between @@ -22,7 +22,8 @@ PUT my_index "mappings": { "properties": { "my_vector": { - "type": "dense_vector" + "type": "dense_vector", + "dims": 3 <1> }, "my_text" : { "type" : "keyword" @@ -40,13 +41,14 @@ PUT my_index/_doc/1 PUT my_index/_doc/2 { "my_text" : "text2", - "my_vector" : [-0.5, 10, 10, 4] + "my_vector" : [-0.5, 10, 10] } -------------------------------------------------- // CONSOLE +<1> dims—the number of dimensions in the vector, required parameter. + Internally, each document's dense vector is encoded as a binary doc value. Its size in bytes is equal to -`4 * NUMBER_OF_DIMENSIONS`, where `NUMBER_OF_DIMENSIONS` - -number of the vector's dimensions. \ No newline at end of file +`4 * dims`, where `dims`—the number of the vector's dimensions. \ No newline at end of file diff --git a/docs/reference/mapping/types/flattened.asciidoc b/docs/reference/mapping/types/flattened.asciidoc new file mode 100644 index 0000000000000..80fd72c3dcc1f --- /dev/null +++ b/docs/reference/mapping/types/flattened.asciidoc @@ -0,0 +1,188 @@ +[role="xpack"] +[testenv="basic"] + +[[flattened]] +=== Flattened datatype + +By default, each subfield in an object is mapped and indexed separately. If +the names or types of the subfields are not known in advance, then they are +<>. + +The `flattened` type provides an alternative approach, where the entire +object is mapped as a single field. Given an object, the `flattened` +mapping will parse out its leaf values and index them into one field as +keywords. The object's contents can then be searched through simple queries +and aggregations. + +This data type can be useful for indexing objects with a large or unknown +number of unique keys. Only one field mapping is created for the whole JSON +object, which can help prevent a <> +from having too many distinct field mappings. + +On the other hand, flattened object fields present a trade-off in terms of +search functionality. Only basic queries are allowed, with no support for +numeric range queries or highlighting. Further information on the limitations +can be found in the <> section. + +NOTE: The `flattened` mapping type should **not** be used for indexing all +document content, as it treats all values as keywords and does not provide full +search functionality. The default approach, where each subfield has its own +entry in the mappings, works well in the majority of cases. + +An flattened object field can be created as follows: +[source,js] +-------------------------------- +PUT bug_reports +{ + "mappings": { + "properties": { + "title": { + "type": "text" + }, + "labels": { + "type": "flattened" + } + } + } +} + +POST bug_reports/_doc/1 +{ + "title": "Results are not sorted correctly.", + "labels": { + "priority": "urgent", + "release": ["v1.2.5", "v1.3.0"], + "timestamp": { + "created": 1541458026, + "closed": 1541457010 + } + } +} +-------------------------------- +// CONSOLE +// TESTSETUP + +During indexing, tokens are created for each leaf value in the JSON object. The +values are indexed as string keywords, without analysis or special handling for +numbers or dates. + +Querying the top-level `flattened` field searches all leaf values in the +object: + +[source,js] +-------------------------------- +POST bug_reports/_search +{ + "query": { + "term": {"labels": "urgent"} + } +} +-------------------------------- +// CONSOLE + +To query on a specific key in the flattened object, object dot notation is used: +[source,js] +-------------------------------- +POST bug_reports/_search +{ + "query": { + "term": {"labels.release": "v1.3.0"} + } +} +-------------------------------- +// CONSOLE + +[[supported-operations]] +==== Supported operations + +Because of the similarities in the way values are indexed, `flattened` +fields share much of the same mapping and search functionality as +<> fields. + +Currently, flattened object fields can be used with the following query types: + +- `term`, `terms`, and `terms_set` +- `prefix` +- `range` +- `match` and `multi_match` +- `query_string` and `simple_query_string` +- `exists` + +When querying, it is not possible to refer to field keys using wildcards, as in +`{ "term": {"labels.time*": 1541457010}}`. Note that all queries, including +`range`, treat the values as string keywords. Highlighting is not supported on +`flattened` fields. + +It is possible to sort on an flattened object field, as well as perform simple +keyword-style aggregations such as `terms`. As with queries, there is no +special support for numerics -- all values in the JSON object are treated as +keywords. When sorting, this implies that values are compared +lexicographically. + +Flattened object fields currently cannot be stored. It is not possible to +specify the <> parameter in the mapping. + +[[flattened-params]] +==== Parameters for flattened object fields + +The following mapping parameters are accepted: + +[horizontal] + +<>:: + + Mapping field-level query time boosting. Accepts a floating point number, + defaults to `1.0`. + +`depth_limit`:: + + The maximum allowed depth of the flattened object field, in terms of nested + inner objects. If a flattened object field exceeds this limit, then an + error will be thrown. Defaults to `20`. + +<>:: + + Should the field be stored on disk in a column-stride fashion, so that it + can later be used for sorting, aggregations, or scripting? Accepts `true` + (default) or `false`. + +<>:: + + Should global ordinals be loaded eagerly on refresh? Accepts `true` or + `false` (default). Enabling this is a good idea on fields that are + frequently used for terms aggregations. + +<>:: + + Leaf values longer than this limit will not be indexed. By default, there + is no limit and all values will be indexed. Note that this limit applies + to the leaf values within the flattened object field, and not the length of + the entire field. + +<>:: + + Determines if the field should be searchable. Accepts `true` (default) or + `false`. + +<>:: + + What information should be stored in the index for scoring purposes. + Defaults to `docs` but can also be set to `freqs` to take term frequency + into account when computing scores. + +<>:: + + A string value which is substituted for any explicit `null` values within + the flattened object field. Defaults to `null`, which means null sields are + treated as if it were missing. + +<>:: + + Which scoring algorithm or _similarity_ should be used. Defaults + to `BM25`. + +`split_queries_on_whitespace`:: + + Whether <> should split the input on + whitespace when building a query for this field. Accepts `true` or `false` + (default). diff --git a/docs/reference/mapping/types/geo-shape.asciidoc b/docs/reference/mapping/types/geo-shape.asciidoc index 26f59e1058c09..be5308db41a3d 100644 --- a/docs/reference/mapping/types/geo-shape.asciidoc +++ b/docs/reference/mapping/types/geo-shape.asciidoc @@ -108,6 +108,8 @@ geo-points containing any more than latitude and longitude (two dimensions) valu and reject the whole document. | `true` +|`coerce` |If `true` unclosed linear rings in polygons will be automatically closed. +| `false` |======================================================================= diff --git a/docs/reference/mapping/types/nested.asciidoc b/docs/reference/mapping/types/nested.asciidoc index 5969dcfd6956b..63bb4591369e5 100644 --- a/docs/reference/mapping/types/nested.asciidoc +++ b/docs/reference/mapping/types/nested.asciidoc @@ -159,6 +159,22 @@ Nested documents can be: * sorted with <>. * retrieved and highlighted with <>. +[IMPORTANT] +============================================= + +Because nested documents are indexed as separate documents, they can only be +accessed within the scope of the `nested` query, the +`nested`/`reverse_nested` aggregations, or <>. + +For instance, if a string field within a nested document has +<> set to `offsets` to allow use of the postings +during the highlighting, these offsets will not be available during the main highlighting +phase. Instead, highlighting needs to be performed via +<>. The same consideration applies when loading +fields during a search through <> +or <>. + +============================================= [[nested-params]] ==== Parameters for `nested` fields @@ -178,35 +194,32 @@ The following parameters are accepted by `nested` fields: may be added to an existing nested object. -[IMPORTANT] -============================================= +[float] +=== Limits on `nested` mappings and objects -Because nested documents are indexed as separate documents, they can only be -accessed within the scope of the `nested` query, the -`nested`/`reverse_nested` aggregations, or <>. +As described earlier, each nested object is indexed as a separate document under the hood. +Continuing with the example above, if we indexed a single document containing 100 `user` objects, +then 101 Lucene documents would be created -- one for the parent document, and one for each +nested object. Because of the expense associated with `nested` mappings, Elasticsearch puts +settings in place to guard against performance problems: -For instance, if a string field within a nested document has -<> set to `offsets` to allow use of the postings -during the highlighting, these offsets will not be available during the main highlighting -phase. Instead, highlighting needs to be performed via -<>. +`index.mapping.nested_fields.limit`:: -============================================= + The `nested` type should only be used in special cases, when arrays of objects need to be + queried independently of each other. To safeguard against poorly designed mappings, this setting + limits the number of unique `nested` types per index. In our example, the `user` mapping would + count as only 1 towards this limit. Defaults to 50. +`index.mapping.nested_objects.limit`:: -==== Limiting the number of `nested` fields + This setting limits the number of nested objects that a single document may contain across all + `nested` types, in order to prevent out of memory errors when a document contains too many nested + objects. To illustrate how the setting works, say we added another `nested` type called `comments` + to our example mapping above. Then for each document, the combined number of `user` and `comment` + objects it contains must be below the limit. Defaults to 10000. -Indexing a document with 100 nested fields actually indexes 101 documents as each nested -document is indexed as a separate document. To safeguard against ill-defined mappings -the number of nested fields that can be defined per index has been limited to 50. See -<>. +Additional background on these settings, including information on their default values, can be found +in <>. -[[limit-nested-json-objects-number]] -==== Limiting the number of `nested` json objects -Indexing a document with an array of 100 objects within a nested field, will actually -create 101 documents, as each nested object will be indexed as a separate document. -To prevent out of memory errors when a single document contains too many nested json -objects, the number of nested json objects that a single document may contain across all fields -has been limited to 10000. See <>. diff --git a/docs/reference/mapping/types/sparse-vector.asciidoc b/docs/reference/mapping/types/sparse-vector.asciidoc index 70b2ce4ed3198..de63a1a822a0a 100644 --- a/docs/reference/mapping/types/sparse-vector.asciidoc +++ b/docs/reference/mapping/types/sparse-vector.asciidoc @@ -1,3 +1,5 @@ +[role="xpack"] +[testenv="basic"] [[sparse-vector]] === Sparse vector datatype diff --git a/docs/reference/migration/migrate_8_0.asciidoc b/docs/reference/migration/migrate_8_0.asciidoc index b697abf9a9f25..ff3f5030ed9fb 100644 --- a/docs/reference/migration/migrate_8_0.asciidoc +++ b/docs/reference/migration/migrate_8_0.asciidoc @@ -15,6 +15,7 @@ coming[8.0.0] * <> * <> * <> +* <> * <> * <> * <> @@ -24,6 +25,7 @@ coming[8.0.0] * <> * <> * <> +* <> * <> //NOTE: The notable-breaking-changes tagged regions are re-used in the @@ -51,6 +53,7 @@ include::migrate_8_0/analysis.asciidoc[] include::migrate_8_0/discovery.asciidoc[] include::migrate_8_0/mappings.asciidoc[] include::migrate_8_0/packaging.asciidoc[] +include::migrate_8_0/rollup.asciidoc[] include::migrate_8_0/snapshots.asciidoc[] include::migrate_8_0/security.asciidoc[] include::migrate_8_0/ilm.asciidoc[] @@ -60,4 +63,5 @@ include::migrate_8_0/node.asciidoc[] include::migrate_8_0/transport.asciidoc[] include::migrate_8_0/http.asciidoc[] include::migrate_8_0/reindex.asciidoc[] +include::migrate_8_0/search.asciidoc[] include::migrate_8_0/settings.asciidoc[] diff --git a/docs/reference/migration/migrate_8_0/mappings.asciidoc b/docs/reference/migration/migrate_8_0/mappings.asciidoc index 371e9fc44c415..16e75473885c6 100644 --- a/docs/reference/migration/migrate_8_0/mappings.asciidoc +++ b/docs/reference/migration/migrate_8_0/mappings.asciidoc @@ -14,3 +14,12 @@ The number of completion contexts within a single completion field has been limited to 10. + +[float] +==== Defining multi-fields within multi-fields + +Previously, it was possible to define a multi-field within a multi-field. +Defining chained multi-fields was deprecated in 7.3 and is now no longer +supported. To migrate the mappings, all instances of `fields` that occur within +a `fields` block should be removed, either by flattening the chained `fields` +blocks into a single level, or by switching to `copy_to` if appropriate. \ No newline at end of file diff --git a/docs/reference/migration/migrate_8_0/node.asciidoc b/docs/reference/migration/migrate_8_0/node.asciidoc index a1dcd654807e1..b1187e88b5d90 100644 --- a/docs/reference/migration/migrate_8_0/node.asciidoc +++ b/docs/reference/migration/migrate_8_0/node.asciidoc @@ -14,3 +14,25 @@ The `node.max_local_storage_nodes` setting was deprecated in 7.x and has been removed in 8.0. Nodes should be run on separate data paths to ensure that each node is consistently assigned to the same data path. + +[float] +==== Change of data folder layout + +Each node's data is now stored directly in the data directory set by the +`path.data` setting, rather than in `${path.data}/nodes/0`, because the removal +of the `node.max_local_storage_nodes` setting means that nodes may no longer +share a data path. At startup, Elasticsearch will automatically migrate the data +path to the new layout. This automatic migration will not proceed if the data +path contains data for more than one node. You should move to a configuration in +which each node has its own data path before upgrading. + +If you try to upgrade a configuration in which there is data for more than one +node in a data path then the automatic migration will fail and Elasticsearch +will refuse to start. To resolve this you will need to perform the migration +manually. The data for the extra nodes are stored in folders named +`${path.data}/nodes/1`, `${path.data}/nodes/2` and so on, and you should move +each of these folders to an appropriate location and then configure the +corresponding node to use this location for its data path. If your nodes each +have more than one data path in their `path.data` settings then you should move +all the corresponding subfolders in parallel. Each node uses the same subfolder +(e.g. `nodes/2`) across all its data paths. \ No newline at end of file diff --git a/docs/reference/migration/migrate_8_0/reindex.asciidoc b/docs/reference/migration/migrate_8_0/reindex.asciidoc index ef4f5aed147ca..912f0f9dbf11b 100644 --- a/docs/reference/migration/migrate_8_0/reindex.asciidoc +++ b/docs/reference/migration/migrate_8_0/reindex.asciidoc @@ -12,4 +12,18 @@ Instead, please specify the index-name without any encoding. [float] ==== Removal of types -The `/{index}/{type}/_delete_by_query` and `/{index}/{type}/_update_by_query` REST endpoints have been removed in favour of `/{index}/_delete_by_query` and `/{index}/_update_by_query`, since indexes no longer contain types, these typed endpoints are obsolete. \ No newline at end of file +The `/{index}/{type}/_delete_by_query` and `/{index}/{type}/_update_by_query` REST endpoints have been removed in favour of `/{index}/_delete_by_query` and `/{index}/_update_by_query`, since indexes no longer contain types, these typed endpoints are obsolete. + +[float] +==== Removal of size parameter + +Previously, a `_reindex` request had two different size specifications in the body: + +- Outer level, determining the maximum number of documents to process +- Inside the `source` element, determining the scroll/batch size. + +The outer level `size` parameter has now been renamed to `max_docs` to +avoid confusion and clarify its semantics. + +Similarly, the `size` parameter has been renamed to `max_docs` for +`_delete_by_query` and `_update_by_query` to keep the 3 interfaces consistent. \ No newline at end of file diff --git a/docs/reference/migration/migrate_8_0/rollup.asciidoc b/docs/reference/migration/migrate_8_0/rollup.asciidoc new file mode 100644 index 0000000000000..71c9b38d444d6 --- /dev/null +++ b/docs/reference/migration/migrate_8_0/rollup.asciidoc @@ -0,0 +1,20 @@ +[float] +[[breaking_80_rollup_changes]] +=== Rollup changes + +//NOTE: The notable-breaking-changes tagged regions are re-used in the +//Installation and Upgrade Guide + +//tag::notable-breaking-changes[] + +// end::notable-breaking-changes[] + +[float] +==== StartRollupJob endpoint returns success if job already started + +Previously, attempting to start an already-started rollup job would +result in a `500 InternalServerError: Cannot start task for Rollup Job +[job] because state was [STARTED]` exception. + +Now, attempting to start a job that is already started will just +return a successful `200 OK: started` response. \ No newline at end of file diff --git a/docs/reference/migration/migrate_8_0/search.asciidoc b/docs/reference/migration/migrate_8_0/search.asciidoc index 82886d35bc6a5..97796a10fca22 100644 --- a/docs/reference/migration/migrate_8_0/search.asciidoc +++ b/docs/reference/migration/migrate_8_0/search.asciidoc @@ -7,4 +7,21 @@ The `/{index}/{type}/_search`, `/{index}/{type}/_msearch`, `/{index}/{type}/_search/template` and `/{index}/{type}/_msearch/template` REST endpoints have been removed in favour of `/{index}/_search`, `/{index}/_msearch`, `/{index}/_search/template` and `/{index}/_msearch/template`, since indexes no longer contain types, these typed endpoints are obsolete.. -The `/{index}/{type}/_termvectors`, `/{index}/{type}/{id}/_termvectors` and `/{index}/{type}/_mtermvectors` REST endpoints have been removed in favour of `/{index}/_termvectors`, `/{index}/{id}/_termvectors` and `/{index}/_mtermvectors`, since indexes no longer contain types, these typed endpoints are obsolete.. \ No newline at end of file +The `/{index}/{type}/_termvectors`, `/{index}/{type}/{id}/_termvectors` and `/{index}/{type}/_mtermvectors` REST endpoints have been removed in favour of `/{index}/_termvectors`, `/{index}/{id}/_termvectors` and `/{index}/_mtermvectors`, since indexes no longer contain types, these typed endpoints are obsolete.. + +[float] +==== Removal of queries + +The `common` query was deprecated in 7.x and has been removed in 8.0. +The same functionality can be achieved by the `match` query if the total number of hits is not tracked. + +[float] +===== Removal of query parameters + +The `cutoff_frequency` parameter was deprecated in 7.x and has been removed in 8.0 from `match` and `multi_match` queries. +The same functionality can be achieved without any configuration provided that the total number of hits is not tracked. + +[float] +===== Removal of sort parameters + +The `nested_filter` and `nested_path` options, deprecated in 6.x, have been removed in favor of the `nested` context. diff --git a/docs/reference/migration/migrate_8_0/security.asciidoc b/docs/reference/migration/migrate_8_0/security.asciidoc index a7cacef8ff017..f3ee4fd97ef85 100644 --- a/docs/reference/migration/migrate_8_0/security.asciidoc +++ b/docs/reference/migration/migrate_8_0/security.asciidoc @@ -33,3 +33,11 @@ The `elasticsearch-migrate` tool provided a way to convert file realm users and roles into the native realm. It has been deprecated since 7.2.0. Users and roles should now be created in the native realm directly. + +[float] +[[separating-node-and-client-traffic]] +==== The `transport.profiles.*.xpack.security.type` setting has been removed + +The `transport.profiles.*.xpack.security.type` setting has been removed since +the Transport Client has been removed and therefore all client traffic now uses +the HTTP transport. Transport profiles using this setting should be removed. diff --git a/docs/reference/migration/migrate_8_0/snapshots.asciidoc b/docs/reference/migration/migrate_8_0/snapshots.asciidoc index 791e5b28da057..27a0afa92b71e 100644 --- a/docs/reference/migration/migrate_8_0/snapshots.asciidoc +++ b/docs/reference/migration/migrate_8_0/snapshots.asciidoc @@ -9,6 +9,59 @@ // end::notable-breaking-changes[] +[float] +=== Get snapshots response format is changed +It's possible to get snapshots from multiple repositories in one go. The response format has changed +and now contains separate response for each repository. + +For example, requesting one snapshot from particular repository + +[source,js] +----------------------------------- +GET _snapshot/repo1/snap1 +----------------------------------- +// CONSOLE +// TEST[skip:no repo and snapshots are created] + +produces the following response + +[source,js] +----------------------------------- +{ + "responses": [ + { + "repository": "repo1", + "snapshots": [ + { + "snapshot": "snap1", + "uuid": "cEzdqUKxQ5G6MyrJAcYwmA", + "version_id": 8000099, + "version": "8.0.0", + "indices": [], + "include_global_state": true, + "state": "SUCCESS", + "start_time": "2019-05-10T17:01:57.868Z", + "start_time_in_millis": 1557507717868, + "end_time": "2019-05-10T17:01:57.909Z", + "end_time_in_millis": 1557507717909, + "duration_in_millis": 41, + "failures": [], + "shards": { + "total": 0, + "failed": 0, + "successful": 0 + } + } + ] + } + ] +} +----------------------------------- +// TESTRESPONSE +// TEST[skip:no repo and snapshots are created] + +See <> for more information. + [float] ==== Deprecated node level compress setting removed diff --git a/docs/reference/ml/apis/close-job.asciidoc b/docs/reference/ml/apis/close-job.asciidoc index c5f9b5fc2444a..2a38648e48fe5 100644 --- a/docs/reference/ml/apis/close-job.asciidoc +++ b/docs/reference/ml/apis/close-job.asciidoc @@ -12,8 +12,8 @@ A job can be opened and closed multiple times throughout its lifecycle. A closed job cannot receive data or perform analysis operations, but you can still explore and navigate results. - -==== Request +[[ml-close-job-request]] +==== {api-request-title} `POST _ml/anomaly_detectors//_close` + @@ -21,8 +21,15 @@ operations, but you can still explore and navigate results. `POST _ml/anomaly_detectors/_all/_close` + +[[ml-close-job-prereqs]] +==== {api-prereq-title} + +* If the {es} {security-features} are enabled, you must have `manage_ml` or +`manage` cluster privileges to use this API. See +{stack-ov}/security-privileges.html[Security privileges]. -==== Description +[[ml-close-job-desc]] +==== {api-description-title} You can close multiple jobs in a single API request by using a group name, a comma-separated list of jobs, or a wildcard expression. You can close all jobs @@ -47,32 +54,26 @@ after the close job API returns. The `force` query parameter should only be use situations where the job has already failed, or where you are not interested in results the job might have recently produced or might produce in the future. +[[ml-close-job-path-parms]] +==== {api-path-parms-title} -==== Path Parameters - -`job_id`:: +`` (Required):: (string) Identifier for the job. It can be a job identifier, a group name, or a wildcard expression. +[[ml-close-job-query-parms]] +==== {api-query-parms-title} -==== Query Parameters - -`force`:: +`force` (Optional):: (boolean) Use to close a failed job, or to forcefully close a job which has not responded to its initial close request. -`timeout`:: +`timeout` (Optional):: (time units) Controls the time to wait until a job has closed. The default value is 30 minutes. - -==== Authorization - -You must have `manage_ml`, or `manage` cluster privileges to use this API. -For more information, see {xpack-ref}/security-privileges.html[Security Privileges]. - - -==== Examples +[[ml-close-job-example]] +==== {api-examples-title} The following example closes the `total-requests` job: diff --git a/docs/reference/ml/apis/datafeedresource.asciidoc b/docs/reference/ml/apis/datafeedresource.asciidoc index 33fce3dbf7c9d..5c1e3e74a6ae8 100644 --- a/docs/reference/ml/apis/datafeedresource.asciidoc +++ b/docs/reference/ml/apis/datafeedresource.asciidoc @@ -61,12 +61,12 @@ A {dfeed} resource has the following properties: `delayed_data_check_config`:: (object) Specifies whether the data feed checks for missing data and - and the size of the window. For example: + the size of the window. For example: `{"enabled": true, "check_window": "1h"}` See <>. [[ml-datafeed-chunking-config]] -==== Chunking Configuration Objects +==== Chunking configuration objects {dfeeds-cap} might be required to search over long time periods, for several months or years. This search is split into time chunks in order to ensure the load @@ -88,31 +88,33 @@ A chunking configuration object has the following properties: For example: `3h`. [[ml-datafeed-delayed-data-check-config]] -==== Delayed Data Check Configuration Objects +==== Delayed data check configuration objects The {dfeed} can optionally search over indices that have already been read in -an effort to find if any data has since been added to the index. If missing data -is found, it is a good indication that the `query_delay` option is set too low and -the data is being indexed after the {dfeed} has passed that moment in time. See +an effort to determine whether any data has subsequently been added to the index. +If missing data is found, it is a good indication that the `query_delay` option +is set too low and the data is being indexed after the {dfeed} has passed that +moment in time. See {stack-ov}/ml-delayed-data-detection.html[Working with delayed data]. -This check only runs on real-time {dfeeds} +This check runs only on real-time {dfeeds}. The configuration object has the following properties: `enabled`:: - (boolean) Should the {dfeed} periodically check for data being indexed after reading. - Defaults to `true` + (boolean) Specifies whether the {dfeed} periodically checks for delayed data. + Defaults to `true`. `check_window`:: - (time units) The window of time before the latest finalized bucket that should be searched - for late data. Defaults to `null` which causes an appropriate `check_window` to be calculated - when the real-time {dfeed} runs. - The default `check_window` span calculation is the max between `2h` or `8 * bucket_span`. + (time units) The window of time that is searched for late data. This window of + time ends with the latest finalized bucket. It defaults to `null`, which + causes an appropriate `check_window` to be calculated when the real-time + {dfeed} runs. In particular, the default `check_window` span calculation is + based on the maximum of `2h` or `8 * bucket_span`. [float] [[ml-datafeed-counts]] -==== {dfeed-cap} Counts +==== {dfeed-cap} counts The get {dfeed} statistics API provides information about the operational progress of a {dfeed}. All of these properties are informational; you cannot diff --git a/docs/reference/ml/apis/delete-calendar-event.asciidoc b/docs/reference/ml/apis/delete-calendar-event.asciidoc index 68f7a0738375d..0aa9ce5cc8d92 100644 --- a/docs/reference/ml/apis/delete-calendar-event.asciidoc +++ b/docs/reference/ml/apis/delete-calendar-event.asciidoc @@ -8,34 +8,37 @@ Deletes scheduled events from a calendar. - -==== Request +[[ml-delete-calendar-event-request]] +==== {api-request-title} `DELETE _ml/calendars//events/` +[[ml-delete-calendar-event-prereqs]] +==== {api-prereq-title} + +* If the {es} {security-features} are enabled, you must have `manage_ml` or +`manage` cluster privileges to use this API. See +{stack-ov}/security-privileges.html[Security privileges]. -==== Description +[[ml-delete-calendar-event-desc]] +==== {api-description-title} This API removes individual events from a calendar. To remove all scheduled events and delete the calendar, see the <>. -==== Path Parameters +[[ml-delete-calendar-event-path-parms]] +==== {api-path-parms-title} -`calendar_id`(required):: +`` (Required):: (string) Identifier for the calendar. -`event_id` (required):: +`` (Required):: (string) Identifier for the scheduled event. You can obtain this identifier by using the <>. - -==== Authorization - -You must have `manage_ml`, or `manage` cluster privileges to use this API. -For more information, see {xpack-ref}/security-privileges.html[Security Privileges]. - -==== Examples +[[ml-delete-calendar-event-example]] +==== {api-examples-title} The following example deletes a scheduled event from the `planned-outages` calendar: diff --git a/docs/reference/ml/apis/delete-calendar-job.asciidoc b/docs/reference/ml/apis/delete-calendar-job.asciidoc index 118a706d29460..a555b3d3b922b 100644 --- a/docs/reference/ml/apis/delete-calendar-job.asciidoc +++ b/docs/reference/ml/apis/delete-calendar-job.asciidoc @@ -8,28 +8,30 @@ Deletes jobs from a calendar. - -==== Request +[[ml-delete-calendar-job-request]] +==== {api-request-title} `DELETE _ml/calendars//jobs/` +[[ml-delete-calendar-job-prereqs]] +==== {api-prereq-title} -==== Path Parameters - -`calendar_id`(required):: - (string) Identifier for the calendar. +* If the {es} {security-features} are enabled, you must have `manage_ml` or +`manage` cluster privileges to use this API. See +{stack-ov}/security-privileges.html[Security privileges]. -`job_id` (required):: - (string) An identifier for the job. It can be a job identifier, a group name, or a - comma-separated list of jobs or groups. +[[ml-delete-calendar-job-path-parms]] +==== {api-path-parms-title} +`` (Required):: + (string) Identifier for the calendar. -==== Authorization - -You must have `manage_ml`, or `manage` cluster privileges to use this API. -For more information, see {xpack-ref}/security-privileges.html[Security Privileges]. +`` (Required):: + (string) An identifier for the job. It can be a job identifier, a group name, + or a comma-separated list of jobs or groups. -==== Examples +[[ml-delete-calendar-job-example]] +==== {api-examples-title} The following example removes the association between the `planned-outages` calendar and `total-requests` job: diff --git a/docs/reference/ml/apis/delete-calendar.asciidoc b/docs/reference/ml/apis/delete-calendar.asciidoc index 2707f3175e04c..065c117c49c63 100644 --- a/docs/reference/ml/apis/delete-calendar.asciidoc +++ b/docs/reference/ml/apis/delete-calendar.asciidoc @@ -8,31 +8,32 @@ Deletes a calendar. - -==== Request +[[ml-delete-calendar-request]] +==== {api-request-title} `DELETE _ml/calendars/` +[[ml-delete-calendar-prereqs]] +==== {api-prereq-title} + +* If the {es} {security-features} are enabled, you must have `manage_ml` or +`manage` cluster privileges to use this API. See +{stack-ov}/security-privileges.html[Security privileges]. -==== Description +[[ml-delete-calendar-desc]] +==== {api-description-title} This API removes all scheduled events from the calendar then deletes the calendar. +[[ml-delete-calendar-path-parms]] +==== {api-path-parms-title} -==== Path Parameters - -`calendar_id` (required):: +`` (Required):: (string) Identifier for the calendar. - -==== Authorization - -You must have `manage_ml`, or `manage` cluster privileges to use this API. -For more information, see {xpack-ref}/security-privileges.html[Security Privileges]. - - -==== Examples +[[ml-delete-calendar-example]] +==== {api-examples-title} The following example deletes the `planned-outages` calendar: diff --git a/docs/reference/ml/apis/delete-datafeed.asciidoc b/docs/reference/ml/apis/delete-datafeed.asciidoc index d2a7845be7584..23917bf9e3365 100644 --- a/docs/reference/ml/apis/delete-datafeed.asciidoc +++ b/docs/reference/ml/apis/delete-datafeed.asciidoc @@ -10,38 +10,39 @@ Deletes an existing {dfeed}. - -==== Request +[[ml-delete-datafeed-request]] +==== {api-request-title} `DELETE _ml/datafeeds/` +[[ml-delete-datafeed-prereqs]] +==== {api-prereq-title} -==== Description - -NOTE: Unless the `force` parameter is used, the {dfeed} must be stopped before it can be deleted. - - -==== Path Parameters - -`feed_id` (required):: - (string) Identifier for the {dfeed} - +* If the {es} {security-features} are enabled, you must have `manage_ml` or +`manage` cluster privileges to use this API. See +{stack-ov}/security-privileges.html[Security privileges]. -===== Query Parameters +[[ml-delete-datafeed-desc]] +==== {api-description-title} -`force`:: - (boolean) Use to forcefully delete a started {dfeed}; this method is quicker than - stopping and deleting the {dfeed}. +NOTE: Unless you use the `force` parameter, you must stop the {dfeed} before you +can delete it. +[[ml-delete-datafeed-path-parms]] +==== {api-path-parms-title} -===== Authorization +`` (Required):: + (string) Identifier for the {dfeed}. -You must have `manage_ml`, or `manage` cluster privileges to use this API. -For more information, see {xpack-ref}/security-privileges.html[Security Privileges]. -//<>. +[[ml-delete-datafeed-query-parms]] +==== {api-query-parms-title} +`force` (Optional):: + (boolean) Use to forcefully delete a started {dfeed}; this method is quicker + than stopping and deleting the {dfeed}. -==== Examples +[[ml-delete-datafeed-example]] +==== {api-examples-title} The following example deletes the `datafeed-total-requests` {dfeed}: diff --git a/docs/reference/ml/apis/delete-expired-data.asciidoc b/docs/reference/ml/apis/delete-expired-data.asciidoc index 8814a1686736e..ada9ec1c8c34e 100644 --- a/docs/reference/ml/apis/delete-expired-data.asciidoc +++ b/docs/reference/ml/apis/delete-expired-data.asciidoc @@ -8,25 +8,27 @@ Deletes expired and unused machine learning data. -==== Request +[[ml-delete-expired-data-request]] +==== {api-request-title} `DELETE _ml/_delete_expired_data` -==== Description +[[ml-delete-expired-data-prereqs]] +==== {api-prereq-title} + +* If the {es} {security-features} are enabled, you must have `manage_ml` or +`manage` cluster privileges to use this API. See +{stack-ov}/security-privileges.html[Security privileges]. + +[[ml-delete-expired-data-desc]] +==== {api-description-title} Deletes all job results, model snapshots and forecast data that have exceeded their `retention days` period. Machine learning state documents that are not associated with any job are also deleted. -==== Authorization - -You must have `manage_ml`, or `manage` cluster privileges to use this API. -For more information, see -{stack-ov}/security-privileges.html[Security Privileges] and -{stack-ov}/built-in-roles.html[Built-in Roles]. - - -==== Examples +[[ml-delete-expired-data-example]] +==== {api-examples-title} The endpoint takes no arguments: diff --git a/docs/reference/ml/apis/delete-filter.asciidoc b/docs/reference/ml/apis/delete-filter.asciidoc index b2cbb7ef2832d..1962db29ad74e 100644 --- a/docs/reference/ml/apis/delete-filter.asciidoc +++ b/docs/reference/ml/apis/delete-filter.asciidoc @@ -8,32 +8,33 @@ Deletes a filter. - -==== Request +[[ml-delete-filter-request]] +==== {api-request-title} `DELETE _ml/filters/` +[[ml-delete-filter-prereqs]] +==== {api-prereq-title} + +* If the {es} {security-features} are enabled, you must have `manage_ml` or +`manage` cluster privileges to use this API. See +{stack-ov}/security-privileges.html[Security privileges]. -==== Description +[[ml-delete-filter-desc]] +==== {api-description-title} This API deletes a {stack-ov}/ml-rules.html[filter]. If a {ml} job references the filter, you cannot delete the filter. You must update or delete the job before you can delete the filter. +[[ml-delete-filter-path-parms]] +==== {api-path-parms-title} -==== Path Parameters - -`filter_id` (required):: +`` (Required):: (string) Identifier for the filter. - -==== Authorization - -You must have `manage_ml`, or `manage` cluster privileges to use this API. -For more information, see {xpack-ref}/security-privileges.html[Security Privileges]. - - -==== Examples +[[ml-delete-filter-example]] +==== {api-examples-title} The following example deletes the `safe_domains` filter: diff --git a/docs/reference/ml/apis/delete-forecast.asciidoc b/docs/reference/ml/apis/delete-forecast.asciidoc index 133b9105e478a..aac054217fced 100644 --- a/docs/reference/ml/apis/delete-forecast.asciidoc +++ b/docs/reference/ml/apis/delete-forecast.asciidoc @@ -8,7 +8,8 @@ Deletes forecasts from a {ml} job. -==== Request +[[ml-delete-forecast-request]] +==== {api-request-title} `DELETE _ml/anomaly_detectors//_forecast` + @@ -16,48 +17,54 @@ Deletes forecasts from a {ml} job. `DELETE _ml/anomaly_detectors//_forecast/_all` +[[ml-delete-forecast-prereqs]] +==== {api-prereq-title} -==== Description +* If the {es} {security-features} are enabled, you must have `manage_ml` or +`manage` cluster privileges to use this API. See +{stack-ov}/security-privileges.html[Security privileges]. -By default, forecasts are retained for 14 days. You can specify a different -retention period with the `expires_in` parameter in the <>. The delete forecast API enables you to delete one or more forecasts before they expire. +[[ml-delete-forecast-desc]] +==== {api-description-title} -NOTE: When you delete a job its associated forecasts are deleted. +By default, forecasts are retained for 14 days. You can specify a different +retention period with the `expires_in` parameter in the +<>. The delete forecast API enables you to delete +one or more forecasts before they expire. -For more information, see {stack-ov}/ml-overview.html#ml-forecasting[Forecasting the Future]. +NOTE: When you delete a job, its associated forecasts are deleted. +For more information, see +{stack-ov}/ml-overview.html#ml-forecasting[Forecasting the future]. -==== Path Parameters +[[ml-delete-forecast-path-parms]] +==== {api-path-parms-title} -`job_id` (required):: +`` (Required):: (string) Identifier for the job. -`forecast_id`:: +`forecast_id` (Optional):: (string) A comma-separated list of forecast identifiers. If you do not specify this optional parameter or if you specify `_all`, the API deletes all forecasts from the job. -==== Request Parameters +[[ml-delete-forecast-query-parms]] +==== {api-query-parms-title} -`allow_no_forecasts`:: +`allow_no_forecasts` (Optional):: (boolean) Specifies whether an error occurs when there are no forecasts. In particular, if this parameter is set to `false` and there are no forecasts associated with the job, attempts to delete all forecasts return an error. The default value is `true`. -`timeout`:: +`timeout` (Optional):: (time units) Specifies the period of time to wait for the completion of the delete operation. When this period of time elapses, the API fails and returns an error. The default value is `30s`. For more information about time units, see <>. - - -==== Authorization - -You must have `manage_ml`, or `manage` cluster privileges to use this API. -For more information, see {stack-ov}/security-privileges.html[Security Privileges]. -==== Examples +[[ml-delete-forecast-example]] +==== {api-examples-title} The following example deletes all forecasts from the `total-requests` job: diff --git a/docs/reference/ml/apis/delete-job.asciidoc b/docs/reference/ml/apis/delete-job.asciidoc index a52c434f93cfe..efd172ef5fb60 100644 --- a/docs/reference/ml/apis/delete-job.asciidoc +++ b/docs/reference/ml/apis/delete-job.asciidoc @@ -8,13 +8,20 @@ Deletes an existing anomaly detection job. - -==== Request +[[ml-delete-job-request]] +==== {api-request-title} `DELETE _ml/anomaly_detectors/` +[[ml-delete-job-prereqs]] +==== {api-prereq-title} + +* If {es} {security-features} are enabled, you must have `manage_ml` or `manage` +cluster privileges to use this API. See +{stack-ov}/security-privileges.html[Security privileges]. -==== Description +[[ml-delete-job-desc]] +==== {api-description-title} All job configuration, model state and results are deleted. @@ -30,29 +37,25 @@ is used the job must be closed before it can be deleted. It is not currently possible to delete multiple jobs using wildcards or a comma separated list. -==== Path Parameters +[[ml-delete-job-path-parms]] +==== {api-path-parms-title} -`job_id` (required):: - (string) Identifier for the job +`` (Required):: + (string) Identifier for the job. -===== Query Parameters +[[ml-delete-job-query-parms]] +==== {api-query-parms-title} -`force`:: +`force` (Optional):: (boolean) Use to forcefully delete an opened job; this method is quicker than closing and deleting the job. -`wait_for_completion`:: +`wait_for_completion` (Optional):: (boolean) Specifies whether the request should return immediately or wait until the job deletion completes. Defaults to `true`. -==== Authorization - -If {es} {security-features} are enabled, you must have `manage_ml`, or `manage` -cluster privileges to use this API. -For more information, see {stack-ov}/security-privileges.html[Security Privileges]. - - -==== Examples +[[ml-delete-job-example]] +==== {api-examples-title} The following example deletes the `total-requests` job: diff --git a/docs/reference/ml/apis/delete-snapshot.asciidoc b/docs/reference/ml/apis/delete-snapshot.asciidoc index 18092ff8e89c1..0e696f2a01139 100644 --- a/docs/reference/ml/apis/delete-snapshot.asciidoc +++ b/docs/reference/ml/apis/delete-snapshot.asciidoc @@ -8,34 +8,36 @@ Deletes an existing model snapshot. - -==== Request +[[ml-delete-snapshot-request]] +==== {api-request-title} `DELETE _ml/anomaly_detectors//model_snapshots/` +[[ml-delete-snapshot-prereqs]] +==== {api-prereq-title} + +* If the {es} {security-features} are enabled, you must have `manage_ml` or +`manage` cluster privileges to use this API. See +{stack-ov}/security-privileges.html[Security privileges]. -==== Description +[[ml-delete-snapshot-desc]] +==== {api-description-title} IMPORTANT: You cannot delete the active model snapshot. To delete that snapshot, first revert to a different one. To identify the active model snapshot, refer to the `model_snapshot_id` in the results from the get jobs API. -==== Path Parameters - -`job_id` (required):: - (string) Identifier for the job - -`snapshot_id` (required):: - (string) Identifier for the model snapshot - - -==== Authorization +[[ml-delete-snapshot-path-parms]] +==== {api-path-parms-title} -You must have `manage_ml`, or `manage` cluster privileges to use this API. -For more information, see {xpack-ref}/security-privileges.html[Security Privileges]. +`` (Required):: + (string) Identifier for the job. +`` (Required):: + (string) Identifier for the model snapshot. -==== Examples +[[ml-delete-snapshot-example]] +==== {api-examples-title} The following example deletes the `1491948163` snapshot: diff --git a/docs/reference/ml/apis/find-file-structure.asciidoc b/docs/reference/ml/apis/find-file-structure.asciidoc index e9d9da479c0f2..212e80c7e1bd2 100644 --- a/docs/reference/ml/apis/find-file-structure.asciidoc +++ b/docs/reference/ml/apis/find-file-structure.asciidoc @@ -11,12 +11,20 @@ experimental[] Finds the structure of a text file. The text file must contain data that is suitable to be ingested into {es}. -==== Request +[[ml-find-file-structure-request]] +==== {api-request-title} `POST _ml/find_file_structure` +[[ml-find-file-structure-prereqs]] +==== {api-prereq-title} -==== Description +* If the {es} {security-features} are enabled, you must have `monitor_ml` or +`monitor` cluster privileges to use this API. See +{stack-ov}/security-privileges.html[Security privileges]. + +[[ml-find-file-structure-desc]] +==== {api-description-title} This API provides a starting point for ingesting data into {es} in a format that is suitable for subsequent use with other {ml} functionality. @@ -47,38 +55,39 @@ specify the `explain` query parameter. It causes an `explanation` to appear in the response, which should help in determining why the returned structure was chosen. -==== Query Parameters +[[ml-find-file-structure-query-parms]] +==== {api-query-parms-title} -`charset`:: +`charset` (Optional):: (string) The file's character set. It must be a character set that is supported by the JVM that {es} uses. For example, `UTF-8`, `UTF-16LE`, `windows-1252`, or `EUC-JP`. If this parameter is not specified, the structure finder chooses an appropriate character set. -`column_names`:: +`column_names` (Optional):: (string) If you have set `format` to `delimited`, you can specify the column names in a comma-separated list. If this parameter is not specified, the structure finder uses the column names from the header row of the file. If the file does not have a header role, columns are named "column1", "column2", "column3", etc. -`delimiter`:: +`delimiter` (Optional):: (string) If you have set `format` to `delimited`, you can specify the character used to delimit the values in each row. Only a single character is supported; the delimiter cannot have multiple characters. If this parameter is not specified, the structure finder considers the following possibilities: comma, tab, semi-colon, and pipe (`|`). -`explain`:: +`explain` (Optional):: (boolean) If this parameter is set to `true`, the response includes a field named `explanation`, which is an array of strings that indicate how the structure finder produced its result. The default value is `false`. -`format`:: +`format` (Optional):: (string) The high level structure of the file. Valid values are `ndjson`, `xml`, `delimited`, and `semi_structured_text`. If this parameter is not specified, the structure finder chooses one. -`grok_pattern`:: +`grok_pattern` (Optional):: (string) If you have set `format` to `semi_structured_text`, you can specify a Grok pattern that is used to extract fields from every message in the file. The name of the timestamp field in the Grok pattern must match what is specified @@ -86,13 +95,20 @@ chosen. name of the timestamp field in the Grok pattern must match "timestamp". If `grok_pattern` is not specified, the structure finder creates a Grok pattern. -`has_header_row`:: +`has_header_row` (Optional):: (boolean) If you have set `format` to `delimited`, you can use this parameter to indicate whether the column names are in the first row of the file. If this parameter is not specified, the structure finder guesses based on the similarity of the first row of the file to other rows. -`lines_to_sample`:: +`line_merge_size_limit` (Optional):: + (unsigned integer) The maximum number of characters in a message when lines are + merged to form messages while analyzing semi-structured files. The default + is 10000. If you have extremely long messages you may need to increase this, but + be aware that this may lead to very long processing times if the way to group + lines into messages is misdetected. + +`lines_to_sample` (Optional):: (unsigned integer) The number of lines to include in the structural analysis, starting from the beginning of the file. The minimum is 2; the default is 1000. If the value of this parameter is greater than the number of lines in @@ -108,7 +124,7 @@ efficient to upload a sample file with more variety in the first 1000 lines than to request analysis of 100000 lines to achieve some variety. -- -`quote`:: +`quote` (Optional):: (string) If you have set `format` to `delimited`, you can specify the character used to quote the values in each row if they contain newlines or the delimiter character. Only a single character is supported. If this parameter is not @@ -116,18 +132,18 @@ to request analysis of 100000 lines to achieve some variety. format does not use quoting, a workaround is to set this argument to a character that does not appear anywhere in the sample. -`should_trim_fields`:: +`should_trim_fields` (Optional):: (boolean) If you have set `format` to `delimited`, you can specify whether values between delimiters should have whitespace trimmed from them. If this parameter is not specified and the delimiter is pipe (`|`), the default value is `true`. Otherwise, the default value is `false`. -`timeout`:: +`timeout` (Optional):: (time) Sets the maximum amount of time that the structure analysis make take. If the analysis is still running when the timeout expires then it will be aborted. The default value is 25 seconds. -`timestamp_field`:: +`timestamp_field` (Optional):: (string) The name of the field that contains the primary timestamp of each record in the file. In particular, if the file were ingested into an index, this is the field that would be used to populate the `@timestamp` field. + @@ -146,7 +162,7 @@ field (if any) is the primary timestamp field. For structured file formats, it is not compulsory to have a timestamp in the file. -- -`timestamp_format`:: +`timestamp_format` (Optional):: (string) The Java time format of the timestamp field in the file. + + -- @@ -190,22 +206,16 @@ format from a built-in set. -- -==== Request Body +[[ml-find-file-structure-request-body]] +==== {api-request-body-title} The text file that you want to analyze. It must contain data that is suitable to be ingested into {es}. It does not need to be in JSON format and it does not need to be UTF-8 encoded. The size is limited to the {es} HTTP receive buffer size, which defaults to 100 Mb. - -==== Authorization - -You must have `monitor_ml`, or `monitor` cluster privileges to use this API. -For more information, see {stack-ov}/security-privileges.html[Security Privileges]. - - [[ml-find-file-structure-examples]] -==== Examples +==== {api-examples-title} Suppose you have a newline-delimited JSON file that contains information about some books. You can send the contents to the `find_file_structure` endpoint: @@ -438,6 +448,8 @@ If the request does not encounter errors, you receive the following result: "release_date" : { "count" : 24, "cardinality" : 20, + "earliest" : "1932-06-01", + "latest" : "2011-06-02", "top_hits" : [ { "value" : "1985-06-01", @@ -1145,6 +1157,8 @@ If the request does not encounter errors, you receive the following result: "tpep_dropoff_datetime" : { "count" : 19998, "cardinality" : 9066, + "earliest" : "2018-05-31 06:18:15", + "latest" : "2018-06-02 02:25:44", "top_hits" : [ { "value" : "2018-06-01 01:12:12", @@ -1191,6 +1205,8 @@ If the request does not encounter errors, you receive the following result: "tpep_pickup_datetime" : { "count" : 19998, "cardinality" : 8760, + "earliest" : "2018-05-31 06:08:31", + "latest" : "2018-06-02 01:21:21", "top_hits" : [ { "value" : "2018-06-01 00:01:23", @@ -1450,6 +1466,8 @@ this: "timestamp" : { "count" : 53, "cardinality" : 28, + "earliest" : "2018-09-27T14:39:28,518", + "latest" : "2018-09-27T14:39:37,012", "top_hits" : [ { "value" : "2018-09-27T14:39:29,859", @@ -1712,6 +1730,8 @@ this: "timestamp" : { "count" : 53, "cardinality" : 28, + "earliest" : "2018-09-27T14:39:28,518", + "latest" : "2018-09-27T14:39:37,012", "top_hits" : [ { "value" : "2018-09-27T14:39:29,859", diff --git a/docs/reference/ml/apis/flush-job.asciidoc b/docs/reference/ml/apis/flush-job.asciidoc index e2793b2c1a174..590f866ca1799 100644 --- a/docs/reference/ml/apis/flush-job.asciidoc +++ b/docs/reference/ml/apis/flush-job.asciidoc @@ -8,13 +8,20 @@ Forces any buffered data to be processed by the job. - -==== Request +[[ml-flush-job-request]] +==== {api-request-title} `POST _ml/anomaly_detectors//_flush` +[[ml-flush-job-prereqs]] +==== {api-prereq-title} + +* If the {es} {security-features} are enabled, you must have `manage_ml` or +`manage` cluster privileges to use this API. See +{stack-ov}/security-privileges.html[Security privileges]. -==== Description +[[ml-flush-job-desc]] +==== {api-description-title} The flush jobs API is only applicable when sending data for analysis using the <>. Depending on the content of the buffer, then it @@ -26,44 +33,38 @@ remains open and is available to continue analyzing data. A close operation additionally prunes and persists the model state to disk and the job must be opened again before analyzing further data. +[[ml-flush-job-path-parms]] +==== {api-path-parms-title} -==== Path Parameters - -`job_id` (required):: -(string) Identifier for the job - +`` (Required):: +(string) Identifier for the job. -==== Query Parameters +[[ml-flush-job-query-parms]] +==== {api-query-parms-title} -`advance_time`:: +`advance_time` (Optional):: (string) Specifies to advance to a particular time value. Results are generated and the model is updated for data from the specified time interval. -`calc_interim`:: +`calc_interim` (Optional):: (boolean) If true, calculates the interim results for the most recent bucket or all buckets within the latency period. -`end`:: +`end` (Optional):: (string) When used in conjunction with `calc_interim`, specifies the range of buckets on which to calculate interim results. -`skip_time`:: +`skip_time` (Optional):: (string) Specifies to skip to a particular time value. Results are not generated and the model is not updated for data from the specified time interval. -`start`:: +`start` (Optional):: (string) When used in conjunction with `calc_interim`, specifies the range of buckets on which to calculate interim results. - -==== Authorization - -You must have `manage_ml`, or `manage` cluster privileges to use this API. -For more information, see {xpack-ref}/security-privileges.html[Security Privileges]. - - -==== Examples +[[ml-flush-job-example]] +==== {api-examples-title} The following example flushes the `total-requests` job: diff --git a/docs/reference/ml/apis/forecast.asciidoc b/docs/reference/ml/apis/forecast.asciidoc index 71a7e1db2b185..d137b2e1be3ce 100644 --- a/docs/reference/ml/apis/forecast.asciidoc +++ b/docs/reference/ml/apis/forecast.asciidoc @@ -8,14 +8,22 @@ Predicts the future behavior of a time series by using its historical behavior. -==== Request +[[ml-forecast-request]] +==== {api-request-title} `POST _ml/anomaly_detectors//_forecast` +[[ml-forecast-prereqs]] +==== {api-prereq-title} -==== Description +* If the {es} {security-features} are enabled, you must have `manage_ml` or +`manage` cluster privileges to use this API. See +{stack-ov}/security-privileges.html[Security privileges]. -See {xpack-ref}/ml-overview.html#ml-forecasting[Forecasting the Future]. +[[ml-forecast-desc]] +==== {api-description-title} + +See {stack-ov}/ml-overview.html#ml-forecasting[Forecasting the future]. [NOTE] =============================== @@ -25,33 +33,29 @@ forecast. For more information about this property, see <>. * The job must be open when you create a forecast. Otherwise, an error occurs. =============================== -==== Path Parameters +[[ml-forecast-path-parms]] +==== {api-path-parms-title} -`job_id`:: +`` (Required):: (string) Identifier for the job. +[[ml-forecast-request-body]] +==== {api-request-body-title} -==== Request Parameters - -`duration`:: +`duration` (Optional):: (time units) A period of time that indicates how far into the future to forecast. For example, `30d` corresponds to 30 days. The default value is 1 day. The forecast starts at the last record that was processed. For more information about time units, see <>. -`expires_in`:: +`expires_in` (Optional):: (time units) The period of time that forecast results are retained. After a forecast expires, the results are deleted. The default value is 14 days. If set to a value of `0`, the forecast is never automatically deleted. For more information about time units, see <>. -==== Authorization - -You must have `manage_ml`, or `manage` cluster privileges to use this API. -For more information, see {xpack-ref}/security-privileges.html[Security Privileges]. - - -==== Examples +[[ml-forecast-example]] +==== {api-examples-title} The following example requests a 10 day forecast for the `total-requests` job: diff --git a/docs/reference/ml/apis/get-bucket.asciidoc b/docs/reference/ml/apis/get-bucket.asciidoc index 39c548dd64e8d..2a73d0f5d3538 100644 --- a/docs/reference/ml/apis/get-bucket.asciidoc +++ b/docs/reference/ml/apis/get-bucket.asciidoc @@ -8,64 +8,75 @@ Retrieves job results for one or more buckets. - -==== Request +[[ml-get-bucket-request]] +==== {api-request-title} `GET _ml/anomaly_detectors//results/buckets` + `GET _ml/anomaly_detectors//results/buckets/` +[[ml-get-bucket-prereqs]] +==== {api-prereq-title} + +* If the {es} {security-features} are enabled, you must have `monitor_ml`, +`monitor`, `manage_ml`, or `manage` cluster privileges to use this API. You also +need `read` index privilege on the index that stores the results. The +`machine_learning_admin` and `machine_learning_user` roles provide these +privileges. For more information, see +{stack-ov}/security-privileges.html[Security privileges] and +{stack-ov}/built-in-roles.html[Built-in roles]. -==== Description +[[ml-get-bucket-desc]] +==== {api-description-title} The get buckets API presents a chronological view of the records, grouped by bucket. +[[ml-get-bucket-path-parms]] +==== {api-path-parms-title} -==== Path Parameters - -`job_id`:: +`` (Required):: (string) Identifier for the job -`timestamp`:: +`` (Optional):: (string) The timestamp of a single bucket result. - If you do not specify this optional parameter, the API returns information + If you do not specify this parameter, the API returns information about all buckets. +[[ml-get-bucket-request-body]] +==== {api-request-body-title} -==== Request Body - -`anomaly_score`:: +`anomaly_score` (Optional):: (double) Returns buckets with anomaly scores greater or equal than this value. -`desc`:: +`desc` (Optional):: (boolean) If true, the buckets are sorted in descending order. -`end`:: +`end` (Optional):: (string) Returns buckets with timestamps earlier than this time. -`exclude_interim`:: +`exclude_interim` (Optional):: (boolean) If true, the output excludes interim results. By default, interim results are included. -`expand`:: +`expand` (Optional):: (boolean) If true, the output includes anomaly records. -`page`:: +`page` (Optional):: `from`::: (integer) Skips the specified number of buckets. `size`::: (integer) Specifies the maximum number of buckets to obtain. -`sort`:: +`sort` (Optional):: (string) Specifies the sort field for the requested buckets. By default, the buckets are sorted by the `timestamp` field. -`start`:: +`start` (Optional):: (string) Returns buckets with timestamps after this time. - -===== Results +[[ml-get-bucket-results]] +==== {api-response-body-title} The API returns the following information: @@ -73,18 +84,8 @@ The API returns the following information: (array) An array of bucket objects. For more information, see <>. - -==== Authorization - -You must have `monitor_ml`, `monitor`, `manage_ml`, or `manage` cluster -privileges to use this API. You also need `read` index privilege on the index -that stores the results. The `machine_learning_admin` and `machine_learning_user` -roles provide these privileges. For more information, see -{xpack-ref}/security-privileges.html[Security Privileges] and -{xpack-ref}/built-in-roles.html[Built-in Roles]. - - -==== Examples +[[ml-get-bucket-example]] +==== {api-examples-title} The following example gets bucket information for the `it-ops-kpi` job: diff --git a/docs/reference/ml/apis/get-calendar-event.asciidoc b/docs/reference/ml/apis/get-calendar-event.asciidoc index a890f67db0d23..173a249488684 100644 --- a/docs/reference/ml/apis/get-calendar-event.asciidoc +++ b/docs/reference/ml/apis/get-calendar-event.asciidoc @@ -9,39 +9,49 @@ Retrieves information about the scheduled events in calendars. - -==== Request +[[ml-get-calendar-event-request]] +==== {api-request-title} `GET _ml/calendars//events` + `GET _ml/calendars/_all/events` +[[ml-get-calendar-event-prereqs]] +==== {api-prereq-title} + +* If the {es} {security-features} are enabled, you must have `monitor_ml`, +`monitor`, `manage_ml`, or `manage` cluster privileges to use this API. See +{stack-ov}/security-privileges.html[Security privileges]. -===== Description +[[ml-get-calendar-event-desc]] +==== {api-description-title} You can get scheduled event information for a single calendar or for all calendars by using `_all`. -==== Path Parameters +[[ml-get-calendar-event-path-parms]] +==== {api-path-parms-title} -`calendar_id` (required):: +`` (Required):: (string) Identifier for the calendar. -==== Request Body +[[ml-get-calendar-event-request-body]] +==== {api-request-body-title} -`end`:: +`end` (Optional):: (string) Specifies to get events with timestamps earlier than this time. -`from`:: +`from` (Optional):: (integer) Skips the specified number of events. -`size`:: +`size` (Optional):: (integer) Specifies the maximum number of events to obtain. -`start`:: +`start` (Optional):: (string) Specifies to get events with timestamps after this time. -==== Results +[[ml-get-calendar-event-results]] +==== {api-response-body-title} The API returns the following information: @@ -49,15 +59,8 @@ The API returns the following information: (array) An array of scheduled event resources. For more information, see <>. - -==== Authorization - -You must have `monitor_ml`, `monitor`, `manage_ml`, or `manage` cluster -privileges to use this API. For more information, see -{xpack-ref}/security-privileges.html[Security Privileges]. - - -==== Examples +[[ml-get-calendar-event-example]] +==== {api-examples-title} The following example gets information about the scheduled events in the `planned-outages` calendar: diff --git a/docs/reference/ml/apis/get-calendar.asciidoc b/docs/reference/ml/apis/get-calendar.asciidoc index 09e429b1f6de3..3d55f825bdb86 100644 --- a/docs/reference/ml/apis/get-calendar.asciidoc +++ b/docs/reference/ml/apis/get-calendar.asciidoc @@ -8,37 +8,44 @@ Retrieves configuration information for calendars. - -==== Request +[[ml-get-calendar-request]] +==== {api-request-title} `GET _ml/calendars/` + `GET _ml/calendars/_all` +[[ml-get-calendar-prereqs]] +==== {api-prereq-title} + +* If the {es} {security-features} are enabled, you must have `monitor_ml`, +`monitor`, `manage_ml`, or `manage` cluster privileges to use this API. See +{stack-ov}/security-privileges.html[Security privileges]. -===== Description +[[ml-get-calendar-desc]] +==== {api-description-title} You can get information for a single calendar or for all calendars by using `_all`. +[[ml-get-calendar-path-parms]] +==== {api-path-parms-title} -==== Path Parameters - -`calendar_id`:: +`` (Required):: (string) Identifier for the calendar. +[[ml-get-calendar-request-body]] +==== {api-request-body-title} -==== Request Body - -`page`:: +`page` (Optional):: `from`::: (integer) Skips the specified number of calendars. -`size`::: +`size` (Optional)::: (integer) Specifies the maximum number of calendars to obtain. - -==== Results +[[ml-get-calendar-results]] +==== {api-response-body-title} The API returns the following information: @@ -46,15 +53,8 @@ The API returns the following information: (array) An array of calendar resources. For more information, see <>. - -==== Authorization - -You must have `monitor_ml`, `monitor`, `manage_ml`, or `manage` cluster -privileges to use this API. For more information, see -{xpack-ref}/security-privileges.html[Security Privileges]. - - -==== Examples +[[ml-get-calendar-example]] +==== {api-examples-title} The following example gets configuration information for the `planned-outages` calendar: diff --git a/docs/reference/ml/apis/get-category.asciidoc b/docs/reference/ml/apis/get-category.asciidoc index 1fbfda20eccc0..6301eaf13a538 100644 --- a/docs/reference/ml/apis/get-category.asciidoc +++ b/docs/reference/ml/apis/get-category.asciidoc @@ -8,38 +8,50 @@ Retrieves job results for one or more categories. - -==== Request +[[ml-get-category-request]] +==== {api-request-title} `GET _ml/anomaly_detectors//results/categories` + `GET _ml/anomaly_detectors//results/categories/` -==== Description +[[ml-get-category-prereqs]] +==== {api-prereq-title} + +* If the {es} {security-features} are enabled, you must have `monitor_ml`, +`monitor`, `manage_ml`, or `manage` cluster privileges to use this API. You also +need `read` index privilege on the index that stores the results. The +`machine_learning_admin` and `machine_learning_user` roles provide these +privileges. See {stack-ov}/security-privileges.html[Security privileges] and +{stack-ov}/built-in-roles.html[Built-in roles]. + +[[ml-get-category-desc]] +==== {api-description-title} For more information about categories, see -{xpack-ref}/ml-configuring-categories.html[Categorizing Log Messages]. +{stack-ov}/ml-configuring-categories.html[Categorizing log messages]. -==== Path Parameters +[[ml-get-category-path-parms]] +==== {api-path-parms-title} -`job_id`:: +`` (Required):: (string) Identifier for the job. -`category_id`:: - (long) Identifier for the category. If you do not specify this optional parameter, +`` (Optional):: + (long) Identifier for the category. If you do not specify this parameter, the API returns information about all categories in the job. +[[ml-get-category-request-body]] +==== {api-request-body-title} -==== Request Body - -`page`:: +`page` (Optional):: `from`::: (integer) Skips the specified number of categories. `size`::: (integer) Specifies the maximum number of categories to obtain. - -==== Results +[[ml-get-category-results]] +==== {api-response-body-title} The API returns the following information: @@ -47,18 +59,8 @@ The API returns the following information: (array) An array of category objects. For more information, see <>. - -==== Authorization - -You must have `monitor_ml`, `monitor`, `manage_ml`, or `manage` cluster -privileges to use this API. You also need `read` index privilege on the index -that stores the results. The `machine_learning_admin` and `machine_learning_user` -roles provide these privileges. For more information, see -{xpack-ref}/security-privileges.html[Security Privileges] and -{xpack-ref}/built-in-roles.html[Built-in Roles]. - - -==== Examples +[[ml-get-category-example]] +==== {api-examples-title} The following example gets information about one category for the `esxi_log` job: diff --git a/docs/reference/ml/apis/get-datafeed-stats.asciidoc b/docs/reference/ml/apis/get-datafeed-stats.asciidoc index d1b842509b408..6ce99785912af 100644 --- a/docs/reference/ml/apis/get-datafeed-stats.asciidoc +++ b/docs/reference/ml/apis/get-datafeed-stats.asciidoc @@ -10,9 +10,8 @@ Retrieves usage information for {dfeeds}. - -==== Request - +[[ml-get-datafeed-stats-request]] +==== {api-request-title} `GET _ml/datafeeds//_stats` + @@ -20,11 +19,17 @@ Retrieves usage information for {dfeeds}. `GET _ml/datafeeds/_stats` + -`GET _ml/datafeeds/_all/_stats` + +`GET _ml/datafeeds/_all/_stats` +[[ml-get-datafeed-stats-prereqs]] +==== {api-prereq-title} +* If the {es} {security-features} are enabled, you must have `monitor_ml`, +`monitor`, `manage_ml`, or `manage` cluster privileges to use this API. See +{stack-ov}/security-privileges.html[Security privileges]. -==== Description +[[ml-get-datafeed-stats-desc]] +==== {api-description-title} You can get statistics for multiple {dfeeds} in a single API request by using a comma-separated list of {dfeeds} or a wildcard expression. You can get @@ -36,15 +41,16 @@ If the {dfeed} is stopped, the only information you receive is the IMPORTANT: This API returns a maximum of 10,000 {dfeeds}. -==== Path Parameters +[[ml-get-datafeed-stats-path-parms]] +==== {api-path-parms-title} -`feed_id`:: +`` (Optional):: (string) Identifier for the {dfeed}. It can be a {dfeed} identifier or a wildcard expression. If you do not specify one of these options, the API returns statistics for all {dfeeds}. - -==== Results +[[ml-get-datafeed-stats-results]] +==== {api-response-body-title} The API returns the following information: @@ -52,15 +58,8 @@ The API returns the following information: (array) An array of {dfeed} count objects. For more information, see <>. - -==== Authorization - -You must have `monitor_ml`, `monitor`, `manage_ml`, or `manage` cluster -privileges to use this API. For more information, see -{xpack-ref}/security-privileges.html[Security Privileges]. - - -==== Examples +[[ml-get-datafeed-stats-example]] +==== {api-examples-title} The following example gets usage information for the `datafeed-total-requests` {dfeed}: diff --git a/docs/reference/ml/apis/get-datafeed.asciidoc b/docs/reference/ml/apis/get-datafeed.asciidoc index 2c9ef7e3aec74..abc79ae5c7d71 100644 --- a/docs/reference/ml/apis/get-datafeed.asciidoc +++ b/docs/reference/ml/apis/get-datafeed.asciidoc @@ -10,8 +10,8 @@ Retrieves configuration information for {dfeeds}. -==== Request - +[[ml-get-datafeed-request]] +==== {api-request-title} `GET _ml/datafeeds/` + @@ -19,10 +19,17 @@ Retrieves configuration information for {dfeeds}. `GET _ml/datafeeds/` + -`GET _ml/datafeeds/_all` + +`GET _ml/datafeeds/_all` + +[[ml-get-datafeed-prereqs]] +==== {api-prereq-title} +* If the {es} {security-features} are enabled, you must have `monitor_ml`, +`monitor`, `manage_ml`, or `manage` cluster privileges to use this API. See +{stack-ov}/security-privileges.html[Security privileges]. -===== Description +[[ml-get-datafeed-desc]] +==== {api-description-title} You can get information for multiple {dfeeds} in a single API request by using a comma-separated list of {dfeeds} or a wildcard expression. You can get @@ -31,15 +38,16 @@ information for all {dfeeds} by using `_all`, by specifying `*` as the IMPORTANT: This API returns a maximum of 10,000 {dfeeds}. -==== Path Parameters +[[ml-get-datafeed-path-parms]] +==== {api-path-parms-title} -`feed_id`:: +`` (Optional):: (string) Identifier for the {dfeed}. It can be a {dfeed} identifier or a wildcard expression. If you do not specify one of these options, the API returns information about all {dfeeds}. - -==== Results +[[ml-get-datafeed-results]] +==== {api-response-body-title} The API returns the following information: @@ -47,15 +55,8 @@ The API returns the following information: (array) An array of {dfeed} objects. For more information, see <>. - -==== Authorization - -You must have `monitor_ml`, `monitor`, `manage_ml`, or `manage` cluster -privileges to use this API. For more information, see -{xpack-ref}/security-privileges.html[Security Privileges]. - - -==== Examples +[[ml-get-datafeed-example]] +==== {api-examples-title} The following example gets configuration information for the `datafeed-total-requests` {dfeed}: diff --git a/docs/reference/ml/apis/get-filter.asciidoc b/docs/reference/ml/apis/get-filter.asciidoc index 53dfab2530a10..ad5fee343f6d5 100644 --- a/docs/reference/ml/apis/get-filter.asciidoc +++ b/docs/reference/ml/apis/get-filter.asciidoc @@ -8,36 +8,43 @@ Retrieves filters. - -==== Request +[[ml-get-filter-request]] +==== {api-request-title} `GET _ml/filters/` + `GET _ml/filters/` +[[ml-get-filter-prereqs]] +==== {api-prereq-title} + +* If the {es} {security-features} are enabled, you must have `monitor_ml`, +`monitor`, `manage_ml`, or `manage` cluster privileges to use this API. See +{stack-ov}/security-privileges.html[Security privileges]. -===== Description +[[ml-get-filter-desc]] +==== {api-description-title} You can get a single filter or all filters. For more information, see {stack-ov}/ml-rules.html[Machine learning custom rules]. +[[ml-get-filter-path-parms]] +==== {api-path-parms-title} -==== Path Parameters - -`filter_id`:: +`` (Optional):: (string) Identifier for the filter. +[[ml-get-filter-query-parms]] +==== {api-query-parms-title} -==== Querystring Parameters - -`from`::: +`from` (Optional)::: (integer) Skips the specified number of filters. -`size`::: +`size` (Optional)::: (integer) Specifies the maximum number of filters to obtain. - -==== Results +[[ml-get-filter-results]] +==== {api-response-body-title} The API returns the following information: @@ -45,15 +52,8 @@ The API returns the following information: (array) An array of filter resources. For more information, see <>. - -==== Authorization - -You must have `monitor_ml`, `monitor`, `manage_ml`, or `manage` cluster -privileges to use this API. For more information, see -{xpack-ref}/security-privileges.html[Security Privileges]. - - -==== Examples +[[ml-get-filter-example]] +==== {api-examples-title} The following example gets configuration information for the `safe_domains` filter: diff --git a/docs/reference/ml/apis/get-influencer.asciidoc b/docs/reference/ml/apis/get-influencer.asciidoc index 7425a734ed441..8d7ca889a264f 100644 --- a/docs/reference/ml/apis/get-influencer.asciidoc +++ b/docs/reference/ml/apis/get-influencer.asciidoc @@ -8,48 +8,58 @@ Retrieves job results for one or more influencers. - -==== Request +[[ml-get-influencer-request]] +==== {api-request-title} `GET _ml/anomaly_detectors//results/influencers` -//===== Description +[[ml-get-influencer-prereqs]] +==== {api-prereq-title} + +* If the {es} {security-features} are enabled, you must have `monitor_ml`, +`monitor`, `manage_ml`, or `manage` cluster privileges to use this API. You also +need `read` index privilege on the index that stores the results. The +`machine_learning_admin` and `machine_learning_user` roles provide these +privileges. See {stack-ov}/security-privileges.html[Security privileges] and +{stack-ov}/built-in-roles.html[Built-in roles]. -==== Path Parameters +[[ml-get-influencer-path-parms]] +==== {api-path-parms-title} -`job_id`:: +`` (Required):: (string) Identifier for the job. -==== Request Body +[[ml-get-influencer-request-body]] +==== {api-request-body-title} -`desc`:: +`desc` (Optional):: (boolean) If true, the results are sorted in descending order. -`end`:: +`end` (Optional):: (string) Returns influencers with timestamps earlier than this time. -`exclude_interim`:: +`exclude_interim` (Optional):: (boolean) If true, the output excludes interim results. By default, interim results are included. -`influencer_score`:: +`influencer_score` (Optional):: (double) Returns influencers with anomaly scores greater or equal than this value. -`page`:: +`page` (Optional):: `from`::: (integer) Skips the specified number of influencers. `size`::: (integer) Specifies the maximum number of influencers to obtain. -`sort`:: +`sort` (Optional):: (string) Specifies the sort field for the requested influencers. By default the influencers are sorted by the `influencer_score` value. -`start`:: +`start` (Optional):: (string) Returns influencers with timestamps after this time. - -==== Results +[[ml-get-influencer-results]] +==== {api-response-body-title} The API returns the following information: @@ -57,19 +67,8 @@ The API returns the following information: (array) An array of influencer objects. For more information, see <>. - -==== Authorization - -You must have `monitor_ml`, `monitor`, `manage_ml`, or `manage` cluster -privileges to use this API. You also need `read` index privilege on the index -that stores the results. The `machine_learning_admin` and `machine_learning_user` -roles provide these privileges. For more information, see -{xpack-ref}/security-privileges.html[Security Privileges] and -{xpack-ref}/built-in-roles.html[Built-in Roles]. -//<> and <>. - - -==== Examples +[[ml-get-influencer-example]] +==== {api-examples-title} The following example gets influencer information for the `it_ops_new_kpi` job: diff --git a/docs/reference/ml/apis/get-job-stats.asciidoc b/docs/reference/ml/apis/get-job-stats.asciidoc index b674b01802bee..8a705d7ff9ed1 100644 --- a/docs/reference/ml/apis/get-job-stats.asciidoc +++ b/docs/reference/ml/apis/get-job-stats.asciidoc @@ -8,10 +8,8 @@ Retrieves usage information for jobs. - -==== Request - - +[[ml-get-job-stats-request]] +==== {api-request-title} `GET _ml/anomaly_detectors//_stats` @@ -19,10 +17,17 @@ Retrieves usage information for jobs. `GET _ml/anomaly_detectors/_stats` + -`GET _ml/anomaly_detectors/_all/_stats` + +`GET _ml/anomaly_detectors/_all/_stats` +[[ml-get-job-stats-prereqs]] +==== {api-prereq-title} -===== Description +* If the {es} {security-features} are enabled, you must have `monitor_ml`, +`monitor`, `manage_ml`, or `manage` cluster privileges to use this API. See +{stack-ov}/security-privileges.html[Security privileges]. + +[[ml-get-job-stats-desc]] +==== {api-description-title} You can get statistics for multiple jobs in a single API request by using a group name, a comma-separated list of jobs, or a wildcard expression. You can @@ -31,16 +36,16 @@ get statistics for all jobs by using `_all`, by specifying `*` as the IMPORTANT: This API returns a maximum of 10,000 jobs. +[[ml-get-job-stats-path-parms]] +==== {api-path-parms-title} -==== Path Parameters - -`job_id`:: +`` (Optional):: (string) An identifier for the job. It can be a job identifier, a group name, or a wildcard expression. If you do not specify one of these options, the API returns statistics for all jobs. - -==== Results +[[ml-get-job-stats-results]] +==== {api-response-body-title} The API returns the following information: @@ -48,15 +53,8 @@ The API returns the following information: (array) An array of job statistics objects. For more information, see <>. - -==== Authorization - -You must have `monitor_ml`, `monitor`, `manage_ml`, or `manage` cluster -privileges to use this API. For more information, see -{xpack-ref}/security-privileges.html[Security Privileges]. - - -==== Examples +[[ml-get-job-stats-example]] +==== {api-examples-title} The following example gets usage information for the `farequote` job: @@ -105,7 +103,14 @@ The API returns the following results: "log_time": 1491948163000, "timestamp": 1455234600000 }, - "state": "closed" + "state": "closed", + "timing_stats": { + "job_id": "farequote", + "minimum_bucket_processing_time_ms": 0.0, + "maximum_bucket_processing_time_ms": 15.0, + "average_bucket_processing_time_ms": 8.75, + "exponential_average_bucket_processing_time_ms": 6.1435899 + } } ] } diff --git a/docs/reference/ml/apis/get-job.asciidoc b/docs/reference/ml/apis/get-job.asciidoc index 4eb7eaf5a7f07..176ca09fc56c5 100644 --- a/docs/reference/ml/apis/get-job.asciidoc +++ b/docs/reference/ml/apis/get-job.asciidoc @@ -8,8 +8,8 @@ Retrieves configuration information for jobs. - -==== Request +[[ml-get-job-request]] +==== {api-request-title} `GET _ml/anomaly_detectors/` + @@ -19,8 +19,15 @@ Retrieves configuration information for jobs. `GET _ml/anomaly_detectors/_all` +[[ml-get-job-prereqs]] +==== {api-prereq-title} + +* If the {es} {security-features} are enabled, you must have `monitor_ml`, +`monitor`, `manage_ml`, or `manage` cluster privileges to use this API. See +{stack-ov}/security-privileges.html[Security privileges]. -===== Description +[[ml-get-job-desc]] +==== {api-description-title} You can get information for multiple jobs in a single API request by using a group name, a comma-separated list of jobs, or a wildcard expression. You can @@ -29,15 +36,16 @@ get information for all jobs by using `_all`, by specifying `*` as the IMPORTANT: This API returns a maximum of 10,000 jobs. +[[ml-get-job-path-parms]] +==== {api-path-parms-title} -==== Path Parameters - -`job_id`:: +` (Optional)`:: (string) Identifier for the job. It can be a job identifier, a group name, or a wildcard expression. If you do not specify one of these options, the API returns information for all jobs. -==== Results +[[ml-get-job-results]] +==== {api-response-body-title} The API returns the following information: @@ -45,15 +53,8 @@ The API returns the following information: (array) An array of job resources. For more information, see <>. - -==== Authorization - -You must have `monitor_ml`, `monitor`, `manage_ml`, or `manage` cluster -privileges to use this API. For more information, see -{xpack-ref}/security-privileges.html[Security Privileges]. - - -==== Examples +[[ml-get-job-example]] +==== {api-examples-title} The following example gets configuration information for the `total-requests` job: diff --git a/docs/reference/ml/apis/get-ml-info.asciidoc b/docs/reference/ml/apis/get-ml-info.asciidoc index 00c57c8cf7f5b..2c486741ffd42 100644 --- a/docs/reference/ml/apis/get-ml-info.asciidoc +++ b/docs/reference/ml/apis/get-ml-info.asciidoc @@ -10,28 +10,30 @@ Returns defaults and limits used by machine learning. -==== Request +[[get-ml-info-request]] +==== {api-request-title} `GET _ml/info` -==== Description +[[get-ml-info-prereqs]] +==== {api-prereq-title} + +* If the {es} {security-features} are enabled, you must have `monitor_ml`, +`monitor`, `manage_ml`, or `manage` cluster privileges to use this API. The +`machine_learning_admin` and `machine_learning_user` roles provide these +privileges. See {stack-ov}/security-privileges.html[Security privileges] and +{stack-ov}/built-in-roles.html[Built-in roles]. + +[[get-ml-info-desc]] +==== {api-description-title} This endpoint is designed to be used by a user interface that needs to fully understand machine learning configurations where some options are not specified, meaning that the defaults should be used. This endpoint may be used to find out what those defaults are. - -==== Authorization - -You must have `monitor_ml`, `monitor`, `manage_ml`, or `manage` cluster -privileges to use this API. The `machine_learning_admin` and `machine_learning_user` -roles provide these privileges. For more information, see -{stack-ov}/security-privileges.html[Security Privileges] and -{stack-ov}/built-in-roles.html[Built-in Roles]. - - -==== Examples +[[get-ml-info-example]] +==== {api-examples-title} The endpoint takes no arguments: @@ -57,7 +59,13 @@ This is a possible response: } }, "upgrade_mode": false, + "native_code" : { + "version": "7.0.0", + "build_hash": "99a07c016d5a73" + }, "limits" : { } } ---- // TESTRESPONSE[s/"upgrade_mode": false/"upgrade_mode": $body.upgrade_mode/] +// TESTRESPONSE[s/"version": "7.0.0",/"version": "$body.native_code.version",/] +// TESTRESPONSE[s/"build_hash": "99a07c016d5a73"/"build_hash": "$body.native_code.build_hash"/] diff --git a/docs/reference/ml/apis/get-overall-buckets.asciidoc b/docs/reference/ml/apis/get-overall-buckets.asciidoc index d8592e6516bbb..4d8287f9a54f7 100644 --- a/docs/reference/ml/apis/get-overall-buckets.asciidoc +++ b/docs/reference/ml/apis/get-overall-buckets.asciidoc @@ -9,7 +9,8 @@ Retrieves overall bucket results that summarize the bucket results of multiple jobs. -==== Request +[[ml-get-overall-buckets-request]] +==== {api-request-title} `GET _ml/anomaly_detectors//results/overall_buckets` + @@ -17,7 +18,18 @@ bucket results of multiple jobs. `GET _ml/anomaly_detectors/_all/results/overall_buckets` -==== Description +[[ml-get-overall-buckets-prereqs]] +==== {api-prereq-title} + +* If the {es} {security-features} are enabled, you must have `monitor_ml`, +`monitor`, `manage_ml`, or `manage` cluster privileges to use this API. You also +need `read` index privilege on the index that stores the results. The +`machine_learning_admin` and `machine_learning_user` roles provide these +privileges. See {stack-ov}/security-privileges.html[Security privileges] and +{stack-ov}/built-in-roles.html[Built-in roles]. + +[[ml-get-overall-buckets-desc]] +==== {api-description-title} You can summarize the bucket results for all jobs by using `_all` or by specifying `*` as the ``. @@ -41,43 +53,46 @@ to request overall buckets that span longer than the largest job's `bucket_span` When set, the `overall_score` will be the max `overall_score` of the corresponding overall buckets with a span equal to the largest job's `bucket_span`. -==== Path Parameters +[[ml-get-overall-buckets-path-parms]] +==== {api-path-parms-title} -`job_id`:: +`` (Required):: (string) Identifier for the job. It can be a job identifier, a group name, a comma-separated list of jobs or groups, or a wildcard expression. -==== Request Body +[[ml-get-overall-buckets-request-body]] +==== {api-request-body-title} -`allow_no_jobs`:: +`allow_no_jobs` (Optional):: (boolean) If `false` and the `job_id` does not match any job an error will be returned. The default value is `true`. -`bucket_span`:: +`bucket_span` (Optional):: (string) The span of the overall buckets. Must be greater or equal to the largest job's `bucket_span`. Defaults to the largest job's `bucket_span`. -`end`:: +`end` (Optional):: (string) Returns overall buckets with timestamps earlier than this time. -`exclude_interim`:: +`exclude_interim` (Optional):: (boolean) If `true`, the output excludes interim overall buckets. Overall buckets are interim if any of the job buckets within the overall bucket interval are interim. By default, interim results are included. -`overall_score`:: - (double) Returns overall buckets with overall scores greater or equal than this value. +`overall_score` (Optional):: + (double) Returns overall buckets with overall scores greater or equal than + this value. -`start`:: +`start` (Optional):: (string) Returns overall buckets with timestamps after this time. -`top_n`:: +`top_n` (Optional):: (integer) The number of top job bucket scores to be used in the `overall_score` calculation. The default value is `1`. - -===== Results +[[ml-get-overall-buckets-results]] +==== {api-response-body-title} The API returns the following information: @@ -85,18 +100,8 @@ The API returns the following information: (array) An array of overall bucket objects. For more information, see <>. - -==== Authorization - -You must have `monitor_ml`, `monitor`, `manage_ml`, or `manage` cluster -privileges to use this API. You also need `read` index privilege on the index -that stores the results. The `machine_learning_admin` and `machine_learning_user` -roles provide these privileges. For more information, see -{xpack-ref}/security-privileges.html[Security Privileges] and -{xpack-ref}/built-in-roles.html[Built-in Roles]. - - -==== Examples +[[ml-get-overall-buckets-example]] +==== {api-examples-title} The following example gets overall buckets for jobs with IDs matching `job-*`: diff --git a/docs/reference/ml/apis/get-record.asciidoc b/docs/reference/ml/apis/get-record.asciidoc index afc7d2733c872..0acc3e0e49fce 100644 --- a/docs/reference/ml/apis/get-record.asciidoc +++ b/docs/reference/ml/apis/get-record.asciidoc @@ -8,49 +8,58 @@ Retrieves anomaly records for a job. - -==== Request +[[ml-get-record-request]] +==== {api-request-title} `GET _ml/anomaly_detectors//results/records` -//===== Description +[[ml-get-record-prereqs]] +==== {api-prereq-title} -==== Path Parameters +* If the {es} {security-features} are enabled, you must have `monitor_ml`, +`monitor`, `manage_ml`, or `manage` cluster privileges to use this API. You also +need `read` index privilege on the index that stores the results. The +`machine_learning_admin` and `machine_learning_user` roles provide these +privileges. See {stack-ov}/security-privileges.html[Security privileges] and +{stack-ov}/built-in-roles.html[Built-in roles]. -`job_id`:: - (string) Identifier for the job. +[[ml-get-record-path-parms]] +==== {api-path-parms-title} +`job_id` (Required):: + (string) Identifier for the job. -==== Request Body +[[ml-get-record-request-body]] +==== {api-request-body-title} -`desc`:: +`desc` (Optional):: (boolean) If true, the results are sorted in descending order. -`end`:: +`end` (Optional):: (string) Returns records with timestamps earlier than this time. -`exclude_interim`:: +`exclude_interim` (Optional):: (boolean) If true, the output excludes interim results. By default, interim results are included. -`page`:: +`page` (Optional):: `from`::: (integer) Skips the specified number of records. `size`::: (integer) Specifies the maximum number of records to obtain. -`record_score`:: +`record_score` (Optional):: (double) Returns records with anomaly scores greater or equal than this value. -`sort`:: +`sort` (Optional):: (string) Specifies the sort field for the requested records. By default, the records are sorted by the `anomaly_score` value. -`start`:: +`start` (Optional):: (string) Returns records with timestamps after this time. - -==== Results +[[ml-get-record-results]] +==== {api-response-body-title} The API returns the following information: @@ -58,19 +67,8 @@ The API returns the following information: (array) An array of record objects. For more information, see <>. - -==== Authorization - -You must have `monitor_ml`, `monitor`, `manage_ml`, or `manage` cluster -privileges to use this API. You also need `read` index privilege on the index -that stores the results. The `machine_learning_admin` and `machine_learning_user` -roles provide these privileges. For more information, see -{xpack-ref}/security-privileges.html[Security Privileges] and -{xpack-ref}/built-in-roles.html[Built-in Roles]. -//<> and <>. - - -==== Examples +[[ml-get-record-example]] +==== {api-examples-title} The following example gets record information for the `it-ops-kpi` job: diff --git a/docs/reference/ml/apis/get-snapshot.asciidoc b/docs/reference/ml/apis/get-snapshot.asciidoc index 4935a6e2d238f..ea1b15df33f33 100644 --- a/docs/reference/ml/apis/get-snapshot.asciidoc +++ b/docs/reference/ml/apis/get-snapshot.asciidoc @@ -8,47 +8,54 @@ Retrieves information about model snapshots. - -==== Request +[[ml-get-snapshot-request]] +==== {api-request-title} `GET _ml/anomaly_detectors//model_snapshots` + `GET _ml/anomaly_detectors//model_snapshots/` -//===== Description +[[ml-get-snapshot-prereqs]] +==== {api-prereq-title} + +* If the {es} {security-features} are enabled, you must have `monitor_ml`, +`monitor`, `manage_ml`, or `manage` cluster privileges to use this API. See +{stack-ov}/security-privileges.html[Security privileges]. -==== Path Parameters +[[ml-get-snapshot-path-parms]] +==== {api-path-parms-title} -`job_id`:: +`` (Required):: (string) Identifier for the job. -`snapshot_id`:: +`` (Optional):: (string) Identifier for the model snapshot. If you do not specify this optional parameter, the API returns information about all model snapshots. -==== Request Body +[[ml-get-snapshot-request-body]] +==== {api-request-body-title} -`desc`:: +`desc` (Optional):: (boolean) If true, the results are sorted in descending order. -`end`:: +`end` (Optional):: (date) Returns snapshots with timestamps earlier than this time. -`from`:: +`from` (Optional):: (integer) Skips the specified number of snapshots. -`size`:: +`size` (Optional):: (integer) Specifies the maximum number of snapshots to obtain. -`sort`:: +`sort` (Optional):: (string) Specifies the sort field for the requested snapshots. By default, the snapshots are sorted by their timestamp. -`start`:: +`start` (Optional):: (string) Returns snapshots with timestamps after this time. - -==== Results +[[ml-get-snapshot-results]] +==== {api-response-body-title} The API returns the following information: @@ -56,16 +63,8 @@ The API returns the following information: (array) An array of model snapshot objects. For more information, see <>. - -==== Authorization - -You must have `monitor_ml`, `monitor`, `manage_ml`, or `manage` cluster -privileges to use this API. For more information, see -{xpack-ref}/security-privileges.html[Security Privileges]. -//<>. - - -==== Examples +[[ml-get-snapshot-example]] +==== {api-examples-title} The following example gets model snapshot information for the `it_ops_new_logs` job: diff --git a/docs/reference/ml/apis/jobcounts.asciidoc b/docs/reference/ml/apis/jobcounts.asciidoc index ed69a8b0a796b..3b92224034b60 100644 --- a/docs/reference/ml/apis/jobcounts.asciidoc +++ b/docs/reference/ml/apis/jobcounts.asciidoc @@ -19,11 +19,15 @@ progress of a job. `model_size_stats`:: (object) An object that provides information about the size and contents of the model. - See <> + See <>. `forecasts_stats`:: (object) An object that provides statistical information about forecasts - of this job. See <> + of this job. See <>. + +`timing_stats`:: + (object) An object that provides statistical information about timing aspect + of this job. See <>. `node`:: (object) For open jobs only, contains information about the node where the @@ -209,6 +213,31 @@ The `forecasts_stats` object shows statistics about forecasts. It has the follow NOTE: `memory_bytes`, `records`, `processing_time_ms` and `status` require at least 1 forecast, otherwise these fields are omitted. +[float] +[[ml-timingstats]] +==== Timing Stats Objects + +The `timing_stats` object shows timing-related statistics about the job's progress. It has the following properties: + +`job_id`:: + (string) A numerical character string that uniquely identifies the job. + +`bucket_count`:: + (long) The number of buckets processed. + +`minimum_bucket_processing_time_ms`:: + (double) Minimum among all bucket processing times in milliseconds. + +`maximum_bucket_processing_time_ms`:: + (double) Maximum among all bucket processing times in milliseconds. + +`average_bucket_processing_time_ms`:: + (double) Average of all bucket processing times in milliseconds. + +`exponential_average_bucket_processing_time_ms`:: + (double) Exponential moving average of all bucket processing times in milliseconds. + + [float] [[ml-stats-node]] ==== Node Objects diff --git a/docs/reference/ml/apis/open-job.asciidoc b/docs/reference/ml/apis/open-job.asciidoc index 08c7b97d9c050..84000cb89b0de 100644 --- a/docs/reference/ml/apis/open-job.asciidoc +++ b/docs/reference/ml/apis/open-job.asciidoc @@ -10,41 +10,42 @@ Opens one or more jobs. A job must be opened in order for it to be ready to receive and analyze data. A job can be opened and closed multiple times throughout its lifecycle. - -==== Request +[[ml-open-job-request]] +==== {api-request-title} `POST _ml/anomaly_detectors/{job_id}/_open` +[[ml-open-job-prereqs]] +==== {api-prereq-title} -==== Description - -When you open a new job, it starts with an empty model. +* If the {es} {security-features} are enabled, you must have `manage_ml` or +`manage` cluster privileges to use this API. See +{stack-ov}/security-privileges.html[Security privileges]. -When you open an existing job, the most recent model state is automatically loaded. -The job is ready to resume its analysis from where it left off, once new data is received. +[[ml-open-job-desc]] +==== {api-description-title} +When you open a new job, it starts with an empty model. -==== Path Parameters +When you open an existing job, the most recent model state is automatically +loaded. The job is ready to resume its analysis from where it left off, once new +data is received. -`job_id` (required):: -(string) Identifier for the job +[[ml-open-job-path-parms]] +==== {api-path-parms-title} +`` (Required):: + (string) Identifier for the job -==== Request Body +[[ml-open-job-request-body]] +==== {api-request-body-title} -`timeout`:: +`timeout` (Optional):: (time) Controls the time to wait until a job has opened. The default value is 30 minutes. - -==== Authorization - -You must have `manage_ml`, or `manage` cluster privileges to use this API. -For more information, see -{xpack-ref}/security-privileges.html[Security Privileges]. - - -==== Examples +[[ml-open-job-example]] +==== {api-examples-title} The following example opens the `total-requests` job and sets an optional property: diff --git a/docs/reference/ml/apis/post-calendar-event.asciidoc b/docs/reference/ml/apis/post-calendar-event.asciidoc index 5d122a5d6d1a8..88d771f3b7f18 100644 --- a/docs/reference/ml/apis/post-calendar-event.asciidoc +++ b/docs/reference/ml/apis/post-calendar-event.asciidoc @@ -8,38 +8,40 @@ Posts scheduled events in a calendar. -==== Request +[[ml-post-calendar-event-request]] +==== {api-request-title} `POST _ml/calendars//events` +[[ml-post-calendar-event-prereqs]] +==== {api-prereq-title} -==== Description +* If the {es} {security-features} are enabled, you must have `manage_ml` or +`manage` cluster privileges to use this API. See +{stack-ov}/security-privileges.html[Security privileges]. -This API accepts a list of {xpack-ref}/ml-calendars.html[scheduled events], each +[[ml-post-calendar-event-desc]] +==== {api-description-title} + +This API accepts a list of {stack-ov}/ml-calendars.html[scheduled events], each of which must have a start time, end time, and description. -==== Path Parameters +[[ml-post-calendar-event-path-parms]] +==== {api-path-parms-title} -`calendar_id` (required):: +`` (Required):: (string) Identifier for the calendar. +[[ml-post-calendar-event-request-body]] +==== {api-request-body-title} -==== Request Body - -`events`:: - (array) A list of one of more scheduled events. The event's start and end times - may be specified as integer milliseconds since the epoch or as a string in ISO 8601 - format. See <>. - - -==== Authorization - -You must have `manage_ml`, or `manage` cluster privileges to use this API. -For more information, see -{xpack-ref}/security-privileges.html[Security Privileges]. - +`events` (Required):: + (array) A list of one of more scheduled events. The event's start and end + times may be specified as integer milliseconds since the epoch or as a string + in ISO 8601 format. See <>. -==== Examples +[[ml-post-calendar-event-example]] +==== {api-examples-title} You can add scheduled events to the `planned-outages` calendar as follows: diff --git a/docs/reference/ml/apis/post-data.asciidoc b/docs/reference/ml/apis/post-data.asciidoc index 2df0df69e9030..3c2d0e49fde93 100644 --- a/docs/reference/ml/apis/post-data.asciidoc +++ b/docs/reference/ml/apis/post-data.asciidoc @@ -8,13 +8,20 @@ Sends data to an anomaly detection job for analysis. - -==== Request +[[ml-post-data-request]] +==== {api-request-title} `POST _ml/anomaly_detectors//_data` +[[ml-post-data-prereqs]] +==== {api-prereq-title} + +* If the {es} {security-features} are enabled, you must have `manage_ml` or +`manage` cluster privileges to use this API. See +{stack-ov}/security-privileges.html[Security privileges]. -==== Description +[[ml-post-data-desc]] +==== {api-description-title} The job must have a state of `open` to receive and process the data. @@ -42,39 +49,32 @@ IMPORTANT: For each job, data can only be accepted from a single connection at a time. It is not currently possible to post data to multiple jobs using wildcards or a comma-separated list. +[[ml-post-data-path-parms]] +==== {api-path-parms-title} -==== Path Parameters - -`job_id` (required):: - (string) Identifier for the job - - -==== Query Parameters +`` (Required):: + (string) Identifier for the job. -`reset_start`:: - (string) Specifies the start of the bucket resetting range +[[ml-post-data-query-parms]] +==== {api-query-parms-title} -`reset_end`:: - (string) Specifies the end of the bucket resetting range +`reset_start` (Optional):: + (string) Specifies the start of the bucket resetting range. +`reset_end` (Optional):: + (string) Specifies the end of the bucket resetting range. -==== Request Body +[[ml-post-data-request-body]] +==== {api-request-body-title} A sequence of one or more JSON documents containing the data to be analyzed. Only whitespace characters are permitted in between the documents. +[[ml-post-data-example]] +==== {api-examples-title} -==== Authorization - -You must have `manage_ml`, or `manage` cluster privileges to use this API. -For more information, see -{xpack-ref}/security-privileges.html[Security Privileges]. -//<>. - - -==== Examples - -The following example posts data from the it_ops_new_kpi.json file to the `it_ops_new_kpi` job: +The following example posts data from the `it_ops_new_kpi.json` file to the +`it_ops_new_kpi` job: [source,js] -------------------------------------------------- @@ -83,8 +83,8 @@ $ curl -s -H "Content-type: application/json" --data-binary @it_ops_new_kpi.json -------------------------------------------------- -When the data is sent, you receive information about the operational progress of the job. -For example: +When the data is sent, you receive information about the operational progress of +the job. For example: [source,js] ---- diff --git a/docs/reference/ml/apis/preview-datafeed.asciidoc b/docs/reference/ml/apis/preview-datafeed.asciidoc index 83af6a78057cc..4ca3ebcd10e40 100644 --- a/docs/reference/ml/apis/preview-datafeed.asciidoc +++ b/docs/reference/ml/apis/preview-datafeed.asciidoc @@ -10,45 +10,41 @@ Previews a {dfeed}. - -==== Request +[[ml-preview-datafeed-request]] +==== {api-request-title} `GET _ml/datafeeds//_preview` +[[ml-preview-datafeed-prereqs]] +==== {api-prereq-title} + +* If {es} {security-features} are enabled, you must have `monitor_ml`, `monitor`, +`manage_ml`, or `manage` cluster privileges to use this API. See +{stack-ov}/security-privileges.html[Security privileges]. -==== Description +[[ml-preview-datafeed-desc]] +==== {api-description-title} The preview {dfeeds} API returns the first "page" of results from the `search` that is created by using the current {dfeed} settings. This preview shows the structure of the data that will be passed to the anomaly detection engine. - -==== Path Parameters - -`datafeed_id` (required):: - (string) Identifier for the {dfeed} - - -==== Authorization - -If {es} {security-features} are enabled, you must have `monitor_ml`, `monitor`, -`manage_ml`, or `manage` cluster privileges to use this API. For more -information, see -{stack-ov}/security-privileges.html[Security Privileges]. - - -==== Security Integration - -When {es} {security-features} are enabled, the {dfeed} query is previewed using -the credentials of the user calling the preview {dfeed} API. When the {dfeed} -is started it runs the query using the roles of the last user to -create or update it. If the two sets of roles differ then the preview may -not accurately reflect what the {dfeed} will return when started. To avoid +IMPORTANT: When {es} {security-features} are enabled, the {dfeed} query is +previewed using the credentials of the user calling the preview {dfeed} API. +When the {dfeed} is started it runs the query using the roles of the last user +to create or update it. If the two sets of roles differ then the preview may +not accurately reflect what the {dfeed} will return when started. To avoid such problems, the same user that creates/updates the {dfeed} should preview it to ensure it is returning the expected data. +[[ml-preview-datafeed-path-parms]] +==== {api-path-parms-title} + +`` (Required):: + (string) Identifier for the {dfeed}. -==== Examples +[[ml-preview-datafeed-example]] +==== {api-examples-title} The following example obtains a preview of the `datafeed-farequote` {dfeed}: diff --git a/docs/reference/ml/apis/put-calendar-job.asciidoc b/docs/reference/ml/apis/put-calendar-job.asciidoc index cafc5f670627c..0a1ee2fcc6de0 100644 --- a/docs/reference/ml/apis/put-calendar-job.asciidoc +++ b/docs/reference/ml/apis/put-calendar-job.asciidoc @@ -8,28 +8,30 @@ Adds a job to a calendar. -==== Request +[[ml-put-calendar-job-request]] +==== {api-request-title} `PUT _ml/calendars//jobs/` +[[ml-put-calendar-job-prereqs]] +==== {api-prereq-title} -==== Path Parameters +* If the {es} {security-features} are enabled, you must have `manage_ml` or +`manage` cluster privileges to use this API. See +{stack-ov}/security-privileges.html[Security privileges]. -`calendar_id` (required):: - (string) Identifier for the calendar. - -`job_id` (required):: - (string) An identifier for the job. It can be a job identifier, a group name, or a - comma-separated list of jobs or groups. +[[ml-put-calendar-job-path-parms]] +==== {api-path-parms-title} -==== Authorization - -You must have `manage_ml`, or `manage` cluster privileges to use this API. -For more information, see -{xpack-ref}/security-privileges.html[Security Privileges]. +`` (Required):: + (string) Identifier for the calendar. +`` (Required):: + (string) An identifier for the job. It can be a job identifier, a group name, + or a comma-separated list of jobs or groups. -==== Examples +[[ml-put-calendar-job-example]] +==== {api-examples-title} The following example associates the `planned-outages` calendar with the `total-requests` job: diff --git a/docs/reference/ml/apis/put-calendar.asciidoc b/docs/reference/ml/apis/put-calendar.asciidoc index 9b1e781e3cc06..f98dd541d6753 100644 --- a/docs/reference/ml/apis/put-calendar.asciidoc +++ b/docs/reference/ml/apis/put-calendar.asciidoc @@ -8,35 +8,38 @@ Instantiates a calendar. -==== Request +[[ml-put-calendar-request]] +==== {api-request-title} `PUT _ml/calendars/` -===== Description +[[ml-put-calendar-prereqs]] +==== {api-prereq-title} + +* If the {es} {security-features} are enabled, you must have `manage_ml` or +`manage` cluster privileges to use this API. See +{stack-ov}/security-privileges.html[Security privileges]. + +[[ml-put-calendar-desc]] +==== {api-description-title} For more information, see -{xpack-ref}/ml-calendars.html[Calendars and Scheduled Events]. +{stack-ov}/ml-calendars.html[Calendars and Scheduled Events]. -==== Path Parameters +[[ml-put-calendar-path-parms]] +==== {api-path-parms-title} -`calendar_id` (required):: +`` (Required):: (string) Identifier for the calendar. +[[ml-put-calendar-request-body]] +==== {api-request-body-title} -==== Request Body - -`description`:: +`description` (Optional):: (string) A description of the calendar. - -==== Authorization - -You must have `manage_ml`, or `manage` cluster privileges to use this API. -For more information, see -{xpack-ref}/security-privileges.html[Security Privileges]. - - -==== Examples +[[ml-put-calendar-example]] +==== {api-examples-title} The following example creates the `planned-outages` calendar: diff --git a/docs/reference/ml/apis/put-datafeed.asciidoc b/docs/reference/ml/apis/put-datafeed.asciidoc index 52728dd093da5..6c4578abb1671 100644 --- a/docs/reference/ml/apis/put-datafeed.asciidoc +++ b/docs/reference/ml/apis/put-datafeed.asciidoc @@ -10,101 +10,100 @@ Instantiates a {dfeed}. - -==== Request +[[ml-put-datafeed-request]] +==== {api-request-title} `PUT _ml/datafeeds/` +[[ml-put-datafeed-prereqs]] +==== {api-prereq-title} + +* If {es} {security-features} are enabled, you must have `manage_ml` or `manage` +cluster privileges to use this API. See +{stack-ov}/security-privileges.html[Security privileges]. -==== Description +[[ml-put-datafeed-desc]] +==== {api-description-title} You must create a job before you create a {dfeed}. You can associate only one {dfeed} to each job. -IMPORTANT: You must use {kib} or this API to create a {dfeed}. Do not put a {dfeed} - directly to the `.ml-config` index using the Elasticsearch index API. - If {es} {security-features} are enabled, do not give users `write` - privileges on the `.ml-config` index. - - -==== Path Parameters - -`feed_id` (required):: +[IMPORTANT] +==== +* You must use {kib} or this API to create a {dfeed}. Do not put a +{dfeed} directly to the `.ml-config` index using the {es} index API. If {es} +{security-features} are enabled, do not give users `write` privileges on the +`.ml-config` index. +* When {es} {security-features} are enabled, your {dfeed} remembers which roles +the user who created it had at the time of creation and runs the query using +those same roles. +==== + +[[ml-put-datafeed-path-parms]] +==== {api-path-parms-title} + +`` (Required):: (string) A numerical character string that uniquely identifies the {dfeed}. This identifier can contain lowercase alphanumeric characters (a-z and 0-9), hyphens, and underscores. It must start and end with alphanumeric characters. +[[ml-put-datafeed-request-body]] +==== {api-request-body-title} -==== Request Body - -`aggregations`:: +`aggregations` (Optional):: (object) If set, the {dfeed} performs aggregation searches. For more information, see <>. -`chunking_config`:: +`chunking_config` (Optional):: (object) Specifies how data searches are split into time chunks. See <>. -`frequency`:: +`delayed_data_check_config` (Optional):: + (object) Specifies whether the data feed checks for missing data and + the size of the window. See + <>. + +`frequency` (Optional):: (time units) The interval at which scheduled queries are made while the {dfeed} runs in real time. The default value is either the bucket span for short bucket spans, or, for longer bucket spans, a sensible fraction of the bucket span. For example: `150s`. -`indices` (required):: +`indices` (Required):: (array) An array of index names. Wildcards are supported. For example: `["it_ops_metrics", "server*"]`. -`job_id` (required):: +`job_id` (Required):: (string) A numerical character string that uniquely identifies the job. -`query`:: +`query` (Optional):: (object) The {es} query domain-specific language (DSL). This value corresponds to the query object in an {es} search POST body. All the options that are supported by {Es} can be used, as this object is passed verbatim to {es}. By default, this property has the following value: `{"match_all": {"boost": 1}}`. -`query_delay`:: +`query_delay` (Optional):: (time units) The number of seconds behind real time that data is queried. For example, if data from 10:04 a.m. might not be searchable in {es} until 10:06 a.m., set this property to 120 seconds. The default value is `60s`. -`script_fields`:: +`script_fields` (Optional):: (object) Specifies scripts that evaluate custom expressions and returns script fields to the {dfeed}. The <> in a job can contain - functions that use these script fields. - For more information, + functions that use these script fields. For more information, see {ref}/search-request-script-fields.html[Script Fields]. -`scroll_size`:: +`scroll_size` (Optional):: (unsigned integer) The `size` parameter that is used in {es} searches. The default value is `1000`. -`delayed_data_check_config`:: - (object) Specifies if and with how large a window should the data feed check - for missing data. See <>. - For more information about these properties, see <>. - -==== Authorization - -If {es} {security-features} are enabled, you must have `manage_ml`, or `manage` -cluster privileges to use this API. For more information, see -{stack-ov}/security-privileges.html[Security Privileges]. - - -==== Security integration - -When {es} {security-features} are enabled, your {dfeed} remembers which roles the -user who created it had at the time of creation and runs the query using those -same roles. - - -==== Examples +[[ml-put-datafeed-example]] +==== {api-examples-title} The following example creates the `datafeed-total-requests` {dfeed}: diff --git a/docs/reference/ml/apis/put-filter.asciidoc b/docs/reference/ml/apis/put-filter.asciidoc index abe52dfb13b25..ad0d6d34ea81d 100644 --- a/docs/reference/ml/apis/put-filter.asciidoc +++ b/docs/reference/ml/apis/put-filter.asciidoc @@ -8,42 +8,45 @@ Instantiates a filter. -==== Request +[[ml-put-filter-request]] +==== {api-request-title} `PUT _ml/filters/` -===== Description +[[ml-put-filter-prereqs]] +==== {api-prereq-title} + +* If the {es} {security-features} are enabled, you must have `manage_ml` or +`manage` cluster privileges to use this API. See +{stack-ov}/security-privileges.html[Security privileges]. + +[[ml-put-filter-desc]] +==== {api-description-title} A {stack-ov}/ml-rules.html[filter] contains a list of strings. It can be used by one or more jobs. Specifically, filters are referenced in the `custom_rules` property of <>. -==== Path Parameters +[[ml-put-filter-path-parms]] +==== {api-path-parms-title} -`filter_id` (required):: +`` (Required):: (string) Identifier for the filter. +[[ml-put-filter-request-body]] +==== {api-request-body-title} -==== Request Body - -`description`:: +`description` (Optional):: (string) A description of the filter. -`items`:: +`items` (Required):: (array of strings) The items of the filter. A wildcard `*` can be used at the beginning or the end of an item. Up to 10000 items are allowed in each filter. - -==== Authorization - -You must have `manage_ml`, or `manage` cluster privileges to use this API. -For more information, see -{xpack-ref}/security-privileges.html[Security Privileges]. - - -==== Examples +[[ml-put-filter-example]] +==== {api-examples-title} The following example creates the `safe_domains` filter: diff --git a/docs/reference/ml/apis/put-job.asciidoc b/docs/reference/ml/apis/put-job.asciidoc index e3d80c276dc55..c60de488180d9 100644 --- a/docs/reference/ml/apis/put-job.asciidoc +++ b/docs/reference/ml/apis/put-job.asciidoc @@ -8,84 +8,87 @@ Instantiates a job. -==== Request +[[ml-put-job-request]] +==== {api-request-title} `PUT _ml/anomaly_detectors/` -===== Description +[[ml-put-job-prereqs]] +==== {api-prereq-title} + +* If the {es} {security-features} are enabled, you must have `manage_ml` or +`manage` cluster privileges to use this API. See +{stack-ov}/security-privileges.html[Security privileges]. + +[[ml-put-job-desc]] +==== {api-description-title} IMPORTANT: You must use {kib} or this API to create a {ml} job. Do not put a job directly to the `.ml-config` index using the Elasticsearch index API. If {es} {security-features} are enabled, do not give users `write` privileges on the `.ml-config` index. +[[ml-put-job-path-parms]] +==== {api-path-parms-title} -==== Path Parameters - -`job_id` (required):: +`` (Required):: (string) Identifier for the job. This identifier can contain lowercase alphanumeric characters (a-z and 0-9), hyphens, and underscores. It must start and end with alphanumeric characters. +[[ml-put-job-request-body]] +==== {api-request-body-title} -==== Request Body - -`analysis_config`:: +`analysis_config` (Required):: (object) The analysis configuration, which specifies how to analyze the data. See <>. -`analysis_limits`:: +`analysis_limits` (Optional):: (object) Specifies runtime limits for the job. See <>. -`background_persist_interval`:: +`background_persist_interval` (Optional):: (time units) Advanced configuration option. The time between each periodic persistence of the model. See <>. -`custom_settings`:: +`custom_settings` (Optional):: (object) Advanced configuration option. Contains custom meta data about the job. See <>. -`data_description` (required):: +`data_description` (Required):: (object) Describes the format of the input data. This object is required, but it can be empty (`{}`). See <>. -`description`:: +`description` (Optional):: (string) A description of the job. -`groups`:: +`groups` (Optional):: (array of strings) A list of job groups. See <>. -`model_plot_config`:: +`model_plot_config` (Optional):: (object) Advanced configuration option. Specifies to store model information along with the results. This adds overhead to the performance of the system and is not feasible for jobs with many entities, see <>. -`model_snapshot_retention_days`:: +`model_snapshot_retention_days` (Optional):: (long) The time in days that model snapshots are retained for the job. Older snapshots are deleted. The default value is `1`, which means snapshots are retained for one day (twenty-four hours). -`renormalization_window_days`:: +`renormalization_window_days` (Optional):: (long) Advanced configuration option. The period over which adjustments to the score are applied, as new data is seen. See <>. -`results_index_name`:: +`results_index_name` (Optional):: (string) A text string that affects the name of the {ml} results index. The default value is `shared`, which generates an index named `.ml-anomalies-shared`. -`results_retention_days`:: +`results_retention_days` (Optional):: (long) Advanced configuration option. The number of days for which job results are retained. See <>. -==== Authorization - -You must have `manage_ml`, or `manage` cluster privileges to use this API. -For more information, see -{xpack-ref}/security-privileges.html[Security Privileges]. - - -==== Examples +[[ml-put-job-example]] +==== {api-examples-title} The following example creates the `total-requests` job: diff --git a/docs/reference/ml/apis/revert-snapshot.asciidoc b/docs/reference/ml/apis/revert-snapshot.asciidoc index b560f7b041206..86d3d4c14a93c 100644 --- a/docs/reference/ml/apis/revert-snapshot.asciidoc +++ b/docs/reference/ml/apis/revert-snapshot.asciidoc @@ -8,12 +8,20 @@ Reverts to a specific snapshot. -==== Request +[[ml-revert-snapshot-request]] +==== {api-request-title} `POST _ml/anomaly_detectors//model_snapshots//_revert` +[[ml-revert-snapshot-prereqs]] +==== {api-prereq-title} -==== Description +* If the {es} {security-features} are enabled, you must have `manage_ml` or +`manage` cluster privileges to use this API. See +{stack-ov}/security-privileges.html[Security privileges]. + +[[ml-revert-snapshot-desc]] +==== {api-description-title} The {ml} feature in {xpack} reacts quickly to anomalous input, learning new behaviors in data. Highly anomalous input increases the variance in the models @@ -25,18 +33,19 @@ Friday or a critical system failure. IMPORTANT: Before you revert to a saved snapshot, you must close the job. +[[ml-revert-snapshot-path-parms]] +==== {api-path-parms-title} -==== Path Parameters - -`job_id` (required):: - (string) Identifier for the job +`` (Required):: + (string) Identifier for the job. -`snapshot_id` (required):: - (string) Identifier for the model snapshot +`` (Required):: + (string) Identifier for the model snapshot. -==== Request Body +[[ml-revert-snapshot-request-body]] +==== {api-request-body-title} -`delete_intervening_results`:: +`delete_intervening_results` (Optional):: (boolean) If true, deletes the results in the time period between the latest results and the time of the reverted snapshot. It also resets the model to accept records for this time period. The default value is false. @@ -45,15 +54,8 @@ NOTE: If you choose not to delete intervening results when reverting a snapshot, the job will not accept input data that is older than the current time. If you want to resend data, then delete the intervening results. - -==== Authorization - -You must have `manage_ml`, or `manage` cluster privileges to use this API. -For more information, see -{xpack-ref}/security-privileges.html[Security Privileges]. - - -==== Examples +[[ml-revert-snapshot-example]] +==== {api-examples-title} The following example reverts to the `1491856080` snapshot for the `it_ops_new_kpi` job: diff --git a/docs/reference/ml/apis/set-upgrade-mode.asciidoc b/docs/reference/ml/apis/set-upgrade-mode.asciidoc index 5434d70d4e61e..6a00656430c66 100644 --- a/docs/reference/ml/apis/set-upgrade-mode.asciidoc +++ b/docs/reference/ml/apis/set-upgrade-mode.asciidoc @@ -9,7 +9,8 @@ Sets a cluster wide upgrade_mode setting that prepares {ml} indices for an upgrade. -==== Request +[[ml-set-upgrade-mode-request]] +==== {api-request-title} ////////////////////////// [source,js] @@ -25,7 +26,15 @@ POST /_ml/set_upgrade_mode?enabled=false&timeout=10m `POST _ml/set_upgrade_mode` -==== Description +[[ml-set-upgrade-mode-prereqs]] +==== {api-prereq-title} + +* If the {es} {security-features} are enabled, you must have `manage_ml` or +`manage` cluster privileges to use this API. See +{stack-ov}/security-privileges.html[Security privileges]. + +[[ml-set-upgrade-mode-desc]] +==== {api-description-title} When upgrading your cluster, in some circumstances you must restart your nodes and reindex your {ml} indices. In those circumstances, there must be no {ml} jobs running. @@ -37,7 +46,6 @@ though stopping jobs is not a requirement in that case. For more information, see {stack-ref}/upgrading-elastic-stack.html[Upgrading the {stack}]. - When `enabled=true` this API temporarily halts all job and {dfeed} tasks and prohibits new job and {dfeed} tasks from starting. @@ -50,23 +58,18 @@ You can see the current value for the `upgrade_mode` setting by using the IMPORTANT: No new {ml} jobs can be opened while the `upgrade_mode` setting is `true`. -==== Query Parameters +[[ml-set-upgrade-mode-query-parms]] +==== {api-query-parms-title} -`enabled`:: +`enabled` (Optional):: (boolean) When `true`, this enables `upgrade_mode`. Defaults to `false` -`timeout`:: +`timeout` (Optional):: (time) The time to wait for the request to be completed. The default value is 30 seconds. -==== Authorization - -You must have `manage_ml`, or `manage` cluster privileges to use this API. -For more information, see -{stack-ov}/security-privileges.html[Security privileges]. - - -==== Examples +[[ml-set-upgrade-mode-example]] +==== {api-examples-title} The following example enables `upgrade_mode` for the cluster: diff --git a/docs/reference/ml/apis/snapshotresource.asciidoc b/docs/reference/ml/apis/snapshotresource.asciidoc index 73e2e9b3b8bc1..13a26e26e8d70 100644 --- a/docs/reference/ml/apis/snapshotresource.asciidoc +++ b/docs/reference/ml/apis/snapshotresource.asciidoc @@ -3,7 +3,7 @@ [[ml-snapshot-resource]] === Model snapshot resources -Model snapshots are saved to disk periodically. +Model snapshots are saved to an internal index within the Elasticsearch cluster. By default, this is occurs approximately every 3 hours to 4 hours and is configurable with the `background_persist_interval` property. diff --git a/docs/reference/ml/apis/start-datafeed.asciidoc b/docs/reference/ml/apis/start-datafeed.asciidoc index aee237b72c837..05cf0766e9522 100644 --- a/docs/reference/ml/apis/start-datafeed.asciidoc +++ b/docs/reference/ml/apis/start-datafeed.asciidoc @@ -12,11 +12,20 @@ Starts one or more {dfeeds}. A {dfeed} must be started in order to retrieve data from {es}. A {dfeed} can be started and stopped multiple times throughout its lifecycle. -==== Request +[[ml-start-datafeed-request]] +==== {api-request-title} `POST _ml/datafeeds//_start` -==== Description +[[ml-start-datafeed-prereqs]] +==== {api-prereq-title} + +* If {es} {security-features} are enabled, you must have `manage_ml` or `manage` +cluster privileges to use this API. See +{stack-ov}/security-privileges.html[Security privileges]. + +[[ml-start-datafeed-desc]] +==== {api-description-title} NOTE: Before you can start a {dfeed}, the job must be open. Otherwise, an error occurs. @@ -56,42 +65,33 @@ If you specify a `start` value that is earlier than the timestamp of the latest processed record, the {dfeed} continues from 1 millisecond after the timestamp of the latest processed record. +IMPORTANT: When {es} {security-features} are enabled, your {dfeed} remembers +which roles the last user to create or update it had at the time of +creation/update and runs the query using those same roles. -==== Path Parameters +[[ml-start-datafeed-path-parms]] +==== {api-path-parms-title} -`feed_id` (required):: -(string) Identifier for the {dfeed} +`` (Required):: + (string) Identifier for the {dfeed}. -==== Request Body +[[ml-start-datafeed-request-body]] +==== {api-request-body-title} -`end`:: +`end` (Optional):: (string) The time that the {dfeed} should end. This value is exclusive. The default value is an empty string. -`start`:: +`start` (Optional):: (string) The time that the {dfeed} should begin. This value is inclusive. The default value is an empty string. -`timeout`:: +`timeout` (Optional):: (time) Controls the amount of time to wait until a {dfeed} starts. The default value is 20 seconds. - -==== Authorization - -If {es} {security-features} are enabled, you must have `manage_ml`, or `manage` -cluster privileges to use this API. For more information, see -{stack-ov}/security-privileges.html[Security Privileges]. - - -==== Security integration - -When {es} {security-features} are enabled, your {dfeed} remembers which roles the -last user to create or update it had at the time of creation/update and runs the -query using those same roles. - - -==== Examples +[[ml-start-datafeed-example]] +==== {api-examples-title} The following example starts the `datafeed-it-ops-kpi` {dfeed}: diff --git a/docs/reference/ml/apis/stop-datafeed.asciidoc b/docs/reference/ml/apis/stop-datafeed.asciidoc index 1489137b9db07..bdac8d51fab04 100644 --- a/docs/reference/ml/apis/stop-datafeed.asciidoc +++ b/docs/reference/ml/apis/stop-datafeed.asciidoc @@ -10,10 +10,8 @@ Stops one or more {dfeeds}. -A {dfeed} that is stopped ceases to retrieve data from {es}. -A {dfeed} can be started and stopped multiple times throughout its lifecycle. - -==== Request +[[ml-stop-datafeed-request]] +==== {api-request-title} `POST _ml/datafeeds//_stop` + @@ -21,39 +19,42 @@ A {dfeed} can be started and stopped multiple times throughout its lifecycle. `POST _ml/datafeeds/_all/_stop` +[[ml-stop-datafeed-prereqs]] +==== {api-prereq-title} + +* If the {es} {security-features} are enabled, you must have `manage_ml` or +`manage` cluster privileges to use this API. See +{stack-ov}/security-privileges.html[Security privileges]. -===== Description +[[ml-stop-datafeed-desc]] +==== {api-description-title} + +A {dfeed} that is stopped ceases to retrieve data from {es}. +A {dfeed} can be started and stopped multiple times throughout its lifecycle. You can stop multiple {dfeeds} in a single API request by using a comma-separated list of {dfeeds} or a wildcard expression. You can close all {dfeeds} by using `_all` or by specifying `*` as the ``. +[[ml-stop-datafeed-path-parms]] +==== {api-path-parms-title} -==== Path Parameters - -`feed_id`:: +`` (Required):: (string) Identifier for the {dfeed}. It can be a {dfeed} identifier or a wildcard expression. +[[ml-stop-datafeed-request-body]] +==== {api-request-body-title} -==== Request Body - -`force`:: +`force` (Optional):: (boolean) If true, the {dfeed} is stopped forcefully. -`timeout`:: +`timeout` (Optional):: (time) Controls the amount of time to wait until a {dfeed} stops. The default value is 20 seconds. - -==== Authorization - -You must have `manage_ml`, or `manage` cluster privileges to use this API. -For more information, see -{xpack-ref}/security-privileges.html[Security Privileges]. - - -==== Examples +[[ml-stop-datafeed-example]] +==== {api-examples-title} The following example stops the `datafeed-total-requests` {dfeed}: diff --git a/docs/reference/ml/apis/update-datafeed.asciidoc b/docs/reference/ml/apis/update-datafeed.asciidoc index a370c1acef9d7..b57088673d845 100644 --- a/docs/reference/ml/apis/update-datafeed.asciidoc +++ b/docs/reference/ml/apis/update-datafeed.asciidoc @@ -10,55 +10,77 @@ Updates certain properties of a {dfeed}. -==== Request +[[ml-update-datafeed-request]] +==== {api-request-title} `POST _ml/datafeeds//_update` -//===== Description +[[ml-update-datafeed-prereqs]] +==== {api-prereq-title} -==== Path Parameters +* If {es} {security-features} are enabled, you must have `manage_ml`, or `manage` +cluster privileges to use this API. See +{stack-ov}/security-privileges.html[Security privileges]. -`feed_id` (required):: - (string) Identifier for the {dfeed} +[[ml-update-datafeed-desc]] +==== {api-description-title} -==== Request Body +If you update the `delayed_data_check_config` property, you must stop and +start the {dfeed} for the change to be applied. + +IMPORTANT: When {es} {security-features} are enabled, your {dfeed} remembers +which roles the user who updated it had at the time of update and runs the query +using those same roles. + +[[ml-update-datafeed-path-parms]] +==== {api-path-parms-title} + +`` (Required):: + (string) Identifier for the {dfeed}. + +[[ml-update-datafeed-request-body]] +==== {api-request-body-title} The following properties can be updated after the {dfeed} is created: -`aggregations`:: +`aggregations` (Optional):: (object) If set, the {dfeed} performs aggregation searches. For more information, see <>. -`chunking_config`:: +`chunking_config` (Optional):: (object) Specifies how data searches are split into time chunks. See <>. + +`delayed_data_check_config` (Optional):: + (object) Specifies whether the data feed checks for missing data and + the size of the window. See <>. -`frequency`:: +`frequency` (Optional):: (time units) The interval at which scheduled queries are made while the {dfeed} runs in real time. The default value is either the bucket span for short bucket spans, or, for longer bucket spans, a sensible fraction of the bucket span. For example: `150s`. -`indices`:: +`indices` (Optional):: (array) An array of index names. Wildcards are supported. For example: `["it_ops_metrics", "server*"]`. -`job_id`:: +`job_id` (Optional):: (string) A numerical character string that uniquely identifies the job. -`query`:: +`query` (Optional):: (object) The {es} query domain-specific language (DSL). This value corresponds to the query object in an {es} search POST body. All the options that are supported by {es} can be used, as this object is passed verbatim to {es}. By default, this property has the following value: `{"match_all": {"boost": 1}}`. -`query_delay`:: +`query_delay` (Optional):: (time units) The number of seconds behind real-time that data is queried. For example, if data from 10:04 a.m. might not be searchable in {es} until 10:06 a.m., set this property to 120 seconds. The default value is `60s`. -`script_fields`:: +`script_fields` (Optional):: (object) Specifies scripts that evaluate custom expressions and returns script fields to the {dfeed}. The <> in a job can contain @@ -66,29 +88,15 @@ The following properties can be updated after the {dfeed} is created: For more information, see {ref}/search-request-script-fields.html[Script Fields]. -`scroll_size`:: +`scroll_size` (Optional):: (unsigned integer) The `size` parameter that is used in {es} searches. The default value is `1000`. For more information about these properties, see <>. - -==== Authorization - -If {es} {security-features} are enabled, you must have `manage_ml`, or `manage` -cluster privileges to use this API. For more information, see -{stack-ov}/security-privileges.html[Security Privileges]. - - -==== Security Integration - -When {es} {security-features} are enabled, your {dfeed} remembers which roles the -user who updated it had at the time of update and runs the query using those -same roles. - - -==== Examples +[[ml-update-datafeed-example]] +==== {api-examples-title} The following example updates the query for the `datafeed-total-requests` {dfeed} so that only log entries of error level are analyzed: diff --git a/docs/reference/ml/apis/update-filter.asciidoc b/docs/reference/ml/apis/update-filter.asciidoc index 45c294a0b8bc6..df8f3056d12cc 100644 --- a/docs/reference/ml/apis/update-filter.asciidoc +++ b/docs/reference/ml/apis/update-filter.asciidoc @@ -8,40 +8,41 @@ Updates the description of a filter, adds items, or removes items. -==== Request +[[ml-update-filter-request]] +==== {api-request-title} `POST _ml/filters//_update` -//==== Description +[[ml-update-filter-prereqs]] +==== {api-prereq-title} -==== Path Parameters +* If the {es} {security-features} are enabled, you must have `manage_ml` or +`manage` cluster privileges to use this API. See +{stack-ov}/security-privileges.html[Security privileges]. -`filter_id` (required):: - (string) Identifier for the filter. +[[ml-update-filter-path-parms]] +==== {api-path-parms-title} +`` (Required):: + (string) Identifier for the filter. -==== Request Body +[[ml-update-filter-request-body]] +==== {api-request-body-title} -`description`:: +`description` (Optional):: (string) A description for the filter. See <>. -`add_items`:: +`add_items` (Optional):: (array of strings) The items to add to the filter. -`remove_items`:: +`remove_items` (Optional):: (array of strings) The items to remove from the filter. +[[ml-update-filter-example]] +==== {api-examples-title} -==== Authorization - -You must have `manage_ml`, or `manage` cluster privileges to use this API. -For more information, see -{xpack-ref}/security-privileges.html[Security Privileges]. - - -==== Examples - -You can change the description, add and remove items to the `safe_domains` filter as follows: +You can change the description, add and remove items to the `safe_domains` +filter as follows: [source,js] -------------------------------------------------- diff --git a/docs/reference/ml/apis/update-job.asciidoc b/docs/reference/ml/apis/update-job.asciidoc index 3382e7fe34675..e78bda613d801 100644 --- a/docs/reference/ml/apis/update-job.asciidoc +++ b/docs/reference/ml/apis/update-job.asciidoc @@ -8,17 +8,27 @@ Updates certain properties of a job. -==== Request +[[ml-update-job-request]] +==== {api-request-title} `POST _ml/anomaly_detectors//_update` +[[ml-update-job-prereqs]] +==== {api-prereq-title} -==== Path Parameters +* If the {es} {security-features} are enabled, you must have `manage_ml` or +`manage` cluster privileges to use this API. See +{stack-ov}/security-privileges.html[Security privileges]. -`job_id` (required):: - (string) Identifier for the job -==== Request Body +[[ml-update-job-path-parms]] +==== {api-path-parms-title} + +`` (Required):: + (string) Identifier for the job. + +[[ml-update-job-request-body]] +==== {api-request-body-title} The following properties can be updated after the job is created: @@ -86,14 +96,8 @@ A detector update object has the following properties: No other detector property can be updated. -==== Authorization - -You must have `manage_ml`, or `manage` cluster privileges to use this API. -For more information, see -{xpack-ref}/security-privileges.html[Security Privileges]. - - -==== Examples +[[ml-update-job-example]] +==== {api-examples-title} The following example updates the `total-requests` job: diff --git a/docs/reference/ml/apis/update-snapshot.asciidoc b/docs/reference/ml/apis/update-snapshot.asciidoc index ffd38f590b1e2..1fe2ed5384bc0 100644 --- a/docs/reference/ml/apis/update-snapshot.asciidoc +++ b/docs/reference/ml/apis/update-snapshot.asciidoc @@ -8,45 +8,45 @@ Updates certain properties of a snapshot. -==== Request +[[ml-update-snapshot-request]] +==== {api-request-title} `POST _ml/anomaly_detectors//model_snapshots//_update` +[[ml-update-snapshot-prereqs]] +==== {api-prereq-title} -//==== Description +* If the {es} {security-features} are enabled, you must have `manage_ml` or +`manage` cluster privileges to use this API. See +{stack-ov}/security-privileges.html[Security privileges]. -==== Path Parameters -`job_id` (required):: - (string) Identifier for the job +[[ml-update-snapshot-path-parms]] +==== {api-path-parms-title} -`snapshot_id` (required):: - (string) Identifier for the model snapshot +`` (Required):: + (string) Identifier for the job. -==== Request Body +`` (Required):: + (string) Identifier for the model snapshot. + +[[ml-update-snapshot-request-body]] +==== {api-request-body-title} The following properties can be updated after the model snapshot is created: -`description`:: - (string) An optional description of the model snapshot. For example, +`description` (Optional):: + (string) A description of the model snapshot. For example, "Before black friday". -`retain`:: +`retain` (Optional):: (boolean) If true, this snapshot will not be deleted during automatic cleanup of snapshots older than `model_snapshot_retention_days`. Note that this snapshot will still be deleted when the job is deleted. The default value is false. - -==== Authorization - -You must have `manage_ml`, or `manage` cluster privileges to use this API. -For more information, see -{xpack-ref}/security-privileges.html[Security Privileges]. -//<>. - - -==== Examples +[[ml-update-snapshot-example]] +==== {api-examples-title} The following example updates the snapshot identified as `1491852978`: diff --git a/docs/reference/ml/apis/validate-detector.asciidoc b/docs/reference/ml/apis/validate-detector.asciidoc index 0f9fe9902e36e..2e5896b95cc93 100644 --- a/docs/reference/ml/apis/validate-detector.asciidoc +++ b/docs/reference/ml/apis/validate-detector.asciidoc @@ -8,30 +8,32 @@ Validates detector configuration information. -==== Request +[[ml-valid-detector-request]] +==== {api-request-title} `POST _ml/anomaly_detectors/_validate/detector` -==== Description +[[ml-valid-detector-prereqs]] +==== {api-prereq-title} + +* If the {es} {security-features} are enabled, you must have `manage_ml` or +`manage` cluster privileges to use this API. See +{stack-ov}/security-privileges.html[Security privileges]. + +[[ml-valid-detector-desc]] +==== {api-description-title} The validate detectors API enables you validate the detector configuration before you create a job. - -==== Request Body +[[ml-valid-detector-request-body]] +==== {api-request-body-title} For a list of the properties that you can specify in the body of this API, see <>. - -==== Authorization - -You must have `manage_ml`, or `manage` cluster privileges to use this API. -For more information, see -{xpack-ref}/security-privileges.html[Security Privileges]. - - -==== Examples +[[ml-valid-detector-example]] +==== {api-examples-title} The following example validates detector configuration information: diff --git a/docs/reference/ml/apis/validate-job.asciidoc b/docs/reference/ml/apis/validate-job.asciidoc index 5fbfb62dd28a6..faa7cab2f3928 100644 --- a/docs/reference/ml/apis/validate-job.asciidoc +++ b/docs/reference/ml/apis/validate-job.asciidoc @@ -8,30 +8,32 @@ Validates job configuration information. -==== Request +[[ml-valid-job-request]] +==== {api-request-title} `POST _ml/anomaly_detectors/_validate` -==== Description +[[ml-valid-job-prereqs]] +==== {api-prereq-title} + +* If the {es} {security-features} are enabled, you must have `manage_ml` or +`manage` cluster privileges to use this API. See +{stack-ov}/security-privileges.html[Security privileges]. + +[[ml-valid-job-desc]] +==== {api-description-title} The validate jobs API enables you validate the job configuration before you create the job. - -==== Request Body +[[ml-valid-job-request-body]] +==== {api-request-body-title} For a list of the properties that you can specify in the body of this API, see <>. - -==== Authorization - -You must have `manage_ml`, or `manage` cluster privileges to use this API. -For more information, see -{xpack-ref}/security-privileges.html[Security Privileges]. - - -==== Examples +[[ml-valid-job-example]] +==== {api-examples-title} The following example validates job configuration information: diff --git a/docs/reference/ml/configuring.asciidoc b/docs/reference/ml/configuring.asciidoc index 9304a93d360c7..a27818e0cf1f5 100644 --- a/docs/reference/ml/configuring.asciidoc +++ b/docs/reference/ml/configuring.asciidoc @@ -5,7 +5,7 @@ If you want to use {ml-features}, there must be at least one {ml} node in your cluster and all master-eligible nodes must have {ml} enabled. By default, all nodes are {ml} nodes. For more information about these settings, see -{ref}/modules-node.html#modules-node-xpack[{ml} nodes]. +{ref}/modules-node.html#ml-node[{ml} nodes]. To use the {ml-features} to analyze your data, you must create a job and send your data to that job. diff --git a/docs/reference/ml/functions/geo.asciidoc b/docs/reference/ml/functions/geo.asciidoc index 130e17d85dcfe..ee666c83094f0 100644 --- a/docs/reference/ml/functions/geo.asciidoc +++ b/docs/reference/ml/functions/geo.asciidoc @@ -54,8 +54,9 @@ detects anomalies where the geographic location of a credit card transaction is unusual for a particular customer’s credit card. An anomaly might indicate fraud. IMPORTANT: The `field_name` that you supply must be a single string that contains -two comma-separated numbers of the form `latitude,longitude`. The `latitude` and -`longitude` must be in the range -180 to 180 and represent a point on the +two comma-separated numbers of the form `latitude,longitude`, a `geo_point` field, +a `geo_shape` field that contains point values, or a `geo_centroid` aggregation. +The `latitude` and `longitude` must be in the range -180 to 180 and represent a point on the surface of the Earth. For example, JSON data might contain the following transaction coordinates: @@ -71,34 +72,9 @@ For example, JSON data might contain the following transaction coordinates: // NOTCONSOLE In {es}, location data is likely to be stored in `geo_point` fields. For more -information, see {ref}/geo-point.html[Geo-point datatype]. This data type is not -supported natively in {ml-features}. You can, however, use Painless scripts -in `script_fields` in your {dfeed} to transform the data into an appropriate -format. For example, the following Painless script transforms -`"coords": {"lat" : 41.44, "lon":90.5}` into `"lat-lon": "41.44,90.5"`: - -[source,js] --------------------------------------------------- -PUT _ml/datafeeds/datafeed-test2 -{ - "job_id": "farequote", - "indices": ["farequote"], - "query": { - "match_all": { - "boost": 1 - } - }, - "script_fields": { - "lat-lon": { - "script": { - "source": "doc['coords'].lat + ',' + doc['coords'].lon", - "lang": "painless" - } - } - } -} --------------------------------------------------- -// CONSOLE -// TEST[skip:setup:farequote_job] +information, see {ref}/geo-point.html[Geo-point datatype]. This data type is +supported natively in {ml-features}. Specifically, {dfeed} when pulling data from +a `geo_point` field, will transform the data into the appropriate `lat,lon` string +format before sending to the {ml} job. For more information, see <>. diff --git a/docs/reference/modules.asciidoc b/docs/reference/modules.asciidoc index 5928c676b1e46..e9f187a98e3c7 100644 --- a/docs/reference/modules.asciidoc +++ b/docs/reference/modules.asciidoc @@ -46,20 +46,10 @@ The modules in this section are: A Java node client joins the cluster, but doesn't hold data or act as a master node. -<>:: - - A built-in scripting language for Elasticsearch that's designed to be as secure as possible. - <>:: Using plugins to extend Elasticsearch. -<>:: - - Custom scripting available in Lucene Expressions, ad Groovy. You can also - write scripts in the built-in scripting language, - <>. - <>:: Backup your data with snapshot/restore. @@ -102,8 +92,6 @@ include::modules/node.asciidoc[] :edit_url: include::modules/plugins.asciidoc[] -include::modules/scripting.asciidoc[] - include::modules/snapshots.asciidoc[] include::modules/threadpool.asciidoc[] diff --git a/docs/reference/modules/cluster/allocation_awareness.asciidoc b/docs/reference/modules/cluster/allocation_awareness.asciidoc index 4381d0281cc9d..5fc9197d449d5 100644 --- a/docs/reference/modules/cluster/allocation_awareness.asciidoc +++ b/docs/reference/modules/cluster/allocation_awareness.asciidoc @@ -46,7 +46,7 @@ You can also set custom attributes when you start a node: + [source,sh] -------------------------------------------------------- -`./bin/elasticsearch -Enode.attr.rack_id=rack_one` +./bin/elasticsearch -Enode.attr.rack_id=rack_one -------------------------------------------------------- . Tell {es} to take one or more awareness attributes into account when diff --git a/docs/reference/modules/discovery/adding-removing-nodes.asciidoc b/docs/reference/modules/discovery/adding-removing-nodes.asciidoc index ccc0e99125371..2a62bb5e49dd3 100644 --- a/docs/reference/modules/discovery/adding-removing-nodes.asciidoc +++ b/docs/reference/modules/discovery/adding-removing-nodes.asciidoc @@ -35,30 +35,36 @@ four of the nodes at once: to do so would leave only three nodes remaining, which is less than half of the voting configuration, which means the cluster cannot take any further actions. +More precisely, if you shut down half or more of the master-eligible nodes all +at the same time then the cluster will normally become unavailable. If this +happens then you can bring the cluster back online by starting the removed +nodes again. + As long as there are at least three master-eligible nodes in the cluster, as a general rule it is best to remove nodes one-at-a-time, allowing enough time for the cluster to <> the voting configuration and adapt the fault tolerance level to the new set of nodes. If there are only two master-eligible nodes remaining then neither node can be -safely removed since both are required to reliably make progress. You must first -inform Elasticsearch that one of the nodes should not be part of the voting -configuration, and that the voting power should instead be given to other nodes. -You can then take the excluded node offline without preventing the other node -from making progress. A node which is added to a voting configuration exclusion -list still works normally, but Elasticsearch tries to remove it from the voting -configuration so its vote is no longer required. Importantly, Elasticsearch -will never automatically move a node on the voting exclusions list back into the -voting configuration. Once an excluded node has been successfully +safely removed since both are required to reliably make progress. To remove one +of these nodes you must first inform {es} that it should not be part of the +voting configuration, and that the voting power should instead be given to the +other node. You can then take the excluded node offline without preventing the +other node from making progress. A node which is added to a voting +configuration exclusion list still works normally, but {es} tries to remove it +from the voting configuration so its vote is no longer required. Importantly, +{es} will never automatically move a node on the voting exclusions list back +into the voting configuration. Once an excluded node has been successfully auto-reconfigured out of the voting configuration, it is safe to shut it down without affecting the cluster's master-level availability. A node can be added -to the voting configuration exclusion list using the <> API. For example: +to the voting configuration exclusion list using the +<> API. For example: [source,js] -------------------------------------------------- -# Add node to voting configuration exclusions list and wait for the system to -# auto-reconfigure the node out of the voting configuration up to the default -# timeout of 30 seconds +# Add node to voting configuration exclusions list and wait for the system +# to auto-reconfigure the node out of the voting configuration up to the +# default timeout of 30 seconds POST /_cluster/voting_config_exclusions/node_name # Add node to voting configuration exclusions list and wait for diff --git a/docs/reference/modules/discovery/bootstrapping.asciidoc b/docs/reference/modules/discovery/bootstrapping.asciidoc index 2b17af17ec5da..cc7cb0ea91227 100644 --- a/docs/reference/modules/discovery/bootstrapping.asciidoc +++ b/docs/reference/modules/discovery/bootstrapping.asciidoc @@ -6,8 +6,9 @@ set of <> to be explicitly defined on one or more of the master-eligible nodes in the cluster. This is known as _cluster bootstrapping_. This is only required the very first time the cluster starts up: nodes that have already joined a cluster store this information in their -data folder and freshly-started nodes that are joining an existing cluster -obtain this information from the cluster's elected master. +data folder for use in a <>, and +freshly-started nodes that are joining a running cluster obtain this +information from the cluster's elected master. The initial set of master-eligible nodes is defined in the <>. This should be @@ -58,19 +59,6 @@ cluster.initial_master_nodes: - master-c -------------------------------------------------- -If it is not possible to use the names of the nodes then you can also use IP -addresses, or IP addresses and ports, or even a mix of IP addresses and node -names: - -[source,yaml] --------------------------------------------------- -cluster.initial_master_nodes: - - 10.0.10.101 - - 10.0.10.102:9300 - - 10.0.10.102:9301 - - master-node-name --------------------------------------------------- - Like all node settings, it is also possible to specify the initial set of master nodes on the command-line that is used to start Elasticsearch: @@ -139,3 +127,29 @@ in the <>: * `discovery.seed_providers` * `discovery.seed_hosts` * `cluster.initial_master_nodes` + +[NOTE] +================================================== + +[[modules-discovery-bootstrap-cluster-joining]] If you start an {es} node +without configuring these settings then it will start up in development mode and +auto-bootstrap itself into a new cluster. If you start some {es} nodes on +different hosts then by default they will not discover each other and will form +a different cluster on each host. {es} will not merge separate clusters together +after they have formed, even if you subsequently try and configure all the nodes +into a single cluster. This is because there is no way to merge these separate +clusters together without a risk of data loss. You can tell that you have formed +separate clusters by checking the cluster UUID reported by `GET /` on each node. +If you intended to form a single cluster then you should start again: + +* Take a <> of each of the single-host clusters if + you do not want to lose any data that they hold. Note that each cluster must + use its own snapshot repository. +* Shut down all the nodes. +* Completely wipe each node by deleting the contents of their + <>. +* Configure `cluster.initial_master_nodes` as described above. +* Restart all the nodes and verify that they have formed a single cluster. +* <> any snapshots as required. + +================================================== diff --git a/docs/reference/modules/discovery/discovery-settings.asciidoc b/docs/reference/modules/discovery/discovery-settings.asciidoc index 84472552cfced..850f0100c8939 100644 --- a/docs/reference/modules/discovery/discovery-settings.asciidoc +++ b/docs/reference/modules/discovery/discovery-settings.asciidoc @@ -16,14 +16,21 @@ Discovery and cluster formation are affected by the following settings: to use to obtain the addresses of the seed nodes used to start the discovery process. By default, it is the <>. - + +`discovery.type`:: + + Specifies whether {es} should form a multiple-node cluster. By default, {es} + discovers other nodes when forming a cluster and allows other nodes to join + the cluster later. If `discovery.type` is set to `single-node`, {es} forms a + single-node cluster. For more information about when you might use this + setting, see <>. + `cluster.initial_master_nodes`:: - Sets a list of the <> or transport addresses of the - initial set of master-eligible nodes in a brand-new cluster. By default - this list is empty, meaning that this node expects to join a cluster that - has already been bootstrapped. See <>. - + Sets the initial set of master-eligible nodes in a brand-new cluster. By + default this list is empty, meaning that this node expects to join a cluster + that has already been bootstrapped. See <>. + [float] ==== Expert settings diff --git a/docs/reference/modules/indices/search-settings.asciidoc b/docs/reference/modules/indices/search-settings.asciidoc index 30137fa382779..071471a1803ca 100644 --- a/docs/reference/modules/indices/search-settings.asciidoc +++ b/docs/reference/modules/indices/search-settings.asciidoc @@ -10,8 +10,8 @@ The following _expert_ setting can be set to manage global search limits. This setting limits the number of clauses a Lucene BooleanQuery can have. The default of 1024 is quite high and should normally be sufficient. This limit does not only affect Elasticsearchs `bool` query, but many other queries are rewritten to Lucene's -BooleanQuery internally. The limit is in place to prevent searches from becoming to large -and taking up too much CPU and memory. In case you consider to increase this setting, -make sure you exhausted all other options to avoid having to do this. Higher values can lead +BooleanQuery internally. The limit is in place to prevent searches from becoming too large +and taking up too much CPU and memory. In case you're considering increasing this setting, +make sure you've exhausted all other options to avoid having to do this. Higher values can lead to performance degradations and memory issues, especially in clusters with a high load or few resources. diff --git a/docs/reference/modules/memcached.asciidoc b/docs/reference/modules/memcached.asciidoc deleted file mode 100644 index 508d328671b26..0000000000000 --- a/docs/reference/modules/memcached.asciidoc +++ /dev/null @@ -1,69 +0,0 @@ -[[modules-memcached]] -== memcached - -The memcached module allows to expose *Elasticsearch* -APIs over the memcached protocol (as closely -as possible). - -It is provided as a plugin called `transport-memcached` and installing -is explained -https://github.com/elastic/elasticsearch-transport-memcached[here] -. Another option is to download the memcached plugin and placing it -under the `plugins` directory. - -The memcached protocol supports both the binary and the text protocol, -automatically detecting the correct one to use. - -[float] -=== Mapping REST to Memcached Protocol - -Memcached commands are mapped to REST and handled by the same generic -REST layer in Elasticsearch. Here is a list of the memcached commands -supported: - -[float] -==== GET - -The memcached `GET` command maps to a REST `GET`. The key used is the -URI (with parameters). The main downside is the fact that the memcached -`GET` does not allow body in the request (and `SET` does not allow to -return a result...). For this reason, most REST APIs (like search) allow -to accept the "source" as a URI parameter as well. - -[float] -==== SET - -The memcached `SET` command maps to a REST `POST`. The key used is the -URI (with parameters), and the body maps to the REST body. - -[float] -==== DELETE - -The memcached `DELETE` command maps to a REST `DELETE`. The key used is -the URI (with parameters). - -[float] -==== QUIT - -The memcached `QUIT` command is supported and disconnects the client. - -[float] -=== Settings - -The following are the settings the can be configured for memcached: - -[cols="<,<",options="header",] -|=============================================================== -|Setting |Description -|`memcached.port` |A bind port range. Defaults to `11211-11311`. -|=============================================================== - -It also uses the common -<>. - -[float] -=== Disable memcached - -The memcached module can be completely disabled and not started using by -setting `memcached.enabled` to `false`. By default it is enabled once it -is detected as a plugin. diff --git a/docs/reference/modules/ml-node.asciidoc b/docs/reference/modules/ml-node.asciidoc deleted file mode 100644 index 365a08dfe7bb1..0000000000000 --- a/docs/reference/modules/ml-node.asciidoc +++ /dev/null @@ -1,116 +0,0 @@ -[float] -[[modules-node-xpack]] -== [xpack]#X-Pack node settings# - -//This content is referenced from the elastic/elasticsearch/docs/reference/modules/node.asciidoc - -If {xpack} is installed, there is an additional node type: - -<>:: - -A node that has `xpack.ml.enabled` and `node.ml` set to `true`, which is the -default behavior when {xpack} is installed. If you want to use {ml-features}, there must be at least one {ml} node in your cluster. For more -information about {ml-features}, -see {stack-ov}/xpack-ml.html[Machine learning in the {stack}]. - -IMPORTANT: Do not set use the `node.ml` setting unless {xpack} is installed. -Otherwise, the node fails to start. - -If {xpack} is installed, nodes are master-eligible, data, ingest, and {ml} -nodes by default. As the cluster grows and in particular if you have large -{ml} jobs, consider separating dedicated master-eligible nodes from dedicated -data nodes and dedicated {ml} nodes. - -To create a dedicated master-eligible node when {xpack} is installed, set: - -[source,yaml] -------------------- -node.master: true <1> -node.data: false <2> -node.ingest: false <3> -node.ml: false <4> -xpack.ml.enabled: true <5> -------------------- -<1> The `node.master` role is enabled by default. -<2> Disable the `node.data` role (enabled by default). -<3> Disable the `node.ingest` role (enabled by default). -<4> Disable the `node.ml` role (enabled by default in {xpack}). -<5> The `xpack.ml.enabled` setting is enabled by default in {xpack}. - -To create a dedicated data node when {xpack} is installed, set: - -[source,yaml] -------------------- -node.master: false <1> -node.data: true <2> -node.ingest: false <3> -node.ml: false <4> -------------------- -<1> Disable the `node.master` role (enabled by default). -<2> The `node.data` role is enabled by default. -<3> Disable the `node.ingest` role (enabled by default). -<4> Disable the `node.ml` role (enabled by default in {xpack}). - -To create a dedicated ingest node when {xpack} is installed, set: - -[source,yaml] -------------------- -node.master: false <1> -node.data: false <2> -node.ingest: true <3> -cluster.remote.connect: false <4> -node.ml: false <5> -------------------- -<1> Disable the `node.master` role (enabled by default). -<2> Disable the `node.data` role (enabled by default). -<3> The `node.ingest` role is enabled by default. -<4> Disable {ccs} (enabled by default). -<5> Disable the `node.ml` role (enabled by default in {xpack}). - -To create a dedicated coordinating node when {xpack} is installed, set: - -[source,yaml] -------------------- -node.master: false <1> -node.data: false <2> -node.ingest: false <3> -cluster.remote.connect: false <4> -node.ml: false <5> -------------------- -<1> Disable the `node.master` role (enabled by default). -<2> Disable the `node.data` role (enabled by default). -<3> Disable the `node.ingest` role (enabled by default). -<4> Disable {ccs} (enabled by default). -<5> Disable the `node.ml` role (enabled by default in {xpack}). - -[float] -[[ml-node]] -=== [xpack]#Machine learning node# - -The {ml-features} provide {ml} nodes, which run jobs and handle {ml} API -requests. If `xpack.ml.enabled` is set to true and `node.ml` is set to `false`, -the node can service API requests but it cannot run jobs. - -If you want to use {ml-features} in your cluster, you must enable {ml} -(set `xpack.ml.enabled` to `true`) on all master-eligible nodes. Do not use -these settings if you do not have {xpack} installed. - -For more information about these settings, see <>. - -To create a dedicated {ml} node, set: - -[source,yaml] -------------------- -node.master: false <1> -node.data: false <2> -node.ingest: false <3> -cluster.remote.connect: false <4> -node.ml: true <5> -xpack.ml.enabled: true <6> -------------------- -<1> Disable the `node.master` role (enabled by default). -<2> Disable the `node.data` role (enabled by default). -<3> Disable the `node.ingest` role (enabled by default). -<4> Disable {ccs} (enabled by default). -<5> The `node.ml` role is enabled by default in {xpack}. -<6> The `xpack.ml.enabled` setting is enabled by default in {xpack}. diff --git a/docs/reference/modules/network.asciidoc b/docs/reference/modules/network.asciidoc index e60b56fe12d74..4136f07bb9b31 100644 --- a/docs/reference/modules/network.asciidoc +++ b/docs/reference/modules/network.asciidoc @@ -24,7 +24,9 @@ this host to other nodes in the cluster. Accepts an IP address, hostname, a <>, or an array of any combination of these. Note that any values containing a `:` (e.g., an IPv6 address or containing one of the <>) must be -quoted because `:` is a special character in YAML. +quoted because `:` is a special character in YAML. `0.0.0.0` is an acceptable +IP address and will bind to all network interfaces. The value `0` has the +same effect as the value `0.0.0.0`. + Defaults to `_local_`. @@ -173,11 +175,10 @@ settings, but may be further configured independently: TCP Transport:: -Used for communication between nodes in the cluster, by the Java -{javaclient}/transport-client.html[Transport client]. +Used for communication between nodes in the cluster. See the <> for more information. HTTP:: -Exposes the JSON-over-HTTP interface used by all clients other than the Java -clients. See the <> for more information. +Exposes the JSON-over-HTTP interface used by all clients. +See the <> for more information. diff --git a/docs/reference/modules/node.asciidoc b/docs/reference/modules/node.asciidoc index 031138dada3f1..a25ab22260d3d 100644 --- a/docs/reference/modules/node.asciidoc +++ b/docs/reference/modules/node.asciidoc @@ -2,19 +2,24 @@ == Node Any time that you start an instance of Elasticsearch, you are starting a -_node_. A collection of connected nodes is called a +_node_. A collection of connected nodes is called a <>. If you are running a single node of Elasticsearch, then you have a cluster of one node. Every node in the cluster can handle <> and <> traffic by default. The transport layer -is used exclusively for communication between nodes and the -{javaclient}/transport-client.html[Java `TransportClient`]; the HTTP layer is -used only by external REST clients. +is used exclusively for communication between nodes; the HTTP layer is +used by REST clients. All nodes know about all the other nodes in the cluster and can forward client -requests to the appropriate node. Besides that, each node serves one or more -purpose: +requests to the appropriate node. + +By default, a node is all of the following types: master-eligible, data, ingest, +and machine learning (if available). + +TIP: As the cluster grows and in particular if you have large {ml} jobs, +consider separating dedicated master-eligible nodes from dedicated data nodes +and dedicated {ml} nodes. <>:: @@ -35,6 +40,17 @@ and enrich the document before indexing. With a heavy ingest load, it makes sense to use dedicated ingest nodes and to mark the master and data nodes as `node.ingest: false`. +<>:: + +A node that has `xpack.ml.enabled` and `node.ml` set to `true`, which is the +default behavior in the {es} {default-dist}. If you want to use {ml-features}, +there must be at least one {ml} node in your cluster. For more information about +{ml-features}, see +{stack-ov}/xpack-ml.html[Machine learning in the {stack}]. ++ +IMPORTANT: If you use the {oss-dist}, do not set `node.ml`. Otherwise, the node +fails to start. + [NOTE] [[coordinating-node]] .Coordinating node @@ -68,8 +84,9 @@ creating or deleting an index, tracking which nodes are part of the cluster, and deciding which shards to allocate to which nodes. It is important for cluster health to have a stable master node. -Any master-eligible node (all nodes by default) may be elected to become the -master node by the <>. +Any master-eligible node that is not a <> may +be elected to become the master node by the <>. IMPORTANT: Master nodes must have access to the `data/` directory (just like `data` nodes) as this is where the cluster state is persisted between node restarts. @@ -86,7 +103,27 @@ better _not_ to use dedicated master nodes for this purpose. It is important for the stability of the cluster that master-eligible nodes do as little work as possible. -To create a dedicated master-eligible node, set: +To create a dedicated master-eligible node in the {default-dist}, set: + +[source,yaml] +------------------- +node.master: true <1> +node.voting_only: false <2> +node.data: false <3> +node.ingest: false <4> +node.ml: false <5> +xpack.ml.enabled: true <6> +cluster.remote.connect: false <7> +------------------- +<1> The `node.master` role is enabled by default. +<2> The `node.voting_only` role is disabled by default. +<3> Disable the `node.data` role (enabled by default). +<4> Disable the `node.ingest` role (enabled by default). +<5> Disable the `node.ml` role (enabled by default). +<6> The `xpack.ml.enabled` setting is enabled by default. +<7> Disable {ccs} (enabled by default). + +To create a dedicated master-eligible node in the {oss-dist}, set: [source,yaml] ------------------- @@ -100,10 +137,70 @@ cluster.remote.connect: false <4> <3> Disable the `node.ingest` role (enabled by default). <4> Disable {ccs} (enabled by default). -ifdef::include-xpack[] -NOTE: These settings apply only when {xpack} is not installed. To create a -dedicated master-eligible node when {xpack} is installed, see <>. -endif::include-xpack[] +[float] +[[voting-only-node]] +==== Voting-only master-eligible node + +A voting-only master-eligible node is a node that participates in +<> but which will not act as the cluster's +elected master node. In particular, a voting-only node can serve as a tiebreaker +in elections. + +It may seem confusing to use the term "master-eligible" to describe a +voting-only node since such a node is not actually eligible to become the master +at all. This terminology is an unfortunate consequence of history: +master-eligible nodes are those nodes that participate in elections and perform +certain tasks during cluster state publications, and voting-only nodes have the +same responsibilities even if they can never become the elected master. + +To configure a master-eligible node as a voting-only node, set the following +setting: + +[source,yaml] +------------------- +node.voting_only: true <1> +------------------- +<1> The default for `node.voting_only` is `false`. + +IMPORTANT: The `voting_only` role requires the {default-dist} of Elasticsearch +and is not supported in the {oss-dist}. If you use the {oss-dist} and set +`node.voting_only` then the node will fail to start. Also note that only +master-eligible nodes can be marked as voting-only. + +High availability (HA) clusters require at least three master-eligible nodes, at +least two of which are not voting-only nodes. Such a cluster will be able to +elect a master node even if one of the nodes fails. + +Since voting-only nodes never act as the cluster's elected master, they may +require require less heap and a less powerful CPU than the true master nodes. +However all master-eligible nodes, including voting-only nodes, require +reasonably fast persistent storage and a reliable and low-latency network +connection to the rest of the cluster, since they are on the critical path for +<>. + +Voting-only master-eligible nodes may also fill other roles in your cluster. +For instance, a node may be both a data node and a voting-only master-eligible +node. A _dedicated_ voting-only master-eligible nodes is a voting-only +master-eligible node that fills no other roles in the cluster. To create a +dedicated voting-only master-eligible node in the {default-dist}, set: + +[source,yaml] +------------------- +node.master: true <1> +node.voting_only: true <2> +node.data: false <3> +node.ingest: false <4> +node.ml: false <5> +xpack.ml.enabled: true <6> +cluster.remote.connect: false <7> +------------------- +<1> The `node.master` role is enabled by default. +<2> Enable the `node.voting_only` role (disabled by default). +<3> Disable the `node.data` role (enabled by default). +<4> Disable the `node.ingest` role (enabled by default). +<5> Disable the `node.ml` role (enabled by default). +<6> The `xpack.ml.enabled` setting is enabled by default. +<7> Disable {ccs} (enabled by default). [float] [[data-node]] @@ -117,8 +214,24 @@ monitor these resources and to add more data nodes if they are overloaded. The main benefit of having dedicated data nodes is the separation of the master and data roles. -To create a dedicated data node, set: +To create a dedicated data node in the {default-dist}, set: +[source,yaml] +------------------- +node.master: false <1> +node.voting_only: false <2> +node.data: true <3> +node.ingest: false <4> +node.ml: false <5> +cluster.remote.connect: false <6> +------------------- +<1> Disable the `node.master` role (enabled by default). +<2> The `node.voting_only` role is disabled by default. +<3> The `node.data` role is enabled by default. +<4> Disable the `node.ingest` role (enabled by default). +<5> Disable the `node.ml` role (enabled by default). +<6> Disable {ccs} (enabled by default). +To create a dedicated data node in the {oss-dist}, set: [source,yaml] ------------------- node.master: false <1> @@ -131,11 +244,6 @@ cluster.remote.connect: false <4> <3> Disable the `node.ingest` role (enabled by default). <4> Disable {ccs} (enabled by default). -ifdef::include-xpack[] -NOTE: These settings apply only when {xpack} is not installed. To create a -dedicated data node when {xpack} is installed, see <>. -endif::include-xpack[] - [float] [[node-ingest-node]] === Ingest Node @@ -145,7 +253,25 @@ ingest processors. Depending on the type of operations performed by the ingest processors and the required resources, it may make sense to have dedicated ingest nodes, that will only perform this specific task. -To create a dedicated ingest node, set: +To create a dedicated ingest node in the {default-dist}, set: + +[source,yaml] +------------------- +node.master: false <1> +node.voting_only: false <2> +node.data: false <3> +node.ingest: true <4> +node.ml: false <5> +cluster.remote.connect: false <6> +------------------- +<1> Disable the `node.master` role (enabled by default). +<2> The `node.voting_only` role is disabled by default. +<3> Disable the `node.data` role (enabled by default). +<4> The `node.ingest` role is enabled by default. +<5> Disable the `node.ml` role (enabled by default). +<6> Disable {ccs} (enabled by default). + +To create a dedicated ingest node in the {oss-dist}, set: [source,yaml] ------------------- @@ -159,11 +285,6 @@ cluster.remote.connect: false <4> <3> The `node.ingest` role is enabled by default. <4> Disable {ccs} (enabled by default). -ifdef::include-xpack[] -NOTE: These settings apply only when {xpack} is not installed. To create a -dedicated ingest node when {xpack} is installed, see <>. -endif::include-xpack[] - [float] [[coordinating-only-node]] === Coordinating only node @@ -185,7 +306,25 @@ acknowledgement of cluster state updates from every node! The benefit of coordinating only nodes should not be overstated -- data nodes can happily serve the same purpose. -To create a dedicated coordinating node, set: +To create a dedicated coordinating node in the {default-dist}, set: + +[source,yaml] +------------------- +node.master: false <1> +node.voting_only: false <2> +node.data: false <3> +node.ingest: false <4> +node.ml: false <5> +cluster.remote.connect: false <6> +------------------- +<1> Disable the `node.master` role (enabled by default). +<2> The `node.voting_only` role is disabled by default. +<3> Disable the `node.data` role (enabled by default). +<4> Disable the `node.ingest` role (enabled by default). +<5> Disable the `node.ml` role (enabled by default). +<6> Disable {ccs} (enabled by default). + +To create a dedicated coordinating node in the {oss-dist}, set: [source,yaml] ------------------- @@ -199,10 +338,39 @@ cluster.remote.connect: false <4> <3> Disable the `node.ingest` role (enabled by default). <4> Disable {ccs} (enabled by default). -ifdef::include-xpack[] -NOTE: These settings apply only when {xpack} is not installed. To create a -dedicated coordinating node when {xpack} is installed, see <>. -endif::include-xpack[] +[float] +[[ml-node]] +=== [xpack]#Machine learning node# + +The {ml-features} provide {ml} nodes, which run jobs and handle {ml} API +requests. If `xpack.ml.enabled` is set to true and `node.ml` is set to `false`, +the node can service API requests but it cannot run jobs. + +If you want to use {ml-features} in your cluster, you must enable {ml} +(set `xpack.ml.enabled` to `true`) on all master-eligible nodes. If you have the +{oss-dist}, do not use these settings. + +For more information about these settings, see <>. + +To create a dedicated {ml} node in the {default-dist}, set: + +[source,yaml] +------------------- +node.master: false <1> +node.voting_only: false <2> +node.data: false <3> +node.ingest: false <4> +node.ml: true <5> +xpack.ml.enabled: true <6> +cluster.remote.connect: false <7> +------------------- +<1> Disable the `node.master` role (enabled by default). +<2> The `node.voting_only` role is disabled by default. +<3> Disable the `node.data` role (enabled by default). +<4> Disable the `node.ingest` role (enabled by default). +<5> The `node.ml` role is enabled by default. +<6> The `xpack.ml.enabled` setting is enabled by default. +<7> Disable {ccs} (enabled by default). [float] [[change-node-role]] @@ -283,7 +451,3 @@ your data! The RPM and Debian distributions do this for you already. More node settings can be found in <>. Of particular note are the <>, the <> and the <>. - -ifdef::include-xpack[] -include::ml-node.asciidoc[] -endif::include-xpack[] diff --git a/docs/reference/modules/remote-clusters.asciidoc b/docs/reference/modules/remote-clusters.asciidoc index e654a50e6122e..52fe0ba42064e 100644 --- a/docs/reference/modules/remote-clusters.asciidoc +++ b/docs/reference/modules/remote-clusters.asciidoc @@ -30,24 +30,25 @@ node to the selected remote _gateway nodes_ only. The _gateway nodes_ selection depends on the following criteria: - *version*: Remote nodes must be compatible with the cluster they are -registered to. This is subject to the same rules as <>. -Any node can communicate with any other node on the same major version (e.g. -6.0 can talk to any 6.x node). Only nodes on the last minor version of a -certain major version can communicate with nodes on the following major -version (e.g. 6.7 can communicate with 7.0, as well as any 7.x node, while - 6.6 or earlier cannot talk to any 7.x node). Note that version compatibility - is symmetric, meaning that if 6.7 can communicate with 7.0, 7.0 can also - communicate with 6.7. The matrix below summarizes compatibility as described - above. - -[cols="^,^,^,^,^,^"] +registered to. This is subject to rules that are similar to those for +<>. Any node can communicate with any other node on the same +major version (e.g. 7.0 can talk to any 7.x node). Only nodes on the last minor +version of a certain major version can communicate with nodes on the following +major version. Note that in the 6.x series, 6.8 can communicate with any 7.x +node, while 6.7 can only communicate with 7.0. Version compatibility is +symmetric, meaning that if 6.7 can communicate with 7.0, 7.0 can also +communicate with 6.7. The matrix below summarizes compatibility as described above. + +[cols="^,^,^,^,^,^,^,^"] |==== -| Compatibility | 5.0->5.5 | 5.6 | 6.0->6.6 | 6.7 | 7.x -| 5.0->5.5 | Yes | Yes | No | No | No -| 5.6 | Yes | Yes | Yes | Yes | No -| 6.0->6.6 | No | Yes | Yes | Yes | No -| 6.7 | No | Yes | Yes | Yes | Yes -| 7.x | No | No | No | Yes | Yes +| Compatibility | 5.0->5.5 | 5.6 | 6.0->6.6 | 6.7 | 6.8 | 7.0 | 7.1->7.x +| 5.0->5.5 | Yes | Yes | No | No | No | No | No +| 5.6 | Yes | Yes | Yes | Yes | Yes | No | No +| 6.0->6.6 | No | Yes | Yes | Yes | Yes | No | No +| 6.7 | No | Yes | Yes | Yes | Yes | Yes | No +| 6.8 | No | Yes | Yes | Yes | Yes | Yes | Yes +| 7.0 | No | No | No | Yes | Yes | Yes | Yes +| 7.1->7.x | No | No | No | No | Yes | Yes | Yes |==== - *role*: Dedicated master nodes never get selected. diff --git a/docs/reference/modules/snapshots.asciidoc b/docs/reference/modules/snapshots.asciidoc index ec7916d5a3445..ed52c0958fec0 100644 --- a/docs/reference/modules/snapshots.asciidoc +++ b/docs/reference/modules/snapshots.asciidoc @@ -1,29 +1,32 @@ [[modules-snapshots]] == Snapshot And Restore +// tag::snapshot-intro[] A snapshot is a backup taken from a running Elasticsearch cluster. You can take a snapshot of individual indices or of the entire cluster and store it in a repository on a shared filesystem, and there are plugins that support remote repositories on S3, HDFS, Azure, Google Cloud Storage and more. -Snapshots are taken incrementally. This means that when creating a snapshot of -an index Elasticsearch will avoid copying any data that is already stored in -the repository as part of an earlier snapshot of the same index. Therefore it -can be efficient to take snapshots of your cluster quite frequently. - -Snapshots can be restored into a running cluster via the restore API. When -restoring an index it is possible to alter the name of the restored index as -well as some of its settings, allowing a great deal of flexibility in how the -snapshot and restore functionality can be used. - -WARNING: It is not possible to back up an Elasticsearch cluster simply by -taking a copy of the data directories of all of its nodes. Elasticsearch may be -making changes to the contents of its data directories while it is running, and -this means that copying its data directories cannot be expected to capture a -consistent picture of their contents. Attempts to restore a cluster from such a -backup may fail, reporting corruption and/or missing files, or may appear to -have succeeded having silently lost some of its data. The only reliable way to -back up a cluster is by using the snapshot and restore functionality. +Snapshots are taken incrementally. This means that when it creates a snapshot of +an index, Elasticsearch avoids copying any data that is already stored in the +repository as part of an earlier snapshot of the same index. Therefore it can be +efficient to take snapshots of your cluster quite frequently. + +You can restore snapshots into a running cluster via the +<>. When you restore an index, you can alter the +name of the restored index as well as some of its settings. There is a great +deal of flexibility in how the snapshot and restore functionality can be used. + +WARNING: You cannot back up an Elasticsearch cluster by simply taking a copy of +the data directories of all of its nodes. Elasticsearch may be making changes to +the contents of its data directories while it is running; copying its data +directories cannot be expected to capture a consistent picture of their contents. +If you try to restore a cluster from such a backup, it may fail and report +corruption and/or missing files. Alternatively, it may appear to have succeeded +though it silently lost some of its data. The only reliable way to back up a +cluster is by using the snapshot and restore functionality. + +// end::snapshot-intro[] [float] === Version compatibility @@ -349,7 +352,11 @@ PUT /_snapshot/my_backup/snapshot_2?wait_for_completion=true { "indices": "index_1,index_2", "ignore_unavailable": true, - "include_global_state": false + "include_global_state": false, + "_meta": { + "taken_by": "kimchy", + "taken_because": "backup before upgrading" + } } ----------------------------------- // CONSOLE @@ -363,6 +370,9 @@ By setting `include_global_state` to false it's possible to prevent the cluster the snapshot. By default, the entire snapshot will fail if one or more indices participating in the snapshot don't have all primary shards available. This behaviour can be changed by setting `partial` to `true`. +The `_meta` field can be used to attach arbitrary metadata to the snapshot. This may be a record of who took the snapshot, +why it was taken, or any other data that might be useful. + Snapshot names can be automatically derived using <>, similarly as when creating new indices. Note that special characters need to be URI encoded. @@ -465,6 +475,16 @@ that setting `verbose` to `false` will omit all other information about the snap such as status information, the number of snapshotted shards, etc. The default value of the `verbose` parameter is `true`. +It is also possible to retrieve snapshots from multiple repositories in one go, for example: +[source,sh] +----------------------------------- +GET /_snapshot/_all +GET /_snapshot/my_backup,my_fs_backup +GET /_snapshot/my*/snap* +----------------------------------- +// CONSOLE +// TEST[skip:no my_fs_backup] + A currently running snapshot can be retrieved using the following command: [source,sh] @@ -502,6 +522,7 @@ When a repository is unregistered, Elasticsearch only removes the reference to t the snapshots. The snapshots themselves are left untouched and in place. [float] +[[restore-snapshot]] === Restore A snapshot can be restored using the following command: diff --git a/docs/reference/modules/thrift.asciidoc b/docs/reference/modules/thrift.asciidoc deleted file mode 100644 index 1ea3f81812693..0000000000000 --- a/docs/reference/modules/thrift.asciidoc +++ /dev/null @@ -1,25 +0,0 @@ -[[modules-thrift]] -== Thrift - -The https://thrift.apache.org/[thrift] transport module allows to expose the REST interface of -Elasticsearch using thrift. Thrift should provide better performance -over http. Since thrift provides both the wire protocol and the -transport, it should make using Elasticsearch more efficient (though it has limited -documentation). - -Using thrift requires installing the `transport-thrift` plugin, located -https://github.com/elastic/elasticsearch-transport-thrift[here]. - -The thrift -https://github.com/elastic/elasticsearch-transport-thrift/blob/master/elasticsearch.thrift[schema] -can be used to generate thrift clients. - -[cols="<,<",options="header",] -|======================================================================= -|Setting |Description -|`thrift.port` |The port to bind to. Defaults to 9500-9600 - -|`thrift.frame` |Defaults to `-1`, which means no framing. Set to a -higher value to specify the frame size (like `15mb`). -|======================================================================= - diff --git a/docs/reference/monitoring/configuring-filebeat.asciidoc b/docs/reference/monitoring/configuring-filebeat.asciidoc new file mode 100644 index 0000000000000..fd77dc860ce8a --- /dev/null +++ b/docs/reference/monitoring/configuring-filebeat.asciidoc @@ -0,0 +1,187 @@ +[role="xpack"] +[testenv="basic"] +[[configuring-filebeat]] +=== Collecting {es} log data with {filebeat} + +[subs="attributes"] +++++ +Collecting log data with {filebeat} +++++ + +You can use {filebeat} to monitor the {es} log files, collect log events, and +ship them to the monitoring cluster. Your recent logs are visible on the +*Monitoring* page in {kib}. + +//NOTE: The tagged regions are re-used in the Stack Overview. + +. Verify that {es} is running and that the monitoring cluster is ready to +receive data from {filebeat}. ++ +-- +TIP: In production environments, we strongly recommend using a separate cluster +(referred to as the _monitoring cluster_) to store the data. Using a separate +monitoring cluster prevents production cluster outages from impacting your +ability to access your monitoring data. It also prevents monitoring activities +from impacting the performance of your production cluster. See +{stack-ov}/monitoring-production.html[Monitoring in a production environment]. + +-- + +. Enable the collection of monitoring data on your cluster. ++ +-- +include::configuring-metricbeat.asciidoc[tag=enable-collection] + +For more information, see <> and <>. +-- + +. Identify which logs you want to monitor. ++ +-- +The {filebeat} {es} module can handle +{stack-ov}/audit-log-output.html[audit logs], +{ref}/logging.html#deprecation-logging[deprecation logs], +{ref}/gc-logging.html[gc logs], {ref}/logging.html[server logs], and +{ref}/index-modules-slowlog.html[slow logs]. +For more information about the location of your {es} logs, see the +{ref}/path-settings.html[path.logs] setting. + +IMPORTANT: If there are both structured (`*.json`) and unstructured (plain text) +versions of the logs, you must use the structured logs. Otherwise, they might +not appear in the appropriate context in {kib}. + +-- + +. {filebeat-ref}/filebeat-installation.html[Install {filebeat}] on the {es} +nodes that contain logs that you want to monitor. + +. Identify where to send the log data. ++ +-- +// tag::output-elasticsearch[] +For example, specify {es} output information for your monitoring cluster in +the {filebeat} configuration file (`filebeat.yml`): + +[source,yaml] +---------------------------------- +output.elasticsearch: + # Array of hosts to connect to. + hosts: ["http://es-mon-1:9200", "http://es-mon2:9200"] <1> + + # Optional protocol and basic auth credentials. + #protocol: "https" + #username: "elastic" + #password: "changeme" +---------------------------------- +<1> In this example, the data is stored on a monitoring cluster with nodes +`es-mon-1` and `es-mon-2`. + +If you configured the monitoring cluster to use encrypted communications, you +must access it via HTTPS. For example, use a `hosts` setting like +`https://es-mon-1:9200`. + +IMPORTANT: The {es} {monitor-features} use ingest pipelines, therefore the +cluster that stores the monitoring data must have at least one +<>. + +If {es} {security-features} are enabled on the monitoring cluster, you must +provide a valid user ID and password so that {filebeat} can send metrics +successfully. + +For more information about these configuration options, see +{filebeat-ref}/elasticsearch-output.html[Configure the {es} output]. +// end::output-elasticsearch[] +-- + +. Optional: Identify where to visualize the data. ++ +-- +// tag::setup-kibana[] +{filebeat} provides example {kib} dashboards, visualizations and searches. To +load the dashboards into the appropriate {kib} instance, specify the +`setup.kibana` information in the {filebeat} configuration file +(`filebeat.yml`) on each node: + +[source,yaml] +---------------------------------- +setup.kibana: + host: "localhost:5601" + #username: "my_kibana_user" + #password: "YOUR_PASSWORD" +---------------------------------- + +TIP: In production environments, we strongly recommend using a dedicated {kib} +instance for your monitoring cluster. + +If {security-features} are enabled, you must provide a valid user ID and +password so that {filebeat} can connect to {kib}: + +.. Create a user on the monitoring cluster that has the +{stack-ov}/built-in-roles.html[`kibana_user` built-in role] or equivalent +privileges. + +.. Add the `username` and `password` settings to the {es} output information in +the {filebeat} configuration file. The example shows a hard-coded password, but +you should store sensitive values in the +{filebeat-ref}/keystore.html[secrets keystore]. + +See {filebeat-ref}/setup-kibana-endpoint.html[Configure the {kib} endpoint]. + +// end::setup-kibana[] +-- + +. Enable the {es} module and set up the initial {filebeat} environment on each +node. ++ +-- +// tag::enable-es-module[] +For example: + +["source","sh",subs="attributes,callouts"] +---------------------------------------------------------------------- +filebeat modules enable elasticsearch +filebeat setup -e +---------------------------------------------------------------------- + +For more information, see +{filebeat-ref}/filebeat-module-elasticsearch.html[{es} module]. + +// end::enable-es-module[] +-- + +. Configure the {es} module in {filebeat} on each node. ++ +-- +// tag::configure-es-module[] +If the logs that you want to monitor aren't in the default location, set the +appropriate path variables in the `modules.d/elasticsearch.yml` file. See +{filebeat-ref}/filebeat-module-elasticsearch.html#configuring-elasticsearch-module[Configure the {es} module]. + +IMPORTANT: If there are JSON logs, configure the `var.paths` settings to point +to them instead of the plain text logs. + +// end::configure-es-module[] +-- + +. {filebeat-ref}/filebeat-starting.html[Start {filebeat}] on each node. ++ +-- +NOTE: Depending on how you’ve installed {filebeat}, you might see errors related +to file ownership or permissions when you try to run {filebeat} modules. See +{beats-ref}/config-file-permissions.html[Config file ownership and permissions]. + +-- + +. Check whether the appropriate indices exist on the monitoring cluster. ++ +-- +For example, use the {ref}/cat-indices.html[cat indices] command to verify +that there are new `filebeat-*` indices. + +TIP: If you want to use the *Monitoring* UI in {kib}, there must also be +`.monitoring-*` indices. Those indices are generated when you collect metrics +about {stack} products. For example, see <>. + +-- + +. {kibana-ref}/monitoring-data.html[View the monitoring data in {kib}]. diff --git a/docs/reference/monitoring/configuring-metricbeat.asciidoc b/docs/reference/monitoring/configuring-metricbeat.asciidoc index df578e88da614..c768a7782053e 100644 --- a/docs/reference/monitoring/configuring-metricbeat.asciidoc +++ b/docs/reference/monitoring/configuring-metricbeat.asciidoc @@ -17,17 +17,16 @@ image::monitoring/images/metricbeat.png[Example monitoring architecture] To learn about monitoring in general, see {stack-ov}/xpack-monitoring.html[Monitoring the {stack}]. -. Enable the collection of monitoring data. Set -`xpack.monitoring.collection.enabled` to `true` on each node in the production -cluster. By default, it is is disabled (`false`). -+ +//NOTE: The tagged regions are re-used in the Stack Overview. + +. Enable the collection of monitoring data. + ++ -- -NOTE: You can specify this setting in either the `elasticsearch.yml` on each -node or across the cluster as a dynamic cluster setting. If {es} -{security-features} are enabled, you must have `monitor` cluster privileges to -view the cluster settings and `manage` cluster privileges to change them. +// tag::enable-collection[] +Set `xpack.monitoring.collection.enabled` to `true` on the +production cluster. By default, it is is disabled (`false`). -For example, you can use the following APIs to review and change this setting: +You can use the following APIs to review and change this setting: [source,js] ---------------------------------- @@ -42,42 +41,20 @@ PUT _cluster/settings ---------------------------------- // CONSOLE -For more information, see <> and <>. --- - -. Disable the default collection of {es} monitoring metrics. Set -`xpack.monitoring.elasticsearch.collection.enabled` to `false` on each node in -the production cluster. -+ --- -NOTE: You can specify this setting in either the `elasticsearch.yml` on each -node or across the cluster as a dynamic cluster setting. If {es} -{security-features} are enabled, you must have `monitor` cluster privileges to +If {es} {security-features} are enabled, you must have `monitor` cluster privileges to view the cluster settings and `manage` cluster privileges to change them. -For example, you can use the following API to change this setting: - -[source,js] ----------------------------------- -PUT _cluster/settings -{ - "persistent": { - "xpack.monitoring.elasticsearch.collection.enabled": false - } -} ----------------------------------- -// CONSOLE - -Leave `xpack.monitoring.enabled` set to its default value (`true`). +// end::enable-collection[] +For more information, see <> and <>. -- -. On each {es} node in the production cluster: - -.. {metricbeat-ref}/metricbeat-installation.html[Install {metricbeat}]. +. {metricbeat-ref}/metricbeat-installation.html[Install {metricbeat}] on each +{es} node in the production cluster. -.. Enable the {es} module in {metricbeat}. + +. Enable the {es} {xpack} module in {metricbeat} on each {es} node. + + -- +// tag::enable-es-module[] For example, to enable the default configuration in the `modules.d` directory, run the following command: @@ -89,39 +66,74 @@ metricbeat modules enable elasticsearch-xpack For more information, see {metricbeat-ref}/configuration-metricbeat.html[Specify which modules to run] and {metricbeat-ref}/metricbeat-module-elasticsearch.html[{es} module]. --- - -.. By default the module will collect {es} monitoring metrics from `http://localhost:9200`. -If the local {es} node has a different address, you must specify it via the `hosts` setting -in the `modules.d/elasticsearch-xpack.yml` file. -.. If Elastic {security-features} are enabled, you must also provide a user ID -and password so that {metricbeat} can collect metrics successfully. - -... Create a user on the production cluster that has the -{stack-ov}/built-in-roles.html[`remote_monitoring_collector` built-in role]. -Alternatively, use the {stack-ov}/built-in-users.html[`remote_monitoring_user` built-in user]. +// end::enable-es-module[] +-- -... Add the `username` and `password` settings to the {es} module configuration -file. +. Configure the {es} {xpack} module in {metricbeat} on each {es} node. + + -- -For example, add the following settings in the `modules.d/elasticsearch-xpack.yml` file: +// tag::configure-es-module[] +The `modules.d/elasticsearch-xpack.yml` file contains the following settings: [source,yaml] ---------------------------------- -- module: elasticsearch - ... - username: remote_monitoring_user - password: YOUR_PASSWORD + - module: elasticsearch + metricsets: + - ccr + - cluster_stats + - index + - index_recovery + - index_summary + - ml_job + - node_stats + - shard + period: 10s + hosts: ["http://localhost:9200"] + #username: "user" + #password: "secret" + xpack.enabled: true ---------------------------------- + +By default, the module collects {es} monitoring metrics from +`http://localhost:9200`. If that host and port number are not correct, you must +update the `hosts` setting. If you configured {es} to use encrypted +communications, you must access it via HTTPS. For example, use a `hosts` setting +like `https://localhost:9200`. +// end::configure-es-module[] + +// tag::remote-monitoring-user[] +If Elastic {security-features} are enabled, you must also provide a user ID +and password so that {metricbeat} can collect metrics successfully: + +.. Create a user on the production cluster that has the +{stack-ov}/built-in-roles.html[`remote_monitoring_collector` built-in role]. +Alternatively, use the +{stack-ov}/built-in-users.html[`remote_monitoring_user` built-in user]. + +.. Add the `username` and `password` settings to the {es} module configuration +file. +// end::remote-monitoring-user[] +-- + +. Optional: Disable the system module in {metricbeat}. ++ -- +// tag::disable-system-module[] +By default, the {metricbeat-ref}/metricbeat-module-system.html[system module] is +enabled. The information it collects, however, is not shown on the *Monitoring* +page in {kib}. Unless you want to use that information for other purposes, run +the following command: -.. If you configured {es} to use <>, -you must access it via HTTPS. For example, use a `hosts` setting like -`https://localhost:9200` in the `modules.d/elasticsearch-xpack.yml` file. +["source","sh",subs="attributes,callouts"] +---------------------------------------------------------------------- +metricbeat modules disable system +---------------------------------------------------------------------- -.. Identify where to send the monitoring data. + +// end::disable-system-module[] +-- + +. Identify where to send the monitoring data. + + -- TIP: In production environments, we strongly recommend using a separate cluster @@ -136,48 +148,68 @@ configuration file (`metricbeat.yml`): [source,yaml] ---------------------------------- output.elasticsearch: + # Array of hosts to connect to. hosts: ["http://es-mon-1:9200", "http://es-mon2:9200"] <1> + + # Optional protocol and basic auth credentials. + #protocol: "https" + #username: "elastic" + #password: "changeme" ---------------------------------- <1> In this example, the data is stored on a monitoring cluster with nodes `es-mon-1` and `es-mon-2`. +If you configured the monitoring cluster to use encrypted communications, you +must access it via HTTPS. For example, use a `hosts` setting like +`https://es-mon-1:9200`. + IMPORTANT: The {es} {monitor-features} use ingest pipelines, therefore the cluster that stores the monitoring data must have at least one <>. -For more information about these configuration options, see -{metricbeat-ref}/elasticsearch-output.html[Configure the {es} output]. --- +If {es} {security-features} are enabled on the monitoring cluster, you must +provide a valid user ID and password so that {metricbeat} can send metrics +successfully: -.. If {es} {security-features} are enabled on the monitoring cluster, you -must provide a valid user ID and password so that {metricbeat} can send metrics -successfully. - -... Create a user on the monitoring cluster that has the +.. Create a user on the monitoring cluster that has the {stack-ov}/built-in-roles.html[`remote_monitoring_agent` built-in role]. Alternatively, use the {stack-ov}/built-in-users.html[`remote_monitoring_user` built-in user]. -... Add the `username` and `password` settings to the {es} output information in -the {metricbeat} configuration file (`metricbeat.yml`): +.. Add the `username` and `password` settings to the {es} output information in +the {metricbeat} configuration file. + +For more information about these configuration options, see +{metricbeat-ref}/elasticsearch-output.html[Configure the {es} output]. +-- + +. {metricbeat-ref}/metricbeat-starting.html[Start {metricbeat}] on each node. + +. Disable the default collection of {es} monitoring metrics. + + -- -[source,yaml] +// tag::disable-default-collection[] +Set `xpack.monitoring.elasticsearch.collection.enabled` to `false` on the +production cluster. + +You can use the following API to change this setting: + +[source,js] ---------------------------------- -output.elasticsearch: - ... - username: remote_monitoring_user - password: YOUR_PASSWORD +PUT _cluster/settings +{ + "persistent": { + "xpack.monitoring.elasticsearch.collection.enabled": false + } +} ---------------------------------- --- - -.. If you configured the monitoring cluster to use -<>, you must access it via -HTTPS. For example, use a `hosts` setting like `https://es-mon-1:9200` in the -`metricbeat.yml` file. +// CONSOLE -. <>. +If {es} {security-features} are enabled, you must have `monitor` cluster +privileges to view the cluster settings and `manage` cluster privileges +to change them. -. {metricbeat-ref}/metricbeat-starting.html[Start {metricbeat}]. +// end::disable-default-collection[] +-- . {kibana-ref}/monitoring-data.html[View the monitoring data in {kib}]. diff --git a/docs/reference/monitoring/configuring-monitoring.asciidoc b/docs/reference/monitoring/configuring-monitoring.asciidoc index 9fe5b71d7a18c..e129999e3a51b 100644 --- a/docs/reference/monitoring/configuring-monitoring.asciidoc +++ b/docs/reference/monitoring/configuring-monitoring.asciidoc @@ -12,9 +12,12 @@ methods to collect metrics about {es}: * <> * <> +You can also <>. + To learn about monitoring in general, see {stack-ov}/xpack-monitoring.html[Monitoring the {stack}]. include::collecting-monitoring-data.asciidoc[] include::configuring-metricbeat.asciidoc[] +include::configuring-filebeat.asciidoc[] include::indices.asciidoc[] \ No newline at end of file diff --git a/docs/reference/query-dsl/boosting-query.asciidoc b/docs/reference/query-dsl/boosting-query.asciidoc index 5cd12ce1f00b7..c57235e71606d 100644 --- a/docs/reference/query-dsl/boosting-query.asciidoc +++ b/docs/reference/query-dsl/boosting-query.asciidoc @@ -1,36 +1,60 @@ [[query-dsl-boosting-query]] === Boosting Query -The `boosting` query can be used to effectively demote results that -match a given query. Unlike the "NOT" clause in bool query, this still -selects documents that contain undesirable terms, but reduces their -overall score. +Returns documents matching a `positive` query while reducing the +<> of documents that also match a +`negative` query. -It accepts a `positive` query and a `negative` query. -Only documents that match the `positive` query will be included -in the results list, but documents that also match the `negative` query -will be downgraded by multiplying the original `_score` of the document -with the `negative_boost`. +You can use the `boosting` query to demote certain documents without +excluding them from the search results. + +[[boosting-query-ex-request]] +==== Example request [source,js] --------------------------------------------------- +---- GET /_search { "query": { "boosting" : { "positive" : { "term" : { - "field1" : "value1" + "text" : "apple" } }, "negative" : { "term" : { - "field2" : "value2" + "text" : "pie tart fruit crumble tree" } }, - "negative_boost" : 0.2 + "negative_boost" : 0.5 } } } --------------------------------------------------- +---- // CONSOLE + +[[boosting-top-level-params]] +==== Top-level parameters for `boosting` + +`positive` (Required):: +Query you wish to run. Any returned documents must match this query. + +`negative` (Required):: ++ +-- +Query used to decrease the <> of matching +documents. + +If a returned document matches the `positive` query and this query, the +`boosting` query calculates the final <> +for the document as follows: + +. Take the original relevance score from the `positive` query. +. Multiply the score by the `negative_boost` value. +-- + +`negative_boost` (Required):: +Floating point number between `0` and `1.0` used to decrease the +<> of documents matching the `negative` +query. \ No newline at end of file diff --git a/docs/reference/query-dsl/common-terms-query.asciidoc b/docs/reference/query-dsl/common-terms-query.asciidoc deleted file mode 100644 index 87288778246a6..0000000000000 --- a/docs/reference/query-dsl/common-terms-query.asciidoc +++ /dev/null @@ -1,299 +0,0 @@ -[[query-dsl-common-terms-query]] -=== Common Terms Query - -The `common` terms query is a modern alternative to stopwords which -improves the precision and recall of search results (by taking stopwords -into account), without sacrificing performance. - -[float] -==== The problem - -Every term in a query has a cost. A search for `"The brown fox"` -requires three term queries, one for each of `"the"`, `"brown"` and -`"fox"`, all of which are executed against all documents in the index. -The query for `"the"` is likely to match many documents and thus has a -much smaller impact on relevance than the other two terms. - -Previously, the solution to this problem was to ignore terms with high -frequency. By treating `"the"` as a _stopword_, we reduce the index size -and reduce the number of term queries that need to be executed. - -The problem with this approach is that, while stopwords have a small -impact on relevance, they are still important. If we remove stopwords, -we lose precision, (eg we are unable to distinguish between `"happy"` -and `"not happy"`) and we lose recall (eg text like `"The The"` or -`"To be or not to be"` would simply not exist in the index). - -[float] -==== The solution - -The `common` terms query divides the query terms into two groups: more -important (ie _low frequency_ terms) and less important (ie _high -frequency_ terms which would previously have been stopwords). - -First it searches for documents which match the more important terms. -These are the terms which appear in fewer documents and have a greater -impact on relevance. - -Then, it executes a second query for the less important terms -- terms -which appear frequently and have a low impact on relevance. But instead -of calculating the relevance score for *all* matching documents, it only -calculates the `_score` for documents already matched by the first -query. In this way the high frequency terms can improve the relevance -calculation without paying the cost of poor performance. - -If a query consists only of high frequency terms, then a single query is -executed as an `AND` (conjunction) query, in other words all terms are -required. Even though each individual term will match many documents, -the combination of terms narrows down the resultset to only the most -relevant. The single query can also be executed as an `OR` with a -specific -<>, -in this case a high enough value should probably be used. - -Terms are allocated to the high or low frequency groups based on the -`cutoff_frequency`, which can be specified as an absolute frequency -(`>=1`) or as a relative frequency (`0.0 .. 1.0`). (Remember that document -frequencies are computed on a per shard level as explained in the blog post -{defguide}/relevance-is-broken.html[Relevance is broken].) - -Perhaps the most interesting property of this query is that it adapts to -domain specific stopwords automatically. For example, on a video hosting -site, common terms like `"clip"` or `"video"` will automatically behave -as stopwords without the need to maintain a manual list. - -[float] -==== Examples - -In this example, words that have a document frequency greater than 0.1% -(eg `"this"` and `"is"`) will be treated as _common terms_. - -[source,js] --------------------------------------------------- -GET /_search -{ - "query": { - "common": { - "body": { - "query": "this is bonsai cool", - "cutoff_frequency": 0.001 - } - } - } -} --------------------------------------------------- -// CONSOLE - -The number of terms which should match can be controlled with the -<> -(`high_freq`, `low_freq`), `low_freq_operator` (default `"or"`) and -`high_freq_operator` (default `"or"`) parameters. - -For low frequency terms, set the `low_freq_operator` to `"and"` to make -all terms required: - -[source,js] --------------------------------------------------- -GET /_search -{ - "query": { - "common": { - "body": { - "query": "nelly the elephant as a cartoon", - "cutoff_frequency": 0.001, - "low_freq_operator": "and" - } - } - } -} --------------------------------------------------- -// CONSOLE - -which is roughly equivalent to: - -[source,js] --------------------------------------------------- -GET /_search -{ - "query": { - "bool": { - "must": [ - { "term": { "body": "nelly"}}, - { "term": { "body": "elephant"}}, - { "term": { "body": "cartoon"}} - ], - "should": [ - { "term": { "body": "the"}}, - { "term": { "body": "as"}}, - { "term": { "body": "a"}} - ] - } - } -} --------------------------------------------------- -// CONSOLE - -Alternatively use -<> -to specify a minimum number or percentage of low frequency terms which -must be present, for instance: - -[source,js] --------------------------------------------------- -GET /_search -{ - "query": { - "common": { - "body": { - "query": "nelly the elephant as a cartoon", - "cutoff_frequency": 0.001, - "minimum_should_match": 2 - } - } - } -} --------------------------------------------------- -// CONSOLE - -which is roughly equivalent to: - -[source,js] --------------------------------------------------- -GET /_search -{ - "query": { - "bool": { - "must": { - "bool": { - "should": [ - { "term": { "body": "nelly"}}, - { "term": { "body": "elephant"}}, - { "term": { "body": "cartoon"}} - ], - "minimum_should_match": 2 - } - }, - "should": [ - { "term": { "body": "the"}}, - { "term": { "body": "as"}}, - { "term": { "body": "a"}} - ] - } - } -} --------------------------------------------------- -// CONSOLE - -A different -<> -can be applied for low and high frequency terms with the additional -`low_freq` and `high_freq` parameters. Here is an example when providing -additional parameters (note the change in structure): - -[source,js] --------------------------------------------------- -GET /_search -{ - "query": { - "common": { - "body": { - "query": "nelly the elephant not as a cartoon", - "cutoff_frequency": 0.001, - "minimum_should_match": { - "low_freq" : 2, - "high_freq" : 3 - } - } - } - } -} --------------------------------------------------- -// CONSOLE - -which is roughly equivalent to: - -[source,js] --------------------------------------------------- -GET /_search -{ - "query": { - "bool": { - "must": { - "bool": { - "should": [ - { "term": { "body": "nelly"}}, - { "term": { "body": "elephant"}}, - { "term": { "body": "cartoon"}} - ], - "minimum_should_match": 2 - } - }, - "should": { - "bool": { - "should": [ - { "term": { "body": "the"}}, - { "term": { "body": "not"}}, - { "term": { "body": "as"}}, - { "term": { "body": "a"}} - ], - "minimum_should_match": 3 - } - } - } - } -} --------------------------------------------------- -// CONSOLE - -In this case it means the high frequency terms have only an impact on -relevance when there are at least three of them. But the most -interesting use of the -<> -for high frequency terms is when there are only high frequency terms: - -[source,js] --------------------------------------------------- -GET /_search -{ - "query": { - "common": { - "body": { - "query": "how not to be", - "cutoff_frequency": 0.001, - "minimum_should_match": { - "low_freq" : 2, - "high_freq" : 3 - } - } - } - } -} --------------------------------------------------- -// CONSOLE - -which is roughly equivalent to: - -[source,js] --------------------------------------------------- -GET /_search -{ - "query": { - "bool": { - "should": [ - { "term": { "body": "how"}}, - { "term": { "body": "not"}}, - { "term": { "body": "to"}}, - { "term": { "body": "be"}} - ], - "minimum_should_match": "3<50%" - } - } -} --------------------------------------------------- -// CONSOLE - -The high frequency generated query is then slightly less restrictive -than with an `AND`. - -The `common` terms query also supports `boost` and `analyzer` as -parameters. diff --git a/docs/reference/query-dsl/constant-score-query.asciidoc b/docs/reference/query-dsl/constant-score-query.asciidoc index aa7ee60aa5c6f..bfcece8d62e2f 100644 --- a/docs/reference/query-dsl/constant-score-query.asciidoc +++ b/docs/reference/query-dsl/constant-score-query.asciidoc @@ -1,12 +1,12 @@ [[query-dsl-constant-score-query]] === Constant Score Query -A query that wraps another query and simply returns a -constant score equal to the query boost for every document in the -filter. Maps to Lucene `ConstantScoreQuery`. +Wraps a <> and returns every matching +document with a <> equal to the `boost` +parameter value. [source,js] --------------------------------------------------- +---- GET /_search { "query": { @@ -18,8 +18,22 @@ GET /_search } } } --------------------------------------------------- +---- // CONSOLE -Filter clauses are executed in <>, -meaning that scoring is ignored and clauses are considered for caching. +[[constant-score-top-level-params]] +==== Top-level parameters for `constant_score` +`filter`:: ++ +-- +<> you wish to run. Any returned documents +must match this query. Required. + +Filter queries do not calculate <>. To +speed up performance, {es} automatically caches frequently used filter queries. +-- + +`boost`:: +Floating point number used as the constant <> for every document matching the `filter` query. Default is `1.0`. +Optional. \ No newline at end of file diff --git a/docs/reference/query-dsl/dis-max-query.asciidoc b/docs/reference/query-dsl/dis-max-query.asciidoc index f05f97107a0a4..9a0f1fb7b039c 100644 --- a/docs/reference/query-dsl/dis-max-query.asciidoc +++ b/docs/reference/query-dsl/dis-max-query.asciidoc @@ -1,48 +1,63 @@ [[query-dsl-dis-max-query]] -=== Dis Max Query - -A query that generates the union of documents produced by its -subqueries, and that scores each document with the maximum score for -that document as produced by any subquery, plus a tie breaking increment -for any additional matching subqueries. - -This is useful when searching for a word in multiple fields with -different boost factors (so that the fields cannot be combined -equivalently into a single search field). We want the primary score to -be the one associated with the highest boost, not the sum of the field -scores (as Boolean Query would give). If the query is "albino elephant" -this ensures that "albino" matching one field and "elephant" matching -another gets a higher score than "albino" matching both fields. To get -this result, use both Boolean Query and DisjunctionMax Query: for each -term a DisjunctionMaxQuery searches for it in each field, while the set -of these DisjunctionMaxQuery's is combined into a BooleanQuery. - -The tie breaker capability allows results that include the same term in -multiple fields to be judged better than results that include this term -in only the best of those multiple fields, without confusing this with -the better case of two different terms in the multiple fields. The -default `tie_breaker` is `0.0`. - -This query maps to Lucene `DisjunctionMaxQuery`. +=== Disjunction Max Query + +Returns documents matching one or more wrapped queries, called query clauses or +clauses. + +If a returned document matches multiple query clauses, the `dis_max` query +assigns the document the highest relevance score from any matching clause, plus +a tie breaking increment for any additional matching subqueries. + +You can use the `dis_max` to search for a term in fields mapped with different +<> factors. + +[[query-dsl-dis-max-query-ex-request]] +==== Example request [source,js] --------------------------------------------------- +---- GET /_search { "query": { "dis_max" : { - "tie_breaker" : 0.7, - "boost" : 1.2, "queries" : [ - { - "term" : { "age" : 34 } - }, - { - "term" : { "age" : 35 } - } - ] + { "term" : { "title" : "Quick pets" }}, + { "term" : { "body" : "Quick pets" }} + ], + "tie_breaker" : 0.7 } } } --------------------------------------------------- +---- // CONSOLE + +[[query-dsl-dis-max-query-top-level-params]] +==== Top-level parameters for `dis_max` + +`queries` (Required):: +(array of query objects) Contains one or more query clauses. Returned documents +**must match one or more** of these queries. If a document matches multiple +queries, {es} uses the highest <>. + +`tie_breaker` (Optional):: ++ +-- +(float) Floating point number between `0` and `1.0` used to increase the +<> of documents matching multiple query +clauses. Defaults to `0.0`. + +You can use the `tie_breaker` value to assign higher relevance scores to +documents that contain the same term in multiple fields than documents that +contain this term in only the best of those multiple fields, without confusing +this with the better case of two different terms in the multiple fields. + +If a document matches multiple clauses, the `dis_max` query calculates the +relevance score for the document as follows: + +. Take the relevance score from a matching clause with the highest score. +. Multiply the score from any other matching clauses by the `tie_breaker` value. +. Add the highest score to the multiplied scores. + +If the `tie_breaker` value is greater than `0.0`, all matching clauses count, +but the clause with the highest score counts most. +-- \ No newline at end of file diff --git a/docs/reference/query-dsl/exists-query.asciidoc b/docs/reference/query-dsl/exists-query.asciidoc index 9c7553e314c43..f35e97db8ec18 100644 --- a/docs/reference/query-dsl/exists-query.asciidoc +++ b/docs/reference/query-dsl/exists-query.asciidoc @@ -1,8 +1,14 @@ [[query-dsl-exists-query]] === Exists Query -Returns documents that contain a value other than `null` or `[]` in a provided -field. +Returns documents that contain an indexed value for a field. + +An indexed value may not exist for a document's field due to a variety of reasons: + +* The field in the source JSON is `null` or `[]` +* The field has `"index" : false` set in the mapping +* The length of the field value exceeded an `ignore_above` setting in the mapping +* The field value was malformed and `ignore_malformed` was defined in the mapping [[exists-query-ex-request]] ==== Example request @@ -25,8 +31,7 @@ GET /_search `field`:: Name of the field you wish to search. + -To return a document, this field must exist and contain a value other -than `null` or `[]`. These values can include: +While a field is deemed non-existant if the JSON value is `null` or `[]`, these values will indicate the field does exist: + * Empty strings, such as `""` or `"-"` * Arrays containing `null` and another value, such as `[null, "foo"]` @@ -36,13 +41,13 @@ than `null` or `[]`. These values can include: ==== Notes [[find-docs-null-values]] -===== Find documents with null values -To find documents that contain only `null` values or `[]` in a provided field, +===== Find documents missing indexed values +To find documents that are missing an indexed value for a field, use the `must_not` <> with the `exists` query. -The following search returns documents that contain only `null` values or `[]` -in the `user` field. +The following search returns documents that are missing an indexed value for +the `user` field. [source,js] ---- diff --git a/docs/reference/query-dsl/full-text-queries.asciidoc b/docs/reference/query-dsl/full-text-queries.asciidoc index 0af99b61f194f..8fc53bc7e9b8a 100644 --- a/docs/reference/query-dsl/full-text-queries.asciidoc +++ b/docs/reference/query-dsl/full-text-queries.asciidoc @@ -29,10 +29,6 @@ The queries in this group are: The multi-field version of the `match` query. -<>:: - - A more specialized query which gives more preference to uncommon words. - <>:: Supports the compact Lucene <>, @@ -59,8 +55,6 @@ include::match-bool-prefix-query.asciidoc[] include::multi-match-query.asciidoc[] -include::common-terms-query.asciidoc[] - include::query-string-query.asciidoc[] include::simple-query-string-query.asciidoc[] diff --git a/docs/reference/query-dsl/geo-bounding-box-query.asciidoc b/docs/reference/query-dsl/geo-bounding-box-query.asciidoc index 487e944c09e10..1a088a350145f 100644 --- a/docs/reference/query-dsl/geo-bounding-box-query.asciidoc +++ b/docs/reference/query-dsl/geo-bounding-box-query.asciidoc @@ -39,7 +39,7 @@ Then the following simple query can be executed with a [source,js] -------------------------------------------------- -GET /_search +GET my_locations/_search { "query": { "bool" : { @@ -94,7 +94,7 @@ representations of the geo point, the filter can accept it as well: [source,js] -------------------------------------------------- -GET /_search +GET my_locations/_search { "query": { "bool" : { @@ -129,7 +129,7 @@ conform with http://geojson.org/[GeoJSON]. [source,js] -------------------------------------------------- -GET /_search +GET my_locations/_search { "query": { "bool" : { @@ -157,7 +157,7 @@ Format in `lat,lon`. [source,js] -------------------------------------------------- -GET /_search +GET my_locations/_search { "query": { "bool" : { @@ -183,7 +183,7 @@ GET /_search [source,js] -------------------------------------------------- -GET /_search +GET my_locations/_search { "query": { "bool" : { @@ -208,7 +208,7 @@ GET /_search [source,js] -------------------------------------------------- -GET /_search +GET my_locations/_search { "query": { "bool" : { @@ -243,7 +243,7 @@ geohash the geohash can be specified in both `top_left` and [source,js] -------------------------------------------------- -GET /_search +GET my_locations/_search { "query": { "geo_bounding_box" : { @@ -273,7 +273,7 @@ values separately. [source,js] -------------------------------------------------- -GET /_search +GET my_locations/_search { "query": { "bool" : { @@ -323,7 +323,7 @@ are not supported. Here is an example: [source,js] -------------------------------------------------- -GET /_search +GET my_locations/_search { "query": { "bool" : { diff --git a/docs/reference/query-dsl/intervals-query.asciidoc b/docs/reference/query-dsl/intervals-query.asciidoc index f5788783f7e63..7353ca137f3e1 100644 --- a/docs/reference/query-dsl/intervals-query.asciidoc +++ b/docs/reference/query-dsl/intervals-query.asciidoc @@ -40,7 +40,6 @@ POST _search } ] }, - "boost" : 2.0, "_name" : "favourite_food" } } @@ -83,6 +82,53 @@ to search across multiple fields as if they were all the same field; for example you could index the same text into stemmed and unstemmed fields, and search for stemmed tokens near unstemmed ones. +[[intervals-prefix]] +==== `prefix` + +The `prefix` rule finds terms that start with a specified prefix. The prefix will +expand to match at most 128 terms; if there are more matching terms in the index, +then an error will be returned. To avoid this limit, enable the +<> option on the field being searched. + +[horizontal] +`prefix`:: +Match terms starting with this prefix +`analyzer`:: +Which analyzer should be used to normalize the `prefix`. By default, the +search analyzer of the top-level field will be used. +`use_field`:: +If specified, then match intervals from this field rather than the top-level field. +The `prefix` will be normalized using the search analyzer from this field, unless +`analyzer` is specified separately. + +[[intervals-wildcard]] +==== `wildcard` + +The `wildcard` rule finds terms that match a wildcard pattern. The pattern will +expand to match at most 128 terms; if there are more matching terms in the index, +then an error will be returned. + +[horizontal] +`pattern`:: +Find terms matching this pattern ++ +-- +This parameter supports two wildcard operators: + +* `?`, which matches any single character +* `*`, which can match zero or more characters, including an empty one + +WARNING: Avoid beginning patterns with `*` or `?`. This can increase +the iterations needed to find matching terms and slow search performance. +-- +`analyzer`:: +Which analyzer should be used to normalize the `pattern`. By default, the +search analyzer of the top-level field will be used. +`use_field`:: +If specified, then match intervals from this field rather than the top-level field. +The `pattern` will be normalized using the search analyzer from this field, unless +`analyzer` is specified separately. + [[intervals-all_of]] ==== `all_of` @@ -298,4 +344,4 @@ POST _search } } -------------------------------------------------- -// CONSOLE \ No newline at end of file +// CONSOLE diff --git a/docs/reference/query-dsl/match-query.asciidoc b/docs/reference/query-dsl/match-query.asciidoc index 89a0a942b79ce..4fcb40a76ec9c 100644 --- a/docs/reference/query-dsl/match-query.asciidoc +++ b/docs/reference/query-dsl/match-query.asciidoc @@ -75,7 +75,8 @@ rewritten. Fuzzy transpositions (`ab` -> `ba`) are allowed by default but can be disabled by setting `fuzzy_transpositions` to `false`. -Note that fuzzy matching is not applied to terms with synonyms, as under the hood +NOTE: Fuzzy matching is not applied to terms with synonyms or in cases where the +analysis process produces multiple tokens at the same position. Under the hood these terms are expanded to a special synonym query that blends term frequencies, which does not support fuzzy expansion. @@ -119,50 +120,6 @@ GET /_search -------------------------------------------------- // CONSOLE -[[query-dsl-match-query-cutoff]] -===== Cutoff frequency - -The match query supports a `cutoff_frequency` that allows -specifying an absolute or relative document frequency where high -frequency terms are moved into an optional subquery and are only scored -if one of the low frequency (below the cutoff) terms in the case of an -`or` operator or all of the low frequency terms in the case of an `and` -operator match. - -This query allows handling `stopwords` dynamically at runtime, is domain -independent and doesn't require a stopword file. It prevents scoring / -iterating high frequency terms and only takes the terms into account if a -more significant / lower frequency term matches a document. Yet, if all -of the query terms are above the given `cutoff_frequency` the query is -automatically transformed into a pure conjunction (`and`) query to -ensure fast execution. - -The `cutoff_frequency` can either be relative to the total number of -documents if in the range `[0..1)` or absolute if greater or equal to -`1.0`. - -Here is an example showing a query composed of stopwords exclusively: - -[source,js] --------------------------------------------------- -GET /_search -{ - "query": { - "match" : { - "message" : { - "query" : "to be or not to be", - "cutoff_frequency" : 0.001 - } - } - } -} --------------------------------------------------- -// CONSOLE - -IMPORTANT: The `cutoff_frequency` option operates on a per-shard-level. This means -that when trying it out on test indexes with low document numbers you -should follow the advice in {defguide}/relevance-is-broken.html[Relevance is broken]. - [[query-dsl-match-query-synonyms]] ===== Synonyms diff --git a/docs/reference/query-dsl/mlt-query.asciidoc b/docs/reference/query-dsl/mlt-query.asciidoc index 509b4a9b44094..e59cd24292d48 100644 --- a/docs/reference/query-dsl/mlt-query.asciidoc +++ b/docs/reference/query-dsl/mlt-query.asciidoc @@ -232,9 +232,8 @@ The syntax is the same as the <>. +`auto_generate_synonyms_phrase_query` and `fuzzy_transpositions`, +as explained in <>. [IMPORTANT] [[operator-min]] @@ -247,9 +247,7 @@ The score from each `match` clause is added together, then divided by the number of `match` clauses. Also, accepts `analyzer`, `boost`, `operator`, `minimum_should_match`, -`fuzziness`, `lenient`, `prefix_length`, `max_expansions`, `rewrite`, `zero_terms_query` -and `cutoff_frequency`, as explained in <>, but -*see <>*. +`fuzziness`, `lenient`, `prefix_length`, `max_expansions`, `rewrite`, and `zero_terms_query`. [[type-phrase]] ==== `phrase` and `phrase_prefix` @@ -292,9 +290,9 @@ GET /_search -------------------------------------------------- // CONSOLE -Also, accepts `analyzer`, `boost`, `lenient`, `slop` and `zero_terms_query` as explained -in <>. Type `phrase_prefix` additionally accepts -`max_expansions`. +Also, accepts `analyzer`, <>, `lenient` and `zero_terms_query` as explained +in <>, as well as `slop` which is explained in <>. +Type `phrase_prefix` additionally accepts `max_expansions`. [IMPORTANT] [[phrase-fuzziness]] @@ -389,8 +387,7 @@ explanation: +blended("smith", fields: [first_name, last_name]) Also, accepts `analyzer`, `boost`, `operator`, `minimum_should_match`, -`lenient`, `zero_terms_query` and `cutoff_frequency`, as explained in -<>. +`lenient` and `zero_terms_query`. [[cross-field-analysis]] ===== `cross_field` and analysis @@ -554,5 +551,4 @@ explained in <> are supported. The construct term queries, but do not have an effect on the prefix query constructed from the final term. -The `slop` and `cutoff_frequency` parameters are not supported by this query -type. +The `slop` parameter is not supported by this query type. diff --git a/docs/reference/query-dsl/query_filter_context.asciidoc b/docs/reference/query-dsl/query_filter_context.asciidoc index 79f8c4bd960b0..6e7177ed19c18 100644 --- a/docs/reference/query-dsl/query_filter_context.asciidoc +++ b/docs/reference/query-dsl/query_filter_context.asciidoc @@ -52,12 +52,12 @@ GET /_search "query": { <1> "bool": { <2> "must": [ - { "match": { "title": "Search" }}, <2> - { "match": { "content": "Elasticsearch" }} <2> + { "match": { "title": "Search" }}, + { "match": { "content": "Elasticsearch" }} ], "filter": [ <3> - { "term": { "status": "published" }}, <4> - { "range": { "publish_date": { "gte": "2015-01-01" }}} <4> + { "term": { "status": "published" }}, + { "range": { "publish_date": { "gte": "2015-01-01" }}} ] } } @@ -68,11 +68,16 @@ GET /_search <2> The `bool` and two `match` clauses are used in query context, which means that they are used to score how well each document matches. -<3> The `filter` parameter indicates filter context. -<4> The `term` and `range` clauses are used in filter context. - They will filter out documents which do not match, but they will +<3> The `filter` parameter indicates filter context. Its `term` and + `range` clauses are used in filter context. They will filter out + documents which do not match, but they will not affect the score for matching documents. +WARNING: Scores calculated for queries in query context are represented +as single precision floating point numbers; they have only +24 bits for significand's precision. Score calculations that exceed the +significand's precision will be converted to floats with loss of precision. + TIP: Use query clauses in query context for conditions which should affect the score of matching documents (i.e. how well does the document match), and use all other query clauses in filter context. diff --git a/docs/reference/query-dsl/range-query.asciidoc b/docs/reference/query-dsl/range-query.asciidoc index 61c46996949f1..27db882fe1dd3 100644 --- a/docs/reference/query-dsl/range-query.asciidoc +++ b/docs/reference/query-dsl/range-query.asciidoc @@ -1,14 +1,16 @@ [[query-dsl-range-query]] === Range Query -Matches documents with fields that have terms within a certain range. -The type of the Lucene query depends on the field type, for `string` -fields, the `TermRangeQuery`, while for number/date fields, the query is -a `NumericRangeQuery`. The following example returns all documents where -`age` is between `10` and `20`: +Returns documents that contain terms within a provided range. + +[[range-query-ex-request]] +==== Example request + +The following search returns documents where the `age` field contains a term +between `10` and `20`. [source,js] --------------------------------------------------- +---- GET _search { "query": { @@ -21,147 +23,209 @@ GET _search } } } --------------------------------------------------- +---- // CONSOLE -The `range` query accepts the following parameters: +[[range-query-top-level-params]] +==== Top-level parameters for `range` + +``:: ++ +-- +Field you wish to search. +-- + +[[range-query-field-params]] +==== Parameters for `` + +`gt`:: +Greater than. Optional. + +`gte`:: +Greater than or equal to. Optional. + +`lt`:: +Less than. Optional. + +`lte`:: +Less than or equal to. Optional. + +`format`:: ++ +-- +Date format used to convert `date` values in the query. + +By default, {es} uses the <> provided in the +``'s mapping. This value overrides that mapping format. -[horizontal] -`gte`:: Greater-than or equal to -`gt`:: Greater-than -`lte`:: Less-than or equal to -`lt`:: Less-than -`boost`:: Sets the boost value of the query, defaults to `1.0` +For valid syntax, see <>. Optional. +[WARNING] +==== +If a `format` and `date` value are incomplete, {es} replaces any missing year, +month, or date component with the start of +https://en.wikipedia.org/wiki/Unix_time[Unix time], which is January 1st, 1970. + +For example, if the `format` value is `dd`, {es} converts a `gte` value of `10` +to `1970-01-10T00:00:00.000Z`. +==== + +-- + +[[querying-range-fields]] +`relation`:: ++ +-- +Indicates how the range query matches values for `range` fields. Optional. Valid +values are: + +`INTERSECTS` (Default):: +Matches documents with a range field value that intersects the query's range. + +`CONTAINS`:: +Matches documents with a range field value that entirely contains the query's range. + +`WITHIN`:: +Matches documents with a range field value entirely within the query's range. +-- + +`time_zone`:: ++ +-- +https://en.wikipedia.org/wiki/List_of_UTC_time_offsets[Coordinated Universal +Time (UTC) offset] or +https://en.wikipedia.org/wiki/List_of_tz_database_time_zones[IANA time zone] +used to convert `date` values in the query to UTC. Optional. + +Valid values are ISO 8601 UTC offsets, such as `+01:00` or -`08:00`, and IANA +time zone IDs, such as `America/Los_Angeles`. + +For an example query using the `time_zone` parameter, see +<>. + +[NOTE] +==== +The `time_zone` parameter does **not** affect the <> value +of `now`. `now` is always the current system time in UTC. + +However, the `time_zone` parameter does convert dates calculated using `now` and +<>. For example, the `time_zone` parameter will +convert a value of `now/d`. +==== +-- + +`boost`:: ++ +-- +Floating point number used to decrease or increase the +<> of a query. Default is `1.0`. +Optional. + +You can use the `boost` parameter to adjust relevance scores for searches +containing two or more queries. + +Boost values are relative to the default value of `1.0`. A boost value between +`0` and `1.0` decreases the relevance score. A value greater than `1.0` +increases the relevance score. +-- + +[[range-query-notes]] +==== Notes [[ranges-on-dates]] -==== Ranges on date fields +===== Using the `range` query with `date` fields + +When the `` parameter is a <> field datatype, you can use +<> with the following parameters: -When running `range` queries on fields of type <>, ranges can be -specified using <>: +* `gt` +* `gte` +* `lt` +* `lte` + +For example, the following search returns documents where the `timestamp` field +contains a date between today and yesterday. [source,js] --------------------------------------------------- +---- GET _search { "query": { "range" : { - "date" : { + "timestamp" : { "gte" : "now-1d/d", "lt" : "now/d" } } } } --------------------------------------------------- +---- // CONSOLE -===== Date math and rounding - -When using <> to round dates to the nearest day, month, -hour, etc, the rounded dates depend on whether the ends of the ranges are -inclusive or exclusive. -Rounding up moves to the last millisecond of the rounding scope, and rounding -down to the first millisecond of the rounding scope. For example: +[[range-query-date-math-rounding]] +====== Date math and rounding +{es} rounds <> values in parameters as follows: -[horizontal] `gt`:: ++ +-- +Rounds up to the lastest millisecond. - Greater than the date rounded up: `2014-11-18||/M` becomes - `2014-11-30T23:59:59.999`, ie excluding the entire month. +For example, `2014-11-18||/M` rounds up to `2014-11-30T23:59:59.999`, including +the entire month. +-- `gte`:: ++ +-- +Rounds down to the first millisecond. - Greater than or equal to the date rounded down: `2014-11-18||/M` becomes - `2014-11-01`, ie including the entire month. +For example, `2014-11-18||/M` rounds down to `2014-11-01`, excluding +the entire month. +-- `lt`:: ++ +-- +Rounds down to the first millisecond. - Less than the date rounded down: `2014-11-18||/M` becomes `2014-11-01`, ie - excluding the entire month. +For example, `2014-11-18||/M` rounds down to `2014-11-01`, excluding +the entire month. +-- `lte`:: ++ +-- +Rounds up to the lastest millisecond. - Less than or equal to the date rounded up: `2014-11-18||/M` becomes - `2014-11-30T23:59:59.999`, ie including the entire month. +For example, `2014-11-18||/M` rounds up to `2014-11-30T23:59:59.999`, including +the entire month. +-- -===== Date format in range queries +[[range-query-time-zone]] +===== Example query using `time_zone` parameter -Formatted dates will be parsed using the <> -specified on the <> field by default, but it can be overridden by -passing the `format` parameter to the `range` query: +You can use the `time_zone` parameter to convert `date` values to UTC using a +UTC offset. For example: [source,js] --------------------------------------------------- -GET _search -{ - "query": { - "range" : { - "born" : { - "gte": "01/01/2012", - "lte": "2013", - "format": "dd/MM/yyyy||yyyy" - } - } - } -} --------------------------------------------------- -// 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 -time zone in the date value itself (if the <> -accepts it), or it can be specified as the `time_zone` parameter: - -[source,js] --------------------------------------------------- +---- GET _search { "query": { "range" : { "timestamp" : { - "gte": "2015-01-01 00:00:00", <1> - "lte": "now", <2> - "time_zone": "+01:00" + "time_zone": "+01:00", <1> + "gte": "2015-01-01 00:00:00", <2> + "lte": "now" <3> } } } } --------------------------------------------------- +---- // CONSOLE -<1> This date will be converted to `2014-12-31T23:00:00 UTC`. -<2> `now` is not affected by the `time_zone` parameter, its always the current system time (in UTC). -However, when using <> (e.g. down to the nearest day using `now/d`), -the provided `time_zone` will be considered. - - -[[querying-range-fields]] -==== Querying range fields - -`range` queries can be used on fields of type <>, allowing to -match a range specified in the query with a range field value in the document. -The `relation` parameter controls how these two ranges are matched: - -[horizontal] -`WITHIN`:: - - Matches documents who's range field is entirely within the query's range. - -`CONTAINS`:: - - Matches documents who's range field entirely contains the query's range. - -`INTERSECTS`:: - - Matches documents who's range field intersects the query's range. - This is the default value when querying range fields. - -For examples, see <> mapping type. +<1> Indicates that `date` values use a UTC offset of `+01:00`. +<2> With a UTC offset of `+01:00`, {es} converts this date to +`2014-12-31T23:00:00 UTC`. +<3> The `time_zone` parameter does not affect the `now` value. \ No newline at end of file diff --git a/docs/reference/query-dsl/regexp-syntax.asciidoc b/docs/reference/query-dsl/regexp-syntax.asciidoc index 6a929ba98d502..74094b0cab1b0 100644 --- a/docs/reference/query-dsl/regexp-syntax.asciidoc +++ b/docs/reference/query-dsl/regexp-syntax.asciidoc @@ -224,7 +224,7 @@ follows a tilde `"~"` is negated. For instance, `"ab~cd" means: * Starts with `a` * Followed by `b` -* Followed by a string of any length that it anything but `c` +* Followed by a string of any length that is anything but `c` * Ends with `d` For the string `"abcdef"`: diff --git a/docs/reference/query-dsl/script-score-query.asciidoc b/docs/reference/query-dsl/script-score-query.asciidoc index e8d97a31fa95f..401d323f6fff4 100644 --- a/docs/reference/query-dsl/script-score-query.asciidoc +++ b/docs/reference/query-dsl/script-score-query.asciidoc @@ -76,6 +76,8 @@ to be the most efficient by using the internal mechanisms. -------------------------------------------------- // NOTCONSOLE +[role="xpack"] +[testenv="basic"] [[vector-functions]] ===== Functions for vector fields @@ -197,8 +199,7 @@ a vector function is executed, 0 is returned as a result for this document. NOTE: If a document's dense vector field has a number of dimensions -different from the query's vector, 0 is used for missing dimensions -in the calculations of vector functions. +different from the query's vector, an error will be thrown. [[random-score-function]] diff --git a/docs/reference/query-dsl/term-level-queries.asciidoc b/docs/reference/query-dsl/term-level-queries.asciidoc index dd7ea38819f01..5941aea76c2fa 100644 --- a/docs/reference/query-dsl/term-level-queries.asciidoc +++ b/docs/reference/query-dsl/term-level-queries.asciidoc @@ -1,68 +1,59 @@ [[term-level-queries]] -== Term level queries +== Term-level queries -While the <> will analyze the query -string before executing, the _term-level queries_ operate on the exact terms -that are stored in the inverted index, and will normalize terms before executing -only for <> fields with <> property. +You can use **term-level queries** to find documents based on precise values in +structured data. Examples of structured data include date ranges, IP addresses, +prices, or product IDs. -These queries are usually used for structured data like numbers, dates, and -enums, rather than full text fields. Alternatively, they allow you to craft -low-level queries, foregoing the analysis process. +Unlike <>, term-level queries do not +analyze search terms. Instead, term-level queries match the exact terms stored +in a field. -The queries in this group are: -<>:: +[NOTE] +==== +Term-level queries still normalize search terms for `keyword` fields with the +`normalizer` property. For more details, see <>. +==== - Find documents which contain the exact term specified in the field - specified. +[float] +[[term-level-query-types]] +=== Types of term-level queries -<>:: +<>:: +Returns documents that contain an exact term in a provided field. - Find documents which contain any of the exact terms specified in the field - specified. +<>:: +Returns documents that contain one or more exact terms in a provided field. <>:: - - Find documents which match with one or more of the specified terms. The - number of terms that must match depend on the specified minimum should - match field or script. +Returns documents that contain a minimum number of exact terms in a provided +field. You can define the minimum number of matching terms using a field or +script. <>:: - - Find documents where the field specified contains values (dates, numbers, - or strings) in the range specified. +Returns documents that contain terms within a provided range. <>:: - - Find documents where the field specified contains any non-null value. +Returns documents that contain any indexed value for a field. <>:: - - Find documents where the field specified contains terms which begin with - the exact prefix specified. +Returns documents that contain a specific prefix in a provided field. <>:: - - Find documents where the field specified contains terms which match the - pattern specified, where the pattern supports single character wildcards - (`?`) and multi-character wildcards (`*`) +Returns documents that contain terms matching a wildcard pattern. <>:: - - Find documents where the field specified contains terms which match the - <> specified. +Returns documents that contain terms matching a +https://en.wikipedia.org/wiki/Regular_expression[regular expression]. <>:: - - Find documents where the field specified contains terms which are fuzzily - similar to the specified term. Fuzziness is measured as a - http://en.wikipedia.org/wiki/Damerau%E2%80%93Levenshtein_distance[Levenshtein edit distance] - of 1 or 2. +Returns documents that contain terms similar to the search term. {es} measures +similarity, or fuzziness, using a +http://en.wikipedia.org/wiki/Levenshtein_distance[Levenshtein edit distance]. <>:: - - Find documents with the specified IDs. +Returns documents based on their <>. include::term-query.asciidoc[] diff --git a/docs/reference/query-dsl/terms-query.asciidoc b/docs/reference/query-dsl/terms-query.asciidoc index db4597fbea504..53ae0163f804f 100644 --- a/docs/reference/query-dsl/terms-query.asciidoc +++ b/docs/reference/query-dsl/terms-query.asciidoc @@ -1,121 +1,252 @@ [[query-dsl-terms-query]] === Terms Query -Filters documents that have fields that match any of the provided terms -(*not analyzed*). For example: +Returns documents that contain one or more *exact* terms in a provided field. + +The `terms` query is the same as the <>, +except you can search for multiple values. + +[[terms-query-ex-request]] +==== Example request + +The following search returns documents where the `user` field contains `kimchy` +or `elasticsearch`. [source,js] --------------------------------------------------- +---- GET /_search { - "query": { - "terms" : { "user" : ["kimchy", "elasticsearch"]} + "query" : { + "terms" : { + "user" : ["kimchy", "elasticsearch"], + "boost" : 1.0 + } } } --------------------------------------------------- +---- // CONSOLE -NOTE: Highlighting `terms` queries is best-effort only, so terms of a `terms` -query might not be highlighted depending on the highlighter implementation that -is selected and on the number of terms in the `terms` query. +[[terms-top-level-params]] +==== Top-level parameters for `terms` +``:: ++ +-- +Field you wish to search. + +The value of this parameter is an array of terms you wish to find in the +provided field. To return a document, one or more terms must exactly match a +field value, including whitespace and capitalization. + +By default, {es} limits the `terms` query to a maximum of 65,536 +terms. You can change this limit using the <> setting. + +[NOTE] +To use the field values of an existing document as search terms, use the +<> parameters. +-- + +`boost`:: ++ +-- +Floating point number used to decrease or increase the +<> of a query. Default is `1.0`. +Optional. + +You can use the `boost` parameter to adjust relevance scores for searches +containing two or more queries. + +Boost values are relative to the default value of `1.0`. A boost value between +`0` and `1.0` decreases the relevance score. A value greater than `1.0` +increases the relevance score. +-- + +[[terms-query-notes]] +==== Notes + +[[query-dsl-terms-query-highlighting]] +===== Highlighting `terms` queries +<> is best-effort only. {es} may not +return highlight results for `terms` queries depending on: + +* Highlighter type +* Number of terms in the query -[float] [[query-dsl-terms-lookup]] -===== Terms lookup mechanism +===== Terms lookup +Terms lookup fetches the field values of an existing document. {es} then uses +those values as search terms. This can be helpful when searching for a large set +of terms. -When it's needed to specify a `terms` filter with a lot of terms it can -be beneficial to fetch those term values from a document in an index. A -concrete example would be to filter tweets tweeted by your followers. -Potentially the amount of user ids specified in the terms filter can be -a lot. In this scenario it makes sense to use the terms filter's terms -lookup mechanism. +Because terms lookup fetches values from a document, the <> mapping field must be enabled to use terms lookup. The `_source` +field is enabled by default. -The terms lookup mechanism supports the following options: +[NOTE] +By default, {es} limits the `terms` query to a maximum of 65,536 +terms. This includes terms fetched using terms lookup. You can change +this limit using the <> setting. -[horizontal] +To perform a terms lookup, use the following parameters. + +[[query-dsl-terms-lookup-params]] +====== Terms lookup parameters `index`:: - The index to fetch the term values from. +Name of the index from which to fetch field values. `id`:: - The id of the document to fetch the term values from. +<> of the document from which to fetch field values. `path`:: - The field specified as path to fetch the actual values for the - `terms` filter. ++ +-- +Name of the field from which to fetch field values. {es} uses +these values as search terms for the query. + +If the field values include an array of nested inner objects, you can access +those objects using dot notation syntax. +-- `routing`:: - A custom routing value to be used when retrieving the - external terms doc. - -The values for the `terms` filter will be fetched from a field in a -document with the specified id in the specified type and index. -Internally a get request is executed to fetch the values from the -specified path. At the moment for this feature to work the `_source` -needs to be stored. - -Also, consider using an index with a single shard and fully replicated -across all nodes if the "reference" terms data is not large. The lookup -terms filter will prefer to execute the get request on a local node if -possible, reducing the need for networking. - -[WARNING] -Executing a Terms Query request with a lot of terms can be quite slow, -as each additional term demands extra processing and memory. -To safeguard against this, the maximum number of terms that can be used -in a Terms Query both directly or through lookup has been limited to `65536`. -This default maximum can be changed for a particular index with the index setting - `index.max_terms_count`. - -[float] -===== Terms lookup twitter example -At first we index the information for user with id 2, specifically, its -followers, then index a tweet from user with id 1. Finally we search on -all the tweets that match the followers of user 2. +Custom <> of the document from which to +fetch term values. If a custom routing value was provided when the document was +indexed, this parameter is required. + +[[query-dsl-terms-lookup-example]] +====== Terms lookup example + +To see how terms lookup works, try the following example. + +. Create an index with a `keyword` field named `color`. ++ +-- [source,js] --------------------------------------------------- -PUT /users/_doc/2 +---- +PUT my_index { - "followers" : ["1", "3"] + "mappings" : { + "properties" : { + "color" : { "type" : "keyword" } + } + } } +---- +// CONSOLE +-- -PUT /tweets/_doc/1 +. Index a document with an ID of 1 and values of `["blue", "green"]` in the +`color` field. ++ +-- + +[source,js] +---- +PUT my_index/_doc/1 { - "user" : "1" + "color": ["blue", "green"] } +---- +// CONSOLE +// TEST[continued] +-- -GET /tweets/_search +. Index another document with an ID of 2 and value of `blue` in the `color` +field. ++ +-- + +[source,js] +---- +PUT my_index/_doc/2 { - "query" : { - "terms" : { - "user" : { - "index" : "users", - "id" : "2", - "path" : "followers" - } - } - } + "color": "blue" } --------------------------------------------------- +---- +// CONSOLE +// TEST[continued] +-- + +. Use the `terms` query with terms lookup parameters to find documents +containing one or more of the same terms as document 2. Include the `pretty` +parameter so the response is more readable. ++ +-- + +//// + +[source,js] +---- +POST my_index/_refresh +---- // CONSOLE +// TEST[continued] -The structure of the external terms document can also include an array of -inner objects, for example: +//// [source,js] --------------------------------------------------- -PUT /users/_doc/2 +---- +GET my_index/_search?pretty { - "followers" : [ - { - "id" : "1" - }, - { - "id" : "2" - } - ] + "query": { + "terms": { + "color" : { + "index" : "my_index", + "id" : "2", + "path" : "color" + } + } + } } --------------------------------------------------- +---- // CONSOLE +// TEST[continued] + +Because document 2 and document 1 both contain `blue` as a value in the `color` +field, {es} returns both documents. -In which case, the lookup path will be `followers.id`. +[source,js] +---- +{ + "took" : 17, + "timed_out" : false, + "_shards" : { + "total" : 1, + "successful" : 1, + "skipped" : 0, + "failed" : 0 + }, + "hits" : { + "total" : { + "value" : 2, + "relation" : "eq" + }, + "max_score" : 1.0, + "hits" : [ + { + "_index" : "my_index", + "_type" : "_doc", + "_id" : "1", + "_score" : 1.0, + "_source" : { + "color" : [ + "blue", + "green" + ] + } + }, + { + "_index" : "my_index", + "_type" : "_doc", + "_id" : "2", + "_score" : 1.0, + "_source" : { + "color" : "blue" + } + } + ] + } +} +---- +// TESTRESPONSE[s/"took" : 17/"took" : $body.took/] +-- \ No newline at end of file diff --git a/docs/reference/query-dsl/terms-set-query.asciidoc b/docs/reference/query-dsl/terms-set-query.asciidoc index 3ebfb672e205f..0f097e494bfda 100644 --- a/docs/reference/query-dsl/terms-set-query.asciidoc +++ b/docs/reference/query-dsl/terms-set-query.asciidoc @@ -1,121 +1,231 @@ [[query-dsl-terms-set-query]] === Terms Set Query -Returns any documents that match with at least one or more of the -provided terms. The terms are not analyzed and thus must match exactly. -The number of terms that must match varies per document and is either -controlled by a minimum should match field or computed per document in -a minimum should match script. +Returns documents that contain a minimum number of *exact* terms in a provided +field. -The field that controls the number of required terms that must match must -be a number field: +The `terms_set` query is the same as the <>, except you can define the number of matching terms required to +return a document. For example: + +* A field, `programming_languages`, contains a list of known programming +languages, such as `c++`, `java`, or `php` for job candidates. You can use the +`terms_set` query to return documents that match at least two of these +languages. + +* A field, `permissions`, contains a list of possible user permissions for an +application. You can use the `terms_set` query to return documents that +match a subset of these permissions. + +[[terms-set-query-ex-request]] +==== Example request + +[[terms-set-query-ex-request-index-setup]] +===== Index setup +In most cases, you'll need to include a <> field mapping in +your index to use the `terms_set` query. This numeric field contains the +number of matching terms required to return a document. + +To see how you can set up an index for the `terms_set` query, try the +following example. + +. Create an index, `job-candidates`, with the following field mappings: ++ +-- + +* `name`, a <> field. This field contains the name of the +job candidate. + +* `programming_languages`, a <> field. This field contains +programming languages known by the job candidate. + +* `required_matches`, a <> `long` field. This field contains +the number of matching terms required to return a document. [source,js] --------------------------------------------------- -PUT /my-index +---- +PUT /job-candidates { "mappings": { "properties": { + "name": { + "type": "keyword" + }, + "programming_languages": { + "type": "keyword" + }, "required_matches": { "type": "long" } } } } +---- +// CONSOLE +// TESTSETUP + +-- -PUT /my-index/_doc/1?refresh +. Index a document with an ID of `1` and the following values: ++ +-- + +* `Jane Smith` in the `name` field. + +* `["c++", "java"]` in the `programming_languages` field. + +* `2` in the `required_matches` field. + +Include the `?refresh` parameter so the document is immediately available for +search. + +[source,js] +---- +PUT /job-candidates/_doc/1?refresh { - "codes": ["ghi", "jkl"], + "name": "Jane Smith", + "programming_languages": ["c++", "java"], "required_matches": 2 } +---- +// CONSOLE + +-- + +. Index another document with an ID of `2` and the following values: ++ +-- + +* `Jason Response` in the `name` field. + +* `["java", "php"]` in the `programming_languages` field. + +* `2` in the `required_matches` field. -PUT /my-index/_doc/2?refresh +[source,js] +---- +PUT /job-candidates/_doc/2?refresh { - "codes": ["def", "ghi"], + "name": "Jason Response", + "programming_languages": ["java", "php"], "required_matches": 2 } --------------------------------------------------- +---- // CONSOLE -// TESTSETUP -An example that uses the minimum should match field: +-- + +You can now use the `required_matches` field value as the number of +matching terms required to return a document in the `terms_set` query. + +[[terms-set-query-ex-request-query]] +===== Example query + +The following search returns documents where the `programming_languages` field +contains at least two of the following terms: + +* `c++` +* `java` +* `php` + +The `minimum_should_match_field` is `required_matches`. This means the +number of matching terms required is `2`, the value of the `required_matches` +field. [source,js] --------------------------------------------------- -GET /my-index/_search +---- +GET /job-candidates/_search { "query": { "terms_set": { - "codes" : { - "terms" : ["abc", "def", "ghi"], + "programming_languages": { + "terms": ["c++", "java", "php"], "minimum_should_match_field": "required_matches" } } } } --------------------------------------------------- +---- // CONSOLE -Response: +[[terms-set-top-level-params]] +==== Top-level parameters for `terms_set` -[source,js] --------------------------------------------------- -{ - "took": 13, - "timed_out": false, - "_shards": { - "total": 1, - "successful": 1, - "skipped" : 0, - "failed": 0 - }, - "hits": { - "total" : { - "value": 1, - "relation": "eq" - }, - "max_score": 0.87546873, - "hits": [ - { - "_index": "my-index", - "_type": "_doc", - "_id": "2", - "_score": 0.87546873, - "_source": { - "codes": ["def", "ghi"], - "required_matches": 2 - } - } - ] - } -} --------------------------------------------------- -// TESTRESPONSE[s/"took": 13,/"took": "$body.took",/] +``:: +Field you wish to search. + +[[terms-set-field-params]] +==== Parameters for `` -Scripts can also be used to control how many terms are required to match -in a more dynamic way. For example a create date or a popularity field -can be used as basis for the number of required terms to match. +`terms`:: ++ +-- +Array of terms you wish to find in the provided ``. To return a document, +a required number of terms must exactly match the field values, including +whitespace and capitalization. -Also the `params.num_terms` parameter is available in the script to indicate the -number of terms that have been specified. +The required number of matching terms is defined in the +`minimum_should_match_field` or `minimum_should_match_script` parameter. +-- -An example that always limits the number of required terms to match to never -become larger than the number of terms specified: +`minimum_should_match_field`:: +<> field containing the number of matching terms +required to return a document. + +`minimum_should_match_script`:: ++ +-- +Custom script containing the number of matching terms required to return a +document. + +For parameters and valid values, see <>. + +For an example query using the `minimum_should_match_script` parameter, see +<>. +-- + +[[terms-set-query-notes]] +==== Notes + +[[terms-set-query-script]] +===== How to use the `minimum_should_match_script` parameter +You can use `minimum_should_match_script` to define the required number of +matching terms using a script. This is useful if you need to set the number of +required terms dynamically. + +[[terms-set-query-script-ex]] +====== Example query using `minimum_should_match_script` + +The following search returns documents where the `programming_languages` field +contains at least two of the following terms: + +* `c++` +* `java` +* `php` + +The `source` parameter of this query indicates: + +* The required number of terms to match cannot exceed `params.num_terms`, the +number of terms provided in the `terms` field. +* The required number of terms to match is `2`, the value of the +`required_matches` field. [source,js] --------------------------------------------------- -GET /my-index/_search +---- +GET /job-candidates/_search { "query": { "terms_set": { - "codes" : { - "terms" : ["abc", "def", "ghi"], + "programming_languages": { + "terms": ["c++", "java", "php"], "minimum_should_match_script": { "source": "Math.min(params.num_terms, doc['required_matches'].value)" - } + }, + "boost": 1.0 } } } } --------------------------------------------------- -// CONSOLE +---- +// CONSOLE \ No newline at end of file diff --git a/docs/reference/query-dsl/wildcard-query.asciidoc b/docs/reference/query-dsl/wildcard-query.asciidoc index ba1c72bb1e53b..b2e8eb0adf772 100644 --- a/docs/reference/query-dsl/wildcard-query.asciidoc +++ b/docs/reference/query-dsl/wildcard-query.asciidoc @@ -1,51 +1,67 @@ [[query-dsl-wildcard-query]] === Wildcard Query +Returns documents that contain terms matching a wildcard pattern. -Matches documents that have fields matching a wildcard expression (*not -analyzed*). Supported wildcards are `*`, which matches any character -sequence (including the empty one), and `?`, which matches any single -character. Note that this query can be slow, as it needs to iterate over many -terms. In order to prevent extremely slow wildcard queries, a wildcard -term should not start with one of the wildcards `*` or `?`. The wildcard -query maps to Lucene `WildcardQuery`. +A wildcard operator is a placeholder that matches one or more characters. For +example, the `*` wildcard operator matches zero or more characters. You can +combine wildcard operators with other characters to create a wildcard pattern. -[source,js] --------------------------------------------------- -GET /_search -{ - "query": { - "wildcard" : { "user" : "ki*y" } - } -} --------------------------------------------------- -// CONSOLE +[[wildcard-query-ex-request]] +==== Example request -A boost can also be associated with the query: +The following search returns documents where the `user` field contains a term +that begins with `ki` and ends with `y`. These matching terms can include `kiy`, +`kity`, or `kimchy`. [source,js] --------------------------------------------------- +---- GET /_search { "query": { - "wildcard" : { "user" : { "value" : "ki*y", "boost" : 2.0 } } + "wildcard": { + "user": { + "value": "ki*y", + "boost": 1.0, + "rewrite": "constant_score" + } + } } } --------------------------------------------------- +---- // CONSOLE -Or : +[[wildcard-top-level-params]] +==== Top-level parameters for `wildcard` +``:: +Field you wish to search. -[source,js] --------------------------------------------------- -GET /_search -{ - "query": { - "wildcard" : { "user" : { "wildcard" : "ki*y", "boost" : 2.0 } } - } -} --------------------------------------------------- -// CONSOLE +[[wildcard-query-field-params]] +==== Parameters for `` +`value`:: +Wildcard pattern for terms you wish to find in the provided ``. ++ +-- +This parameter supports two wildcard operators: + +* `?`, which matches any single character +* `*`, which can match zero or more characters, including an empty one + +WARNING: Avoid beginning patterns with `*` or `?`. This can increase +the iterations needed to find matching terms and slow search performance. +-- + +`boost`:: +Floating point number used to decrease or increase the +<> of a query. Default is `1.0`. +Optional. ++ +You can use the `boost` parameter to adjust relevance scores for searches +containing two or more queries. ++ +Boost values are relative to the default value of `1.0`. A boost value between +`0` and `1.0` decreases the relevance score. A value greater than `1.0` +increases the relevance score. -This multi term query allows to control how it gets rewritten using the -<> -parameter. +`rewrite` (Expert):: +Method used to rewrite the query. For valid values and more information, see the +<>. Optional. \ No newline at end of file diff --git a/docs/reference/redirects.asciidoc b/docs/reference/redirects.asciidoc index 6f68d781f4856..4d15ee7e25503 100644 --- a/docs/reference/redirects.asciidoc +++ b/docs/reference/redirects.asciidoc @@ -82,15 +82,13 @@ The `_uid` field has been removed in favour of the <> or the -{javaclient}/index.html[Java API]. +interface over <>. [role="exclude",id="modules-thrift"] === Thrift The `thrift` transport is no longer supported. Instead use the REST -interface over <> or the -{javaclient}/index.html[Java API]. +interface over <>. // QUERY DSL @@ -610,4 +608,16 @@ The `TransportClient` is deprecated in favour of the {java-rest}/java-rest-high.html[Java High Level REST Client] and was removed in Elasticsearch 8.0. The {java-rest}/java-rest-high-level-migration.html[migration guide] describes all -the steps needed to migrate. \ No newline at end of file +the steps needed to migrate. + +[role="exclude",id="query-dsl-common-terms-query"] +=== Common Terms Query + +The `common` terms query is deprecated. Use the <> instead. The `match` query skips blocks of documents efficiently, +without any configuration, if the total number of hits is not tracked. + +[role="exclude",id="xpack-api"] +=== X-Pack APIs + +{es} {xpack} APIs are now documented in <>. \ No newline at end of file diff --git a/docs/reference/rest-api/defs.asciidoc b/docs/reference/rest-api/defs.asciidoc index 823b63cbe579d..65a0384a3d3bf 100644 --- a/docs/reference/rest-api/defs.asciidoc +++ b/docs/reference/rest-api/defs.asciidoc @@ -12,6 +12,7 @@ These resource definitions are used in APIs related to {ml-features} and * <> * <> * <> +* <> * <> * <> * <> @@ -22,6 +23,7 @@ include::{es-repo-dir}/ml/apis/filterresource.asciidoc[] include::{es-repo-dir}/ml/apis/jobresource.asciidoc[] include::{es-repo-dir}/ml/apis/jobcounts.asciidoc[] include::{es-repo-dir}/ml/apis/snapshotresource.asciidoc[] +include::{es-repo-dir}/data-frames/apis/pivotresource.asciidoc[] include::{xes-repo-dir}/rest-api/security/role-mapping-resources.asciidoc[] include::{es-repo-dir}/ml/apis/resultsresource.asciidoc[] include::{es-repo-dir}/ml/apis/eventresource.asciidoc[] diff --git a/docs/reference/rest-api/index.asciidoc b/docs/reference/rest-api/index.asciidoc index a5b93747dfa1a..9f70c2a3cef65 100644 --- a/docs/reference/rest-api/index.asciidoc +++ b/docs/reference/rest-api/index.asciidoc @@ -1,17 +1,21 @@ -[role="xpack"] -[[xpack-api]] -= {xpack} APIs +[[rest-apis]] += REST APIs [partintro] -- -{xpack} exposes REST APIs that are used by the UI components and can be called -directly to configure and access {xpack} features. +{es} exposes REST APIs that are used by the UI components and can be called +directly to configure and access {es} features. + +[NOTE] +We are working on including more {es} APIs in this section. Some content might +not be included yet. * <> * <> * <> * <> * <>, <> +* <> * <> * <> * <> @@ -35,4 +39,5 @@ include::{es-repo-dir}/rollup/rollup-api.asciidoc[] include::{xes-repo-dir}/rest-api/security.asciidoc[] include::{es-repo-dir}/indices/apis/unfreeze.asciidoc[] include::{xes-repo-dir}/rest-api/watcher.asciidoc[] +include::{es-repo-dir}/indices/apis/reload-analyzers.asciidoc[] include::defs.asciidoc[] diff --git a/docs/reference/rest-api/info.asciidoc b/docs/reference/rest-api/info.asciidoc index e626b491da17d..adbf5f01979a0 100644 --- a/docs/reference/rest-api/info.asciidoc +++ b/docs/reference/rest-api/info.asciidoc @@ -64,61 +64,58 @@ Example response: }, "features" : { "ccr" : { - "description" : "Cross Cluster Replication", "available" : true, "enabled" : true }, "data_frame" : { - "description" : "Data Frame for the Elastic Stack", "available" : true, "enabled" : true }, + "flattened" : { + "available" : true, + "enabled" : true + }, "graph" : { - "description" : "Graph Data Exploration for the Elastic Stack", "available" : true, "enabled" : true }, "ilm" : { - "description" : "Index lifecycle management for the Elastic Stack", "available" : true, "enabled" : true }, "logstash" : { - "description" : "Logstash management component for X-Pack", "available" : true, "enabled" : true }, "ml" : { - "description" : "Machine Learning for the Elastic Stack", "available" : true, - "enabled" : true, - "native_code_info" : { - "version" : "7.0.0-alpha1-SNAPSHOT", - "build_hash" : "99a07c016d5a73" - } + "enabled" : true }, "monitoring" : { - "description" : "Monitoring for the Elastic Stack", "available" : true, "enabled" : true }, "rollup": { - "description": "Time series pre-aggregation and rollup", "available": true, "enabled": true }, "security" : { - "description" : "Security for the Elastic Stack", "available" : true, "enabled" : false }, "sql" : { - "description" : "SQL access to Elasticsearch", + "available" : true, + "enabled" : true + }, + "vectors" : { + "available" : true, + "enabled" : true + }, + "voting_only" : { "available" : true, "enabled" : true }, "watcher" : { - "description" : "Alerting, Notification and Automation for the Elastic Stack", "available" : true, "enabled" : true } diff --git a/docs/reference/rollup/api-quickref.asciidoc b/docs/reference/rollup/api-quickref.asciidoc index 21eefefb4b12b..d1ea03b6284d7 100644 --- a/docs/reference/rollup/api-quickref.asciidoc +++ b/docs/reference/rollup/api-quickref.asciidoc @@ -5,7 +5,7 @@ experimental[] -Most {rollup} endpoints have the following base: +Most rollup endpoints have the following base: [source,js] ---- diff --git a/docs/reference/rollup/rollup-api.asciidoc b/docs/reference/rollup/rollup-api.asciidoc index 099686fb4329d..5981336d0a054 100644 --- a/docs/reference/rollup/rollup-api.asciidoc +++ b/docs/reference/rollup/rollup-api.asciidoc @@ -9,7 +9,7 @@ * <>, <>, * <>, <>, -* <> +* <> * <> [float] diff --git a/docs/reference/modules/scripting.asciidoc b/docs/reference/scripting.asciidoc similarity index 86% rename from docs/reference/modules/scripting.asciidoc rename to docs/reference/scripting.asciidoc index 44696ea94bb89..33b8795a58114 100644 --- a/docs/reference/modules/scripting.asciidoc +++ b/docs/reference/scripting.asciidoc @@ -1,9 +1,11 @@ [[modules-scripting]] -== Scripting += Scripting -The scripting module enables you to use scripts to evaluate custom -expressions. For example, you could use a script to return "script fields" -as part of a search request or evaluate a custom score for a query. +[partintro] +-- +With scripting, you can evaluate custom expressions in {es}. For example, you +could use a script to return "script fields" as part of a search request or +evaluate a custom score for a query. The default scripting language is <>. Additional `lang` plugins enable you to run scripts written in other languages. @@ -11,7 +13,7 @@ Everywhere a script can be used, you can include a `lang` parameter to specify the language of the script. [float] -=== General-purpose languages: +== General-purpose languages These languages can be used for any purpose in the scripting APIs, and give the most flexibility. @@ -29,7 +31,7 @@ and give the most flexibility. |======================================================================= [float] -=== Special-purpose languages: +== Special-purpose languages These languages are less flexible, but typically have higher performance for certain tasks. @@ -67,7 +69,7 @@ sandboxed languages can be a security issue, please read <> for more details. ================================================= - +-- include::scripting/using.asciidoc[] diff --git a/docs/reference/modules/scripting/engine.asciidoc b/docs/reference/scripting/engine.asciidoc similarity index 97% rename from docs/reference/modules/scripting/engine.asciidoc rename to docs/reference/scripting/engine.asciidoc index da3b4529daacc..b4a2cd29fdd72 100644 --- a/docs/reference/modules/scripting/engine.asciidoc +++ b/docs/reference/scripting/engine.asciidoc @@ -1,5 +1,5 @@ [[modules-scripting-engine]] -=== Advanced scripts using script engines +== Advanced scripts using script engines A `ScriptEngine` is a backend for implementing a scripting language. It may also be used to write scripts that need to use advanced internals of scripting. For example, diff --git a/docs/reference/modules/scripting/expression.asciidoc b/docs/reference/scripting/expression.asciidoc similarity index 99% rename from docs/reference/modules/scripting/expression.asciidoc rename to docs/reference/scripting/expression.asciidoc index de3704a899e52..fe58cbbdf131b 100644 --- a/docs/reference/modules/scripting/expression.asciidoc +++ b/docs/reference/scripting/expression.asciidoc @@ -1,5 +1,5 @@ [[modules-scripting-expression]] -=== Lucene Expressions Language +== Lucene expressions language Lucene's expressions compile a `javascript` expression to bytecode. They are designed for high-performance custom ranking and sorting functions and are diff --git a/docs/reference/modules/scripting/fields.asciidoc b/docs/reference/scripting/fields.asciidoc similarity index 98% rename from docs/reference/modules/scripting/fields.asciidoc rename to docs/reference/scripting/fields.asciidoc index 681e8d6e54067..cf8905189f799 100644 --- a/docs/reference/modules/scripting/fields.asciidoc +++ b/docs/reference/scripting/fields.asciidoc @@ -1,5 +1,5 @@ [[modules-scripting-fields]] -=== Accessing document fields and special variables +== Accessing document fields and special variables Depending on where a script is used, it will have access to certain special variables and document fields. @@ -17,7 +17,7 @@ API will have access to the `ctx` variable which exposes: `ctx._index` etc:: Access to <>, some of which may be read-only. [float] -== Search and Aggregation scripts +== Search and aggregation scripts With the exception of <> which are executed once per search hit, scripts used in search and aggregations will be @@ -80,7 +80,7 @@ GET my_index/_search [float] [[modules-scripting-doc-vals]] -=== Doc Values +=== Doc values By far the fastest most efficient way to access a field value from a script is to use the `doc['field_name']` syntax, which retrieves the field @@ -140,7 +140,7 @@ access `text` fields from scripts. [float] [[modules-scripting-stored]] -=== Stored Fields and `_source` +=== Stored fields and `_source` _Stored fields_ -- fields explicitly marked as <> -- can be accessed using the diff --git a/docs/reference/modules/scripting/painless.asciidoc b/docs/reference/scripting/painless.asciidoc similarity index 97% rename from docs/reference/modules/scripting/painless.asciidoc rename to docs/reference/scripting/painless.asciidoc index 6dd9b50db51ed..1d615c1426e90 100644 --- a/docs/reference/modules/scripting/painless.asciidoc +++ b/docs/reference/scripting/painless.asciidoc @@ -1,5 +1,5 @@ [[modules-scripting-painless]] -=== Painless Scripting Language +== Painless scripting language _Painless_ is a simple, secure scripting language designed specifically for use with Elasticsearch. It is the default scripting language for Elasticsearch and diff --git a/docs/reference/modules/scripting/security.asciidoc b/docs/reference/scripting/security.asciidoc similarity index 99% rename from docs/reference/modules/scripting/security.asciidoc rename to docs/reference/scripting/security.asciidoc index f1946bd0f2876..421cec2ccf760 100644 --- a/docs/reference/modules/scripting/security.asciidoc +++ b/docs/reference/scripting/security.asciidoc @@ -1,5 +1,5 @@ [[modules-scripting-security]] -=== Scripting and security +== Scripting and security While Elasticsearch contributors make every effort to prevent scripts from running amok, security is something best done in diff --git a/docs/reference/modules/scripting/using.asciidoc b/docs/reference/scripting/using.asciidoc similarity index 97% rename from docs/reference/modules/scripting/using.asciidoc rename to docs/reference/scripting/using.asciidoc index 86202a98dd537..6da631ac9f592 100644 --- a/docs/reference/modules/scripting/using.asciidoc +++ b/docs/reference/scripting/using.asciidoc @@ -1,5 +1,5 @@ [[modules-scripting-using]] -=== How to use scripts +== How to use scripts Wherever scripting is supported in the Elasticsearch API, the syntax follows the same pattern: @@ -45,7 +45,7 @@ GET my_index/_search // CONSOLE [float] -=== Script Parameters +=== Script parameters `lang`:: @@ -107,7 +107,7 @@ minute will be compiled. You can change this setting dynamically by setting [float] [[modules-scripting-short-script-form]] -=== Short Script Form +=== Short script form A short script form can be used for brevity. In the short form, `script` is represented by a string instead of an object. This string contains the source of the script. @@ -131,12 +131,13 @@ The same script in the normal form: [float] [[modules-scripting-stored-scripts]] -=== Stored Scripts +=== Stored scripts Scripts may be stored in and retrieved from the cluster state using the `_scripts` end-point. -==== Request Examples +[float] +==== Request examples The following are examples of using a stored script that lives at `/_scripts/{id}`. @@ -196,7 +197,7 @@ DELETE _scripts/calculate-score [float] [[modules-scripting-using-caching]] -=== Script Caching +=== Script caching All scripts are cached by default so that they only need to be recompiled when updates occur. By default, scripts do not have a time-based expiration, but diff --git a/docs/reference/search/request/docvalue-fields.asciidoc b/docs/reference/search/request/docvalue-fields.asciidoc index 6697b5bb3e383..784cc94015366 100644 --- a/docs/reference/search/request/docvalue-fields.asciidoc +++ b/docs/reference/search/request/docvalue-fields.asciidoc @@ -67,3 +67,7 @@ on their mappings: `long`, `double` and other numeric fields are formatted as numbers, `keyword` fields are formatted as strings, `date` fields are formatted with the configured `date` format, etc. +NOTE: On its own, `docvalue_fields` cannot be used to load fields in nested +objects -- if a field contains a nested object in its path, then no data will +be returned for that docvalue field. To access nested fields, `docvalue_fields` +must be used within an <> block. \ No newline at end of file diff --git a/docs/reference/search/request/preference.asciidoc b/docs/reference/search/request/preference.asciidoc index 5f3fcb2efa6b7..7412f04844c08 100644 --- a/docs/reference/search/request/preference.asciidoc +++ b/docs/reference/search/request/preference.asciidoc @@ -6,8 +6,12 @@ default, Elasticsearch selects from the available shard copies in an unspecified order, taking the <> and <> configuration into account. However, it may sometimes be desirable to try and route certain -searches to certain sets of shard copies, for instance to make better use of -per-copy caches. +searches to certain sets of shard copies. + +A possible use case would be to make use of per-copy caches like the +<>. Doing this, however, runs contrary to the +idea of search parallelization and can create hotspots on certain nodes because +the load might not be evenly distributed anymore. The `preference` is a query string parameter which can be set to: @@ -64,6 +68,10 @@ GET /_search?preference=xyzabc123 ------------------------------------------------ // CONSOLE +This can be an effective strategy to increase usage of e.g. the request cache for +unique users running similar searches repeatedly by always hitting the same cache, while +requests of different users are still spread across all shard copies. + NOTE: The `_only_local` preference guarantees only to use shard copies on the local node, which is sometimes useful for troubleshooting. All other options do not _fully_ guarantee that any particular shard copies are used in a search, diff --git a/docs/reference/search/request/script-fields.asciidoc b/docs/reference/search/request/script-fields.asciidoc index da5868ea7d65e..1bd61e0048182 100644 --- a/docs/reference/search/request/script-fields.asciidoc +++ b/docs/reference/search/request/script-fields.asciidoc @@ -33,7 +33,7 @@ GET /_search // CONSOLE // TEST[setup:sales] -Script fields can work on fields that are not stored (`my_field_name` in +Script fields can work on fields that are not stored (`price` in the above case), and allow to return custom values to be returned (the evaluated value of the script). diff --git a/docs/reference/search/request/sort.asciidoc b/docs/reference/search/request/sort.asciidoc index c12ec3a679a72..ccbc3da6e063b 100644 --- a/docs/reference/search/request/sort.asciidoc +++ b/docs/reference/search/request/sort.asciidoc @@ -252,7 +252,7 @@ field support has a `nested` sort option with the following properties: A filter that the inner objects inside the nested path should match with in order for its field values to be taken into account by sorting. Common case is to repeat the query / filter inside the - nested filter or query. By default no `nested_filter` is active. + nested filter or query. By default no `filter` is active. `max_children`:: The maximum number of children to consider per root document when picking the sort value. Defaults to unlimited. @@ -260,14 +260,8 @@ field support has a `nested` sort option with the following properties: Same as top-level `nested` but applies to another nested path within the current nested object. -[WARNING] -.Nested sort options before Elasticsearch 6.1 -============================================ - -The `nested_path` and `nested_filter` options have been deprecated in -favor of the options documented above. - -============================================ +NOTE: Elasticsearch will throw an error if a nested field is defined in a sort without +a `nested` context. ===== Nested sorting examples @@ -300,7 +294,7 @@ POST /_search // CONSOLE In the below example `parent` and `child` fields are of type `nested`. -The `nested_path` needs to be specified at each level; otherwise, Elasticsearch doesn't know on what nested level sort values need to be captured. +The `nested.path` needs to be specified at each level; otherwise, Elasticsearch doesn't know on what nested level sort values need to be captured. [source,js] -------------------------------------------------- @@ -374,7 +368,7 @@ GET /_search // CONSOLE NOTE: If a nested inner object doesn't match with -the `nested_filter` then a missing value is used. +the `nested.filter` then a missing value is used. ==== Ignoring Unmapped Fields diff --git a/docs/reference/search/request/stored-fields.asciidoc b/docs/reference/search/request/stored-fields.asciidoc index 195dc39f11e73..b55e0fce45757 100644 --- a/docs/reference/search/request/stored-fields.asciidoc +++ b/docs/reference/search/request/stored-fields.asciidoc @@ -49,6 +49,11 @@ Script fields can also be automatically detected and used as fields, so things like `_source.obj1.field1` can be used, though not recommended, as `obj1.field1` will work as well. +NOTE: On its own, `stored_fields` cannot be used to load fields in nested +objects -- if a field contains a nested object in its path, then no data will +be returned for that stored field. To access nested fields, `stored_fields` +must be used within an <> block. + ==== Disable stored fields entirely To disable the stored fields (and metadata fields) entirely use: `_none_`: diff --git a/docs/reference/search/suggesters/phrase-suggest.asciidoc b/docs/reference/search/suggesters/phrase-suggest.asciidoc index d92c32eddf033..2d0e3e218e62c 100644 --- a/docs/reference/search/suggesters/phrase-suggest.asciidoc +++ b/docs/reference/search/suggesters/phrase-suggest.asciidoc @@ -224,7 +224,7 @@ The response contains suggestions scored by the most likely spell correction fir [source,js] -------------------------------------------------- -POST _search +POST test/_search { "suggest": { "text" : "noble prize", @@ -267,7 +267,10 @@ POST _search The `phrase` suggester supports multiple smoothing models to balance weight between infrequent grams (grams (shingles) are not existing in -the index) and frequent grams (appear at least once in the index). +the index) and frequent grams (appear at least once in the index). The +smoothing model can be selected by setting the `smoothing` parameter +to one of the following options. Each smoothing model supports specific +properties that can be configured. [horizontal] `stupid_backoff`:: @@ -288,6 +291,28 @@ the index) and frequent grams (appear at least once in the index). All parameters (`trigram_lambda`, `bigram_lambda`, `unigram_lambda`) must be supplied. +[source,js] +-------------------------------------------------- +POST test/_search +{ + "suggest": { + "text" : "obel prize", + "simple_phrase" : { + "phrase" : { + "field" : "title.trigram", + "size" : 1, + "smoothing" : { + "laplace" : { + "alpha" : 0.7 + } + } + } + } + } +} +-------------------------------------------------- +// CONSOLE + ==== Candidate Generators The `phrase` suggester uses candidate generators to produce a list of @@ -389,7 +414,7 @@ accept ordinary analyzer names. [source,js] -------------------------------------------------- -POST _search +POST test/_search { "suggest": { "text" : "obel prize", diff --git a/docs/reference/security/securing-communications/configuring-tls-docker.asciidoc b/docs/reference/security/securing-communications/configuring-tls-docker.asciidoc index 1d23430e37eec..2190f17e4588f 100644 --- a/docs/reference/security/securing-communications/configuring-tls-docker.asciidoc +++ b/docs/reference/security/securing-communications/configuring-tls-docker.asciidoc @@ -43,11 +43,13 @@ instances: `.env`: [source,yaml] ---- -CERTS_DIR=/usr/share/elasticsearch/config/certificates <1> -ELASTIC_PASSWORD=PleaseChangeMe <2> +COMPOSE_PROJECT_NAME=es <1> +CERTS_DIR=/usr/share/elasticsearch/config/certificates <2> +ELASTIC_PASSWORD=PleaseChangeMe <3> ---- -<1> The path, inside the Docker image, where certificates are expected to be found. -<2> Initial password for the `elastic` user. +<1> Use an `es_` prefix for all volumes and networks created by docker-compose. +<2> The path, inside the Docker image, where certificates are expected to be found. +<3> Initial password for the `elastic` user. [[getting-starter-tls-create-certs-composefile]] `create-certs.yml`: @@ -69,21 +71,21 @@ services: image: {docker-image} command: > bash -c ' - if [[ ! -d config/certificates/certs ]]; then - mkdir config/certificates/certs; + yum install -y -q -e 0 unzip; + if [[ ! -f /certs/bundle.zip ]]; then + bin/elasticsearch-certutil cert --silent --pem --in config/certificates/instances.yml -out /certs/bundle.zip; + unzip /certs/bundle.zip -d /certs; <1> fi; - if [[ ! -f /local/certs/bundle.zip ]]; then - bin/elasticsearch-certgen --silent --in config/certificates/instances.yml --out config/certificates/certs/bundle.zip; - unzip config/certificates/certs/bundle.zip -d config/certificates/certs; <1> - fi; - chgrp -R 0 config/certificates/certs + chown -R 1000:0 /certs ' - user: $\{UID:-1000\} + user: "0" working_dir: /usr/share/elasticsearch - volumes: ['.:/usr/share/elasticsearch/config/certificates'] + volumes: ['certs:/certs', '.:/usr/share/elasticsearch/config/certificates'] + +volumes: {"certs"} ---- -<1> The new node certificates and CA certificate+key are placed under the local directory `certs`. +<1> The new node certificates and CA certificate+key are placed in a docker volume `es_certs`. endif::[] [[getting-starter-tls-create-docker-compose]] @@ -106,7 +108,7 @@ services: image: {docker-image} environment: - node.name=es01 - - discovery.seed_hosts=es02 + - discovery.seed_hosts=es01,es02 - cluster.initial_master_nodes=es01,es02 - ELASTIC_PASSWORD=$ELASTIC_PASSWORD <1> - "ES_JAVA_OPTS=-Xms512m -Xmx512m" @@ -121,7 +123,7 @@ services: - xpack.security.transport.ssl.certificate_authorities=$CERTS_DIR/ca/ca.crt - xpack.security.transport.ssl.certificate=$CERTS_DIR/es01/es01.crt - xpack.security.transport.ssl.key=$CERTS_DIR/es01/es01.key - volumes: ['esdata_01:/usr/share/elasticsearch/data', './certs:$CERTS_DIR'] + volumes: ['data01:/usr/share/elasticsearch/data', 'certs:$CERTS_DIR'] ports: - 9200:9200 healthcheck: @@ -135,7 +137,7 @@ services: image: {docker-image} environment: - node.name=es02 - - discovery.seed_hosts=es01 + - discovery.seed_hosts=es01,es02 - cluster.initial_master_nodes=es01,es02 - ELASTIC_PASSWORD=$ELASTIC_PASSWORD - "ES_JAVA_OPTS=-Xms512m -Xmx512m" @@ -150,14 +152,14 @@ services: - xpack.security.transport.ssl.certificate_authorities=$CERTS_DIR/ca/ca.crt - xpack.security.transport.ssl.certificate=$CERTS_DIR/es02/es02.crt - xpack.security.transport.ssl.key=$CERTS_DIR/es02/es02.key - volumes: ['esdata_02:/usr/share/elasticsearch/data', './certs:$CERTS_DIR'] + volumes: ['data02:/usr/share/elasticsearch/data', 'certs:$CERTS_DIR'] wait_until_ready: image: {docker-image} command: /usr/bin/true depends_on: {"es01": {"condition": "service_healthy"}} -volumes: {"esdata_01": {"driver": "local"}, "esdata_02": {"driver": "local"}} +volumes: {"data01", "data02", "certs"} ---- <1> Bootstrap `elastic` with the password defined in `.env`. See @@ -175,7 +177,7 @@ endif::[] -- ["source","sh"] ---- -docker-compose -f create-certs.yml up +docker-compose -f create-certs.yml run --rm create_certs ---- -- . Start two {es} nodes configured for SSL/TLS: @@ -189,9 +191,9 @@ docker-compose up -d . Access the {es} API over SSL/TLS using the bootstrapped password: + -- -["source","sh"] +["source","sh",subs="attributes"] ---- -curl --cacert certs/ca/ca.crt -u elastic:PleaseChangeMe https://localhost:9200 +docker run --rm -v es_certs:/certs --network=es_default {docker-image} curl --cacert /certs/ca/ca.crt -u elastic:PleaseChangeMe https://es01:9200 ---- // NOTCONSOLE -- @@ -210,3 +212,13 @@ auto --batch \ --url https://localhost:9200" ---- -- + +[float] +==== Tear everything down +To remove all the Docker resources created by the example, issue: +-- +["source","sh"] +---- +docker-compose down -v +---- +-- diff --git a/docs/reference/security/securing-communications/separating-node-client-traffic.asciidoc b/docs/reference/security/securing-communications/separating-node-client-traffic.asciidoc deleted file mode 100644 index 61ebd3e682594..0000000000000 --- a/docs/reference/security/securing-communications/separating-node-client-traffic.asciidoc +++ /dev/null @@ -1,68 +0,0 @@ -[role="xpack"] -[[separating-node-client-traffic]] -=== Separating node-to-node and client traffic - -Elasticsearch has the feature of so called -{ref}/modules-transport.html[TCP transport profiles] -that allows it to bind to several ports and addresses. The {es} -{security-features} extend on this functionality to enhance the security of the -cluster by enabling the separation of node-to-node transport traffic from client -transport traffic. This is important if the client transport traffic is not -trusted and could potentially be malicious. To separate the node-to-node traffic -from the client traffic, add the following to `elasticsearch.yml`: - -[source, yaml] --------------------------------------------------- -transport.profiles.client: <1> - port: 9500-9600 <2> - xpack.security: - type: client <3> --------------------------------------------------- -<1> `client` is the name of this example profile -<2> The port range that will be used by transport clients to communicate with - this cluster -<3> Categorizes the profile as a `client`. This accounts for additional security - filters by denying request attempts on for internal cluster operations - (e.g shard level actions and ping requests) from this profile. - -If supported by your environment, an internal network can be used for node-to-node -traffic and public network can be used for client traffic by adding the following -to `elasticsearch.yml`: - -[source, yaml] --------------------------------------------------- -transport.profiles.default.bind_host: 10.0.0.1 <1> -transport.profiles.client.bind_host: 1.1.1.1 <2> --------------------------------------------------- -<1> The bind address for the network that will be used for node-to-node communication -<2> The bind address for the network used for client communication - -If separate networks are not available, then -{stack-ov}/ip-filtering.html[IP Filtering] can -be enabled to limit access to the profiles. - -When using SSL for transport, a different set of certificates can also be used -for the client traffic by adding the following to `elasticsearch.yml`: - -[source, yaml] --------------------------------------------------- -transport.profiles.client.xpack.security.ssl.truststore: - path: /path/to/another/truststore - password: x-pack-test-password - -transport.profiles.client.xpack.security.ssl.keystore: - path: /path/to/another/keystore - password: x-pack-test-password --------------------------------------------------- - -To change the default behavior that requires certificates for transport clients, -set the following value in the `elasticsearch.yml` file: - -[source, yaml] --------------------------------------------------- -transport.profiles.client.xpack.security.ssl.client_authentication: none --------------------------------------------------- - -This setting keeps certificate authentication active for node-to-node traffic, -but removes the requirement to distribute a signed certificate to transport -clients. \ No newline at end of file diff --git a/docs/reference/settings/ml-settings.asciidoc b/docs/reference/settings/ml-settings.asciidoc index 09fb8adad8523..91afcbe5b3466 100644 --- a/docs/reference/settings/ml-settings.asciidoc +++ b/docs/reference/settings/ml-settings.asciidoc @@ -109,3 +109,11 @@ cluster and the job is assigned to run on that node. IMPORTANT: This setting assumes some external process is capable of adding ML nodes to the cluster. This setting is only useful when used in conjunction with such an external process. + +`xpack.ml.process_connect_timeout` (<>):: +The connection timeout for {ml} processes that run separately from the {es} JVM. +Defaults to `10s`. Some {ml} processing is done by processes that run separately +to the {es} JVM. When such processes are started they must connect to the {es} +JVM. If such a process does not connect within the time period specified by this +setting then the process is assumed to have failed. Defaults to `10s`. The minimum +value for this setting is `5s`. diff --git a/docs/reference/settings/monitoring-settings.asciidoc b/docs/reference/settings/monitoring-settings.asciidoc index c48b7d8764d2c..18379577e6a3e 100644 --- a/docs/reference/settings/monitoring-settings.asciidoc +++ b/docs/reference/settings/monitoring-settings.asciidoc @@ -283,5 +283,6 @@ For example: `["elasticsearch_version_mismatch","xpack_license_expiration"]`. :component: {monitoring} :verifies: :server!: +:ssl-context: monitoring include::ssl-settings.asciidoc[] diff --git a/docs/reference/settings/notification-settings.asciidoc b/docs/reference/settings/notification-settings.asciidoc index 77f755b09e285..a2eb84bc2110e 100644 --- a/docs/reference/settings/notification-settings.asciidoc +++ b/docs/reference/settings/notification-settings.asciidoc @@ -37,14 +37,8 @@ required. For more information, see {xpack-ref}/encrypting-data.html[Encrypting sensitive data in {watcher}]. `xpack.watcher.history.cleaner_service.enabled`:: -ifdef::asciidoctor[] added:[6.3.0,Default changed to `true`.] deprecated:[7.0.0,Watcher history indices are now managed by the `watch-history-ilm-policy` ILM policy] -endif::[] -ifndef::asciidoctor[] -added[6.3.0,Default changed to `true`.] -deprecated[7.0.0,Watcher history indices are now managed by the `watch-history-ilm-policy` ILM policy] -endif::[] + Set to `true` (default) to enable the cleaner service. If this setting is `true`, the `xpack.monitoring.enabled` setting must also be set to `true` with @@ -85,6 +79,7 @@ corresponding endpoints are whitelisted as well. :component: {watcher} :verifies: :server!: +:ssl-context: watcher include::ssl-settings.asciidoc[] diff --git a/docs/reference/settings/security-settings.asciidoc b/docs/reference/settings/security-settings.asciidoc index 3cfe1d1e58769..00c1941f65256 100644 --- a/docs/reference/settings/security-settings.asciidoc +++ b/docs/reference/settings/security-settings.asciidoc @@ -1566,6 +1566,7 @@ a PKCS#12 container includes trusted certificate ("anchor") entries look for :client-auth-default: none :verifies!: :server: +:ssl-context: security-http include::ssl-settings.asciidoc[] @@ -1575,6 +1576,7 @@ include::ssl-settings.asciidoc[] :client-auth-default!: :verifies: :server: +:ssl-context: security-transport include::ssl-settings.asciidoc[] diff --git a/docs/reference/settings/ssl-settings.asciidoc b/docs/reference/settings/ssl-settings.asciidoc index f392d0c2fb816..a9c8576a8c4e9 100644 --- a/docs/reference/settings/ssl-settings.asciidoc +++ b/docs/reference/settings/ssl-settings.asciidoc @@ -1,4 +1,3 @@ - ==== {component} TLS/SSL Settings You can configure the following TLS/SSL settings. If the settings are not configured, the {ref}/security-settings.html#ssl-tls-settings[Default TLS/SSL Settings] @@ -39,6 +38,7 @@ endif::verifies[] Supported cipher suites can be found in Oracle's http://docs.oracle.com/javase/8/docs/technotes/guides/security/SunProviders.html[ Java Cryptography Architecture documentation]. Defaults to ``. +[#{ssl-context}-tls-ssl-key-trusted-certificate-settings] ===== {component} TLS/SSL Key and Trusted Certificate Settings The following settings are used to specify a private key, certificate, and the @@ -105,6 +105,7 @@ Password to the truststore. +{ssl-prefix}.ssl.truststore.secure_password+ (<>):: Password to the truststore. +[#{ssl-context}-pkcs12-files] ===== PKCS#12 Files {es} can be configured to use PKCS#12 container files (`.p12` or `.pfx` files) @@ -143,6 +144,7 @@ Password to the PKCS#12 file. +{ssl-prefix}.ssl.truststore.secure_password+ (<>):: Password to the PKCS#12 file. +[#{ssl-context}-pkcs11-tokens] ===== PKCS#11 Tokens {es} can be configured to use a PKCS#11 token that contains the private key, diff --git a/docs/reference/setup/important-settings/discovery-settings.asciidoc b/docs/reference/setup/important-settings/discovery-settings.asciidoc index 245852b209609..942c076a33a0d 100644 --- a/docs/reference/setup/important-settings/discovery-settings.asciidoc +++ b/docs/reference/setup/important-settings/discovery-settings.asciidoc @@ -39,9 +39,9 @@ first election. In <>, with no discovery settings configured, this step is automatically performed by the nodes themselves. As this auto-bootstrapping is <>, when you start a brand new cluster in <>, you must explicitly list the names or IP addresses of the -master-eligible nodes whose votes should be counted in the very first election. -This list is set using the `cluster.initial_master_nodes` setting. +mode>>, you must explicitly list the master-eligible nodes whose votes should be +counted in the very first election. This list is set using the +`cluster.initial_master_nodes` setting. [source,yaml] -------------------------------------------------- diff --git a/docs/reference/setup/install.asciidoc b/docs/reference/setup/install.asciidoc index 800cecb40652a..7e03ad3947b4c 100644 --- a/docs/reference/setup/install.asciidoc +++ b/docs/reference/setup/install.asciidoc @@ -59,6 +59,13 @@ downloaded from the Elastic Docker Registry. + {ref}/docker.html[Install {es} with Docker] +`brew`:: + +Formulae are available from the Elastic Homebrew tap for installing +{es} on macOS with the Homebrew package manager. ++ +{ref}/brew.html[Install {es} on macOS with Homebrew] + [float] [[config-mgmt-tools]] === Configuration Management Tools @@ -84,3 +91,4 @@ include::install/windows.asciidoc[] include::install/docker.asciidoc[] +include::install/brew.asciidoc[] diff --git a/docs/reference/setup/install/brew.asciidoc b/docs/reference/setup/install/brew.asciidoc new file mode 100644 index 0000000000000..cfc9e4c3e05be --- /dev/null +++ b/docs/reference/setup/install/brew.asciidoc @@ -0,0 +1,69 @@ +[[brew]] +=== Install {es} on macOS with Homebrew + +Elastic publishes Homebrew formulae so you can install {es} with the +https://brew.sh/[Homebrew] package manager. + +To install with Homebrew, you first need to tap the +Elastic Homebrew repository: + +[source,sh] +------------------------- +brew tap elastic/tap +------------------------- + +Once you've tapped the Elastic Homebrew repo, you can use `brew install` to +install the default distribution of {es}: + +[source,sh] +------------------------- +brew install elastic/tap/elasticsearch-full +------------------------- + +This installs the most recently released default distribution of {es}. +To install the OSS distribution, specify `elastic/tap/elasticsearch-oss`. + +[[brew-layout]] +==== Directory layout for Homebrew installs + +When you install {es} with `brew install` the config files, logs, +and data directory are stored in the following locations. + +[cols="> + +| data + | The location of the data files of each index / shard allocated + on the node. Can hold multiple locations. + | /usr/local/var/lib/elasticsearch + | path.data + +| logs + | Log files location. + | /usr/local/var/log/elasticsearch + | path.logs + +| plugins + | Plugin files location. Each plugin will be contained in a subdirectory. + | /usr/local/var/homebrew/linked/elasticsearch/plugins + | + +|======================================================================= + +include::next-steps.asciidoc[] diff --git a/docs/reference/setup/install/docker.asciidoc b/docs/reference/setup/install/docker.asciidoc index 1fcc261d68e1f..e8dd1ee95957a 100644 --- a/docs/reference/setup/install/docker.asciidoc +++ b/docs/reference/setup/install/docker.asciidoc @@ -332,7 +332,7 @@ data through a bind-mount: As a last resort, you can also force the container to mutate the ownership of any bind-mounts used for the <> through the -environment variable `TAKE_FILE_OWNERSHIP`. Inn this case, they will be owned by +environment variable `TAKE_FILE_OWNERSHIP`. In this case, they will be owned by uid:gid `1000:0` providing read/write access to the {es} process as required. -- @@ -372,6 +372,12 @@ published ports with `--publish-all`, unless you are pinning one container per h . Use the `ES_JAVA_OPTS` environment variable to set heap size. For example, to use 16GB, use `-e ES_JAVA_OPTS="-Xms16g -Xmx16g"` with `docker run`. ++ +-- +NOTE: You still need to <> even if you are +https://docs.docker.com/config/containers/resource_constraints/#limit-a-containers-access-to-memory[limiting +memory access] to the container. +-- . Pin your deployments to a specific version of the {es} Docker image, for example +docker.elastic.co/elasticsearch/elasticsearch:{version}+. diff --git a/docs/reference/setup/secure-settings.asciidoc b/docs/reference/setup/secure-settings.asciidoc index 4e3799db75bde..82b61848a846d 100644 --- a/docs/reference/setup/secure-settings.asciidoc +++ b/docs/reference/setup/secure-settings.asciidoc @@ -7,8 +7,12 @@ keystore and the `elasticsearch-keystore` tool to manage the settings in the key NOTE: All commands here should be run as the user which will run Elasticsearch. -NOTE: Only some settings are designed to be read from the keystore. See -documentation for each setting to see if it is supported as part of the keystore. +IMPORTANT: Only some settings are designed to be read from the keystore. However, +the keystore has no validation to block unsupported settings. +Adding unsupported settings to the keystore will cause {es} +Additional unsupported settings being added to the keystore will cause Elasticsearch +to fail to start. See documentation for each setting to see if it is supported +as part of the keystore. NOTE: All the modifications to the keystore take affect only after restarting Elasticsearch. diff --git a/docs/reference/sql/endpoints/jdbc.asciidoc b/docs/reference/sql/endpoints/jdbc.asciidoc index 4e9de8934895a..fdacb8b97a232 100644 --- a/docs/reference/sql/endpoints/jdbc.asciidoc +++ b/docs/reference/sql/endpoints/jdbc.asciidoc @@ -207,3 +207,8 @@ connection. For example: -------------------------------------------------- include-tagged::{jdbc-tests}/SimpleExampleTestCase.java[simple_example] -------------------------------------------------- + +NOTE:: {es-sql} doesn't provide a connection pooling mechanism, thus the connections +the JDBC driver creates are not pooled. In order to achieve pooled connections, +a third-party connection pooling mechanism is required. Configuring and setting up the +third-party provider is outside the scope of this documentation. \ No newline at end of file diff --git a/docs/reference/sql/endpoints/rest.asciidoc b/docs/reference/sql/endpoints/rest.asciidoc index e44649f3a8767..8140fe162d9de 100644 --- a/docs/reference/sql/endpoints/rest.asciidoc +++ b/docs/reference/sql/endpoints/rest.asciidoc @@ -3,6 +3,16 @@ [[sql-rest]] == SQL REST API +* <> +* <> +* <> +* <> +* <> +* <> + +[[sql-rest-overview]] +=== Overview + The SQL REST API accepts SQL in a JSON document, executes it, and returns the results. For example: @@ -30,18 +40,17 @@ Alastair Reynolds|Revelation Space |585 |2000-03-15T00:00:00.000Z James S.A. Corey |Leviathan Wakes |561 |2011-06-02T00:00:00.000Z -------------------------------------------------- // TESTRESPONSE[s/\|/\\|/ s/\+/\\+/] -// TESTRESPONSE[_cat] +// TESTRESPONSE[non_json] [[sql-kibana-console]] .Using Kibana Console -If you are using {kibana-ref}/console-kibana.html[Kibana Console]. +If you are using {kibana-ref}/console-kibana.html[Kibana Console] (which is highly recommended), take advantage of the triple quotes `"""` when creating the query. This not only automatically escapes double quotes (`"`) inside the query string but also support multi-line as shown below: image:images/sql/rest/console-triple-quotes.png[] [[sql-rest-format]] -[float] === Response Data Formats While the textual format is nice for humans, computers prefer something @@ -94,6 +103,35 @@ s|Description |=== +Here are some examples for the human readable formats: + +==== CSV + +[source,js] +-------------------------------------------------- +POST /_sql?format=csv +{ + "query": "SELECT * FROM library ORDER BY page_count DESC", + "fetch_size": 5 +} +-------------------------------------------------- +// CONSOLE +// TEST[setup:library] + +Which returns: + +[source,text] +-------------------------------------------------- +author,name,page_count,release_date +Peter F. Hamilton,Pandora's Star,768,2004-03-02T00:00:00.000Z +Vernor Vinge,A Fire Upon the Deep,613,1992-06-01T00:00:00.000Z +Frank Herbert,Dune,604,1965-06-01T00:00:00.000Z +Alastair Reynolds,Revelation Space,585,2000-03-15T00:00:00.000Z +James S.A. Corey,Leviathan Wakes,561,2011-06-02T00:00:00.000Z +-------------------------------------------------- +// TESTRESPONSE[non_json] + +==== JSON [source,js] -------------------------------------------------- @@ -129,8 +167,113 @@ Which returns: -------------------------------------------------- // TESTRESPONSE[s/sDXF1ZXJ5QW5kRmV0Y2gBAAAAAAAAAAEWWWdrRlVfSS1TbDYtcW9lc1FJNmlYdw==:BAFmBmF1dGhvcgFmBG5hbWUBZgpwYWdlX2NvdW50AWYMcmVsZWFzZV9kYXRl\+v\/\/\/w8=/$body.cursor/] +==== TSV + +[source,js] +-------------------------------------------------- +POST /_sql?format=tsv +{ + "query": "SELECT * FROM library ORDER BY page_count DESC", + "fetch_size": 5 +} +-------------------------------------------------- +// CONSOLE +// TEST[setup:library] + +Which returns: + +[source,text] +-------------------------------------------------- +author name page_count release_date +Peter F. Hamilton Pandora's Star 768 2004-03-02T00:00:00.000Z +Vernor Vinge A Fire Upon the Deep 613 1992-06-01T00:00:00.000Z +Frank Herbert Dune 604 1965-06-01T00:00:00.000Z +Alastair Reynolds Revelation Space 585 2000-03-15T00:00:00.000Z +James S.A. Corey Leviathan Wakes 561 2011-06-02T00:00:00.000Z +-------------------------------------------------- +// TESTRESPONSE[s/\t/ /] +// TESTRESPONSE[non_json] + +==== TXT + +[source,js] +-------------------------------------------------- +POST /_sql?format=txt +{ + "query": "SELECT * FROM library ORDER BY page_count DESC", + "fetch_size": 5 +} +-------------------------------------------------- +// CONSOLE +// TEST[setup:library] + +Which returns: + +[source,text] +-------------------------------------------------- + author | name | page_count | release_date +-----------------+--------------------+---------------+------------------------ +Peter F. Hamilton|Pandora's Star |768 |2004-03-02T00:00:00.000Z +Vernor Vinge |A Fire Upon the Deep|613 |1992-06-01T00:00:00.000Z +Frank Herbert |Dune |604 |1965-06-01T00:00:00.000Z +Alastair Reynolds|Revelation Space |585 |2000-03-15T00:00:00.000Z +James S.A. Corey |Leviathan Wakes |561 |2011-06-02T00:00:00.000Z +-------------------------------------------------- +// TESTRESPONSE[s/\|/\\|/ s/\+/\\+/] +// TESTRESPONSE[non_json] + +==== YAML + +[source,js] +-------------------------------------------------- +POST /_sql?format=yaml +{ + "query": "SELECT * FROM library ORDER BY page_count DESC", + "fetch_size": 5 +} +-------------------------------------------------- +// CONSOLE +// TEST[setup:library] + +Which returns: + +[source,yaml] +-------------------------------------------------- +columns: +- name: "author" + type: "text" +- name: "name" + type: "text" +- name: "page_count" + type: "short" +- name: "release_date" + type: "datetime" +rows: +- - "Peter F. Hamilton" + - "Pandora's Star" + - 768 + - "2004-03-02T00:00:00.000Z" +- - "Vernor Vinge" + - "A Fire Upon the Deep" + - 613 + - "1992-06-01T00:00:00.000Z" +- - "Frank Herbert" + - "Dune" + - 604 + - "1965-06-01T00:00:00.000Z" +- - "Alastair Reynolds" + - "Revelation Space" + - 585 + - "2000-03-15T00:00:00.000Z" +- - "James S.A. Corey" + - "Leviathan Wakes" + - 561 + - "2011-06-02T00:00:00.000Z" +cursor: "sDXF1ZXJ5QW5kRmV0Y2gBAAAAAAAAAAEWWWdrRlVfSS1TbDYtcW9lc1FJNmlYdw==:BAFmBmF1dGhvcgFmBG5hbWUBZgpwYWdlX2NvdW50AWYMcmVsZWFzZV9kYXRl+v///w8=" +-------------------------------------------------- +// TESTRESPONSE[s/sDXF1ZXJ5QW5kRmV0Y2gBAAAAAAAAAAEWWWdrRlVfSS1TbDYtcW9lc1FJNmlYdw==:BAFmBmF1dGhvcgFmBG5hbWUBZgpwYWdlX2NvdW50AWYMcmVsZWFzZV9kYXRl\+v\/\/\/w8=/$body.cursor/] + [[sql-pagination]] -[float] === Paginating through a large response Using the example above, one can continue to the next page by sending back the `cursor` field. In @@ -198,7 +341,6 @@ Which will like return the [[sql-rest-filtering]] -[float] === Filtering using {es} query DSL You can filter the results that SQL will run on using a standard @@ -233,10 +375,9 @@ Which returns: Douglas Adams |The Hitchhiker's Guide to the Galaxy|180 |1979-10-12T00:00:00.000Z -------------------------------------------------- // TESTRESPONSE[s/\|/\\|/ s/\+/\\+/] -// TESTRESPONSE[_cat] +// TESTRESPONSE[non_json] [[sql-rest-columnar]] -[float] === Columnar results The most well known way of displaying the results of an SQL query result in general is the one where each @@ -311,7 +452,6 @@ Which looks like: // TESTRESPONSE[s/46ToAwFzQERYRjFaWEo1UVc1a1JtVjBZMmdCQUFBQUFBQUFBQUVXWjBaNlFXbzNOV0pVY21Wa1NUZDJhV2t3V2xwblp3PT3\/\/\/\/\/DwQBZgZhdXRob3IBBHRleHQAAAFmBG5hbWUBBHRleHQAAAFmCnBhZ2VfY291bnQBBGxvbmcBAAFmDHJlbGVhc2VfZGF0ZQEIZGF0ZXRpbWUBAAEP/$body.cursor/] [[sql-rest-fields]] -[float] === Supported REST parameters In addition to the `query` and `fetch_size`, a request a number of user-defined fields for specifying diff --git a/docs/reference/sql/functions/conditional.asciidoc b/docs/reference/sql/functions/conditional.asciidoc index d0b8e7d2ff3f1..0206115c182c1 100644 --- a/docs/reference/sql/functions/conditional.asciidoc +++ b/docs/reference/sql/functions/conditional.asciidoc @@ -33,17 +33,17 @@ If the condition’s result is true, the value of the result expression that fol the subsequent when clauses will be skipped and not processed. -["source","sql",subs="attributes,callouts,macros"] +[source, sql] ---- include-tagged::{sql-specs}/docs/docs.csv-spec[case] ---- -["source","sql",subs="attributes,callouts,macros"] +[source, sql] ---- include-tagged::{sql-specs}/docs/docs.csv-spec[caseReturnNull] ---- -["source","sql",subs="attributes,callouts,macros"] +[source, sql] ---- include-tagged::{sql-specs}/docs/docs.csv-spec[caseWithElse] ---- @@ -70,12 +70,12 @@ CASE WHEN expression = value1 THEN result1 END ---- -["source","sql",subs="attributes,callouts,macros"] +[source, sql] ---- include-tagged::{sql-specs}/docs/docs.csv-spec[caseWithOperand] ---- -["source","sql",subs="attributes,callouts,macros"] +[source, sql] ---- include-tagged::{sql-specs}/docs/docs.csv-spec[caseWithOperandAndElse] ---- @@ -155,12 +155,12 @@ If all arguments are null, then it returns `null`. -["source","sql",subs="attributes,callouts,macros"] +[source, sql] ---- include-tagged::{sql-specs}/docs/docs.csv-spec[coalesceReturnNonNull] ---- -["source","sql",subs="attributes,callouts,macros"] +[source, sql] ---- include-tagged::{sql-specs}/docs/docs.csv-spec[coalesceReturnNull] ---- @@ -199,12 +199,12 @@ If all arguments are null, then it returns `null`. -["source","sql",subs="attributes,callouts,macros"] +[source, sql] ---- include-tagged::{sql-specs}/docs/docs.csv-spec[greatestReturnNonNull] ---- -["source","sql",subs="attributes,callouts,macros"] +[source, sql] ---- include-tagged::{sql-specs}/docs/docs.csv-spec[greatestReturnNull] ---- @@ -237,12 +237,12 @@ If all arguments are null, then it returns `null`. -["source","sql",subs="attributes,callouts,macros"] +[source, sql] ---- include-tagged::{sql-specs}/docs/docs.csv-spec[ifNullReturnFirst] ---- -["source","sql",subs="attributes,callouts,macros"] +[source, sql] ---- include-tagged::{sql-specs}/docs/docs.csv-spec[ifNullReturnSecond] ---- @@ -277,12 +277,12 @@ logic of programming languages. If the 3rd expression is not provided and the co `null` is returned. -["source","sql",subs="attributes,callouts,macros"] +[source, sql] ---- include-tagged::{sql-specs}/docs/docs.csv-spec[iifWithDefaultValue] ---- -["source","sql",subs="attributes,callouts,macros"] +[source, sql] ---- include-tagged::{sql-specs}/docs/docs.csv-spec[iifWithoutDefaultValue] ---- @@ -325,12 +325,12 @@ If all arguments are null, then it returns `null`. -["source","sql",subs="attributes,callouts,macros"] +[source, sql] ---- include-tagged::{sql-specs}/docs/docs.csv-spec[isNullReturnFirst] ---- -["source","sql",subs="attributes,callouts,macros"] +[source, sql] ---- include-tagged::{sql-specs}/docs/docs.csv-spec[isNullReturnSecond] ---- @@ -370,12 +370,12 @@ If all arguments are null, then it returns `null`. -["source","sql",subs="attributes,callouts,macros"] +[source, sql] ---- include-tagged::{sql-specs}/docs/docs.csv-spec[leastReturnNonNull] ---- -["source","sql",subs="attributes,callouts,macros"] +[source, sql] ---- include-tagged::{sql-specs}/docs/docs.csv-spec[leastReturnNull] ---- @@ -407,12 +407,12 @@ Returns `null` when the two input expressions are equal and if not, it returns the 1st expression. -["source","sql",subs="attributes,callouts,macros"] +[source, sql] ---- include-tagged::{sql-specs}/docs/docs.csv-spec[nullIfReturnFirst] ---- -["source","sql",subs="attributes,callouts,macros"] +[source, sql] ---- include-tagged::{sql-specs}/docs/docs.csv-spec[nullIfReturnNull] ---- @@ -446,12 +446,12 @@ If all arguments are null, then it returns `null`. -["source","sql",subs="attributes,callouts,macros"] +[source, sql] ---- include-tagged::{sql-specs}/docs/docs.csv-spec[nvlReturnFirst] ---- -["source","sql",subs="attributes,callouts,macros"] +[source, sql] ---- include-tagged::{sql-specs}/docs/docs.csv-spec[nvlReturnSecond] ---- diff --git a/docs/reference/sql/functions/date-time.asciidoc b/docs/reference/sql/functions/date-time.asciidoc index d9d5e7bcf14e5..45231393521c7 100644 --- a/docs/reference/sql/functions/date-time.asciidoc +++ b/docs/reference/sql/functions/date-time.asciidoc @@ -57,32 +57,32 @@ s|Description Basic arithmetic operators (`+`, `-`, etc) support date/time parameters as indicated below: -["source","sql",subs="attributes,callouts,macros"] +[source, sql] -------------------------------------------------- include-tagged::{sql-specs}/docs/docs.csv-spec[dtIntervalPlusInterval] -------------------------------------------------- -["source","sql",subs="attributes,callouts,macros"] +[source, sql] -------------------------------------------------- include-tagged::{sql-specs}/docs/docs.csv-spec[dtDateTimePlusInterval] -------------------------------------------------- -["source","sql",subs="attributes,callouts,macros"] +[source, sql] -------------------------------------------------- include-tagged::{sql-specs}/docs/docs.csv-spec[dtMinusInterval] -------------------------------------------------- -["source","sql",subs="attributes,callouts,macros"] +[source, sql] -------------------------------------------------- include-tagged::{sql-specs}/docs/docs.csv-spec[dtIntervalMinusInterval] -------------------------------------------------- -["source","sql",subs="attributes,callouts,macros"] +[source, sql] -------------------------------------------------- include-tagged::{sql-specs}/docs/docs.csv-spec[dtDateTimeMinusInterval] -------------------------------------------------- -["source","sql",subs="attributes,callouts,macros"] +[source, sql] -------------------------------------------------- include-tagged::{sql-specs}/docs/docs.csv-spec[dtIntervalMul] -------------------------------------------------- @@ -116,17 +116,17 @@ Unlike CURRENT_DATE, `CURDATE()` can only be used as a function with no argument This method always returns the same value for its every occurrence within the same query. -["source","sql",subs="attributes,callouts,macros"] +[source, sql] -------------------------------------------------- include-tagged::{sql-specs}/docs/docs.csv-spec[currentDate] -------------------------------------------------- -["source","sql",subs="attributes,callouts,macros"] +[source, sql] -------------------------------------------------- include-tagged::{sql-specs}/docs/docs.csv-spec[currentDateFunction] -------------------------------------------------- -["source","sql",subs="attributes,callouts,macros"] +[source, sql] -------------------------------------------------- include-tagged::{sql-specs}/docs/docs.csv-spec[curDateFunction] -------------------------------------------------- @@ -134,7 +134,7 @@ include-tagged::{sql-specs}/docs/docs.csv-spec[curDateFunction] Typically, this function (as well as its twin <> function is used for relative date filtering: -["source","sql",subs="attributes,callouts,macros"] +[source, sql] -------------------------------------------------- include-tagged::{sql-specs}/docs/docs.csv-spec[filterToday] -------------------------------------------------- @@ -165,29 +165,29 @@ meaning a milliseconds precision current time will be returned. This method always returns the same value for its every occurrence within the same query. -["source","sql",subs="attributes,callouts,macros"] +[source, sql] -------------------------------------------------- include-tagged::{sql-specs}/docs/docs.csv-spec[currentTime] -------------------------------------------------- -["source","sql",subs="attributes,callouts,macros"] +[source, sql] -------------------------------------------------- include-tagged::{sql-specs}/docs/docs.csv-spec[currentTimeFunction] -------------------------------------------------- -["source","sql",subs="attributes,callouts,macros"] +[source, sql] -------------------------------------------------- include-tagged::{sql-specs}/docs/docs.csv-spec[curTimeFunction] -------------------------------------------------- -["source","sql",subs="attributes,callouts,macros"] +[source, sql] -------------------------------------------------- include-tagged::{sql-specs}/docs/docs.csv-spec[currentTimeFunctionPrecision] -------------------------------------------------- Typically, this function is used for relative date/time filtering: -["source","sql",subs="attributes,callouts,macros"] +[source, sql] -------------------------------------------------- include-tagged::{sql-specs}/docs/docs.csv-spec[filterCurrentTime] -------------------------------------------------- @@ -221,17 +221,17 @@ meaning a milliseconds precision current date/time will be returned. This method always returns the same value for its every occurrence within the same query. -["source","sql",subs="attributes,callouts,macros"] +[source, sql] -------------------------------------------------- include-tagged::{sql-specs}/docs/docs.csv-spec[curTs] -------------------------------------------------- -["source","sql",subs="attributes,callouts,macros"] +[source, sql] -------------------------------------------------- include-tagged::{sql-specs}/docs/docs.csv-spec[curTsFunction] -------------------------------------------------- -["source","sql",subs="attributes,callouts,macros"] +[source, sql] -------------------------------------------------- include-tagged::{sql-specs}/docs/docs.csv-spec[curTsFunctionPrecision] -------------------------------------------------- @@ -239,7 +239,7 @@ include-tagged::{sql-specs}/docs/docs.csv-spec[curTsFunctionPrecision] Typically, this function (as well as its twin <> function is used for relative date/time filtering: -["source","sql",subs="attributes,callouts,macros"] +[source, sql] -------------------------------------------------- include-tagged::{sql-specs}/docs/docs.csv-spec[filterNow] -------------------------------------------------- @@ -267,7 +267,7 @@ DAY_OF_MONTH(datetime_exp) <1> Extract the day of the month from a date/datetime. -["source","sql",subs="attributes,callouts,macros"] +[source, sql] -------------------------------------------------- include-tagged::{sql-specs}/docs/docs.csv-spec[dayOfMonth] -------------------------------------------------- @@ -291,7 +291,7 @@ DAY_OF_WEEK(datetime_exp) <1> Extract the day of the week from a date/datetime. Sunday is `1`, Monday is `2`, etc. -["source","sql",subs="attributes,callouts,macros"] +[source, sql] -------------------------------------------------- include-tagged::{sql-specs}/docs/docs.csv-spec[dayOfWeek] -------------------------------------------------- @@ -315,7 +315,7 @@ DAY_OF_YEAR(datetime_exp) <1> Extract the day of the year from a date/datetime. -["source","sql",subs="attributes,callouts,macros"] +[source, sql] -------------------------------------------------- include-tagged::{sql-specs}/docs/docs.csv-spec[dayOfYear] -------------------------------------------------- @@ -339,7 +339,7 @@ DAY_NAME(datetime_exp) <1> Extract the day of the week from a date/datetime in text format (`Monday`, `Tuesday`...). -["source","sql",subs="attributes,callouts,macros"] +[source, sql] -------------------------------------------------- include-tagged::{sql-specs}/docs/docs.csv-spec[dayName] -------------------------------------------------- @@ -363,7 +363,7 @@ HOUR_OF_DAY(datetime_exp) <1> Extract the hour of the day from a date/datetime. -["source","sql",subs="attributes,callouts,macros"] +[source, sql] -------------------------------------------------- include-tagged::{sql-specs}/docs/docs.csv-spec[hourOfDay] -------------------------------------------------- @@ -388,7 +388,7 @@ ISO_DAY_OF_WEEK(datetime_exp) <1> Extract the day of the week from a date/datetime, following the https://en.wikipedia.org/wiki/ISO_week_date[ISO 8601 standard]. Monday is `1`, Tuesday is `2`, etc. -["source","sql",subs="attributes,callouts,macros"] +[source, sql] -------------------------------------------------- include-tagged::{sql-specs}/docs/docs.csv-spec[isoDayOfWeek] -------------------------------------------------- @@ -413,7 +413,7 @@ ISO_WEEK_OF_YEAR(datetime_exp) <1> Extract the week of the year from a date/datetime, following https://en.wikipedia.org/wiki/ISO_week_date[ISO 8601 standard]. The first week of a year is the first week with a majority (4 or more) of its days in January. -["source","sql",subs="attributes,callouts,macros"] +[source, sql] -------------------------------------------------- include-tagged::{sql-specs}/docs/docs.csv-spec[isoWeekOfYear] -------------------------------------------------- @@ -437,7 +437,7 @@ MINUTE_OF_DAY(datetime_exp) <1> Extract the minute of the day from a date/datetime. -["source","sql",subs="attributes,callouts,macros"] +[source, sql] -------------------------------------------------- include-tagged::{sql-specs}/docs/docs.csv-spec[minuteOfDay] -------------------------------------------------- @@ -461,7 +461,7 @@ MINUTE_OF_HOUR(datetime_exp) <1> Extract the minute of the hour from a date/datetime. -["source","sql",subs="attributes,callouts,macros"] +[source, sql] -------------------------------------------------- include-tagged::{sql-specs}/docs/docs.csv-spec[minuteOfHour] -------------------------------------------------- @@ -485,7 +485,7 @@ MONTH(datetime_exp) <1> Extract the month of the year from a date/datetime. -["source","sql",subs="attributes,callouts,macros"] +[source, sql] -------------------------------------------------- include-tagged::{sql-specs}/docs/docs.csv-spec[monthOfYear] -------------------------------------------------- @@ -509,7 +509,7 @@ MONTH_NAME(datetime_exp) <1> Extract the month from a date/datetime in text format (`January`, `February`...). -["source","sql",subs="attributes,callouts,macros"] +[source, sql] -------------------------------------------------- include-tagged::{sql-specs}/docs/docs.csv-spec[monthName] -------------------------------------------------- @@ -533,7 +533,7 @@ This function offers the same functionality as <> function is used for relative date/time filtering: -["source","sql",subs="attributes,callouts,macros"] +[source, sql] -------------------------------------------------- include-tagged::{sql-specs}/docs/docs.csv-spec[filterNow] -------------------------------------------------- @@ -565,7 +565,7 @@ SECOND_OF_MINUTE(datetime_exp) <1> Extract the second of the minute from a date/datetime. -["source","sql",subs="attributes,callouts,macros"] +[source, sql] -------------------------------------------------- include-tagged::{sql-specs}/docs/docs.csv-spec[secondOfMinute] -------------------------------------------------- @@ -589,7 +589,7 @@ QUARTER(datetime_exp) <1> Extract the year quarter the date/datetime falls in. -["source","sql",subs="attributes,callouts,macros"] +[source, sql] -------------------------------------------------- include-tagged::{sql-specs}/docs/docs.csv-spec[quarter] -------------------------------------------------- @@ -613,7 +613,7 @@ This function offers the same functionality as <> function is used for relative date filtering: -["source","sql",subs="attributes,callouts,macros"] +[source, sql] -------------------------------------------------- include-tagged::{sql-specs}/docs/docs.csv-spec[filterToday] -------------------------------------------------- @@ -645,7 +645,7 @@ WEEK_OF_YEAR(datetime_exp) <1> Extract the week of the year from a date/datetime. -["source","sql",subs="attributes,callouts,macros"] +[source, sql] -------------------------------------------------- include-tagged::{sql-specs}/docs/docs.csv-spec[weekOfYear] -------------------------------------------------- @@ -669,7 +669,7 @@ YEAR(datetime_exp) <1> Extract the year from a date/datetime. -["source","sql",subs="attributes,callouts,macros"] +[source, sql] -------------------------------------------------- include-tagged::{sql-specs}/docs/docs.csv-spec[year] -------------------------------------------------- @@ -697,14 +697,14 @@ EXTRACT( Extract fields from a date/datetime by specifying the name of a <>. The following -["source","sql",subs="attributes,callouts,macros"] +[source, sql] -------------------------------------------------- include-tagged::{sql-specs}/docs/docs.csv-spec[extractDayOfYear] -------------------------------------------------- is the equivalent to -["source","sql",subs="attributes,callouts,macros"] +[source, sql] -------------------------------------------------- include-tagged::{sql-specs}/docs/docs.csv-spec[dayOfYear] -------------------------------------------------- diff --git a/docs/reference/sql/functions/geo.asciidoc b/docs/reference/sql/functions/geo.asciidoc index 72f69af85529f..fc9a85ce97e4d 100644 --- a/docs/reference/sql/functions/geo.asciidoc +++ b/docs/reference/sql/functions/geo.asciidoc @@ -147,7 +147,7 @@ ST_Y( .Description: -Returns the the latitude of the first point in the geometry. +Returns the latitude of the first point in the geometry. ["source","sql",subs="attributes,macros"] -------------------------------------------------- @@ -206,4 +206,4 @@ Returns the distance between geometries in meters. Both geometries have to be po ["source","sql",subs="attributes,macros"] -------------------------------------------------- include-tagged::{sql-specs}/docs/geo.csv-spec[distance] --------------------------------------------------- \ No newline at end of file +-------------------------------------------------- diff --git a/docs/reference/sql/functions/grouping.asciidoc b/docs/reference/sql/functions/grouping.asciidoc index 0a498a1aacef0..6f2f5a1b6e4c2 100644 --- a/docs/reference/sql/functions/grouping.asciidoc +++ b/docs/reference/sql/functions/grouping.asciidoc @@ -44,14 +44,14 @@ NOTE:: The histogram in SQL does *NOT* return empty buckets for missing interval `Histogram` can be applied on either numeric fields: -["source","sql",subs="attributes,callouts,macros"] +[source, sql] ---- include-tagged::{sql-specs}/docs/docs.csv-spec[histogramNumeric] ---- or date/time fields: -["source","sql",subs="attributes,callouts,macros"] +[source, sql] ---- include-tagged::{sql-specs}/docs/docs.csv-spec[histogramDateTime] ---- @@ -59,14 +59,14 @@ include-tagged::{sql-specs}/docs/docs.csv-spec[histogramDateTime] Expressions inside the histogram are also supported as long as the return type is numeric: -["source","sql",subs="attributes,callouts,macros"] +[source, sql] ---- include-tagged::{sql-specs}/docs/docs.csv-spec[histogramNumericExpression] ---- Do note that histograms (and grouping functions in general) allow custom expressions but cannot have any functions applied to them in the `GROUP BY`. In other words, the following statement is *NOT* allowed: -["source","sql",subs="attributes,callouts,macros"] +[source, sql] ---- include-tagged::{sql-specs}/docs/docs.csv-spec[expressionOnHistogramNotAllowed] ---- @@ -75,7 +75,7 @@ as it requires two groupings (one for histogram followed by a second for applyin Instead one can rewrite the query to move the expression on the histogram _inside_ of it: -["source","sql",subs="attributes,callouts,macros"] +[source, sql] ---- include-tagged::{sql-specs}/docs/docs.csv-spec[histogramDateTimeExpression] ---- diff --git a/docs/reference/sql/functions/like-rlike.asciidoc b/docs/reference/sql/functions/like-rlike.asciidoc index 73212bc113542..2d5ef0b62f93f 100644 --- a/docs/reference/sql/functions/like-rlike.asciidoc +++ b/docs/reference/sql/functions/like-rlike.asciidoc @@ -38,7 +38,7 @@ with the `LIKE` operator: The percent sign represents zero, one or multiple characters. The underscore represents a single number or character. These symbols can be used in combinations. -["source","sql",subs="attributes,callouts,macros"] +[source, sql] ---- include-tagged::{sql-specs}/docs/docs.csv-spec[simpleLike] ---- @@ -75,7 +75,7 @@ and underscore (`_`); the pattern in this case is a regular expression which all For more details about the regular expressions syntax, https://docs.oracle.com/en/java/javase/11/docs/api/java.base/java/util/regex/Pattern.html[Java's Pattern class javadoc] is a good starting point. -["source","sql",subs="attributes,callouts,macros"] +[source, sql] ---- include-tagged::{sql-specs}/docs/docs.csv-spec[simpleRLike] ---- diff --git a/docs/reference/sql/functions/operators.asciidoc b/docs/reference/sql/functions/operators.asciidoc index 4b7e8990290dd..02841c84b58e5 100644 --- a/docs/reference/sql/functions/operators.asciidoc +++ b/docs/reference/sql/functions/operators.asciidoc @@ -8,7 +8,7 @@ Boolean operator for comparing against one or multiple expressions. [[sql-operators-equality]] ==== `Equality (=)` -["source","sql",subs="attributes,callouts,macros"] +[source, sql] -------------------------------------------------- include-tagged::{sql-specs}/filter.sql-spec[whereFieldEquality] -------------------------------------------------- @@ -16,12 +16,12 @@ include-tagged::{sql-specs}/filter.sql-spec[whereFieldEquality] [[sql-operators-null-safe-equality]] ==== `Null safe Equality (<=>)` -["source","sql",subs="attributes,callouts,macros"] +[source, sql] -------------------------------------------------- include-tagged::{sql-specs}/docs/docs.csv-spec[nullEqualsCompareWithNull] -------------------------------------------------- -["source","sql",subs="attributes,callouts,macros"] +[source, sql] -------------------------------------------------- include-tagged::{sql-specs}/docs/docs.csv-spec[nullEqualsCompareTwoNulls] -------------------------------------------------- @@ -29,7 +29,7 @@ include-tagged::{sql-specs}/docs/docs.csv-spec[nullEqualsCompareTwoNulls] [[sql-operators-inequality]] ==== `Inequality (<> or !=)` -["source","sql",subs="attributes,callouts,macros"] +[source, sql] -------------------------------------------------- include-tagged::{sql-specs}/filter.sql-spec[whereFieldNonEquality] -------------------------------------------------- @@ -37,7 +37,7 @@ include-tagged::{sql-specs}/filter.sql-spec[whereFieldNonEquality] [[sql-operators-comparison]] ==== `Comparison (<, <=, >, >=)` -["source","sql",subs="attributes,callouts,macros"] +[source, sql] -------------------------------------------------- include-tagged::{sql-specs}/filter.sql-spec[whereFieldLessThan] -------------------------------------------------- @@ -45,7 +45,7 @@ include-tagged::{sql-specs}/filter.sql-spec[whereFieldLessThan] [[sql-operators-between]] ==== `BETWEEN` -["source","sql",subs="attributes,callouts,macros"] +[source, sql] -------------------------------------------------- include-tagged::{sql-specs}/filter.sql-spec[whereBetween] -------------------------------------------------- @@ -53,7 +53,7 @@ include-tagged::{sql-specs}/filter.sql-spec[whereBetween] [[sql-operators-is-null]] ==== `IS NULL/IS NOT NULL` -["source","sql",subs="attributes,callouts,macros"] +[source, sql] -------------------------------------------------- include-tagged::{sql-specs}/filter.sql-spec[whereIsNotNullAndIsNull] -------------------------------------------------- @@ -61,7 +61,7 @@ include-tagged::{sql-specs}/filter.sql-spec[whereIsNotNullAndIsNull] [[sql-operators-in]] ==== `IN (, , ...)` -["source","sql",subs="attributes,callouts,macros"] +[source, sql] -------------------------------------------------- include-tagged::{sql-specs}/filter.sql-spec[whereWithInAndMultipleValues] -------------------------------------------------- @@ -74,7 +74,7 @@ Boolean operator for evaluating one or two expressions. [[sql-operators-and]] ==== `AND` -["source","sql",subs="attributes,callouts,macros"] +[source, sql] -------------------------------------------------- include-tagged::{sql-specs}/filter.sql-spec[whereFieldAndComparison] -------------------------------------------------- @@ -82,7 +82,7 @@ include-tagged::{sql-specs}/filter.sql-spec[whereFieldAndComparison] [[sql-operators-or]] ==== `OR` -["source","sql",subs="attributes,callouts,macros"] +[source, sql] -------------------------------------------------- include-tagged::{sql-specs}/filter.sql-spec[whereFieldOrComparison] -------------------------------------------------- @@ -90,7 +90,7 @@ include-tagged::{sql-specs}/filter.sql-spec[whereFieldOrComparison] [[sql-operators-not]] ==== `NOT` -["source","sql",subs="attributes,callouts,macros"] +[source, sql] -------------------------------------------------- include-tagged::{sql-specs}/filter.sql-spec[whereFieldEqualityNot] -------------------------------------------------- @@ -104,7 +104,7 @@ The result is a value of numeric type. [[sql-operators-plus]] ==== `Add (+)` -["source","sql",subs="attributes,callouts,macros"] +[source, sql] -------------------------------------------------- include-tagged::{sql-specs}/arithmetic.sql-spec[plus] -------------------------------------------------- @@ -112,7 +112,7 @@ include-tagged::{sql-specs}/arithmetic.sql-spec[plus] [[sql-operators-subtract]] ==== `Subtract (infix -)` -["source","sql",subs="attributes,callouts,macros"] +[source, sql] -------------------------------------------------- include-tagged::{sql-specs}/arithmetic.sql-spec[minus] -------------------------------------------------- @@ -120,7 +120,7 @@ include-tagged::{sql-specs}/arithmetic.sql-spec[minus] [[sql-operators-negate]] ==== `Negate (unary -)` -["source","sql",subs="attributes,callouts,macros"] +[source, sql] -------------------------------------------------- include-tagged::{sql-specs}/arithmetic.sql-spec[unaryMinus] -------------------------------------------------- @@ -128,7 +128,7 @@ include-tagged::{sql-specs}/arithmetic.sql-spec[unaryMinus] [[sql-operators-multiply]] ==== `Multiply (*)` -["source","sql",subs="attributes,callouts,macros"] +[source, sql] -------------------------------------------------- include-tagged::{sql-specs}/arithmetic.sql-spec[multiply] -------------------------------------------------- @@ -136,7 +136,7 @@ include-tagged::{sql-specs}/arithmetic.sql-spec[multiply] [[sql-operators-divide]] ==== `Divide (/)` -["source","sql",subs="attributes,callouts,macros"] +[source, sql] -------------------------------------------------- include-tagged::{sql-specs}/arithmetic.sql-spec[divide] -------------------------------------------------- @@ -144,7 +144,7 @@ include-tagged::{sql-specs}/arithmetic.sql-spec[divide] [[sql-operators-remainder]] ==== `Modulo or Remainder(%)` -["source","sql",subs="attributes,callouts,macros"] +[source, sql] -------------------------------------------------- include-tagged::{sql-specs}/arithmetic.sql-spec[mod] -------------------------------------------------- @@ -157,7 +157,7 @@ include-tagged::{sql-specs}/arithmetic.sql-spec[mod] `::` provides an alternative syntax to the <> function. -["source","sql",subs="attributes,callouts,macros"] +[source, sql] -------------------------------------------------- include-tagged::{sql-specs}/docs/docs.csv-spec[conversionStringToLongCastOperator] -------------------------------------------------- diff --git a/docs/reference/sql/functions/search.asciidoc b/docs/reference/sql/functions/search.asciidoc index 6990f6669d69c..34716e070434f 100644 --- a/docs/reference/sql/functions/search.asciidoc +++ b/docs/reference/sql/functions/search.asciidoc @@ -33,7 +33,7 @@ and <> {es} queries. The first parameter is the field or fields to match against. In case it receives one value only, {es-sql} will use a `match` query to perform the search: -["source","sql",subs="attributes,callouts,macros"] +[source, sql] ---- include-tagged::{sql-specs}/docs/docs.csv-spec[simpleMatch] ---- @@ -41,7 +41,7 @@ include-tagged::{sql-specs}/docs/docs.csv-spec[simpleMatch] However, it can also receive a list of fields and their corresponding optional `boost` value. In this case, {es-sql} will use a `multi_match` query to match the documents: -["source","sql",subs="attributes,callouts,macros"] +[source, sql] ---- include-tagged::{sql-specs}/docs/docs.csv-spec[multiFieldsMatch] ---- @@ -53,22 +53,17 @@ the final score than the `author` field when searching for `frank dune` text in Both options above can be used in combination with the optional third parameter of the `MATCH()` predicate, where one can specify additional configuration parameters (separated by semicolon `;`) for either `match` or `multi_match` queries. For example: -["source","sql",subs="attributes,callouts,macros"] +[source, sql] ---- include-tagged::{sql-specs}/docs/docs.csv-spec[optionalParamsForMatch] ---- -In the more advanced example above, the `cutoff_frequency` parameter allows specifying an absolute or relative document frequency where -high frequency terms are moved into an optional subquery and are only scored if one of the low frequency (below the cutoff) terms in the -case of an `or` operator or all of the low frequency terms in the case of an `and` operator match. More about this you can find in the -<> page. - NOTE: The allowed optional parameters for a single-field `MATCH()` variant (for the `match` {es} query) are: `analyzer`, `auto_generate_synonyms_phrase_query`, -`cutoff_frequency`, `lenient`, `fuzziness`, `fuzzy_transpositions`, `fuzzy_rewrite`, `minimum_should_match`, `operator`, +`lenient`, `fuzziness`, `fuzzy_transpositions`, `fuzzy_rewrite`, `minimum_should_match`, `operator`, `max_expansions`, `prefix_length`. NOTE: The allowed optional parameters for a multi-field `MATCH()` variant (for the `multi_match` {es} query) are: `analyzer`, `auto_generate_synonyms_phrase_query`, -`cutoff_frequency`, `lenient`, `fuzziness`, `fuzzy_transpositions`, `fuzzy_rewrite`, `minimum_should_match`, `operator`, +`lenient`, `fuzziness`, `fuzzy_transpositions`, `fuzzy_rewrite`, `minimum_should_match`, `operator`, `max_expansions`, `prefix_length`, `slop`, `tie_breaker`, `type`. @@ -95,14 +90,14 @@ Just like `MATCH`, `QUERY` is a full-text search predicate that gives the user c The first parameter is basically the input that will be passed as is to the `query_string` query, which means that anything that `query_string` accepts in its `query` field can be used here as well: -["source","sql",subs="attributes,callouts,macros"] +[source, sql] ---- include-tagged::{sql-specs}/docs/docs.csv-spec[simpleQueryQuery] ---- A more advanced example, showing more of the features that `query_string` supports, of course possible with {es-sql}: -["source","sql",subs="attributes,callouts,macros"] +[source, sql] ---- include-tagged::{sql-specs}/docs/docs.csv-spec[advancedQueryQuery] ---- @@ -113,7 +108,7 @@ regex and fuzziness queries for the `name` field. If one needs to customize various configuration options that `query_string` exposes, this can be done using the second _optional_ parameter. Multiple settings can be specified separated by a semicolon `;`: -["source","sql",subs="attributes,callouts,macros"] +[source, sql] ---- include-tagged::{sql-specs}/docs/docs.csv-spec[optionalParameterQuery] ---- @@ -149,14 +144,14 @@ combined using the same rules as {es}'s Typically `SCORE` is used for ordering the results of a query based on their relevance: -["source","sql",subs="attributes,callouts,macros"] +[source, sql] ---- include-tagged::{sql-specs}/docs/docs.csv-spec[orderByScore] ---- However, it is perfectly fine to return the score without sorting by it: -["source","sql",subs="attributes,callouts,macros"] +[source, sql] ---- include-tagged::{sql-specs}/docs/docs.csv-spec[scoreWithMatch] ---- diff --git a/docs/reference/sql/functions/string.asciidoc b/docs/reference/sql/functions/string.asciidoc index 7acc358763512..a82ac66adce1c 100644 --- a/docs/reference/sql/functions/string.asciidoc +++ b/docs/reference/sql/functions/string.asciidoc @@ -24,7 +24,7 @@ ASCII(string_exp) <1> Returns the ASCII code value of the leftmost character of `string_exp` as an integer. -["source","sql",subs="attributes,callouts,macros"] +[source, sql] -------------------------------------------------- include-tagged::{sql-specs}/docs/docs.csv-spec[stringAscii] -------------------------------------------------- @@ -47,7 +47,7 @@ BIT_LENGTH(string_exp) <1> Returns the length in bits of the `string_exp` input expression. -["source","sql",subs="attributes,callouts,macros"] +[source, sql] -------------------------------------------------- include-tagged::{sql-specs}/docs/docs.csv-spec[stringBitLength] -------------------------------------------------- @@ -70,7 +70,7 @@ CHAR(code) <1> Returns the character that has the ASCII code value specified by the numeric input. The value should be between 0 and 255; otherwise, the return value is data source–dependent. -["source","sql",subs="attributes,callouts,macros"] +[source, sql] -------------------------------------------------- include-tagged::{sql-specs}/docs/docs.csv-spec[stringChar] -------------------------------------------------- @@ -93,7 +93,7 @@ CHAR_LENGTH(string_exp) <1> Returns the length in characters of the input, if the string expression is of a character data type; otherwise, returns the length in bytes of the string expression (the smallest integer not less than the number of bits divided by 8). -["source","sql",subs="attributes,callouts,macros"] +[source, sql] -------------------------------------------------- include-tagged::{sql-specs}/docs/docs.csv-spec[stringCharLength] -------------------------------------------------- @@ -119,7 +119,7 @@ CONCAT( Returns a character string that is the result of concatenating `string_exp1` to `string_exp2`. If one of the string is `NULL`, the other string will be returned. -["source","sql",subs="attributes,callouts,macros"] +[source, sql] -------------------------------------------------- include-tagged::{sql-specs}/docs/docs.csv-spec[stringConcat] -------------------------------------------------- @@ -149,7 +149,7 @@ INSERT( Returns a string where `length` characters have been deleted from `source`, beginning at `start`, and where `replacement` has been inserted into `source`, beginning at `start`. -["source","sql",subs="attributes,callouts,macros"] +[source, sql] -------------------------------------------------- include-tagged::{sql-specs}/docs/docs.csv-spec[stringInsert] -------------------------------------------------- @@ -172,7 +172,7 @@ LCASE(string_exp) <1> Returns a string equal to that in `string_exp`, with all uppercase characters converted to lowercase. -["source","sql",subs="attributes,callouts,macros"] +[source, sql] -------------------------------------------------- include-tagged::{sql-specs}/docs/docs.csv-spec[stringLCase] -------------------------------------------------- @@ -198,7 +198,7 @@ LEFT( Returns the leftmost count characters of `string_exp`. -["source","sql",subs="attributes,callouts,macros"] +[source, sql] -------------------------------------------------- include-tagged::{sql-specs}/docs/docs.csv-spec[stringLeft] -------------------------------------------------- @@ -221,7 +221,7 @@ LENGTH(string_exp) <1> Returns the number of characters in `string_exp`, excluding trailing blanks. -["source","sql",subs="attributes,callouts,macros"] +[source, sql] -------------------------------------------------- include-tagged::{sql-specs}/docs/docs.csv-spec[stringLength] -------------------------------------------------- @@ -250,12 +250,12 @@ LOCATE( Returns the starting position of the first occurrence of `pattern` within `source`. The search for the first occurrence of `pattern` begins with the first character position in `source` unless the optional argument, `start`, is specified. If `start` is specified, the search begins with the character position indicated by the value of `start`. The first character position in `source` is indicated by the value 1. If `pattern` is not found within `source`, the value 0 is returned. -["source","sql",subs="attributes,callouts,macros"] +[source, sql] -------------------------------------------------- include-tagged::{sql-specs}/docs/docs.csv-spec[stringLocateWoStart] -------------------------------------------------- -["source","sql",subs="attributes,callouts,macros"] +[source, sql] -------------------------------------------------- include-tagged::{sql-specs}/docs/docs.csv-spec[stringLocateWithStart] -------------------------------------------------- @@ -278,7 +278,7 @@ LTRIM(string_exp) <1> Returns the characters of `string_exp`, with leading blanks removed. -["source","sql",subs="attributes,callouts,macros"] +[source, sql] -------------------------------------------------- include-tagged::{sql-specs}/docs/docs.csv-spec[stringLTrim] -------------------------------------------------- @@ -301,7 +301,7 @@ OCTET_LENGTH(string_exp) <1> Returns the length in bytes of the `string_exp` input expression. -["source","sql",subs="attributes,callouts,macros"] +[source, sql] -------------------------------------------------- include-tagged::{sql-specs}/docs/docs.csv-spec[stringOctetLength] -------------------------------------------------- @@ -327,7 +327,7 @@ POSITION( Returns the position of the `string_exp1` in `string_exp2`. The result is an exact numeric. -["source","sql",subs="attributes,callouts,macros"] +[source, sql] -------------------------------------------------- include-tagged::{sql-specs}/docs/docs.csv-spec[stringPosition] -------------------------------------------------- @@ -353,7 +353,7 @@ REPEAT( Returns a character string composed of `string_exp` repeated `count` times. -["source","sql",subs="attributes,callouts,macros"] +[source, sql] -------------------------------------------------- include-tagged::{sql-specs}/docs/docs.csv-spec[stringRepeat] -------------------------------------------------- @@ -381,7 +381,7 @@ REPLACE( Search `source` for occurrences of `pattern`, and replace with `replacement`. -["source","sql",subs="attributes,callouts,macros"] +[source, sql] -------------------------------------------------- include-tagged::{sql-specs}/docs/docs.csv-spec[stringReplace] -------------------------------------------------- @@ -407,7 +407,7 @@ RIGHT( Returns the rightmost count characters of `string_exp`. -["source","sql",subs="attributes,callouts,macros"] +[source, sql] -------------------------------------------------- include-tagged::{sql-specs}/docs/docs.csv-spec[stringRight] -------------------------------------------------- @@ -430,7 +430,7 @@ RTRIM(string_exp) <1> Returns the characters of `string_exp` with trailing blanks removed. -["source","sql",subs="attributes,callouts,macros"] +[source, sql] -------------------------------------------------- include-tagged::{sql-specs}/docs/docs.csv-spec[stringRTrim] -------------------------------------------------- @@ -453,7 +453,7 @@ SPACE(count) <1> Returns a character string consisting of `count` spaces. -["source","sql",subs="attributes,callouts,macros"] +[source, sql] -------------------------------------------------- include-tagged::{sql-specs}/docs/docs.csv-spec[stringSpace] -------------------------------------------------- @@ -481,7 +481,7 @@ SUBSTRING( Returns a character string that is derived from `source`, beginning at the character position specified by `start` for `length` characters. -["source","sql",subs="attributes,callouts,macros"] +[source, sql] -------------------------------------------------- include-tagged::{sql-specs}/docs/docs.csv-spec[stringSubString] -------------------------------------------------- @@ -504,7 +504,7 @@ UCASE(string_exp) <1> Returns a string equal to that of the input, with all lowercase characters converted to uppercase. -["source","sql",subs="attributes,callouts,macros"] +[source, sql] -------------------------------------------------- include-tagged::{sql-specs}/docs/docs.csv-spec[stringUCase] -------------------------------------------------- diff --git a/docs/reference/sql/functions/system.asciidoc b/docs/reference/sql/functions/system.asciidoc index dfca7d526d3a6..b2d604728c165 100644 --- a/docs/reference/sql/functions/system.asciidoc +++ b/docs/reference/sql/functions/system.asciidoc @@ -24,7 +24,7 @@ Returns the name of the database being queried. In the case of Elasticsearch SQL is the name of the Elasticsearch cluster. This function should always return a non-null value. -["source","sql",subs="attributes,callouts,macros"] +[source, sql] -------------------------------------------------- include-tagged::{sql-specs}/docs/docs.csv-spec[database] -------------------------------------------------- @@ -46,7 +46,7 @@ USER() Returns the username of the authenticated user executing the query. This function can return `null` in case {stack-ov}/elasticsearch-security.html[Security] is disabled. -["source","sql",subs="attributes,callouts,macros"] +[source, sql] -------------------------------------------------- include-tagged::{sql-specs}/docs/docs.csv-spec[user] -------------------------------------------------- diff --git a/docs/reference/sql/functions/type-conversion.asciidoc b/docs/reference/sql/functions/type-conversion.asciidoc index 7f8488be40f64..c6c761305519e 100644 --- a/docs/reference/sql/functions/type-conversion.asciidoc +++ b/docs/reference/sql/functions/type-conversion.asciidoc @@ -25,17 +25,17 @@ Casts the result of the given expression to the target <> with slightly differen Moreover, apart from the standard <> it supports the corresponding https://docs.microsoft.com/en-us/sql/odbc/reference/appendixes/explicit-data-type-conversion-function?view=sql-server-2017[ODBC data types]. -["source","sql",subs="attributes,callouts,macros"] +[source, sql] ---- include-tagged::{sql-specs}/docs/docs.csv-spec[conversionStringToIntConvertODBCDataType] ---- -["source","sql",subs="attributes,callouts,macros"] +[source, sql] ---- include-tagged::{sql-specs}/docs/docs.csv-spec[conversionStringToIntConvertESDataType] ---- diff --git a/docs/reference/sql/getting-started.asciidoc b/docs/reference/sql/getting-started.asciidoc index 3ce7b6ebdf3e3..ffd52060e2b0f 100644 --- a/docs/reference/sql/getting-started.asciidoc +++ b/docs/reference/sql/getting-started.asciidoc @@ -40,7 +40,7 @@ Dan Simmons |Hyperion |482 |1989-05-26T00:00:00.000Z Frank Herbert |Dune |604 |1965-06-01T00:00:00.000Z -------------------------------------------------- // TESTRESPONSE[s/\|/\\|/ s/\+/\\+/] -// TESTRESPONSE[_cat] +// TESTRESPONSE[non_json] You can also use the <>. There is a script to start it shipped in x-pack's bin directory: diff --git a/docs/reference/sql/language/data-types.asciidoc b/docs/reference/sql/language/data-types.asciidoc index ad9b2a320c0c6..7c5b4db045f9b 100644 --- a/docs/reference/sql/language/data-types.asciidoc +++ b/docs/reference/sql/language/data-types.asciidoc @@ -1,7 +1,7 @@ [role="xpack"] [testenv="basic"] [[sql-data-types]] -== Data Types +=== Data Types [cols="^,^m,^,^"] @@ -89,7 +89,7 @@ s|SQL precision [[sql-multi-field]] [float] -=== SQL and multi-fields +==== SQL and multi-fields A core concept in {es} is that of an `analyzed` field, that is a full-text value that is interpreted in order to be effectively indexed. These fields are of type <> and are not used for sorting or aggregations as their actual value depends on the <> used hence why {es} also offers the <> type for storing the _exact_ diff --git a/docs/reference/sql/language/indices.asciidoc b/docs/reference/sql/language/indices.asciidoc index 82c7f30fb041e..78834470df5db 100644 --- a/docs/reference/sql/language/indices.asciidoc +++ b/docs/reference/sql/language/indices.asciidoc @@ -1,20 +1,20 @@ [role="xpack"] [testenv="basic"] [[sql-index-patterns]] -== Index patterns +=== Index patterns {es-sql} supports two types of patterns for matching multiple indices or tables: [[sql-index-patterns-multi]] [float] -=== {es} multi-index +==== {es} multi-index The {es} notation for enumerating, including or excluding <> is supported _as long_ as it is quoted or escaped as a table identifier. For example: -["source","sql",subs="attributes,callouts,macros"] +[source, sql] ---- include-tagged::{sql-specs}/docs/docs.csv-spec[showTablesEsMultiIndex] ---- @@ -28,7 +28,7 @@ The same kind of patterns can also be used to query multiple indices or tables. For example: -["source","sql",subs="attributes,callouts,macros"] +[source, sql] ---- include-tagged::{sql-specs}/docs/docs.csv-spec[fromTablePatternQuoted] ---- @@ -37,14 +37,14 @@ NOTE: There is the restriction that all resolved concrete tables have the exact [[sql-index-patterns-like]] [float] -=== SQL `LIKE` notation +==== SQL `LIKE` notation The common `LIKE` statement (including escaping if needed) to match a wildcard pattern, based on one `_` or multiple `%` characters. Using `SHOW TABLES` command again: -["source","sql",subs="attributes,callouts,macros"] +[source, sql] ---- include-tagged::{sql-specs}/docs/docs.csv-spec[showTablesLikeWildcard] ---- @@ -53,7 +53,7 @@ The pattern matches all tables that start with `emp`. This command supports _escaping_ as well, for example: -["source","sql",subs="attributes,callouts,macros"] +[source, sql] ---- include-tagged::{sql-specs}/docs/docs.csv-spec[showTablesLikeEscape] ---- @@ -86,7 +86,7 @@ NOTE: As the query type of quoting between the two patterns is fairly similar (` requires the keyword `LIKE` for SQL `LIKE` pattern. [[sql-index-frozen]] -== Frozen Indices +=== Frozen Indices {es} <> are a useful and powerful tool for hot/warm architecture introduced in {es} 6.6, essentially by trading speed for memory. @@ -101,13 +101,13 @@ Set to `true` properties `index_include_frozen` in the <> or `index.in dedicated keyword:: Explicitly perform the inclusion through the dedicated `FROZEN` keyword in the `FROM` clause or `INCLUDE FROZEN` in the `SHOW` commands: -["source","sql",subs="attributes,callouts,macros"] +[source, sql] ---- include-tagged::{sql-specs}/docs/docs.csv-spec[showTablesIncludeFrozen] ---- -["source","sql",subs="attributes,callouts,macros"] +[source, sql] ---- include-tagged::{sql-specs}/docs/docs.csv-spec[fromTableIncludeFrozen] ---- diff --git a/docs/reference/sql/language/syntax/commands/describe-table.asciidoc b/docs/reference/sql/language/syntax/commands/describe-table.asciidoc index da02f1fa23817..9aad578da479e 100644 --- a/docs/reference/sql/language/syntax/commands/describe-table.asciidoc +++ b/docs/reference/sql/language/syntax/commands/describe-table.asciidoc @@ -30,7 +30,7 @@ DESC `DESC` and `DESCRIBE` are aliases to <>. -["source","sql",subs="attributes,callouts,macros"] +[source, sql] ---- include-tagged::{sql-specs}/docs/docs.csv-spec[describeTable] ---- diff --git a/docs/reference/sql/language/syntax/commands/index.asciidoc b/docs/reference/sql/language/syntax/commands/index.asciidoc index 4af8f19d7034b..78259d07914a8 100644 --- a/docs/reference/sql/language/syntax/commands/index.asciidoc +++ b/docs/reference/sql/language/syntax/commands/index.asciidoc @@ -1,7 +1,7 @@ [role="xpack"] [testenv="basic"] [[sql-commands]] -== SQL Commands +=== SQL Commands This section contains the list of SQL commands supported by {es-sql} along with their syntax: diff --git a/docs/reference/sql/language/syntax/commands/select.asciidoc b/docs/reference/sql/language/syntax/commands/select.asciidoc index 08ebe0ae96497..5fe9769774cef 100644 --- a/docs/reference/sql/language/syntax/commands/select.asciidoc +++ b/docs/reference/sql/language/syntax/commands/select.asciidoc @@ -36,7 +36,7 @@ The general execution of `SELECT` is as follows: As with a table, every output column of a `SELECT` has a name which can be either specified per column through the `AS` keyword : -["source","sql",subs="attributes,callouts,macros"] +[source, sql] ---- include-tagged::{sql-specs}/docs/docs.csv-spec[selectColumnAlias] ---- @@ -46,14 +46,14 @@ which is why it is recommended to specify it. assigned by {es-sql} if no name is given: -["source","sql",subs="attributes,callouts,macros"] +[source, sql] ---- include-tagged::{sql-specs}/docs/docs.csv-spec[selectInline] ---- or if it's a simple column reference, use its name as the column name: -["source","sql",subs="attributes,callouts,macros"] +[source, sql] ---- include-tagged::{sql-specs}/docs/docs.csv-spec[selectColumn] ---- @@ -63,7 +63,7 @@ include-tagged::{sql-specs}/docs/docs.csv-spec[selectColumn] To select all the columns in the source, one can use `*`: -["source","sql",subs="attributes,callouts,macros"] +[source, sql] ---- include-tagged::{sql-specs}/docs/docs.csv-spec[wildcardWithOrder] ---- @@ -89,14 +89,14 @@ Represents the name (optionally qualified) of an existing table, either a concre If the table name contains special SQL characters (such as `.`,`-`,`*`,etc...) use double quotes to escape them: -["source","sql",subs="attributes,callouts,macros"] +[source, sql] ---- include-tagged::{sql-specs}/docs/docs.csv-spec[fromTableQuoted] ---- The name can be a <> pointing to multiple indices (likely requiring quoting as mentioned above) with the restriction that *all* resolved concrete tables have **exact mapping**. -["source","sql",subs="attributes,callouts,macros"] +[source, sql] ---- include-tagged::{sql-specs}/docs/docs.csv-spec[fromTablePatternQuoted] ---- @@ -104,7 +104,7 @@ include-tagged::{sql-specs}/docs/docs.csv-spec[fromTablePatternQuoted] `alias`:: A substitute name for the `FROM` item containing the alias. An alias is used for brevity or to eliminate ambiguity. When an alias is provided, it completely hides the actual name of the table and must be used in its place. -["source","sql",subs="attributes,callouts,macros"] +[source, sql] ---- include-tagged::{sql-specs}/docs/docs.csv-spec[fromTableAlias] ---- @@ -125,7 +125,7 @@ where: Represents an expression that evaluates to a `boolean`. Only the rows that match the condition (to `true`) are returned. -["source","sql",subs="attributes,callouts,macros"] +[source, sql] ---- include-tagged::{sql-specs}/docs/docs.csv-spec[basicWhere] ---- @@ -148,34 +148,34 @@ Represents an expression on which rows are being grouped _on_. It can be a colum A common, group by column name: -["source","sql",subs="attributes,callouts,macros"] +[source, sql] ---- include-tagged::{sql-specs}/docs/docs.csv-spec[groupByColumn] ---- Grouping by output ordinal: -["source","sql",subs="attributes,callouts,macros"] +[source, sql] ---- include-tagged::{sql-specs}/docs/docs.csv-spec[groupByOrdinal] ---- Grouping by alias: -["source","sql",subs="attributes,callouts,macros"] +[source, sql] ---- include-tagged::{sql-specs}/docs/docs.csv-spec[groupByAlias] ---- And grouping by column expression (typically used along-side an alias): -["source","sql",subs="attributes,callouts,macros"] +[source, sql] ---- include-tagged::{sql-specs}/docs/docs.csv-spec[groupByExpression] ---- Or a mixture of the above: -["source","sql",subs="attributes,callouts,macros"] +[source, sql] ---- include-tagged::{sql-specs}/docs/docs.csv-spec[groupByMulti] ---- @@ -185,27 +185,27 @@ When a `GROUP BY` clause is used in a `SELECT`, _all_ output expressions must be To wit: -["source","sql",subs="attributes,callouts,macros"] +[source, sql] ---- include-tagged::{sql-specs}/docs/docs.csv-spec[groupByAndAgg] ---- Expressions over aggregates used in output: -["source","sql",subs="attributes,callouts,macros"] +[source, sql] ---- include-tagged::{sql-specs}/docs/docs.csv-spec[groupByAndAggExpression] ---- Multiple aggregates used: -["source","sql",subs="attributes,callouts,macros"] +[source, sql] ---- include-tagged::{sql-specs}/docs/docs.csv-spec[groupByAndMultipleAggs] ---- [TIP] -If custom bucketing is required, it can be achieved with the use of `<>`, +If custom bucketing is required, it can be achieved with the use of <>, as shown <>. [[sql-syntax-group-by-implicit]] @@ -216,14 +216,14 @@ As such, the query emits only a single row (as there is only a single group). A common example is counting the number of records: -["source","sql",subs="attributes,callouts,macros"] +[source, sql] ---- include-tagged::{sql-specs}/docs/docs.csv-spec[groupByImplicitCount] ---- Of course, multiple aggregations can be applied: -["source","sql",subs="attributes,callouts,macros"] +[source, sql] ---- include-tagged::{sql-specs}/docs/docs.csv-spec[groupByImplicitMultipleAggs] ---- @@ -249,14 +249,14 @@ Both `WHERE` and `HAVING` are used for filtering however there are several signi . `WHERE` works on individual *rows*, `HAVING` works on the *groups* created by ``GROUP BY`` . `WHERE` is evaluated *before* grouping, `HAVING` is evaluated *after* grouping -["source","sql",subs="attributes,callouts,macros"] +[source, sql] ---- include-tagged::{sql-specs}/docs/docs.csv-spec[groupByHaving] ---- Further more, one can use multiple aggregate expressions inside `HAVING` even ones that are not used in the output (`SELECT`): -["source","sql",subs="attributes,callouts,macros"] +[source, sql] ---- include-tagged::{sql-specs}/docs/docs.csv-spec[groupByHavingMultiple] ---- @@ -269,14 +269,14 @@ As such, the query emits only a single row (as there is only a single group) and In this example, `HAVING` matches: -["source","sql",subs="attributes,callouts,macros"] +[source, sql] ---- include-tagged::{sql-specs}/docs/docs.csv-spec[groupByHavingImplicitMatch] ---- //However `HAVING` can also not match, in which case an empty result is returned: // -//["source","sql",subs="attributes,callouts,macros"] +//[source, sql] //---- //include-tagged::{sql-specs}/docs/docs.csv-spec[groupByHavingImplicitNoMatch] //---- @@ -304,7 +304,7 @@ IMPORTANT: When used along-side, `GROUP BY` expression can point _only_ to the c For example, the following query sorts by an arbitrary input field (`page_count`): -["source","sql",subs="attributes,callouts,macros"] +[source, sql] ---- include-tagged::{sql-specs}/docs/docs.csv-spec[orderByBasic] ---- @@ -318,26 +318,26 @@ NOTE: With `GROUP BY`, make sure the ordering targets the resulting group - appl For example, to order groups simply indicate the grouping key: -["source","sql",subs="attributes,callouts,macros"] +[source, sql] ---- include-tagged::{sql-specs}/docs/docs.csv-spec[orderByGroup] ---- Multiple keys can be specified of course: -["source","sql",subs="attributes,callouts,macros"] +[source, sql] ---- include-tagged::{sql-specs}/docs/docs.csv-spec[groupByMulti] ---- Further more, it is possible to order groups based on aggregations of their values: -["source","sql",subs="attributes,callouts,macros"] +[source, sql] ---- include-tagged::{sql-specs}/docs/docs.csv-spec[orderByAgg] ---- -IMPORTANT: Ordering by aggregation is possible for up to 512 entries for memory consumption reasons. -In cases where the results pass this threshold, use <<`LIMIT`, sql-syntax-limit>> to reduce the number +IMPORTANT: Ordering by aggregation is possible for up to *10000* entries for memory consumption reasons. +In cases where the results pass this threshold, use <> to reduce the number of results. [[sql-syntax-order-by-score]] @@ -352,7 +352,7 @@ combined using the same rules as {es}'s To sort based on the `score`, use the special function `SCORE()`: -["source","sql",subs="attributes,callouts,macros"] +[source, sql] ---- include-tagged::{sql-specs}/docs/docs.csv-spec[orderByScore] ---- @@ -360,7 +360,7 @@ include-tagged::{sql-specs}/docs/docs.csv-spec[orderByScore] Note that you can return `SCORE()` by using a full-text search predicate in the `WHERE` clause. This is possible even if `SCORE()` is not used for sorting: -["source","sql",subs="attributes,callouts,macros"] +[source, sql] ---- include-tagged::{sql-specs}/docs/docs.csv-spec[orderByScoreWithMatch] ---- @@ -387,7 +387,7 @@ ALL:: indicates there is no limit and thus all results are being returned. To return -["source","sql",subs="attributes,callouts,macros"] +[source, sql] ---- include-tagged::{sql-specs}/docs/docs.csv-spec[limitBasic] ---- diff --git a/docs/reference/sql/language/syntax/commands/show-columns.asciidoc b/docs/reference/sql/language/syntax/commands/show-columns.asciidoc index b21c02358e526..9cb90af6b656f 100644 --- a/docs/reference/sql/language/syntax/commands/show-columns.asciidoc +++ b/docs/reference/sql/language/syntax/commands/show-columns.asciidoc @@ -21,7 +21,7 @@ patterns. List the columns in table and their data type (and other attributes). -["source","sql",subs="attributes,callouts,macros"] +[source, sql] ---- include-tagged::{sql-specs}/docs/docs.csv-spec[showColumns] ---- diff --git a/docs/reference/sql/language/syntax/commands/show-functions.asciidoc b/docs/reference/sql/language/syntax/commands/show-functions.asciidoc index 47c000e81d9fd..8689788867c20 100644 --- a/docs/reference/sql/language/syntax/commands/show-functions.asciidoc +++ b/docs/reference/sql/language/syntax/commands/show-functions.asciidoc @@ -15,7 +15,7 @@ SHOW FUNCTIONS [LIKE pattern?]? <1> List all the SQL functions and their type. The `LIKE` clause can be used to restrict the list of names to the given pattern. -["source","sql",subs="attributes,callouts,macros"] +[source, sql] ---- include-tagged::{sql-specs}/docs/docs.csv-spec[showFunctions] ---- @@ -23,25 +23,25 @@ include-tagged::{sql-specs}/docs/docs.csv-spec[showFunctions] The list of functions returned can be customized based on the pattern. It can be an exact match: -["source","sql",subs="attributes,callouts,macros"] +[source, sql] ---- include-tagged::{sql-specs}/docs/docs.csv-spec[showFunctionsLikeExact] ---- A wildcard for exactly one character: -["source","sql",subs="attributes,callouts,macros"] +[source, sql] ---- include-tagged::{sql-specs}/docs/docs.csv-spec[showFunctionsLikeChar] ---- A wildcard matching zero or more characters: -["source","sql",subs="attributes,callouts,macros"] +[source, sql] ---- include-tagged::{sql-specs}/docs/docs.csv-spec[showFunctionsLikeWildcard] ---- Or of course, a variation of the above: -["source","sql",subs="attributes,callouts,macros"] +[source, sql] ---- include-tagged::{sql-specs}/docs/docs.csv-spec[showFunctionsWithPattern] ---- diff --git a/docs/reference/sql/language/syntax/commands/show-tables.asciidoc b/docs/reference/sql/language/syntax/commands/show-tables.asciidoc index 554819e24b178..d5a40337713d4 100644 --- a/docs/reference/sql/language/syntax/commands/show-tables.asciidoc +++ b/docs/reference/sql/language/syntax/commands/show-tables.asciidoc @@ -24,7 +24,7 @@ patterns. List the tables available to the current user and their type. -["source","sql",subs="attributes,callouts,macros"] +[source, sql] ---- include-tagged::{sql-specs}/docs/docs.csv-spec[showTables] ---- @@ -32,7 +32,7 @@ include-tagged::{sql-specs}/docs/docs.csv-spec[showTables] Match multiple indices by using {es} <> notation: -["source","sql",subs="attributes,callouts,macros"] +[source, sql] ---- include-tagged::{sql-specs}/docs/docs.csv-spec[showTablesEsMultiIndex] ---- @@ -40,26 +40,26 @@ include-tagged::{sql-specs}/docs/docs.csv-spec[showTablesEsMultiIndex] One can also use the `LIKE` clause to restrict the list of names to the given pattern. The pattern can be an exact match: -["source","sql",subs="attributes,callouts,macros"] +[source, sql] ---- include-tagged::{sql-specs}/docs/docs.csv-spec[showTablesLikeExact] ---- Multiple chars: -["source","sql",subs="attributes,callouts,macros"] +[source, sql] ---- include-tagged::{sql-specs}/docs/docs.csv-spec[showTablesLikeWildcard] ---- A single char: -["source","sql",subs="attributes,callouts,macros"] +[source, sql] ---- include-tagged::{sql-specs}/docs/docs.csv-spec[showTablesLikeOneChar] ---- Or a mixture of single and multiple chars: -["source","sql",subs="attributes,callouts,macros"] +[source, sql] ---- include-tagged::{sql-specs}/docs/docs.csv-spec[showTablesLikeMixed] ---- diff --git a/docs/reference/sql/language/syntax/lexic/index.asciidoc b/docs/reference/sql/language/syntax/lexic/index.asciidoc index 014193a0469dd..a668ee724e56f 100644 --- a/docs/reference/sql/language/syntax/lexic/index.asciidoc +++ b/docs/reference/sql/language/syntax/lexic/index.asciidoc @@ -1,7 +1,7 @@ [role="xpack"] [testenv="basic"] [[sql-lexical-structure]] -== Lexical Structure +=== Lexical Structure This section covers the major lexical structure of SQL, which for the most part, is going to resemble that of ANSI SQL itself hence why low-levels details are not discussed in depth. @@ -10,8 +10,7 @@ This section covers the major lexical structure of SQL, which for the most part, A token can be a __key word__, an _identifier_ (_quoted_ or _unquoted_), a _literal_ (or constant) or a special character symbol (typically a delimiter). Tokens are typically separated by whitespace (be it space, tab) though in some cases, where there is no ambiguity (typically due to a character symbol) this is not needed - however for readability purposes this should be avoided. [[sql-syntax-keywords]] -[float] -=== Key Words +==== Key Words Take the following example: @@ -35,8 +34,7 @@ Identifiers however are not - as {es} is case sensitive, {es-sql} uses the recei To help differentiate between the two, through-out the documentation the SQL key words are upper-cased a convention we find increases readability and thus recommend to others. [[sql-syntax-identifiers]] -[float] -=== Identifiers +==== Identifiers Identifiers can be of two types: __quoted__ and __unquoted__: @@ -59,14 +57,13 @@ The first identifier from needs to quoted as otherwise it clashes with the `FROM Hence why in general, *especially* when dealing with user input it is *highly* recommended to use quotes for identifiers. It adds minimal increase to your queries and in return offers clarity and disambiguation. [[sql-syntax-literals]] -[float] -=== Literals (Constants) +==== Literals (Constants) {es-sql} supports two kind of __implicitly-typed__ literals: strings and numbers. [[sql-syntax-string-literals]] [float] -==== String Literals +===== String Literals A string literal is an arbitrary number of characters bounded by single quotes `'`: `'Giant Robot'`. To include a single quote in the string, escape it using another single quote: `'Captain EO''s Voyage'`. @@ -75,7 +72,7 @@ NOTE: An escaped single quote is *not* a double quote (`"`), but a single quote [sql-syntax-numeric-literals] [float] -==== Numeric Literals +===== Numeric Literals Numeric literals are accepted both in decimal and scientific notation with exponent marker (`e` or `E`), starting either with a digit or decimal point `.`: @@ -92,7 +89,7 @@ Numeric literals that contain a decimal point are always interpreted as being of [[sql-syntax-generic-literals]] [float] -==== Generic Literals +===== Generic Literals When dealing with arbitrary type literal, one creates the object by casting, typically, the string representation to the desired type. This can be achieved through the dedicated <> and <>: @@ -106,8 +103,7 @@ CONVERT('10.0.0.1', IP) -- cast '10.0.0.1' to an IP Do note that {es-sql} provides functions that out of the box return popular literals (like `E()`) or provide dedicated parsing for certain strings. [[sql-syntax-single-vs-double-quotes]] -[float] -=== Single vs Double Quotes +==== Single vs Double Quotes It is worth pointing out that in SQL, single quotes `'` and double quotes `"` have different meaning and *cannot* be used interchangeably. Single quotes are used to declare a <> while double quotes for <>. @@ -126,8 +122,7 @@ SELECT "first_name" <1> <2> Single quotes `'` used for a string literal [[sql-syntax-special-chars]] -[float] -=== Special characters +==== Special characters A few characters that are not alphanumeric have a dedicated meaning different from that of an operator. For completeness these are specified below: @@ -146,8 +141,7 @@ s|Description |=== [[sql-syntax-operators]] -[float] -=== Operators +==== Operators Most operators in {es-sql} have the same precedence and are left-associative. As this is done at parsing time, parenthesis need to be used to enforce a different precedence. @@ -205,8 +199,7 @@ s|Description [[sql-syntax-comments]] -[float] -=== Comments +==== Comments {es-sql} allows comments which are sequence of characters ignored by the parsers. diff --git a/docs/reference/sql/limitations.asciidoc b/docs/reference/sql/limitations.asciidoc index c5b334480c993..2d46d016c6b89 100644 --- a/docs/reference/sql/limitations.asciidoc +++ b/docs/reference/sql/limitations.asciidoc @@ -87,8 +87,8 @@ It is recommended to use `LIMIT` for queries that use sorting by aggregation, es SELECT * FROM test GROUP BY age ORDER BY COUNT(*) LIMIT 100; -------------------------------------------------- -It is possible to run the same queries without a `LIMIT` however in that case if the maximum size (*512*) is passed, an exception will be -returned as {es-sql} is unable to track (and sort) all the results returned. +It is possible to run the same queries without a `LIMIT` however in that case if the maximum size (*10000*) is passed, +an exception will be returned as {es-sql} is unable to track (and sort) all the results returned. [float] === Using aggregation functions on top of scalar functions diff --git a/docs/reference/sql/security.asciidoc b/docs/reference/sql/security.asciidoc index ad946c33e2d98..cbf41b46997a4 100644 --- a/docs/reference/sql/security.asciidoc +++ b/docs/reference/sql/security.asciidoc @@ -33,7 +33,7 @@ the API require `cluster:monitor/main`. The following example configures a role that can run SQL in JDBC querying the `test` and `bort` indices: -["source","yaml",subs="attributes,callouts,macros"] +[source, yaml] -------------------------------------------------- include-tagged::{sql-tests}/security/roles.yml[cli_drivers] -------------------------------------------------- diff --git a/docs/reference/upgrade.asciidoc b/docs/reference/upgrade.asciidoc index e5e447aff75f6..3bd35197dcd5f 100644 --- a/docs/reference/upgrade.asciidoc +++ b/docs/reference/upgrade.asciidoc @@ -7,8 +7,8 @@ process so upgrading does not interrupt service. Rolling upgrades are supported: * Between minor versions -* From 5.6 to 6.7 -* From 6.7 to {version} +* From 5.6 to 6.8 +* From 6.8 to {version} {es} can read indices created in the previous major version. If you have indices created in 5.x or before, you must reindex or delete them diff --git a/docs/reference/upgrade/cluster_restart.asciidoc b/docs/reference/upgrade/cluster_restart.asciidoc index 1865f005dccc8..52b14d642b620 100644 --- a/docs/reference/upgrade/cluster_restart.asciidoc +++ b/docs/reference/upgrade/cluster_restart.asciidoc @@ -5,7 +5,7 @@ To upgrade directly to {es} {version} from versions 6.0-6.6, you must shut down all nodes in the cluster, upgrade each node to {version}, and restart the cluster. NOTE: If you are running a version prior to 6.0, -https://www.elastic.co/guide/en/elastic-stack/6.7/upgrading-elastic-stack.html[upgrade to 6.7] +{stack-ref-68}/upgrading-elastic-stack.html[upgrade to 6.8] and reindex your old indices or bring up a new {version} cluster and <>. diff --git a/docs/reference/upgrade/reindex_upgrade.asciidoc b/docs/reference/upgrade/reindex_upgrade.asciidoc index 7fc6c320263c3..faa8fbc1639b4 100644 --- a/docs/reference/upgrade/reindex_upgrade.asciidoc +++ b/docs/reference/upgrade/reindex_upgrade.asciidoc @@ -36,7 +36,7 @@ been deleted. [[reindex-upgrade-inplace]] === Reindex in place -You can use the Upgrade Assistant in {kib} 6.7 to automatically reindex 5.x +You can use the Upgrade Assistant in {kib} 6.8 to automatically reindex 5.x indices you need to carry forward to {version}. To manually reindex your old indices in place: @@ -103,7 +103,7 @@ endif::include-xpack[] You can use <> to migrate indices from your old cluster to a new {version} cluster. This enables you move to {version} -from a pre-6.7 cluster without interrupting service. +from a pre-6.8 cluster without interrupting service. [WARNING] ============================================= @@ -196,4 +196,4 @@ monitor progress of the reindex job with the <>: `30s` and `1`). .. Once reindexing is complete and the status of the new index is `green`, - you can delete the old index. \ No newline at end of file + you can delete the old index. diff --git a/docs/reference/upgrade/rolling_upgrade.asciidoc b/docs/reference/upgrade/rolling_upgrade.asciidoc index 2cf1061e67ba7..7ad4a0bb904c1 100644 --- a/docs/reference/upgrade/rolling_upgrade.asciidoc +++ b/docs/reference/upgrade/rolling_upgrade.asciidoc @@ -10,8 +10,8 @@ running the older version. Rolling upgrades are supported: * Between minor versions -* {stack-ref-67}/upgrading-elastic-stack.html[From 5.6 to 6.7] -* {stack-ref-70}/upgrading-elastic-stack.html[From 6.7 to 7.0] +* {stack-ref-68}/upgrading-elastic-stack.html[From 5.6 to 6.8] +* {stack-ref-70}/upgrading-elastic-stack.html[From 6.8 to 7.0] * From {prev-major-version} to {version} Upgrading directly to {version} from 6.6 or earlier requires a diff --git a/gradle.properties b/gradle.properties index 491770edd7c52..63b1dc3cd7288 100644 --- a/gradle.properties +++ b/gradle.properties @@ -1,3 +1,3 @@ org.gradle.daemon=true -org.gradle.jvmargs=-Xmx2g -XX:+HeapDumpOnOutOfMemoryError -Xss2m +org.gradle.jvmargs=-Xmx3g -XX:+HeapDumpOnOutOfMemoryError -Xss2m options.forkOptions.memoryMaximumSize=2g diff --git a/gradle/wrapper/gradle-wrapper.properties b/gradle/wrapper/gradle-wrapper.properties index 47216b872e431..9fabbb9bd78d9 100644 --- a/gradle/wrapper/gradle-wrapper.properties +++ b/gradle/wrapper/gradle-wrapper.properties @@ -1,6 +1,6 @@ distributionBase=GRADLE_USER_HOME distributionPath=wrapper/dists -distributionUrl=https\://services.gradle.org/distributions/gradle-5.4.1-all.zip +distributionUrl=https\://services.gradle.org/distributions/gradle-5.5-all.zip zipStoreBase=GRADLE_USER_HOME zipStorePath=wrapper/dists -distributionSha256Sum=14cd15fc8cc8705bd69dcfa3c8fefb27eb7027f5de4b47a8b279218f76895a91 +distributionSha256Sum=302b7df46730ce75c582542c056c9bf5cac2b94fbf2cc656d0e37e41e8a5d371 diff --git a/gradlew b/gradlew index b0d6d0ab5deb5..8e25e6c19d574 100755 --- a/gradlew +++ b/gradlew @@ -7,7 +7,7 @@ # 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 +# https://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, diff --git a/gradlew.bat b/gradlew.bat index 15e1ee37a70d7..24467a141f791 100644 --- a/gradlew.bat +++ b/gradlew.bat @@ -5,7 +5,7 @@ @rem you may not use this file except in compliance with the License. @rem You may obtain a copy of the License at @rem -@rem http://www.apache.org/licenses/LICENSE-2.0 +@rem https://www.apache.org/licenses/LICENSE-2.0 @rem @rem Unless required by applicable law or agreed to in writing, software @rem distributed under the License is distributed on an "AS IS" BASIS, diff --git a/libs/build.gradle b/libs/build.gradle index b0924aa1f54f1..03b5d2c611e71 100644 --- a/libs/build.gradle +++ b/libs/build.gradle @@ -26,19 +26,19 @@ subprojects { /* * Subprojects may depend on the "core" lib but may not depend on any - * other libs. This keeps are dependencies simpler. + * other libs. This keeps our dependencies simpler. */ project.afterEvaluate { configurations.all { Configuration conf -> - dependencies.all { Dependency dep -> - Project depProject = dependencyToProject(dep) + dependencies.matching { it instanceof ProjectDependency }.all { ProjectDependency dep -> + Project depProject = dep.dependencyProject if (depProject != null - && false == depProject.path.equals(':libs:core') + && false == depProject.path.equals(':libs:elasticsearch-core') && false == isEclipse && depProject.path.startsWith(':libs')) { throw new InvalidUserDataException("projects in :libs " + "may not depend on other projects libs except " - + ":libs:core but " + + ":libs:elasticsearch-core but " + "${project.path} depends on ${depProject.path}") } } diff --git a/libs/cli/build.gradle b/libs/cli/build.gradle index b1f3b338255c4..a97c62096a512 100644 --- a/libs/cli/build.gradle +++ b/libs/cli/build.gradle @@ -23,7 +23,7 @@ apply plugin: 'nebula.maven-scm' dependencies { compile 'net.sf.jopt-simple:jopt-simple:5.0.2' - compile "org.elasticsearch:elasticsearch-core:${version}" + compile project(':libs:elasticsearch-core') } test.enabled = false diff --git a/libs/core/build.gradle b/libs/core/build.gradle index 86778c3d23744..b67b213d7b41c 100644 --- a/libs/core/build.gradle +++ b/libs/core/build.gradle @@ -21,30 +21,20 @@ apply plugin: 'nebula.optional-base' apply plugin: 'nebula.maven-base-publish' apply plugin: 'nebula.maven-scm' -archivesBaseName = 'elasticsearch-core' - -publishing { - publications { - nebula { - artifactId = archivesBaseName - } - } -} - dependencies { testCompile "com.carrotsearch.randomizedtesting:randomizedtesting-runner:${versions.randomizedrunner}" testCompile "junit:junit:${versions.junit}" testCompile "org.hamcrest:hamcrest:${versions.hamcrest}" - if (isEclipse == false || project.path == ":libs:core-tests") { - testCompile("org.elasticsearch.test:framework:${version}") { + if (isEclipse == false || project.path == ":libs:elasticsearch-core-tests") { + testCompile(project(":test:framework")) { exclude group: 'org.elasticsearch', module: 'elasticsearch-core' } } } forbiddenApisMain { - // :libs:core does not depend on server + // :libs:elasticsearch-core does not depend on server // TODO: Need to decide how we want to handle for forbidden signatures with the changes to server replaceSignatureFiles 'jdk-signatures' } @@ -52,7 +42,7 @@ forbiddenApisMain { if (isEclipse) { // in eclipse the project is under a fake root, we need to change around the source sets sourceSets { - if (project.path == ":libs:core") { + if (project.path == ":libs:elasticsearch-core") { main.java.srcDirs = ['java'] main.resources.srcDirs = ['resources'] } else { diff --git a/libs/core/src/main/java/org/elasticsearch/common/unit/TimeValue.java b/libs/core/src/main/java/org/elasticsearch/common/unit/TimeValue.java index f40fbbe73a9dc..c208e7d795391 100644 --- a/libs/core/src/main/java/org/elasticsearch/common/unit/TimeValue.java +++ b/libs/core/src/main/java/org/elasticsearch/common/unit/TimeValue.java @@ -31,6 +31,26 @@ public class TimeValue implements Comparable { public static final TimeValue MINUS_ONE = timeValueMillis(-1); public static final TimeValue ZERO = timeValueMillis(0); + private static final long C0 = 1L; + private static final long C1 = C0 * 1000L; + private static final long C2 = C1 * 1000L; + private static final long C3 = C2 * 1000L; + private static final long C4 = C3 * 60L; + private static final long C5 = C4 * 60L; + private static final long C6 = C5 * 24L; + + private final long duration; + private final TimeUnit timeUnit; + + public TimeValue(long millis) { + this(millis, TimeUnit.MILLISECONDS); + } + + public TimeValue(long duration, TimeUnit timeUnit) { + this.duration = duration; + this.timeUnit = timeUnit; + } + public static TimeValue timeValueNanos(long nanos) { return new TimeValue(nanos, TimeUnit.NANOSECONDS); } @@ -51,17 +71,6 @@ public static TimeValue timeValueHours(long hours) { return new TimeValue(hours, TimeUnit.HOURS); } - private final long duration; - - /** - * @return the number of {@link #timeUnit()} units this value contains - */ - public long duration() { - return duration; - } - - private final TimeUnit timeUnit; - /** * @return the unit used for the this time value, see {@link #duration()} */ @@ -69,13 +78,11 @@ public TimeUnit timeUnit() { return timeUnit; } - public TimeValue(long millis) { - this(millis, TimeUnit.MILLISECONDS); - } - - public TimeValue(long duration, TimeUnit timeUnit) { - this.duration = duration; - this.timeUnit = timeUnit; + /** + * @return the number of {@link #timeUnit()} units this value contains + */ + public long duration() { + return duration; } public long nanos() { @@ -187,9 +194,31 @@ public double getDaysFrac() { * * Note that this method might produce fractional time values (ex 1.6m) which cannot be * parsed by method like {@link TimeValue#parse(String, String, String)}. + * + * Also note that the maximum string value that will be generated is + * {@code 106751.9d} due to the way that values are internally converted + * to nanoseconds (106751.9 days is Long.MAX_VALUE nanoseconds) */ @Override public String toString() { + return this.toHumanReadableString(1); + } + + /** + * Returns a {@link String} representation of the current {@link TimeValue}. + * + * Note that this method might produce fractional time values (ex 1.6m) which cannot be + * parsed by method like {@link TimeValue#parse(String, String, String)}. The number of + * fractional decimals (up to 10 maximum) are truncated to the number of fraction pieces + * specified. + * + * Also note that the maximum string value that will be generated is + * {@code 106751.9d} due to the way that values are internally converted + * to nanoseconds (106751.9 days is Long.MAX_VALUE nanoseconds) + * + * @param fractionPieces the number of decimal places to include + */ + public String toHumanReadableString(int fractionPieces) { if (duration < 0) { return Long.toString(duration); } @@ -218,25 +247,68 @@ public String toString() { value = microsFrac(); suffix = "micros"; } - return formatDecimal(value) + suffix; + // Limit fraction pieces to a min of 0 and maximum of 10 + return formatDecimal(value, Math.min(10, Math.max(0, fractionPieces))) + suffix; } - private static String formatDecimal(double value) { + private static String formatDecimal(double value, int fractionPieces) { String p = String.valueOf(value); + int totalLength = p.length(); int ix = p.indexOf('.') + 1; int ex = p.indexOf('E'); - char fraction = p.charAt(ix); - if (fraction == '0') { + // Location where the fractional values end + int fractionEnd = ex == -1 ? Math.min(ix + fractionPieces, totalLength) : ex; + + // Determine the value of the fraction, so if it were .000 the + // actual long value is 0, in which case, it can be elided. + long fractionValue; + try { + fractionValue = Long.parseLong(p.substring(ix, fractionEnd)); + } catch (NumberFormatException e) { + fractionValue = 0; + } + + if (fractionValue == 0 || fractionPieces <= 0) { + // Either the part of the fraction we were asked to report is + // zero, or the user requested 0 fraction pieces, so return + // only the integral value if (ex != -1) { return p.substring(0, ix - 1) + p.substring(ex); } else { return p.substring(0, ix - 1); } } else { + // Build up an array of fraction characters, without going past + // the end of the string. This keeps track of trailing '0' chars + // that should be truncated from the end to avoid getting a + // string like "1.3000d" (returning "1.3d" instead) when the + // value is 1.30000009 + char[] fractions = new char[fractionPieces]; + int fracCount = 0; + int truncateCount = 0; + for (int i = 0; i < fractionPieces; i++) { + int position = ix + i; + if (position >= fractionEnd) { + // No more pieces, the fraction has ended + break; + } + char fraction = p.charAt(position); + if (fraction == '0') { + truncateCount++; + } else { + truncateCount = 0; + } + fractions[i] = fraction; + fracCount++; + } + + // Generate the fraction string from the char array, truncating any trailing zeros + String fractionStr = new String(fractions, 0, fracCount - truncateCount); + if (ex != -1) { - return p.substring(0, ix) + fraction + p.substring(ex); + return p.substring(0, ix) + fractionStr + p.substring(ex); } else { - return p.substring(0, ix) + fraction; + return p.substring(0, ix) + fractionStr; } } } @@ -317,14 +389,6 @@ private static long parse(final String initialInput, final String normalized, fi } } - private static final long C0 = 1L; - private static final long C1 = C0 * 1000L; - private static final long C2 = C1 * 1000L; - private static final long C3 = C2 * 1000L; - private static final long C4 = C3 * 60L; - private static final long C5 = C4 * 60L; - private static final long C6 = C5 * 24L; - @Override public boolean equals(Object o) { if (this == o) return true; diff --git a/libs/core/src/main/java/org/elasticsearch/core/internal/io/IOUtils.java b/libs/core/src/main/java/org/elasticsearch/core/internal/io/IOUtils.java index 46d19d2a814fe..d3e9afd4970df 100644 --- a/libs/core/src/main/java/org/elasticsearch/core/internal/io/IOUtils.java +++ b/libs/core/src/main/java/org/elasticsearch/core/internal/io/IOUtils.java @@ -24,6 +24,7 @@ import java.nio.file.FileVisitResult; import java.nio.file.FileVisitor; import java.nio.file.Files; +import java.nio.file.NoSuchFileException; import java.nio.file.Path; import java.nio.file.StandardOpenOption; import java.nio.file.attribute.BasicFileAttributes; @@ -249,6 +250,7 @@ public FileVisitResult visitFileFailed(final Path file, final IOException exc) t } // TODO: replace with constants class if needed (cf. org.apache.lucene.util.Constants) + private static final boolean WINDOWS = System.getProperty("os.name").startsWith("Windows"); private static final boolean LINUX = System.getProperty("os.name").startsWith("Linux"); private static final boolean MAC_OS_X = System.getProperty("os.name").startsWith("Mac OS X"); @@ -263,18 +265,28 @@ public FileVisitResult visitFileFailed(final Path file, final IOException exc) t * systems and operating systems allow to fsync on a directory) */ public static void fsync(final Path fileToSync, final boolean isDir) throws IOException { + if (isDir && WINDOWS) { + // opening a directory on Windows fails, directories can not be fsynced there + if (Files.exists(fileToSync) == false) { + // yet do not suppress trying to fsync directories that do not exist + throw new NoSuchFileException(fileToSync.toString()); + } + return; + } try (FileChannel file = FileChannel.open(fileToSync, isDir ? StandardOpenOption.READ : StandardOpenOption.WRITE)) { - file.force(true); - } catch (final IOException ioe) { - if (isDir) { - assert (LINUX || MAC_OS_X) == false : - "on Linux and MacOSX fsyncing a directory should not throw IOException, "+ - "we just don't want to rely on that in production (undocumented); got: " + ioe; - // ignore exception if it is a directory - return; + try { + file.force(true); + } catch (final IOException e) { + if (isDir) { + assert (LINUX || MAC_OS_X) == false : + "on Linux and MacOSX fsyncing a directory should not throw IOException, "+ + "we just don't want to rely on that in production (undocumented); got: " + e; + // ignore exception if it is a directory + return; + } + // throw original exception + throw e; } - // throw original exception - throw ioe; } } } diff --git a/libs/core/src/test/eclipse-build.gradle b/libs/core/src/test/eclipse-build.gradle index b5fe04174287f..9fb1d2cac390f 100644 --- a/libs/core/src/test/eclipse-build.gradle +++ b/libs/core/src/test/eclipse-build.gradle @@ -2,5 +2,5 @@ apply from: '../../build.gradle' dependencies { - testCompile project(':libs:core') + testCompile project(':libs:elasticsearch-core') } diff --git a/libs/core/src/test/java/org/elasticsearch/common/unit/TimeValueTests.java b/libs/core/src/test/java/org/elasticsearch/common/unit/TimeValueTests.java index af6b89be5fffe..3dec990b7b251 100644 --- a/libs/core/src/test/java/org/elasticsearch/common/unit/TimeValueTests.java +++ b/libs/core/src/test/java/org/elasticsearch/common/unit/TimeValueTests.java @@ -47,6 +47,26 @@ public void testToString() { assertThat("1.5m", equalTo(new TimeValue(90, TimeUnit.SECONDS).toString())); assertThat("1.5h", equalTo(new TimeValue(90, TimeUnit.MINUTES).toString())); assertThat("1.5d", equalTo(new TimeValue(36, TimeUnit.HOURS).toString())); + assertThat("1d", equalTo(new TimeValue(36, TimeUnit.HOURS).toHumanReadableString(0))); + assertThat("1d", equalTo(new TimeValue(36, TimeUnit.HOURS).toHumanReadableString(-4))); + assertThat("1.5d", equalTo(new TimeValue(36, TimeUnit.HOURS).toHumanReadableString(2))); + assertThat("1.45d", equalTo(new TimeValue(35, TimeUnit.HOURS).toHumanReadableString(2))); + assertThat("1.4583333333d", equalTo(new TimeValue(35, TimeUnit.HOURS).toHumanReadableString(10))); + assertThat("1d", equalTo(new TimeValue(103723200, TimeUnit.MILLISECONDS).toHumanReadableString(0))); + assertThat("1.2d", equalTo(new TimeValue(103723200, TimeUnit.MILLISECONDS).toHumanReadableString(1))); + assertThat("1.2d", equalTo(new TimeValue(103723200, TimeUnit.MILLISECONDS).toHumanReadableString(2))); + assertThat("1.2d", equalTo(new TimeValue(103723200, TimeUnit.MILLISECONDS).toHumanReadableString(3))); + assertThat("1.2005d", equalTo(new TimeValue(103723200, TimeUnit.MILLISECONDS).toHumanReadableString(4))); + assertThat("1d", equalTo(new TimeValue(86400077, TimeUnit.MILLISECONDS).toHumanReadableString(0))); + assertThat("1d", equalTo(new TimeValue(86400077, TimeUnit.MILLISECONDS).toHumanReadableString(1))); + assertThat("1d", equalTo(new TimeValue(86400077, TimeUnit.MILLISECONDS).toHumanReadableString(2))); + assertThat("1d", equalTo(new TimeValue(86400077, TimeUnit.MILLISECONDS).toHumanReadableString(3))); + assertThat("1d", equalTo(new TimeValue(86400077, TimeUnit.MILLISECONDS).toHumanReadableString(4))); + assertThat("1d", equalTo(new TimeValue(86400077, TimeUnit.MILLISECONDS).toHumanReadableString(5))); + assertThat("1d", equalTo(new TimeValue(86400077, TimeUnit.MILLISECONDS).toHumanReadableString(6))); + assertThat("1.0000008d", equalTo(new TimeValue(86400077, TimeUnit.MILLISECONDS).toHumanReadableString(7))); + assertThat("1.00000089d", equalTo(new TimeValue(86400077, TimeUnit.MILLISECONDS).toHumanReadableString(8))); + assertThat("1.4583333333d", equalTo(new TimeValue(35, TimeUnit.HOURS).toHumanReadableString(Integer.MAX_VALUE))); assertThat("1000d", equalTo(new TimeValue(1000, TimeUnit.DAYS).toString())); } diff --git a/libs/core/src/test/java/org/elasticsearch/core/internal/io/IOUtilsTests.java b/libs/core/src/test/java/org/elasticsearch/core/internal/io/IOUtilsTests.java index ee5af323b5219..8af0f2a707e24 100644 --- a/libs/core/src/test/java/org/elasticsearch/core/internal/io/IOUtilsTests.java +++ b/libs/core/src/test/java/org/elasticsearch/core/internal/io/IOUtilsTests.java @@ -19,6 +19,7 @@ import org.apache.lucene.mockfile.FilterFileSystemProvider; import org.apache.lucene.mockfile.FilterPath; +import org.apache.lucene.util.Constants; import org.elasticsearch.common.CheckedConsumer; import org.elasticsearch.common.io.PathUtils; import org.elasticsearch.test.ESTestCase; @@ -27,14 +28,20 @@ import java.io.IOException; import java.io.OutputStream; import java.net.URI; +import java.nio.channels.FileChannel; import java.nio.charset.StandardCharsets; import java.nio.file.AccessDeniedException; import java.nio.file.FileSystem; import java.nio.file.Files; +import java.nio.file.NoSuchFileException; +import java.nio.file.OpenOption; import java.nio.file.Path; +import java.nio.file.attribute.FileAttribute; import java.util.ArrayList; import java.util.Arrays; import java.util.List; +import java.util.Objects; +import java.util.Set; import java.util.function.Function; import static org.hamcrest.Matchers.arrayWithSize; @@ -214,6 +221,43 @@ public void testFsyncDirectory() throws Exception { // no exception } + private static final class AccessDeniedWhileOpeningDirectoryFileSystem extends FilterFileSystemProvider { + + AccessDeniedWhileOpeningDirectoryFileSystem(final FileSystem delegate) { + super("access_denied://", Objects.requireNonNull(delegate)); + } + + @Override + public FileChannel newFileChannel( + final Path path, + final Set options, + final FileAttribute... attrs) throws IOException { + if (Files.isDirectory(path)) { + throw new AccessDeniedException(path.toString()); + } + return delegate.newFileChannel(path, options, attrs); + } + + } + + public void testFsyncAccessDeniedOpeningDirectory() throws Exception { + final Path path = createTempDir().toRealPath(); + final FileSystem fs = new AccessDeniedWhileOpeningDirectoryFileSystem(path.getFileSystem()).getFileSystem(URI.create("file:///")); + final Path wrapped = new FilterPath(path, fs); + if (Constants.WINDOWS) { + // no exception, we early return and do not even try to open the directory + IOUtils.fsync(wrapped, true); + } else { + expectThrows(AccessDeniedException.class, () -> IOUtils.fsync(wrapped, true)); + } + } + + public void testFsyncNonExistentDirectory() throws Exception { + final Path dir = FilterPath.unwrap(createTempDir()).toRealPath(); + final Path nonExistentDir = dir.resolve("non-existent"); + expectThrows(NoSuchFileException.class, () -> IOUtils.fsync(nonExistentDir, true)); + } + public void testFsyncFile() throws IOException { final Path path = createTempDir().toRealPath(); final Path subPath = path.resolve(randomAlphaOfLength(8)); diff --git a/libs/dissect/build.gradle b/libs/dissect/build.gradle index 853c78646c25b..7e71f86f64f44 100644 --- a/libs/dissect/build.gradle +++ b/libs/dissect/build.gradle @@ -17,17 +17,15 @@ * under the License. */ -archivesBaseName = 'elasticsearch-dissect' - dependencies { - if (isEclipse == false || project.path == ":libs:dissect-tests") { - testCompile("org.elasticsearch.test:framework:${version}") { - exclude group: 'org.elasticsearch', module: 'dissect' + if (isEclipse == false || project.path == ":libs:elasticsearch-dissect-tests") { + testCompile(project(":test:framework")) { + exclude group: 'org.elasticsearch', module: 'elasticsearch-dissect' } } testCompile "com.fasterxml.jackson.core:jackson-core:${versions.jackson}" - testCompile("com.fasterxml.jackson.core:jackson-annotations:${versions.jackson}") - testCompile("com.fasterxml.jackson.core:jackson-databind:${versions.jackson}") + testCompile "com.fasterxml.jackson.core:jackson-annotations:${versions.jackson}" + testCompile "com.fasterxml.jackson.core:jackson-databind:${versions.jackson}" } forbiddenApisMain { @@ -37,7 +35,7 @@ forbiddenApisMain { if (isEclipse) { // in eclipse the project is under a fake root, we need to change around the source sets sourceSets { - if (project.path == ":libs:dissect") { + if (project.path == ":libs:elasticsearch-dissect") { main.java.srcDirs = ['java'] main.resources.srcDirs = ['resources'] } else { diff --git a/libs/dissect/src/test/eclipse-build.gradle b/libs/dissect/src/test/eclipse-build.gradle index 56d632f23b129..c10fea5e2b748 100644 --- a/libs/dissect/src/test/eclipse-build.gradle +++ b/libs/dissect/src/test/eclipse-build.gradle @@ -3,5 +3,5 @@ apply from: '../../build.gradle' dependencies { - testCompile project(':libs:dissect') + testCompile project(':libs:elasticsearch-dissect') } diff --git a/libs/geo/build.gradle b/libs/geo/build.gradle index ab3419b93b9b8..e2e5a11d53586 100644 --- a/libs/geo/build.gradle +++ b/libs/geo/build.gradle @@ -22,8 +22,8 @@ apply plugin: 'nebula.maven-base-publish' apply plugin: 'nebula.maven-scm' dependencies { - if (isEclipse == false || project.path == ":libs:geo-tests") { - testCompile("org.elasticsearch.test:framework:${version}") { + if (isEclipse == false || project.path == ":libs:elasticsearch-geo-tests") { + testCompile(project(":test:framework")) { exclude group: 'org.elasticsearch', module: 'elasticsearch-geo' } } @@ -38,7 +38,7 @@ forbiddenApisMain { if (isEclipse) { // in eclipse the project is under a fake root, we need to change around the source sets sourceSets { - if (project.path == ":libs:geo") { + if (project.path == ":libs:elasticsearch-geo") { main.java.srcDirs = ['java'] main.resources.srcDirs = ['resources'] } else { diff --git a/libs/geo/src/main/java/org/elasticsearch/geo/geometry/Circle.java b/libs/geo/src/main/java/org/elasticsearch/geo/geometry/Circle.java index cb8e2c4cb33e1..ad9881ab72fba 100644 --- a/libs/geo/src/main/java/org/elasticsearch/geo/geometry/Circle.java +++ b/libs/geo/src/main/java/org/elasticsearch/geo/geometry/Circle.java @@ -49,8 +49,6 @@ public Circle(final double lat, final double lon, final double alt, final double if (radiusMeters < 0 ) { throw new IllegalArgumentException("Circle radius [" + radiusMeters + "] cannot be negative"); } - GeometryUtils.checkLatitude(lat); - GeometryUtils.checkLongitude(lon); } @Override diff --git a/libs/geo/src/main/java/org/elasticsearch/geo/geometry/GeometryUtils.java b/libs/geo/src/main/java/org/elasticsearch/geo/geometry/GeometryUtils.java deleted file mode 100644 index 9a7d4b99d3e53..0000000000000 --- a/libs/geo/src/main/java/org/elasticsearch/geo/geometry/GeometryUtils.java +++ /dev/null @@ -1,70 +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.geo.geometry; - -/** - * Geometry-related utility methods - */ -final class GeometryUtils { - /** - * Minimum longitude value. - */ - static final double MIN_LON_INCL = -180.0D; - - /** - * Maximum longitude value. - */ - static final double MAX_LON_INCL = 180.0D; - - /** - * Minimum latitude value. - */ - static final double MIN_LAT_INCL = -90.0D; - - /** - * Maximum latitude value. - */ - static final double MAX_LAT_INCL = 90.0D; - - // No instance: - private GeometryUtils() { - } - - /** - * validates latitude value is within standard +/-90 coordinate bounds - */ - static void checkLatitude(double latitude) { - if (Double.isNaN(latitude) || latitude < MIN_LAT_INCL || latitude > MAX_LAT_INCL) { - throw new IllegalArgumentException( - "invalid latitude " + latitude + "; must be between " + MIN_LAT_INCL + " and " + MAX_LAT_INCL); - } - } - - /** - * validates longitude value is within standard +/-180 coordinate bounds - */ - static void checkLongitude(double longitude) { - if (Double.isNaN(longitude) || longitude < MIN_LON_INCL || longitude > MAX_LON_INCL) { - throw new IllegalArgumentException( - "invalid longitude " + longitude + "; must be between " + MIN_LON_INCL + " and " + MAX_LON_INCL); - } - } - -} diff --git a/libs/geo/src/main/java/org/elasticsearch/geo/geometry/Line.java b/libs/geo/src/main/java/org/elasticsearch/geo/geometry/Line.java index c2c9cb4b83a18..20f4314246950 100644 --- a/libs/geo/src/main/java/org/elasticsearch/geo/geometry/Line.java +++ b/libs/geo/src/main/java/org/elasticsearch/geo/geometry/Line.java @@ -59,10 +59,6 @@ public Line(double[] lats, double[] lons, double[] alts) { if (alts != null && alts.length != lats.length) { throw new IllegalArgumentException("alts and lats must be equal length"); } - for (int i = 0; i < lats.length; i++) { - GeometryUtils.checkLatitude(lats[i]); - GeometryUtils.checkLongitude(lons[i]); - } } public int length() { diff --git a/libs/geo/src/main/java/org/elasticsearch/geo/geometry/Point.java b/libs/geo/src/main/java/org/elasticsearch/geo/geometry/Point.java index 248f433b96a13..88fd5eb06fe79 100644 --- a/libs/geo/src/main/java/org/elasticsearch/geo/geometry/Point.java +++ b/libs/geo/src/main/java/org/elasticsearch/geo/geometry/Point.java @@ -42,8 +42,6 @@ public Point(double lat, double lon) { } public Point(double lat, double lon, double alt) { - GeometryUtils.checkLatitude(lat); - GeometryUtils.checkLongitude(lon); this.lat = lat; this.lon = lon; this.alt = alt; diff --git a/libs/geo/src/main/java/org/elasticsearch/geo/geometry/Rectangle.java b/libs/geo/src/main/java/org/elasticsearch/geo/geometry/Rectangle.java index ca7ec2e57c98d..75ba25721e755 100644 --- a/libs/geo/src/main/java/org/elasticsearch/geo/geometry/Rectangle.java +++ b/libs/geo/src/main/java/org/elasticsearch/geo/geometry/Rectangle.java @@ -71,10 +71,6 @@ public Rectangle(double minLat, double maxLat, double minLon, double maxLon) { * Constructs a bounding box by first validating the provided latitude and longitude coordinates */ public Rectangle(double minLat, double maxLat, double minLon, double maxLon, double minAlt, double maxAlt) { - GeometryUtils.checkLatitude(minLat); - GeometryUtils.checkLatitude(maxLat); - GeometryUtils.checkLongitude(minLon); - GeometryUtils.checkLongitude(maxLon); this.minLon = minLon; this.maxLon = maxLon; this.minLat = minLat; @@ -90,17 +86,6 @@ public Rectangle(double minLat, double maxLat, double minLon, double maxLon, dou } } - public double getWidth() { - if (crossesDateline()) { - return GeometryUtils.MAX_LON_INCL - minLon + maxLon - GeometryUtils.MIN_LON_INCL; - } - return maxLon - minLon; - } - - public double getHeight() { - return maxLat - minLat; - } - public double getMinLat() { return minLat; } @@ -156,21 +141,6 @@ public String toString() { return b.toString(); } - /** - * Returns true if this bounding box crosses the dateline - */ - public boolean crossesDateline() { - return maxLon < minLon; - } - - /** returns true if rectangle (defined by minLat, maxLat, minLon, maxLon) contains the lat lon point */ - public boolean containsPoint(final double lat, final double lon) { - if (lat >= minLat && lat <= maxLat) { - return crossesDateline() ? lon >= minLon || lon <= maxLon : lon >= minLon && lon <= maxLon; - } - return false; - } - @Override public boolean equals(Object o) { if (this == o) return true; diff --git a/libs/geo/src/main/java/org/elasticsearch/geo/utils/GeographyValidator.java b/libs/geo/src/main/java/org/elasticsearch/geo/utils/GeographyValidator.java new file mode 100644 index 0000000000000..756792358abd4 --- /dev/null +++ b/libs/geo/src/main/java/org/elasticsearch/geo/utils/GeographyValidator.java @@ -0,0 +1,178 @@ +/* + * 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.geo.utils; + +import org.elasticsearch.geo.geometry.Circle; +import org.elasticsearch.geo.geometry.Geometry; +import org.elasticsearch.geo.geometry.GeometryCollection; +import org.elasticsearch.geo.geometry.GeometryVisitor; +import org.elasticsearch.geo.geometry.Line; +import org.elasticsearch.geo.geometry.LinearRing; +import org.elasticsearch.geo.geometry.MultiLine; +import org.elasticsearch.geo.geometry.MultiPoint; +import org.elasticsearch.geo.geometry.MultiPolygon; +import org.elasticsearch.geo.geometry.Point; +import org.elasticsearch.geo.geometry.Polygon; +import org.elasticsearch.geo.geometry.Rectangle; + +/** + * Validator that checks that lats are between -90 and +90 and lons are between -180 and +180 and altitude is present only if + * ignoreZValue is set to true + */ +public class GeographyValidator implements GeometryValidator { + + /** + * Minimum longitude value. + */ + private static final double MIN_LON_INCL = -180.0D; + + /** + * Maximum longitude value. + */ + private static final double MAX_LON_INCL = 180.0D; + + /** + * Minimum latitude value. + */ + private static final double MIN_LAT_INCL = -90.0D; + + /** + * Maximum latitude value. + */ + private static final double MAX_LAT_INCL = 90.0D; + + private final boolean ignoreZValue; + + public GeographyValidator(boolean ignoreZValue) { + this.ignoreZValue = ignoreZValue; + } + + /** + * validates latitude value is within standard +/-90 coordinate bounds + */ + protected void checkLatitude(double latitude) { + if (Double.isNaN(latitude) || latitude < MIN_LAT_INCL || latitude > MAX_LAT_INCL) { + throw new IllegalArgumentException( + "invalid latitude " + latitude + "; must be between " + MIN_LAT_INCL + " and " + MAX_LAT_INCL); + } + } + + /** + * validates longitude value is within standard +/-180 coordinate bounds + */ + protected void checkLongitude(double longitude) { + if (Double.isNaN(longitude) || longitude < MIN_LON_INCL || longitude > MAX_LON_INCL) { + throw new IllegalArgumentException( + "invalid longitude " + longitude + "; must be between " + MIN_LON_INCL + " and " + MAX_LON_INCL); + } + } + + protected void checkAltitude(double zValue) { + if (ignoreZValue == false && Double.isNaN(zValue) == false) { + throw new IllegalArgumentException("found Z value [" + zValue + "] but [ignore_z_value] " + + "parameter is [" + ignoreZValue + "]"); + } + } + + @Override + public void validate(Geometry geometry) { + geometry.visit(new GeometryVisitor() { + + @Override + public Void visit(Circle circle) throws RuntimeException { + checkLatitude(circle.getLat()); + checkLongitude(circle.getLon()); + checkAltitude(circle.getAlt()); + return null; + } + + @Override + public Void visit(GeometryCollection collection) throws RuntimeException { + for (Geometry g : collection) { + g.visit(this); + } + return null; + } + + @Override + public Void visit(Line line) throws RuntimeException { + for (int i = 0; i < line.length(); i++) { + checkLatitude(line.getLat(i)); + checkLongitude(line.getLon(i)); + checkAltitude(line.getAlt(i)); + } + return null; + } + + @Override + public Void visit(LinearRing ring) throws RuntimeException { + for (int i = 0; i < ring.length(); i++) { + checkLatitude(ring.getLat(i)); + checkLongitude(ring.getLon(i)); + checkAltitude(ring.getAlt(i)); + } + return null; + } + + @Override + public Void visit(MultiLine multiLine) throws RuntimeException { + return visit((GeometryCollection) multiLine); + } + + @Override + public Void visit(MultiPoint multiPoint) throws RuntimeException { + return visit((GeometryCollection) multiPoint); + } + + @Override + public Void visit(MultiPolygon multiPolygon) throws RuntimeException { + return visit((GeometryCollection) multiPolygon); + } + + @Override + public Void visit(Point point) throws RuntimeException { + checkLatitude(point.getLat()); + checkLongitude(point.getLon()); + checkAltitude(point.getAlt()); + return null; + } + + @Override + public Void visit(Polygon polygon) throws RuntimeException { + polygon.getPolygon().visit(this); + for (int i = 0; i < polygon.getNumberOfHoles(); i++) { + polygon.getHole(i).visit(this); + } + return null; + } + + @Override + public Void visit(Rectangle rectangle) throws RuntimeException { + checkLatitude(rectangle.getMinLat()); + checkLatitude(rectangle.getMaxLat()); + checkLongitude(rectangle.getMinLon()); + checkLongitude(rectangle.getMaxLon()); + checkAltitude(rectangle.getMinAlt()); + checkAltitude(rectangle.getMaxAlt()); + return null; + } + }); + } +} diff --git a/libs/geo/src/main/java/org/elasticsearch/geo/utils/GeometryValidator.java b/libs/geo/src/main/java/org/elasticsearch/geo/utils/GeometryValidator.java new file mode 100644 index 0000000000000..2caf6738ed469 --- /dev/null +++ b/libs/geo/src/main/java/org/elasticsearch/geo/utils/GeometryValidator.java @@ -0,0 +1,34 @@ +/* + * 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.geo.utils; + +import org.elasticsearch.geo.geometry.Geometry; + +/** + * Generic geometry validator that can be used by the parser to verify the validity of the parsed geometry + */ +public interface GeometryValidator { + + /** + * Validates the geometry and throws IllegalArgumentException if the geometry is not valid + */ + void validate(Geometry geometry); + +} diff --git a/libs/geo/src/main/java/org/elasticsearch/geo/utils/WellKnownText.java b/libs/geo/src/main/java/org/elasticsearch/geo/utils/WellKnownText.java index e1af54e3383e0..4fd4bdb6fd150 100644 --- a/libs/geo/src/main/java/org/elasticsearch/geo/utils/WellKnownText.java +++ b/libs/geo/src/main/java/org/elasticsearch/geo/utils/WellKnownText.java @@ -52,17 +52,25 @@ public class WellKnownText { public static final String COMMA = ","; public static final String NAN = "NaN"; - private static final String NUMBER = ""; - private static final String EOF = "END-OF-STREAM"; - private static final String EOL = "END-OF-LINE"; + private final String NUMBER = ""; + private final String EOF = "END-OF-STREAM"; + private final String EOL = "END-OF-LINE"; - public static String toWKT(Geometry geometry) { + private final boolean coerce; + private final GeometryValidator validator; + + public WellKnownText(boolean coerce, GeometryValidator validator) { + this.coerce = coerce; + this.validator = validator; + } + + public String toWKT(Geometry geometry) { StringBuilder builder = new StringBuilder(); toWKT(geometry, builder); return builder.toString(); } - public static void toWKT(Geometry geometry, StringBuilder sb) { + public void toWKT(Geometry geometry, StringBuilder sb) { sb.append(getWKTName(geometry)); sb.append(SPACE); if (geometry.isEmpty()) { @@ -216,7 +224,7 @@ public Void visit(Rectangle rectangle) { } } - public static Geometry fromWKT(String wkt) throws IOException, ParseException { + public Geometry fromWKT(String wkt) throws IOException, ParseException { StringReader reader = new StringReader(wkt); try { // setup the tokenizer; configured to read words w/o numbers @@ -234,7 +242,9 @@ public static Geometry fromWKT(String wkt) throws IOException, ParseException { tokenizer.whitespaceChars('\r', '\r'); tokenizer.whitespaceChars('\n', '\n'); tokenizer.commentChar('#'); - return parseGeometry(tokenizer); + Geometry geometry = parseGeometry(tokenizer); + validator.validate(geometry); + return geometry; } finally { reader.close(); } @@ -243,7 +253,7 @@ public static Geometry fromWKT(String wkt) throws IOException, ParseException { /** * parse geometry from the stream tokenizer */ - private static Geometry parseGeometry(StreamTokenizer stream) throws IOException, ParseException { + private Geometry parseGeometry(StreamTokenizer stream) throws IOException, ParseException { final String type = nextWord(stream).toLowerCase(Locale.ROOT); switch (type) { case "point": @@ -268,7 +278,7 @@ private static Geometry parseGeometry(StreamTokenizer stream) throws IOException throw new IllegalArgumentException("Unknown geometry type: " + type); } - private static GeometryCollection parseGeometryCollection(StreamTokenizer stream) throws IOException, ParseException { + private GeometryCollection parseGeometryCollection(StreamTokenizer stream) throws IOException, ParseException { if (nextEmptyOrOpen(stream).equals(EMPTY)) { return GeometryCollection.EMPTY; } @@ -280,7 +290,7 @@ private static GeometryCollection parseGeometryCollection(StreamTokeni return new GeometryCollection<>(shapes); } - private static Point parsePoint(StreamTokenizer stream) throws IOException, ParseException { + private Point parsePoint(StreamTokenizer stream) throws IOException, ParseException { if (nextEmptyOrOpen(stream).equals(EMPTY)) { return Point.EMPTY; } @@ -296,7 +306,7 @@ private static Point parsePoint(StreamTokenizer stream) throws IOException, Pars return pt; } - private static void parseCoordinates(StreamTokenizer stream, ArrayList lats, ArrayList lons, ArrayList alts) + private void parseCoordinates(StreamTokenizer stream, ArrayList lats, ArrayList lons, ArrayList alts) throws IOException, ParseException { parseCoordinate(stream, lats, lons, alts); while (nextCloserOrComma(stream).equals(COMMA)) { @@ -304,7 +314,7 @@ private static void parseCoordinates(StreamTokenizer stream, ArrayList l } } - private static void parseCoordinate(StreamTokenizer stream, ArrayList lats, ArrayList lons, ArrayList alts) + private void parseCoordinate(StreamTokenizer stream, ArrayList lats, ArrayList lons, ArrayList alts) throws IOException, ParseException { lons.add(nextNumber(stream)); lats.add(nextNumber(stream)); @@ -316,7 +326,7 @@ private static void parseCoordinate(StreamTokenizer stream, ArrayList la } } - private static MultiPoint parseMultiPoint(StreamTokenizer stream) throws IOException, ParseException { + private MultiPoint parseMultiPoint(StreamTokenizer stream) throws IOException, ParseException { String token = nextEmptyOrOpen(stream); if (token.equals(EMPTY)) { return MultiPoint.EMPTY; @@ -336,7 +346,7 @@ private static MultiPoint parseMultiPoint(StreamTokenizer stream) throws IOExcep return new MultiPoint(Collections.unmodifiableList(points)); } - private static Line parseLine(StreamTokenizer stream) throws IOException, ParseException { + private Line parseLine(StreamTokenizer stream) throws IOException, ParseException { String token = nextEmptyOrOpen(stream); if (token.equals(EMPTY)) { return Line.EMPTY; @@ -352,7 +362,7 @@ private static Line parseLine(StreamTokenizer stream) throws IOException, ParseE } } - private static MultiLine parseMultiLine(StreamTokenizer stream) throws IOException, ParseException { + private MultiLine parseMultiLine(StreamTokenizer stream) throws IOException, ParseException { String token = nextEmptyOrOpen(stream); if (token.equals(EMPTY)) { return MultiLine.EMPTY; @@ -365,12 +375,13 @@ private static MultiLine parseMultiLine(StreamTokenizer stream) throws IOExcepti return new MultiLine(Collections.unmodifiableList(lines)); } - private static LinearRing parsePolygonHole(StreamTokenizer stream) throws IOException, ParseException { + private LinearRing parsePolygonHole(StreamTokenizer stream) throws IOException, ParseException { nextOpener(stream); ArrayList lats = new ArrayList<>(); ArrayList lons = new ArrayList<>(); ArrayList alts = new ArrayList<>(); parseCoordinates(stream, lats, lons, alts); + closeLinearRingIfCoerced(lats, lons, alts); if (alts.isEmpty()) { return new LinearRing(toArray(lats), toArray(lons)); } else { @@ -378,7 +389,7 @@ private static LinearRing parsePolygonHole(StreamTokenizer stream) throws IOExce } } - private static Polygon parsePolygon(StreamTokenizer stream) throws IOException, ParseException { + private Polygon parsePolygon(StreamTokenizer stream) throws IOException, ParseException { if (nextEmptyOrOpen(stream).equals(EMPTY)) { return Polygon.EMPTY; } @@ -391,6 +402,7 @@ private static Polygon parsePolygon(StreamTokenizer stream) throws IOException, while (nextCloserOrComma(stream).equals(COMMA)) { holes.add(parsePolygonHole(stream)); } + closeLinearRingIfCoerced(lats, lons, alts); LinearRing shell; if (alts.isEmpty()) { shell = new LinearRing(toArray(lats), toArray(lons)); @@ -404,7 +416,25 @@ private static Polygon parsePolygon(StreamTokenizer stream) throws IOException, } } - private static MultiPolygon parseMultiPolygon(StreamTokenizer stream) throws IOException, ParseException { + /** + * Treats supplied arrays as coordinates of a linear ring. If the ring is not closed and coerce is set to true, + * the first set of coordinates (lat, lon and alt if available) are added to the end of the arrays. + */ + private void closeLinearRingIfCoerced(ArrayList lats, ArrayList lons, ArrayList alts) { + if (coerce && lats.isEmpty() == false && lons.isEmpty() == false) { + int last = lats.size() - 1; + if (!lats.get(0).equals(lats.get(last)) || !lons.get(0).equals(lons.get(last)) || + (alts.isEmpty() == false && !alts.get(0).equals(alts.get(last)))) { + lons.add(lons.get(0)); + lats.add(lats.get(0)); + if (alts.isEmpty() == false) { + alts.add(alts.get(0)); + } + } + } + } + + private MultiPolygon parseMultiPolygon(StreamTokenizer stream) throws IOException, ParseException { String token = nextEmptyOrOpen(stream); if (token.equals(EMPTY)) { return MultiPolygon.EMPTY; @@ -417,7 +447,7 @@ private static MultiPolygon parseMultiPolygon(StreamTokenizer stream) throws IOE return new MultiPolygon(Collections.unmodifiableList(polygons)); } - private static Rectangle parseBBox(StreamTokenizer stream) throws IOException, ParseException { + private Rectangle parseBBox(StreamTokenizer stream) throws IOException, ParseException { if (nextEmptyOrOpen(stream).equals(EMPTY)) { return Rectangle.EMPTY; } @@ -434,7 +464,7 @@ private static Rectangle parseBBox(StreamTokenizer stream) throws IOException, P } - private static Circle parseCircle(StreamTokenizer stream) throws IOException, ParseException { + private Circle parseCircle(StreamTokenizer stream) throws IOException, ParseException { if (nextEmptyOrOpen(stream).equals(EMPTY)) { return Circle.EMPTY; } @@ -453,7 +483,7 @@ private static Circle parseCircle(StreamTokenizer stream) throws IOException, Pa /** * next word in the stream */ - private static String nextWord(StreamTokenizer stream) throws ParseException, IOException { + private String nextWord(StreamTokenizer stream) throws ParseException, IOException { switch (stream.nextToken()) { case StreamTokenizer.TT_WORD: final String word = stream.sval; @@ -468,7 +498,7 @@ private static String nextWord(StreamTokenizer stream) throws ParseException, IO throw new ParseException("expected word but found: " + tokenString(stream), stream.lineno()); } - private static double nextNumber(StreamTokenizer stream) throws IOException, ParseException { + private double nextNumber(StreamTokenizer stream) throws IOException, ParseException { if (stream.nextToken() == StreamTokenizer.TT_WORD) { if (stream.sval.equalsIgnoreCase(NAN)) { return Double.NaN; @@ -483,7 +513,7 @@ private static double nextNumber(StreamTokenizer stream) throws IOException, Par throw new ParseException("expected number but found: " + tokenString(stream), stream.lineno()); } - private static String tokenString(StreamTokenizer stream) { + private String tokenString(StreamTokenizer stream) { switch (stream.ttype) { case StreamTokenizer.TT_WORD: return stream.sval; @@ -497,13 +527,13 @@ private static String tokenString(StreamTokenizer stream) { return "'" + (char) stream.ttype + "'"; } - private static boolean isNumberNext(StreamTokenizer stream) throws IOException { + private boolean isNumberNext(StreamTokenizer stream) throws IOException { final int type = stream.nextToken(); stream.pushBack(); return type == StreamTokenizer.TT_WORD; } - private static String nextEmptyOrOpen(StreamTokenizer stream) throws IOException, ParseException { + private String nextEmptyOrOpen(StreamTokenizer stream) throws IOException, ParseException { final String next = nextWord(stream); if (next.equals(EMPTY) || next.equals(LPAREN)) { return next; @@ -512,28 +542,28 @@ private static String nextEmptyOrOpen(StreamTokenizer stream) throws IOException + " but found: " + tokenString(stream), stream.lineno()); } - private static String nextCloser(StreamTokenizer stream) throws IOException, ParseException { + private String nextCloser(StreamTokenizer stream) throws IOException, ParseException { if (nextWord(stream).equals(RPAREN)) { return RPAREN; } throw new ParseException("expected " + RPAREN + " but found: " + tokenString(stream), stream.lineno()); } - private static String nextComma(StreamTokenizer stream) throws IOException, ParseException { + private String nextComma(StreamTokenizer stream) throws IOException, ParseException { if (nextWord(stream).equals(COMMA) == true) { return COMMA; } throw new ParseException("expected " + COMMA + " but found: " + tokenString(stream), stream.lineno()); } - private static String nextOpener(StreamTokenizer stream) throws IOException, ParseException { + private String nextOpener(StreamTokenizer stream) throws IOException, ParseException { if (nextWord(stream).equals(LPAREN)) { return LPAREN; } throw new ParseException("expected " + LPAREN + " but found: " + tokenString(stream), stream.lineno()); } - private static String nextCloserOrComma(StreamTokenizer stream) throws IOException, ParseException { + private String nextCloserOrComma(StreamTokenizer stream) throws IOException, ParseException { String token = nextWord(stream); if (token.equals(COMMA) || token.equals(RPAREN)) { return token; @@ -542,7 +572,7 @@ private static String nextCloserOrComma(StreamTokenizer stream) throws IOExcepti + " but found: " + tokenString(stream), stream.lineno()); } - public static String getWKTName(Geometry geometry) { + private static String getWKTName(Geometry geometry) { return geometry.visit(new GeometryVisitor() { @Override public String visit(Circle circle) { @@ -596,7 +626,7 @@ public String visit(Rectangle rectangle) { }); } - private static double[] toArray(ArrayList doubles) { + private double[] toArray(ArrayList doubles) { return doubles.stream().mapToDouble(i -> i).toArray(); } diff --git a/libs/geo/src/test/java/org/elasticsearch/geo/geometry/BaseGeometryTestCase.java b/libs/geo/src/test/java/org/elasticsearch/geo/geometry/BaseGeometryTestCase.java index cc7dcd340c734..073bff4cb7575 100644 --- a/libs/geo/src/test/java/org/elasticsearch/geo/geometry/BaseGeometryTestCase.java +++ b/libs/geo/src/test/java/org/elasticsearch/geo/geometry/BaseGeometryTestCase.java @@ -22,6 +22,7 @@ import org.elasticsearch.ElasticsearchException; import org.elasticsearch.Version; import org.elasticsearch.common.io.stream.Writeable; +import org.elasticsearch.geo.utils.GeographyValidator; import org.elasticsearch.geo.utils.WellKnownText; import org.elasticsearch.test.AbstractWireTestCase; @@ -53,9 +54,10 @@ protected Writeable.Reader instanceReader() { @SuppressWarnings("unchecked") @Override protected T copyInstance(T instance, Version version) throws IOException { - String text = WellKnownText.toWKT(instance); + WellKnownText wkt = new WellKnownText(true, new GeographyValidator(true)); + String text = wkt.toWKT(instance); try { - return (T) WellKnownText.fromWKT(text); + return (T) wkt.fromWKT(text); } catch (ParseException e) { throw new ElasticsearchException(e); } diff --git a/libs/geo/src/test/java/org/elasticsearch/geo/geometry/CircleTests.java b/libs/geo/src/test/java/org/elasticsearch/geo/geometry/CircleTests.java index a38a29af24b54..e8912a39fb435 100644 --- a/libs/geo/src/test/java/org/elasticsearch/geo/geometry/CircleTests.java +++ b/libs/geo/src/test/java/org/elasticsearch/geo/geometry/CircleTests.java @@ -19,6 +19,8 @@ package org.elasticsearch.geo.geometry; +import org.elasticsearch.geo.utils.GeographyValidator; +import org.elasticsearch.geo.utils.GeometryValidator; import org.elasticsearch.geo.utils.WellKnownText; import java.io.IOException; @@ -36,24 +38,26 @@ protected Circle createTestInstance(boolean hasAlt) { } public void testBasicSerialization() throws IOException, ParseException { - assertEquals("circle (20.0 10.0 15.0)", WellKnownText.toWKT(new Circle(10, 20, 15))); - assertEquals(new Circle(10, 20, 15), WellKnownText.fromWKT("circle (20.0 10.0 15.0)")); + WellKnownText wkt = new WellKnownText(true, new GeographyValidator(true)); + assertEquals("circle (20.0 10.0 15.0)", wkt.toWKT(new Circle(10, 20, 15))); + assertEquals(new Circle(10, 20, 15), wkt.fromWKT("circle (20.0 10.0 15.0)")); - assertEquals("circle (20.0 10.0 15.0 25.0)", WellKnownText.toWKT(new Circle(10, 20, 25, 15))); - assertEquals(new Circle(10, 20, 25, 15), WellKnownText.fromWKT("circle (20.0 10.0 15.0 25.0)")); + assertEquals("circle (20.0 10.0 15.0 25.0)", wkt.toWKT(new Circle(10, 20, 25, 15))); + assertEquals(new Circle(10, 20, 25, 15), wkt.fromWKT("circle (20.0 10.0 15.0 25.0)")); - assertEquals("circle EMPTY", WellKnownText.toWKT(Circle.EMPTY)); - assertEquals(Circle.EMPTY, WellKnownText.fromWKT("circle EMPTY)")); + assertEquals("circle EMPTY", wkt.toWKT(Circle.EMPTY)); + assertEquals(Circle.EMPTY, wkt.fromWKT("circle EMPTY)")); } public void testInitValidation() { - IllegalArgumentException ex = expectThrows(IllegalArgumentException.class, () -> new Circle(10, 20, -1)); + GeometryValidator validator = new GeographyValidator(true); + IllegalArgumentException ex = expectThrows(IllegalArgumentException.class, () -> validator.validate(new Circle(10, 20, -1))); assertEquals("Circle radius [-1.0] cannot be negative", ex.getMessage()); - ex = expectThrows(IllegalArgumentException.class, () -> new Circle(100, 20, 1)); + ex = expectThrows(IllegalArgumentException.class, () -> validator.validate(new Circle(100, 20, 1))); assertEquals("invalid latitude 100.0; must be between -90.0 and 90.0", ex.getMessage()); - ex = expectThrows(IllegalArgumentException.class, () -> new Circle(10, 200, 1)); + ex = expectThrows(IllegalArgumentException.class, () -> validator.validate(new Circle(10, 200, 1))); assertEquals("invalid longitude 200.0; must be between -180.0 and 180.0", ex.getMessage()); } } diff --git a/libs/geo/src/test/java/org/elasticsearch/geo/geometry/GeometryCollectionTests.java b/libs/geo/src/test/java/org/elasticsearch/geo/geometry/GeometryCollectionTests.java index 8b6415270635e..c78c47dfbcd96 100644 --- a/libs/geo/src/test/java/org/elasticsearch/geo/geometry/GeometryCollectionTests.java +++ b/libs/geo/src/test/java/org/elasticsearch/geo/geometry/GeometryCollectionTests.java @@ -19,6 +19,7 @@ package org.elasticsearch.geo.geometry; +import org.elasticsearch.geo.utils.GeographyValidator; import org.elasticsearch.geo.utils.WellKnownText; import java.io.IOException; @@ -35,14 +36,15 @@ protected GeometryCollection createTestInstance(boolean hasAlt) { public void testBasicSerialization() throws IOException, ParseException { + WellKnownText wkt = new WellKnownText(true, new GeographyValidator(true)); assertEquals("geometrycollection (point (20.0 10.0),point EMPTY)", - WellKnownText.toWKT(new GeometryCollection(Arrays.asList(new Point(10, 20), Point.EMPTY)))); + wkt.toWKT(new GeometryCollection(Arrays.asList(new Point(10, 20), Point.EMPTY)))); assertEquals(new GeometryCollection(Arrays.asList(new Point(10, 20), Point.EMPTY)), - WellKnownText.fromWKT("geometrycollection (point (20.0 10.0),point EMPTY)")); + wkt.fromWKT("geometrycollection (point (20.0 10.0),point EMPTY)")); - assertEquals("geometrycollection EMPTY", WellKnownText.toWKT(GeometryCollection.EMPTY)); - assertEquals(GeometryCollection.EMPTY, WellKnownText.fromWKT("geometrycollection EMPTY)")); + assertEquals("geometrycollection EMPTY", wkt.toWKT(GeometryCollection.EMPTY)); + assertEquals(GeometryCollection.EMPTY, wkt.fromWKT("geometrycollection EMPTY)")); } @SuppressWarnings("ConstantConditions") diff --git a/libs/geo/src/test/java/org/elasticsearch/geo/geometry/GeometryValidatorTests.java b/libs/geo/src/test/java/org/elasticsearch/geo/geometry/GeometryValidatorTests.java new file mode 100644 index 0000000000000..c747fc2df50a7 --- /dev/null +++ b/libs/geo/src/test/java/org/elasticsearch/geo/geometry/GeometryValidatorTests.java @@ -0,0 +1,127 @@ +/* + * 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.geo.geometry; + +import org.elasticsearch.geo.utils.GeographyValidator; +import org.elasticsearch.geo.utils.GeometryValidator; +import org.elasticsearch.geo.utils.WellKnownText; +import org.elasticsearch.test.ESTestCase; + +public class GeometryValidatorTests extends ESTestCase { + + public static class NoopValidator implements GeometryValidator { + + @Override + public void validate(Geometry geometry) { + + } + } + + public static class OneValidator extends GeographyValidator { + /** + * Minimum longitude value. + */ + private static final double MIN_LON_INCL = -1D; + + /** + * Maximum longitude value. + */ + private static final double MAX_LON_INCL = 1D; + + /** + * Minimum latitude value. + */ + private static final double MIN_LAT_INCL = -1D; + + /** + * Maximum latitude value. + */ + private static final double MAX_LAT_INCL = 1D; + + /** + * Minimum altitude value. + */ + private static final double MIN_ALT_INCL = -1D; + + /** + * Maximum altitude value. + */ + private static final double MAX_ALT_INCL = 1D; + + public OneValidator() { + super(true); + } + + @Override + protected void checkLatitude(double latitude) { + if (Double.isNaN(latitude) || latitude < MIN_LAT_INCL || latitude > MAX_LAT_INCL) { + throw new IllegalArgumentException( + "invalid latitude " + latitude + "; must be between " + MIN_LAT_INCL + " and " + MAX_LAT_INCL); + } + } + + @Override + protected void checkLongitude(double longitude) { + if (Double.isNaN(longitude) || longitude < MIN_LON_INCL || longitude > MAX_LON_INCL) { + throw new IllegalArgumentException( + "invalid longitude " + longitude + "; must be between " + MIN_LON_INCL + " and " + MAX_LON_INCL); + } + } + + @Override + protected void checkAltitude(double zValue) { + if (Double.isNaN(zValue) == false && (zValue < MIN_ALT_INCL || zValue > MAX_ALT_INCL)) { + throw new IllegalArgumentException( + "invalid altitude " + zValue + "; must be between " + MIN_ALT_INCL + " and " + MAX_ALT_INCL); + } + } + } + + public void testNoopValidator() throws Exception { + WellKnownText parser = new WellKnownText(true, new NoopValidator()); + parser.fromWKT("CIRCLE (10000 20000 30000)"); + parser.fromWKT("POINT (10000 20000)"); + parser.fromWKT("LINESTRING (10000 20000, 0 0)"); + parser.fromWKT("POLYGON ((300 100, 400 200, 500 300, 300 100), (50 150, 250 150, 200 100))"); + parser.fromWKT("MULTIPOINT (10000 20000, 20000 30000)"); + } + + public void testOneValidator() throws Exception { + WellKnownText parser = new WellKnownText(true, new OneValidator()); + parser.fromWKT("POINT (0 1)"); + parser.fromWKT("POINT (0 1 0.5)"); + IllegalArgumentException ex; + ex = expectThrows(IllegalArgumentException.class, () -> parser.fromWKT("CIRCLE (1 2 3)")); + assertEquals("invalid latitude 2.0; must be between -1.0 and 1.0", ex.getMessage()); + ex = expectThrows(IllegalArgumentException.class, () -> parser.fromWKT("POINT (2 1)")); + assertEquals("invalid longitude 2.0; must be between -1.0 and 1.0", ex.getMessage()); + ex = expectThrows(IllegalArgumentException.class, () -> parser.fromWKT("LINESTRING (1 -1 0, 0 0 2)")); + assertEquals("invalid altitude 2.0; must be between -1.0 and 1.0", ex.getMessage()); + ex = expectThrows(IllegalArgumentException.class, () -> parser.fromWKT("POLYGON ((0.3 0.1, 0.4 0.2, 5 0.3, 0.3 0.1))")); + assertEquals("invalid longitude 5.0; must be between -1.0 and 1.0", ex.getMessage()); + ex = expectThrows(IllegalArgumentException.class, () -> parser.fromWKT( + "POLYGON ((0.3 0.1, 0.4 0.2, 0.5 0.3, 0.3 0.1), (0.5 1.5, 2.5 1.5, 2.0 1.0))")); + assertEquals("invalid latitude 1.5; must be between -1.0 and 1.0", ex.getMessage()); + ex = expectThrows(IllegalArgumentException.class, () -> parser.fromWKT("MULTIPOINT (0 1, -2 1)")); + assertEquals("invalid longitude -2.0; must be between -1.0 and 1.0", ex.getMessage()); + } + + +} diff --git a/libs/geo/src/test/java/org/elasticsearch/geo/geometry/LineTests.java b/libs/geo/src/test/java/org/elasticsearch/geo/geometry/LineTests.java index 7156039f9085f..b9f8cb37f5422 100644 --- a/libs/geo/src/test/java/org/elasticsearch/geo/geometry/LineTests.java +++ b/libs/geo/src/test/java/org/elasticsearch/geo/geometry/LineTests.java @@ -19,6 +19,8 @@ package org.elasticsearch.geo.geometry; +import org.elasticsearch.geo.utils.GeographyValidator; +import org.elasticsearch.geo.utils.GeometryValidator; import org.elasticsearch.geo.utils.WellKnownText; import java.io.IOException; @@ -31,26 +33,37 @@ protected Line createTestInstance(boolean hasAlt) { } public void testBasicSerialization() throws IOException, ParseException { - assertEquals("linestring (3.0 1.0, 4.0 2.0)", WellKnownText.toWKT(new Line(new double[]{1, 2}, new double[]{3, 4}))); - assertEquals(new Line(new double[]{1, 2}, new double[]{3, 4}), WellKnownText.fromWKT("linestring (3 1, 4 2)")); + WellKnownText wkt = new WellKnownText(true, new GeographyValidator(true)); + assertEquals("linestring (3.0 1.0, 4.0 2.0)", wkt.toWKT(new Line(new double[]{1, 2}, new double[]{3, 4}))); + assertEquals(new Line(new double[]{1, 2}, new double[]{3, 4}), wkt.fromWKT("linestring (3 1, 4 2)")); - assertEquals("linestring (3.0 1.0 5.0, 4.0 2.0 6.0)", WellKnownText.toWKT(new Line(new double[]{1, 2}, new double[]{3, 4}, + assertEquals("linestring (3.0 1.0 5.0, 4.0 2.0 6.0)", wkt.toWKT(new Line(new double[]{1, 2}, new double[]{3, 4}, new double[]{5, 6}))); assertEquals(new Line(new double[]{1, 2}, new double[]{3, 4}, new double[]{6, 5}), - WellKnownText.fromWKT("linestring (3 1 6, 4 2 5)")); + wkt.fromWKT("linestring (3 1 6, 4 2 5)")); - assertEquals("linestring EMPTY", WellKnownText.toWKT(Line.EMPTY)); - assertEquals(Line.EMPTY, WellKnownText.fromWKT("linestring EMPTY)")); + assertEquals("linestring EMPTY", wkt.toWKT(Line.EMPTY)); + assertEquals(Line.EMPTY, wkt.fromWKT("linestring EMPTY)")); } public void testInitValidation() { - IllegalArgumentException ex = expectThrows(IllegalArgumentException.class, () -> new Line(new double[]{1}, new double[]{3})); + GeometryValidator validator = new GeographyValidator(true); + IllegalArgumentException ex = expectThrows(IllegalArgumentException.class, + () -> validator.validate(new Line(new double[]{1}, new double[]{3}))); assertEquals("at least two points in the line is required", ex.getMessage()); - ex = expectThrows(IllegalArgumentException.class, () -> new Line(new double[]{1, 2, 3, 1}, new double[]{3, 4, 500, 3})); + ex = expectThrows(IllegalArgumentException.class, + () -> validator.validate(new Line(new double[]{1, 2, 3, 1}, new double[]{3, 4, 500, 3}))); assertEquals("invalid longitude 500.0; must be between -180.0 and 180.0", ex.getMessage()); - ex = expectThrows(IllegalArgumentException.class, () -> new Line(new double[]{1, 100, 3, 1}, new double[]{3, 4, 5, 3})); + ex = expectThrows(IllegalArgumentException.class, + () -> validator.validate(new Line(new double[]{1, 100, 3, 1}, new double[]{3, 4, 5, 3}))); assertEquals("invalid latitude 100.0; must be between -90.0 and 90.0", ex.getMessage()); } + + public void testWKTValidation() { + IllegalArgumentException ex = expectThrows(IllegalArgumentException.class, + () -> new WellKnownText(randomBoolean(), new GeographyValidator(false)).fromWKT("linestring (3 1 6, 4 2 5)")); + assertEquals("found Z value [6.0] but [ignore_z_value] parameter is [false]", ex.getMessage()); + } } diff --git a/libs/geo/src/test/java/org/elasticsearch/geo/geometry/LinearRingTests.java b/libs/geo/src/test/java/org/elasticsearch/geo/geometry/LinearRingTests.java index d5b708f558cac..07e9e866233e7 100644 --- a/libs/geo/src/test/java/org/elasticsearch/geo/geometry/LinearRingTests.java +++ b/libs/geo/src/test/java/org/elasticsearch/geo/geometry/LinearRingTests.java @@ -19,6 +19,8 @@ package org.elasticsearch.geo.geometry; +import org.elasticsearch.geo.utils.GeographyValidator; +import org.elasticsearch.geo.utils.GeometryValidator; import org.elasticsearch.geo.utils.WellKnownText; import org.elasticsearch.test.ESTestCase; @@ -26,30 +28,35 @@ public class LinearRingTests extends ESTestCase { public void testBasicSerialization() { UnsupportedOperationException ex = expectThrows(UnsupportedOperationException.class, - () -> WellKnownText.toWKT(new LinearRing(new double[]{1, 2, 3, 1}, new double[]{3, 4, 5, 3}))); + () -> new WellKnownText(true, new GeographyValidator(true)) + .toWKT(new LinearRing(new double[]{1, 2, 3, 1}, new double[]{3, 4, 5, 3}))); assertEquals("line ring cannot be serialized using WKT", ex.getMessage()); } public void testInitValidation() { + GeometryValidator validator = new GeographyValidator(true); IllegalArgumentException ex = expectThrows(IllegalArgumentException.class, - () -> new LinearRing(new double[]{1, 2, 3}, new double[]{3, 4, 5})); + () -> validator.validate(new LinearRing(new double[]{1, 2, 3}, new double[]{3, 4, 5}))); assertEquals("first and last points of the linear ring must be the same (it must close itself): lats[0]=1.0 lats[2]=3.0 " + "lons[0]=3.0 lons[2]=5.0", ex.getMessage()); ex = expectThrows(IllegalArgumentException.class, - () -> new LinearRing(new double[]{1, 2, 1}, new double[]{3, 4, 3}, new double[]{1, 2, 3})); + () -> validator.validate(new LinearRing(new double[]{1, 2, 1}, new double[]{3, 4, 3}, new double[]{1, 2, 3}))); assertEquals("first and last points of the linear ring must be the same (it must close itself): lats[0]=1.0 lats[2]=1.0 " + "lons[0]=3.0 lons[2]=3.0 alts[0]=1.0 alts[2]=3.0", ex.getMessage()); - ex = expectThrows(IllegalArgumentException.class, () -> new LinearRing(new double[]{1}, new double[]{3})); + ex = expectThrows(IllegalArgumentException.class, + () -> validator.validate(new LinearRing(new double[]{1}, new double[]{3}))); assertEquals("at least two points in the line is required", ex.getMessage()); - ex = expectThrows(IllegalArgumentException.class, () -> new LinearRing(new double[]{1, 2, 3, 1}, new double[]{3, 4, 500, 3})); + ex = expectThrows(IllegalArgumentException.class, + () -> validator.validate(new LinearRing(new double[]{1, 2, 3, 1}, new double[]{3, 4, 500, 3}))); assertEquals("invalid longitude 500.0; must be between -180.0 and 180.0", ex.getMessage()); - ex = expectThrows(IllegalArgumentException.class, () -> new LinearRing(new double[]{1, 100, 3, 1}, new double[]{3, 4, 5, 3})); + ex = expectThrows(IllegalArgumentException.class, + () -> validator.validate(new LinearRing(new double[]{1, 100, 3, 1}, new double[]{3, 4, 5, 3}))); assertEquals("invalid latitude 100.0; must be between -90.0 and 90.0", ex.getMessage()); } diff --git a/libs/geo/src/test/java/org/elasticsearch/geo/geometry/MultiLineTests.java b/libs/geo/src/test/java/org/elasticsearch/geo/geometry/MultiLineTests.java index 3fcb84d93b7d2..9ed782e65cc06 100644 --- a/libs/geo/src/test/java/org/elasticsearch/geo/geometry/MultiLineTests.java +++ b/libs/geo/src/test/java/org/elasticsearch/geo/geometry/MultiLineTests.java @@ -19,6 +19,7 @@ package org.elasticsearch.geo.geometry; +import org.elasticsearch.geo.utils.GeographyValidator; import org.elasticsearch.geo.utils.WellKnownText; import java.io.IOException; @@ -40,12 +41,13 @@ protected MultiLine createTestInstance(boolean hasAlt) { } public void testBasicSerialization() throws IOException, ParseException { - assertEquals("multilinestring ((3.0 1.0, 4.0 2.0))", WellKnownText.toWKT( + WellKnownText wkt = new WellKnownText(true, new GeographyValidator(true)); + assertEquals("multilinestring ((3.0 1.0, 4.0 2.0))", wkt.toWKT( new MultiLine(Collections.singletonList(new Line(new double[]{1, 2}, new double[]{3, 4}))))); assertEquals(new MultiLine(Collections.singletonList(new Line(new double[]{1, 2}, new double[]{3, 4}))), - WellKnownText.fromWKT("multilinestring ((3 1, 4 2))")); + wkt.fromWKT("multilinestring ((3 1, 4 2))")); - assertEquals("multilinestring EMPTY", WellKnownText.toWKT(MultiLine.EMPTY)); - assertEquals(MultiLine.EMPTY, WellKnownText.fromWKT("multilinestring EMPTY)")); + assertEquals("multilinestring EMPTY", wkt.toWKT(MultiLine.EMPTY)); + assertEquals(MultiLine.EMPTY, wkt.fromWKT("multilinestring EMPTY)")); } } diff --git a/libs/geo/src/test/java/org/elasticsearch/geo/geometry/MultiPointTests.java b/libs/geo/src/test/java/org/elasticsearch/geo/geometry/MultiPointTests.java index ecdcc0815a82b..c170adf9c9411 100644 --- a/libs/geo/src/test/java/org/elasticsearch/geo/geometry/MultiPointTests.java +++ b/libs/geo/src/test/java/org/elasticsearch/geo/geometry/MultiPointTests.java @@ -19,6 +19,7 @@ package org.elasticsearch.geo.geometry; +import org.elasticsearch.geo.utils.GeographyValidator; import org.elasticsearch.geo.utils.WellKnownText; import java.io.IOException; @@ -41,22 +42,23 @@ protected MultiPoint createTestInstance(boolean hasAlt) { } public void testBasicSerialization() throws IOException, ParseException { - assertEquals("multipoint (2.0 1.0)", WellKnownText.toWKT( + WellKnownText wkt = new WellKnownText(true, new GeographyValidator(true)); + assertEquals("multipoint (2.0 1.0)", wkt.toWKT( new MultiPoint(Collections.singletonList(new Point(1, 2))))); assertEquals(new MultiPoint(Collections.singletonList(new Point(1 ,2))), - WellKnownText.fromWKT("multipoint (2 1)")); + wkt.fromWKT("multipoint (2 1)")); assertEquals("multipoint (2.0 1.0, 3.0 4.0)", - WellKnownText.toWKT(new MultiPoint(Arrays.asList(new Point(1, 2), new Point(4, 3))))); + wkt.toWKT(new MultiPoint(Arrays.asList(new Point(1, 2), new Point(4, 3))))); assertEquals(new MultiPoint(Arrays.asList(new Point(1, 2), new Point(4, 3))), - WellKnownText.fromWKT("multipoint (2 1, 3 4)")); + wkt.fromWKT("multipoint (2 1, 3 4)")); assertEquals("multipoint (2.0 1.0 10.0, 3.0 4.0 20.0)", - WellKnownText.toWKT(new MultiPoint(Arrays.asList(new Point(1, 2, 10), new Point(4, 3, 20))))); + wkt.toWKT(new MultiPoint(Arrays.asList(new Point(1, 2, 10), new Point(4, 3, 20))))); assertEquals(new MultiPoint(Arrays.asList(new Point(1, 2, 10), new Point(4, 3, 20))), - WellKnownText.fromWKT("multipoint (2 1 10, 3 4 20)")); + wkt.fromWKT("multipoint (2 1 10, 3 4 20)")); - assertEquals("multipoint EMPTY", WellKnownText.toWKT(MultiPoint.EMPTY)); - assertEquals(MultiPoint.EMPTY, WellKnownText.fromWKT("multipoint EMPTY)")); + assertEquals("multipoint EMPTY", wkt.toWKT(MultiPoint.EMPTY)); + assertEquals(MultiPoint.EMPTY, wkt.fromWKT("multipoint EMPTY)")); } } diff --git a/libs/geo/src/test/java/org/elasticsearch/geo/geometry/MultiPolygonTests.java b/libs/geo/src/test/java/org/elasticsearch/geo/geometry/MultiPolygonTests.java index 1b82cfd3871a5..9918dfa546c82 100644 --- a/libs/geo/src/test/java/org/elasticsearch/geo/geometry/MultiPolygonTests.java +++ b/libs/geo/src/test/java/org/elasticsearch/geo/geometry/MultiPolygonTests.java @@ -19,6 +19,7 @@ package org.elasticsearch.geo.geometry; +import org.elasticsearch.geo.utils.GeographyValidator; import org.elasticsearch.geo.utils.WellKnownText; import java.io.IOException; @@ -40,14 +41,15 @@ protected MultiPolygon createTestInstance(boolean hasAlt) { } public void testBasicSerialization() throws IOException, ParseException { + WellKnownText wkt = new WellKnownText(true, new GeographyValidator(true)); assertEquals("multipolygon (((3.0 1.0, 4.0 2.0, 5.0 3.0, 3.0 1.0)))", - WellKnownText.toWKT(new MultiPolygon(Collections.singletonList( + wkt.toWKT(new MultiPolygon(Collections.singletonList( new Polygon(new LinearRing(new double[]{1, 2, 3, 1}, new double[]{3, 4, 5, 3})))))); assertEquals(new MultiPolygon(Collections.singletonList( new Polygon(new LinearRing(new double[]{1, 2, 3, 1}, new double[]{3, 4, 5, 3})))), - WellKnownText.fromWKT("multipolygon (((3.0 1.0, 4.0 2.0, 5.0 3.0, 3.0 1.0)))")); + wkt.fromWKT("multipolygon (((3.0 1.0, 4.0 2.0, 5.0 3.0, 3.0 1.0)))")); - assertEquals("multipolygon EMPTY", WellKnownText.toWKT(MultiPolygon.EMPTY)); - assertEquals(MultiPolygon.EMPTY, WellKnownText.fromWKT("multipolygon EMPTY)")); + assertEquals("multipolygon EMPTY", wkt.toWKT(MultiPolygon.EMPTY)); + assertEquals(MultiPolygon.EMPTY, wkt.fromWKT("multipolygon EMPTY)")); } } diff --git a/libs/geo/src/test/java/org/elasticsearch/geo/geometry/PointTests.java b/libs/geo/src/test/java/org/elasticsearch/geo/geometry/PointTests.java index d480f44e30a44..82e8fc40e75e9 100644 --- a/libs/geo/src/test/java/org/elasticsearch/geo/geometry/PointTests.java +++ b/libs/geo/src/test/java/org/elasticsearch/geo/geometry/PointTests.java @@ -19,6 +19,8 @@ package org.elasticsearch.geo.geometry; +import org.elasticsearch.geo.utils.GeographyValidator; +import org.elasticsearch.geo.utils.GeometryValidator; import org.elasticsearch.geo.utils.WellKnownText; import java.io.IOException; @@ -31,21 +33,29 @@ protected Point createTestInstance(boolean hasAlt) { } public void testBasicSerialization() throws IOException, ParseException { - assertEquals("point (20.0 10.0)", WellKnownText.toWKT(new Point(10, 20))); - assertEquals(new Point(10, 20), WellKnownText.fromWKT("point (20.0 10.0)")); + WellKnownText wkt = new WellKnownText(true, new GeographyValidator(true)); + assertEquals("point (20.0 10.0)", wkt.toWKT(new Point(10, 20))); + assertEquals(new Point(10, 20), wkt.fromWKT("point (20.0 10.0)")); - assertEquals("point (20.0 10.0 100.0)", WellKnownText.toWKT(new Point(10, 20, 100))); - assertEquals(new Point(10, 20, 100), WellKnownText.fromWKT("point (20.0 10.0 100.0)")); + assertEquals("point (20.0 10.0 100.0)", wkt.toWKT(new Point(10, 20, 100))); + assertEquals(new Point(10, 20, 100), wkt.fromWKT("point (20.0 10.0 100.0)")); - assertEquals("point EMPTY", WellKnownText.toWKT(Point.EMPTY)); - assertEquals(Point.EMPTY, WellKnownText.fromWKT("point EMPTY)")); + assertEquals("point EMPTY", wkt.toWKT(Point.EMPTY)); + assertEquals(Point.EMPTY, wkt.fromWKT("point EMPTY)")); } public void testInitValidation() { - IllegalArgumentException ex = expectThrows(IllegalArgumentException.class, () -> new Point(100, 10)); + GeometryValidator validator = new GeographyValidator(true); + IllegalArgumentException ex = expectThrows(IllegalArgumentException.class, () -> validator.validate(new Point(100, 10))); assertEquals("invalid latitude 100.0; must be between -90.0 and 90.0", ex.getMessage()); - ex = expectThrows(IllegalArgumentException.class, () -> new Point(10, 500)); + ex = expectThrows(IllegalArgumentException.class, () -> validator.validate(new Point(10, 500))); assertEquals("invalid longitude 500.0; must be between -180.0 and 180.0", ex.getMessage()); } + + public void testWKTValidation() { + IllegalArgumentException ex = expectThrows(IllegalArgumentException.class, + () -> new WellKnownText(randomBoolean(), new GeographyValidator(false)).fromWKT("point (20.0 10.0 100.0)")); + assertEquals("found Z value [100.0] but [ignore_z_value] parameter is [false]", ex.getMessage()); + } } diff --git a/libs/geo/src/test/java/org/elasticsearch/geo/geometry/PolygonTests.java b/libs/geo/src/test/java/org/elasticsearch/geo/geometry/PolygonTests.java index fd3032435891c..adbe1f38cdcc0 100644 --- a/libs/geo/src/test/java/org/elasticsearch/geo/geometry/PolygonTests.java +++ b/libs/geo/src/test/java/org/elasticsearch/geo/geometry/PolygonTests.java @@ -19,6 +19,7 @@ package org.elasticsearch.geo.geometry; +import org.elasticsearch.geo.utils.GeographyValidator; import org.elasticsearch.geo.utils.WellKnownText; import java.io.IOException; @@ -32,18 +33,28 @@ protected Polygon createTestInstance(boolean hasAlt) { } public void testBasicSerialization() throws IOException, ParseException { + WellKnownText wkt = new WellKnownText(true, new GeographyValidator(true)); assertEquals("polygon ((3.0 1.0, 4.0 2.0, 5.0 3.0, 3.0 1.0))", - WellKnownText.toWKT(new Polygon(new LinearRing(new double[]{1, 2, 3, 1}, new double[]{3, 4, 5, 3})))); + wkt.toWKT(new Polygon(new LinearRing(new double[]{1, 2, 3, 1}, new double[]{3, 4, 5, 3})))); assertEquals(new Polygon(new LinearRing(new double[]{1, 2, 3, 1}, new double[]{3, 4, 5, 3})), - WellKnownText.fromWKT("polygon ((3 1, 4 2, 5 3, 3 1))")); + wkt.fromWKT("polygon ((3 1, 4 2, 5 3, 3 1))")); assertEquals("polygon ((3.0 1.0 5.0, 4.0 2.0 4.0, 5.0 3.0 3.0, 3.0 1.0 5.0))", - WellKnownText.toWKT(new Polygon(new LinearRing(new double[]{1, 2, 3, 1}, new double[]{3, 4, 5, 3}, new double[]{5, 4, 3, 5})))); + wkt.toWKT(new Polygon(new LinearRing(new double[]{1, 2, 3, 1}, new double[]{3, 4, 5, 3}, new double[]{5, 4, 3, 5})))); assertEquals(new Polygon(new LinearRing(new double[]{1, 2, 3, 1}, new double[]{3, 4, 5, 3}, new double[]{5, 4, 3, 5})), - WellKnownText.fromWKT("polygon ((3 1 5, 4 2 4, 5 3 3, 3 1 5))")); + wkt.fromWKT("polygon ((3 1 5, 4 2 4, 5 3 3, 3 1 5))")); - assertEquals("polygon EMPTY", WellKnownText.toWKT(Polygon.EMPTY)); - assertEquals(Polygon.EMPTY, WellKnownText.fromWKT("polygon EMPTY)")); + // Auto closing in coerce mode + assertEquals(new Polygon(new LinearRing(new double[]{1, 2, 3, 1}, new double[]{3, 4, 5, 3})), + wkt.fromWKT("polygon ((3 1, 4 2, 5 3))")); + assertEquals(new Polygon(new LinearRing(new double[]{1, 2, 3, 1}, new double[]{3, 4, 5, 3}, new double[]{5, 4, 3, 5})), + wkt.fromWKT("polygon ((3 1 5, 4 2 4, 5 3 3))")); + assertEquals(new Polygon(new LinearRing(new double[]{1, 2, 3, 1}, new double[]{3, 4, 5, 3}), + Collections.singletonList(new LinearRing(new double[]{1.5, 1.5, 1.0, 1.5}, new double[]{0.5, 2.5, 2.0, 0.5}))), + wkt.fromWKT("polygon ((3 1, 4 2, 5 3, 3 1), (0.5 1.5, 2.5 1.5, 2.0 1.0))")); + + assertEquals("polygon EMPTY", wkt.toWKT(Polygon.EMPTY)); + assertEquals(Polygon.EMPTY, wkt.fromWKT("polygon EMPTY)")); } public void testInitValidation() { @@ -60,4 +71,21 @@ public void testInitValidation() { Collections.singletonList(new LinearRing(new double[]{1, 2, 3, 1}, new double[]{3, 4, 5, 3})))); assertEquals("holes must have the same number of dimensions as the polygon", ex.getMessage()); } + + public void testWKTValidation() { + IllegalArgumentException ex = expectThrows(IllegalArgumentException.class, + () -> new WellKnownText(false, new GeographyValidator(true)).fromWKT("polygon ((3 1 5, 4 2 4, 5 3 3))")); + assertEquals("first and last points of the linear ring must be the same (it must close itself): " + + "lats[0]=1.0 lats[2]=3.0 lons[0]=3.0 lons[2]=5.0 alts[0]=5.0 alts[2]=3.0", ex.getMessage()); + + ex = expectThrows(IllegalArgumentException.class, + () -> new WellKnownText(randomBoolean(), new GeographyValidator(false)).fromWKT("polygon ((3 1 5, 4 2 4, 5 3 3, 3 1 5))")); + assertEquals("found Z value [5.0] but [ignore_z_value] parameter is [false]", ex.getMessage()); + + ex = expectThrows(IllegalArgumentException.class, + () -> new WellKnownText(false, new GeographyValidator(randomBoolean())).fromWKT( + "polygon ((3 1, 4 2, 5 3, 3 1), (0.5 1.5, 2.5 1.5, 2.0 1.0))")); + assertEquals("first and last points of the linear ring must be the same (it must close itself): " + + "lats[0]=1.5 lats[2]=1.0 lons[0]=0.5 lons[2]=2.0", ex.getMessage()); + } } diff --git a/libs/geo/src/test/java/org/elasticsearch/geo/geometry/RectangleTests.java b/libs/geo/src/test/java/org/elasticsearch/geo/geometry/RectangleTests.java index 14cb777f94bb9..8bd1494eb34a9 100644 --- a/libs/geo/src/test/java/org/elasticsearch/geo/geometry/RectangleTests.java +++ b/libs/geo/src/test/java/org/elasticsearch/geo/geometry/RectangleTests.java @@ -19,6 +19,8 @@ package org.elasticsearch.geo.geometry; +import org.elasticsearch.geo.utils.GeographyValidator; +import org.elasticsearch.geo.utils.GeometryValidator; import org.elasticsearch.geo.utils.WellKnownText; import java.io.IOException; @@ -32,24 +34,30 @@ protected Rectangle createTestInstance(boolean hasAlt) { } public void testBasicSerialization() throws IOException, ParseException { - assertEquals("bbox (10.0, 20.0, 40.0, 30.0)", WellKnownText.toWKT(new Rectangle(30, 40, 10, 20))); - assertEquals(new Rectangle(30, 40, 10, 20), WellKnownText.fromWKT("bbox (10.0, 20.0, 40.0, 30.0)")); + WellKnownText wkt = new WellKnownText(true, new GeographyValidator(true)); + assertEquals("bbox (10.0, 20.0, 40.0, 30.0)", wkt.toWKT(new Rectangle(30, 40, 10, 20))); + assertEquals(new Rectangle(30, 40, 10, 20), wkt.fromWKT("bbox (10.0, 20.0, 40.0, 30.0)")); - assertEquals("bbox EMPTY", WellKnownText.toWKT(Rectangle.EMPTY)); - assertEquals(Rectangle.EMPTY, WellKnownText.fromWKT("bbox EMPTY)")); + assertEquals("bbox EMPTY", wkt.toWKT(Rectangle.EMPTY)); + assertEquals(Rectangle.EMPTY, wkt.fromWKT("bbox EMPTY)")); } public void testInitValidation() { - IllegalArgumentException ex = expectThrows(IllegalArgumentException.class, () -> new Rectangle(100, 1, 2, 3)); + GeometryValidator validator = new GeographyValidator(true); + IllegalArgumentException ex = expectThrows(IllegalArgumentException.class, + () -> validator.validate(new Rectangle(1, 100, 2, 3))); assertEquals("invalid latitude 100.0; must be between -90.0 and 90.0", ex.getMessage()); - ex = expectThrows(IllegalArgumentException.class, () -> new Rectangle(1, 2, 200, 3)); + ex = expectThrows(IllegalArgumentException.class, + () -> validator.validate(new Rectangle(1, 2, 200, 3))); assertEquals("invalid longitude 200.0; must be between -180.0 and 180.0", ex.getMessage()); - ex = expectThrows(IllegalArgumentException.class, () -> new Rectangle(2, 1, 2, 3)); + ex = expectThrows(IllegalArgumentException.class, + () -> validator.validate(new Rectangle(2, 1, 2, 3))); assertEquals("max lat cannot be less than min lat", ex.getMessage()); - ex = expectThrows(IllegalArgumentException.class, () -> new Rectangle(1, 2, 2, 3, 5, Double.NaN)); + ex = expectThrows(IllegalArgumentException.class, + () -> validator.validate(new Rectangle(1, 2, 2, 3, 5, Double.NaN))); assertEquals("only one altitude value is specified", ex.getMessage()); } } diff --git a/libs/grok/build.gradle b/libs/grok/build.gradle index 9ca02df35aabe..b7ae54e54c3af 100644 --- a/libs/grok/build.gradle +++ b/libs/grok/build.gradle @@ -17,16 +17,14 @@ * under the License. */ -archivesBaseName = 'elasticsearch-grok' - dependencies { compile 'org.jruby.joni:joni:2.1.6' // joni dependencies: - compile 'org.jruby.jcodings:jcodings:1.0.12' + compile 'org.jruby.jcodings:jcodings:1.0.44' - if (isEclipse == false || project.path == ":libs:grok-tests") { - testCompile("org.elasticsearch.test:framework:${version}") { - exclude group: 'org.elasticsearch', module: 'grok' + if (isEclipse == false || project.path == ":libs:elasticsearch-grok-tests") { + testCompile(project(":test:framework")) { + exclude group: 'org.elasticsearch', module: 'elasticsearch-grok' } } } @@ -38,7 +36,7 @@ forbiddenApisMain { if (isEclipse) { // in eclipse the project is under a fake root, we need to change around the source sets sourceSets { - if (project.path == ":libs:grok") { + if (project.path == ":libs:elasticsearch-grok") { main.java.srcDirs = ['java'] main.resources.srcDirs = ['resources'] } else { diff --git a/libs/grok/licenses/jcodings-1.0.12.jar.sha1 b/libs/grok/licenses/jcodings-1.0.12.jar.sha1 deleted file mode 100644 index b097e32ece493..0000000000000 --- a/libs/grok/licenses/jcodings-1.0.12.jar.sha1 +++ /dev/null @@ -1 +0,0 @@ -6bc17079fcaa8823ea8cd0d4c66516335b558db8 \ No newline at end of file diff --git a/libs/grok/licenses/jcodings-1.0.44.jar.sha1 b/libs/grok/licenses/jcodings-1.0.44.jar.sha1 new file mode 100644 index 0000000000000..4449009d3395e --- /dev/null +++ b/libs/grok/licenses/jcodings-1.0.44.jar.sha1 @@ -0,0 +1 @@ +a6884b2fd8fd9a56874db05afaa22435043a2e3e \ No newline at end of file diff --git a/libs/grok/src/test/eclipse-build.gradle b/libs/grok/src/test/eclipse-build.gradle index c5d791c166324..606aaffa121ab 100644 --- a/libs/grok/src/test/eclipse-build.gradle +++ b/libs/grok/src/test/eclipse-build.gradle @@ -3,5 +3,5 @@ apply from: '../../build.gradle' dependencies { - testCompile project(':libs:grok') + testCompile project(':libs:elasticsearch-grok') } diff --git a/libs/nio/build.gradle b/libs/nio/build.gradle index 66436bb040e18..d6d0eaea0ab83 100644 --- a/libs/nio/build.gradle +++ b/libs/nio/build.gradle @@ -19,25 +19,15 @@ apply plugin: 'nebula.maven-base-publish' apply plugin: 'nebula.maven-scm' -archivesBaseName = 'elasticsearch-nio' - -publishing { - publications { - nebula { - artifactId = archivesBaseName - } - } -} - dependencies { - compile "org.elasticsearch:elasticsearch-core:${version}" + compile project(':libs:elasticsearch-core') testCompile "com.carrotsearch.randomizedtesting:randomizedtesting-runner:${versions.randomizedrunner}" testCompile "junit:junit:${versions.junit}" testCompile "org.hamcrest:hamcrest:${versions.hamcrest}" - if (isEclipse == false || project.path == ":libs:nio-tests") { - testCompile("org.elasticsearch.test:framework:${version}") { + if (isEclipse == false || project.path == ":libs:elasticsearch-nio-tests") { + testCompile(project(":test:framework")) { exclude group: 'org.elasticsearch', module: 'elasticsearch-nio' } } @@ -46,7 +36,7 @@ dependencies { if (isEclipse) { // in eclipse the project is under a fake root, we need to change around the source sets sourceSets { - if (project.path == ":libs:nio") { + if (project.path == ":libs:elasticsearch-nio") { main.java.srcDirs = ['java'] main.resources.srcDirs = ['resources'] } else { diff --git a/libs/nio/src/main/java/org/elasticsearch/nio/BytesChannelContext.java b/libs/nio/src/main/java/org/elasticsearch/nio/BytesChannelContext.java index 8b174eac468e1..211e609ba4c0d 100644 --- a/libs/nio/src/main/java/org/elasticsearch/nio/BytesChannelContext.java +++ b/libs/nio/src/main/java/org/elasticsearch/nio/BytesChannelContext.java @@ -21,19 +21,12 @@ import java.io.IOException; import java.util.function.Consumer; -import java.util.function.Predicate; public class BytesChannelContext extends SocketChannelContext { public BytesChannelContext(NioSocketChannel channel, NioSelector selector, Consumer exceptionHandler, - ReadWriteHandler handler, InboundChannelBuffer channelBuffer) { - this(channel, selector, exceptionHandler, handler, channelBuffer, ALWAYS_ALLOW_CHANNEL); - } - - public BytesChannelContext(NioSocketChannel channel, NioSelector selector, Consumer exceptionHandler, - ReadWriteHandler handler, InboundChannelBuffer channelBuffer, - Predicate allowChannelPredicate) { - super(channel, selector, exceptionHandler, handler, channelBuffer, allowChannelPredicate); + NioChannelHandler handler, InboundChannelBuffer channelBuffer) { + super(channel, selector, exceptionHandler, handler, channelBuffer); } @Override diff --git a/libs/nio/src/main/java/org/elasticsearch/nio/BytesWriteHandler.java b/libs/nio/src/main/java/org/elasticsearch/nio/BytesWriteHandler.java index 07333aa2eebc4..48d83d2169248 100644 --- a/libs/nio/src/main/java/org/elasticsearch/nio/BytesWriteHandler.java +++ b/libs/nio/src/main/java/org/elasticsearch/nio/BytesWriteHandler.java @@ -24,7 +24,7 @@ import java.util.List; import java.util.function.BiConsumer; -public abstract class BytesWriteHandler implements ReadWriteHandler { +public abstract class BytesWriteHandler implements NioChannelHandler { private static final List EMPTY_LIST = Collections.emptyList(); @@ -48,6 +48,11 @@ public List pollFlushOperations() { return EMPTY_LIST; } + @Override + public boolean closeNow() { + return false; + } + @Override public void close() {} } diff --git a/libs/nio/src/main/java/org/elasticsearch/nio/ChannelFactory.java b/libs/nio/src/main/java/org/elasticsearch/nio/ChannelFactory.java index 97750af2432c3..423168b35d09b 100644 --- a/libs/nio/src/main/java/org/elasticsearch/nio/ChannelFactory.java +++ b/libs/nio/src/main/java/org/elasticsearch/nio/ChannelFactory.java @@ -177,7 +177,9 @@ SocketChannel openNioChannel(InetSocketAddress remoteAddress) throws IOException SocketChannel acceptNioChannel(ServerChannelContext serverContext) throws IOException { ServerSocketChannel rawChannel = serverContext.getChannel().getRawChannel(); + assert rawChannel.isBlocking() == false; SocketChannel socketChannel = accept(rawChannel); + assert rawChannel.isBlocking() == false; if (socketChannel == null) { return null; } diff --git a/libs/nio/src/main/java/org/elasticsearch/nio/DelegatingHandler.java b/libs/nio/src/main/java/org/elasticsearch/nio/DelegatingHandler.java new file mode 100644 index 0000000000000..d928b0bf9349d --- /dev/null +++ b/libs/nio/src/main/java/org/elasticsearch/nio/DelegatingHandler.java @@ -0,0 +1,68 @@ +/* + * 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.nio; + +import java.io.IOException; +import java.util.List; +import java.util.function.BiConsumer; + +public abstract class DelegatingHandler implements NioChannelHandler { + + private NioChannelHandler delegate; + + public DelegatingHandler(NioChannelHandler delegate) { + this.delegate = delegate; + } + + @Override + public void channelRegistered() { + this.delegate.channelRegistered(); + } + + @Override + public WriteOperation createWriteOperation(SocketChannelContext context, Object message, BiConsumer listener) { + return delegate.createWriteOperation(context, message, listener); + } + + @Override + public List writeToBytes(WriteOperation writeOperation) { + return delegate.writeToBytes(writeOperation); + } + + @Override + public List pollFlushOperations() { + return delegate.pollFlushOperations(); + } + + @Override + public int consumeReads(InboundChannelBuffer channelBuffer) throws IOException { + return delegate.consumeReads(channelBuffer); + } + + @Override + public boolean closeNow() { + return delegate.closeNow(); + } + + @Override + public void close() throws IOException { + delegate.close(); + } +} diff --git a/libs/nio/src/main/java/org/elasticsearch/nio/ReadWriteHandler.java b/libs/nio/src/main/java/org/elasticsearch/nio/NioChannelHandler.java similarity index 90% rename from libs/nio/src/main/java/org/elasticsearch/nio/ReadWriteHandler.java rename to libs/nio/src/main/java/org/elasticsearch/nio/NioChannelHandler.java index 92b276ad2d6da..61bda9a450768 100644 --- a/libs/nio/src/main/java/org/elasticsearch/nio/ReadWriteHandler.java +++ b/libs/nio/src/main/java/org/elasticsearch/nio/NioChannelHandler.java @@ -24,9 +24,9 @@ import java.util.function.BiConsumer; /** - * Implements the application specific logic for handling inbound and outbound messages for a channel. + * Implements the application specific logic for handling channel operations. */ -public interface ReadWriteHandler { +public interface NioChannelHandler { /** * This method is called when the channel is registered with its selector. @@ -72,5 +72,12 @@ public interface ReadWriteHandler { */ int consumeReads(InboundChannelBuffer channelBuffer) throws IOException; + /** + * This method indicates if the underlying channel should be closed. + * + * @return if the channel should be closed + */ + boolean closeNow(); + void close() throws IOException; } diff --git a/libs/nio/src/main/java/org/elasticsearch/nio/NioSelector.java b/libs/nio/src/main/java/org/elasticsearch/nio/NioSelector.java index 1d0af24ae2cba..fff07f625ff13 100644 --- a/libs/nio/src/main/java/org/elasticsearch/nio/NioSelector.java +++ b/libs/nio/src/main/java/org/elasticsearch/nio/NioSelector.java @@ -61,6 +61,7 @@ public class NioSelector implements Closeable { private final AtomicBoolean isClosed = new AtomicBoolean(false); private final CompletableFuture isRunningFuture = new CompletableFuture<>(); private final AtomicReference thread = new AtomicReference<>(null); + private final AtomicBoolean wokenUp = new AtomicBoolean(false); public NioSelector(EventHandler eventHandler) throws IOException { this(eventHandler, Selector.open()); @@ -153,7 +154,7 @@ void singleLoop() { preSelect(); long nanosUntilNextTask = taskScheduler.nanosUntilNextTask(System.nanoTime()); int ready; - if (nanosUntilNextTask == 0) { + if (wokenUp.getAndSet(false) || nanosUntilNextTask == 0) { ready = selector.selectNow(); } else { long millisUntilNextTask = TimeUnit.NANOSECONDS.toMillis(nanosUntilNextTask); @@ -221,13 +222,10 @@ void processKey(SelectionKey selectionKey) { if (selectionKey.isAcceptable()) { assert context instanceof ServerChannelContext : "Only server channels can receive accept events"; ServerChannelContext serverChannelContext = (ServerChannelContext) context; - int ops = selectionKey.readyOps(); - if ((ops & SelectionKey.OP_ACCEPT) != 0) { - try { - eventHandler.acceptChannel(serverChannelContext); - } catch (IOException e) { - eventHandler.acceptException(serverChannelContext, e); - } + try { + eventHandler.acceptChannel(serverChannelContext); + } catch (IOException e) { + eventHandler.acceptException(serverChannelContext, e); } } else { assert context instanceof SocketChannelContext : "Only sockets channels can receive non-accept events"; @@ -279,29 +277,36 @@ private void handleTask(Runnable task) { /** * Queues a write operation to be handled by the event loop. This can be called by any thread and is the - * api available for non-selector threads to schedule writes. + * api available for non-selector threads to schedule writes. When invoked from the selector thread the write will be executed + * right away. * * @param writeOperation to be queued */ public void queueWrite(WriteOperation writeOperation) { - queuedWrites.offer(writeOperation); - if (isOpen() == false) { - boolean wasRemoved = queuedWrites.remove(writeOperation); - if (wasRemoved) { - writeOperation.getListener().accept(null, new ClosedSelectorException()); - } + if (isOnCurrentThread()) { + writeToChannel(writeOperation); } else { - wakeup(); + queuedWrites.offer(writeOperation); + if (isOpen() == false) { + boolean wasRemoved = queuedWrites.remove(writeOperation); + if (wasRemoved) { + writeOperation.getListener().accept(null, new ClosedSelectorException()); + } + } else { + wakeup(); + } } } public void queueChannelClose(NioChannel channel) { ChannelContext context = channel.getContext(); assert context.getSelector() == this : "Must schedule a channel for closure with its selector"; - channelsToClose.offer(context); if (isOnCurrentThread() == false) { + channelsToClose.offer(context); ensureSelectorOpenForEnqueuing(channelsToClose, context); wakeup(); + } else { + closeChannel(context); } } @@ -313,9 +318,13 @@ public void queueChannelClose(NioChannel channel) { */ public void scheduleForRegistration(NioChannel channel) { ChannelContext context = channel.getContext(); - channelsToRegister.add(context); - ensureSelectorOpenForEnqueuing(channelsToRegister, context); - wakeup(); + if (isOnCurrentThread() == false) { + channelsToRegister.add(context); + ensureSelectorOpenForEnqueuing(channelsToRegister, context); + wakeup(); + } else { + registerChannel(context); + } } /** @@ -326,7 +335,7 @@ public void scheduleForRegistration(NioChannel channel) { * * @param writeOperation to be queued in a channel's buffer */ - public void writeToChannel(WriteOperation writeOperation) { + private void writeToChannel(WriteOperation writeOperation) { assertOnSelectorThread(); SocketChannelContext context = writeOperation.getChannel(); // If the channel does not currently have anything that is ready to flush, we should flush after @@ -380,8 +389,10 @@ private void cleanupPendingWrites() { } private void wakeup() { - // TODO: Do we need the wakeup optimizations that some other libraries use? - selector.wakeup(); + assert isOnCurrentThread() == false; + if (wokenUp.compareAndSet(false, true)) { + selector.wakeup(); + } } private void handleWrite(SocketChannelContext context) { @@ -414,30 +425,38 @@ private void attemptConnect(SocketChannelContext context, boolean connectEvent) private void setUpNewChannels() { ChannelContext newChannel; while ((newChannel = this.channelsToRegister.poll()) != null) { - assert newChannel.getSelector() == this : "The channel must be registered with the selector with which it was created"; - try { - if (newChannel.isOpen()) { - eventHandler.handleRegistration(newChannel); - if (newChannel instanceof SocketChannelContext) { - attemptConnect((SocketChannelContext) newChannel, false); - } - } else { - eventHandler.registrationException(newChannel, new ClosedChannelException()); + registerChannel(newChannel); + } + } + + private void registerChannel(ChannelContext newChannel) { + assert newChannel.getSelector() == this : "The channel must be registered with the selector with which it was created"; + try { + if (newChannel.isOpen()) { + eventHandler.handleRegistration(newChannel); + if (newChannel instanceof SocketChannelContext) { + attemptConnect((SocketChannelContext) newChannel, false); } - } catch (Exception e) { - eventHandler.registrationException(newChannel, e); + } else { + eventHandler.registrationException(newChannel, new ClosedChannelException()); } + } catch (Exception e) { + eventHandler.registrationException(newChannel, e); } } private void closePendingChannels() { ChannelContext channelContext; while ((channelContext = channelsToClose.poll()) != null) { - try { - eventHandler.handleClose(channelContext); - } catch (Exception e) { - eventHandler.closeException(channelContext, e); - } + closeChannel(channelContext); + } + } + + private void closeChannel(final ChannelContext channelContext) { + try { + eventHandler.handleClose(channelContext); + } catch (Exception e) { + eventHandler.closeException(channelContext, e); } } @@ -470,7 +489,7 @@ private void handleQueuedWrites() { * @param the object type */ private void ensureSelectorOpenForEnqueuing(ConcurrentLinkedQueue queue, O objectAdded) { - if (isOpen() == false && isOnCurrentThread() == false) { + if (isOpen() == false) { if (queue.remove(objectAdded)) { throw new IllegalStateException("selector is already closed"); } diff --git a/libs/nio/src/main/java/org/elasticsearch/nio/SocketChannelContext.java b/libs/nio/src/main/java/org/elasticsearch/nio/SocketChannelContext.java index 22d85472126c1..f77ccb17aef39 100644 --- a/libs/nio/src/main/java/org/elasticsearch/nio/SocketChannelContext.java +++ b/libs/nio/src/main/java/org/elasticsearch/nio/SocketChannelContext.java @@ -32,7 +32,6 @@ import java.util.concurrent.atomic.AtomicBoolean; import java.util.function.BiConsumer; import java.util.function.Consumer; -import java.util.function.Predicate; /** * This context should implement the specific logic for a channel. When a channel receives a notification @@ -45,13 +44,10 @@ */ public abstract class SocketChannelContext extends ChannelContext { - protected static final Predicate ALWAYS_ALLOW_CHANNEL = (c) -> true; - protected final NioSocketChannel channel; protected final InboundChannelBuffer channelBuffer; protected final AtomicBoolean isClosing = new AtomicBoolean(false); - private final ReadWriteHandler readWriteHandler; - private final Predicate allowChannelPredicate; + private final NioChannelHandler readWriteHandler; private final NioSelector selector; private final CompletableContext connectContext = new CompletableContext<>(); private final LinkedList pendingFlushes = new LinkedList<>(); @@ -59,14 +55,12 @@ public abstract class SocketChannelContext extends ChannelContext private Exception connectException; protected SocketChannelContext(NioSocketChannel channel, NioSelector selector, Consumer exceptionHandler, - ReadWriteHandler readWriteHandler, InboundChannelBuffer channelBuffer, - Predicate allowChannelPredicate) { + NioChannelHandler readWriteHandler, InboundChannelBuffer channelBuffer) { super(channel.getRawChannel(), exceptionHandler); this.selector = selector; this.channel = channel; this.readWriteHandler = readWriteHandler; this.channelBuffer = channelBuffer; - this.allowChannelPredicate = allowChannelPredicate; } @Override @@ -135,13 +129,7 @@ public void sendMessage(Object message, BiConsumer listener) { WriteOperation writeOperation = readWriteHandler.createWriteOperation(this, message, listener); - NioSelector selector = getSelector(); - if (selector.isOnCurrentThread() == false) { - selector.queueWrite(writeOperation); - return; - } - - selector.writeToChannel(writeOperation); + getSelector().queueWrite(writeOperation); } public void queueWriteOperation(WriteOperation writeOperation) { @@ -171,9 +159,6 @@ protected FlushOperation getPendingFlush() { protected void register() throws IOException { super.register(); readWriteHandler.channelRegistered(); - if (allowChannelPredicate.test(channel) == false) { - closeNow = true; - } } @Override @@ -233,7 +218,7 @@ public boolean readyForFlush() { public abstract boolean selectorShouldClose(); protected boolean closeNow() { - return closeNow; + return closeNow || readWriteHandler.closeNow(); } protected void setCloseNow() { @@ -278,7 +263,7 @@ protected int readFromChannel(InboundChannelBuffer channelBuffer) throws IOExcep // Currently we limit to 64KB. This is a trade-off which means more syscalls, in exchange for less // copying. - private final int WRITE_LIMIT = 1 << 16; + private static final int WRITE_LIMIT = 1 << 16; protected int flushToChannel(FlushOperation flushOperation) throws IOException { ByteBuffer ioBuffer = getSelector().getIoBuffer(); diff --git a/libs/nio/src/main/java/org/elasticsearch/nio/WriteOperation.java b/libs/nio/src/main/java/org/elasticsearch/nio/WriteOperation.java index 3d17519be7e1f..b5f60bd28a1d6 100644 --- a/libs/nio/src/main/java/org/elasticsearch/nio/WriteOperation.java +++ b/libs/nio/src/main/java/org/elasticsearch/nio/WriteOperation.java @@ -23,7 +23,7 @@ /** * This is a basic write operation that can be queued with a channel. The only requirements of a write * operation is that is has a listener and a reference to its channel. The actual conversion of the write - * operation implementation to bytes will be performed by the {@link ReadWriteHandler}. + * operation implementation to bytes will be performed by the {@link NioChannelHandler}. */ public interface WriteOperation { diff --git a/libs/nio/src/test/eclipse-build.gradle b/libs/nio/src/test/eclipse-build.gradle index e30e76b0da59f..f7e152311ad5d 100644 --- a/libs/nio/src/test/eclipse-build.gradle +++ b/libs/nio/src/test/eclipse-build.gradle @@ -3,5 +3,5 @@ apply from: '../../build.gradle' dependencies { - testCompile project(':libs:nio') + testCompile project(':libs:elasticsearch-nio') } diff --git a/libs/nio/src/test/java/org/elasticsearch/nio/EventHandlerTests.java b/libs/nio/src/test/java/org/elasticsearch/nio/EventHandlerTests.java index 1b42e8be60d7c..578890b152ff1 100644 --- a/libs/nio/src/test/java/org/elasticsearch/nio/EventHandlerTests.java +++ b/libs/nio/src/test/java/org/elasticsearch/nio/EventHandlerTests.java @@ -44,7 +44,7 @@ public class EventHandlerTests extends ESTestCase { private Consumer channelExceptionHandler; private Consumer genericExceptionHandler; - private ReadWriteHandler readWriteHandler; + private NioChannelHandler readWriteHandler; private EventHandler handler; private DoNotRegisterSocketContext context; private DoNotRegisterServerContext serverContext; @@ -56,7 +56,7 @@ public class EventHandlerTests extends ESTestCase { public void setUpHandler() throws IOException { channelExceptionHandler = mock(Consumer.class); genericExceptionHandler = mock(Consumer.class); - readWriteHandler = mock(ReadWriteHandler.class); + readWriteHandler = mock(NioChannelHandler.class); channelFactory = mock(ChannelFactory.class); NioSelector selector = mock(NioSelector.class); ArrayList selectors = new ArrayList<>(); @@ -260,7 +260,7 @@ private class DoNotRegisterSocketContext extends BytesChannelContext { DoNotRegisterSocketContext(NioSocketChannel channel, NioSelector selector, Consumer exceptionHandler, - ReadWriteHandler handler) { + NioChannelHandler handler) { super(channel, selector, exceptionHandler, handler, InboundChannelBuffer.allocatingInstance()); } diff --git a/libs/nio/src/test/java/org/elasticsearch/nio/NioSelectorTests.java b/libs/nio/src/test/java/org/elasticsearch/nio/NioSelectorTests.java index 7a641315fe285..55d2e645cadee 100644 --- a/libs/nio/src/test/java/org/elasticsearch/nio/NioSelectorTests.java +++ b/libs/nio/src/test/java/org/elasticsearch/nio/NioSelectorTests.java @@ -19,7 +19,9 @@ package org.elasticsearch.nio; +import org.elasticsearch.common.CheckedRunnable; import org.elasticsearch.common.unit.TimeValue; +import org.elasticsearch.common.util.concurrent.AbstractRunnable; import org.elasticsearch.test.ESTestCase; import org.junit.Before; import org.mockito.ArgumentCaptor; @@ -108,14 +110,14 @@ public void testQueueChannelForClosed() throws IOException { } @SuppressWarnings({"unchecked", "rawtypes"}) - public void testCloseException() throws IOException { + public void testCloseException() throws IOException, InterruptedException { IOException ioException = new IOException(); NioChannel channel = mock(NioChannel.class); ChannelContext context = mock(ChannelContext.class); when(channel.getContext()).thenReturn(context); when(context.getSelector()).thenReturn(selector); - selector.queueChannelClose(channel); + executeOnNewThread(() -> selector.queueChannelClose(channel)); doThrow(ioException).when(eventHandler).handleClose(context); @@ -198,9 +200,10 @@ public void testIOExceptionWhileSelect() throws IOException { verify(eventHandler).selectorException(ioException); } - public void testSelectorClosedIfOpenAndEventLoopNotRunning() throws IOException { + public void testSelectorClosedIfOpenAndEventLoopNotRunning() throws Exception { when(rawSelector.isOpen()).thenReturn(true); - selector.close(); + executeOnNewThread(() -> selector.close()); + verify(rawSelector).close(); } @@ -222,8 +225,7 @@ public void testClosedServerChannelWillNotBeRegistered() { } public void testRegisterServerChannelFailsDueToException() throws Exception { - selector.scheduleForRegistration(serverChannel); - + executeOnNewThread(() -> selector.scheduleForRegistration(serverChannel)); ClosedChannelException closedChannelException = new ClosedChannelException(); doThrow(closedChannelException).when(eventHandler).handleRegistration(serverChannelContext); @@ -242,16 +244,18 @@ public void testClosedSocketChannelWillNotBeRegistered() throws Exception { verify(eventHandler, times(0)).handleConnect(channelContext); } - public void testRegisterSocketChannelFailsDueToException() throws Exception { - selector.scheduleForRegistration(channel); + public void testRegisterSocketChannelFailsDueToException() throws InterruptedException { + executeOnNewThread(() -> { + selector.scheduleForRegistration(channel); - ClosedChannelException closedChannelException = new ClosedChannelException(); - doThrow(closedChannelException).when(eventHandler).handleRegistration(channelContext); + ClosedChannelException closedChannelException = new ClosedChannelException(); + doThrow(closedChannelException).when(eventHandler).handleRegistration(channelContext); - selector.preSelect(); + selector.preSelect(); - verify(eventHandler).registrationException(channelContext, closedChannelException); - verify(eventHandler, times(0)).handleConnect(channelContext); + verify(eventHandler).registrationException(channelContext, closedChannelException); + verify(eventHandler, times(0)).handleConnect(channelContext); + }); } public void testAcceptEvent() throws IOException { @@ -292,17 +296,17 @@ public void testSuccessfullyRegisterChannelWillAttemptConnect() throws Exception } public void testQueueWriteWhenNotRunning() throws Exception { - selector.close(); - - selector.queueWrite(new FlushReadyWrite(channelContext, buffers, listener)); - + executeOnNewThread(() -> { + selector.close(); + selector.queueWrite(new FlushReadyWrite(channelContext, buffers, listener)); + }); verify(listener).accept(isNull(Void.class), any(ClosedSelectorException.class)); } public void testQueueWriteChannelIsClosed() throws Exception { WriteOperation writeOperation = new FlushReadyWrite(channelContext, buffers, listener); - selector.queueWrite(writeOperation); + executeOnNewThread(() -> selector.queueWrite(writeOperation)); when(channelContext.isOpen()).thenReturn(false); selector.preSelect(); @@ -315,7 +319,7 @@ public void testQueueWriteSelectionKeyThrowsException() throws Exception { WriteOperation writeOperation = new FlushReadyWrite(channelContext, buffers, listener); CancelledKeyException cancelledKeyException = new CancelledKeyException(); - selector.queueWrite(writeOperation); + executeOnNewThread(() -> selector.queueWrite(writeOperation)); when(channelContext.getSelectionKey()).thenReturn(selectionKey); when(selectionKey.interestOps(anyInt())).thenThrow(cancelledKeyException); @@ -327,7 +331,7 @@ public void testQueueWriteSelectionKeyThrowsException() throws Exception { public void testQueueWriteSuccessful() throws Exception { WriteOperation writeOperation = new FlushReadyWrite(channelContext, buffers, listener); - selector.queueWrite(writeOperation); + executeOnNewThread(() -> selector.queueWrite(writeOperation)); assertTrue((selectionKey.interestOps() & SelectionKey.OP_WRITE) == 0); @@ -343,7 +347,7 @@ public void testQueueDirectlyInChannelBufferSuccessful() throws Exception { assertEquals(0, (selectionKey.interestOps() & SelectionKey.OP_WRITE)); when(channelContext.readyForFlush()).thenReturn(true); - selector.writeToChannel(writeOperation); + selector.queueWrite(writeOperation); verify(channelContext).queueWriteOperation(writeOperation); verify(eventHandler, times(0)).handleWrite(channelContext); @@ -357,7 +361,7 @@ public void testShouldFlushIfNoPendingFlushes() throws Exception { assertEquals(0, (selectionKey.interestOps() & SelectionKey.OP_WRITE)); when(channelContext.readyForFlush()).thenReturn(false); - selector.writeToChannel(writeOperation); + selector.queueWrite(writeOperation); verify(channelContext).queueWriteOperation(writeOperation); verify(eventHandler).handleWrite(channelContext); @@ -374,7 +378,7 @@ public void testQueueDirectlyInChannelBufferSelectionKeyThrowsException() throws when(channelContext.getSelectionKey()).thenReturn(selectionKey); when(channelContext.readyForFlush()).thenReturn(false); when(selectionKey.interestOps(anyInt())).thenThrow(cancelledKeyException); - selector.writeToChannel(writeOperation); + selector.queueWrite(writeOperation); verify(channelContext, times(0)).queueWriteOperation(writeOperation); verify(eventHandler, times(0)).handleWrite(channelContext); @@ -477,14 +481,17 @@ public void testWillCallPostHandleAfterChannelHandling() throws Exception { public void testCleanup() throws Exception { NioSocketChannel unregisteredChannel = mock(NioSocketChannel.class); SocketChannelContext unregisteredContext = mock(SocketChannelContext.class); + when(unregisteredContext.getSelector()).thenReturn(selector); when(unregisteredChannel.getContext()).thenReturn(unregisteredContext); - selector.scheduleForRegistration(channel); + executeOnNewThread(() -> selector.scheduleForRegistration(channel)); selector.preSelect(); - selector.queueWrite(new FlushReadyWrite(channelContext, buffers, listener)); - selector.scheduleForRegistration(unregisteredChannel); + executeOnNewThread(() -> { + selector.queueWrite(new FlushReadyWrite(channelContext, buffers, listener)); + selector.scheduleForRegistration(unregisteredChannel); + }); TestSelectionKey testSelectionKey = new TestSelectionKey(0); testSelectionKey.attach(channelContext); @@ -496,4 +503,20 @@ public void testCleanup() throws Exception { verify(eventHandler).handleClose(channelContext); verify(eventHandler).handleClose(unregisteredContext); } + + private static void executeOnNewThread(CheckedRunnable runnable) throws InterruptedException { + final Thread thread = new Thread(new AbstractRunnable() { + @Override + protected void doRun() throws Exception { + runnable.run(); + } + + @Override + public void onFailure(Exception e) { + throw new AssertionError(e); + } + }); + thread.start(); + thread.join(); + } } diff --git a/libs/nio/src/test/java/org/elasticsearch/nio/SocketChannelContextTests.java b/libs/nio/src/test/java/org/elasticsearch/nio/SocketChannelContextTests.java index 0040f70df85ae..5563ccc43063b 100644 --- a/libs/nio/src/test/java/org/elasticsearch/nio/SocketChannelContextTests.java +++ b/libs/nio/src/test/java/org/elasticsearch/nio/SocketChannelContextTests.java @@ -35,7 +35,6 @@ import java.util.function.BiConsumer; import java.util.function.Consumer; import java.util.function.IntFunction; -import java.util.function.Predicate; import static org.mockito.Matchers.any; import static org.mockito.Matchers.anyInt; @@ -54,7 +53,7 @@ public class SocketChannelContextTests extends ESTestCase { private NioSocketChannel channel; private BiConsumer listener; private NioSelector selector; - private ReadWriteHandler readWriteHandler; + private NioChannelHandler readWriteHandler; private ByteBuffer ioBuffer = ByteBuffer.allocate(1024); @SuppressWarnings("unchecked") @@ -68,7 +67,7 @@ public void setup() throws Exception { when(channel.getRawChannel()).thenReturn(rawChannel); exceptionHandler = mock(Consumer.class); selector = mock(NioSelector.class); - readWriteHandler = mock(ReadWriteHandler.class); + readWriteHandler = mock(NioChannelHandler.class); InboundChannelBuffer channelBuffer = InboundChannelBuffer.allocatingInstance(); context = new TestSocketChannelContext(channel, selector, exceptionHandler, readWriteHandler, channelBuffer); @@ -102,22 +101,6 @@ public void testSignalWhenPeerClosed() throws IOException { assertTrue(context.closeNow()); } - public void testValidateInRegisterCanSucceed() throws IOException { - InboundChannelBuffer channelBuffer = InboundChannelBuffer.allocatingInstance(); - context = new TestSocketChannelContext(channel, selector, exceptionHandler, readWriteHandler, channelBuffer, (c) -> true); - assertFalse(context.closeNow()); - context.register(); - assertFalse(context.closeNow()); - } - - public void testValidateInRegisterCanFail() throws IOException { - InboundChannelBuffer channelBuffer = InboundChannelBuffer.allocatingInstance(); - context = new TestSocketChannelContext(channel, selector, exceptionHandler, readWriteHandler, channelBuffer, (c) -> false); - assertFalse(context.closeNow()); - context.register(); - assertTrue(context.closeNow()); - } - public void testConnectSucceeds() throws IOException { AtomicBoolean listenerCalled = new AtomicBoolean(false); when(rawChannel.finishConnect()).thenReturn(false, true); @@ -192,7 +175,7 @@ public void testSendMessageFromSameThreadIsQueuedInChannel() { when(readWriteHandler.createWriteOperation(context, buffers, listener)).thenReturn(writeOperation); context.sendMessage(buffers, listener); - verify(selector).writeToChannel(writeOpCaptor.capture()); + verify(selector).queueWrite(writeOpCaptor.capture()); WriteOperation writeOp = writeOpCaptor.getValue(); assertSame(writeOperation, writeOp); @@ -394,14 +377,8 @@ public void testFlushBuffersHandlesIOExceptionSecondTimeThroughLoop() throws IOE private static class TestSocketChannelContext extends SocketChannelContext { private TestSocketChannelContext(NioSocketChannel channel, NioSelector selector, Consumer exceptionHandler, - ReadWriteHandler readWriteHandler, InboundChannelBuffer channelBuffer) { - this(channel, selector, exceptionHandler, readWriteHandler, channelBuffer, ALWAYS_ALLOW_CHANNEL); - } - - private TestSocketChannelContext(NioSocketChannel channel, NioSelector selector, Consumer exceptionHandler, - ReadWriteHandler readWriteHandler, InboundChannelBuffer channelBuffer, - Predicate allowChannelPredicate) { - super(channel, selector, exceptionHandler, readWriteHandler, channelBuffer, allowChannelPredicate); + NioChannelHandler readWriteHandler, InboundChannelBuffer channelBuffer) { + super(channel, selector, exceptionHandler, readWriteHandler, channelBuffer); } @Override diff --git a/libs/secure-sm/build.gradle b/libs/secure-sm/build.gradle index bbd44afc70a6d..3e79d9ee2e84b 100644 --- a/libs/secure-sm/build.gradle +++ b/libs/secure-sm/build.gradle @@ -19,16 +19,6 @@ apply plugin: 'nebula.maven-base-publish' apply plugin: 'nebula.maven-scm' -archivesBaseName = 'elasticsearch-secure-sm' - -publishing { - publications { - nebula { - artifactId = archivesBaseName - } - } -} - dependencies { // do not add non-test compile dependencies to secure-sm without a good reason to do so @@ -36,9 +26,9 @@ dependencies { testCompile "junit:junit:${versions.junit}" testCompile "org.hamcrest:hamcrest:${versions.hamcrest}" - if (isEclipse == false || project.path == ":libs:secure-sm-tests") { - testCompile("org.elasticsearch.test:framework:${version}") { - exclude group: 'org.elasticsearch', module: 'secure-sm' + if (isEclipse == false || project.path == ":libs:elasticsearch-secure-sm-tests") { + testCompile(project(":test:framework")) { + exclude group: 'org.elasticsearch', module: 'elasticsearch-secure-sm' } } } @@ -50,7 +40,7 @@ forbiddenApisMain { if (isEclipse) { // in Eclipse the project is under a fake root so we need to change around the source sets sourceSets { - if (project.path == ":libs:secure-sm") { + if (project.path == ":libs:elasticsearch-secure-sm") { main.java.srcDirs = ['java'] main.resources.srcDirs = ['resources'] } else { diff --git a/libs/secure-sm/src/test/eclipse-build.gradle b/libs/secure-sm/src/test/eclipse-build.gradle index 56dcdcbac27fa..63d610b75c1db 100644 --- a/libs/secure-sm/src/test/eclipse-build.gradle +++ b/libs/secure-sm/src/test/eclipse-build.gradle @@ -3,5 +3,5 @@ apply from: '../../build.gradle' dependencies { - testCompile project(':libs:secure-sm') + testCompile project(':libs:elasticsearch-secure-sm') } diff --git a/libs/ssl-config/build.gradle b/libs/ssl-config/build.gradle index 860cdcd9e6fc8..71ebd642a4339 100644 --- a/libs/ssl-config/build.gradle +++ b/libs/ssl-config/build.gradle @@ -19,10 +19,10 @@ apply plugin: "nebula.maven-scm" dependencies { - compile "org.elasticsearch:elasticsearch-core:${version}" + compile project(':libs:elasticsearch-core') - if (isEclipse == false || project.path == ":libs:ssl-config-tests") { - testCompile("org.elasticsearch.test:framework:${version}") { + if (isEclipse == false || project.path == ":libs:elasticsearch-ssl-config-tests") { + testCompile(project(":test:framework")) { exclude group: 'org.elasticsearch', module: 'elasticsearch-ssl-config' } } @@ -35,7 +35,7 @@ dependencies { if (isEclipse) { // in eclipse the project is under a fake root, we need to change around the source sets sourceSets { - if (project.path == ":libs:ssl-config") { + if (project.path == ":libs:elasticsearch-ssl-config") { main.java.srcDirs = ['java'] main.resources.srcDirs = ['resources'] } else { diff --git a/libs/ssl-config/src/test/java/org/elasticsearch/common/ssl/PemTrustConfigTests.java b/libs/ssl-config/src/test/java/org/elasticsearch/common/ssl/PemTrustConfigTests.java index 44a39c2e183d6..8d87c1599352b 100644 --- a/libs/ssl-config/src/test/java/org/elasticsearch/common/ssl/PemTrustConfigTests.java +++ b/libs/ssl-config/src/test/java/org/elasticsearch/common/ssl/PemTrustConfigTests.java @@ -22,7 +22,6 @@ import org.elasticsearch.test.ESTestCase; import org.hamcrest.Matchers; -import javax.net.ssl.X509ExtendedTrustManager; import java.nio.file.Files; import java.nio.file.NoSuchFileException; import java.nio.file.Path; @@ -37,6 +36,8 @@ import java.util.stream.Collectors; import java.util.stream.Stream; +import javax.net.ssl.X509ExtendedTrustManager; + public class PemTrustConfigTests extends ESTestCase { public void testBuildTrustConfigFromSinglePemFile() throws Exception { @@ -57,7 +58,7 @@ public void testBuildTrustConfigFromMultiplePemFiles() throws Exception { public void testBadFileFormatFails() throws Exception { final Path ca = createTempFile("ca", ".crt"); - Files.write(ca, randomByteArrayOfLength(128), StandardOpenOption.APPEND); + Files.write(ca, generateRandomByteArrayOfLength(128), StandardOpenOption.APPEND); final PemTrustConfig trustConfig = new PemTrustConfig(Collections.singletonList(ca)); assertThat(trustConfig.getDependentFiles(), Matchers.containsInAnyOrder(ca)); assertInvalidFileFormat(trustConfig, ca); @@ -106,7 +107,7 @@ public void testTrustConfigReloadsFileContents() throws Exception { Files.delete(ca1); assertFileNotFound(trustConfig, ca1); - Files.write(ca1, randomByteArrayOfLength(128), StandardOpenOption.CREATE); + Files.write(ca1, generateRandomByteArrayOfLength(128), StandardOpenOption.CREATE); assertInvalidFileFormat(trustConfig, ca1); } @@ -148,4 +149,24 @@ private void assertFileNotFound(PemTrustConfig trustConfig, Path file) { assertThat(exception.getMessage(), Matchers.containsString(file.toAbsolutePath().toString())); assertThat(exception.getCause(), Matchers.instanceOf(NoSuchFileException.class)); } + + private byte[] generateRandomByteArrayOfLength(int length) { + byte[] bytes = randomByteArrayOfLength(length); + /* + * If the bytes represent DER encoded value indicating ASN.1 SEQUENCE followed by length byte if it is zero then while trying to + * parse PKCS7 block from the encoded stream, it failed parsing the content type. The DerInputStream.getSequence() method in this + * case returns an empty DerValue array but ContentType does not check the length of array before accessing the array resulting in a + * ArrayIndexOutOfBoundsException. This check ensures that when we create random stream of bytes we do not create ASN.1 SEQUENCE + * followed by zero length which fails the test intermittently. + */ + while(checkRandomGeneratedBytesRepresentZeroLengthDerSequenceCausingArrayIndexOutOfBound(bytes)) { + bytes = randomByteArrayOfLength(length); + } + return bytes; + } + + private static boolean checkRandomGeneratedBytesRepresentZeroLengthDerSequenceCausingArrayIndexOutOfBound(byte[] bytes) { + // Tag value indicating an ASN.1 "SEQUENCE". Reference: sun.security.util.DerValue.tag_Sequence = 0x30 + return bytes[0] == 0x30 && bytes[1] == 0x00; + } } diff --git a/libs/x-content/build.gradle b/libs/x-content/build.gradle index 0e99d80da1e00..e54427c058304 100644 --- a/libs/x-content/build.gradle +++ b/libs/x-content/build.gradle @@ -21,18 +21,8 @@ apply plugin: 'elasticsearch.build' apply plugin: 'nebula.maven-base-publish' apply plugin: 'nebula.maven-scm' -archivesBaseName = 'elasticsearch-x-content' - -publishing { - publications { - nebula { - artifactId = archivesBaseName - } - } -} - dependencies { - compile "org.elasticsearch:elasticsearch-core:${version}" + compile project(':libs:elasticsearch-core') compile "org.yaml:snakeyaml:${versions.snakeyaml}" compile "com.fasterxml.jackson.core:jackson-core:${versions.jackson}" @@ -44,8 +34,8 @@ dependencies { testCompile "junit:junit:${versions.junit}" testCompile "org.hamcrest:hamcrest:${versions.hamcrest}" - if (isEclipse == false || project.path == ":libs:x-content-tests") { - testCompile("org.elasticsearch.test:framework:${version}") { + if (isEclipse == false || project.path == ":libs:elasticsearch-x-content-tests") { + testCompile(project(":test:framework")) { exclude group: 'org.elasticsearch', module: 'elasticsearch-x-content' } } @@ -61,7 +51,7 @@ forbiddenApisMain { if (isEclipse) { // in eclipse the project is under a fake root, we need to change around the source sets sourceSets { - if (project.path == ":libs:x-content") { + if (project.path == ":libs:elasticsearch-x-content") { main.java.srcDirs = ['java'] main.resources.srcDirs = ['resources'] } else { diff --git a/libs/x-content/src/main/java/org/elasticsearch/common/xcontent/ObjectParser.java b/libs/x-content/src/main/java/org/elasticsearch/common/xcontent/ObjectParser.java index ee5e3347f8d99..c80c5bdb0d09a 100644 --- a/libs/x-content/src/main/java/org/elasticsearch/common/xcontent/ObjectParser.java +++ b/libs/x-content/src/main/java/org/elasticsearch/common/xcontent/ObjectParser.java @@ -78,14 +78,63 @@ public static BiConsumer> fromLi }; } + private interface UnknownFieldParser { + + void acceptUnknownField(String parserName, String field, XContentLocation location, XContentParser parser, + Value value, Context context) throws IOException; + } + + private static UnknownFieldParser ignoreUnknown() { + return (n, f, l, p, v, c) -> p.skipChildren(); + } + + private static UnknownFieldParser errorOnUnknown() { + return (n, f, l, p, v, c) -> { + throw new XContentParseException(l, "[" + n + "] unknown field [" + f + "], parser not found"); + }; + } + + /** + * Defines how to consume a parsed undefined field + */ + public interface UnknownFieldConsumer { + void accept(Value target, String field, Object value); + } + + private static UnknownFieldParser consumeUnknownField(UnknownFieldConsumer consumer) { + return (parserName, field, location, parser, value, context) -> { + XContentParser.Token t = parser.currentToken(); + switch (t) { + case VALUE_STRING: + consumer.accept(value, field, parser.text()); + break; + case VALUE_NUMBER: + consumer.accept(value, field, parser.numberValue()); + break; + case VALUE_BOOLEAN: + consumer.accept(value, field, parser.booleanValue()); + break; + case VALUE_NULL: + consumer.accept(value, field, null); + break; + case START_OBJECT: + consumer.accept(value, field, parser.map()); + break; + case START_ARRAY: + consumer.accept(value, field, parser.list()); + break; + default: + throw new XContentParseException(parser.getTokenLocation(), + "[" + parserName + "] cannot parse field [" + field + "] with value type [" + t + "]"); + } + }; + } + private final Map fieldParserMap = new HashMap<>(); private final String name; private final Supplier valueSupplier; - /** - * Should this parser ignore unknown fields? This should generally be set to true only when parsing responses from external systems, - * never when parsing requests from users. - */ - private final boolean ignoreUnknownFields; + + private final UnknownFieldParser unknownFieldParser; /** * Creates a new ObjectParser instance with a name. This name is used to reference the parser in exceptions and messages. @@ -95,25 +144,45 @@ public ObjectParser(String name) { } /** - * Creates a new ObjectParser instance which a name. + * Creates a new ObjectParser instance with a name. * @param name the parsers name, used to reference the parser in exceptions and messages. * @param valueSupplier a supplier that creates a new Value instance used when the parser is used as an inner object parser. */ public ObjectParser(String name, @Nullable Supplier valueSupplier) { - this(name, false, valueSupplier); + this(name, errorOnUnknown(), valueSupplier); } /** - * Creates a new ObjectParser instance which a name. + * Creates a new ObjectParser instance with a name. * @param name the parsers name, used to reference the parser in exceptions and messages. * @param ignoreUnknownFields Should this parser ignore unknown fields? This should generally be set to true only when parsing * responses from external systems, never when parsing requests from users. * @param valueSupplier a supplier that creates a new Value instance used when the parser is used as an inner object parser. */ public ObjectParser(String name, boolean ignoreUnknownFields, @Nullable Supplier valueSupplier) { + this(name, ignoreUnknownFields ? ignoreUnknown() : errorOnUnknown(), valueSupplier); + } + + /** + * Creates a new ObjectParser instance with a name. + * @param name the parsers name, used to reference the parser in exceptions and messages. + * @param unknownFieldConsumer how to consume parsed unknown fields + * @param valueSupplier a supplier that creates a new Value instance used when the parser is used as an inner object parser. + */ + public ObjectParser(String name, UnknownFieldConsumer unknownFieldConsumer, @Nullable Supplier valueSupplier) { + this(name, consumeUnknownField(unknownFieldConsumer), valueSupplier); + } + + /** + * Creates a new ObjectParser instance with a name. + * @param name the parsers name, used to reference the parser in exceptions and messages. + * @param unknownFieldParser how to parse unknown fields + * @param valueSupplier a supplier that creates a new Value instance used when the parser is used as an inner object parser. + */ + private ObjectParser(String name, UnknownFieldParser unknownFieldParser, @Nullable Supplier valueSupplier) { this.name = name; this.valueSupplier = valueSupplier; - this.ignoreUnknownFields = ignoreUnknownFields; + this.unknownFieldParser = unknownFieldParser; } /** @@ -152,17 +221,18 @@ public Value parse(XContentParser parser, Value value, Context context) throws I FieldParser fieldParser = null; String currentFieldName = null; + XContentLocation currentPosition = null; while ((token = parser.nextToken()) != XContentParser.Token.END_OBJECT) { if (token == XContentParser.Token.FIELD_NAME) { currentFieldName = parser.currentName(); - fieldParser = getParser(currentFieldName, parser); + currentPosition = parser.getTokenLocation(); + fieldParser = fieldParserMap.get(currentFieldName); } else { if (currentFieldName == null) { throw new XContentParseException(parser.getTokenLocation(), "[" + name + "] no field found"); } if (fieldParser == null) { - assert ignoreUnknownFields : "this should only be possible if configured to ignore known fields"; - parser.skipChildren(); // noop if parser points to a value, skips children if parser is start object or start array + unknownFieldParser.acceptUnknownField(name, currentFieldName, currentPosition, parser, value, context); } else { fieldParser.assertSupports(name, parser, currentFieldName); parseSub(parser, fieldParser, currentFieldName, value, context); @@ -363,15 +433,6 @@ private void parseSub(XContentParser parser, FieldParser fieldParser, String cur } } - private FieldParser getParser(String fieldName, XContentParser xContentParser) { - FieldParser parser = fieldParserMap.get(fieldName); - if (parser == null && false == ignoreUnknownFields) { - throw new XContentParseException(xContentParser.getTokenLocation(), - "[" + name + "] unknown field [" + fieldName + "], parser not found"); - } - return parser; - } - private class FieldParser { private final Parser parser; private final EnumSet supportedTokens; diff --git a/libs/x-content/src/main/java/org/elasticsearch/common/xcontent/XContentParser.java b/libs/x-content/src/main/java/org/elasticsearch/common/xcontent/XContentParser.java index 6d4da08bfaa59..82a663bd9dc5d 100644 --- a/libs/x-content/src/main/java/org/elasticsearch/common/xcontent/XContentParser.java +++ b/libs/x-content/src/main/java/org/elasticsearch/common/xcontent/XContentParser.java @@ -117,7 +117,7 @@ public boolean isValue() { } enum NumberType { - INT, LONG, FLOAT, DOUBLE + INT, BIG_INTEGER, LONG, FLOAT, DOUBLE, BIG_DECIMAL } XContentType contentType(); @@ -136,11 +136,9 @@ enum NumberType { Map mapStrings() throws IOException; - Map mapStringsOrdered() throws IOException; - /** * Returns an instance of {@link Map} holding parsed map. - * Serves as a replacement for the "map", "mapOrdered", "mapStrings" and "mapStringsOrdered" methods above. + * Serves as a replacement for the "map", "mapOrdered" and "mapStrings" methods above. * * @param mapFactory factory for creating new {@link Map} objects * @param mapValueParser parser for parsing a single map value diff --git a/libs/x-content/src/main/java/org/elasticsearch/common/xcontent/XContentSubParser.java b/libs/x-content/src/main/java/org/elasticsearch/common/xcontent/XContentSubParser.java index 252bfea7ca9c0..9a8686001e2dc 100644 --- a/libs/x-content/src/main/java/org/elasticsearch/common/xcontent/XContentSubParser.java +++ b/libs/x-content/src/main/java/org/elasticsearch/common/xcontent/XContentSubParser.java @@ -108,11 +108,6 @@ public Map mapStrings() throws IOException { return parser.mapStrings(); } - @Override - public Map mapStringsOrdered() throws IOException { - return parser.mapStringsOrdered(); - } - @Override public Map map( Supplier> mapFactory, CheckedFunction mapValueParser) throws IOException { diff --git a/libs/x-content/src/main/java/org/elasticsearch/common/xcontent/json/JsonXContentParser.java b/libs/x-content/src/main/java/org/elasticsearch/common/xcontent/json/JsonXContentParser.java index be0d01763b893..7489222df2e76 100644 --- a/libs/x-content/src/main/java/org/elasticsearch/common/xcontent/json/JsonXContentParser.java +++ b/libs/x-content/src/main/java/org/elasticsearch/common/xcontent/json/JsonXContentParser.java @@ -199,12 +199,16 @@ private NumberType convertNumberType(JsonParser.NumberType numberType) { switch (numberType) { case INT: return NumberType.INT; + case BIG_INTEGER: + return NumberType.BIG_INTEGER; case LONG: return NumberType.LONG; case FLOAT: return NumberType.FLOAT; case DOUBLE: return NumberType.DOUBLE; + case BIG_DECIMAL: + return NumberType.BIG_DECIMAL; } throw new IllegalStateException("No matching token for number_type [" + numberType + "]"); } diff --git a/libs/x-content/src/main/java/org/elasticsearch/common/xcontent/support/AbstractXContentParser.java b/libs/x-content/src/main/java/org/elasticsearch/common/xcontent/support/AbstractXContentParser.java index 68e03e34a1a17..264af205e488b 100644 --- a/libs/x-content/src/main/java/org/elasticsearch/common/xcontent/support/AbstractXContentParser.java +++ b/libs/x-content/src/main/java/org/elasticsearch/common/xcontent/support/AbstractXContentParser.java @@ -276,11 +276,6 @@ public Map mapStrings() throws IOException { return readMapStrings(this); } - @Override - public Map mapStringsOrdered() throws IOException { - return readOrderedMapStrings(this); - } - @Override public Map map( Supplier> mapFactory, CheckedFunction mapValueParser) throws IOException { @@ -303,8 +298,6 @@ public List listOrderedMap() throws IOException { static final Supplier> SIMPLE_MAP_STRINGS_FACTORY = HashMap::new; - static final Supplier> ORDERED_MAP_STRINGS_FACTORY = LinkedHashMap::new; - static Map readMap(XContentParser parser) throws IOException { return readMap(parser, SIMPLE_MAP_FACTORY); } @@ -314,11 +307,7 @@ static Map readOrderedMap(XContentParser parser) throws IOExcept } static Map readMapStrings(XContentParser parser) throws IOException { - return readMapStrings(parser, SIMPLE_MAP_STRINGS_FACTORY); - } - - static Map readOrderedMapStrings(XContentParser parser) throws IOException { - return readMapStrings(parser, ORDERED_MAP_STRINGS_FACTORY); + return readGenericMap(parser, SIMPLE_MAP_STRINGS_FACTORY, XContentParser::text); } static List readList(XContentParser parser) throws IOException { @@ -333,10 +322,6 @@ static Map readMap(XContentParser parser, Supplier readValue(p, mapFactory)); } - static Map readMapStrings(XContentParser parser, Supplier> mapFactory) throws IOException { - return readGenericMap(parser, mapFactory, XContentParser::text); - } - static Map readGenericMap( XContentParser parser, Supplier> mapFactory, @@ -382,7 +367,7 @@ static List readList(XContentParser parser, Supplier return list; } - static Object readValue(XContentParser parser, Supplier> mapFactory) throws IOException { + public static Object readValue(XContentParser parser, Supplier> mapFactory) throws IOException { switch (parser.currentToken()) { case VALUE_STRING: return parser.text(); case VALUE_NUMBER: return parser.numberValue(); diff --git a/libs/x-content/src/test/eclipse-build.gradle b/libs/x-content/src/test/eclipse-build.gradle index f456f71a4c310..a8d81f9f95831 100644 --- a/libs/x-content/src/test/eclipse-build.gradle +++ b/libs/x-content/src/test/eclipse-build.gradle @@ -3,5 +3,5 @@ apply from: '../../build.gradle' dependencies { - testCompile project(':libs:x-content') + testCompile project(':libs:elasticsearch-x-content') } diff --git a/libs/x-content/src/test/java/org/elasticsearch/common/xcontent/ObjectParserTests.java b/libs/x-content/src/test/java/org/elasticsearch/common/xcontent/ObjectParserTests.java index e089b8a956ac8..6002c6bd35076 100644 --- a/libs/x-content/src/test/java/org/elasticsearch/common/xcontent/ObjectParserTests.java +++ b/libs/x-content/src/test/java/org/elasticsearch/common/xcontent/ObjectParserTests.java @@ -33,7 +33,9 @@ import java.util.ArrayList; import java.util.Arrays; import java.util.Collections; +import java.util.HashMap; import java.util.List; +import java.util.Map; import java.util.concurrent.atomic.AtomicReference; import static org.hamcrest.Matchers.containsString; @@ -733,4 +735,41 @@ public void setFoo(int foo) { this.foo = foo; } } + + private static class ObjectWithArbitraryFields { + String name; + Map fields = new HashMap<>(); + void setField(String key, Object value) { + fields.put(key, value); + } + void setName(String name) { + this.name = name; + } + } + + public void testConsumeUnknownFields() throws IOException { + XContentParser parser = createParser(JsonXContent.jsonXContent, + "{\n" + + " \"test\" : \"foo\",\n" + + " \"test_number\" : 2,\n" + + " \"name\" : \"geoff\",\n" + + " \"test_boolean\" : true,\n" + + " \"test_null\" : null,\n" + + " \"test_array\": [1,2,3,4],\n" + + " \"test_nested\": { \"field\" : \"value\", \"field2\" : [ \"list1\", \"list2\" ] }\n" + + "}"); + ObjectParser op + = new ObjectParser<>("unknown", ObjectWithArbitraryFields::setField, ObjectWithArbitraryFields::new); + op.declareString(ObjectWithArbitraryFields::setName, new ParseField("name")); + + ObjectWithArbitraryFields o = op.parse(parser, null); + assertEquals("geoff", o.name); + assertEquals(6, o.fields.size()); + assertEquals("foo", o.fields.get("test")); + assertEquals(2, o.fields.get("test_number")); + assertEquals(true, o.fields.get("test_boolean")); + assertNull(o.fields.get("test_null")); + assertEquals(List.of(1, 2, 3, 4), o.fields.get("test_array")); + assertEquals(Map.of("field", "value", "field2", List.of("list1", "list2")), o.fields.get("test_nested")); + } } diff --git a/libs/x-content/src/test/java/org/elasticsearch/common/xcontent/XContentParserTests.java b/libs/x-content/src/test/java/org/elasticsearch/common/xcontent/XContentParserTests.java index c519880224ccb..31a00c4025ab2 100644 --- a/libs/x-content/src/test/java/org/elasticsearch/common/xcontent/XContentParserTests.java +++ b/libs/x-content/src/test/java/org/elasticsearch/common/xcontent/XContentParserTests.java @@ -189,7 +189,7 @@ private Map readMapStrings(String source) throws IOException { assertThat(parser.currentName(), equalTo("foo")); token = parser.nextToken(); assertThat(token, equalTo(XContentParser.Token.START_OBJECT)); - return randomBoolean() ? parser.mapStringsOrdered() : parser.mapStrings(); + return parser.mapStrings(); } } diff --git a/modules/aggs-matrix-stats/build.gradle b/modules/aggs-matrix-stats/build.gradle index d3af124b8af71..b3060fa178621 100644 --- a/modules/aggs-matrix-stats/build.gradle +++ b/modules/aggs-matrix-stats/build.gradle @@ -20,5 +20,4 @@ esplugin { description 'Adds aggregations whose input are a list of numeric fields and output includes a matrix.' classname 'org.elasticsearch.search.aggregations.matrix.MatrixAggregationPlugin' - hasClientJar = true } diff --git a/modules/aggs-matrix-stats/src/main/java/org/elasticsearch/search/aggregations/matrix/stats/InternalMatrixStats.java b/modules/aggs-matrix-stats/src/main/java/org/elasticsearch/search/aggregations/matrix/stats/InternalMatrixStats.java index 7dcdff426711b..c70ddea7e3b22 100644 --- a/modules/aggs-matrix-stats/src/main/java/org/elasticsearch/search/aggregations/matrix/stats/InternalMatrixStats.java +++ b/modules/aggs-matrix-stats/src/main/java/org/elasticsearch/search/aggregations/matrix/stats/InternalMatrixStats.java @@ -256,12 +256,16 @@ public InternalAggregation doReduce(List aggregations, Redu } @Override - protected int doHashCode() { - return Objects.hash(stats, results); + public int hashCode() { + return Objects.hash(super.hashCode(), stats, results); } @Override - protected boolean doEquals(Object obj) { + public boolean equals(Object obj) { + if (this == obj) return true; + if (obj == null || getClass() != obj.getClass()) return false; + if (super.equals(obj) == false) return false; + InternalMatrixStats other = (InternalMatrixStats) obj; return Objects.equals(this.stats, other.stats) && Objects.equals(this.results, other.results); diff --git a/modules/aggs-matrix-stats/src/main/java/org/elasticsearch/search/aggregations/matrix/stats/MatrixStatsAggregationBuilder.java b/modules/aggs-matrix-stats/src/main/java/org/elasticsearch/search/aggregations/matrix/stats/MatrixStatsAggregationBuilder.java index 75f24eb78037a..90f51236252ce 100644 --- a/modules/aggs-matrix-stats/src/main/java/org/elasticsearch/search/aggregations/matrix/stats/MatrixStatsAggregationBuilder.java +++ b/modules/aggs-matrix-stats/src/main/java/org/elasticsearch/search/aggregations/matrix/stats/MatrixStatsAggregationBuilder.java @@ -81,7 +81,7 @@ public MultiValueMode multiValueMode() { @Override protected MatrixStatsAggregatorFactory innerBuild(SearchContext context, Map> configs, - AggregatorFactory parent, AggregatorFactories.Builder subFactoriesBuilder) throws IOException { + AggregatorFactory parent, AggregatorFactories.Builder subFactoriesBuilder) throws IOException { return new MatrixStatsAggregatorFactory(name, configs, multiValueMode, context, parent, subFactoriesBuilder, metaData); } @@ -91,16 +91,6 @@ public XContentBuilder doXContentBody(XContentBuilder builder, ToXContent.Params return builder; } - @Override - protected int innerHashCode() { - return 0; - } - - @Override - protected boolean innerEquals(Object obj) { - return true; - } - @Override public String getType() { return NAME; diff --git a/modules/aggs-matrix-stats/src/main/java/org/elasticsearch/search/aggregations/matrix/stats/MatrixStatsAggregatorFactory.java b/modules/aggs-matrix-stats/src/main/java/org/elasticsearch/search/aggregations/matrix/stats/MatrixStatsAggregatorFactory.java index fb456d75bb78b..469cd0dad8fcb 100644 --- a/modules/aggs-matrix-stats/src/main/java/org/elasticsearch/search/aggregations/matrix/stats/MatrixStatsAggregatorFactory.java +++ b/modules/aggs-matrix-stats/src/main/java/org/elasticsearch/search/aggregations/matrix/stats/MatrixStatsAggregatorFactory.java @@ -32,14 +32,13 @@ import java.util.List; import java.util.Map; -final class MatrixStatsAggregatorFactory - extends ArrayValuesSourceAggregatorFactory { +final class MatrixStatsAggregatorFactory extends ArrayValuesSourceAggregatorFactory { private final MultiValueMode multiValueMode; MatrixStatsAggregatorFactory(String name, Map> configs, MultiValueMode multiValueMode, - SearchContext context, AggregatorFactory parent, AggregatorFactories.Builder subFactoriesBuilder, + SearchContext context, AggregatorFactory parent, AggregatorFactories.Builder subFactoriesBuilder, Map metaData) throws IOException { super(name, configs, context, parent, subFactoriesBuilder, metaData); this.multiValueMode = multiValueMode; diff --git a/modules/aggs-matrix-stats/src/main/java/org/elasticsearch/search/aggregations/support/ArrayValuesSourceAggregationBuilder.java b/modules/aggs-matrix-stats/src/main/java/org/elasticsearch/search/aggregations/support/ArrayValuesSourceAggregationBuilder.java index eb8152e0fe0b8..a3d7fc5d64290 100644 --- a/modules/aggs-matrix-stats/src/main/java/org/elasticsearch/search/aggregations/support/ArrayValuesSourceAggregationBuilder.java +++ b/modules/aggs-matrix-stats/src/main/java/org/elasticsearch/search/aggregations/support/ArrayValuesSourceAggregationBuilder.java @@ -239,10 +239,10 @@ public Map missingMap() { } @Override - protected final ArrayValuesSourceAggregatorFactory doBuild(SearchContext context, AggregatorFactory parent, + protected final ArrayValuesSourceAggregatorFactory doBuild(SearchContext context, AggregatorFactory parent, AggregatorFactories.Builder subFactoriesBuilder) throws IOException { Map> configs = resolveConfig(context); - ArrayValuesSourceAggregatorFactory factory = innerBuild(context, configs, parent, subFactoriesBuilder); + ArrayValuesSourceAggregatorFactory factory = innerBuild(context, configs, parent, subFactoriesBuilder); return factory; } @@ -255,9 +255,9 @@ protected Map> resolveConfig(SearchContext contex return configs; } - protected abstract ArrayValuesSourceAggregatorFactory innerBuild(SearchContext context, + protected abstract ArrayValuesSourceAggregatorFactory innerBuild(SearchContext context, Map> configs, - AggregatorFactory parent, + AggregatorFactory parent, AggregatorFactories.Builder subFactoriesBuilder) throws IOException; public ValuesSourceConfig config(SearchContext context, String field, Script script) { @@ -354,30 +354,21 @@ public final XContentBuilder internalXContent(XContentBuilder builder, Params pa protected abstract XContentBuilder doXContentBody(XContentBuilder builder, Params params) throws IOException; @Override - protected final int doHashCode() { - return Objects.hash(fields, format, missing, targetValueType, valueType, valuesSourceType, - innerHashCode()); + public int hashCode() { + return Objects.hash(super.hashCode(), fields, format, missing, targetValueType, valueType, valuesSourceType); } - protected abstract int innerHashCode(); - @Override - protected final boolean doEquals(Object obj) { + public boolean equals(Object obj) { + if (this == obj) return true; + if (obj == null || getClass() != obj.getClass()) return false; + if (super.equals(obj) == false) return false; ArrayValuesSourceAggregationBuilder other = (ArrayValuesSourceAggregationBuilder) obj; - if (!Objects.equals(fields, other.fields)) - return false; - if (!Objects.equals(format, other.format)) - return false; - if (!Objects.equals(missing, other.missing)) - return false; - if (!Objects.equals(targetValueType, other.targetValueType)) - return false; - if (!Objects.equals(valueType, other.valueType)) - return false; - if (!Objects.equals(valuesSourceType, other.valuesSourceType)) - return false; - return innerEquals(obj); + return Objects.equals(fields, other.fields) + && Objects.equals(format, other.format) + && Objects.equals(missing, other.missing) + && Objects.equals(targetValueType, other.targetValueType) + && Objects.equals(valueType, other.valueType) + && Objects.equals(valuesSourceType, other.valuesSourceType); } - - protected abstract boolean innerEquals(Object obj); } diff --git a/modules/aggs-matrix-stats/src/main/java/org/elasticsearch/search/aggregations/support/ArrayValuesSourceAggregatorFactory.java b/modules/aggs-matrix-stats/src/main/java/org/elasticsearch/search/aggregations/support/ArrayValuesSourceAggregatorFactory.java index ce8eeecd19036..754cb9576c7dd 100644 --- a/modules/aggs-matrix-stats/src/main/java/org/elasticsearch/search/aggregations/support/ArrayValuesSourceAggregatorFactory.java +++ b/modules/aggs-matrix-stats/src/main/java/org/elasticsearch/search/aggregations/support/ArrayValuesSourceAggregatorFactory.java @@ -30,13 +30,13 @@ import java.util.List; import java.util.Map; -public abstract class ArrayValuesSourceAggregatorFactory> - extends AggregatorFactory { +public abstract class ArrayValuesSourceAggregatorFactory + extends AggregatorFactory { protected Map> configs; public ArrayValuesSourceAggregatorFactory(String name, Map> configs, - SearchContext context, AggregatorFactory parent, + SearchContext context, AggregatorFactory parent, AggregatorFactories.Builder subFactoriesBuilder, Map metaData) throws IOException { super(name, context, parent, subFactoriesBuilder, metaData); diff --git a/modules/analysis-common/src/main/java/org/elasticsearch/analysis/common/CommonAnalysisPlugin.java b/modules/analysis-common/src/main/java/org/elasticsearch/analysis/common/CommonAnalysisPlugin.java index f095b766ee1d5..a655f42a36c26 100644 --- a/modules/analysis-common/src/main/java/org/elasticsearch/analysis/common/CommonAnalysisPlugin.java +++ b/modules/analysis-common/src/main/java/org/elasticsearch/analysis/common/CommonAnalysisPlugin.java @@ -83,6 +83,7 @@ import org.apache.lucene.analysis.miscellaneous.TruncateTokenFilter; import org.apache.lucene.analysis.miscellaneous.WordDelimiterFilter; import org.apache.lucene.analysis.miscellaneous.WordDelimiterGraphFilter; +import org.apache.lucene.analysis.miscellaneous.WordDelimiterIterator; import org.apache.lucene.analysis.ngram.EdgeNGramTokenFilter; import org.apache.lucene.analysis.ngram.EdgeNGramTokenizer; import org.apache.lucene.analysis.ngram.NGramTokenFilter; @@ -110,6 +111,7 @@ import org.apache.lucene.analysis.tr.TurkishAnalyzer; import org.apache.lucene.analysis.util.ElisionFilter; import org.apache.lucene.util.SetOnce; +import org.elasticsearch.Version; import org.elasticsearch.client.Client; import org.elasticsearch.cluster.service.ClusterService; import org.elasticsearch.common.io.stream.NamedWriteableRegistry; @@ -237,7 +239,7 @@ public Map> getTokenFilters() { filters.put("dutch_stem", DutchStemTokenFilterFactory::new); filters.put("edge_ngram", EdgeNGramTokenFilterFactory::new); filters.put("edgeNGram", EdgeNGramTokenFilterFactory::new); - filters.put("elision", ElisionTokenFilterFactory::new); + filters.put("elision", requiresAnalysisSettings(ElisionTokenFilterFactory::new)); filters.put("fingerprint", FingerprintTokenFilterFactory::new); filters.put("flatten_graph", FlattenGraphTokenFilterFactory::new); filters.put("french_stem", FrenchStemTokenFilterFactory::new); @@ -320,9 +322,6 @@ public Map> getTokenizers() { @Override public List getPreBuiltAnalyzerProviderFactories() { List analyzers = new ArrayList<>(); - // TODO remove in 8.0 - analyzers.add(new PreBuiltAnalyzerProviderFactory("standard_html_strip", CachingStrategy.ELASTICSEARCH, - () -> new StandardHtmlStripAnalyzer(CharArraySet.EMPTY_SET))); analyzers.add(new PreBuiltAnalyzerProviderFactory("pattern", CachingStrategy.ELASTICSEARCH, () -> new PatternAnalyzer(Regex.compile("\\W+" /*PatternAnalyzer.NON_WORD_PATTERN*/, null), true, CharArraySet.EMPTY_SET))); @@ -393,10 +392,6 @@ public List getPreConfiguredTokenFilters() { input -> new CommonGramsFilter(input, CharArraySet.EMPTY_SET))); filters.add(PreConfiguredTokenFilter.singleton("czech_stem", false, CzechStemFilter::new)); filters.add(PreConfiguredTokenFilter.singleton("decimal_digit", true, DecimalDigitFilter::new)); - filters.add(PreConfiguredTokenFilter.singleton("delimited_payload_filter", false, input -> - new DelimitedPayloadTokenFilter(input, - DelimitedPayloadTokenFilterFactory.DEFAULT_DELIMITER, - DelimitedPayloadTokenFilterFactory.DEFAULT_ENCODER))); filters.add(PreConfiguredTokenFilter.singleton("delimited_payload", false, input -> new DelimitedPayloadTokenFilter(input, DelimitedPayloadTokenFilterFactory.DEFAULT_DELIMITER, @@ -455,13 +450,15 @@ public List getPreConfiguredTokenFilters() { | WordDelimiterFilter.SPLIT_ON_CASE_CHANGE | WordDelimiterFilter.SPLIT_ON_NUMERICS | WordDelimiterFilter.STEM_ENGLISH_POSSESSIVE, null))); - filters.add(PreConfiguredTokenFilter.singleton("word_delimiter_graph", false, false, input -> - new WordDelimiterGraphFilter(input, + filters.add(PreConfiguredTokenFilter.singletonWithVersion("word_delimiter_graph", false, false, (input, version) -> { + boolean adjustOffsets = version.onOrAfter(Version.V_7_3_0); + return new WordDelimiterGraphFilter(input, adjustOffsets, WordDelimiterIterator.DEFAULT_WORD_DELIM_TABLE, WordDelimiterGraphFilter.GENERATE_WORD_PARTS | WordDelimiterGraphFilter.GENERATE_NUMBER_PARTS | WordDelimiterGraphFilter.SPLIT_ON_CASE_CHANGE | WordDelimiterGraphFilter.SPLIT_ON_NUMERICS - | WordDelimiterGraphFilter.STEM_ENGLISH_POSSESSIVE, null))); + | WordDelimiterGraphFilter.STEM_ENGLISH_POSSESSIVE, null); + })); return filters; } @@ -475,8 +472,12 @@ public List getPreConfiguredTokenizers() { tokenizers.add(PreConfiguredTokenizer.singleton("letter", LetterTokenizer::new)); tokenizers.add(PreConfiguredTokenizer.singleton("whitespace", WhitespaceTokenizer::new)); tokenizers.add(PreConfiguredTokenizer.singleton("ngram", NGramTokenizer::new)); - tokenizers.add(PreConfiguredTokenizer.singleton("edge_ngram", - () -> new EdgeNGramTokenizer(EdgeNGramTokenizer.DEFAULT_MIN_GRAM_SIZE, EdgeNGramTokenizer.DEFAULT_MAX_GRAM_SIZE))); + tokenizers.add(PreConfiguredTokenizer.elasticsearchVersion("edge_ngram", (version) -> { + if (version.onOrAfter(Version.V_7_3_0)) { + return new EdgeNGramTokenizer(NGramTokenizer.DEFAULT_MIN_NGRAM_SIZE, NGramTokenizer.DEFAULT_MAX_NGRAM_SIZE); + } + return new EdgeNGramTokenizer(EdgeNGramTokenizer.DEFAULT_MIN_GRAM_SIZE, EdgeNGramTokenizer.DEFAULT_MAX_GRAM_SIZE); + })); tokenizers.add(PreConfiguredTokenizer.singleton("pattern", () -> new PatternTokenizer(Regex.compile("\\W+", null), -1))); tokenizers.add(PreConfiguredTokenizer.singleton("thai", ThaiTokenizer::new)); // TODO deprecate and remove in API @@ -485,8 +486,12 @@ public List getPreConfiguredTokenizers() { // Temporary shim for aliases. TODO deprecate after they are moved tokenizers.add(PreConfiguredTokenizer.singleton("nGram", NGramTokenizer::new)); - tokenizers.add(PreConfiguredTokenizer.singleton("edgeNGram", - () -> new EdgeNGramTokenizer(EdgeNGramTokenizer.DEFAULT_MIN_GRAM_SIZE, EdgeNGramTokenizer.DEFAULT_MAX_GRAM_SIZE))); + tokenizers.add(PreConfiguredTokenizer.elasticsearchVersion("edgeNGram", (version) -> { + if (version.onOrAfter(Version.V_7_3_0)) { + return new EdgeNGramTokenizer(NGramTokenizer.DEFAULT_MIN_NGRAM_SIZE, NGramTokenizer.DEFAULT_MAX_NGRAM_SIZE); + } + return new EdgeNGramTokenizer(EdgeNGramTokenizer.DEFAULT_MIN_GRAM_SIZE, EdgeNGramTokenizer.DEFAULT_MAX_GRAM_SIZE); + })); tokenizers.add(PreConfiguredTokenizer.singleton("PathHierarchy", PathHierarchyTokenizer::new)); return tokenizers; diff --git a/modules/analysis-common/src/main/java/org/elasticsearch/analysis/common/ElisionTokenFilterFactory.java b/modules/analysis-common/src/main/java/org/elasticsearch/analysis/common/ElisionTokenFilterFactory.java index 52cb69952b836..39d042caa8c25 100644 --- a/modules/analysis-common/src/main/java/org/elasticsearch/analysis/common/ElisionTokenFilterFactory.java +++ b/modules/analysis-common/src/main/java/org/elasticsearch/analysis/common/ElisionTokenFilterFactory.java @@ -36,6 +36,9 @@ public class ElisionTokenFilterFactory extends AbstractTokenFilterFactory implem ElisionTokenFilterFactory(IndexSettings indexSettings, Environment env, String name, Settings settings) { super(indexSettings, name, settings); this.articles = Analysis.parseArticles(env, settings); + if (this.articles == null) { + throw new IllegalArgumentException("elision filter requires [articles] or [articles_path] setting"); + } } @Override diff --git a/modules/analysis-common/src/main/java/org/elasticsearch/analysis/common/StandardHtmlStripAnalyzer.java b/modules/analysis-common/src/main/java/org/elasticsearch/analysis/common/StandardHtmlStripAnalyzer.java deleted file mode 100644 index a35a0ea2a4a0b..0000000000000 --- a/modules/analysis-common/src/main/java/org/elasticsearch/analysis/common/StandardHtmlStripAnalyzer.java +++ /dev/null @@ -1,58 +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.analysis.common; - -import org.apache.lucene.analysis.CharArraySet; -import org.apache.lucene.analysis.LowerCaseFilter; -import org.apache.lucene.analysis.StopFilter; -import org.apache.lucene.analysis.StopwordAnalyzerBase; -import org.apache.lucene.analysis.TokenStream; -import org.apache.lucene.analysis.Tokenizer; -import org.apache.lucene.analysis.en.EnglishAnalyzer; -import org.apache.lucene.analysis.standard.StandardTokenizer; - -public class StandardHtmlStripAnalyzer extends StopwordAnalyzerBase { - - /** - * @deprecated use {@link StandardHtmlStripAnalyzer#StandardHtmlStripAnalyzer(CharArraySet)} instead - */ - @Deprecated - public StandardHtmlStripAnalyzer() { - super(EnglishAnalyzer.ENGLISH_STOP_WORDS_SET); - } - /** - * @deprecated in 6.5, can not create in 7.0, and we remove this in 8.0 - */ - @Deprecated - StandardHtmlStripAnalyzer(CharArraySet stopwords) { - super(stopwords); - } - - @Override - protected TokenStreamComponents createComponents(final String fieldName) { - final Tokenizer src = new StandardTokenizer(); - TokenStream tok = new LowerCaseFilter(src); - if (!stopwords.isEmpty()) { - tok = new StopFilter(tok, stopwords); - } - return new TokenStreamComponents(src, tok); - } - -} diff --git a/modules/analysis-common/src/main/java/org/elasticsearch/analysis/common/SynonymTokenFilterFactory.java b/modules/analysis-common/src/main/java/org/elasticsearch/analysis/common/SynonymTokenFilterFactory.java index 75d4eca4254f8..5d6135549b882 100644 --- a/modules/analysis-common/src/main/java/org/elasticsearch/analysis/common/SynonymTokenFilterFactory.java +++ b/modules/analysis-common/src/main/java/org/elasticsearch/analysis/common/SynonymTokenFilterFactory.java @@ -30,6 +30,7 @@ import org.elasticsearch.index.IndexSettings; import org.elasticsearch.index.analysis.AbstractTokenFilterFactory; import org.elasticsearch.index.analysis.Analysis; +import org.elasticsearch.index.analysis.AnalysisMode; import org.elasticsearch.index.analysis.CharFilterFactory; import org.elasticsearch.index.analysis.CustomAnalyzer; import org.elasticsearch.index.analysis.TokenFilterFactory; @@ -50,6 +51,7 @@ public class SynonymTokenFilterFactory extends AbstractTokenFilterFactory { private final boolean lenient; protected final Settings settings; protected final Environment environment; + private final boolean updateable; SynonymTokenFilterFactory(IndexSettings indexSettings, Environment env, String name, Settings settings) { @@ -65,9 +67,15 @@ public class SynonymTokenFilterFactory extends AbstractTokenFilterFactory { this.expand = settings.getAsBoolean("expand", true); this.lenient = settings.getAsBoolean("lenient", false); this.format = settings.get("format", ""); + this.updateable = settings.getAsBoolean("updateable", false); this.environment = env; } + @Override + public AnalysisMode getAnalysisMode() { + return this.updateable ? AnalysisMode.SEARCH_TIME : AnalysisMode.ALL; + } + @Override public TokenStream create(TokenStream tokenStream) { throw new IllegalStateException("Call createPerAnalyzerSynonymFactory to specialize this factory for an analysis chain first"); @@ -98,6 +106,11 @@ public TokenFilterFactory getSynonymFilter() { // which doesn't support stacked input tokens return IDENTITY_FILTER; } + + @Override + public AnalysisMode getAnalysisMode() { + return updateable ? AnalysisMode.SEARCH_TIME : AnalysisMode.ALL; + } }; } diff --git a/modules/analysis-common/src/test/java/org/elasticsearch/analysis/common/EdgeNGramTokenizerTests.java b/modules/analysis-common/src/test/java/org/elasticsearch/analysis/common/EdgeNGramTokenizerTests.java new file mode 100644 index 0000000000000..0172f7cbc2657 --- /dev/null +++ b/modules/analysis-common/src/test/java/org/elasticsearch/analysis/common/EdgeNGramTokenizerTests.java @@ -0,0 +1,98 @@ +/* + * 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.analysis.common; + +import org.elasticsearch.Version; +import org.elasticsearch.cluster.metadata.IndexMetaData; +import org.elasticsearch.common.settings.Settings; +import org.elasticsearch.env.Environment; +import org.elasticsearch.env.TestEnvironment; +import org.elasticsearch.index.IndexSettings; +import org.elasticsearch.index.analysis.IndexAnalyzers; +import org.elasticsearch.index.analysis.NamedAnalyzer; +import org.elasticsearch.indices.analysis.AnalysisModule; +import org.elasticsearch.test.ESTokenStreamTestCase; +import org.elasticsearch.test.IndexSettingsModule; +import org.elasticsearch.test.VersionUtils; + +import java.io.IOException; +import java.util.Collections; + +public class EdgeNGramTokenizerTests extends ESTokenStreamTestCase { + + private IndexAnalyzers buildAnalyzers(Version version, String tokenizer) throws IOException { + Settings settings = Settings.builder() + .put(Environment.PATH_HOME_SETTING.getKey(), createTempDir().toString()) + .build(); + Settings indexSettings = Settings.builder() + .put(IndexMetaData.SETTING_VERSION_CREATED, version) + .put("index.analysis.analyzer.my_analyzer.tokenizer", tokenizer) + .build(); + IndexSettings idxSettings = IndexSettingsModule.newIndexSettings("index", indexSettings); + return new AnalysisModule(TestEnvironment.newEnvironment(settings), + Collections.singletonList(new CommonAnalysisPlugin())).getAnalysisRegistry().build(idxSettings); + } + + public void testPreConfiguredTokenizer() throws IOException { + + // Before 7.3 we return ngrams of length 1 only + { + Version version = VersionUtils.randomVersionBetween(random(), Version.V_7_0_0, + VersionUtils.getPreviousVersion(Version.V_7_3_0)); + try (IndexAnalyzers indexAnalyzers = buildAnalyzers(version, "edge_ngram")) { + NamedAnalyzer analyzer = indexAnalyzers.get("my_analyzer"); + assertNotNull(analyzer); + assertAnalyzesTo(analyzer, "test", new String[]{"t"}); + } + } + + // Check deprecated name as well + { + Version version = VersionUtils.randomVersionBetween(random(), Version.V_7_0_0, + VersionUtils.getPreviousVersion(Version.V_7_3_0)); + try (IndexAnalyzers indexAnalyzers = buildAnalyzers(version, "edgeNGram")) { + NamedAnalyzer analyzer = indexAnalyzers.get("my_analyzer"); + assertNotNull(analyzer); + assertAnalyzesTo(analyzer, "test", new String[]{"t"}); + } + } + + // Afterwards, we return ngrams of length 1 and 2, to match the default factory settings + { + try (IndexAnalyzers indexAnalyzers = buildAnalyzers(Version.CURRENT, "edge_ngram")) { + NamedAnalyzer analyzer = indexAnalyzers.get("my_analyzer"); + assertNotNull(analyzer); + assertAnalyzesTo(analyzer, "test", new String[]{"t", "te"}); + } + } + + // Check deprecated name as well + { + try (IndexAnalyzers indexAnalyzers = buildAnalyzers(Version.CURRENT, "edgeNGram")) { + NamedAnalyzer analyzer = indexAnalyzers.get("my_analyzer"); + assertNotNull(analyzer); + assertAnalyzesTo(analyzer, "test", new String[]{"t", "te"}); + + } + } + + } + +} diff --git a/modules/analysis-common/src/test/java/org/elasticsearch/analysis/common/ElisionFilterFactoryTests.java b/modules/analysis-common/src/test/java/org/elasticsearch/analysis/common/ElisionFilterFactoryTests.java new file mode 100644 index 0000000000000..dbfd49d5649d5 --- /dev/null +++ b/modules/analysis-common/src/test/java/org/elasticsearch/analysis/common/ElisionFilterFactoryTests.java @@ -0,0 +1,43 @@ +/* + * 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.analysis.common; + +import org.elasticsearch.common.settings.Settings; +import org.elasticsearch.env.Environment; +import org.elasticsearch.index.analysis.AnalysisTestsHelper; +import org.elasticsearch.test.ESTokenStreamTestCase; + +import java.io.IOException; + +public class ElisionFilterFactoryTests extends ESTokenStreamTestCase { + + public void testElisionFilterWithNoArticles() throws IOException { + Settings settings = Settings.builder() + .put("index.analysis.filter.elision.type", "elision") + .put(Environment.PATH_HOME_SETTING.getKey(), createTempDir().toString()) + .build(); + + IllegalArgumentException e = expectThrows(IllegalArgumentException.class, + () -> AnalysisTestsHelper.createTestAnalysisFromSettings(settings, new CommonAnalysisPlugin())); + + assertEquals("elision filter requires [articles] or [articles_path] setting", e.getMessage()); + } + +} diff --git a/modules/analysis-common/src/test/java/org/elasticsearch/analysis/common/HighlighterWithAnalyzersTests.java b/modules/analysis-common/src/test/java/org/elasticsearch/analysis/common/HighlighterWithAnalyzersTests.java index 8f58a074cf102..c4cc7589678ad 100644 --- a/modules/analysis-common/src/test/java/org/elasticsearch/analysis/common/HighlighterWithAnalyzersTests.java +++ b/modules/analysis-common/src/test/java/org/elasticsearch/analysis/common/HighlighterWithAnalyzersTests.java @@ -108,7 +108,7 @@ public void testNgramHighlightingWithBrokenPositions() throws IOException { client().prepareIndex("test", "test", "1") .setSource("name", "ARCOTEL Hotels Deutschland").get(); refresh(); - SearchResponse search = client().prepareSearch("test").setTypes("test") + SearchResponse search = client().prepareSearch("test") .setQuery(matchQuery("name.autocomplete", "deut tel").operator(Operator.OR)) .highlighter(new HighlightBuilder().field("name.autocomplete")).get(); assertHighlight(search, 0, "name.autocomplete", 0, diff --git a/modules/analysis-common/src/test/java/org/elasticsearch/analysis/common/WordDelimiterGraphTokenFilterFactoryTests.java b/modules/analysis-common/src/test/java/org/elasticsearch/analysis/common/WordDelimiterGraphTokenFilterFactoryTests.java index d799674f231a1..c8e3699ea840d 100644 --- a/modules/analysis-common/src/test/java/org/elasticsearch/analysis/common/WordDelimiterGraphTokenFilterFactoryTests.java +++ b/modules/analysis-common/src/test/java/org/elasticsearch/analysis/common/WordDelimiterGraphTokenFilterFactoryTests.java @@ -20,14 +20,24 @@ import org.apache.lucene.analysis.Tokenizer; import org.apache.lucene.analysis.core.WhitespaceTokenizer; +import org.elasticsearch.Version; +import org.elasticsearch.cluster.metadata.IndexMetaData; import org.elasticsearch.common.settings.Settings; import org.elasticsearch.env.Environment; +import org.elasticsearch.env.TestEnvironment; +import org.elasticsearch.index.IndexSettings; import org.elasticsearch.index.analysis.AnalysisTestsHelper; +import org.elasticsearch.index.analysis.IndexAnalyzers; +import org.elasticsearch.index.analysis.NamedAnalyzer; import org.elasticsearch.index.analysis.TokenFilterFactory; +import org.elasticsearch.indices.analysis.AnalysisModule; import org.elasticsearch.test.ESTestCase; +import org.elasticsearch.test.IndexSettingsModule; +import org.elasticsearch.test.VersionUtils; import java.io.IOException; import java.io.StringReader; +import java.util.Collections; public class WordDelimiterGraphTokenFilterFactoryTests extends BaseWordDelimiterTokenFilterFactoryTestCase { @@ -107,4 +117,51 @@ public void testAdjustingOffsets() throws IOException { assertTokenStreamContents(tokenFilter.create(tokenizer), expected, expectedStartOffsets, expectedEndOffsets, null, expectedIncr, expectedPosLen, null); } + + public void testPreconfiguredFilter() throws IOException { + // Before 7.3 we don't adjust offsets + { + Settings settings = Settings.builder() + .put(Environment.PATH_HOME_SETTING.getKey(), createTempDir().toString()) + .build(); + Settings indexSettings = Settings.builder() + .put(IndexMetaData.SETTING_VERSION_CREATED, + VersionUtils.randomVersionBetween(random(), Version.V_7_0_0, VersionUtils.getPreviousVersion(Version.V_7_3_0))) + .put("index.analysis.analyzer.my_analyzer.tokenizer", "standard") + .putList("index.analysis.analyzer.my_analyzer.filter", "word_delimiter_graph") + .build(); + IndexSettings idxSettings = IndexSettingsModule.newIndexSettings("index", indexSettings); + + try (IndexAnalyzers indexAnalyzers = new AnalysisModule(TestEnvironment.newEnvironment(settings), + Collections.singletonList(new CommonAnalysisPlugin())).getAnalysisRegistry().build(idxSettings)) { + + NamedAnalyzer analyzer = indexAnalyzers.get("my_analyzer"); + assertNotNull(analyzer); + assertAnalyzesTo(analyzer, "h100", new String[]{"h", "100"}, new int[]{ 0, 0 }, new int[]{ 4, 4 }); + + } + } + + // Afger 7.3 we do adjust offsets + { + Settings settings = Settings.builder() + .put(Environment.PATH_HOME_SETTING.getKey(), createTempDir().toString()) + .build(); + Settings indexSettings = Settings.builder() + .put(IndexMetaData.SETTING_VERSION_CREATED, Version.CURRENT) + .put("index.analysis.analyzer.my_analyzer.tokenizer", "standard") + .putList("index.analysis.analyzer.my_analyzer.filter", "word_delimiter_graph") + .build(); + IndexSettings idxSettings = IndexSettingsModule.newIndexSettings("index", indexSettings); + + try (IndexAnalyzers indexAnalyzers = new AnalysisModule(TestEnvironment.newEnvironment(settings), + Collections.singletonList(new CommonAnalysisPlugin())).getAnalysisRegistry().build(idxSettings)) { + + NamedAnalyzer analyzer = indexAnalyzers.get("my_analyzer"); + assertNotNull(analyzer); + assertAnalyzesTo(analyzer, "h100", new String[]{"h", "100"}, new int[]{ 0, 1 }, new int[]{ 1, 4 }); + + } + } + } } diff --git a/modules/analysis-common/src/test/resources/rest-api-spec/test/analysis-common/20_analyzers.yml b/modules/analysis-common/src/test/resources/rest-api-spec/test/analysis-common/20_analyzers.yml index 15de6fe664786..2904cc3e95b58 100644 --- a/modules/analysis-common/src/test/resources/rest-api-spec/test/analysis-common/20_analyzers.yml +++ b/modules/analysis-common/src/test/resources/rest-api-spec/test/analysis-common/20_analyzers.yml @@ -67,15 +67,6 @@ - length: { tokens: 1 } - match: { tokens.0.token: a1 b2 c3 d4 } ---- -"standard_html_strip": - - do: - catch: /\[standard_html_strip\] analyzer is not supported for new indices, use a custom analyzer using \[standard\] tokenizer and \[html_strip\] char_filter, plus \[lowercase\] filter/ - indices.analyze: - body: - text: - analyzer: standard_html_strip - --- "pattern": - do: diff --git a/modules/analysis-common/src/test/resources/rest-api-spec/test/analysis-common/40_token_filters.yml b/modules/analysis-common/src/test/resources/rest-api-spec/test/analysis-common/40_token_filters.yml index 3486b9defd9d2..92d0dce7b6201 100644 --- a/modules/analysis-common/src/test/resources/rest-api-spec/test/analysis-common/40_token_filters.yml +++ b/modules/analysis-common/src/test/resources/rest-api-spec/test/analysis-common/40_token_filters.yml @@ -587,6 +587,20 @@ - length: { tokens: 1 } - match: { tokens.0.token: avion } + - do: + catch: bad_request + indices.create: + index: test2 + body: + settings: + analysis: + filter: + my_elision: + type: elision + - match: { status: 400 } + - match: { error.type: illegal_argument_exception } + - match: { error.reason: "elision filter requires [articles] or [articles_path] setting" } + --- "stemmer": - do: diff --git a/modules/analysis-common/src/test/resources/rest-api-spec/test/search.query/50_queries_with_synonyms.yml b/modules/analysis-common/src/test/resources/rest-api-spec/test/search.query/50_queries_with_synonyms.yml index 784ffd9dd123a..dca56565e6954 100644 --- a/modules/analysis-common/src/test/resources/rest-api-spec/test/search.query/50_queries_with_synonyms.yml +++ b/modules/analysis-common/src/test/resources/rest-api-spec/test/search.query/50_queries_with_synonyms.yml @@ -46,117 +46,6 @@ field1: the quick lazy huge brown fox jumps over the tree refresh: true - - do: - search: - rest_total_hits_as_int: true - body: - query: - common: - field1: - query: the fast brown - cutoff_frequency: 3 - low_freq_operator: or - - match: { hits.total: 3 } - - match: { hits.hits.0._id: "1" } - - match: { hits.hits.1._id: "2" } - - match: { hits.hits.2._id: "3" } - - - do: - search: - rest_total_hits_as_int: true - body: - query: - common: - field1: - query: the fast brown - cutoff_frequency: 3 - low_freq_operator: and - - match: { hits.total: 2 } - - match: { hits.hits.0._id: "1" } - - match: { hits.hits.1._id: "2" } - - - do: - search: - rest_total_hits_as_int: true - body: - query: - common: - field1: - query: the fast brown - cutoff_frequency: 3 - - match: { hits.total: 3 } - - match: { hits.hits.0._id: "1" } - - match: { hits.hits.1._id: "2" } - - match: { hits.hits.2._id: "3" } - - - do: - search: - rest_total_hits_as_int: true - body: - query: - common: - field1: - query: the fast huge fox - minimum_should_match: - low_freq: 3 - - match: { hits.total: 1 } - - match: { hits.hits.0._id: "2" } - - - do: - search: - rest_total_hits_as_int: true - body: - query: - common: - field1: - query: the fast lazy fox brown - cutoff_frequency: 1 - minimum_should_match: - high_freq: 5 - - match: { hits.total: 2 } - - match: { hits.hits.0._id: "2" } - - match: { hits.hits.1._id: "1" } - - - do: - search: - rest_total_hits_as_int: true - body: - query: - common: - field1: - query: the fast lazy fox brown - cutoff_frequency: 1 - minimum_should_match: - high_freq: 6 - - match: { hits.total: 1 } - - match: { hits.hits.0._id: "2" } - - - do: - search: - rest_total_hits_as_int: true - body: - query: - common: - field1: - query: the fast lazy fox brown - cutoff_frequency: 1 - - match: { hits.total: 1 } - - match: { hits.hits.0._id: "2" } - - - do: - search: - rest_total_hits_as_int: true - body: - query: - common: - field1: - query: the quick brown - cutoff_frequency: 3 - - match: { hits.total: 3 } - - match: { hits.hits.0._id: "1" } - - match: { hits.hits.1._id: "2" } - - match: { hits.hits.2._id: "3" } - - do: search: rest_total_hits_as_int: true @@ -165,7 +54,6 @@ match: field1: query: the fast brown - cutoff_frequency: 3 operator: and - match: { hits.total: 2 } - match: { hits.hits.0._id: "1" } @@ -179,7 +67,6 @@ match: field1: query: the fast brown - cutoff_frequency: 3 operator: or - match: { hits.total: 3 } - match: { hits.hits.0._id: "1" } @@ -194,13 +81,13 @@ match: field1: query: the fast brown - cutoff_frequency: 3 minimum_should_match: 3 - match: { hits.total: 2 } - match: { hits.hits.0._id: "1" } - match: { hits.hits.1._id: "2" } - do: + warnings: search: rest_total_hits_as_int: true body: @@ -208,7 +95,6 @@ multi_match: query: the fast brown fields: [ "field1", "field2" ] - cutoff_frequency: 3 operator: and - match: { hits.total: 3 } - match: { hits.hits.0._id: "3" } diff --git a/modules/ingest-common/build.gradle b/modules/ingest-common/build.gradle index a94c375afc3e1..00c444e50e87d 100644 --- a/modules/ingest-common/build.gradle +++ b/modules/ingest-common/build.gradle @@ -25,6 +25,6 @@ esplugin { dependencies { compileOnly project(':modules:lang-painless') - compile project(':libs:grok') - compile project(':libs:dissect') + compile project(':libs:elasticsearch-grok') + compile project(':libs:elasticsearch-dissect') } \ No newline at end of file diff --git a/modules/ingest-common/src/main/java/org/elasticsearch/ingest/common/GrokProcessorGetAction.java b/modules/ingest-common/src/main/java/org/elasticsearch/ingest/common/GrokProcessorGetAction.java index 1141a4cf7e891..f6eadab8014d2 100644 --- a/modules/ingest-common/src/main/java/org/elasticsearch/ingest/common/GrokProcessorGetAction.java +++ b/modules/ingest-common/src/main/java/org/elasticsearch/ingest/common/GrokProcessorGetAction.java @@ -18,11 +18,11 @@ */ package org.elasticsearch.ingest.common; -import org.elasticsearch.action.Action; import org.elasticsearch.action.ActionListener; import org.elasticsearch.action.ActionRequest; import org.elasticsearch.action.ActionRequestValidationException; import org.elasticsearch.action.ActionResponse; +import org.elasticsearch.action.StreamableResponseActionType; import org.elasticsearch.action.support.ActionFilters; import org.elasticsearch.action.support.HandledTransportAction; import org.elasticsearch.client.node.NodeClient; @@ -45,7 +45,7 @@ import static org.elasticsearch.ingest.common.IngestCommonPlugin.GROK_PATTERNS; import static org.elasticsearch.rest.RestRequest.Method.GET; -public class GrokProcessorGetAction extends Action { +public class GrokProcessorGetAction extends StreamableResponseActionType { static final GrokProcessorGetAction INSTANCE = new GrokProcessorGetAction(); static final String NAME = "cluster:admin/ingest/processor/grok/get"; diff --git a/modules/ingest-common/src/test/java/org/elasticsearch/ingest/common/RenameProcessorTests.java b/modules/ingest-common/src/test/java/org/elasticsearch/ingest/common/RenameProcessorTests.java index 5b3d88879ba13..5c3b0b0e7f3b5 100644 --- a/modules/ingest-common/src/test/java/org/elasticsearch/ingest/common/RenameProcessorTests.java +++ b/modules/ingest-common/src/test/java/org/elasticsearch/ingest/common/RenameProcessorTests.java @@ -132,7 +132,11 @@ public void testRenameExistingFieldNullValue() throws Exception { String newFieldName = randomValueOtherThanMany(ingestDocument::hasField, () -> RandomDocumentPicks.randomFieldName(random())); Processor processor = createRenameProcessor(fieldName, newFieldName, false); processor.execute(ingestDocument); - assertThat(ingestDocument.hasField(fieldName), equalTo(false)); + if (newFieldName.startsWith(fieldName + '.')) { + assertThat(ingestDocument.getFieldValue(fieldName, Object.class), instanceOf(Map.class)); + } else { + assertThat(ingestDocument.hasField(fieldName), equalTo(false)); + } assertThat(ingestDocument.hasField(newFieldName), equalTo(true)); assertThat(ingestDocument.getFieldValue(newFieldName, Object.class), nullValue()); } diff --git a/modules/ingest-common/src/test/resources/rest-api-spec/test/ingest/200_default_pipeline.yml b/modules/ingest-common/src/test/resources/rest-api-spec/test/ingest/200_default_pipeline.yml index 86f4821ddaa23..3d100fad3b027 100644 --- a/modules/ingest-common/src/test/resources/rest-api-spec/test/ingest/200_default_pipeline.yml +++ b/modules/ingest-common/src/test/resources/rest-api-spec/test/ingest/200_default_pipeline.yml @@ -119,6 +119,12 @@ teardown: {"doc":{"bytes_source_field":"2kb"}, "doc_as_upsert":true} {"update":{"_id":"8","_index":"test"}} {"script": "ctx._source.ran_script = true","upsert":{"bytes_source_field":"3kb"}, "scripted_upsert" : true} + {"update":{"_id":"6_alias","_index":"test_alias"}} + {"script":"ctx._source.ran_script = true","upsert":{"bytes_source_field":"1kb"}} + {"update":{"_id":"7_alias","_index":"test_alias"}} + {"doc":{"bytes_source_field":"2kb"}, "doc_as_upsert":true} + {"update":{"_id":"8_alias","_index":"test_alias"}} + {"script": "ctx._source.ran_script = true","upsert":{"bytes_source_field":"3kb"}, "scripted_upsert" : true} - do: mget: @@ -127,6 +133,9 @@ teardown: - { _index: "test", _id: "6" } - { _index: "test", _id: "7" } - { _index: "test", _id: "8" } + - { _index: "test", _id: "6_alias" } + - { _index: "test", _id: "7_alias" } + - { _index: "test", _id: "8_alias" } - match: { docs.0._index: "test" } - match: { docs.0._id: "6" } - match: { docs.0._source.bytes_source_field: "1kb" } @@ -141,6 +150,20 @@ teardown: - match: { docs.2._source.bytes_source_field: "3kb" } - match: { docs.2._source.bytes_target_field: 3072 } - match: { docs.2._source.ran_script: true } + - match: { docs.3._index: "test" } + - match: { docs.3._id: "6_alias" } + - match: { docs.3._source.bytes_source_field: "1kb" } + - match: { docs.3._source.bytes_target_field: 1024 } + - is_false: docs.3._source.ran_script + - match: { docs.4._index: "test" } + - match: { docs.4._id: "7_alias" } + - match: { docs.4._source.bytes_source_field: "2kb" } + - match: { docs.4._source.bytes_target_field: 2048 } + - match: { docs.5._index: "test" } + - match: { docs.5._id: "8_alias" } + - match: { docs.5._source.bytes_source_field: "3kb" } + - match: { docs.5._source.bytes_target_field: 3072 } + - match: { docs.5._source.ran_script: true } # explicit no default pipeline - do: diff --git a/modules/lang-expression/src/main/java/org/elasticsearch/script/expression/ExpressionAggregationScript.java b/modules/lang-expression/src/main/java/org/elasticsearch/script/expression/ExpressionAggregationScript.java index 199f52c40319d..ba7b1b68c41e5 100644 --- a/modules/lang-expression/src/main/java/org/elasticsearch/script/expression/ExpressionAggregationScript.java +++ b/modules/lang-expression/src/main/java/org/elasticsearch/script/expression/ExpressionAggregationScript.java @@ -38,20 +38,37 @@ class ExpressionAggregationScript implements AggregationScript.LeafFactory { final Expression exprScript; final SimpleBindings bindings; final DoubleValuesSource source; + final boolean needsScore; final ReplaceableConstDoubleValueSource specialValue; // _value - ExpressionAggregationScript(Expression e, SimpleBindings b, ReplaceableConstDoubleValueSource v) { + ExpressionAggregationScript(Expression e, SimpleBindings b, boolean n, ReplaceableConstDoubleValueSource v) { exprScript = e; bindings = b; source = exprScript.getDoubleValuesSource(bindings); + needsScore = n; specialValue = v; } + @Override + public boolean needs_score() { + return needsScore; + } + @Override public AggregationScript newInstance(final LeafReaderContext leaf) throws IOException { return new AggregationScript() { // Fake the scorer until setScorer is called. - DoubleValues values = source.getValues(leaf, null); + DoubleValues values = source.getValues(leaf, new DoubleValues() { + @Override + public double doubleValue() throws IOException { + return get_score().doubleValue(); + } + + @Override + public boolean advanceExact(int doc) throws IOException { + return true; + } + }); @Override public Object execute() { @@ -84,10 +101,4 @@ public void setNextAggregationValue(Object value) { } }; } - - @Override - public boolean needs_score() { - return false; - } - } 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 1ad0fff49b78a..abd8738b0e4c1 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 @@ -221,10 +221,14 @@ private AggregationScript.LeafFactory newAggregationScript(Expression expr, Sear // NOTE: if we need to do anything complicated with bindings in the future, we can just extend Bindings, // instead of complicating SimpleBindings (which should stay simple) SimpleBindings bindings = new SimpleBindings(); + boolean needsScores = false; ReplaceableConstDoubleValueSource specialValue = null; for (String variable : expr.variables) { try { - if (variable.equals("_value")) { + if (variable.equals("_score")) { + bindings.add(new SortField("_score", SortField.Type.SCORE)); + needsScores = true; + } else if (variable.equals("_value")) { specialValue = new ReplaceableConstDoubleValueSource(); bindings.add("_value", specialValue); // noop: _value is special for aggregations, and is handled in ExpressionScriptBindings @@ -237,6 +241,7 @@ private AggregationScript.LeafFactory newAggregationScript(Expression expr, Sear // delegate valuesource creation based on field's type // there are three types of "fields" to expressions, and each one has a different "api" of variables and methods. final ValueSource valueSource = getDocValueSource(variable, lookup); + needsScores |= valueSource.getSortField(false).needsScores(); bindings.add(variable, valueSource.asDoubleValuesSource()); } } catch (Exception e) { @@ -244,7 +249,7 @@ private AggregationScript.LeafFactory newAggregationScript(Expression expr, Sear throw convertToScriptException("link error", expr.sourceText, variable, e); } } - return new ExpressionAggregationScript(expr, bindings, specialValue); + return new ExpressionAggregationScript(expr, bindings, needsScores, specialValue); } private FieldScript.LeafFactory newFieldScript(Expression expr, SearchLookup lookup, @Nullable Map vars) { diff --git a/modules/lang-expression/src/test/java/org/elasticsearch/script/expression/ExpressionFieldScriptTests.java b/modules/lang-expression/src/test/java/org/elasticsearch/script/expression/ExpressionFieldScriptTests.java index 205e638314fe3..abc1b8e882389 100644 --- a/modules/lang-expression/src/test/java/org/elasticsearch/script/expression/ExpressionFieldScriptTests.java +++ b/modules/lang-expression/src/test/java/org/elasticsearch/script/expression/ExpressionFieldScriptTests.java @@ -64,7 +64,7 @@ public void setUp() throws Exception { when(fieldData.load(anyObject())).thenReturn(atomicFieldData); service = new ExpressionScriptEngine(); - lookup = new SearchLookup(mapperService, ignored -> fieldData, null); + lookup = new SearchLookup(mapperService, ignored -> fieldData); } private FieldScript.LeafFactory compile(String expression) { diff --git a/modules/lang-expression/src/test/java/org/elasticsearch/script/expression/ExpressionNumberSortScriptTests.java b/modules/lang-expression/src/test/java/org/elasticsearch/script/expression/ExpressionNumberSortScriptTests.java index e6bd503bfabe1..94acf6b35ab98 100644 --- a/modules/lang-expression/src/test/java/org/elasticsearch/script/expression/ExpressionNumberSortScriptTests.java +++ b/modules/lang-expression/src/test/java/org/elasticsearch/script/expression/ExpressionNumberSortScriptTests.java @@ -19,9 +19,6 @@ package org.elasticsearch.script.expression; -import java.io.IOException; -import java.text.ParseException; -import java.util.Collections; import org.elasticsearch.index.fielddata.AtomicNumericFieldData; import org.elasticsearch.index.fielddata.IndexNumericFieldData; import org.elasticsearch.index.fielddata.SortedNumericDoubleValues; @@ -33,6 +30,10 @@ import org.elasticsearch.search.lookup.SearchLookup; import org.elasticsearch.test.ESTestCase; +import java.io.IOException; +import java.text.ParseException; +import java.util.Collections; + import static org.mockito.Matchers.anyInt; import static org.mockito.Matchers.anyObject; import static org.mockito.Mockito.mock; @@ -63,7 +64,7 @@ public void setUp() throws Exception { when(fieldData.load(anyObject())).thenReturn(atomicFieldData); service = new ExpressionScriptEngine(); - lookup = new SearchLookup(mapperService, ignored -> fieldData, null); + lookup = new SearchLookup(mapperService, ignored -> fieldData); } private NumberSortScript.LeafFactory compile(String expression) { diff --git a/modules/lang-expression/src/test/java/org/elasticsearch/script/expression/ExpressionTermsSetQueryTests.java b/modules/lang-expression/src/test/java/org/elasticsearch/script/expression/ExpressionTermsSetQueryTests.java index 137f8e058cd85..cc67501eba319 100644 --- a/modules/lang-expression/src/test/java/org/elasticsearch/script/expression/ExpressionTermsSetQueryTests.java +++ b/modules/lang-expression/src/test/java/org/elasticsearch/script/expression/ExpressionTermsSetQueryTests.java @@ -19,9 +19,6 @@ package org.elasticsearch.script.expression; -import java.io.IOException; -import java.text.ParseException; -import java.util.Collections; import org.elasticsearch.index.fielddata.AtomicNumericFieldData; import org.elasticsearch.index.fielddata.IndexNumericFieldData; import org.elasticsearch.index.fielddata.SortedNumericDoubleValues; @@ -33,6 +30,10 @@ import org.elasticsearch.search.lookup.SearchLookup; import org.elasticsearch.test.ESTestCase; +import java.io.IOException; +import java.text.ParseException; +import java.util.Collections; + import static org.mockito.Matchers.anyInt; import static org.mockito.Matchers.anyObject; import static org.mockito.Mockito.mock; @@ -63,7 +64,7 @@ public void setUp() throws Exception { when(fieldData.load(anyObject())).thenReturn(atomicFieldData); service = new ExpressionScriptEngine(); - lookup = new SearchLookup(mapperService, ignored -> fieldData, null); + lookup = new SearchLookup(mapperService, ignored -> fieldData); } private TermsSetQueryScript.LeafFactory compile(String expression) { diff --git a/modules/lang-expression/src/test/java/org/elasticsearch/script/expression/MoreExpressionTests.java b/modules/lang-expression/src/test/java/org/elasticsearch/script/expression/MoreExpressionTests.java index 4f8fa5f463edd..6e1c0efb7ded9 100644 --- a/modules/lang-expression/src/test/java/org/elasticsearch/script/expression/MoreExpressionTests.java +++ b/modules/lang-expression/src/test/java/org/elasticsearch/script/expression/MoreExpressionTests.java @@ -28,8 +28,8 @@ import org.elasticsearch.common.xcontent.XContentBuilder; import org.elasticsearch.common.xcontent.XContentFactory; import org.elasticsearch.index.query.QueryBuilders; -import org.elasticsearch.index.query.functionscore.ScoreFunctionBuilder; import org.elasticsearch.index.query.functionscore.ScoreFunctionBuilders; +import org.elasticsearch.index.query.functionscore.ScriptScoreFunctionBuilder; import org.elasticsearch.plugins.Plugin; import org.elasticsearch.script.Script; import org.elasticsearch.script.ScriptType; @@ -120,7 +120,7 @@ public void testScore() throws Exception { client().prepareIndex("test", "doc", "1").setSource("text", "hello goodbye"), client().prepareIndex("test", "doc", "2").setSource("text", "hello hello hello goodbye"), client().prepareIndex("test", "doc", "3").setSource("text", "hello hello goodebye")); - ScoreFunctionBuilder score = ScoreFunctionBuilders.scriptFunction( + ScriptScoreFunctionBuilder score = ScoreFunctionBuilders.scriptFunction( new Script(ScriptType.INLINE, "expression", "1 / _score", Collections.emptyMap())); SearchRequestBuilder req = client().prepareSearch().setIndices("test"); req.setQuery(QueryBuilders.functionScoreQuery(QueryBuilders.termQuery("text", "hello"), score).boostMode(CombineFunction.REPLACE)); @@ -132,6 +132,15 @@ public void testScore() throws Exception { assertEquals("1", hits.getAt(0).getId()); assertEquals("3", hits.getAt(1).getId()); assertEquals("2", hits.getAt(2).getId()); + + req = client().prepareSearch().setIndices("test"); + req.setQuery(QueryBuilders.functionScoreQuery(QueryBuilders.termQuery("text", "hello"), score).boostMode(CombineFunction.REPLACE)); + score = ScoreFunctionBuilders.scriptFunction( + new Script(ScriptType.INLINE, "expression", "1 / _score", Collections.emptyMap())); + req.addAggregation(AggregationBuilders.max("max_score").script((score).getScript())); + req.setSearchType(SearchType.DFS_QUERY_THEN_FETCH); // make sure DF is consistent + rsp = req.get(); + assertSearchResponse(rsp); } public void testDateMethods() throws Exception { diff --git a/modules/lang-expression/src/test/java/org/elasticsearch/script/expression/StoredExpressionTests.java b/modules/lang-expression/src/test/java/org/elasticsearch/script/expression/StoredExpressionTests.java index 7f7f30f271acf..29e13eebf5845 100644 --- a/modules/lang-expression/src/test/java/org/elasticsearch/script/expression/StoredExpressionTests.java +++ b/modules/lang-expression/src/test/java/org/elasticsearch/script/expression/StoredExpressionTests.java @@ -67,7 +67,7 @@ public void testAllOpsDisabledIndexedScripts() throws IOException { client().prepareSearch() .setSource(new SearchSourceBuilder().scriptField("test1", new Script(ScriptType.STORED, null, "script1", Collections.emptyMap()))) - .setIndices("test").setTypes("scriptTest").get(); + .setIndices("test").get(); fail("search script should have been rejected"); } catch(Exception e) { assertThat(e.toString(), containsString("cannot execute scripts using [field] context")); diff --git a/modules/lang-expression/src/test/resources/rest-api-spec/test/lang_expression/20_search.yml b/modules/lang-expression/src/test/resources/rest-api-spec/test/lang_expression/20_search.yml index fd8a2ebf1183b..33bad01a1232a 100644 --- a/modules/lang-expression/src/test/resources/rest-api-spec/test/lang_expression/20_search.yml +++ b/modules/lang-expression/src/test/resources/rest-api-spec/test/lang_expression/20_search.yml @@ -25,9 +25,25 @@ setup: rest_total_hits_as_int: true body: script_fields: - my_field : + my_field: script: lang: expression source: 'doc["age"].value + 19' - - match: { hits.hits.0.fields.my_field.0: 42.0 } + - match: { hits.hits.0.fields.my_field.0: 42.0 } + +--- +"Expressions aggregation score test": + + - do: + search: + rest_total_hits_as_int: true + body: + aggs: + max_score: + max: + script: + lang: expression + source: '_score' + + - match: { aggregations.max_score.value: 1.0 } diff --git a/modules/lang-mustache/build.gradle b/modules/lang-mustache/build.gradle index 2a46bd9ed2efa..f493a85ab67d4 100644 --- a/modules/lang-mustache/build.gradle +++ b/modules/lang-mustache/build.gradle @@ -20,7 +20,6 @@ esplugin { description 'Mustache scripting integration for Elasticsearch' classname 'org.elasticsearch.script.mustache.MustachePlugin' - hasClientJar = true // For the template apis and query } dependencies { diff --git a/modules/lang-mustache/src/main/java/org/elasticsearch/script/mustache/MultiSearchTemplateAction.java b/modules/lang-mustache/src/main/java/org/elasticsearch/script/mustache/MultiSearchTemplateAction.java index a9a44d0471586..a096a89951e7c 100644 --- a/modules/lang-mustache/src/main/java/org/elasticsearch/script/mustache/MultiSearchTemplateAction.java +++ b/modules/lang-mustache/src/main/java/org/elasticsearch/script/mustache/MultiSearchTemplateAction.java @@ -19,10 +19,10 @@ package org.elasticsearch.script.mustache; -import org.elasticsearch.action.Action; +import org.elasticsearch.action.ActionType; import org.elasticsearch.common.io.stream.Writeable; -public class MultiSearchTemplateAction extends Action { +public class MultiSearchTemplateAction extends ActionType { public static final MultiSearchTemplateAction INSTANCE = new MultiSearchTemplateAction(); public static final String NAME = "indices:data/read/msearch/template"; @@ -31,11 +31,6 @@ private MultiSearchTemplateAction() { super(NAME); } - @Override - public MultiSearchTemplateResponse newResponse() { - throw new UnsupportedOperationException("usage of Streamable is to be replaced by Writeable"); - } - @Override public Writeable.Reader getResponseReader() { return MultiSearchTemplateResponse::new; diff --git a/modules/lang-mustache/src/main/java/org/elasticsearch/script/mustache/SearchTemplateAction.java b/modules/lang-mustache/src/main/java/org/elasticsearch/script/mustache/SearchTemplateAction.java index 5d905ec39e1ab..2e8417c993990 100644 --- a/modules/lang-mustache/src/main/java/org/elasticsearch/script/mustache/SearchTemplateAction.java +++ b/modules/lang-mustache/src/main/java/org/elasticsearch/script/mustache/SearchTemplateAction.java @@ -19,10 +19,10 @@ package org.elasticsearch.script.mustache; -import org.elasticsearch.action.Action; +import org.elasticsearch.action.ActionType; import org.elasticsearch.common.io.stream.Writeable; -public class SearchTemplateAction extends Action { +public class SearchTemplateAction extends ActionType { public static final SearchTemplateAction INSTANCE = new SearchTemplateAction(); public static final String NAME = "indices:data/read/search/template"; @@ -31,11 +31,6 @@ private SearchTemplateAction() { super(NAME); } - @Override - public SearchTemplateResponse newResponse() { - throw new UnsupportedOperationException("usage of Streamable is to be replaced by Writeable"); - } - @Override public Writeable.Reader getResponseReader() { return SearchTemplateResponse::new; diff --git a/modules/lang-mustache/src/test/java/org/elasticsearch/script/mustache/MultiSearchTemplateRequestTests.java b/modules/lang-mustache/src/test/java/org/elasticsearch/script/mustache/MultiSearchTemplateRequestTests.java index 39400197a3871..3b5c7562472e4 100644 --- a/modules/lang-mustache/src/test/java/org/elasticsearch/script/mustache/MultiSearchTemplateRequestTests.java +++ b/modules/lang-mustache/src/test/java/org/elasticsearch/script/mustache/MultiSearchTemplateRequestTests.java @@ -56,13 +56,10 @@ public void testParseRequest() throws Exception { assertThat(request.requests().get(0).getRequest().preference(), nullValue()); assertThat(request.requests().get(1).getRequest().indices()[0], equalTo("test2")); assertThat(request.requests().get(1).getRequest().indices()[1], equalTo("test3")); - assertThat(request.requests().get(1).getRequest().types()[0], equalTo("type1")); assertThat(request.requests().get(1).getRequest().requestCache(), nullValue()); assertThat(request.requests().get(1).getRequest().preference(), equalTo("_local")); assertThat(request.requests().get(2).getRequest().indices()[0], equalTo("test4")); assertThat(request.requests().get(2).getRequest().indices()[1], equalTo("test1")); - assertThat(request.requests().get(2).getRequest().types()[0], equalTo("type2")); - assertThat(request.requests().get(2).getRequest().types()[1], equalTo("type1")); assertThat(request.requests().get(2).getRequest().routing(), equalTo("123")); assertNotNull(request.requests().get(0).getScript()); assertNotNull(request.requests().get(1).getScript()); diff --git a/modules/lang-mustache/src/test/java/org/elasticsearch/script/mustache/SearchTemplateIT.java b/modules/lang-mustache/src/test/java/org/elasticsearch/script/mustache/SearchTemplateIT.java index 0fbc3fa16afd2..3ff2bb649a79b 100644 --- a/modules/lang-mustache/src/test/java/org/elasticsearch/script/mustache/SearchTemplateIT.java +++ b/modules/lang-mustache/src/test/java/org/elasticsearch/script/mustache/SearchTemplateIT.java @@ -185,7 +185,7 @@ public void testIndexedTemplateClient() throws Exception { templateParams.put("fieldParam", "foo"); SearchTemplateResponse searchResponse = new SearchTemplateRequestBuilder(client()) - .setRequest(new SearchRequest("test").types("type")) + .setRequest(new SearchRequest("test")) .setScript("testTemplate").setScriptType(ScriptType.STORED).setScriptParams(templateParams) .get(); assertHitCount(searchResponse.getResponse(), 4); @@ -235,7 +235,7 @@ public void testIndexedTemplate() throws Exception { templateParams.put("fieldParam", "foo"); SearchTemplateResponse searchResponse = new SearchTemplateRequestBuilder(client()) - .setRequest(new SearchRequest().indices("test").types("type")) + .setRequest(new SearchRequest().indices("test")) .setScript("1a") .setScriptType(ScriptType.STORED) .setScriptParams(templateParams) @@ -243,7 +243,7 @@ public void testIndexedTemplate() throws Exception { assertHitCount(searchResponse.getResponse(), 4); expectThrows(ResourceNotFoundException.class, () -> new SearchTemplateRequestBuilder(client()) - .setRequest(new SearchRequest().indices("test").types("type")) + .setRequest(new SearchRequest().indices("test")) .setScript("1000") .setScriptType(ScriptType.STORED) .setScriptParams(templateParams) @@ -251,7 +251,7 @@ public void testIndexedTemplate() throws Exception { templateParams.put("fieldParam", "bar"); searchResponse = new SearchTemplateRequestBuilder(client()) - .setRequest(new SearchRequest("test").types("type")) + .setRequest(new SearchRequest("test")) .setScript("2").setScriptType(ScriptType.STORED).setScriptParams(templateParams) .get(); assertHitCount(searchResponse.getResponse(), 1); @@ -297,7 +297,7 @@ public void testIndexedTemplateOverwrite() throws Exception { templateParams.put("P_Keyword1", "dev"); IllegalArgumentException e = expectThrows(IllegalArgumentException.class, () -> new SearchTemplateRequestBuilder(client()) - .setRequest(new SearchRequest("testindex").types("test")) + .setRequest(new SearchRequest("testindex")) .setScript("git01").setScriptType(ScriptType.STORED).setScriptParams(templateParams) .get()); assertThat(e.getMessage(), containsString("No negative slop allowed")); @@ -308,7 +308,7 @@ public void testIndexedTemplateOverwrite() throws Exception { ); SearchTemplateResponse searchResponse = new SearchTemplateRequestBuilder(client()) - .setRequest(new SearchRequest("testindex").types("test")) + .setRequest(new SearchRequest("testindex")) .setScript("git01").setScriptType(ScriptType.STORED).setScriptParams(templateParams) .get(); assertHitCount(searchResponse.getResponse(), 1); @@ -352,7 +352,7 @@ public void testIndexedTemplateWithArray() throws Exception { arrayTemplateParams.put("fieldParam", fieldParams); SearchTemplateResponse searchResponse = new SearchTemplateRequestBuilder(client()) - .setRequest(new SearchRequest("test").types("type")) + .setRequest(new SearchRequest("test")) .setScript("4").setScriptType(ScriptType.STORED).setScriptParams(arrayTemplateParams) .get(); assertHitCount(searchResponse.getResponse(), 5); diff --git a/modules/lang-mustache/src/test/java/org/elasticsearch/script/mustache/SearchTemplateRequestTests.java b/modules/lang-mustache/src/test/java/org/elasticsearch/script/mustache/SearchTemplateRequestTests.java index 7d4a6479727e2..b91054ee131d2 100644 --- a/modules/lang-mustache/src/test/java/org/elasticsearch/script/mustache/SearchTemplateRequestTests.java +++ b/modules/lang-mustache/src/test/java/org/elasticsearch/script/mustache/SearchTemplateRequestTests.java @@ -63,8 +63,8 @@ protected SearchTemplateRequest mutateInstance(SearchTemplateRequest instance) t mutators.add(request -> request.setExplain(!request.isExplain())); mutators.add(request -> request.setSimulate(!request.isSimulate())); - mutators.add(request -> request.setRequest( - RandomSearchRequestGenerator.randomSearchRequest(SearchSourceBuilder::searchSource))); + mutators.add(request -> request.setRequest(randomValueOtherThan(request.getRequest(), + () -> RandomSearchRequestGenerator.randomSearchRequest(SearchSourceBuilder::searchSource)))); SearchTemplateRequest mutatedInstance = copyInstance(instance); Consumer mutator = randomFrom(mutators); diff --git a/modules/lang-mustache/src/test/resources/org/elasticsearch/script/mustache/simple-msearch-template.json b/modules/lang-mustache/src/test/resources/org/elasticsearch/script/mustache/simple-msearch-template.json index 11a0091492c4d..1809b4012fde1 100644 --- a/modules/lang-mustache/src/test/resources/org/elasticsearch/script/mustache/simple-msearch-template.json +++ b/modules/lang-mustache/src/test/resources/org/elasticsearch/script/mustache/simple-msearch-template.json @@ -1,6 +1,6 @@ {"index":["test0", "test1"], "request_cache": true} {"source": {"query" : {"match_{{template}}" :{}}}, "params": {"template": "all" } } -{"index" : "test2,test3", "type" : "type1", "preference": "_local"} +{"index" : "test2,test3", "preference": "_local"} {"source": {"query" : {"match_{{template}}" :{}}}, "params": {"template": "all" } } -{"index" : ["test4", "test1"], "type" : [ "type2", "type1" ], "routing": "123"} +{"index" : ["test4", "test1"], "routing": "123"} {"source": {"query" : {"match_{{template}}" :{}}}, "params": {"template": "all" } } diff --git a/modules/lang-mustache/src/test/resources/rest-api-spec/test/lang_mustache/30_search_template.yml b/modules/lang-mustache/src/test/resources/rest-api-spec/test/lang_mustache/30_search_template.yml index c0cc784250b90..f4d4f3a97e5e0 100644 --- a/modules/lang-mustache/src/test/resources/rest-api-spec/test/lang_mustache/30_search_template.yml +++ b/modules/lang-mustache/src/test/resources/rest-api-spec/test/lang_mustache/30_search_template.yml @@ -133,6 +133,7 @@ index: test id: 1 body: {} + refresh: true - do: put_script: @@ -145,5 +146,5 @@ search_template: body: { "id": "template_1", "params": {} } - - match: { hits.total.value: 0 } + - match: { hits.total.value: 1 } - match: { hits.total.relation: eq } diff --git a/modules/lang-painless/build.gradle b/modules/lang-painless/build.gradle index 1f6b722ec308c..6cce3850232e0 100644 --- a/modules/lang-painless/build.gradle +++ b/modules/lang-painless/build.gradle @@ -69,7 +69,7 @@ sourceSets { } dependencies { - docCompile "org.elasticsearch:elasticsearch:${project.versions.elasticsearch}" + docCompile project(':server') docCompile project(':modules:lang-painless') } diff --git a/modules/lang-painless/spi/build.gradle b/modules/lang-painless/spi/build.gradle index 7e43a242a23a9..3f25f247a2b67 100644 --- a/modules/lang-painless/spi/build.gradle +++ b/modules/lang-painless/spi/build.gradle @@ -33,7 +33,7 @@ publishing { } dependencies { - compile "org.elasticsearch:elasticsearch:${version}" + compile project(":server") } // no tests...yet? diff --git a/modules/lang-painless/spi/src/main/java/org/elasticsearch/painless/spi/Whitelist.java b/modules/lang-painless/spi/src/main/java/org/elasticsearch/painless/spi/Whitelist.java index b08b38d2bfc7b..d79c605986880 100644 --- a/modules/lang-painless/spi/src/main/java/org/elasticsearch/painless/spi/Whitelist.java +++ b/modules/lang-painless/spi/src/main/java/org/elasticsearch/painless/spi/Whitelist.java @@ -19,6 +19,8 @@ package org.elasticsearch.painless.spi; +import org.elasticsearch.painless.spi.annotation.WhitelistAnnotationParser; + import java.util.Collections; import java.util.List; import java.util.Objects; @@ -52,7 +54,8 @@ public final class Whitelist { }; public static final List BASE_WHITELISTS = - Collections.singletonList(WhitelistLoader.loadFromResourceFiles(Whitelist.class, BASE_WHITELIST_FILES)); + Collections.singletonList(WhitelistLoader.loadFromResourceFiles( + Whitelist.class, WhitelistAnnotationParser.BASE_ANNOTATION_PARSERS, BASE_WHITELIST_FILES)); /** The {@link ClassLoader} used to look up the whitelisted Java classes, constructors, methods, and fields. */ public final ClassLoader classLoader; diff --git a/modules/lang-painless/spi/src/main/java/org/elasticsearch/painless/spi/WhitelistClass.java b/modules/lang-painless/spi/src/main/java/org/elasticsearch/painless/spi/WhitelistClass.java index 7b3eb75aa3ecd..3f2bf3e997a27 100644 --- a/modules/lang-painless/spi/src/main/java/org/elasticsearch/painless/spi/WhitelistClass.java +++ b/modules/lang-painless/spi/src/main/java/org/elasticsearch/painless/spi/WhitelistClass.java @@ -19,9 +19,12 @@ package org.elasticsearch.painless.spi; +import java.util.AbstractMap; import java.util.Collections; import java.util.List; +import java.util.Map; import java.util.Objects; +import java.util.stream.Collectors; /** * Class represents the equivalent of a Java class in Painless complete with super classes, @@ -46,11 +49,6 @@ public final class WhitelistClass { /** The Java class name this class represents. */ public final String javaClassName; - /** - * Allow the Java class name to only be specified as the fully-qualified name. - */ - public final boolean noImport; - /** The {@link List} of whitelisted ({@link WhitelistConstructor}s) available to this class. */ public final List whitelistConstructors; @@ -60,17 +58,27 @@ public final class WhitelistClass { /** The {@link List} of whitelisted ({@link WhitelistField}s) available to this class. */ public final List whitelistFields; + /** The {@link Map} of annotations for this class. */ + public final Map, Object> painlessAnnotations; + /** Standard constructor. All values must be not {@code null}. */ - public WhitelistClass(String origin, String javaClassName, boolean noImport, - List whitelistConstructors, List whitelistMethods, List whitelistFields) - { + public WhitelistClass(String origin, String javaClassName, + List whitelistConstructors, List whitelistMethods, List whitelistFields, + List painlessAnnotations) { this.origin = Objects.requireNonNull(origin); this.javaClassName = Objects.requireNonNull(javaClassName); - this.noImport = noImport; this.whitelistConstructors = Collections.unmodifiableList(Objects.requireNonNull(whitelistConstructors)); this.whitelistMethods = Collections.unmodifiableList(Objects.requireNonNull(whitelistMethods)); this.whitelistFields = Collections.unmodifiableList(Objects.requireNonNull(whitelistFields)); + + if (painlessAnnotations.isEmpty()) { + this.painlessAnnotations = Collections.emptyMap(); + } else { + this.painlessAnnotations = Collections.unmodifiableMap(Objects.requireNonNull(painlessAnnotations).stream() + .map(painlessAnnotation -> new AbstractMap.SimpleEntry<>(painlessAnnotation.getClass(), painlessAnnotation)) + .collect(Collectors.toMap(Map.Entry::getKey, Map.Entry::getValue))); + } } } diff --git a/modules/lang-painless/spi/src/main/java/org/elasticsearch/painless/spi/WhitelistClassBinding.java b/modules/lang-painless/spi/src/main/java/org/elasticsearch/painless/spi/WhitelistClassBinding.java index da19917464820..f3ba052156dfa 100644 --- a/modules/lang-painless/spi/src/main/java/org/elasticsearch/painless/spi/WhitelistClassBinding.java +++ b/modules/lang-painless/spi/src/main/java/org/elasticsearch/painless/spi/WhitelistClassBinding.java @@ -19,8 +19,12 @@ package org.elasticsearch.painless.spi; +import java.util.AbstractMap; +import java.util.Collections; import java.util.List; +import java.util.Map; import java.util.Objects; +import java.util.stream.Collectors; /** * A class binding represents a method call that stores state. Each class binding's Java class must @@ -51,9 +55,13 @@ public class WhitelistClassBinding { */ public final List canonicalTypeNameParameters; + /** The {@link Map} of annotations for this class binding. */ + public final Map, Object> painlessAnnotations; + /** Standard constructor. All values must be not {@code null}. */ public WhitelistClassBinding(String origin, String targetJavaClassName, - String methodName, String returnCanonicalTypeName, List canonicalTypeNameParameters) { + String methodName, String returnCanonicalTypeName, List canonicalTypeNameParameters, + List painlessAnnotations) { this.origin = Objects.requireNonNull(origin); this.targetJavaClassName = Objects.requireNonNull(targetJavaClassName); @@ -61,5 +69,13 @@ public WhitelistClassBinding(String origin, String targetJavaClassName, this.methodName = Objects.requireNonNull(methodName); this.returnCanonicalTypeName = Objects.requireNonNull(returnCanonicalTypeName); this.canonicalTypeNameParameters = Objects.requireNonNull(canonicalTypeNameParameters); + + if (painlessAnnotations.isEmpty()) { + this.painlessAnnotations = Collections.emptyMap(); + } else { + this.painlessAnnotations = Collections.unmodifiableMap(Objects.requireNonNull(painlessAnnotations).stream() + .map(painlessAnnotation -> new AbstractMap.SimpleEntry<>(painlessAnnotation.getClass(), painlessAnnotation)) + .collect(Collectors.toMap(Map.Entry::getKey, Map.Entry::getValue))); + } } } diff --git a/modules/lang-painless/spi/src/main/java/org/elasticsearch/painless/spi/WhitelistConstructor.java b/modules/lang-painless/spi/src/main/java/org/elasticsearch/painless/spi/WhitelistConstructor.java index 032ef397def01..13aec63fceb77 100644 --- a/modules/lang-painless/spi/src/main/java/org/elasticsearch/painless/spi/WhitelistConstructor.java +++ b/modules/lang-painless/spi/src/main/java/org/elasticsearch/painless/spi/WhitelistConstructor.java @@ -19,9 +19,12 @@ package org.elasticsearch.painless.spi; +import java.util.AbstractMap; import java.util.Collections; import java.util.List; +import java.util.Map; import java.util.Objects; +import java.util.stream.Collectors; /** * Constructor represents the equivalent of a Java constructor available as a whitelisted class @@ -40,9 +43,20 @@ public final class WhitelistConstructor { */ public final List canonicalTypeNameParameters; + /** The {@link Map} of annotations for this constructor. */ + public final Map, Object> painlessAnnotations; + /** Standard constructor. All values must be not {@code null}. */ - public WhitelistConstructor(String origin, List canonicalTypeNameParameters) { + public WhitelistConstructor(String origin, List canonicalTypeNameParameters, List painlessAnnotations) { this.origin = Objects.requireNonNull(origin); this.canonicalTypeNameParameters = Collections.unmodifiableList(Objects.requireNonNull(canonicalTypeNameParameters)); + + if (painlessAnnotations.isEmpty()) { + this.painlessAnnotations = Collections.emptyMap(); + } else { + this.painlessAnnotations = Collections.unmodifiableMap(Objects.requireNonNull(painlessAnnotations).stream() + .map(painlessAnnotation -> new AbstractMap.SimpleEntry<>(painlessAnnotation.getClass(), painlessAnnotation)) + .collect(Collectors.toMap(Map.Entry::getKey, Map.Entry::getValue))); + } } } diff --git a/modules/lang-painless/spi/src/main/java/org/elasticsearch/painless/spi/WhitelistField.java b/modules/lang-painless/spi/src/main/java/org/elasticsearch/painless/spi/WhitelistField.java index 44ed31a227e1c..84f12e907584c 100644 --- a/modules/lang-painless/spi/src/main/java/org/elasticsearch/painless/spi/WhitelistField.java +++ b/modules/lang-painless/spi/src/main/java/org/elasticsearch/painless/spi/WhitelistField.java @@ -19,7 +19,12 @@ package org.elasticsearch.painless.spi; +import java.util.AbstractMap; +import java.util.Collections; +import java.util.List; +import java.util.Map; import java.util.Objects; +import java.util.stream.Collectors; /** * Field represents the equivalent of a Java field available as a whitelisted class field @@ -37,10 +42,21 @@ public class WhitelistField { /** The canonical type name for the field which can be used to look up the Java field through reflection. */ public final String canonicalTypeNameParameter; + /** The {@link Map} of annotations for this field. */ + public final Map, Object> painlessAnnotations; + /** Standard constructor. All values must be not {@code null}. */ - public WhitelistField(String origin, String fieldName, String canonicalTypeNameParameter) { + public WhitelistField(String origin, String fieldName, String canonicalTypeNameParameter, List painlessAnnotations) { this.origin = Objects.requireNonNull(origin); this.fieldName = Objects.requireNonNull(fieldName); this.canonicalTypeNameParameter = Objects.requireNonNull(canonicalTypeNameParameter); + + if (painlessAnnotations.isEmpty()) { + this.painlessAnnotations = Collections.emptyMap(); + } else { + this.painlessAnnotations = Collections.unmodifiableMap(Objects.requireNonNull(painlessAnnotations).stream() + .map(painlessAnnotation -> new AbstractMap.SimpleEntry<>(painlessAnnotation.getClass(), painlessAnnotation)) + .collect(Collectors.toMap(Map.Entry::getKey, Map.Entry::getValue))); + } } } diff --git a/modules/lang-painless/spi/src/main/java/org/elasticsearch/painless/spi/WhitelistInstanceBinding.java b/modules/lang-painless/spi/src/main/java/org/elasticsearch/painless/spi/WhitelistInstanceBinding.java index 46c2f0f91fe02..2a41f873bf8e5 100644 --- a/modules/lang-painless/spi/src/main/java/org/elasticsearch/painless/spi/WhitelistInstanceBinding.java +++ b/modules/lang-painless/spi/src/main/java/org/elasticsearch/painless/spi/WhitelistInstanceBinding.java @@ -19,8 +19,12 @@ package org.elasticsearch.painless.spi; +import java.util.AbstractMap; +import java.util.Collections; import java.util.List; +import java.util.Map; import java.util.Objects; +import java.util.stream.Collectors; /** * An instance binding represents a method call that stores state. Each instance binding must provide @@ -47,9 +51,13 @@ public class WhitelistInstanceBinding { */ public final List canonicalTypeNameParameters; + /** The {@link Map} of annotations for this instance binding. */ + public final Map, Object> painlessAnnotations; + /** Standard constructor. All values must be not {@code null}. */ public WhitelistInstanceBinding(String origin, Object targetInstance, - String methodName, String returnCanonicalTypeName, List canonicalTypeNameParameters) { + String methodName, String returnCanonicalTypeName, List canonicalTypeNameParameters, + List painlessAnnotations) { this.origin = Objects.requireNonNull(origin); this.targetInstance = Objects.requireNonNull(targetInstance); @@ -57,5 +65,13 @@ public WhitelistInstanceBinding(String origin, Object targetInstance, this.methodName = Objects.requireNonNull(methodName); this.returnCanonicalTypeName = Objects.requireNonNull(returnCanonicalTypeName); this.canonicalTypeNameParameters = Objects.requireNonNull(canonicalTypeNameParameters); + + if (painlessAnnotations.isEmpty()) { + this.painlessAnnotations = Collections.emptyMap(); + } else { + this.painlessAnnotations = Collections.unmodifiableMap(Objects.requireNonNull(painlessAnnotations).stream() + .map(painlessAnnotation -> new AbstractMap.SimpleEntry<>(painlessAnnotation.getClass(), painlessAnnotation)) + .collect(Collectors.toMap(Map.Entry::getKey, Map.Entry::getValue))); + } } } diff --git a/modules/lang-painless/spi/src/main/java/org/elasticsearch/painless/spi/WhitelistLoader.java b/modules/lang-painless/spi/src/main/java/org/elasticsearch/painless/spi/WhitelistLoader.java index d896c345a47e9..a9f6efea0e0b6 100644 --- a/modules/lang-painless/spi/src/main/java/org/elasticsearch/painless/spi/WhitelistLoader.java +++ b/modules/lang-painless/spi/src/main/java/org/elasticsearch/painless/spi/WhitelistLoader.java @@ -19,6 +19,8 @@ package org.elasticsearch.painless.spi; +import org.elasticsearch.painless.spi.annotation.WhitelistAnnotationParser; + import java.io.InputStreamReader; import java.io.LineNumberReader; import java.lang.reflect.Constructor; @@ -30,11 +32,22 @@ import java.util.ArrayList; import java.util.Arrays; import java.util.Collections; +import java.util.HashMap; import java.util.List; +import java.util.Map; /** Loads and creates a {@link Whitelist} from one to many text files. */ public final class WhitelistLoader { + /** + * Loads and creates a {@link Whitelist} from one to many text files using only the base annotation parsers. + * See {@link #loadFromResourceFiles(Class, Map, String...)} for information on how to structure a whitelist + * text file. + */ + public static Whitelist loadFromResourceFiles(Class resource, String... filepaths) { + return loadFromResourceFiles(resource, WhitelistAnnotationParser.BASE_ANNOTATION_PARSERS, filepaths); + } + /** * Loads and creates a {@link Whitelist} from one to many text files. The file paths are passed in as an array of * {@link String}s with a single {@link Class} to be be used to load the resources where each {@link String} @@ -54,7 +67,7 @@ public final class WhitelistLoader { * a Painless type name with the exception that any dollar symbols used as part of inner classes will * be replaced with dot symbols. *
  • short Java type name - The text after the final dot symbol of any specified Java class. A - * short type Java name may be excluded by using the 'no_import' token during Painless class parsing + * short type Java name may be excluded by using the 'no_import' attribute during Painless class parsing * as described later.
  • * * @@ -65,8 +78,8 @@ public final class WhitelistLoader { * be ignored by the parser. *
  • Primitive types may be specified starting with 'class' and followed by the Java type name, * an opening bracket, a newline, a closing bracket, and a final newline.
  • - *
  • Complex types may be specified starting with 'class' and followed the fully-qualified Java - * class name, optionally followed by an 'no_import' token, an opening bracket, a newline, + *
  • Complex types may be specified starting with 'class' and followed by the fully-qualified Java + * class name, optionally followed by a 'no_import' attribute, an opening bracket, a newline, * constructor/method/field specifications, a closing bracket, and a final newline. Within a complex * type the following may be parsed: *
      @@ -90,6 +103,10 @@ public final class WhitelistLoader { * of the field, followed by the Java name of the field (which all be the Painless name * for the field), and a newline. *
    + *
  • Annotations may be added starting with an at, followed by a name, optionally an opening brace, + * a parameter name, an equals, an opening quote, an argument value, a closing quote, (possibly repeated + * for multiple arguments,) and a closing brace. Multiple annotations may be added after a class (before + * the opening bracket), after a method, or after field.
  • * * * Note there must be a one-to-one correspondence of Painless type names to Java type/class names. @@ -110,7 +127,7 @@ public final class WhitelistLoader { * * # complex types * - * class my.package.Example no_import { + * class my.package.Example @no_import { * # constructors * () * (int) @@ -120,7 +137,8 @@ public final class WhitelistLoader { * # method * Example add(int, def) * int add(Example, Example) - * void example() + * void example() @deprecated[use example 2 instead] + * void example2() * * # augmented * Example some.other.Class sub(Example, int, def) @@ -132,7 +150,7 @@ public final class WhitelistLoader { * } * } */ - public static Whitelist loadFromResourceFiles(Class resource, String... filepaths) { + public static Whitelist loadFromResourceFiles(Class resource, Map parsers, String... filepaths) { List whitelistClasses = new ArrayList<>(); List whitelistStatics = new ArrayList<>(); List whitelistClassBindings = new ArrayList<>(); @@ -149,10 +167,10 @@ public static Whitelist loadFromResourceFiles(Class resource, String... filep String parseType = null; String whitelistClassOrigin = null; String javaClassName = null; - boolean noImport = false; List whitelistConstructors = null; List whitelistMethods = null; List whitelistFields = null; + List classAnnotations = null; while ((line = reader.readLine()) != null) { number = reader.getLineNumber(); @@ -164,7 +182,7 @@ public static Whitelist loadFromResourceFiles(Class resource, String... filep } // Handle a new class by resetting all the variables necessary to construct a new WhitelistClass for the whitelist. - // Expects the following format: 'class' ID 'no_import'? '{' '\n' + // Expects the following format: 'class' ID annotations? '{' '\n' if (line.startsWith("class ")) { // Ensure the final token of the line is '{'. if (line.endsWith("{") == false) { @@ -176,19 +194,19 @@ public static Whitelist loadFromResourceFiles(Class resource, String... filep throw new IllegalArgumentException("invalid definition: cannot embed class definition [" + line + "]"); } - // Parse the Java class name. - String[] tokens = line.substring(5, line.length() - 1).trim().split("\\s+"); + // Parse the Java class name and annotations if they exist. + int annotationIndex = line.indexOf('@'); - // Ensure the correct number of tokens. - if (tokens.length == 2 && "no_import".equals(tokens[1])) { - noImport = true; - } else if (tokens.length != 1) { - throw new IllegalArgumentException("invalid class definition: failed to parse class name [" + line + "]"); + if (annotationIndex == -1) { + annotationIndex = line.length() - 1; + classAnnotations = Collections.emptyList(); + } else { + classAnnotations = parseWhitelistAnnotations(parsers, line.substring(annotationIndex, line.length() - 1)); } parseType = "class"; whitelistClassOrigin = "[" + filepath + "]:[" + number + "]"; - javaClassName = tokens[0]; + javaClassName = line.substring(5, annotationIndex).trim(); // Reset all the constructors, methods, and fields to support a new class. whitelistConstructors = new ArrayList<>(); @@ -217,22 +235,22 @@ public static Whitelist loadFromResourceFiles(Class resource, String... filep // Create a new WhitelistClass with all the previously gathered constructors, methods, // augmented methods, and fields, and add it to the list of whitelisted classes. if ("class".equals(parseType)) { - whitelistClasses.add(new WhitelistClass(whitelistClassOrigin, javaClassName, noImport, - whitelistConstructors, whitelistMethods, whitelistFields)); + whitelistClasses.add(new WhitelistClass(whitelistClassOrigin, javaClassName, + whitelistConstructors, whitelistMethods, whitelistFields, classAnnotations)); whitelistClassOrigin = null; javaClassName = null; - noImport = false; whitelistConstructors = null; whitelistMethods = null; whitelistFields = null; + classAnnotations = null; } // Reset the parseType. parseType = null; // Handle static import definition types. - // Expects the following format: ID ID '(' ( ID ( ',' ID )* )? ')' ( 'from_class' | 'bound_to' ) ID '\n' + // Expects the following format: ID ID '(' ( ID ( ',' ID )* )? ')' ( 'from_class' | 'bound_to' ) ID annotations? '\n' } else if ("static_import".equals(parseType)) { // Mark the origin of this parsable object. String origin = "[" + filepath + "]:[" + number + "]"; @@ -274,8 +292,19 @@ public static Whitelist loadFromResourceFiles(Class resource, String... filep canonicalTypeNameParameters = new String[0]; } + // Parse the annotations if they exist. + List annotations; + int annotationIndex = line.indexOf('@'); + + if (annotationIndex == -1) { + annotationIndex = line.length(); + annotations = Collections.emptyList(); + } else { + annotations = parseWhitelistAnnotations(parsers, line.substring(annotationIndex)); + } + // Parse the static import type and class. - tokens = line.substring(parameterEndIndex + 1).trim().split("\\s+"); + tokens = line.substring(parameterEndIndex + 1, annotationIndex).trim().split("\\s+"); String staticImportType; String targetJavaClassName; @@ -291,10 +320,12 @@ public static Whitelist loadFromResourceFiles(Class resource, String... filep // Add a static import method or binding depending on the static import type. if ("from_class".equals(staticImportType)) { whitelistStatics.add(new WhitelistMethod(origin, targetJavaClassName, - methodName, returnCanonicalTypeName, Arrays.asList(canonicalTypeNameParameters))); + methodName, returnCanonicalTypeName, Arrays.asList(canonicalTypeNameParameters), + annotations)); } else if ("bound_to".equals(staticImportType)) { whitelistClassBindings.add(new WhitelistClassBinding(origin, targetJavaClassName, - methodName, returnCanonicalTypeName, Arrays.asList(canonicalTypeNameParameters))); + methodName, returnCanonicalTypeName, Arrays.asList(canonicalTypeNameParameters), + annotations)); } else { throw new IllegalArgumentException("invalid static import definition: " + "unexpected static import type [" + staticImportType + "] [" + line + "]"); @@ -306,36 +337,38 @@ public static Whitelist loadFromResourceFiles(Class resource, String... filep String origin = "[" + filepath + "]:[" + number + "]"; // Handle the case for a constructor definition. - // Expects the following format: '(' ( ID ( ',' ID )* )? ')' '\n' + // Expects the following format: '(' ( ID ( ',' ID )* )? ')' annotations? '\n' if (line.startsWith("(")) { - // Ensure the final token of the line is ')'. - if (line.endsWith(")") == false) { + // Parse the constructor parameters. + int parameterEndIndex = line.indexOf(')'); + + if (parameterEndIndex == -1) { throw new IllegalArgumentException( - "invalid constructor definition: expected a closing parenthesis [" + line + "]"); + "illegal constructor definition: end of constructor parameters not found [" + line + "]"); } - // Parse the constructor parameters. - String[] tokens = line.substring(1, line.length() - 1).replaceAll("\\s+", "").split(","); + String[] canonicalTypeNameParameters = line.substring(1, parameterEndIndex).replaceAll("\\s+", "").split(","); // Handle the case for a constructor with no parameters. - if ("".equals(tokens[0])) { - tokens = new String[0]; + if ("".equals(canonicalTypeNameParameters[0])) { + canonicalTypeNameParameters = new String[0]; } - whitelistConstructors.add(new WhitelistConstructor(origin, Arrays.asList(tokens))); + // Parse the annotations if they exist. + List annotations; + int annotationIndex = line.indexOf('@'); + annotations = annotationIndex == -1 ? + Collections.emptyList() : parseWhitelistAnnotations(parsers, line.substring(annotationIndex)); - // Handle the case for a method or augmented method definition. - // Expects the following format: ID ID? ID '(' ( ID ( ',' ID )* )? ')' '\n' - } else if (line.contains("(")) { - // Ensure the final token of the line is ')'. - if (line.endsWith(")") == false) { - throw new IllegalArgumentException( - "invalid method definition: expected a closing parenthesis [" + line + "]"); - } + whitelistConstructors.add(new WhitelistConstructor( + origin, Arrays.asList(canonicalTypeNameParameters), annotations)); + // Handle the case for a method or augmented method definition. + // Expects the following format: ID ID? ID '(' ( ID ( ',' ID )* )? ')' annotations? '\n' + } else if (line.contains("(")) { // Parse the tokens prior to the method parameters. - int parameterIndex = line.indexOf('('); - String[] tokens = line.substring(0, parameterIndex).trim().split("\\s+"); + int parameterStartIndex = line.indexOf('('); + String[] tokens = line.substring(0, parameterStartIndex).trim().split("\\s+"); String methodName; String javaAugmentedClassName; @@ -354,28 +387,54 @@ public static Whitelist loadFromResourceFiles(Class resource, String... filep String returnCanonicalTypeName = tokens[0]; // Parse the method parameters. - tokens = line.substring(parameterIndex + 1, line.length() - 1).replaceAll("\\s+", "").split(","); + int parameterEndIndex = line.indexOf(')'); + + if (parameterEndIndex == -1) { + throw new IllegalArgumentException( + "illegal static import definition: end of method parameters not found [" + line + "]"); + } + + String[] canonicalTypeNameParameters = + line.substring(parameterStartIndex + 1, parameterEndIndex).replaceAll("\\s+", "").split(","); // Handle the case for a method with no parameters. - if ("".equals(tokens[0])) { - tokens = new String[0]; + if ("".equals(canonicalTypeNameParameters[0])) { + canonicalTypeNameParameters = new String[0]; } + // Parse the annotations if they exist. + List annotations; + int annotationIndex = line.indexOf('@'); + annotations = annotationIndex == -1 ? + Collections.emptyList() : parseWhitelistAnnotations(parsers, line.substring(annotationIndex)); + whitelistMethods.add(new WhitelistMethod(origin, javaAugmentedClassName, methodName, - returnCanonicalTypeName, Arrays.asList(tokens))); + returnCanonicalTypeName, Arrays.asList(canonicalTypeNameParameters), + annotations)); - // Handle the case for a field definition. - // Expects the following format: ID ID '\n' + // Handle the case for a field definition. + // Expects the following format: ID ID annotations? '\n' } else { + // Parse the annotations if they exist. + List annotations; + int annotationIndex = line.indexOf('@'); + + if (annotationIndex == -1) { + annotationIndex = line.length(); + annotations = Collections.emptyList(); + } else { + annotations = parseWhitelistAnnotations(parsers, line.substring(annotationIndex)); + } + // Parse the field tokens. - String[] tokens = line.split("\\s+"); + String[] tokens = line.substring(0, annotationIndex).split("\\s+"); // Ensure the correct number of tokens. if (tokens.length != 2) { throw new IllegalArgumentException("invalid field definition: unexpected format [" + line + "]"); } - whitelistFields.add(new WhitelistField(origin, tokens[1], tokens[0])); + whitelistFields.add(new WhitelistField(origin, tokens[1], tokens[0], annotations)); } } else { throw new IllegalArgumentException("invalid definition: unable to parse line [" + line + "]"); @@ -396,5 +455,85 @@ public static Whitelist loadFromResourceFiles(Class resource, String... filep return new Whitelist(loader, whitelistClasses, whitelistStatics, whitelistClassBindings, Collections.emptyList()); } + private static List parseWhitelistAnnotations( + Map parsers, String line) { + + List annotations; + + if (line.isBlank()) { + annotations = Collections.emptyList(); + } else { + line = line.trim(); + + if (line.charAt(0) != '@') { + throw new IllegalArgumentException("invalid annotation: expected at symbol [" + line + "]"); + } + + if (line.length() < 2) { + throw new IllegalArgumentException("invalid annotation: expected name [" + line + "]"); + } + + String[] annotationStrings = line.substring(1).split("@"); + annotations = new ArrayList<>(annotationStrings.length); + + for (String annotationString : annotationStrings) { + String name; + Map arguments; + + annotationString = annotationString.trim(); + int index = annotationString.indexOf('['); + + if (index == -1) { + name = annotationString; + arguments = Collections.emptyMap(); + } else { + if (annotationString.charAt(annotationString.length() - 1) != ']') { + throw new IllegalArgumentException("invalid annotation: expected closing brace [" + line + "]"); + } + + name = annotationString.substring(0, index); + arguments = new HashMap<>(); + + String[] argumentsStrings = annotationString.substring(index + 1, annotationString.length() - 1).split(","); + + for (String argumentString : argumentsStrings) { + String[] argumentKeyValue = argumentString.split("="); + + if (argumentKeyValue.length != 2) { + throw new IllegalArgumentException("invalid annotation: expected key=\"value\" [" + line + "]"); + } + + String argumentKey = argumentKeyValue[0].trim(); + + if (argumentKey.isEmpty()) { + throw new IllegalArgumentException("invalid annotation: expected key=\"value\" [" + line + "]"); + } + + String argumentValue = argumentKeyValue[1]; + + if (argumentValue.length() < 3 || argumentValue.charAt(0) != '"' || + argumentValue.charAt(argumentValue.length() - 1) != '"') { + throw new IllegalArgumentException("invalid annotation: expected key=\"value\" [" + line + "]"); + } + + argumentValue = argumentValue.substring(1, argumentValue.length() - 1); + + arguments.put(argumentKey, argumentValue); + } + } + + WhitelistAnnotationParser parser = parsers.get(name); + + if (parser == null) { + throw new IllegalArgumentException("invalid annotation: parser not found for [" + name + "] [" + line + "]"); + } + + annotations.add(parser.parse(arguments)); + } + } + + return annotations; + } + private WhitelistLoader() {} } diff --git a/modules/lang-painless/spi/src/main/java/org/elasticsearch/painless/spi/WhitelistMethod.java b/modules/lang-painless/spi/src/main/java/org/elasticsearch/painless/spi/WhitelistMethod.java index f450ee0238d19..838fcba5034ac 100644 --- a/modules/lang-painless/spi/src/main/java/org/elasticsearch/painless/spi/WhitelistMethod.java +++ b/modules/lang-painless/spi/src/main/java/org/elasticsearch/painless/spi/WhitelistMethod.java @@ -19,9 +19,12 @@ package org.elasticsearch.painless.spi; +import java.util.AbstractMap; import java.util.Collections; import java.util.List; +import java.util.Map; import java.util.Objects; +import java.util.stream.Collectors; /** * Method represents the equivalent of a Java method available as a whitelisted class method @@ -61,18 +64,30 @@ public class WhitelistMethod { */ public final List canonicalTypeNameParameters; + /** The {@link Map} of annotations for this method. */ + public final Map, Object> painlessAnnotations; + /** * Standard constructor. All values must be not {@code null} with the exception of * augmentedCanonicalClassName; augmentedCanonicalClassName will be {@code null} unless the method * is augmented as described in the class documentation. */ public WhitelistMethod(String origin, String augmentedCanonicalClassName, String methodName, - String returnCanonicalTypeName, List canonicalTypeNameParameters) { + String returnCanonicalTypeName, List canonicalTypeNameParameters, + List painlessAnnotations) { this.origin = Objects.requireNonNull(origin); this.augmentedCanonicalClassName = augmentedCanonicalClassName; this.methodName = methodName; this.returnCanonicalTypeName = Objects.requireNonNull(returnCanonicalTypeName); this.canonicalTypeNameParameters = Collections.unmodifiableList(Objects.requireNonNull(canonicalTypeNameParameters)); + + if (painlessAnnotations.isEmpty()) { + this.painlessAnnotations = Collections.emptyMap(); + } else { + this.painlessAnnotations = Collections.unmodifiableMap(Objects.requireNonNull(painlessAnnotations).stream() + .map(painlessAnnotation -> new AbstractMap.SimpleEntry<>(painlessAnnotation.getClass(), painlessAnnotation)) + .collect(Collectors.toMap(Map.Entry::getKey, Map.Entry::getValue))); + } } } diff --git a/server/src/main/java/org/elasticsearch/action/admin/indices/exists/types/TypesExistsAction.java b/modules/lang-painless/spi/src/main/java/org/elasticsearch/painless/spi/annotation/DeprecatedAnnotation.java similarity index 63% rename from server/src/main/java/org/elasticsearch/action/admin/indices/exists/types/TypesExistsAction.java rename to modules/lang-painless/spi/src/main/java/org/elasticsearch/painless/spi/annotation/DeprecatedAnnotation.java index 0b508110d7f50..b7162e3d6bf99 100644 --- a/server/src/main/java/org/elasticsearch/action/admin/indices/exists/types/TypesExistsAction.java +++ b/modules/lang-painless/spi/src/main/java/org/elasticsearch/painless/spi/annotation/DeprecatedAnnotation.java @@ -16,21 +16,20 @@ * specific language governing permissions and limitations * under the License. */ -package org.elasticsearch.action.admin.indices.exists.types; -import org.elasticsearch.action.Action; +package org.elasticsearch.painless.spi.annotation; -public class TypesExistsAction extends Action { +public class DeprecatedAnnotation { - public static final TypesExistsAction INSTANCE = new TypesExistsAction(); - public static final String NAME = "indices:admin/types/exists"; + public static final String NAME = "deprecated"; - private TypesExistsAction() { - super(NAME); + private final String message; + + public DeprecatedAnnotation(String message) { + this.message = message; } - @Override - public TypesExistsResponse newResponse() { - return new TypesExistsResponse(); + public String getMessage() { + return message; } } diff --git a/modules/lang-painless/spi/src/main/java/org/elasticsearch/painless/spi/annotation/DeprecatedAnnotationParser.java b/modules/lang-painless/spi/src/main/java/org/elasticsearch/painless/spi/annotation/DeprecatedAnnotationParser.java new file mode 100644 index 0000000000000..c0992969c2278 --- /dev/null +++ b/modules/lang-painless/spi/src/main/java/org/elasticsearch/painless/spi/annotation/DeprecatedAnnotationParser.java @@ -0,0 +1,44 @@ +/* + * 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.painless.spi.annotation; + +import java.util.Map; + +public class DeprecatedAnnotationParser implements WhitelistAnnotationParser { + + public static final DeprecatedAnnotationParser INSTANCE = new DeprecatedAnnotationParser(); + + public static final String MESSAGE = "message"; + + private DeprecatedAnnotationParser() { + + } + + @Override + public Object parse(Map arguments) { + String message = arguments.getOrDefault(MESSAGE, ""); + + if ((arguments.isEmpty() || arguments.size() == 1 && arguments.containsKey(MESSAGE)) == false) { + throw new IllegalArgumentException("unexpected parameters for [@deprecation] annotation, found " + arguments); + } + + return new DeprecatedAnnotation(message); + } +} diff --git a/modules/mapper-extras/src/test/java/org/elasticsearch/index/mapper/SparseVectorFieldTypeTests.java b/modules/lang-painless/spi/src/main/java/org/elasticsearch/painless/spi/annotation/NoImportAnnotation.java similarity index 75% rename from modules/mapper-extras/src/test/java/org/elasticsearch/index/mapper/SparseVectorFieldTypeTests.java rename to modules/lang-painless/spi/src/main/java/org/elasticsearch/painless/spi/annotation/NoImportAnnotation.java index 9fe8d17e697e6..ceec4740b3989 100644 --- a/modules/mapper-extras/src/test/java/org/elasticsearch/index/mapper/SparseVectorFieldTypeTests.java +++ b/modules/lang-painless/spi/src/main/java/org/elasticsearch/painless/spi/annotation/NoImportAnnotation.java @@ -17,12 +17,15 @@ * under the License. */ -package org.elasticsearch.index.mapper; +package org.elasticsearch.painless.spi.annotation; -public class SparseVectorFieldTypeTests extends FieldTypeTestCase { +public class NoImportAnnotation { + + public static final String NAME = "no_import"; + + public static final NoImportAnnotation INSTANCE = new NoImportAnnotation(); + + private NoImportAnnotation() { - @Override - protected MappedFieldType createDefaultFieldType() { - return new SparseVectorFieldMapper.SparseVectorFieldType(); } } diff --git a/server/src/main/java/org/elasticsearch/action/admin/indices/exists/indices/IndicesExistsAction.java b/modules/lang-painless/spi/src/main/java/org/elasticsearch/painless/spi/annotation/NoImportAnnotationParser.java similarity index 59% rename from server/src/main/java/org/elasticsearch/action/admin/indices/exists/indices/IndicesExistsAction.java rename to modules/lang-painless/spi/src/main/java/org/elasticsearch/painless/spi/annotation/NoImportAnnotationParser.java index b878994549f5a..e8027b56e4109 100644 --- a/server/src/main/java/org/elasticsearch/action/admin/indices/exists/indices/IndicesExistsAction.java +++ b/modules/lang-painless/spi/src/main/java/org/elasticsearch/painless/spi/annotation/NoImportAnnotationParser.java @@ -17,21 +17,24 @@ * under the License. */ -package org.elasticsearch.action.admin.indices.exists.indices; +package org.elasticsearch.painless.spi.annotation; -import org.elasticsearch.action.Action; +import java.util.Map; -public class IndicesExistsAction extends Action { +public class NoImportAnnotationParser implements WhitelistAnnotationParser { - public static final IndicesExistsAction INSTANCE = new IndicesExistsAction(); - public static final String NAME = "indices:admin/exists"; + public static final NoImportAnnotationParser INSTANCE = new NoImportAnnotationParser(); + + private NoImportAnnotationParser() { - private IndicesExistsAction() { - super(NAME); } @Override - public IndicesExistsResponse newResponse() { - return new IndicesExistsResponse(); + public Object parse(Map arguments) { + if (arguments.isEmpty() == false) { + throw new IllegalArgumentException("unexpected parameters for [@no_import] annotation, found " + arguments); + } + + return NoImportAnnotation.INSTANCE; } } diff --git a/modules/lang-painless/spi/src/main/java/org/elasticsearch/painless/spi/annotation/WhitelistAnnotationParser.java b/modules/lang-painless/spi/src/main/java/org/elasticsearch/painless/spi/annotation/WhitelistAnnotationParser.java new file mode 100644 index 0000000000000..d5dcbab36f34f --- /dev/null +++ b/modules/lang-painless/spi/src/main/java/org/elasticsearch/painless/spi/annotation/WhitelistAnnotationParser.java @@ -0,0 +1,42 @@ +/* + * 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.painless.spi.annotation; + +import java.util.AbstractMap; +import java.util.Collections; +import java.util.Map; +import java.util.stream.Collectors; +import java.util.stream.Stream; + +/** + * WhitelistAnnotationParser is an interface used to define how to + * parse an annotation against any whitelist object while loading. + */ +public interface WhitelistAnnotationParser { + + Map BASE_ANNOTATION_PARSERS = Collections.unmodifiableMap( + Stream.of( + new AbstractMap.SimpleEntry<>(NoImportAnnotation.NAME, NoImportAnnotationParser.INSTANCE), + new AbstractMap.SimpleEntry<>(DeprecatedAnnotation.NAME, DeprecatedAnnotationParser.INSTANCE) + ).collect(Collectors.toMap(Map.Entry::getKey, Map.Entry::getValue)) + ); + + Object parse(Map arguments); +} diff --git a/modules/lang-painless/src/doc/java/org/elasticsearch/painless/ContextDocGenerator.java b/modules/lang-painless/src/doc/java/org/elasticsearch/painless/ContextDocGenerator.java index babc3e10e55db..f09dd4f521ade 100644 --- a/modules/lang-painless/src/doc/java/org/elasticsearch/painless/ContextDocGenerator.java +++ b/modules/lang-painless/src/doc/java/org/elasticsearch/painless/ContextDocGenerator.java @@ -798,6 +798,7 @@ private static Map getDisplayNames(List createGuiceModules() { - return Collections.singleton(b -> b.bind(PainlessScriptEngine.class).toInstance(painlessScriptEngine.get())); + public Collection createComponents(Client client, ClusterService clusterService, ThreadPool threadPool, + ResourceWatcherService resourceWatcherService, ScriptService scriptService, + NamedXContentRegistry xContentRegistry, Environment environment, + NodeEnvironment nodeEnvironment, NamedWriteableRegistry namedWriteableRegistry) { + // this is a hack to bind the painless script engine in guice (all components are added to guice), so that + // the painless context api. this is a temporary measure until transport actions do no require guice + return Collections.singletonList(painlessScriptEngine.get()); } @Override diff --git a/modules/lang-painless/src/main/java/org/elasticsearch/painless/action/PainlessContextAction.java b/modules/lang-painless/src/main/java/org/elasticsearch/painless/action/PainlessContextAction.java index c947de1fd82d3..5e941a2d0e2e0 100644 --- a/modules/lang-painless/src/main/java/org/elasticsearch/painless/action/PainlessContextAction.java +++ b/modules/lang-painless/src/main/java/org/elasticsearch/painless/action/PainlessContextAction.java @@ -19,7 +19,7 @@ package org.elasticsearch.painless.action; -import org.elasticsearch.action.Action; +import org.elasticsearch.action.ActionType; import org.elasticsearch.action.ActionListener; import org.elasticsearch.action.ActionRequest; import org.elasticsearch.action.ActionRequestValidationException; @@ -64,7 +64,7 @@ * retrieves all available information about the API for this specific context * */ -public class PainlessContextAction extends Action { +public class PainlessContextAction extends ActionType { public static final PainlessContextAction INSTANCE = new PainlessContextAction(); private static final String NAME = "cluster:admin/scripts/painless/context"; @@ -75,11 +75,6 @@ private PainlessContextAction() { super(NAME); } - @Override - public Response newResponse() { - throw new UnsupportedOperationException(); - } - @Override public Writeable.Reader getResponseReader() { return Response::new; diff --git a/modules/lang-painless/src/main/java/org/elasticsearch/painless/action/PainlessExecuteAction.java b/modules/lang-painless/src/main/java/org/elasticsearch/painless/action/PainlessExecuteAction.java index cb407978da83e..d0a5d5cc9611b 100644 --- a/modules/lang-painless/src/main/java/org/elasticsearch/painless/action/PainlessExecuteAction.java +++ b/modules/lang-painless/src/main/java/org/elasticsearch/painless/action/PainlessExecuteAction.java @@ -30,8 +30,7 @@ import org.apache.lucene.search.Scorer; import org.apache.lucene.search.Weight; import org.apache.lucene.store.RAMDirectory; -import org.elasticsearch.Version; -import org.elasticsearch.action.Action; +import org.elasticsearch.action.ActionType; import org.elasticsearch.action.ActionRequestValidationException; import org.elasticsearch.action.ActionResponse; import org.elasticsearch.action.support.ActionFilters; @@ -90,7 +89,7 @@ import static org.elasticsearch.rest.RestRequest.Method.GET; import static org.elasticsearch.rest.RestRequest.Method.POST; -public class PainlessExecuteAction extends Action { +public class PainlessExecuteAction extends ActionType { public static final PainlessExecuteAction INSTANCE = new PainlessExecuteAction(); private static final String NAME = "cluster:admin/scripts/painless/execute"; @@ -100,8 +99,8 @@ private PainlessExecuteAction() { } @Override - public Response newResponse() { - throw new UnsupportedOperationException("usage of Streamable is to be replaced by Writeable"); + public Writeable.Reader getResponseReader() { + return Response::new; } public static class Request extends SingleShardRequest implements ToXContentObject { @@ -258,9 +257,9 @@ public XContentBuilder toXContent(XContentBuilder builder, Params params) throws } - private Script script; - private ScriptContext context = PainlessTestScript.CONTEXT; - private ContextSetup contextSetup; + private final Script script; + private final ScriptContext context; + private final ContextSetup contextSetup; static Request parse(XContentParser parser) throws IOException { return PARSER.parse(parser, null); @@ -268,16 +267,20 @@ static Request parse(XContentParser parser) throws IOException { Request(Script script, String scriptContextName, ContextSetup setup) { this.script = Objects.requireNonNull(script); - if (scriptContextName != null) { - this.context = fromScriptContextName(scriptContextName); - } + this.context = scriptContextName != null ? fromScriptContextName(scriptContextName) : PainlessTestScript.CONTEXT; if (setup != null) { this.contextSetup = setup; index(contextSetup.index); + } else { + contextSetup = null; } } - Request() { + Request(StreamInput in) throws IOException { + super(in); + script = new Script(in); + context = fromScriptContextName(in.readString()); + contextSetup = in.readOptionalWriteable(ContextSetup::new); } public Script getScript() { @@ -309,29 +312,12 @@ public ActionRequestValidationException validate() { return validationException; } - @Override - public void readFrom(StreamInput in) throws IOException { - super.readFrom(in); - script = new Script(in); - if (in.getVersion().before(Version.V_6_4_0)) { - byte scriptContextId = in.readByte(); - assert scriptContextId == 0; - } else { - context = fromScriptContextName(in.readString()); - contextSetup = in.readOptionalWriteable(ContextSetup::new); - } - } - @Override public void writeTo(StreamOutput out) throws IOException { super.writeTo(out); script.writeTo(out); - if (out.getVersion().before(Version.V_6_4_0)) { - out.writeByte((byte) 0); - } else { - out.writeString(context.name); - out.writeOptionalWriteable(contextSetup); - } + out.writeString(context.name); + out.writeOptionalWriteable(contextSetup); } // For testing only: diff --git a/modules/lang-painless/src/main/java/org/elasticsearch/painless/api/Augmentation.java b/modules/lang-painless/src/main/java/org/elasticsearch/painless/api/Augmentation.java index bbbbc3dfc37cf..d0745dc982c36 100644 --- a/modules/lang-painless/src/main/java/org/elasticsearch/painless/api/Augmentation.java +++ b/modules/lang-painless/src/main/java/org/elasticsearch/painless/api/Augmentation.java @@ -25,6 +25,7 @@ import java.util.Collection; import java.util.LinkedHashMap; import java.util.List; +import java.util.Locale; import java.util.Map; import java.util.TreeMap; import java.util.function.BiConsumer; @@ -34,6 +35,7 @@ import java.util.function.Function; import java.util.function.ObjIntConsumer; import java.util.function.Predicate; +import java.util.function.Supplier; import java.util.function.ToDoubleFunction; import java.util.regex.Matcher; import java.util.regex.Pattern; @@ -552,4 +554,115 @@ public static String[] splitOnToken(String receiver, String token, int limit) { // O(N) or faster depending on implementation return result.toArray(new String[0]); } + + /** + * Access values in nested containers with a dot separated path. Path elements are treated + * as strings for Maps and integers for Lists. + * @throws IllegalArgumentException if any of the following: + * - path is empty + * - path contains a trailing '.' or a repeated '.' + * - an element of the path does not exist, ie key or index not present + * - there is a non-container type at a non-terminal path element + * - a path element for a List is not an integer + * @return object at path + */ + public static Object getByPath(List receiver, String path) { + return getByPathDispatch(receiver, splitPath(path), 0, throwCantFindValue(path)); + } + + /** + * Same as {@link #getByPath(List, String)}, but for Map. + */ + public static Object getByPath(Map receiver, String path) { + return getByPathDispatch(receiver, splitPath(path), 0, throwCantFindValue(path)); + } + + /** + * Same as {@link #getByPath(List, String)}, but with a default value. + * @return element at path or {@code defaultValue} if the terminal path element does not exist. + */ + public static Object getByPath(List receiver, String path, Object defaultValue) { + return getByPathDispatch(receiver, splitPath(path), 0, () -> defaultValue); + } + + /** + * Same as {@link #getByPath(List, String, Object)}, but for Map. + */ + public static Object getByPath(Map receiver, String path, Object defaultValue) { + return getByPathDispatch(receiver, splitPath(path), 0, () -> defaultValue); + } + + // Dispatches to getByPathMap, getByPathList or returns obj if done. See handleMissing for dealing with missing + // elements. + private static Object getByPathDispatch(Object obj, String[] elements, int i, Supplier defaultSupplier) { + if (i > elements.length - 1) { + return obj; + } else if (elements[i].length() == 0 ) { + String format = "Extra '.' in path [%s] at index [%d]"; + throw new IllegalArgumentException(String.format(Locale.ROOT, format, String.join(".", elements), i)); + } else if (obj instanceof Map) { + return getByPathMap((Map) obj, elements, i, defaultSupplier); + } else if (obj instanceof List) { + return getByPathList((List) obj, elements, i, defaultSupplier); + } + return handleMissing(obj, elements, i, defaultSupplier); + } + + // lookup existing key in map, call back to dispatch. + private static Object getByPathMap(Map map, String[] elements, int i, Supplier defaultSupplier) { + String element = elements[i]; + if (map.containsKey(element)) { + return getByPathDispatch(map.get(element), elements, i + 1, defaultSupplier); + } + return handleMissing(map, elements, i, defaultSupplier); + } + + // lookup existing index in list, call back to dispatch. Throws IllegalArgumentException with NumberFormatException + // if index can't be parsed as an int. + private static Object getByPathList(List list, String[] elements, int i, Supplier defaultSupplier) { + String element = elements[i]; + try { + int elemInt = Integer.parseInt(element); + if (list.size() >= elemInt) { + return getByPathDispatch(list.get(elemInt), elements, i + 1, defaultSupplier); + } + } catch (NumberFormatException e) { + String format = "Could not parse [%s] as a int index into list at path [%s] and index [%d]"; + throw new IllegalArgumentException(String.format(Locale.ROOT, format, element, String.join(".", elements), i), e); + } + return handleMissing(list, elements, i, defaultSupplier); + } + + // Split path on '.', throws IllegalArgumentException for empty paths and paths ending in '.' + private static String[] splitPath(String path) { + if (path.length() == 0) { + throw new IllegalArgumentException("Missing path"); + } + if (path.endsWith(".")) { + String format = "Trailing '.' in path [%s]"; + throw new IllegalArgumentException(String.format(Locale.ROOT, format, path)); + } + return path.split("\\."); + } + + // A supplier that throws IllegalArgumentException + private static Supplier throwCantFindValue(String path) { + return () -> { + throw new IllegalArgumentException(String.format(Locale.ROOT, "Could not find value at path [%s]", path)); + }; + } + + // Use defaultSupplier if at last path element, otherwise throw IllegalArgumentException + private static Object handleMissing(Object obj, String[] elements, int i, Supplier defaultSupplier) { + if (obj instanceof List || obj instanceof Map) { + if (elements.length - 1 == i) { + return defaultSupplier.get(); + } + String format = "Container does not have [%s], for non-terminal index [%d] in path [%s]"; + throw new IllegalArgumentException(String.format(Locale.ROOT, format, elements[i], i, String.join(".", elements))); + } + String format = "Non-container [%s] at [%s], index [%d] in path [%s]"; + throw new IllegalArgumentException( + String.format(Locale.ROOT, format, obj.getClass().getName(), elements[i], i, String.join(".", elements))); + } } diff --git a/modules/lang-painless/src/main/java/org/elasticsearch/painless/lookup/PainlessLookupBuilder.java b/modules/lang-painless/src/main/java/org/elasticsearch/painless/lookup/PainlessLookupBuilder.java index 6f4400cb4fbd4..0cef518e0d607 100644 --- a/modules/lang-painless/src/main/java/org/elasticsearch/painless/lookup/PainlessLookupBuilder.java +++ b/modules/lang-painless/src/main/java/org/elasticsearch/painless/lookup/PainlessLookupBuilder.java @@ -30,6 +30,7 @@ import org.elasticsearch.painless.spi.WhitelistField; import org.elasticsearch.painless.spi.WhitelistInstanceBinding; import org.elasticsearch.painless.spi.WhitelistMethod; +import org.elasticsearch.painless.spi.annotation.NoImportAnnotation; import org.objectweb.asm.ClassWriter; import org.objectweb.asm.Opcodes; import org.objectweb.asm.commons.GeneratorAdapter; @@ -119,7 +120,8 @@ public static PainlessLookup buildFromWhitelists(List whitelists) { for (WhitelistClass whitelistClass : whitelist.whitelistClasses) { origin = whitelistClass.origin; painlessLookupBuilder.addPainlessClass( - whitelist.classLoader, whitelistClass.javaClassName, whitelistClass.noImport == false); + whitelist.classLoader, whitelistClass.javaClassName, + whitelistClass.painlessAnnotations.containsKey(NoImportAnnotation.class) == false); } } diff --git a/modules/lang-painless/src/main/resources/org/elasticsearch/painless/spi/java.util.txt b/modules/lang-painless/src/main/resources/org/elasticsearch/painless/spi/java.util.txt index 94f302a891d48..958ac927a66dd 100644 --- a/modules/lang-painless/src/main/resources/org/elasticsearch/painless/spi/java.util.txt +++ b/modules/lang-painless/src/main/resources/org/elasticsearch/painless/spi/java.util.txt @@ -126,6 +126,8 @@ class java.util.List { int org.elasticsearch.painless.api.Augmentation getLength() void sort(Comparator) List subList(int,int) + Object org.elasticsearch.painless.api.Augmentation getByPath(String) + Object org.elasticsearch.painless.api.Augmentation getByPath(String, Object) } class java.util.ListIterator { @@ -161,6 +163,8 @@ class java.util.Map { void replaceAll(BiFunction) int size() Collection values() + Object org.elasticsearch.painless.api.Augmentation getByPath(String) + Object org.elasticsearch.painless.api.Augmentation getByPath(String, Object) # some adaptations of groovy methods List org.elasticsearch.painless.api.Augmentation collect(BiFunction) diff --git a/modules/lang-painless/src/main/resources/org/elasticsearch/painless/spi/org.elasticsearch.score.txt b/modules/lang-painless/src/main/resources/org/elasticsearch/painless/spi/org.elasticsearch.score.txt index 03ec9275aa8b7..fd49cbb2d5222 100644 --- a/modules/lang-painless/src/main/resources/org/elasticsearch/painless/spi/org.elasticsearch.score.txt +++ b/modules/lang-painless/src/main/resources/org/elasticsearch/painless/spi/org.elasticsearch.score.txt @@ -19,7 +19,7 @@ # This file contains a whitelist for functions to be used in Score context -class org.elasticsearch.script.ScoreScript no_import { +class org.elasticsearch.script.ScoreScript @no_import { } static_import { @@ -27,10 +27,10 @@ static_import { double sigmoid(double, double, double) from_class org.elasticsearch.script.ScoreScriptUtils double randomScore(org.elasticsearch.script.ScoreScript, int, String) bound_to org.elasticsearch.script.ScoreScriptUtils$RandomScoreField double randomScore(org.elasticsearch.script.ScoreScript, int) bound_to org.elasticsearch.script.ScoreScriptUtils$RandomScoreDoc - double decayGeoLinear(String, String, String, double, GeoPoint) bound_to org.elasticsearch.script.ScoreScriptUtils$DecayGeoLinear - double decayGeoExp(String, String, String, double, GeoPoint) bound_to org.elasticsearch.script.ScoreScriptUtils$DecayGeoExp - double decayGeoGauss(String, String, String, double, GeoPoint) bound_to org.elasticsearch.script.ScoreScriptUtils$DecayGeoGauss - double decayNumericLinear(double, double, double, double, double) bound_to org.elasticsearch.script.ScoreScriptUtils$DecayNumericLinear + double decayGeoLinear(String, String, String, double, GeoPoint) bound_to org.elasticsearch.script.ScoreScriptUtils$DecayGeoLinear + double decayGeoExp(String, String, String, double, GeoPoint) bound_to org.elasticsearch.script.ScoreScriptUtils$DecayGeoExp + double decayGeoGauss(String, String, String, double, GeoPoint) bound_to org.elasticsearch.script.ScoreScriptUtils$DecayGeoGauss + double decayNumericLinear(double, double, double, double, double)bound_to org.elasticsearch.script.ScoreScriptUtils$DecayNumericLinear double decayNumericExp(double, double, double, double, double) bound_to org.elasticsearch.script.ScoreScriptUtils$DecayNumericExp double decayNumericGauss(double, double, double, double, double) bound_to org.elasticsearch.script.ScoreScriptUtils$DecayNumericGauss double decayDateLinear(String, String, String, double, JodaCompatibleZonedDateTime) bound_to org.elasticsearch.script.ScoreScriptUtils$DecayDateLinear @@ -38,4 +38,3 @@ static_import { double decayDateGauss(String, String, String, double, JodaCompatibleZonedDateTime) bound_to org.elasticsearch.script.ScoreScriptUtils$DecayDateGauss } - diff --git a/modules/lang-painless/src/main/resources/org/elasticsearch/painless/spi/org.elasticsearch.txt b/modules/lang-painless/src/main/resources/org/elasticsearch/painless/spi/org.elasticsearch.txt index 412afc62dcf54..031fc342f339b 100644 --- a/modules/lang-painless/src/main/resources/org/elasticsearch/painless/spi/org.elasticsearch.txt +++ b/modules/lang-painless/src/main/resources/org/elasticsearch/painless/spi/org.elasticsearch.txt @@ -24,31 +24,31 @@ #### Primitive types -class void no_import { +class void @no_import { } -class boolean no_import { +class boolean @no_import { } -class byte no_import { +class byte @no_import { } -class short no_import { +class short @no_import { } -class char no_import { +class char @no_import { } -class int no_import { +class int @no_import { } -class long no_import { +class long @no_import { } -class float no_import { +class float @no_import { } -class double no_import { +class double @no_import { } #### Painless debugging API diff --git a/modules/lang-painless/src/test/java/org/elasticsearch/painless/AnnotationTestObject.java b/modules/lang-painless/src/test/java/org/elasticsearch/painless/AnnotationTestObject.java new file mode 100644 index 0000000000000..bc9309bbbb611 --- /dev/null +++ b/modules/lang-painless/src/test/java/org/elasticsearch/painless/AnnotationTestObject.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.painless; + +import org.elasticsearch.painless.spi.annotation.WhitelistAnnotationParser; + +import java.util.Map; + +public class AnnotationTestObject { + + public static class TestAnnotation { + + public static final String NAME = "test_annotation"; + + private final String one; + private final String two; + private final String three; + + public TestAnnotation(String one, String two, String three) { + this.one = one; + this.two = two; + this.three = three; + } + + public String getOne() { + return one; + } + + public String getTwo() { + return two; + } + + public String getThree() { + return three; + } + } + + public static class TestAnnotationParser implements WhitelistAnnotationParser { + + public static final TestAnnotationParser INSTANCE = new TestAnnotationParser(); + + private TestAnnotationParser() { + + } + + @Override + public Object parse(Map arguments) { + if (arguments.size() != 3) { + throw new IllegalArgumentException("expected three arguments"); + } + + String one = arguments.get("one"); + + if (one == null) { + throw new IllegalArgumentException("missing one"); + } + + String two = arguments.get("two"); + + if (two == null) { + throw new IllegalArgumentException("missing two"); + } + + String three = arguments.get("three"); + + if (three == null) { + throw new IllegalArgumentException("missing three"); + } + + return new TestAnnotation(one, two, three); + } + } + + public void deprecatedMethod() { + + } + + public void annotatedTestMethod() { + + } + + public void annotatedMultipleMethod() { + + } +} diff --git a/modules/lang-painless/src/test/java/org/elasticsearch/painless/AugmentationTests.java b/modules/lang-painless/src/test/java/org/elasticsearch/painless/AugmentationTests.java index 70fbb733e2f8f..e462997444165 100644 --- a/modules/lang-painless/src/test/java/org/elasticsearch/painless/AugmentationTests.java +++ b/modules/lang-painless/src/test/java/org/elasticsearch/painless/AugmentationTests.java @@ -232,7 +232,6 @@ public void testString_SplitOnToken() { new SplitCase("1\n1.1.\r\n1\r\n111", "\r\n"), }; for (SplitCase split : cases) { - //System.out.println(String.format("Splitting '%s' by '%s' %d times", split.input, split.token, split.count)); assertArrayEquals( split.input.split(Pattern.quote(split.token), split.count), (String[])exec("return \""+split.input+"\".splitOnToken(\""+split.token+"\", "+split.count+");") diff --git a/modules/lang-painless/src/test/java/org/elasticsearch/painless/BindingsTests.java b/modules/lang-painless/src/test/java/org/elasticsearch/painless/BindingsTests.java index 6aca40fc4cdaf..171880abd7907 100644 --- a/modules/lang-painless/src/test/java/org/elasticsearch/painless/BindingsTests.java +++ b/modules/lang-painless/src/test/java/org/elasticsearch/painless/BindingsTests.java @@ -103,9 +103,9 @@ protected Map, List> scriptContexts() { InstanceBindingTestClass instanceBindingTestClass = new InstanceBindingTestClass(1); WhitelistInstanceBinding getter = new WhitelistInstanceBinding("test", instanceBindingTestClass, - "setInstanceBindingValue", "void", Collections.singletonList("int")); + "setInstanceBindingValue", "void", Collections.singletonList("int"), Collections.emptyList()); WhitelistInstanceBinding setter = new WhitelistInstanceBinding("test", instanceBindingTestClass, - "getInstanceBindingValue", "int", Collections.emptyList()); + "getInstanceBindingValue", "int", Collections.emptyList(), Collections.emptyList()); List instanceBindingsList = new ArrayList<>(); instanceBindingsList.add(getter); instanceBindingsList.add(setter); diff --git a/modules/lang-painless/src/test/java/org/elasticsearch/painless/DateTimeTests.java b/modules/lang-painless/src/test/java/org/elasticsearch/painless/DateTimeTests.java new file mode 100644 index 0000000000000..38ceae74e053a --- /dev/null +++ b/modules/lang-painless/src/test/java/org/elasticsearch/painless/DateTimeTests.java @@ -0,0 +1,194 @@ +/* + * 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.painless; + +import java.time.ZoneId; +import java.time.ZonedDateTime; + +public class DateTimeTests extends ScriptTestCase { + + public void testLongToZonedDateTime() { + assertEquals(ZonedDateTime.of(1983, 10, 13, 22, 15, 30, 0, ZoneId.of("Z")), exec( + "long milliSinceEpoch = 434931330000L;" + + "Instant instant = Instant.ofEpochMilli(milliSinceEpoch);" + + "return ZonedDateTime.ofInstant(instant, ZoneId.of('Z'));" + )); + } + + public void testStringToZonedDateTime() { + assertEquals(ZonedDateTime.of(1983, 10, 13, 22, 15, 30, 0, ZoneId.of("Z")), exec( + "String milliSinceEpochString = '434931330000';" + + "long milliSinceEpoch = Long.parseLong(milliSinceEpochString);" + + "Instant instant = Instant.ofEpochMilli(milliSinceEpoch);" + + "return ZonedDateTime.ofInstant(instant, ZoneId.of('Z'));" + )); + + assertEquals(ZonedDateTime.of(1983, 10, 13, 22, 15, 30, 0, ZoneId.of("Z")), exec( + "String datetime = '1983-10-13T22:15:30Z';" + + "return ZonedDateTime.parse(datetime);" + )); + + assertEquals(ZonedDateTime.of(1983, 10, 13, 22, 15, 30, 0, ZoneId.of("Z")), exec( + "String datetime = 'Thu, 13 Oct 1983 22:15:30 GMT';" + + "return ZonedDateTime.parse(datetime, DateTimeFormatter.RFC_1123_DATE_TIME);" + )); + + assertEquals(ZonedDateTime.of(1983, 10, 13, 22, 15, 30, 0, ZoneId.of("Z")), exec( + "String datetime = 'custom y 1983 m 10 d 13 22:15:30 Z';" + + "DateTimeFormatter dtf = DateTimeFormatter.ofPattern(" + + "\"'custom' 'y' yyyy 'm' MM 'd' dd HH:mm:ss VV\");" + + "return ZonedDateTime.parse(datetime, dtf);" + )); + } + + public void testPiecesToZonedDateTime() { + assertEquals(ZonedDateTime.of(1983, 10, 13, 22, 15, 30, 0, ZoneId.of("Z")), exec( + "int year = 1983;" + + "int month = 10;" + + "int day = 13;" + + "int hour = 22;" + + "int minutes = 15;" + + "int seconds = 30;" + + "int nanos = 0;" + + "String tz = 'Z';" + + "return ZonedDateTime.of(year, month, day, hour, minutes, seconds, nanos, ZoneId.of(tz));" + )); + } + + public void testZonedDatetimeToLong() { + assertEquals(434931330000L, exec( + "ZonedDateTime zdt = ZonedDateTime.of(1983, 10, 13, 22, 15, 30, 0, ZoneId.of('Z'));" + + "return zdt.toInstant().toEpochMilli();" + )); + } + + public void testZonedDateTimeToString() { + assertEquals("1983-10-13T22:15:30Z", exec( + "ZonedDateTime zdt = ZonedDateTime.of(1983, 10, 13, 22, 15, 30, 0, ZoneId.of('Z'));" + + "return zdt.format(DateTimeFormatter.ISO_INSTANT);" + )); + + assertEquals("date: 1983/10/13 time: 22:15:30", exec( + "ZonedDateTime zdt = ZonedDateTime.of(1983, 10, 13, 22, 15, 30, 0, ZoneId.of('Z'));" + + "DateTimeFormatter dtf = DateTimeFormatter.ofPattern(" + + "\"'date:' yyyy/MM/dd 'time:' HH:mm:ss\");" + + "return zdt.format(dtf);" + )); + } + + public void testZonedDateTimeToPieces() { + assertArrayEquals(new int[] {1983, 10, 13, 22, 15, 30, 100}, (int[])exec( + "int[] pieces = new int[7];" + + "ZonedDateTime zdt = ZonedDateTime.of(1983, 10, 13, 22, 15, 30, 100, ZoneId.of('Z'));" + + "pieces[0] = zdt.year;" + + "pieces[1] = zdt.monthValue;" + + "pieces[2] = zdt.dayOfMonth;" + + "pieces[3] = zdt.hour;" + + "pieces[4] = zdt.minute;" + + "pieces[5] = zdt.second;" + + "pieces[6] = zdt.nano;" + + "return pieces;" + )); + } + + public void testLongManipulation() { + assertEquals(ZonedDateTime.of(1983, 10, 13, 22, 15, 27, 0, ZoneId.of("Z")), exec( + "long milliSinceEpoch = 434931330000L;" + + "milliSinceEpoch = milliSinceEpoch - 1000L*3L;" + + "Instant instant = Instant.ofEpochMilli(milliSinceEpoch);" + + "return ZonedDateTime.ofInstant(instant, ZoneId.of('Z'))" + )); + } + + public void testZonedDateTimeManipulation() { + assertEquals(ZonedDateTime.of(1983, 10, 16, 22, 15, 30, 0, ZoneId.of("Z")), exec( + "ZonedDateTime zdt = ZonedDateTime.of(1983, 10, 13, 22, 15, 30, 0, ZoneId.of('Z'));" + + "return zdt.plusDays(3);" + )); + + assertEquals(ZonedDateTime.of(1983, 10, 13, 20, 10, 30, 0, ZoneId.of("Z")), exec( + "ZonedDateTime zdt = ZonedDateTime.of(1983, 10, 13, 22, 15, 30, 0, ZoneId.of('Z'));" + + "return zdt.minusMinutes(125);" + )); + + assertEquals(ZonedDateTime.of(1976, 10, 13, 22, 15, 30, 0, ZoneId.of("Z")), exec( + "ZonedDateTime zdt = ZonedDateTime.of(1983, 10, 13, 22, 15, 30, 0, ZoneId.of('Z'));" + + "return zdt.withYear(1976);" + )); + } + + public void testLongTimeDifference() { + assertEquals(3000L, exec( + "long startTimestamp = 434931327000L;" + + "long endTimestamp = 434931330000L;" + + "return endTimestamp - startTimestamp;" + )); + } + + public void testZonedDateTimeDifference() { + assertEquals(4989L, exec( + "ZonedDateTime zdt1 = ZonedDateTime.of(1983, 10, 13, 22, 15, 30, 11000000, ZoneId.of('Z'));" + + "ZonedDateTime zdt2 = ZonedDateTime.of(1983, 10, 13, 22, 15, 35, 0, ZoneId.of('Z'));" + + "return ChronoUnit.MILLIS.between(zdt1, zdt2);" + )); + + assertEquals(4L, exec( + "ZonedDateTime zdt1 = ZonedDateTime.of(1983, 10, 13, 22, 15, 30, 11000000, ZoneId.of('Z'));" + + "ZonedDateTime zdt2 = ZonedDateTime.of(1983, 10, 17, 22, 15, 35, 0, ZoneId.of('Z'));" + + "return ChronoUnit.DAYS.between(zdt1, zdt2);" + )); + } + + public void compareLongs() { + assertEquals(false, exec( + "long ts1 = 434931327000L;" + + "long ts2 = 434931330000L;" + + "return ts1 > ts2;" + )); + } + + public void compareZonedDateTimes() { + assertEquals(true, exec( + "ZonedDateTime zdt1 = ZonedDateTime.of(1983, 10, 13, 22, 15, 30, 0, ZoneId.of('Z'));" + + "ZonedDateTime zdt2 = ZonedDateTime.of(1983, 10, 17, 22, 15, 35, 0, ZoneId.of('Z'));" + + "return zdt1.isBefore(zdt2);" + )); + + assertEquals(false, exec( + "ZonedDateTime zdt1 = ZonedDateTime.of(1983, 10, 13, 22, 15, 30, 0, ZoneId.of('Z'));" + + "ZonedDateTime zdt2 = ZonedDateTime.of(1983, 10, 17, 22, 15, 35, 0, ZoneId.of('Z'));" + + "return zdt1.isAfter(zdt2);" + )); + } + + public void testTimeZone() { + assertEquals(ZonedDateTime.of(1983, 10, 13, 15, 15, 30, 0, ZoneId.of("America/Los_Angeles")), exec( + "ZonedDateTime utc = ZonedDateTime.of(1983, 10, 13, 22, 15, 30, 0, ZoneId.of('Z'));" + + "return utc.withZoneSameInstant(ZoneId.of('America/Los_Angeles'));")); + + assertEquals("Thu, 13 Oct 1983 15:15:30 -0700", exec( + "String gmtString = 'Thu, 13 Oct 1983 22:15:30 GMT';" + + "ZonedDateTime gmtZdt = ZonedDateTime.parse(gmtString," + + "DateTimeFormatter.RFC_1123_DATE_TIME);" + + "ZonedDateTime pstZdt =" + + "gmtZdt.withZoneSameInstant(ZoneId.of('America/Los_Angeles'));" + + "return pstZdt.format(DateTimeFormatter.RFC_1123_DATE_TIME);")); + } +} diff --git a/modules/lang-painless/src/test/java/org/elasticsearch/painless/GetByPathAugmentationTests.java b/modules/lang-painless/src/test/java/org/elasticsearch/painless/GetByPathAugmentationTests.java new file mode 100644 index 0000000000000..603ab7fd0e60c --- /dev/null +++ b/modules/lang-painless/src/test/java/org/elasticsearch/painless/GetByPathAugmentationTests.java @@ -0,0 +1,259 @@ +/* + * 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.painless; + + +import java.util.ArrayList; +import java.util.HashMap; +import java.util.List; +import java.util.Locale; +import java.util.Map; + +public class GetByPathAugmentationTests extends ScriptTestCase { + + private final String k001Key = "k011"; + private final String k001Value = "b"; + private final Map k001Obj = new HashMap<>(); + private final String k001MapStr = "['" + k001Key + "': '" + k001Value + "']"; + private final String mapMapList = "['k0': ['k01': [['k010': 'a'], " + k001MapStr + "]], 'k1': ['q']]"; + + private final String l2m2l1Index0 = "ll0"; + private final String l2m2l1Index1 = "ll1"; + private final List l2m2l1Obj = new ArrayList<>(); + private final String l2m2l1Str = "['" + l2m2l1Index0 + "', '" + l2m2l1Index1 + "']"; + private final String listMapListList = "[['m0':'v0'],['m1':'v1'],['m2':['l0','l1', " + l2m2l1Str + "]]]"; + + private final String mapList = "['key0': ['a', 'b'], 'key1': ['c', 'd']]"; + private final String mapMap = "['a': ['b': 'c']]"; + + public GetByPathAugmentationTests() { + l2m2l1Obj.add(l2m2l1Index0); + l2m2l1Obj.add(l2m2l1Index1); + k001Obj.put(k001Key, k001Value); + } + + private String toScript(String collection, String key) { + return String.format(Locale.ROOT, "return %s.getByPath('%s')", collection, key); + } + + private String toScript(String collection, String key, String defaultValue) { + return String.format(Locale.ROOT, "return %s.getByPath('%s', %s)", collection, key, defaultValue); + } + + private String numberFormat(String unparsable, String path, int i) { + String format = "Could not parse [%s] as a int index into list at path [%s] and index [%d]"; + return String.format(Locale.ROOT, format, unparsable, path, i); + } + + private String missingValue(String path) { + return String.format(Locale.ROOT, "Could not find value at path [%s]", path); + } + + private void assertPathValue(String collection, String key, Object value) { + assertEquals(value, exec(toScript(collection, key))); + } + + private void assertPathDefaultValue(String collection, String key, Object value, String defaultValue) { + assertEquals(value, exec(toScript(collection, key, defaultValue))); + } + + private IllegalArgumentException assertPathError(String collection, String key, String message) { + return assertPathError(toScript(collection, key), message); + } + + private IllegalArgumentException assertPathError(String collection, String key, String defaultValue, String message) { + return assertPathError(toScript(collection, key, defaultValue), message); + } + + private IllegalArgumentException assertPathError(String script, String message) { + IllegalArgumentException illegal = expectScriptThrows( + IllegalArgumentException.class, + () -> exec(script) + ); + assertEquals(message, illegal.getMessage()); + return illegal; + } + + public void testOneLevelMap() { + assertPathValue("['k0':'v0']", "k0", "v0"); + } + + public void testOneLevelList() { + assertPathValue("['a','b','c','d']", "2", "c"); + } + + public void testTwoLevelMapList() { + assertPathValue("['key0': ['a', 'b'], 'key1': ['c', 'd']]", "key1.0", "c"); + } + + public void testMapDiffSizeList() { + assertPathValue("['k0': ['a','b','c','d'], 'k1': ['q']]", "k0.3", "d"); + } + + public void testBiMapList() { + assertPathValue(mapMapList, "k0.k01.1.k011", k001Value); + } + + public void testBiMapListObject() { + assertPathValue(mapMapList, "k0.k01.1", k001Obj); + } + + public void testListMap() { + assertPathValue("[['key0': 'value0'], ['key1': 'value1']]", "1.key1", "value1"); + } + + public void testTriList() { + assertPathValue("[['a','b'],['c','d'],[['e','f'],['g','h']]]", "2.1.1", "h"); + } + + public void testMapBiListObject() { + assertPathValue(listMapListList, "2.m2.2", l2m2l1Obj); + } + + public void testMapBiList() { + assertPathValue(listMapListList, "2.m2.2.1", l2m2l1Index1); + } + + public void testGetCollection() { + List k1List = new ArrayList<>(); + k1List.add("c"); + k1List.add("d"); + assertPathValue("['key0': ['a', 'b'], 'key1': ['c', 'd']]", "key1", k1List); + } + + public void testMapListDefaultOneLevel() { + assertPathDefaultValue(mapList, "key2", "x", "'x'"); + } + + public void testMapListDefaultTwoLevel() { + assertPathDefaultValue(mapList, "key1.1", "d", "'x'"); + } + + public void testBiMapListDefault() { + assertPathDefaultValue(mapMapList, "k0.k01.1.k012", "foo", "'foo'"); + } + + public void testBiMapListDefaultExists() { + assertPathDefaultValue(mapMapList, "k0.k01.1.k011", "b", "'foo'"); + } + + public void testBiMapListDefaultObjectExists() { + assertPathDefaultValue(mapMapList, "k0.k01.1", k001Obj, "'foo'"); + } + + public void testBiMapListDefaultObject() { + assertPathDefaultValue(mapMapList, "k0.k01.9", k001Obj, k001MapStr); + } + + public void testListMapBiListDefaultExists() { + assertPathDefaultValue(listMapListList, "2.m2.2", l2m2l1Obj, "'foo'"); + } + + public void testListMapBiListDefaultObject() { + assertPathDefaultValue(listMapListList, "2.m2.9", l2m2l1Obj, l2m2l1Str); + } + + public void testBiListBadIndex() { + String path = "1.k0"; + IllegalArgumentException err = assertPathError("[['a','b'],['c','d']]", path, numberFormat("k0", path, 1)); + assertEquals(err.getCause().getClass(), NumberFormatException.class); + } + + public void testBiMapListMissingLast() { + String path = "k0.k01.1.k012"; + assertPathError(mapMapList, path, missingValue(path)); + } + + public void testBiMapListBadIndex() { + String path = "k0.k01.k012"; + IllegalArgumentException err = assertPathError(mapMapList, path, numberFormat("k012", path, 2)); + assertEquals(err.getCause().getClass(), NumberFormatException.class); + } + + public void testListMapBiListMissingObject() { + String path = "2.m2.12"; + assertPathError(listMapListList, path, missingValue(path)); + } + + public void testListMapBiListBadIndexAtObject() { + String path = "2.m2.a8"; + IllegalArgumentException err = assertPathError(listMapListList, path, numberFormat("a8", path, 2)); + assertEquals(err.getCause().getClass(), NumberFormatException.class); + } + + public void testNonContainer() { + assertPathError(mapMap, "a.b.c", "Non-container [java.lang.String] at [c], index [2] in path [a.b.c]"); + } + + public void testMissingPath() { + assertPathError(mapMap, "", "Missing path"); + } + + public void testDoubleDot() { + assertPathError(mapMap, "a..b", "Extra '.' in path [a..b] at index [1]"); + } + + public void testTrailingDot() { + assertPathError(mapMap, "a.b.", "Trailing '.' in path [a.b.]"); + } + + public void testBiListDefaultBadIndex() { + String path = "1.k0"; + IllegalArgumentException err = assertPathError( + "[['a','b'],['c','d']]", + path, + "'foo'", + numberFormat("k0", path, 1)); + assertEquals(err.getCause().getClass(), NumberFormatException.class); + } + + public void testBiMapListDefaultBadIndex() { + String path = "k0.k01.k012"; + IllegalArgumentException err = assertPathError( + mapMapList, + path, + "'foo'", + numberFormat("k012", path, 2)); + assertEquals(err.getCause().getClass(), NumberFormatException.class); + } + + public void testListMapBiListObjectDefaultBadIndex() { + String path = "2.m2.a8"; + IllegalArgumentException err = assertPathError( + listMapListList, + path, + "'foo'", + numberFormat("a8", path, 2)); + assertEquals(err.getCause().getClass(), NumberFormatException.class); + } + + public void testNonContainerDefaultBadIndex() { + assertPathError(mapMap, "a.b.c", "'foo'", + "Non-container [java.lang.String] at [c], index [2] in path [a.b.c]"); + } + + public void testDoubleDotDefault() { + assertPathError(mapMap, "a..b", "'foo'", "Extra '.' in path [a..b] at index [1]"); + } + + public void testTrailingDotDefault() { + assertPathError(mapMap, "a.b.", "'foo'", "Trailing '.' in path [a.b.]"); + } +} diff --git a/modules/lang-painless/src/test/java/org/elasticsearch/painless/NeedsScoreTests.java b/modules/lang-painless/src/test/java/org/elasticsearch/painless/NeedsScoreTests.java index eeb636d6697c6..92816426eaf47 100644 --- a/modules/lang-painless/src/test/java/org/elasticsearch/painless/NeedsScoreTests.java +++ b/modules/lang-painless/src/test/java/org/elasticsearch/painless/NeedsScoreTests.java @@ -47,7 +47,7 @@ public void testNeedsScores() { PainlessScriptEngine service = new PainlessScriptEngine(Settings.EMPTY, contexts); QueryShardContext shardContext = index.newQueryShardContext(0, null, () -> 0, null); - SearchLookup lookup = new SearchLookup(index.mapperService(), shardContext::getForField, null); + SearchLookup lookup = new SearchLookup(index.mapperService(), shardContext::getForField); NumberSortScript.Factory factory = service.compile(null, "1.2", NumberSortScript.CONTEXT, Collections.emptyMap()); NumberSortScript.LeafFactory ss = factory.newFactory(Collections.emptyMap(), lookup); diff --git a/modules/lang-painless/src/test/java/org/elasticsearch/painless/PainlessDocGenerator.java b/modules/lang-painless/src/test/java/org/elasticsearch/painless/PainlessDocGenerator.java deleted file mode 100644 index c1ba6bfbe1c75..0000000000000 --- a/modules/lang-painless/src/test/java/org/elasticsearch/painless/PainlessDocGenerator.java +++ /dev/null @@ -1,451 +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.painless; - -import org.apache.logging.log4j.LogManager; -import org.apache.logging.log4j.Logger; -import org.elasticsearch.common.io.PathUtils; -import org.elasticsearch.core.internal.io.IOUtils; -import org.elasticsearch.painless.lookup.PainlessClass; -import org.elasticsearch.painless.lookup.PainlessConstructor; -import org.elasticsearch.painless.lookup.PainlessField; -import org.elasticsearch.painless.lookup.PainlessLookup; -import org.elasticsearch.painless.lookup.PainlessLookupBuilder; -import org.elasticsearch.painless.lookup.PainlessLookupUtility; -import org.elasticsearch.painless.lookup.PainlessMethod; -import org.elasticsearch.painless.lookup.def; -import org.elasticsearch.painless.spi.Whitelist; - -import java.io.IOException; -import java.io.PrintStream; -import java.lang.reflect.Modifier; -import java.nio.charset.StandardCharsets; -import java.nio.file.Files; -import java.nio.file.Path; -import java.nio.file.StandardOpenOption; -import java.util.Comparator; -import java.util.List; -import java.util.Map; -import java.util.TreeMap; -import java.util.function.Consumer; -import java.util.stream.Collectors; - -import static java.util.Comparator.comparing; - -/** - * Generates an API reference from the method and type whitelists in {@link PainlessLookup}. - */ -public class PainlessDocGenerator { - - private static final PainlessLookup PAINLESS_LOOKUP = PainlessLookupBuilder.buildFromWhitelists(Whitelist.BASE_WHITELISTS); - private static final Logger logger = LogManager.getLogger(PainlessDocGenerator.class); - private static final Comparator FIELD_NAME = comparing(f -> f.javaField.getName()); - private static final Comparator METHOD_NAME = comparing(m -> m.javaMethod.getName()); - private static final Comparator METHOD_NUMBER_OF_PARAMS = comparing(m -> m.typeParameters.size()); - private static final Comparator CONSTRUCTOR_NUMBER_OF_PARAMS = comparing(m -> m.typeParameters.size()); - - public static void main(String[] args) throws IOException { - Path apiRootPath = PathUtils.get(args[0]); - - // Blow away the last execution and recreate it from scratch - IOUtils.rm(apiRootPath); - Files.createDirectories(apiRootPath); - - Path indexPath = apiRootPath.resolve("index.asciidoc"); - logger.info("Starting to write [index.asciidoc]"); - try (PrintStream indexStream = new PrintStream( - Files.newOutputStream(indexPath, StandardOpenOption.CREATE_NEW, StandardOpenOption.WRITE), - false, StandardCharsets.UTF_8.name())) { - emitGeneratedWarning(indexStream); - List> classes = PAINLESS_LOOKUP.getClasses().stream().sorted( - Comparator.comparing(Class::getCanonicalName)).collect(Collectors.toList()); - for (Class clazz : classes) { - PainlessClass struct = PAINLESS_LOOKUP.lookupPainlessClass(clazz); - String canonicalClassName = PainlessLookupUtility.typeToCanonicalTypeName(clazz); - - if (clazz.isPrimitive()) { - // Primitives don't have methods to reference - continue; - } - if (clazz == def.class) { - // def is special but doesn't have any methods all of its own. - continue; - } - indexStream.print("include::"); - indexStream.print(canonicalClassName); - indexStream.println(".asciidoc[]"); - - Path typePath = apiRootPath.resolve(canonicalClassName + ".asciidoc"); - logger.info("Writing [{}.asciidoc]", canonicalClassName); - try (PrintStream typeStream = new PrintStream( - Files.newOutputStream(typePath, StandardOpenOption.CREATE_NEW, StandardOpenOption.WRITE), - false, StandardCharsets.UTF_8.name())) { - emitGeneratedWarning(typeStream); - typeStream.print("[["); - emitAnchor(typeStream, clazz); - typeStream.print("]]++"); - typeStream.print(canonicalClassName); - typeStream.println("++::"); - - Consumer documentField = field -> PainlessDocGenerator.documentField(typeStream, field); - Consumer documentMethod = method -> PainlessDocGenerator.documentMethod(typeStream, method); - Consumer documentConstructor = - constructor -> PainlessDocGenerator.documentConstructor(typeStream, constructor); - struct.staticFields.values().stream().sorted(FIELD_NAME).forEach(documentField); - struct.fields.values().stream().sorted(FIELD_NAME).forEach(documentField); - struct.staticMethods.values().stream().sorted( - METHOD_NAME.thenComparing(METHOD_NUMBER_OF_PARAMS)).forEach(documentMethod); - struct.constructors.values().stream().sorted(CONSTRUCTOR_NUMBER_OF_PARAMS).forEach(documentConstructor); - Map> inherited = new TreeMap<>(); - struct.methods.values().stream().sorted(METHOD_NAME.thenComparing(METHOD_NUMBER_OF_PARAMS)).forEach(method -> { - if (method.targetClass == clazz) { - documentMethod(typeStream, method); - } else { - inherited.put(canonicalClassName, method.targetClass); - } - }); - - if (false == inherited.isEmpty()) { - typeStream.print("* Inherits methods from "); - boolean first = true; - for (Class inheritsFrom : inherited.values()) { - if (first) { - first = false; - } else { - typeStream.print(", "); - } - typeStream.print("++"); - emitStruct(typeStream, inheritsFrom); - typeStream.print("++"); - } - typeStream.println(); - } - } - } - } - logger.info("Done writing [index.asciidoc]"); - } - - private static void documentField(PrintStream stream, PainlessField field) { - stream.print("** [["); - emitAnchor(stream, field); - stream.print("]]"); - - if (Modifier.isStatic(field.javaField.getModifiers())) { - stream.print("static "); - } - - emitType(stream, field.typeParameter); - stream.print(' '); - - String javadocRoot = javadocRoot(field); - emitJavadocLink(stream, javadocRoot, field); - stream.print('['); - stream.print(field.javaField.getName()); - stream.print(']'); - - if (javadocRoot.equals("java8")) { - stream.print(" ("); - emitJavadocLink(stream, "java9", field); - stream.print("[java 9])"); - } - - stream.println(); - } - - /** - * Document a constructor. - */ - private static void documentConstructor(PrintStream stream, PainlessConstructor constructor) { - stream.print("* ++[["); - emitAnchor(stream, constructor); - stream.print("]]"); - - String javadocRoot = javadocRoot(constructor.javaConstructor.getDeclaringClass()); - emitJavadocLink(stream, javadocRoot, constructor); - stream.print('['); - - stream.print(constructorName(constructor)); - - stream.print("]("); - boolean first = true; - for (Class arg : constructor.typeParameters) { - if (first) { - first = false; - } else { - stream.print(", "); - } - emitType(stream, arg); - } - stream.print(")++"); - - if (javadocRoot.equals("java8")) { - stream.print(" ("); - emitJavadocLink(stream, "java9", constructor); - stream.print("[java 9])"); - } - - stream.println(); - } - - /** - * Document a method. - */ - private static void documentMethod(PrintStream stream, PainlessMethod method) { - stream.print("* ++[["); - emitAnchor(stream, method); - stream.print("]]"); - - if (method.targetClass == method.javaMethod.getDeclaringClass() && Modifier.isStatic(method.javaMethod.getModifiers())) { - stream.print("static "); - } - - emitType(stream, method.returnType); - stream.print(' '); - - String javadocRoot = javadocRoot(method); - emitJavadocLink(stream, javadocRoot, method); - stream.print('['); - - stream.print(methodName(method)); - - stream.print("]("); - boolean first = true; - for (Class arg : method.typeParameters) { - if (first) { - first = false; - } else { - stream.print(", "); - } - emitType(stream, arg); - } - stream.print(")++"); - - if (javadocRoot.equals("java8")) { - stream.print(" ("); - emitJavadocLink(stream, "java9", method); - stream.print("[java 9])"); - } - - stream.println(); - } - - /** - * Anchor text for a {@link PainlessClass}. - */ - private static void emitAnchor(PrintStream stream, Class clazz) { - stream.print("painless-api-reference-"); - stream.print(PainlessLookupUtility.typeToCanonicalTypeName(clazz).replace('.', '-')); - } - - /** - * Anchor text for a {@link PainlessConstructor}. - */ - private static void emitAnchor(PrintStream stream, PainlessConstructor constructor) { - emitAnchor(stream, constructor.javaConstructor.getDeclaringClass()); - stream.print('-'); - stream.print(constructorName(constructor)); - stream.print('-'); - stream.print(constructor.typeParameters.size()); - } - - /** - * Anchor text for a {@link PainlessMethod}. - */ - private static void emitAnchor(PrintStream stream, PainlessMethod method) { - emitAnchor(stream, method.targetClass); - stream.print('-'); - stream.print(methodName(method)); - stream.print('-'); - stream.print(method.typeParameters.size()); - } - - /** - * Anchor text for a {@link PainlessField}. - */ - private static void emitAnchor(PrintStream stream, PainlessField field) { - emitAnchor(stream, field.javaField.getDeclaringClass()); - stream.print('-'); - stream.print(field.javaField.getName()); - } - - private static String constructorName(PainlessConstructor constructor) { - return PainlessLookupUtility.typeToCanonicalTypeName(constructor.javaConstructor.getDeclaringClass()); - } - - private static String methodName(PainlessMethod method) { - return PainlessLookupUtility.typeToCanonicalTypeName(method.targetClass); - } - - /** - * Emit a {@link Class}. If the type is primitive or an array of primitives this just emits the name of the type. Otherwise this emits - an internal link with the text. - */ - private static void emitType(PrintStream stream, Class clazz) { - emitStruct(stream, clazz); - while ((clazz = clazz.getComponentType()) != null) { - stream.print("[]"); - } - } - - /** - * Emit a {@link PainlessClass}. If the {@linkplain PainlessClass} is primitive or def this just emits the name of the struct. - * Otherwise this emits an internal link with the name. - */ - private static void emitStruct(PrintStream stream, Class clazz) { - String canonicalClassName = PainlessLookupUtility.typeToCanonicalTypeName(clazz); - - if (false == clazz.isPrimitive() && clazz != def.class) { - stream.print("<<"); - emitAnchor(stream, clazz); - stream.print(','); - stream.print(canonicalClassName); - stream.print(">>"); - } else { - stream.print(canonicalClassName); - } - } - - /** - * Emit an external link to Javadoc for a {@link PainlessMethod}. - * - * @param root name of the root uri variable - */ - private static void emitJavadocLink(PrintStream stream, String root, PainlessConstructor constructor) { - stream.print("link:{"); - stream.print(root); - stream.print("-javadoc}/"); - stream.print(classUrlPath(constructor.javaConstructor.getDeclaringClass())); - stream.print(".html#"); - stream.print(constructorName(constructor)); - stream.print("%2D"); - boolean first = true; - for (Class clazz: constructor.typeParameters) { - if (first) { - first = false; - } else { - stream.print("%2D"); - } - stream.print(clazz.getName()); - if (clazz.isArray()) { - stream.print(":A"); - } - } - stream.print("%2D"); - } - - /** - * Emit an external link to Javadoc for a {@link PainlessMethod}. - * - * @param root name of the root uri variable - */ - private static void emitJavadocLink(PrintStream stream, String root, PainlessMethod method) { - stream.print("link:{"); - stream.print(root); - stream.print("-javadoc}/"); - stream.print(classUrlPath(method.javaMethod.getDeclaringClass())); - stream.print(".html#"); - stream.print(methodName(method)); - stream.print("%2D"); - boolean first = true; - if (method.targetClass != method.javaMethod.getDeclaringClass()) { - first = false; - stream.print(method.javaMethod.getDeclaringClass().getName()); - } - for (Class clazz: method.typeParameters) { - if (first) { - first = false; - } else { - stream.print("%2D"); - } - stream.print(clazz.getName()); - if (clazz.isArray()) { - stream.print(":A"); - } - } - stream.print("%2D"); - } - - /** - * Emit an external link to Javadoc for a {@link PainlessField}. - * - * @param root name of the root uri variable - */ - private static void emitJavadocLink(PrintStream stream, String root, PainlessField field) { - stream.print("link:{"); - stream.print(root); - stream.print("-javadoc}/"); - stream.print(classUrlPath(field.javaField.getDeclaringClass())); - stream.print(".html#"); - stream.print(field.javaField.getName()); - } - - /** - * Pick the javadoc root for a {@link PainlessMethod}. - */ - private static String javadocRoot(PainlessMethod method) { - if (method.targetClass != method.javaMethod.getDeclaringClass()) { - return "painless"; - } - return javadocRoot(method.targetClass); - } - - /** - * Pick the javadoc root for a {@link PainlessField}. - */ - private static String javadocRoot(PainlessField field) { - return javadocRoot(field.javaField.getDeclaringClass()); - } - - /** - * Pick the javadoc root for a {@link Class}. - */ - private static String javadocRoot(Class clazz) { - String classPackage = clazz.getPackage().getName(); - if (classPackage.startsWith("java")) { - return "java8"; - } - if (classPackage.startsWith("org.elasticsearch.painless")) { - return "painless"; - } - if (classPackage.startsWith("org.elasticsearch")) { - return "elasticsearch"; - } - if (classPackage.startsWith("org.joda.time")) { - return "joda-time"; - } - if (classPackage.startsWith("org.apache.lucene")) { - return "lucene-core"; - } - throw new IllegalArgumentException("Unrecognized package: " + classPackage); - } - - private static void emitGeneratedWarning(PrintStream stream) { - stream.println("////"); - stream.println("Automatically generated by PainlessDocGenerator. Do not edit."); - stream.println("Rebuild by running `gradle generatePainlessApi`."); - stream.println("////"); - stream.println(); - } - - private static String classUrlPath(Class clazz) { - return clazz.getName().replace('.', '/').replace('$', '.'); - } -} diff --git a/modules/lang-painless/src/test/java/org/elasticsearch/painless/WhitelistLoaderTests.java b/modules/lang-painless/src/test/java/org/elasticsearch/painless/WhitelistLoaderTests.java new file mode 100644 index 0000000000000..7fe9dcce3bc09 --- /dev/null +++ b/modules/lang-painless/src/test/java/org/elasticsearch/painless/WhitelistLoaderTests.java @@ -0,0 +1,85 @@ +/* + * 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.painless; + +import org.elasticsearch.painless.spi.Whitelist; +import org.elasticsearch.painless.spi.WhitelistClass; +import org.elasticsearch.painless.spi.WhitelistLoader; +import org.elasticsearch.painless.spi.WhitelistMethod; +import org.elasticsearch.painless.spi.annotation.DeprecatedAnnotation; +import org.elasticsearch.painless.spi.annotation.NoImportAnnotation; +import org.elasticsearch.painless.spi.annotation.WhitelistAnnotationParser; + +import java.util.HashMap; +import java.util.Map; + +public class WhitelistLoaderTests extends ScriptTestCase { + + public void testAnnotations() { + Map parsers = new HashMap<>(WhitelistAnnotationParser.BASE_ANNOTATION_PARSERS); + parsers.put(AnnotationTestObject.TestAnnotation.NAME, AnnotationTestObject.TestAnnotationParser.INSTANCE); + Whitelist whitelist = WhitelistLoader.loadFromResourceFiles(Whitelist.class, parsers, "org.elasticsearch.painless.annotation"); + + assertEquals(1, whitelist.whitelistClasses.size()); + + WhitelistClass whitelistClass = whitelist.whitelistClasses.get(0); + + assertNotNull(whitelistClass.painlessAnnotations.get(NoImportAnnotation.class)); + assertEquals(1, whitelistClass.painlessAnnotations.size()); + assertEquals(3, whitelistClass.whitelistMethods.size()); + + int count = 0; + + for (WhitelistMethod whitelistMethod : whitelistClass.whitelistMethods) { + if ("deprecatedMethod".equals(whitelistMethod.methodName)) { + assertEquals("use another method", + ((DeprecatedAnnotation)whitelistMethod.painlessAnnotations.get(DeprecatedAnnotation.class)).getMessage()); + assertEquals(1, whitelistMethod.painlessAnnotations.size()); + ++count; + } + + if ("annotatedTestMethod".equals(whitelistMethod.methodName)) { + AnnotationTestObject.TestAnnotation ta = + ((AnnotationTestObject.TestAnnotation)whitelistMethod.painlessAnnotations.get( + AnnotationTestObject.TestAnnotation.class)); + assertEquals("one", ta.getOne()); + assertEquals("two", ta.getTwo()); + assertEquals("three", ta.getThree()); + assertEquals(1, whitelistMethod.painlessAnnotations.size()); + ++count; + } + + if ("annotatedMultipleMethod".equals(whitelistMethod.methodName)) { + assertEquals("test", + ((DeprecatedAnnotation)whitelistMethod.painlessAnnotations.get(DeprecatedAnnotation.class)).getMessage()); + AnnotationTestObject.TestAnnotation ta = + ((AnnotationTestObject.TestAnnotation)whitelistMethod.painlessAnnotations.get( + AnnotationTestObject.TestAnnotation.class)); + assertEquals("one", ta.getOne()); + assertEquals("two", ta.getTwo()); + assertEquals("three", ta.getThree()); + assertEquals(2, whitelistMethod.painlessAnnotations.size()); + ++count; + } + } + + assertEquals(3, count); + } +} diff --git a/modules/lang-painless/src/test/java/org/elasticsearch/painless/action/PainlessExecuteRequestTests.java b/modules/lang-painless/src/test/java/org/elasticsearch/painless/action/PainlessExecuteRequestTests.java index 51801c29095fd..d0996c4c2591c 100644 --- a/modules/lang-painless/src/test/java/org/elasticsearch/painless/action/PainlessExecuteRequestTests.java +++ b/modules/lang-painless/src/test/java/org/elasticsearch/painless/action/PainlessExecuteRequestTests.java @@ -21,6 +21,7 @@ import org.elasticsearch.common.bytes.BytesReference; import org.elasticsearch.common.io.stream.NamedWriteableRegistry; import org.elasticsearch.common.io.stream.StreamInput; +import org.elasticsearch.common.io.stream.Writeable; import org.elasticsearch.common.settings.Settings; import org.elasticsearch.common.xcontent.LoggingDeprecationHandler; import org.elasticsearch.common.xcontent.NamedXContentRegistry; @@ -35,7 +36,7 @@ import org.elasticsearch.script.ScriptContext; import org.elasticsearch.script.ScriptType; import org.elasticsearch.search.SearchModule; -import org.elasticsearch.test.AbstractStreamableTestCase; +import org.elasticsearch.test.AbstractWireSerializingTestCase; import java.io.IOException; import java.io.UncheckedIOException; @@ -43,7 +44,7 @@ import static org.hamcrest.Matchers.equalTo; -public class PainlessExecuteRequestTests extends AbstractStreamableTestCase { +public class PainlessExecuteRequestTests extends AbstractWireSerializingTestCase { // Testing XContent serialization manually here, because the xContentType field in ContextSetup determines // how the request needs to parse and the xcontent serialization framework randomizes that. The XContentType @@ -72,12 +73,12 @@ public final void testFromXContent() throws Exception { @Override protected NamedWriteableRegistry getNamedWriteableRegistry() { - return new NamedWriteableRegistry(new SearchModule(Settings.EMPTY, false, Collections.emptyList()).getNamedWriteables()); + return new NamedWriteableRegistry(new SearchModule(Settings.EMPTY, Collections.emptyList()).getNamedWriteables()); } @Override protected NamedXContentRegistry xContentRegistry() { - return new NamedXContentRegistry(new SearchModule(Settings.EMPTY, false, Collections.emptyList()).getNamedXContents()); + return new NamedXContentRegistry(new SearchModule(Settings.EMPTY, Collections.emptyList()).getNamedXContents()); } @Override @@ -89,8 +90,8 @@ protected PainlessExecuteAction.Request createTestInstance() { } @Override - protected PainlessExecuteAction.Request createBlankInstance() { - return new PainlessExecuteAction.Request(); + protected Writeable.Reader instanceReader() { + return PainlessExecuteAction.Request::new; } public void testValidate() { diff --git a/modules/lang-painless/src/test/resources/org/elasticsearch/painless/spi/org.elasticsearch.painless.annotation b/modules/lang-painless/src/test/resources/org/elasticsearch/painless/spi/org.elasticsearch.painless.annotation new file mode 100644 index 0000000000000..04e22b6dded96 --- /dev/null +++ b/modules/lang-painless/src/test/resources/org/elasticsearch/painless/spi/org.elasticsearch.painless.annotation @@ -0,0 +1,7 @@ +# whitelist for annotation tests + +class org.elasticsearch.painless.AnnotationTestObject @no_import { + void deprecatedMethod() @deprecated[message="use another method"] + void annotatedTestMethod() @test_annotation[one="one",two="two",three="three"] + void annotatedMultipleMethod() @test_annotation[one="one",two="two",three="three"] @deprecated[message="test"] +} \ No newline at end of file diff --git a/modules/lang-painless/src/test/resources/org/elasticsearch/painless/spi/org.elasticsearch.painless.test b/modules/lang-painless/src/test/resources/org/elasticsearch/painless/spi/org.elasticsearch.painless.test index 37faf50a70126..3ac8fa0ee0c91 100644 --- a/modules/lang-painless/src/test/resources/org/elasticsearch/painless/spi/org.elasticsearch.painless.test +++ b/modules/lang-painless/src/test/resources/org/elasticsearch/painless/spi/org.elasticsearch.painless.test @@ -2,7 +2,7 @@ class org.elasticsearch.painless.BindingsTests$BindingsTestScript { } -class org.elasticsearch.painless.FeatureTestObject no_import { +class org.elasticsearch.painless.FeatureTestObject @no_import { int z () (int,int) diff --git a/rest-api-spec/src/main/resources/rest-api-spec/api/scripts_painless_context.json b/modules/lang-painless/src/test/resources/rest-api-spec/api/scripts_painless_context.json similarity index 88% rename from rest-api-spec/src/main/resources/rest-api-spec/api/scripts_painless_context.json rename to modules/lang-painless/src/test/resources/rest-api-spec/api/scripts_painless_context.json index 1eb490e8ab59f..cb078ec4a00b8 100644 --- a/rest-api-spec/src/main/resources/rest-api-spec/api/scripts_painless_context.json +++ b/modules/lang-painless/src/test/resources/rest-api-spec/api/scripts_painless_context.json @@ -1,8 +1,8 @@ { "scripts_painless_context": { + "stability": "experimental", "methods": ["GET"], "url": { - "path": "/_scripts/painless/_context", "paths": ["/_scripts/painless/_context"], "parts": { }, diff --git a/modules/lang-painless/src/test/resources/rest-api-spec/test/painless/100_terms_agg.yml b/modules/lang-painless/src/test/resources/rest-api-spec/test/painless/100_terms_agg.yml new file mode 100644 index 0000000000000..774a5dd59b0a0 --- /dev/null +++ b/modules/lang-painless/src/test/resources/rest-api-spec/test/painless/100_terms_agg.yml @@ -0,0 +1,104 @@ +setup: + - do: + indices.create: + index: test_1 + body: + settings: + number_of_replicas: 0 + mappings: + properties: + str: + type: keyword + double: + type: double + number: + type: long + + - do: + cluster.health: + wait_for_status: green + + - do: + index: + index: test_1 + id: 1 + body: + str: "abc" + double: 1.0 + number: 1 + + - do: + index: + index: test_1 + id: 2 + body: + str: "abc" + double: 1.0 + number: 1 + + - do: + index: + index: test_1 + id: 3 + body: + str: "bcd" + double: 2.0 + number: 2 + + - do: + indices.refresh: {} + +--- +"String Value Script with doc notation": + + - do: + search: + rest_total_hits_as_int: true + body: { "size" : 0, "aggs" : { "str_terms" : { "terms" : { "field" : "str", "script": { "source": "return doc.str[0] + \"1\""} } } } } + + - match: { hits.total: 3 } + + - length: { aggregations.str_terms.buckets: 2 } + - match: { aggregations.str_terms.buckets.0.key: "abc1" } + - is_false: aggregations.str_terms.buckets.0.key_as_string + - match: { aggregations.str_terms.buckets.0.doc_count: 2 } + - match: { aggregations.str_terms.buckets.1.key: "bcd1" } + - is_false: aggregations.str_terms.buckets.1.key_as_string + - match: { aggregations.str_terms.buckets.1.doc_count: 1 } + +--- +"Long Value Script with doc notation": + + - do: + search: + rest_total_hits_as_int: true + body: { "size" : 0, "aggs" : { "long_terms" : { "terms" : { "field" : "number", "script": { "source": "return doc.number[0] + 1"} } } } } + + - match: { hits.total: 3 } + + - length: { aggregations.long_terms.buckets: 2 } + - match: { aggregations.long_terms.buckets.0.key: 2.0 } + - is_false: aggregations.long_terms.buckets.0.key_as_string + - match: { aggregations.long_terms.buckets.0.doc_count: 2 } + - match: { aggregations.long_terms.buckets.1.key: 3.0 } + - is_false: aggregations.long_terms.buckets.1.key_as_string + - match: { aggregations.long_terms.buckets.1.doc_count: 1 } + +--- +"Double Value Script with doc notation": + + - do: + search: + rest_total_hits_as_int: true + body: { "size" : 0, "aggs" : { "double_terms" : { "terms" : { "field" : "double", "script": { "source": "return doc.double[0] + 1"} } } } } + + - match: { hits.total: 3 } + + - length: { aggregations.double_terms.buckets: 2 } + - match: { aggregations.double_terms.buckets.0.key: 2.0 } + - is_false: aggregations.double_terms.buckets.0.key_as_string + - match: { aggregations.double_terms.buckets.0.doc_count: 2 } + - match: { aggregations.double_terms.buckets.1.key: 3.0 } + - is_false: aggregations.double_terms.buckets.1.key_as_string + - match: { aggregations.double_terms.buckets.1.doc_count: 1 } + diff --git a/modules/lang-painless/src/test/resources/rest-api-spec/test/painless/71_context_api.yml b/modules/lang-painless/src/test/resources/rest-api-spec/test/painless/71_context_api.yml index a185ede4db662..0413661fc586c 100644 --- a/modules/lang-painless/src/test/resources/rest-api-spec/test/painless/71_context_api.yml +++ b/modules/lang-painless/src/test/resources/rest-api-spec/test/painless/71_context_api.yml @@ -17,6 +17,6 @@ - match: { classes.6.methods.0.parameters.0 : java.lang.CharSequence } - match: { classes.6.methods.0.parameters.1 : int } - match: { classes.6.methods.0.parameters.2 : int } - - match: { imported_methods.0.name: dotProduct } - - match: { class_bindings.0.name: cosineSimilarity } + - match: { imported_methods.0.name: saturation } + - match: { class_bindings.0.name: decayDateExp } - match: { instance_bindings: [] } diff --git a/modules/mapper-extras/src/main/java/org/elasticsearch/index/mapper/MapperExtrasPlugin.java b/modules/mapper-extras/src/main/java/org/elasticsearch/index/mapper/MapperExtrasPlugin.java index 45a067d7994d2..569e20a14161f 100644 --- a/modules/mapper-extras/src/main/java/org/elasticsearch/index/mapper/MapperExtrasPlugin.java +++ b/modules/mapper-extras/src/main/java/org/elasticsearch/index/mapper/MapperExtrasPlugin.java @@ -39,8 +39,6 @@ public Map getMappers() { mappers.put(TokenCountFieldMapper.CONTENT_TYPE, new TokenCountFieldMapper.TypeParser()); mappers.put(RankFeatureFieldMapper.CONTENT_TYPE, new RankFeatureFieldMapper.TypeParser()); mappers.put(RankFeaturesFieldMapper.CONTENT_TYPE, new RankFeaturesFieldMapper.TypeParser()); - mappers.put(DenseVectorFieldMapper.CONTENT_TYPE, new DenseVectorFieldMapper.TypeParser()); - mappers.put(SparseVectorFieldMapper.CONTENT_TYPE, new SparseVectorFieldMapper.TypeParser()); mappers.put(SearchAsYouTypeFieldMapper.CONTENT_TYPE, new SearchAsYouTypeFieldMapper.TypeParser()); return Collections.unmodifiableMap(mappers); } diff --git a/modules/mapper-extras/src/main/resources/META-INF/services/org.elasticsearch.painless.spi.PainlessExtension b/modules/mapper-extras/src/main/resources/META-INF/services/org.elasticsearch.painless.spi.PainlessExtension deleted file mode 100644 index f4cc27a362e51..0000000000000 --- a/modules/mapper-extras/src/main/resources/META-INF/services/org.elasticsearch.painless.spi.PainlessExtension +++ /dev/null @@ -1 +0,0 @@ -org.elasticsearch.index.query.DocValuesWhitelistExtension \ No newline at end of file diff --git a/modules/mapper-extras/src/main/resources/org/elasticsearch/index/query/docvalues_whitelist.txt b/modules/mapper-extras/src/main/resources/org/elasticsearch/index/query/docvalues_whitelist.txt deleted file mode 100644 index 3a8989e20b020..0000000000000 --- a/modules/mapper-extras/src/main/resources/org/elasticsearch/index/query/docvalues_whitelist.txt +++ /dev/null @@ -1,32 +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. -# - -class org.elasticsearch.index.query.VectorScriptDocValues { -} -class org.elasticsearch.index.query.VectorScriptDocValues$DenseVectorScriptDocValues { -} -class org.elasticsearch.index.query.VectorScriptDocValues$SparseVectorScriptDocValues { -} - -static_import { - double cosineSimilarity(List, VectorScriptDocValues.DenseVectorScriptDocValues) bound_to org.elasticsearch.index.query.ScoreScriptUtils$CosineSimilarity - double dotProduct(List, VectorScriptDocValues.DenseVectorScriptDocValues) from_class org.elasticsearch.index.query.ScoreScriptUtils - double dotProductSparse(Map, VectorScriptDocValues.SparseVectorScriptDocValues) bound_to org.elasticsearch.index.query.ScoreScriptUtils$DotProductSparse - double cosineSimilaritySparse(Map, VectorScriptDocValues.SparseVectorScriptDocValues) bound_to org.elasticsearch.index.query.ScoreScriptUtils$CosineSimilaritySparse -} \ No newline at end of file diff --git a/modules/mapper-extras/src/test/java/org/elasticsearch/index/mapper/DenseVectorFieldMapperTests.java b/modules/mapper-extras/src/test/java/org/elasticsearch/index/mapper/DenseVectorFieldMapperTests.java deleted file mode 100644 index cf6fc99657756..0000000000000 --- a/modules/mapper-extras/src/test/java/org/elasticsearch/index/mapper/DenseVectorFieldMapperTests.java +++ /dev/null @@ -1,105 +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.mapper; - -import org.apache.lucene.document.BinaryDocValuesField; -import org.apache.lucene.index.IndexableField; -import org.apache.lucene.util.BytesRef; -import org.elasticsearch.common.Strings; -import org.elasticsearch.common.bytes.BytesReference; -import org.elasticsearch.common.compress.CompressedXContent; -import org.elasticsearch.common.xcontent.XContentFactory; -import org.elasticsearch.common.xcontent.XContentType; -import org.elasticsearch.index.IndexService; -import org.elasticsearch.plugins.Plugin; -import org.elasticsearch.test.ESSingleNodeTestCase; -import org.junit.Before; - -import java.io.IOException; -import java.util.Collection; - -import static org.hamcrest.Matchers.containsString; -import static org.hamcrest.Matchers.instanceOf; - -public class DenseVectorFieldMapperTests extends ESSingleNodeTestCase { - private DocumentMapper mapper; - - @Before - public void setUpMapper() throws Exception { - IndexService indexService = createIndex("test-index"); - DocumentMapperParser parser = indexService.mapperService().documentMapperParser(); - String mapping = Strings.toString(XContentFactory.jsonBuilder() - .startObject() - .startObject("_doc") - .startObject("properties") - .startObject("my-dense-vector").field("type", "dense_vector") - .endObject() - .endObject() - .endObject() - .endObject()); - mapper = parser.parse("_doc", new CompressedXContent(mapping)); - } - - @Override - protected Collection> getPlugins() { - return pluginList(MapperExtrasPlugin.class); - } - - public void testDefaults() throws Exception { - float[] expectedArray = {-12.1f, 100.7f, -4}; - ParsedDocument doc1 = mapper.parse(new SourceToParse("test-index", "_doc", "1", BytesReference - .bytes(XContentFactory.jsonBuilder() - .startObject() - .startArray("my-dense-vector").value(expectedArray[0]).value(expectedArray[1]).value(expectedArray[2]).endArray() - .endObject()), - XContentType.JSON)); - IndexableField[] fields = doc1.rootDoc().getFields("my-dense-vector"); - assertEquals(1, fields.length); - assertThat(fields[0], instanceOf(BinaryDocValuesField.class)); - - // assert that after decoding the indexed value is equal to expected - BytesRef vectorBR = ((BinaryDocValuesField) fields[0]).binaryValue(); - float[] decodedValues = VectorEncoderDecoder.decodeDenseVector(vectorBR); - assertArrayEquals( - "Decoded dense vector values is not equal to the indexed one.", - expectedArray, - decodedValues, - 0.001f - ); - } - - public void testDimensionLimit() throws IOException { - float[] validVector = new float[DenseVectorFieldMapper.MAX_DIMS_COUNT]; - BytesReference validDoc = BytesReference.bytes( - XContentFactory.jsonBuilder().startObject() - .array("my-dense-vector", validVector) - .endObject()); - mapper.parse(new SourceToParse("test-index", "_doc", "1", validDoc, XContentType.JSON)); - - float[] invalidVector = new float[DenseVectorFieldMapper.MAX_DIMS_COUNT + 1]; - BytesReference invalidDoc = BytesReference.bytes( - XContentFactory.jsonBuilder().startObject() - .array("my-dense-vector", invalidVector) - .endObject()); - MapperParsingException e = expectThrows(MapperParsingException.class, () -> mapper.parse( - new SourceToParse("test-index", "_doc", "1", invalidDoc, XContentType.JSON))); - assertThat(e.getDetailedMessage(), containsString("has exceeded the maximum allowed number of dimensions")); - } -} diff --git a/modules/mapper-extras/src/test/java/org/elasticsearch/index/mapper/TokenCountFieldMapperIntegrationIT.java b/modules/mapper-extras/src/test/java/org/elasticsearch/index/mapper/TokenCountFieldMapperIntegrationIT.java index b5348ac91465c..d05be9ba2d7f9 100644 --- a/modules/mapper-extras/src/test/java/org/elasticsearch/index/mapper/TokenCountFieldMapperIntegrationIT.java +++ b/modules/mapper-extras/src/test/java/org/elasticsearch/index/mapper/TokenCountFieldMapperIntegrationIT.java @@ -21,6 +21,7 @@ import com.carrotsearch.randomizedtesting.annotations.Name; import com.carrotsearch.randomizedtesting.annotations.ParametersFactory; + import org.elasticsearch.ElasticsearchException; import org.elasticsearch.action.DocWriteResponse; import org.elasticsearch.action.bulk.BulkResponse; @@ -180,7 +181,7 @@ private SearchRequestBuilder searchByNumericRange(int low, int high) { } private SearchRequestBuilder prepareSearch() { - SearchRequestBuilder request = client().prepareSearch("test").setTypes("test"); + SearchRequestBuilder request = client().prepareSearch("test"); request.addStoredField("foo.token_count"); request.addStoredField("foo.token_count_without_position_increments"); if (loadCountedFields) { diff --git a/modules/parent-join/build.gradle b/modules/parent-join/build.gradle index 67bcc9d54e8e7..756a65a371a9b 100644 --- a/modules/parent-join/build.gradle +++ b/modules/parent-join/build.gradle @@ -20,5 +20,4 @@ esplugin { description 'This module adds the support parent-child queries and aggregations' classname 'org.elasticsearch.join.ParentJoinPlugin' - hasClientJar = true } diff --git a/modules/parent-join/src/main/java/org/elasticsearch/join/aggregations/ChildrenAggregationBuilder.java b/modules/parent-join/src/main/java/org/elasticsearch/join/aggregations/ChildrenAggregationBuilder.java index e0fac88cb531c..c10b025d2759f 100644 --- a/modules/parent-join/src/main/java/org/elasticsearch/join/aggregations/ChildrenAggregationBuilder.java +++ b/modules/parent-join/src/main/java/org/elasticsearch/join/aggregations/ChildrenAggregationBuilder.java @@ -95,9 +95,9 @@ protected void innerWriteTo(StreamOutput out) throws IOException { } @Override - protected ValuesSourceAggregatorFactory innerBuild(SearchContext context, + protected ValuesSourceAggregatorFactory innerBuild(SearchContext context, ValuesSourceConfig config, - AggregatorFactory parent, + AggregatorFactory parent, Builder subFactoriesBuilder) throws IOException { return new ChildrenAggregatorFactory(name, config, childFilter, parentFilter, context, parent, subFactoriesBuilder, metaData); @@ -159,12 +159,15 @@ public static ChildrenAggregationBuilder parse(String aggregationName, XContentP } @Override - protected int innerHashCode() { - return Objects.hash(childType); + public int hashCode() { + return Objects.hash(super.hashCode(), childType); } @Override - protected boolean innerEquals(Object obj) { + public boolean equals(Object obj) { + if (this == obj) return true; + if (obj == null || getClass() != obj.getClass()) return false; + if (super.equals(obj) == false) return false; ChildrenAggregationBuilder other = (ChildrenAggregationBuilder) obj; return Objects.equals(childType, other.childType); } diff --git a/modules/parent-join/src/main/java/org/elasticsearch/join/aggregations/ChildrenAggregatorFactory.java b/modules/parent-join/src/main/java/org/elasticsearch/join/aggregations/ChildrenAggregatorFactory.java index 1f466f1020d18..a8cfe62edd9ed 100644 --- a/modules/parent-join/src/main/java/org/elasticsearch/join/aggregations/ChildrenAggregatorFactory.java +++ b/modules/parent-join/src/main/java/org/elasticsearch/join/aggregations/ChildrenAggregatorFactory.java @@ -35,7 +35,7 @@ import java.util.List; import java.util.Map; -public class ChildrenAggregatorFactory extends ValuesSourceAggregatorFactory { +public class ChildrenAggregatorFactory extends ValuesSourceAggregatorFactory { private final Query parentFilter; private final Query childFilter; @@ -45,7 +45,7 @@ public ChildrenAggregatorFactory(String name, Query childFilter, Query parentFilter, SearchContext context, - AggregatorFactory parent, + AggregatorFactory parent, AggregatorFactories.Builder subFactoriesBuilder, Map metaData) throws IOException { super(name, config, context, parent, subFactoriesBuilder, metaData); diff --git a/modules/parent-join/src/main/java/org/elasticsearch/join/aggregations/ParentAggregationBuilder.java b/modules/parent-join/src/main/java/org/elasticsearch/join/aggregations/ParentAggregationBuilder.java index 495a5c0f9ad3f..83575d6d527e6 100644 --- a/modules/parent-join/src/main/java/org/elasticsearch/join/aggregations/ParentAggregationBuilder.java +++ b/modules/parent-join/src/main/java/org/elasticsearch/join/aggregations/ParentAggregationBuilder.java @@ -95,9 +95,9 @@ protected void innerWriteTo(StreamOutput out) throws IOException { } @Override - protected ValuesSourceAggregatorFactory innerBuild(SearchContext context, + protected ValuesSourceAggregatorFactory innerBuild(SearchContext context, ValuesSourceConfig config, - AggregatorFactory parent, + AggregatorFactory parent, Builder subFactoriesBuilder) throws IOException { return new ParentAggregatorFactory(name, config, childFilter, parentFilter, context, parent, subFactoriesBuilder, metaData); @@ -159,12 +159,15 @@ public static ParentAggregationBuilder parse(String aggregationName, XContentPar } @Override - protected int innerHashCode() { - return Objects.hash(childType); + public int hashCode() { + return Objects.hash(super.hashCode(), childType); } @Override - protected boolean innerEquals(Object obj) { + public boolean equals(Object obj) { + if (this == obj) return true; + if (obj == null || getClass() != obj.getClass()) return false; + if (super.equals(obj) == false) return false; ParentAggregationBuilder other = (ParentAggregationBuilder) obj; return Objects.equals(childType, other.childType); } diff --git a/modules/parent-join/src/main/java/org/elasticsearch/join/aggregations/ParentAggregatorFactory.java b/modules/parent-join/src/main/java/org/elasticsearch/join/aggregations/ParentAggregatorFactory.java index 2ae3da7c47af3..fd87744dcf235 100644 --- a/modules/parent-join/src/main/java/org/elasticsearch/join/aggregations/ParentAggregatorFactory.java +++ b/modules/parent-join/src/main/java/org/elasticsearch/join/aggregations/ParentAggregatorFactory.java @@ -35,7 +35,7 @@ import java.util.List; import java.util.Map; -public class ParentAggregatorFactory extends ValuesSourceAggregatorFactory { +public class ParentAggregatorFactory extends ValuesSourceAggregatorFactory { private final Query parentFilter; private final Query childFilter; @@ -45,7 +45,7 @@ public ParentAggregatorFactory(String name, Query childFilter, Query parentFilter, SearchContext context, - AggregatorFactory parent, + AggregatorFactory parent, AggregatorFactories.Builder subFactoriesBuilder, Map metaData) throws IOException { super(name, config, context, parent, subFactoriesBuilder, metaData); diff --git a/modules/parent-join/src/test/java/org/elasticsearch/join/query/HasChildQueryBuilderTests.java b/modules/parent-join/src/test/java/org/elasticsearch/join/query/HasChildQueryBuilderTests.java index 8a50e3a734977..925f85eb684f0 100644 --- a/modules/parent-join/src/test/java/org/elasticsearch/join/query/HasChildQueryBuilderTests.java +++ b/modules/parent-join/src/test/java/org/elasticsearch/join/query/HasChildQueryBuilderTests.java @@ -20,6 +20,7 @@ package org.elasticsearch.join.query; import com.carrotsearch.randomizedtesting.generators.RandomPicks; + import org.apache.lucene.index.Term; import org.apache.lucene.search.BooleanClause; import org.apache.lucene.search.BooleanQuery; @@ -280,13 +281,9 @@ public void testFromJson() throws IOException { } public void testToQueryInnerQueryType() throws IOException { - String[] searchTypes = new String[]{TYPE}; QueryShardContext shardContext = createShardContext(); - shardContext.setTypes(searchTypes); HasChildQueryBuilder hasChildQueryBuilder = hasChildQuery(CHILD_DOC, new IdsQueryBuilder().addIds("id"), ScoreMode.None); Query query = hasChildQueryBuilder.toQuery(shardContext); - //verify that the context types are still the same as the ones we previously set - assertThat(shardContext.getTypes(), equalTo(searchTypes)); assertLateParsingQuery(query, CHILD_DOC, "id"); } diff --git a/modules/parent-join/src/test/java/org/elasticsearch/join/query/HasParentQueryBuilderTests.java b/modules/parent-join/src/test/java/org/elasticsearch/join/query/HasParentQueryBuilderTests.java index ea77ad80799ba..a634690dcac85 100644 --- a/modules/parent-join/src/test/java/org/elasticsearch/join/query/HasParentQueryBuilderTests.java +++ b/modules/parent-join/src/test/java/org/elasticsearch/join/query/HasParentQueryBuilderTests.java @@ -192,14 +192,10 @@ public void testIllegalValues() throws IOException { } public void testToQueryInnerQueryType() throws IOException { - String[] searchTypes = new String[]{TYPE}; QueryShardContext shardContext = createShardContext(); - shardContext.setTypes(searchTypes); HasParentQueryBuilder hasParentQueryBuilder = new HasParentQueryBuilder(PARENT_DOC, new IdsQueryBuilder().addIds("id"), false); Query query = hasParentQueryBuilder.toQuery(shardContext); - //verify that the context types are still the same as the ones we previously set - assertThat(shardContext.getTypes(), equalTo(searchTypes)); HasChildQueryBuilderTests.assertLateParsingQuery(query, PARENT_DOC, "id"); } diff --git a/modules/percolator/build.gradle b/modules/percolator/build.gradle index 0826c91676dea..1296491748c35 100644 --- a/modules/percolator/build.gradle +++ b/modules/percolator/build.gradle @@ -20,7 +20,6 @@ esplugin { description 'Percolator module adds capability to index queries and query these queries by specifying documents' classname 'org.elasticsearch.percolator.PercolatorPlugin' - hasClientJar = true } dependencies { diff --git a/modules/percolator/src/main/java/org/elasticsearch/percolator/PercolateQueryBuilder.java b/modules/percolator/src/main/java/org/elasticsearch/percolator/PercolateQueryBuilder.java index 3021f5b31606e..151dd8b9e61ba 100644 --- a/modules/percolator/src/main/java/org/elasticsearch/percolator/PercolateQueryBuilder.java +++ b/modules/percolator/src/main/java/org/elasticsearch/percolator/PercolateQueryBuilder.java @@ -178,7 +178,7 @@ public PercolateQueryBuilder(String field, String documentType, List documentSupplier) { + protected PercolateQueryBuilder(String field, String documentType, Supplier documentSupplier) { if (field == null) { throw new IllegalArgumentException("[field] is a required argument"); } @@ -491,8 +491,12 @@ protected QueryBuilder doRewrite(QueryRewriteContext queryShardContext) { if (source == null) { return this; // not executed yet } else { - return new PercolateQueryBuilder(field, documentType, Collections.singletonList(source), - XContentHelper.xContentType(source)); + PercolateQueryBuilder rewritten = new PercolateQueryBuilder(field, documentType, + Collections.singletonList(source), XContentHelper.xContentType(source)); + if (name != null) { + rewritten.setName(name); + } + return rewritten; } } GetRequest getRequest; @@ -527,7 +531,12 @@ protected QueryBuilder doRewrite(QueryRewriteContext queryShardContext) { listener.onResponse(null); }, listener::onFailure)); }); - return new PercolateQueryBuilder(field, documentType, documentSupplier::get); + + PercolateQueryBuilder rewritten = new PercolateQueryBuilder(field, documentType, documentSupplier::get); + if (name != null) { + rewritten.setName(name); + } + return rewritten; } @Override @@ -626,6 +635,10 @@ XContentType getXContentType() { return documentXContentType; } + public String getQueryName() { + return name; + } + static IndexSearcher createMultiDocumentSearcher(Analyzer analyzer, Collection docs) { RAMDirectory ramDirectory = new RAMDirectory(); try (IndexWriter indexWriter = new IndexWriter(ramDirectory, new IndexWriterConfig(analyzer))) { diff --git a/modules/percolator/src/main/java/org/elasticsearch/percolator/QueryAnalyzer.java b/modules/percolator/src/main/java/org/elasticsearch/percolator/QueryAnalyzer.java index c245e2cb3a20b..d3c4bdedde7d2 100644 --- a/modules/percolator/src/main/java/org/elasticsearch/percolator/QueryAnalyzer.java +++ b/modules/percolator/src/main/java/org/elasticsearch/percolator/QueryAnalyzer.java @@ -22,7 +22,6 @@ import org.apache.lucene.index.PrefixCodedTerms; import org.apache.lucene.index.Term; import org.apache.lucene.queries.BlendedTermQuery; -import org.apache.lucene.queries.CommonTermsQuery; import org.apache.lucene.search.BooleanClause; import org.apache.lucene.search.BooleanClause.Occur; import org.apache.lucene.search.BooleanQuery; @@ -75,7 +74,6 @@ final class QueryAnalyzer { entry(BoostQuery.class, boostQuery()), entry(TermQuery.class, termQuery()), entry(TermInSetQuery.class, termInSetQuery()), - entry(CommonTermsQuery.class, commonTermsQuery()), entry(BlendedTermQuery.class, blendedTermQuery()), entry(PhraseQuery.class, phraseQuery()), entry(MultiPhraseQuery.class, multiPhraseQuery()), @@ -185,13 +183,6 @@ private static BiFunction synonymQuery() { }; } - private static BiFunction commonTermsQuery() { - return (query, version) -> { - Set terms = ((CommonTermsQuery) query).getTerms().stream().map(QueryExtraction::new).collect(toSet()); - return new Result(false, terms, Math.min(1, terms.size())); - }; - } - private static BiFunction blendedTermQuery() { return (query, version) -> { Set terms = ((BlendedTermQuery) query).getTerms().stream().map(QueryExtraction::new).collect(toSet()); diff --git a/modules/percolator/src/test/java/org/elasticsearch/percolator/CandidateQueryTests.java b/modules/percolator/src/test/java/org/elasticsearch/percolator/CandidateQueryTests.java index b191dd948c574..e487037afaea7 100644 --- a/modules/percolator/src/test/java/org/elasticsearch/percolator/CandidateQueryTests.java +++ b/modules/percolator/src/test/java/org/elasticsearch/percolator/CandidateQueryTests.java @@ -46,7 +46,6 @@ import org.apache.lucene.index.TermsEnum; import org.apache.lucene.index.memory.MemoryIndex; import org.apache.lucene.queries.BlendedTermQuery; -import org.apache.lucene.queries.CommonTermsQuery; import org.apache.lucene.search.BooleanClause.Occur; import org.apache.lucene.search.BooleanQuery; import org.apache.lucene.search.ConstantScoreQuery; @@ -84,8 +83,8 @@ import org.elasticsearch.common.bytes.BytesArray; import org.elasticsearch.common.bytes.BytesReference; import org.elasticsearch.common.compress.CompressedXContent; -import org.elasticsearch.common.lucene.search.function.FunctionScoreQuery; import org.elasticsearch.common.geo.ShapeRelation; +import org.elasticsearch.common.lucene.search.function.FunctionScoreQuery; import org.elasticsearch.common.settings.Settings; import org.elasticsearch.common.xcontent.XContentFactory; import org.elasticsearch.index.IndexService; @@ -530,12 +529,6 @@ public void testDuelIdBased() throws Exception { public void testDuelSpecificQueries() throws Exception { List documents = new ArrayList<>(); - CommonTermsQuery commonTermsQuery = new CommonTermsQuery(Occur.SHOULD, Occur.SHOULD, 128); - commonTermsQuery.add(new Term("field", "quick")); - commonTermsQuery.add(new Term("field", "brown")); - commonTermsQuery.add(new Term("field", "fox")); - addQuery(commonTermsQuery, documents); - BlendedTermQuery blendedTermQuery = BlendedTermQuery.dismaxBlendedQuery(new Term[]{new Term("field", "quick"), new Term("field", "brown"), new Term("field", "fox")}, 1.0f); addQuery(blendedTermQuery, documents); diff --git a/modules/percolator/src/test/java/org/elasticsearch/percolator/PercolateQueryBuilderTests.java b/modules/percolator/src/test/java/org/elasticsearch/percolator/PercolateQueryBuilderTests.java index 6053a92b54a20..a86f93ce40549 100644 --- a/modules/percolator/src/test/java/org/elasticsearch/percolator/PercolateQueryBuilderTests.java +++ b/modules/percolator/src/test/java/org/elasticsearch/percolator/PercolateQueryBuilderTests.java @@ -54,6 +54,7 @@ import java.util.List; import java.util.Map; import java.util.Set; +import java.util.function.Supplier; import static org.elasticsearch.index.seqno.SequenceNumbers.UNASSIGNED_SEQ_NO; import static org.hamcrest.Matchers.equalTo; @@ -295,9 +296,17 @@ private static BytesReference randomSource(Set usedFields) { } } + /** + * Test that this query is never cacheable + */ @Override - protected boolean isCacheable(PercolateQueryBuilder queryBuilder) { - return false; + public void testCacheability() throws IOException { + PercolateQueryBuilder queryBuilder = createTestQueryBuilder(); + QueryShardContext context = createShardContext(); + assert context.isCacheable(); + QueryBuilder rewritten = rewriteQuery(queryBuilder, new QueryShardContext(context)); + assertNotNull(rewritten.toQuery(context)); + assertFalse("query should not be cacheable: " + queryBuilder.toString(), context.isCacheable()); } @Override @@ -331,4 +340,29 @@ public void testFieldAlias() throws IOException { assertEquals(query.getVerifiedMatchesQuery(), aliasQuery.getVerifiedMatchesQuery()); } + public void testSettingNameWhileRewriting() { + String testName = "name1"; + QueryShardContext shardContext = createShardContext(); + PercolateQueryBuilder percolateQueryBuilder = doCreateTestQueryBuilder(true); + percolateQueryBuilder.setName(testName); + + QueryBuilder rewrittenQueryBuilder = percolateQueryBuilder.doRewrite(shardContext); + + assertEquals(testName, ((PercolateQueryBuilder) rewrittenQueryBuilder).getQueryName()); + assertNotEquals(rewrittenQueryBuilder, percolateQueryBuilder); + } + + public void testSettingNameWhileRewritingWhenDocumentSupplierAndSourceNotNull() { + Supplier supplier = () -> new BytesArray("{\"test\": \"test\"}"); + String testName = "name1"; + QueryShardContext shardContext = createShardContext(); + PercolateQueryBuilder percolateQueryBuilder = new PercolateQueryBuilder(queryField, null, supplier); + percolateQueryBuilder.setName(testName); + + QueryBuilder rewrittenQueryBuilder = percolateQueryBuilder.doRewrite(shardContext); + + assertEquals(testName, ((PercolateQueryBuilder) rewrittenQueryBuilder).getQueryName()); + assertNotEquals(rewrittenQueryBuilder, percolateQueryBuilder); + } + } diff --git a/modules/percolator/src/test/java/org/elasticsearch/percolator/PercolatorQuerySearchIT.java b/modules/percolator/src/test/java/org/elasticsearch/percolator/PercolatorQuerySearchIT.java index ee31a81ae168a..57a6ca15ac593 100644 --- a/modules/percolator/src/test/java/org/elasticsearch/percolator/PercolatorQuerySearchIT.java +++ b/modules/percolator/src/test/java/org/elasticsearch/percolator/PercolatorQuerySearchIT.java @@ -44,7 +44,6 @@ import static org.elasticsearch.common.xcontent.XContentFactory.smileBuilder; import static org.elasticsearch.common.xcontent.XContentFactory.yamlBuilder; import static org.elasticsearch.index.query.QueryBuilders.boolQuery; -import static org.elasticsearch.index.query.QueryBuilders.commonTermsQuery; import static org.elasticsearch.index.query.QueryBuilders.geoBoundingBoxQuery; import static org.elasticsearch.index.query.QueryBuilders.geoDistanceQuery; import static org.elasticsearch.index.query.QueryBuilders.geoPolygonQuery; @@ -356,13 +355,10 @@ public void testPercolatorSpecificQueries() throws Exception { ); client().prepareIndex("test", "type", "1") - .setSource(jsonBuilder().startObject().field("query", commonTermsQuery("field1", "quick brown fox")).endObject()) - .get(); - client().prepareIndex("test", "type", "2") .setSource(jsonBuilder().startObject().field("query", multiMatchQuery("quick brown fox", "field1", "field2") .type(MultiMatchQueryBuilder.Type.CROSS_FIELDS)).endObject()) .get(); - client().prepareIndex("test", "type", "3") + client().prepareIndex("test", "type", "2") .setSource(jsonBuilder().startObject().field("query", spanNearQuery(spanTermQuery("field1", "quick"), 0) .addClause(spanTermQuery("field1", "brown")) @@ -372,7 +368,7 @@ public void testPercolatorSpecificQueries() throws Exception { .get(); client().admin().indices().prepareRefresh().get(); - client().prepareIndex("test", "type", "4") + client().prepareIndex("test", "type", "3") .setSource(jsonBuilder().startObject().field("query", spanNotQuery( spanNearQuery(spanTermQuery("field1", "quick"), 0) @@ -387,7 +383,7 @@ public void testPercolatorSpecificQueries() throws Exception { .get(); // doesn't match - client().prepareIndex("test", "type", "5") + client().prepareIndex("test", "type", "4") .setSource(jsonBuilder().startObject().field("query", spanNotQuery( spanNearQuery(spanTermQuery("field1", "quick"), 0) @@ -410,15 +406,13 @@ public void testPercolatorSpecificQueries() throws Exception { .setQuery(new PercolateQueryBuilder("query", source, XContentType.JSON)) .addSort("_id", SortOrder.ASC) .get(); - assertHitCount(response, 4); + assertHitCount(response, 3); assertThat(response.getHits().getAt(0).getId(), equalTo("1")); assertThat(response.getHits().getAt(0).getScore(), equalTo(Float.NaN)); assertThat(response.getHits().getAt(1).getId(), equalTo("2")); assertThat(response.getHits().getAt(1).getScore(), equalTo(Float.NaN)); assertThat(response.getHits().getAt(2).getId(), equalTo("3")); assertThat(response.getHits().getAt(2).getScore(), equalTo(Float.NaN)); - assertThat(response.getHits().getAt(3).getId(), equalTo("4")); - assertThat(response.getHits().getAt(3).getScore(), equalTo(Float.NaN)); } public void testPercolatorQueryWithHighlighting() throws Exception { diff --git a/modules/percolator/src/test/java/org/elasticsearch/percolator/QueryAnalyzerTests.java b/modules/percolator/src/test/java/org/elasticsearch/percolator/QueryAnalyzerTests.java index c07467187f05f..358e9176e19b5 100644 --- a/modules/percolator/src/test/java/org/elasticsearch/percolator/QueryAnalyzerTests.java +++ b/modules/percolator/src/test/java/org/elasticsearch/percolator/QueryAnalyzerTests.java @@ -28,8 +28,8 @@ import org.apache.lucene.document.SortedNumericDocValuesField; import org.apache.lucene.index.Term; import org.apache.lucene.queries.BlendedTermQuery; -import org.apache.lucene.queries.CommonTermsQuery; import org.apache.lucene.search.BooleanClause; +import org.apache.lucene.search.BooleanClause.Occur; import org.apache.lucene.search.BooleanQuery; import org.apache.lucene.search.BoostQuery; import org.apache.lucene.search.ConstantScoreQuery; @@ -44,7 +44,6 @@ import org.apache.lucene.search.TermInSetQuery; import org.apache.lucene.search.TermQuery; import org.apache.lucene.search.TermRangeQuery; -import org.apache.lucene.search.BooleanClause.Occur; import org.apache.lucene.search.join.QueryBitSetProducer; import org.apache.lucene.search.join.ScoreMode; import org.apache.lucene.search.spans.SpanFirstQuery; @@ -520,27 +519,10 @@ public void testExtractQueryMetadata_boostQuery() { assertThat(terms.get(0).bytes(), equalTo(termQuery1.getTerm().bytes())); } - public void testExtractQueryMetadata_commonTermsQuery() { - CommonTermsQuery commonTermsQuery = new CommonTermsQuery(BooleanClause.Occur.SHOULD, BooleanClause.Occur.SHOULD, 100); - commonTermsQuery.add(new Term("_field", "_term1")); - commonTermsQuery.add(new Term("_field", "_term2")); - Result result = analyze(commonTermsQuery, Version.CURRENT); - assertThat(result.verified, is(false)); - assertThat(result.minimumShouldMatch, equalTo(1)); - List terms = new ArrayList<>(result.extractions); - terms.sort(Comparator.comparing(qt -> qt.term)); - assertThat(terms.size(), equalTo(2)); - assertThat(result.minimumShouldMatch, equalTo(1)); - assertThat(terms.get(0).field(), equalTo("_field")); - assertThat(terms.get(0).text(), equalTo("_term1")); - assertThat(terms.get(1).field(), equalTo("_field")); - assertThat(terms.get(1).text(), equalTo("_term2")); - } - public void testExtractQueryMetadata_blendedTermQuery() { Term[] termsArr = new Term[]{new Term("_field", "_term1"), new Term("_field", "_term2")}; - BlendedTermQuery commonTermsQuery = BlendedTermQuery.dismaxBlendedQuery(termsArr, 1.0f); - Result result = analyze(commonTermsQuery, Version.CURRENT); + BlendedTermQuery blendedTermQuery = BlendedTermQuery.dismaxBlendedQuery(termsArr, 1.0f); + Result result = analyze(blendedTermQuery, Version.CURRENT); assertThat(result.verified, is(true)); assertThat(result.minimumShouldMatch, equalTo(1)); List terms = new ArrayList<>(result.extractions); diff --git a/modules/percolator/src/test/java/org/elasticsearch/percolator/QueryBuilderStoreTests.java b/modules/percolator/src/test/java/org/elasticsearch/percolator/QueryBuilderStoreTests.java index 30cad734023d5..88c2a098deb21 100644 --- a/modules/percolator/src/test/java/org/elasticsearch/percolator/QueryBuilderStoreTests.java +++ b/modules/percolator/src/test/java/org/elasticsearch/percolator/QueryBuilderStoreTests.java @@ -55,13 +55,13 @@ public class QueryBuilderStoreTests extends ESTestCase { @Override protected NamedWriteableRegistry writableRegistry() { - SearchModule searchModule = new SearchModule(Settings.EMPTY, false, Collections.emptyList()); + SearchModule searchModule = new SearchModule(Settings.EMPTY, Collections.emptyList()); return new NamedWriteableRegistry(searchModule.getNamedWriteables()); } @Override protected NamedXContentRegistry xContentRegistry() { - SearchModule searchModule = new SearchModule(Settings.EMPTY, false, Collections.emptyList()); + SearchModule searchModule = new SearchModule(Settings.EMPTY, Collections.emptyList()); return new NamedXContentRegistry(searchModule.getNamedXContents()); } diff --git a/modules/rank-eval/build.gradle b/modules/rank-eval/build.gradle index a0fe3bef0a617..3e8e40ae89c40 100644 --- a/modules/rank-eval/build.gradle +++ b/modules/rank-eval/build.gradle @@ -20,7 +20,6 @@ esplugin { description 'The Rank Eval module adds APIs to evaluate ranking quality.' classname 'org.elasticsearch.index.rankeval.RankEvalPlugin' - hasClientJar = true } testClusters.integTest { diff --git a/modules/rank-eval/src/main/java/org/elasticsearch/index/rankeval/RankEvalAction.java b/modules/rank-eval/src/main/java/org/elasticsearch/index/rankeval/RankEvalAction.java index 54e89fe0e98b8..07de8c8a22cad 100644 --- a/modules/rank-eval/src/main/java/org/elasticsearch/index/rankeval/RankEvalAction.java +++ b/modules/rank-eval/src/main/java/org/elasticsearch/index/rankeval/RankEvalAction.java @@ -19,12 +19,12 @@ package org.elasticsearch.index.rankeval; -import org.elasticsearch.action.Action; +import org.elasticsearch.action.StreamableResponseActionType; /** - * Action for explaining evaluating search ranking results. + * ActionType for explaining evaluating search ranking results. */ -public class RankEvalAction extends Action { +public class RankEvalAction extends StreamableResponseActionType { public static final RankEvalAction INSTANCE = new RankEvalAction(); public static final String NAME = "indices:data/read/rank_eval"; diff --git a/modules/rank-eval/src/main/java/org/elasticsearch/index/rankeval/RankEvalRequestBuilder.java b/modules/rank-eval/src/main/java/org/elasticsearch/index/rankeval/RankEvalRequestBuilder.java index 4108a817f046e..1bfb576c37911 100644 --- a/modules/rank-eval/src/main/java/org/elasticsearch/index/rankeval/RankEvalRequestBuilder.java +++ b/modules/rank-eval/src/main/java/org/elasticsearch/index/rankeval/RankEvalRequestBuilder.java @@ -19,13 +19,13 @@ package org.elasticsearch.index.rankeval; -import org.elasticsearch.action.Action; +import org.elasticsearch.action.ActionType; import org.elasticsearch.action.ActionRequestBuilder; import org.elasticsearch.client.ElasticsearchClient; public class RankEvalRequestBuilder extends ActionRequestBuilder { - public RankEvalRequestBuilder(ElasticsearchClient client, Action action, + public RankEvalRequestBuilder(ElasticsearchClient client, ActionType action, RankEvalRequest request) { super(client, action, request); } diff --git a/modules/rank-eval/src/test/java/org/elasticsearch/index/rankeval/RatedRequestsTests.java b/modules/rank-eval/src/test/java/org/elasticsearch/index/rankeval/RatedRequestsTests.java index f77951dd58b6e..74683d8135ed7 100644 --- a/modules/rank-eval/src/test/java/org/elasticsearch/index/rankeval/RatedRequestsTests.java +++ b/modules/rank-eval/src/test/java/org/elasticsearch/index/rankeval/RatedRequestsTests.java @@ -64,7 +64,7 @@ public class RatedRequestsTests extends ESTestCase { @BeforeClass public static void init() { xContentRegistry = new NamedXContentRegistry( - Stream.of(new SearchModule(Settings.EMPTY, false, emptyList()).getNamedXContents().stream()).flatMap(Function.identity()) + Stream.of(new SearchModule(Settings.EMPTY, emptyList()).getNamedXContents().stream()).flatMap(Function.identity()) .collect(toList())); } diff --git a/modules/reindex/build.gradle b/modules/reindex/build.gradle index da184deedaa11..78846e2d81dd5 100644 --- a/modules/reindex/build.gradle +++ b/modules/reindex/build.gradle @@ -26,7 +26,6 @@ apply plugin: 'elasticsearch.test-with-dependencies' esplugin { description 'The Reindex module adds APIs to reindex from one index to another or update documents in place.' classname 'org.elasticsearch.index.reindex.ReindexPlugin' - hasClientJar = true } testClusters.integTest { @@ -55,8 +54,8 @@ test { } dependencies { - compile "org.elasticsearch.client:elasticsearch-rest-client:${version}" - compile "org.elasticsearch:elasticsearch-ssl-config:${version}" + compile project(":client:rest") + compile project(":libs:elasticsearch-ssl-config") // for http - testing reindex from remote testCompile project(path: ':modules:transport-netty4', configuration: 'runtime') // for parent/child testing @@ -77,7 +76,7 @@ forbiddenPatterns { exclude '**/*.p12' } -// Support for testing reindex-from-remote against old Elaticsearch versions +// Support for testing reindex-from-remote against old Elasticsearch versions configurations { oldesFixture es2 @@ -125,7 +124,7 @@ if (Os.isFamily(Os.FAMILY_WINDOWS)) { dependsOn unzip executable = new File(project.runtimeJavaHome, 'bin/java') env 'CLASSPATH', "${ -> project.configurations.oldesFixture.asPath }" - env 'JAVA_HOME', getJavaHome(it, 8) + env 'JAVA_HOME', "${ -> getJavaHome(it, 8)}" args 'oldes.OldElasticsearch', baseDir, unzip.temporaryDir, diff --git a/modules/reindex/src/main/java/org/elasticsearch/index/reindex/AbstractAsyncBulkByScrollAction.java b/modules/reindex/src/main/java/org/elasticsearch/index/reindex/AbstractAsyncBulkByScrollAction.java index a4b28e6c598d3..c6bfcf13eb113 100644 --- a/modules/reindex/src/main/java/org/elasticsearch/index/reindex/AbstractAsyncBulkByScrollAction.java +++ b/modules/reindex/src/main/java/org/elasticsearch/index/reindex/AbstractAsyncBulkByScrollAction.java @@ -75,7 +75,7 @@ import static java.util.Collections.unmodifiableList; import static org.elasticsearch.action.bulk.BackoffPolicy.exponentialBackoff; import static org.elasticsearch.common.unit.TimeValue.timeValueNanos; -import static org.elasticsearch.index.reindex.AbstractBulkByScrollRequest.SIZE_ALL_MATCHES; +import static org.elasticsearch.index.reindex.AbstractBulkByScrollRequest.MAX_DOCS_ALL_MATCHES; import static org.elasticsearch.rest.RestStatus.CONFLICT; import static org.elasticsearch.search.sort.SortBuilders.fieldSort; @@ -263,8 +263,8 @@ void onScrollResponse(TimeValue lastBatchStartTime, int lastBatchSize, Scrollabl return; } long total = response.getTotalHits(); - if (mainRequest.getSize() > 0) { - total = min(total, mainRequest.getSize()); + if (mainRequest.getMaxDocs() > 0) { + total = min(total, mainRequest.getMaxDocs()); } worker.setTotal(total); AbstractRunnable prepareBulkRequestRunnable = new AbstractRunnable() { @@ -304,9 +304,9 @@ void prepareBulkRequest(TimeValue thisBatchStartTime, ScrollableHitSource.Respon } worker.countBatch(); List hits = response.getHits(); - if (mainRequest.getSize() != SIZE_ALL_MATCHES) { - // Truncate the hits if we have more than the request size - long remaining = max(0, mainRequest.getSize() - worker.getSuccessfullyProcessed()); + if (mainRequest.getMaxDocs() != MAX_DOCS_ALL_MATCHES) { + // Truncate the hits if we have more than the request max docs + long remaining = max(0, mainRequest.getMaxDocs() - worker.getSuccessfullyProcessed()); if (remaining < hits.size()) { hits = hits.subList(0, (int) remaining); } @@ -395,7 +395,7 @@ void onBulkResponse(TimeValue thisBatchStartTime, BulkResponse response) { return; } - if (mainRequest.getSize() != SIZE_ALL_MATCHES && worker.getSuccessfullyProcessed() >= mainRequest.getSize()) { + if (mainRequest.getMaxDocs() != MAX_DOCS_ALL_MATCHES && worker.getSuccessfullyProcessed() >= mainRequest.getMaxDocs()) { // We've processed all the requested docs. refreshAndFinish(emptyList(), emptyList(), false); return; diff --git a/modules/reindex/src/main/java/org/elasticsearch/index/reindex/AbstractBaseReindexRestHandler.java b/modules/reindex/src/main/java/org/elasticsearch/index/reindex/AbstractBaseReindexRestHandler.java index bf333352dd55c..63451abb7ccd7 100644 --- a/modules/reindex/src/main/java/org/elasticsearch/index/reindex/AbstractBaseReindexRestHandler.java +++ b/modules/reindex/src/main/java/org/elasticsearch/index/reindex/AbstractBaseReindexRestHandler.java @@ -19,8 +19,8 @@ package org.elasticsearch.index.reindex; +import org.elasticsearch.action.ActionType; import org.elasticsearch.action.ActionRequestValidationException; -import org.elasticsearch.action.Action; import org.elasticsearch.action.support.ActiveShardCount; import org.elasticsearch.client.node.NodeClient; import org.elasticsearch.common.settings.Settings; @@ -38,7 +38,7 @@ public abstract class AbstractBaseReindexRestHandler< Request extends AbstractBulkByScrollRequest, - A extends Action + A extends ActionType > extends BaseRestHandler { private final A action; @@ -105,6 +105,11 @@ protected Request setCommonOptions(RestRequest restRequest, Request request) { if (requestsPerSecond != null) { request.setRequestsPerSecond(requestsPerSecond); } + + if (restRequest.hasParam("max_docs")) { + setMaxDocsValidateIdentical(request, restRequest.paramAsInt("max_docs", -1)); + } + return request; } @@ -170,4 +175,13 @@ public static Float parseRequestsPerSecond(RestRequest request) { } return requestsPerSecond; } + + static void setMaxDocsValidateIdentical(AbstractBulkByScrollRequest request, int maxDocs) { + if (request.getMaxDocs() != AbstractBulkByScrollRequest.MAX_DOCS_ALL_MATCHES && request.getMaxDocs() != maxDocs) { + throw new IllegalArgumentException("[max_docs] set to two different values [" + request.getMaxDocs() + "]" + + " and [" + maxDocs + "]"); + } else { + request.setMaxDocs(maxDocs); + } + } } diff --git a/modules/reindex/src/main/java/org/elasticsearch/index/reindex/AbstractBulkByQueryRestHandler.java b/modules/reindex/src/main/java/org/elasticsearch/index/reindex/AbstractBulkByQueryRestHandler.java index fab94494fe13d..3124d9cb0be0c 100644 --- a/modules/reindex/src/main/java/org/elasticsearch/index/reindex/AbstractBulkByQueryRestHandler.java +++ b/modules/reindex/src/main/java/org/elasticsearch/index/reindex/AbstractBulkByQueryRestHandler.java @@ -19,7 +19,7 @@ package org.elasticsearch.index.reindex; -import org.elasticsearch.action.Action; +import org.elasticsearch.action.ActionType; import org.elasticsearch.action.search.SearchRequest; import org.elasticsearch.common.bytes.BytesReference; import org.elasticsearch.common.settings.Settings; @@ -38,7 +38,7 @@ */ public abstract class AbstractBulkByQueryRestHandler< Request extends AbstractBulkByScrollRequest, - A extends Action> extends AbstractBaseReindexRestHandler { + A extends ActionType> extends AbstractBaseReindexRestHandler { protected AbstractBulkByQueryRestHandler(Settings settings, A action) { super(settings, action); @@ -52,7 +52,7 @@ protected void parseInternalRequest(Request internal, RestRequest restRequest, SearchRequest searchRequest = internal.getSearchRequest(); try (XContentParser parser = extractRequestSpecificFields(restRequest, bodyConsumers)) { - RestSearchAction.parseSearchRequest(searchRequest, restRequest, parser, internal::setSize); + RestSearchAction.parseSearchRequest(searchRequest, restRequest, parser, size -> failOnSizeSpecified()); } searchRequest.source().size(restRequest.paramAsInt("scroll_size", searchRequest.source().size())); @@ -94,4 +94,8 @@ private XContentParser extractRequestSpecificFields(RestRequest restRequest, parser.getDeprecationHandler(), BytesReference.bytes(builder.map(body)).streamInput()); } } + + private static void failOnSizeSpecified() { + throw new IllegalArgumentException("invalid parameter [size], use [max_docs] instead"); + } } diff --git a/modules/reindex/src/main/java/org/elasticsearch/index/reindex/BulkByScrollParallelizationHelper.java b/modules/reindex/src/main/java/org/elasticsearch/index/reindex/BulkByScrollParallelizationHelper.java index 2eca143393567..6df73414e81f3 100644 --- a/modules/reindex/src/main/java/org/elasticsearch/index/reindex/BulkByScrollParallelizationHelper.java +++ b/modules/reindex/src/main/java/org/elasticsearch/index/reindex/BulkByScrollParallelizationHelper.java @@ -19,7 +19,7 @@ package org.elasticsearch.index.reindex; -import org.elasticsearch.action.Action; +import org.elasticsearch.action.ActionType; import org.elasticsearch.action.ActionListener; import org.elasticsearch.action.admin.cluster.shards.ClusterSearchShardsRequest; import org.elasticsearch.action.admin.cluster.shards.ClusterSearchShardsResponse; @@ -61,7 +61,7 @@ private BulkByScrollParallelizationHelper() {} static > void startSlicedAction( Request request, BulkByScrollTask task, - Action action, + ActionType action, ActionListener listener, Client client, DiscoveryNode node, @@ -85,7 +85,7 @@ static > void startSlicedAc private static > void sliceConditionally( Request request, BulkByScrollTask task, - Action action, + ActionType action, ActionListener listener, Client client, DiscoveryNode node, @@ -112,13 +112,13 @@ private static int countSlicesBasedOnShards(ClusterSearchShardsResponse response (sum, term) -> sum + term )); Set counts = new HashSet<>(countsByIndex.values()); - int leastShards = Collections.min(counts); + int leastShards = counts.isEmpty() ? 1 : Collections.min(counts); return Math.min(leastShards, AUTO_SLICE_CEILING); } private static > void sendSubRequests( Client client, - Action action, + ActionType action, String localNodeId, BulkByScrollTask task, Request request, diff --git a/modules/reindex/src/main/java/org/elasticsearch/index/reindex/RestDeleteByQueryAction.java b/modules/reindex/src/main/java/org/elasticsearch/index/reindex/RestDeleteByQueryAction.java index 5d4d140131a14..74e600f32cd75 100644 --- a/modules/reindex/src/main/java/org/elasticsearch/index/reindex/RestDeleteByQueryAction.java +++ b/modules/reindex/src/main/java/org/elasticsearch/index/reindex/RestDeleteByQueryAction.java @@ -58,6 +58,7 @@ protected DeleteByQueryRequest buildRequest(RestRequest request) throws IOExcept Map> consumers = new HashMap<>(); consumers.put("conflicts", o -> internal.setConflicts((String) o)); + consumers.put("max_docs", s -> setMaxDocsValidateIdentical(internal, ((Number) s).intValue())); parseInternalRequest(internal, request, consumers); diff --git a/modules/reindex/src/main/java/org/elasticsearch/index/reindex/RestReindexAction.java b/modules/reindex/src/main/java/org/elasticsearch/index/reindex/RestReindexAction.java index ae9ca0be7ca65..e534717e775f5 100644 --- a/modules/reindex/src/main/java/org/elasticsearch/index/reindex/RestReindexAction.java +++ b/modules/reindex/src/main/java/org/elasticsearch/index/reindex/RestReindexAction.java @@ -19,89 +19,21 @@ package org.elasticsearch.index.reindex; -import org.apache.logging.log4j.LogManager; -import org.elasticsearch.action.index.IndexRequest; import org.elasticsearch.client.node.NodeClient; -import org.elasticsearch.common.ParseField; -import org.elasticsearch.common.Strings; -import org.elasticsearch.common.bytes.BytesReference; -import org.elasticsearch.common.logging.DeprecationLogger; import org.elasticsearch.common.settings.Settings; -import org.elasticsearch.common.unit.TimeValue; -import org.elasticsearch.common.xcontent.ObjectParser; -import org.elasticsearch.common.xcontent.ObjectParser.ValueType; -import org.elasticsearch.common.xcontent.ToXContent; -import org.elasticsearch.common.xcontent.XContentBuilder; -import org.elasticsearch.common.xcontent.XContentFactory; import org.elasticsearch.common.xcontent.XContentParser; -import org.elasticsearch.common.xcontent.json.JsonXContent; -import org.elasticsearch.index.VersionType; import org.elasticsearch.rest.RestController; import org.elasticsearch.rest.RestRequest; -import org.elasticsearch.script.Script; import java.io.IOException; -import java.io.InputStream; -import java.net.URI; -import java.net.URISyntaxException; -import java.util.List; -import java.util.Map; -import static java.util.Collections.emptyMap; -import static java.util.Objects.requireNonNull; import static org.elasticsearch.common.unit.TimeValue.parseTimeValue; -import static org.elasticsearch.index.query.QueryBuilders.matchAllQuery; import static org.elasticsearch.rest.RestRequest.Method.POST; /** * Expose reindex over rest. */ public class RestReindexAction extends AbstractBaseReindexRestHandler { - static final ObjectParser PARSER = new ObjectParser<>("reindex"); - static final String TYPES_DEPRECATION_MESSAGE = "[types removal] Specifying types in reindex requests is deprecated."; - private static final DeprecationLogger deprecationLogger = new DeprecationLogger(LogManager.getLogger(RestReindexAction.class)); - - static { - ObjectParser.Parser sourceParser = (parser, request, context) -> { - // Funky hack to work around Search not having a proper ObjectParser and us wanting to extract query if using remote. - Map source = parser.map(); - String[] indices = extractStringArray(source, "index"); - if (indices != null) { - request.getSearchRequest().indices(indices); - } - String[] types = extractStringArray(source, "type"); - if (types != null) { - deprecationLogger.deprecatedAndMaybeLog("reindex_with_types", TYPES_DEPRECATION_MESSAGE); - request.getSearchRequest().types(types); - } - request.setRemoteInfo(buildRemoteInfo(source)); - XContentBuilder builder = XContentFactory.contentBuilder(parser.contentType()); - builder.map(source); - try (InputStream stream = BytesReference.bytes(builder).streamInput(); - XContentParser innerParser = parser.contentType().xContent() - .createParser(parser.getXContentRegistry(), parser.getDeprecationHandler(), stream)) { - request.getSearchRequest().source().parseXContent(innerParser, false); - } - }; - - ObjectParser destParser = new ObjectParser<>("dest"); - destParser.declareString(IndexRequest::index, new ParseField("index")); - destParser.declareString((request, type) -> { - deprecationLogger.deprecatedAndMaybeLog("reindex_with_types", TYPES_DEPRECATION_MESSAGE); - request.type(type); - }, new ParseField("type")); - destParser.declareString(IndexRequest::routing, new ParseField("routing")); - destParser.declareString(IndexRequest::opType, new ParseField("op_type")); - destParser.declareString(IndexRequest::setPipeline, new ParseField("pipeline")); - destParser.declareString((s, i) -> s.versionType(VersionType.fromString(i)), new ParseField("version_type")); - - PARSER.declareField(sourceParser::parse, new ParseField("source"), ValueType.OBJECT); - PARSER.declareField((p, v, c) -> destParser.parse(p, v.getDestination(), c), new ParseField("dest"), ValueType.OBJECT); - PARSER.declareInt(ReindexRequest::setSize, new ParseField("size")); - PARSER.declareField((p, v, c) -> v.setScript(Script.parse(p)), new ParseField("script"), - ValueType.OBJECT); - PARSER.declareString(ReindexRequest::setConflicts, new ParseField("conflicts")); - } public RestReindexAction(Settings settings, RestController controller) { super(settings, ReindexAction.INSTANCE); @@ -124,123 +56,15 @@ protected ReindexRequest buildRequest(RestRequest request) throws IOException { throw new IllegalArgumentException("_reindex doesn't support [pipeline] as a query parameter. " + "Specify it in the [dest] object instead."); } - ReindexRequest internal = new ReindexRequest(); + + ReindexRequest internal; try (XContentParser parser = request.contentParser()) { - PARSER.parse(parser, internal, null); + internal = ReindexRequest.fromXContent(parser); } + if (request.hasParam("scroll")) { internal.setScroll(parseTimeValue(request.param("scroll"), "scroll")); } return internal; } - - static RemoteInfo buildRemoteInfo(Map source) throws IOException { - @SuppressWarnings("unchecked") - Map remote = (Map) source.remove("remote"); - if (remote == null) { - return null; - } - String username = extractString(remote, "username"); - String password = extractString(remote, "password"); - String hostInRequest = requireNonNull(extractString(remote, "host"), "[host] must be specified to reindex from a remote cluster"); - URI uri; - try { - uri = new URI(hostInRequest); - // URI has less stringent URL parsing than our code. We want to fail if all values are not provided. - if (uri.getPort() == -1) { - throw new URISyntaxException(hostInRequest, "The port was not defined in the [host]"); - } - } catch (URISyntaxException ex) { - throw new IllegalArgumentException("[host] must be of the form [scheme]://[host]:[port](/[pathPrefix])? but was [" - + hostInRequest + "]", ex); - } - - String scheme = uri.getScheme(); - String host = uri.getHost(); - int port = uri.getPort(); - - String pathPrefix = null; - if (uri.getPath().isEmpty() == false) { - pathPrefix = uri.getPath(); - } - - Map headers = extractStringStringMap(remote, "headers"); - TimeValue socketTimeout = extractTimeValue(remote, "socket_timeout", RemoteInfo.DEFAULT_SOCKET_TIMEOUT); - TimeValue connectTimeout = extractTimeValue(remote, "connect_timeout", RemoteInfo.DEFAULT_CONNECT_TIMEOUT); - if (false == remote.isEmpty()) { - throw new IllegalArgumentException( - "Unsupported fields in [remote]: [" + Strings.collectionToCommaDelimitedString(remote.keySet()) + "]"); - } - return new RemoteInfo(scheme, host, port, pathPrefix, queryForRemote(source), - username, password, headers, socketTimeout, connectTimeout); - } - - /** - * Yank a string array from a map. Emulates XContent's permissive String to - * String array conversions. - */ - private static String[] extractStringArray(Map source, String name) { - Object value = source.remove(name); - if (value == null) { - return null; - } - if (value instanceof List) { - @SuppressWarnings("unchecked") - List list = (List) value; - return list.toArray(new String[list.size()]); - } else if (value instanceof String) { - return new String[] {(String) value}; - } else { - throw new IllegalArgumentException("Expected [" + name + "] to be a list of a string but was [" + value + ']'); - } - } - - private static String extractString(Map source, String name) { - Object value = source.remove(name); - if (value == null) { - return null; - } - if (value instanceof String) { - return (String) value; - } - throw new IllegalArgumentException("Expected [" + name + "] to be a string but was [" + value + "]"); - } - - private static Map extractStringStringMap(Map source, String name) { - Object value = source.remove(name); - if (value == null) { - return emptyMap(); - } - if (false == value instanceof Map) { - throw new IllegalArgumentException("Expected [" + name + "] to be an object containing strings but was [" + value + "]"); - } - Map map = (Map) value; - for (Map.Entry entry : map.entrySet()) { - if (false == entry.getKey() instanceof String || false == entry.getValue() instanceof String) { - throw new IllegalArgumentException("Expected [" + name + "] to be an object containing strings but has [" + entry + "]"); - } - } - @SuppressWarnings("unchecked") // We just checked.... - Map safe = (Map) map; - return safe; - } - - private static TimeValue extractTimeValue(Map source, String name, TimeValue defaultValue) { - String string = extractString(source, name); - return string == null ? defaultValue : parseTimeValue(string, name); - } - - private static BytesReference queryForRemote(Map source) throws IOException { - XContentBuilder builder = JsonXContent.contentBuilder().prettyPrint(); - Object query = source.remove("query"); - if (query == null) { - return BytesReference.bytes(matchAllQuery().toXContent(builder, ToXContent.EMPTY_PARAMS)); - } - if (!(query instanceof Map)) { - throw new IllegalArgumentException("Expected [query] to be an object but was [" + query + "]"); - } - @SuppressWarnings("unchecked") - Map map = (Map) query; - return BytesReference.bytes(builder.map(map)); - } } diff --git a/modules/reindex/src/main/java/org/elasticsearch/index/reindex/RestUpdateByQueryAction.java b/modules/reindex/src/main/java/org/elasticsearch/index/reindex/RestUpdateByQueryAction.java index 121f8c372c59c..ea715aee7bd3a 100644 --- a/modules/reindex/src/main/java/org/elasticsearch/index/reindex/RestUpdateByQueryAction.java +++ b/modules/reindex/src/main/java/org/elasticsearch/index/reindex/RestUpdateByQueryAction.java @@ -66,6 +66,7 @@ protected UpdateByQueryRequest buildRequest(RestRequest request) throws IOExcept Map> consumers = new HashMap<>(); consumers.put("conflicts", o -> internal.setConflicts((String) o)); consumers.put("script", o -> internal.setScript(parseScript(o))); + consumers.put("max_docs", s -> setMaxDocsValidateIdentical(internal, ((Number) s).intValue())); parseInternalRequest(internal, request, consumers); diff --git a/modules/reindex/src/main/java/org/elasticsearch/index/reindex/RethrottleAction.java b/modules/reindex/src/main/java/org/elasticsearch/index/reindex/RethrottleAction.java index 3cb2c60c62373..cf04d6d856ddb 100644 --- a/modules/reindex/src/main/java/org/elasticsearch/index/reindex/RethrottleAction.java +++ b/modules/reindex/src/main/java/org/elasticsearch/index/reindex/RethrottleAction.java @@ -19,11 +19,11 @@ package org.elasticsearch.index.reindex; -import org.elasticsearch.action.Action; +import org.elasticsearch.action.ActionType; import org.elasticsearch.action.admin.cluster.node.tasks.list.ListTasksResponse; import org.elasticsearch.common.io.stream.Writeable; -public class RethrottleAction extends Action { +public class RethrottleAction extends ActionType { public static final RethrottleAction INSTANCE = new RethrottleAction(); public static final String NAME = "cluster:admin/reindex/rethrottle"; @@ -31,11 +31,6 @@ private RethrottleAction() { super(NAME); } - @Override - public ListTasksResponse newResponse() { - throw new UnsupportedOperationException("usage of Streamable is to be replaced by Writeable"); - } - @Override public Writeable.Reader getResponseReader() { return ListTasksResponse::new; diff --git a/modules/reindex/src/main/java/org/elasticsearch/index/reindex/RethrottleRequestBuilder.java b/modules/reindex/src/main/java/org/elasticsearch/index/reindex/RethrottleRequestBuilder.java index 25407e6dc93d5..648eb6e441b1a 100644 --- a/modules/reindex/src/main/java/org/elasticsearch/index/reindex/RethrottleRequestBuilder.java +++ b/modules/reindex/src/main/java/org/elasticsearch/index/reindex/RethrottleRequestBuilder.java @@ -19,7 +19,7 @@ package org.elasticsearch.index.reindex; -import org.elasticsearch.action.Action; +import org.elasticsearch.action.ActionType; import org.elasticsearch.action.admin.cluster.node.tasks.list.ListTasksResponse; import org.elasticsearch.action.support.tasks.TasksRequestBuilder; import org.elasticsearch.client.ElasticsearchClient; @@ -29,7 +29,7 @@ */ public class RethrottleRequestBuilder extends TasksRequestBuilder { public RethrottleRequestBuilder(ElasticsearchClient client, - Action action) { + ActionType action) { super(client, action, new RethrottleRequest()); } diff --git a/modules/reindex/src/main/java/org/elasticsearch/index/reindex/remote/RemoteRequestBuilders.java b/modules/reindex/src/main/java/org/elasticsearch/index/reindex/remote/RemoteRequestBuilders.java index 90332c7d55c9c..40c4ba757d176 100644 --- a/modules/reindex/src/main/java/org/elasticsearch/index/reindex/remote/RemoteRequestBuilders.java +++ b/modules/reindex/src/main/java/org/elasticsearch/index/reindex/remote/RemoteRequestBuilders.java @@ -59,7 +59,6 @@ static Request initialSearch(SearchRequest searchRequest, BytesReference query, // It is nasty to build paths with StringBuilder but we'll be careful.... StringBuilder path = new StringBuilder("/"); addIndices(path, searchRequest.indices()); - addTypes(path, searchRequest.types()); path.append("_search"); Request request = new Request("POST", path.toString()); @@ -75,14 +74,13 @@ static Request initialSearch(SearchRequest searchRequest, BytesReference query, request.addParameter("scroll", keepAlive.getStringRep()); } request.addParameter("size", Integer.toString(searchRequest.source().size())); - if (searchRequest.source().version() == null || searchRequest.source().version() == true) { - /* - * Passing `null` here just add the `version` request parameter - * without any value. This way of requesting the version works - * for all supported versions of Elasticsearch. - */ - request.addParameter("version", null); + + if (searchRequest.source().version() == null || searchRequest.source().version() == false) { + request.addParameter("version", Boolean.FALSE.toString()); + } else { + request.addParameter("version", Boolean.TRUE.toString()); } + if (searchRequest.source().sorts() != null) { boolean useScan = false; // Detect if we should use search_type=scan rather than a sort @@ -178,16 +176,6 @@ private static String encodeIndex(String s) { } } - private static void addTypes(StringBuilder path, String[] types) { - if (types == null || types.length == 0) { - return; - } - for (String indexOrType : types) { - checkIndexOrType("Type", indexOrType); - } - path.append(Strings.arrayToCommaDelimitedString(types)).append('/'); - } - private static void checkIndexOrType(String name, String indexOrType) { if (indexOrType.indexOf(',') >= 0) { throw new IllegalArgumentException(name + " containing [,] not supported but got [" + indexOrType + "]"); diff --git a/modules/reindex/src/main/java/org/elasticsearch/index/reindex/remote/RemoteResponseParsers.java b/modules/reindex/src/main/java/org/elasticsearch/index/reindex/remote/RemoteResponseParsers.java index 166eef1be369c..664941127b78e 100644 --- a/modules/reindex/src/main/java/org/elasticsearch/index/reindex/remote/RemoteResponseParsers.java +++ b/modules/reindex/src/main/java/org/elasticsearch/index/reindex/remote/RemoteResponseParsers.java @@ -143,7 +143,7 @@ class Fields { return new SearchFailure(reasonThrowable, index, shardId, nodeId); }); static { - SEARCH_FAILURE_PARSER.declareString(optionalConstructorArg(), new ParseField("index")); + SEARCH_FAILURE_PARSER.declareStringOrNull(optionalConstructorArg(), new ParseField("index")); SEARCH_FAILURE_PARSER.declareInt(optionalConstructorArg(), new ParseField("shard")); SEARCH_FAILURE_PARSER.declareString(optionalConstructorArg(), new ParseField("node")); SEARCH_FAILURE_PARSER.declareField(constructorArg(), (p, c) -> { diff --git a/modules/reindex/src/test/java/org/elasticsearch/client/documentation/ReindexDocumentationIT.java b/modules/reindex/src/test/java/org/elasticsearch/client/documentation/ReindexDocumentationIT.java index 7667fbbcf89f8..c2b907acde44e 100644 --- a/modules/reindex/src/test/java/org/elasticsearch/client/documentation/ReindexDocumentationIT.java +++ b/modules/reindex/src/test/java/org/elasticsearch/client/documentation/ReindexDocumentationIT.java @@ -116,7 +116,7 @@ public void testUpdateByQuery() { new UpdateByQueryRequestBuilder(client, UpdateByQueryAction.INSTANCE); updateByQuery.source("source_index") .filter(QueryBuilders.termQuery("level", "awesome")) - .size(1000) + .maxDocs(1000) .script(new Script(ScriptType.INLINE, "ctx._source.awesome = 'absolutely'", "painless", @@ -139,7 +139,7 @@ public void testUpdateByQuery() { UpdateByQueryRequestBuilder updateByQuery = new UpdateByQueryRequestBuilder(client, UpdateByQueryAction.INSTANCE); updateByQuery.source("source_index") - .size(100) + .maxDocs(100) .source() .addSort("cat", SortOrder.DESC); BulkByScrollResponse response = updateByQuery.get(); diff --git a/modules/reindex/src/test/java/org/elasticsearch/index/reindex/AsyncBulkByScrollActionTests.java b/modules/reindex/src/test/java/org/elasticsearch/index/reindex/AsyncBulkByScrollActionTests.java index bdedc65b7a6d3..3d28ce3bcbc96 100644 --- a/modules/reindex/src/test/java/org/elasticsearch/index/reindex/AsyncBulkByScrollActionTests.java +++ b/modules/reindex/src/test/java/org/elasticsearch/index/reindex/AsyncBulkByScrollActionTests.java @@ -23,7 +23,7 @@ import org.elasticsearch.ElasticsearchException; import org.elasticsearch.ExceptionsHelper; import org.elasticsearch.Version; -import org.elasticsearch.action.Action; +import org.elasticsearch.action.ActionType; import org.elasticsearch.action.ActionListener; import org.elasticsearch.action.ActionRequest; import org.elasticsearch.action.ActionResponse; @@ -761,7 +761,7 @@ private class MyMockClient extends FilterClient { @Override @SuppressWarnings("unchecked") protected - void doExecute(Action action, Request request, ActionListener listener) { + void doExecute(ActionType action, Request request, ActionListener listener) { if (false == expectedHeaders.equals(threadPool().getThreadContext().getHeaders())) { listener.onFailure( new RuntimeException("Expected " + expectedHeaders + " but got " + threadPool().getThreadContext().getHeaders())); diff --git a/modules/reindex/src/test/java/org/elasticsearch/index/reindex/CancelTests.java b/modules/reindex/src/test/java/org/elasticsearch/index/reindex/CancelTests.java index 6d6ae01f0626c..7eaa989b88a32 100644 --- a/modules/reindex/src/test/java/org/elasticsearch/index/reindex/CancelTests.java +++ b/modules/reindex/src/test/java/org/elasticsearch/index/reindex/CancelTests.java @@ -37,7 +37,6 @@ import org.elasticsearch.ingest.IngestTestPlugin; import org.elasticsearch.plugins.Plugin; import org.elasticsearch.tasks.TaskInfo; -import org.elasticsearch.test.junit.annotations.TestLogging; import org.hamcrest.Matcher; import org.junit.Before; @@ -60,7 +59,6 @@ * different cancellation places - that is the responsibility of AsyncBulkByScrollActionTests which have more precise control to * simulate failures but does not exercise important portion of the stack like transport and task management. */ -@TestLogging("org.elasticsearch.index.reindex:DEBUG,org.elasticsearch.action.bulk:DEBUG") public class CancelTests extends ReindexTestCase { protected static final String INDEX = "reindex-cancel-index"; @@ -216,7 +214,7 @@ public void testReindexCancel() throws Exception { assertThat(response, matcher().created(modified).reasonCancelled(equalTo("by user request"))); refresh("dest"); - assertHitCount(client().prepareSearch("dest").setTypes(TYPE).setSize(0).get(), modified); + assertHitCount(client().prepareSearch("dest").setSize(0).get(), modified); }, equalTo("reindex from [" + INDEX + "] to [dest][" + TYPE + "]")); } @@ -251,7 +249,7 @@ public void testReindexCancelWithWorkers() throws Exception { (response, total, modified) -> { assertThat(response, matcher().created(modified).reasonCancelled(equalTo("by user request")).slices(hasSize(5))); refresh("dest"); - assertHitCount(client().prepareSearch("dest").setTypes(TYPE).setSize(0).get(), modified); + assertHitCount(client().prepareSearch("dest").setSize(0).get(), modified); }, equalTo("reindex from [" + INDEX + "] to [dest][" + TYPE + "]")); } diff --git a/modules/reindex/src/test/java/org/elasticsearch/index/reindex/DeleteByQueryBasicTests.java b/modules/reindex/src/test/java/org/elasticsearch/index/reindex/DeleteByQueryBasicTests.java index 5bef735be5e6e..3118021f9056d 100644 --- a/modules/reindex/src/test/java/org/elasticsearch/index/reindex/DeleteByQueryBasicTests.java +++ b/modules/reindex/src/test/java/org/elasticsearch/index/reindex/DeleteByQueryBasicTests.java @@ -63,25 +63,25 @@ public void testBasics() throws Exception { client().prepareIndex("test", "test", "7").setSource("foo", "f") ); - assertHitCount(client().prepareSearch("test").setTypes("test").setSize(0).get(), 7); + assertHitCount(client().prepareSearch("test").setSize(0).get(), 7); // Deletes two docs that matches "foo:a" assertThat(deleteByQuery().source("test").filter(termQuery("foo", "a")).refresh(true).get(), matcher().deleted(2)); - assertHitCount(client().prepareSearch("test").setTypes("test").setSize(0).get(), 5); + assertHitCount(client().prepareSearch("test").setSize(0).get(), 5); // Deletes the two first docs with limit by size DeleteByQueryRequestBuilder request = deleteByQuery().source("test").filter(QueryBuilders.matchAllQuery()).size(2).refresh(true); request.source().addSort("foo.keyword", SortOrder.ASC); assertThat(request.get(), matcher().deleted(2)); - assertHitCount(client().prepareSearch("test").setTypes("test").setSize(0).get(), 3); + assertHitCount(client().prepareSearch("test").setSize(0).get(), 3); // Deletes but match no docs assertThat(deleteByQuery().source("test").filter(termQuery("foo", "no_match")).refresh(true).get(), matcher().deleted(0)); - assertHitCount(client().prepareSearch("test").setTypes("test").setSize(0).get(), 3); + assertHitCount(client().prepareSearch("test").setSize(0).get(), 3); // Deletes all remaining docs assertThat(deleteByQuery().source("test").filter(QueryBuilders.matchAllQuery()).refresh(true).get(), matcher().deleted(3)); - assertHitCount(client().prepareSearch("test").setTypes("test").setSize(0).get(), 0); + assertHitCount(client().prepareSearch("test").setSize(0).get(), 0); } public void testDeleteByQueryWithOneIndex() throws Exception { @@ -236,7 +236,7 @@ public void testSlices() throws Exception { client().prepareIndex("test", "test", "6").setSource("foo", "e"), client().prepareIndex("test", "test", "7").setSource("foo", "f") ); - assertHitCount(client().prepareSearch("test").setTypes("test").setSize(0).get(), 7); + assertHitCount(client().prepareSearch("test").setSize(0).get(), 7); int slices = randomSlices(); int expectedSlices = expectedSliceStatuses(slices, "test"); @@ -251,7 +251,7 @@ public void testSlices() throws Exception { matcher() .deleted(2) .slices(hasSize(expectedSlices))); - assertHitCount(client().prepareSearch("test").setTypes("test").setSize(0).get(), 5); + assertHitCount(client().prepareSearch("test").setSize(0).get(), 5); // Delete remaining docs assertThat( @@ -263,7 +263,7 @@ public void testSlices() throws Exception { matcher() .deleted(5) .slices(hasSize(expectedSlices))); - assertHitCount(client().prepareSearch("test").setTypes("test").setSize(0).get(), 0); + assertHitCount(client().prepareSearch("test").setSize(0).get(), 0); } public void testMultipleSources() throws Exception { @@ -301,9 +301,18 @@ public void testMultipleSources() throws Exception { .slices(hasSize(expectedSlices))); for (String index : docs.keySet()) { - assertHitCount(client().prepareSearch(index).setTypes("test").setSize(0).get(), 0); + assertHitCount(client().prepareSearch(index).setSize(0).get(), 0); } } + public void testMissingSources() { + BulkByScrollResponse response = updateByQuery() + .source("missing-index-*") + .refresh(true) + .setSlices(AbstractBulkByScrollRequest.AUTO_SLICES) + .get(); + assertThat(response, matcher().deleted(0).slices(hasSize(0))); + } + } diff --git a/modules/reindex/src/test/java/org/elasticsearch/index/reindex/ManyDocumentsIT.java b/modules/reindex/src/test/java/org/elasticsearch/index/reindex/ManyDocumentsIT.java index b86f28452cc96..fd8cf49cc39ad 100644 --- a/modules/reindex/src/test/java/org/elasticsearch/index/reindex/ManyDocumentsIT.java +++ b/modules/reindex/src/test/java/org/elasticsearch/index/reindex/ManyDocumentsIT.java @@ -73,18 +73,35 @@ public void testReindexFromRemote() throws IOException { Map http = (Map) nodeInfo.get("http"); String remote = "http://"+ http.get("publish_address"); Request request = new Request("POST", "/_reindex"); - request.setJsonEntity( + if (randomBoolean()) { + request.setJsonEntity( "{\n" + - " \"source\":{\n" + - " \"index\":\"test\",\n" + - " \"remote\":{\n" + - " \"host\":\"" + remote + "\"\n" + - " }\n" + - " }\n," + - " \"dest\":{\n" + - " \"index\":\"des\"\n" + - " }\n" + - "}"); + " \"source\":{\n" + + " \"index\":\"test\",\n" + + " \"remote\":{\n" + + " \"host\":\"" + remote + "\"\n" + + " }\n" + + " }\n," + + " \"dest\":{\n" + + " \"index\":\"des\"\n" + + " }\n" + + "}"); + } else { + // Test with external version_type + request.setJsonEntity( + "{\n" + + " \"source\":{\n" + + " \"index\":\"test\",\n" + + " \"remote\":{\n" + + " \"host\":\"" + remote + "\"\n" + + " }\n" + + " }\n," + + " \"dest\":{\n" + + " \"index\":\"des\",\n" + + " \"version_type\": \"external\"\n" + + " }\n" + + "}"); + } Map response = entityAsMap(client().performRequest(request)); assertThat(response, hasEntry("total", count)); assertThat(response, hasEntry("created", count)); diff --git a/modules/reindex/src/test/java/org/elasticsearch/index/reindex/ReindexBasicTests.java b/modules/reindex/src/test/java/org/elasticsearch/index/reindex/ReindexBasicTests.java index 43764bf25fcbf..55634204eb8f9 100644 --- a/modules/reindex/src/test/java/org/elasticsearch/index/reindex/ReindexBasicTests.java +++ b/modules/reindex/src/test/java/org/elasticsearch/index/reindex/ReindexBasicTests.java @@ -58,8 +58,8 @@ public void testFiltering() throws Exception { assertThat(copy.get(), matcher().created(2)); assertHitCount(client().prepareSearch("dest_half").setSize(0).get(), 2); - // Limit with size - copy = reindex().source("source").destination("dest_size_one", "type").size(1).refresh(true); + // Limit with maxDocs + copy = reindex().source("source").destination("dest_size_one", "type").maxDocs(1).refresh(true); assertThat(copy.get(), matcher().created(1)); assertHitCount(client().prepareSearch("dest_size_one").setSize(0).get(), 1); } @@ -86,7 +86,7 @@ public void testCopyMany() throws Exception { copy = reindex().source("source").destination("dest_half", "type").refresh(true); // Use a small batch size so we have to use more than one batch copy.source().setSize(5); - copy.size(half); // The real "size" of the request. + copy.maxDocs(half); assertThat(copy.get(), matcher().created(half).batches(half, 5)); assertHitCount(client().prepareSearch("dest_half").setSize(0).get(), half); } @@ -109,14 +109,14 @@ public void testCopyManyWithSlices() throws Exception { // Use a small batch size so we have to use more than one batch copy.source().setSize(5); assertThat(copy.get(), matcher().created(max).batches(greaterThanOrEqualTo(max / 5)).slices(hasSize(expectedSlices))); - assertHitCount(client().prepareSearch("dest").setTypes("type").setSize(0).get(), max); + assertHitCount(client().prepareSearch("dest").setSize(0).get(), max); // Copy some of the docs int half = max / 2; copy = reindex().source("source").destination("dest_half", "type").refresh(true).setSlices(slices); // Use a small batch size so we have to use more than one batch copy.source().setSize(5); - copy.size(half); // The real "size" of the request. + copy.maxDocs(half); BulkByScrollResponse response = copy.get(); assertThat(response, matcher().created(lessThanOrEqualTo((long) half)).slices(hasSize(expectedSlices))); assertHitCount(client().prepareSearch("dest_half").setSize(0).get(), response.getCreated()); @@ -157,5 +157,13 @@ public void testMultipleSources() throws Exception { assertHitCount(client().prepareSearch("dest").setSize(0).get(), allDocs.size()); } + public void testMissingSources() { + BulkByScrollResponse response = updateByQuery() + .source("missing-index-*") + .refresh(true) + .setSlices(AbstractBulkByScrollRequest.AUTO_SLICES) + .get(); + assertThat(response, matcher().created(0).slices(hasSize(0))); + } } diff --git a/modules/reindex/src/test/java/org/elasticsearch/index/reindex/ReindexFailureTests.java b/modules/reindex/src/test/java/org/elasticsearch/index/reindex/ReindexFailureTests.java index 917d196b6e9fb..4b9a69cfce1c8 100644 --- a/modules/reindex/src/test/java/org/elasticsearch/index/reindex/ReindexFailureTests.java +++ b/modules/reindex/src/test/java/org/elasticsearch/index/reindex/ReindexFailureTests.java @@ -21,7 +21,6 @@ import org.elasticsearch.action.bulk.BulkItemResponse.Failure; import org.elasticsearch.action.index.IndexRequestBuilder; -import org.elasticsearch.test.junit.annotations.TestLogging; import java.util.ArrayList; import java.util.List; @@ -38,7 +37,6 @@ /** * Tests failure capturing and abort-on-failure behavior of reindex. */ -@TestLogging("_root:DEBUG") public class ReindexFailureTests extends ReindexTestCase { public void testFailuresCauseAbortDefault() throws Exception { /* @@ -114,7 +112,7 @@ public void testResponseOnSearchFailure() throws Exception { * so we *try* and wait for the delete to be fully * complete here. */ - assertBusy(() -> assertFalse(client().admin().indices().prepareExists("source").get().isExists())); + assertBusy(() -> assertFalse(indexExists("source"))); } catch (ExecutionException e) { logger.info("Triggered a reindex failure on the {} attempt: {}", attempt, e.getMessage()); assertThat(e.getMessage(), diff --git a/modules/reindex/src/test/java/org/elasticsearch/index/reindex/ReindexFromRemoteBuildRestClientTests.java b/modules/reindex/src/test/java/org/elasticsearch/index/reindex/ReindexFromRemoteBuildRestClientTests.java index b0b35fbaac74e..73d92e239451f 100644 --- a/modules/reindex/src/test/java/org/elasticsearch/index/reindex/ReindexFromRemoteBuildRestClientTests.java +++ b/modules/reindex/src/test/java/org/elasticsearch/index/reindex/ReindexFromRemoteBuildRestClientTests.java @@ -22,9 +22,11 @@ import org.elasticsearch.client.RestClient; import org.elasticsearch.client.RestClientBuilderTestCase; import org.elasticsearch.common.bytes.BytesArray; +import org.elasticsearch.common.bytes.BytesReference; import org.elasticsearch.common.settings.Settings; import org.elasticsearch.env.Environment; import org.elasticsearch.env.TestEnvironment; +import org.elasticsearch.index.query.MatchAllQueryBuilder; import org.elasticsearch.watcher.ResourceWatcherService; import java.util.ArrayList; @@ -38,9 +40,12 @@ import static org.mockito.Mockito.mock; public class ReindexFromRemoteBuildRestClientTests extends RestClientBuilderTestCase { + + private final BytesReference matchAll = new BytesArray(new MatchAllQueryBuilder().toString()); + public void testBuildRestClient() throws Exception { for(final String path: new String[]{"", null, "/", "path"}) { - RemoteInfo remoteInfo = new RemoteInfo("https", "localhost", 9200, path, new BytesArray("ignored"), null, null, emptyMap(), + RemoteInfo remoteInfo = new RemoteInfo("https", "localhost", 9200, path, matchAll, null, null, emptyMap(), RemoteInfo.DEFAULT_SOCKET_TIMEOUT, RemoteInfo.DEFAULT_CONNECT_TIMEOUT); long taskId = randomLong(); List threads = synchronizedList(new ArrayList<>()); @@ -64,7 +69,7 @@ public void testHeaders() throws Exception { for (int i = 0; i < numHeaders; ++i) { headers.put("header" + i, Integer.toString(i)); } - RemoteInfo remoteInfo = new RemoteInfo("https", "localhost", 9200, null, new BytesArray("ignored"), null, null, + RemoteInfo remoteInfo = new RemoteInfo("https", "localhost", 9200, null, matchAll, null, null, headers, RemoteInfo.DEFAULT_SOCKET_TIMEOUT, RemoteInfo.DEFAULT_CONNECT_TIMEOUT); long taskId = randomLong(); List threads = synchronizedList(new ArrayList<>()); diff --git a/modules/reindex/src/test/java/org/elasticsearch/index/reindex/ReindexFromRemoteWhitelistTests.java b/modules/reindex/src/test/java/org/elasticsearch/index/reindex/ReindexFromRemoteWhitelistTests.java index f7a4e74fa19fd..6982a94146eee 100644 --- a/modules/reindex/src/test/java/org/elasticsearch/index/reindex/ReindexFromRemoteWhitelistTests.java +++ b/modules/reindex/src/test/java/org/elasticsearch/index/reindex/ReindexFromRemoteWhitelistTests.java @@ -20,6 +20,7 @@ package org.elasticsearch.index.reindex; import org.elasticsearch.common.bytes.BytesArray; +import org.elasticsearch.common.bytes.BytesReference; import org.elasticsearch.test.ESTestCase; import java.net.UnknownHostException; @@ -37,6 +38,9 @@ * Tests the reindex-from-remote whitelist of remotes. */ public class ReindexFromRemoteWhitelistTests extends ESTestCase { + + private final BytesReference query = new BytesArray("{ \"foo\" : \"bar\" }"); + public void testLocalRequestWithoutWhitelist() { checkRemoteWhitelist(buildRemoteWhitelist(emptyList()), null); } @@ -49,7 +53,7 @@ public void testLocalRequestWithWhitelist() { * Build a {@link RemoteInfo}, defaulting values that we don't care about in this test to values that don't hurt anything. */ private RemoteInfo newRemoteInfo(String host, int port) { - return new RemoteInfo(randomAlphaOfLength(5), host, port, null, new BytesArray("test"), null, null, emptyMap(), + return new RemoteInfo(randomAlphaOfLength(5), host, port, null, query, null, null, emptyMap(), RemoteInfo.DEFAULT_SOCKET_TIMEOUT, RemoteInfo.DEFAULT_CONNECT_TIMEOUT); } @@ -63,7 +67,7 @@ public void testWhitelistedRemote() { public void testWhitelistedByPrefix() { checkRemoteWhitelist(buildRemoteWhitelist(singletonList("*.example.com:9200")), - new RemoteInfo(randomAlphaOfLength(5), "es.example.com", 9200, null, new BytesArray("test"), null, null, emptyMap(), + new RemoteInfo(randomAlphaOfLength(5), "es.example.com", 9200, null, query, null, null, emptyMap(), RemoteInfo.DEFAULT_SOCKET_TIMEOUT, RemoteInfo.DEFAULT_CONNECT_TIMEOUT)); checkRemoteWhitelist(buildRemoteWhitelist(singletonList("*.example.com:9200")), newRemoteInfo("6e134134a1.us-east-1.aws.example.com", 9200)); diff --git a/modules/reindex/src/test/java/org/elasticsearch/index/reindex/ReindexRestClientSslTests.java b/modules/reindex/src/test/java/org/elasticsearch/index/reindex/ReindexRestClientSslTests.java index a2e00e66fd218..97f803650644f 100644 --- a/modules/reindex/src/test/java/org/elasticsearch/index/reindex/ReindexRestClientSslTests.java +++ b/modules/reindex/src/test/java/org/elasticsearch/index/reindex/ReindexRestClientSslTests.java @@ -199,8 +199,9 @@ public void testClientPassesClientCertificate() throws IOException { } private RemoteInfo getRemoteInfo() { - return new RemoteInfo("https", server.getAddress().getHostName(), server.getAddress().getPort(), "/", new BytesArray("test"), - "user", "password", Collections.emptyMap(), RemoteInfo.DEFAULT_SOCKET_TIMEOUT, RemoteInfo.DEFAULT_CONNECT_TIMEOUT); + return new RemoteInfo("https", server.getAddress().getHostName(), server.getAddress().getPort(), "/", + new BytesArray("{\"match_all\":{}}"), "user", "password", Collections.emptyMap(), RemoteInfo.DEFAULT_SOCKET_TIMEOUT, + RemoteInfo.DEFAULT_CONNECT_TIMEOUT); } @SuppressForbidden(reason = "use http server") diff --git a/modules/reindex/src/test/java/org/elasticsearch/index/reindex/ReindexSourceTargetValidationTests.java b/modules/reindex/src/test/java/org/elasticsearch/index/reindex/ReindexSourceTargetValidationTests.java index 8264d4342c993..dc9d4a548170c 100644 --- a/modules/reindex/src/test/java/org/elasticsearch/index/reindex/ReindexSourceTargetValidationTests.java +++ b/modules/reindex/src/test/java/org/elasticsearch/index/reindex/ReindexSourceTargetValidationTests.java @@ -32,6 +32,7 @@ import org.elasticsearch.cluster.metadata.MetaData; import org.elasticsearch.common.Nullable; import org.elasticsearch.common.bytes.BytesArray; +import org.elasticsearch.common.bytes.BytesReference; import org.elasticsearch.common.settings.ClusterSettings; import org.elasticsearch.common.settings.Settings; import org.elasticsearch.test.ESTestCase; @@ -61,6 +62,8 @@ public class ReindexSourceTargetValidationTests extends ESTestCase { private static final AutoCreateIndex AUTO_CREATE_INDEX = new AutoCreateIndex(Settings.EMPTY, new ClusterSettings(Settings.EMPTY, ClusterSettings.BUILT_IN_CLUSTER_SETTINGS), INDEX_NAME_EXPRESSION_RESOLVER); + private final BytesReference query = new BytesArray("{ \"foo\" : \"bar\" }"); + public void testObviousCases() { fails("target", "target"); fails("target", "foo", "bar", "target", "baz"); @@ -106,10 +109,10 @@ public void testTargetIsWriteAlias() { public void testRemoteInfoSkipsValidation() { // The index doesn't have to exist - succeeds(new RemoteInfo(randomAlphaOfLength(5), "test", 9200, null, new BytesArray("test"), null, null, emptyMap(), + succeeds(new RemoteInfo(randomAlphaOfLength(5), "test", 9200, null, query, null, null, emptyMap(), RemoteInfo.DEFAULT_SOCKET_TIMEOUT, RemoteInfo.DEFAULT_CONNECT_TIMEOUT), "does_not_exist", "target"); // And it doesn't matter if they are the same index. They are considered to be different because the remote one is, well, remote. - succeeds(new RemoteInfo(randomAlphaOfLength(5), "test", 9200, null, new BytesArray("test"), null, null, emptyMap(), + succeeds(new RemoteInfo(randomAlphaOfLength(5), "test", 9200, null, query, null, null, emptyMap(), RemoteInfo.DEFAULT_SOCKET_TIMEOUT, RemoteInfo.DEFAULT_CONNECT_TIMEOUT), "target", "target"); } diff --git a/modules/reindex/src/test/java/org/elasticsearch/index/reindex/RestReindexActionTests.java b/modules/reindex/src/test/java/org/elasticsearch/index/reindex/RestReindexActionTests.java index f0aca38545b4c..3fb85d16a2cc0 100644 --- a/modules/reindex/src/test/java/org/elasticsearch/index/reindex/RestReindexActionTests.java +++ b/modules/reindex/src/test/java/org/elasticsearch/index/reindex/RestReindexActionTests.java @@ -23,7 +23,6 @@ import org.elasticsearch.common.bytes.BytesReference; import org.elasticsearch.common.settings.Settings; import org.elasticsearch.common.xcontent.XContentBuilder; -import org.elasticsearch.common.xcontent.XContentParser; import org.elasticsearch.common.xcontent.XContentType; import org.elasticsearch.common.xcontent.json.JsonXContent; import org.elasticsearch.rest.RestRequest.Method; @@ -32,12 +31,8 @@ import org.junit.Before; import java.io.IOException; -import java.util.Arrays; -import java.util.HashMap; -import java.util.Map; import static java.util.Collections.singletonMap; -import static org.elasticsearch.common.unit.TimeValue.timeValueSeconds; public class RestReindexActionTests extends RestActionTestCase { @@ -48,126 +43,6 @@ public void setUpAction() { action = new RestReindexAction(Settings.EMPTY, controller()); } - public void testBuildRemoteInfoNoRemote() throws IOException { - assertNull(RestReindexAction.buildRemoteInfo(new HashMap<>())); - } - - public void testBuildRemoteInfoFullyLoaded() throws IOException { - Map headers = new HashMap<>(); - headers.put("first", "a"); - headers.put("second", "b"); - headers.put("third", ""); - - Map remote = new HashMap<>(); - remote.put("host", "https://example.com:9200"); - remote.put("username", "testuser"); - remote.put("password", "testpass"); - remote.put("headers", headers); - remote.put("socket_timeout", "90s"); - remote.put("connect_timeout", "10s"); - - Map query = new HashMap<>(); - query.put("a", "b"); - - Map source = new HashMap<>(); - source.put("remote", remote); - source.put("query", query); - - RemoteInfo remoteInfo = RestReindexAction.buildRemoteInfo(source); - assertEquals("https", remoteInfo.getScheme()); - assertEquals("example.com", remoteInfo.getHost()); - assertEquals(9200, remoteInfo.getPort()); - assertEquals("{\n \"a\" : \"b\"\n}", remoteInfo.getQuery().utf8ToString()); - assertEquals("testuser", remoteInfo.getUsername()); - assertEquals("testpass", remoteInfo.getPassword()); - assertEquals(headers, remoteInfo.getHeaders()); - assertEquals(timeValueSeconds(90), remoteInfo.getSocketTimeout()); - assertEquals(timeValueSeconds(10), remoteInfo.getConnectTimeout()); - } - - public void testBuildRemoteInfoWithoutAllParts() throws IOException { - expectThrows(IllegalArgumentException.class, () -> buildRemoteInfoHostTestCase("example.com")); - expectThrows(IllegalArgumentException.class, () -> buildRemoteInfoHostTestCase(":9200")); - expectThrows(IllegalArgumentException.class, () -> buildRemoteInfoHostTestCase("http://:9200")); - expectThrows(IllegalArgumentException.class, () -> buildRemoteInfoHostTestCase("example.com:9200")); - expectThrows(IllegalArgumentException.class, () -> buildRemoteInfoHostTestCase("http://example.com")); - } - - public void testBuildRemoteInfoWithAllHostParts() throws IOException { - RemoteInfo info = buildRemoteInfoHostTestCase("http://example.com:9200"); - assertEquals("http", info.getScheme()); - assertEquals("example.com", info.getHost()); - assertEquals(9200, info.getPort()); - assertNull(info.getPathPrefix()); - assertEquals(RemoteInfo.DEFAULT_SOCKET_TIMEOUT, info.getSocketTimeout()); // Didn't set the timeout so we should get the default - assertEquals(RemoteInfo.DEFAULT_CONNECT_TIMEOUT, info.getConnectTimeout()); // Didn't set the timeout so we should get the default - - info = buildRemoteInfoHostTestCase("https://other.example.com:9201"); - assertEquals("https", info.getScheme()); - assertEquals("other.example.com", info.getHost()); - assertEquals(9201, info.getPort()); - assertNull(info.getPathPrefix()); - assertEquals(RemoteInfo.DEFAULT_SOCKET_TIMEOUT, info.getSocketTimeout()); - assertEquals(RemoteInfo.DEFAULT_CONNECT_TIMEOUT, info.getConnectTimeout()); - - info = buildRemoteInfoHostTestCase("https://[::1]:9201"); - assertEquals("https", info.getScheme()); - assertEquals("[::1]", info.getHost()); - assertEquals(9201, info.getPort()); - assertNull(info.getPathPrefix()); - assertEquals(RemoteInfo.DEFAULT_SOCKET_TIMEOUT, info.getSocketTimeout()); - assertEquals(RemoteInfo.DEFAULT_CONNECT_TIMEOUT, info.getConnectTimeout()); - - info = buildRemoteInfoHostTestCase("https://other.example.com:9201/"); - assertEquals("https", info.getScheme()); - assertEquals("other.example.com", info.getHost()); - assertEquals(9201, info.getPort()); - assertEquals("/", info.getPathPrefix()); - assertEquals(RemoteInfo.DEFAULT_SOCKET_TIMEOUT, info.getSocketTimeout()); - assertEquals(RemoteInfo.DEFAULT_CONNECT_TIMEOUT, info.getConnectTimeout()); - - info = buildRemoteInfoHostTestCase("https://other.example.com:9201/proxy-path/"); - assertEquals("https", info.getScheme()); - assertEquals("other.example.com", info.getHost()); - assertEquals(9201, info.getPort()); - assertEquals("/proxy-path/", info.getPathPrefix()); - assertEquals(RemoteInfo.DEFAULT_SOCKET_TIMEOUT, info.getSocketTimeout()); - assertEquals(RemoteInfo.DEFAULT_CONNECT_TIMEOUT, info.getConnectTimeout()); - - final IllegalArgumentException exception = expectThrows(IllegalArgumentException.class, - () -> buildRemoteInfoHostTestCase("https")); - assertEquals("[host] must be of the form [scheme]://[host]:[port](/[pathPrefix])? but was [https]", - exception.getMessage()); - } - - public void testReindexFromRemoteRequestParsing() throws IOException { - BytesReference request; - try (XContentBuilder b = JsonXContent.contentBuilder()) { - b.startObject(); { - b.startObject("source"); { - b.startObject("remote"); { - b.field("host", "http://localhost:9200"); - } - b.endObject(); - b.field("index", "source"); - } - b.endObject(); - b.startObject("dest"); { - b.field("index", "dest"); - } - b.endObject(); - } - b.endObject(); - request = BytesReference.bytes(b); - } - try (XContentParser p = createParser(JsonXContent.jsonXContent, request)) { - ReindexRequest r = new ReindexRequest(); - RestReindexAction.PARSER.parse(p, r, null); - assertEquals("localhost", r.getRemoteInfo().getHost()); - assertArrayEquals(new String[] {"source"}, r.getSearchRequest().indices()); - } - } - public void testPipelineQueryParameterIsError() throws IOException { FakeRestRequest.Builder request = new FakeRestRequest.Builder(xContentRegistry()); try (XContentBuilder body = JsonXContent.contentBuilder().prettyPrint()) { @@ -206,37 +81,6 @@ public void testSetScrollTimeout() throws IOException { } } - private RemoteInfo buildRemoteInfoHostTestCase(String hostInRest) throws IOException { - Map remote = new HashMap<>(); - remote.put("host", hostInRest); - - Map source = new HashMap<>(); - source.put("remote", remote); - - return RestReindexAction.buildRemoteInfo(source); - } - - /** - * test deprecation is logged if one or more types are used in source search request inside reindex - */ - public void testTypeInSource() throws IOException { - FakeRestRequest.Builder requestBuilder = new FakeRestRequest.Builder(xContentRegistry()) - .withMethod(Method.POST) - .withPath("/_reindex"); - XContentBuilder b = JsonXContent.contentBuilder().startObject(); - { - b.startObject("source"); - { - b.field("type", randomFrom(Arrays.asList("\"t1\"", "[\"t1\", \"t2\"]", "\"_doc\""))); - } - b.endObject(); - } - b.endObject(); - requestBuilder.withContent(new BytesArray(BytesReference.bytes(b).toBytesRef()), XContentType.JSON); - dispatchRequest(requestBuilder.build()); - assertWarnings(RestReindexAction.TYPES_DEPRECATION_MESSAGE); - } - /** * test deprecation is logged if a type is used in the destination index request inside reindex */ @@ -255,6 +99,6 @@ public void testTypeInDestination() throws IOException { b.endObject(); requestBuilder.withContent(new BytesArray(BytesReference.bytes(b).toBytesRef()), XContentType.JSON); dispatchRequest(requestBuilder.build()); - assertWarnings(RestReindexAction.TYPES_DEPRECATION_MESSAGE); + assertWarnings(ReindexRequest.TYPES_DEPRECATION_MESSAGE); } } diff --git a/modules/reindex/src/test/java/org/elasticsearch/index/reindex/RethrottleTests.java b/modules/reindex/src/test/java/org/elasticsearch/index/reindex/RethrottleTests.java index 58067cd2cdbbf..4420fe450bd4d 100644 --- a/modules/reindex/src/test/java/org/elasticsearch/index/reindex/RethrottleTests.java +++ b/modules/reindex/src/test/java/org/elasticsearch/index/reindex/RethrottleTests.java @@ -27,7 +27,6 @@ import org.elasticsearch.action.index.IndexRequestBuilder; import org.elasticsearch.index.query.QueryBuilders; import org.elasticsearch.tasks.TaskId; -import org.elasticsearch.test.junit.annotations.TestLogging; import java.util.ArrayList; import java.util.List; @@ -50,7 +49,6 @@ * too but this is the only place that tests running against multiple nodes so it is the only integration tests that checks for * serialization. */ -@TestLogging("org.elasticsearch.index.reindex:TRACE,org.elasticsearch.action.bulk:TRACE,org.elasticsearch.search.SearchService:TRACE") public class RethrottleTests extends ReindexTestCase { public void testReindex() throws Exception { diff --git a/modules/reindex/src/test/java/org/elasticsearch/index/reindex/RoundTripTests.java b/modules/reindex/src/test/java/org/elasticsearch/index/reindex/RoundTripTests.java index ab1968a2e3582..d401efaae4bf0 100644 --- a/modules/reindex/src/test/java/org/elasticsearch/index/reindex/RoundTripTests.java +++ b/modules/reindex/src/test/java/org/elasticsearch/index/reindex/RoundTripTests.java @@ -51,7 +51,7 @@ public void testReindexRequest() throws IOException { reindex.getDestination().index("test"); if (randomBoolean()) { int port = between(1, Integer.MAX_VALUE); - BytesReference query = new BytesArray(randomAlphaOfLength(5)); + BytesReference query = new BytesArray("{\"match_all\":{}}"); String username = randomBoolean() ? randomAlphaOfLength(5) : null; String password = username != null && randomBoolean() ? randomAlphaOfLength(5) : null; int headersCount = randomBoolean() ? 0 : between(1, 10); @@ -107,7 +107,7 @@ private void randomRequest(AbstractBulkByScrollRequest request) { request.getSearchRequest().indices("test"); request.getSearchRequest().source().size(between(1, 1000)); if (randomBoolean()) { - request.setSize(between(1, Integer.MAX_VALUE)); + request.setMaxDocs(between(1, Integer.MAX_VALUE)); } request.setAbortOnVersionConflict(random().nextBoolean()); request.setRefresh(rarely()); diff --git a/modules/reindex/src/test/java/org/elasticsearch/index/reindex/UpdateByQueryBasicTests.java b/modules/reindex/src/test/java/org/elasticsearch/index/reindex/UpdateByQueryBasicTests.java index 91a92005c2cdc..f55ee1e802e6b 100644 --- a/modules/reindex/src/test/java/org/elasticsearch/index/reindex/UpdateByQueryBasicTests.java +++ b/modules/reindex/src/test/java/org/elasticsearch/index/reindex/UpdateByQueryBasicTests.java @@ -21,7 +21,6 @@ import org.elasticsearch.action.index.IndexRequestBuilder; import org.elasticsearch.search.sort.SortOrder; -import org.elasticsearch.test.junit.annotations.TestLogging; import java.util.ArrayList; import java.util.Collection; @@ -34,14 +33,13 @@ import static org.elasticsearch.test.hamcrest.ElasticsearchAssertions.assertHitCount; import static org.hamcrest.Matchers.hasSize; -@TestLogging("org.elasticsearch.index.reindex:TRACE,org.elasticsearch.action.bulk:TRACE,org.elasticsearch.search.SearchService:TRACE") public class UpdateByQueryBasicTests extends ReindexTestCase { public void testBasics() throws Exception { indexRandom(true, client().prepareIndex("test", "test", "1").setSource("foo", "a"), client().prepareIndex("test", "test", "2").setSource("foo", "a"), client().prepareIndex("test", "test", "3").setSource("foo", "b"), client().prepareIndex("test", "test", "4").setSource("foo", "c")); - assertHitCount(client().prepareSearch("test").setTypes("test").setSize(0).get(), 4); + assertHitCount(client().prepareSearch("test").setSize(0).get(), 4); assertEquals(1, client().prepareGet("test", "test", "1").get().getVersion()); assertEquals(1, client().prepareGet("test", "test", "4").get().getVersion()); @@ -79,7 +77,7 @@ public void testSlices() throws Exception { client().prepareIndex("test", "test", "2").setSource("foo", "a"), client().prepareIndex("test", "test", "3").setSource("foo", "b"), client().prepareIndex("test", "test", "4").setSource("foo", "c")); - assertHitCount(client().prepareSearch("test").setTypes("test").setSize(0).get(), 4); + assertHitCount(client().prepareSearch("test").setSize(0).get(), 4); assertEquals(1, client().prepareGet("test", "test", "1").get().getVersion()); assertEquals(1, client().prepareGet("test", "test", "4").get().getVersion()); @@ -160,4 +158,13 @@ public void testMultipleSources() throws Exception { assertEquals(2, client().prepareGet(index, "test", Integer.toString(randomDoc)).get().getVersion()); } } + + public void testMissingSources() { + BulkByScrollResponse response = updateByQuery() + .source("missing-index-*") + .refresh(true) + .setSlices(AbstractBulkByScrollRequest.AUTO_SLICES) + .get(); + assertThat(response, matcher().updated(0).slices(hasSize(0))); + } } diff --git a/modules/reindex/src/test/java/org/elasticsearch/index/reindex/remote/ReindexFromOldRemoteIT.java b/modules/reindex/src/test/java/org/elasticsearch/index/reindex/remote/ReindexFromOldRemoteIT.java index 9feed83595ff1..27d975c4114f9 100644 --- a/modules/reindex/src/test/java/org/elasticsearch/index/reindex/remote/ReindexFromOldRemoteIT.java +++ b/modules/reindex/src/test/java/org/elasticsearch/index/reindex/remote/ReindexFromOldRemoteIT.java @@ -56,19 +56,38 @@ private void oldEsTestCase(String portPropertyName, String requestsPerSecond) th } Request reindex = new Request("POST", "/_reindex"); - reindex.setJsonEntity( + if (randomBoolean()) { + // Reindex using the external version_type + reindex.setJsonEntity( "{\n" - + " \"source\":{\n" - + " \"index\": \"test\",\n" - + " \"size\": 1,\n" - + " \"remote\": {\n" - + " \"host\": \"http://127.0.0.1:" + oldEsPort + "\"\n" - + " }\n" - + " },\n" - + " \"dest\": {\n" - + " \"index\": \"test\"\n" - + " }\n" - + "}"); + + " \"source\":{\n" + + " \"index\": \"test\",\n" + + " \"size\": 1,\n" + + " \"remote\": {\n" + + " \"host\": \"http://127.0.0.1:" + oldEsPort + "\"\n" + + " }\n" + + " },\n" + + " \"dest\": {\n" + + " \"index\": \"test\",\n" + + " \"version_type\": \"external\"\n" + + " }\n" + + "}"); + } else { + // Reindex using the default internal version_type + reindex.setJsonEntity( + "{\n" + + " \"source\":{\n" + + " \"index\": \"test\",\n" + + " \"size\": 1,\n" + + " \"remote\": {\n" + + " \"host\": \"http://127.0.0.1:" + oldEsPort + "\"\n" + + " }\n" + + " },\n" + + " \"dest\": {\n" + + " \"index\": \"test\"\n" + + " }\n" + + "}"); + } reindex.addParameter("refresh", "true"); reindex.addParameter("pretty", "true"); if (requestsPerSecond != null) { diff --git a/modules/reindex/src/test/java/org/elasticsearch/index/reindex/remote/RemoteInfoTests.java b/modules/reindex/src/test/java/org/elasticsearch/index/reindex/remote/RemoteInfoTests.java index de0ade9c47cc3..f0c4ccf476636 100644 --- a/modules/reindex/src/test/java/org/elasticsearch/index/reindex/remote/RemoteInfoTests.java +++ b/modules/reindex/src/test/java/org/elasticsearch/index/reindex/remote/RemoteInfoTests.java @@ -19,28 +19,28 @@ package org.elasticsearch.index.reindex.remote; -import org.elasticsearch.index.reindex.RemoteInfo; import org.elasticsearch.common.bytes.BytesArray; +import org.elasticsearch.index.reindex.RemoteInfo; import org.elasticsearch.test.ESTestCase; import static java.util.Collections.emptyMap; public class RemoteInfoTests extends ESTestCase { private RemoteInfo newRemoteInfo(String scheme, String prefixPath, String username, String password) { - return new RemoteInfo(scheme, "testhost", 12344, prefixPath, new BytesArray("testquery"), username, password, emptyMap(), - RemoteInfo.DEFAULT_SOCKET_TIMEOUT, RemoteInfo.DEFAULT_CONNECT_TIMEOUT); + return new RemoteInfo(scheme, "testhost", 12344, prefixPath,new BytesArray("{ \"foo\" : \"bar\" }"), username, password, + emptyMap(), RemoteInfo.DEFAULT_SOCKET_TIMEOUT, RemoteInfo.DEFAULT_CONNECT_TIMEOUT); } public void testToString() { - assertEquals("host=testhost port=12344 query=testquery", + assertEquals("host=testhost port=12344 query={ \"foo\" : \"bar\" }", newRemoteInfo("http", null, null, null).toString()); - assertEquals("host=testhost port=12344 query=testquery username=testuser", + assertEquals("host=testhost port=12344 query={ \"foo\" : \"bar\" } username=testuser", newRemoteInfo("http", null, "testuser", null).toString()); - assertEquals("host=testhost port=12344 query=testquery username=testuser password=<<>>", + assertEquals("host=testhost port=12344 query={ \"foo\" : \"bar\" } username=testuser password=<<>>", newRemoteInfo("http", null, "testuser", "testpass").toString()); - assertEquals("scheme=https host=testhost port=12344 query=testquery username=testuser password=<<>>", + assertEquals("scheme=https host=testhost port=12344 query={ \"foo\" : \"bar\" } username=testuser password=<<>>", newRemoteInfo("https", null, "testuser", "testpass").toString()); - assertEquals("scheme=https host=testhost port=12344 pathPrefix=prxy query=testquery username=testuser password=<<>>", + assertEquals("scheme=https host=testhost port=12344 pathPrefix=prxy query={ \"foo\" : \"bar\" } username=testuser password=<<>>", newRemoteInfo("https", "prxy", "testuser", "testpass").toString()); } } diff --git a/modules/reindex/src/test/java/org/elasticsearch/index/reindex/remote/RemoteRequestBuildersTests.java b/modules/reindex/src/test/java/org/elasticsearch/index/reindex/remote/RemoteRequestBuildersTests.java index bf6856754044d..0005fa921b33b 100644 --- a/modules/reindex/src/test/java/org/elasticsearch/index/reindex/remote/RemoteRequestBuildersTests.java +++ b/modules/reindex/src/test/java/org/elasticsearch/index/reindex/remote/RemoteRequestBuildersTests.java @@ -63,50 +63,27 @@ public void testIntialSearchPath() { SearchRequest searchRequest = new SearchRequest().source(new SearchSourceBuilder()); assertEquals("/_search", initialSearch(searchRequest, query, remoteVersion).getEndpoint()); searchRequest.indices("a"); - searchRequest.types("b"); - assertEquals("/a/b/_search", initialSearch(searchRequest, query, remoteVersion).getEndpoint()); + assertEquals("/a/_search", initialSearch(searchRequest, query, remoteVersion).getEndpoint()); searchRequest.indices("a", "b"); - searchRequest.types("c", "d"); - assertEquals("/a,b/c,d/_search", initialSearch(searchRequest, query, remoteVersion).getEndpoint()); + assertEquals("/a,b/_search", initialSearch(searchRequest, query, remoteVersion).getEndpoint()); searchRequest.indices("cat,"); - assertEquals("/cat%2C/c,d/_search", initialSearch(searchRequest, query, remoteVersion).getEndpoint()); + assertEquals("/cat%2C/_search", initialSearch(searchRequest, query, remoteVersion).getEndpoint()); searchRequest.indices("cat/"); - assertEquals("/cat%2F/c,d/_search", initialSearch(searchRequest, query, remoteVersion).getEndpoint()); + assertEquals("/cat%2F/_search", initialSearch(searchRequest, query, remoteVersion).getEndpoint()); searchRequest.indices("cat/", "dog"); - assertEquals("/cat%2F,dog/c,d/_search", initialSearch(searchRequest, query, remoteVersion).getEndpoint()); + assertEquals("/cat%2F,dog/_search", initialSearch(searchRequest, query, remoteVersion).getEndpoint()); // test a specific date math + all characters that need escaping. searchRequest.indices("", "<>/{}|+:,"); - assertEquals("/%3Ccat%7Bnow%2Fd%7D%3E,%3C%3E%2F%7B%7D%7C%2B%3A%2C/c,d/_search", + assertEquals("/%3Ccat%7Bnow%2Fd%7D%3E,%3C%3E%2F%7B%7D%7C%2B%3A%2C/_search", initialSearch(searchRequest, query, remoteVersion).getEndpoint()); // re-escape already escaped (no special handling). searchRequest.indices("%2f", "%3a"); - assertEquals("/%252f,%253a/c,d/_search", initialSearch(searchRequest, query, remoteVersion).getEndpoint()); + assertEquals("/%252f,%253a/_search", initialSearch(searchRequest, query, remoteVersion).getEndpoint()); searchRequest.indices("%2fcat,"); - assertEquals("/%252fcat%2C/c,d/_search", initialSearch(searchRequest, query, remoteVersion).getEndpoint()); + assertEquals("/%252fcat%2C/_search", initialSearch(searchRequest, query, remoteVersion).getEndpoint()); searchRequest.indices("%3ccat/"); - assertEquals("/%253ccat%2F/c,d/_search", initialSearch(searchRequest, query, remoteVersion).getEndpoint()); - - searchRequest.indices("ok"); - searchRequest.types("cat,"); - expectBadStartRequest(searchRequest, "Type", ",", "cat,"); - searchRequest.types("cat,", "dog"); - expectBadStartRequest(searchRequest, "Type", ",", "cat,"); - searchRequest.types("dog", "cat,"); - expectBadStartRequest(searchRequest, "Type", ",", "cat,"); - searchRequest.types("cat/"); - expectBadStartRequest(searchRequest, "Type", "/", "cat/"); - searchRequest.types("cat/", "dog"); - expectBadStartRequest(searchRequest, "Type", "/", "cat/"); - searchRequest.types("dog", "cat/"); - expectBadStartRequest(searchRequest, "Type", "/", "cat/"); - } - - private void expectBadStartRequest(SearchRequest searchRequest, String type, String bad, String failed) { - Version remoteVersion = Version.fromId(between(0, Version.CURRENT.id)); - BytesReference query = new BytesArray("{}"); - IllegalArgumentException e = expectThrows(IllegalArgumentException.class, () -> initialSearch(searchRequest, query, remoteVersion)); - assertEquals(type + " containing [" + bad + "] not supported but got [" + failed + "]", e.getMessage()); + assertEquals("/%253ccat%2F/_search", initialSearch(searchRequest, query, remoteVersion).getEndpoint()); } public void testInitialSearchParamsSort() { @@ -136,7 +113,7 @@ public void testInitialSearchParamsFields() { // Test request without any fields Version remoteVersion = Version.fromId(between(2000099, Version.CURRENT.id)); assertThat(initialSearch(searchRequest, query, remoteVersion).getParameters(), - not(either(hasKey("stored_fields")).or(hasKey("fields")))); + not(either(hasKey("stored_fields")).or(hasKey("fields")))); // Test stored_fields for versions that support it searchRequest = new SearchRequest().source(new SearchSourceBuilder()); @@ -157,14 +134,14 @@ public void testInitialSearchParamsFields() { searchRequest.source().storedField("_source").storedField("_id"); remoteVersion = Version.fromId(between(0, 2000099 - 1)); assertThat(initialSearch(searchRequest, query, remoteVersion).getParameters(), - hasEntry("fields", "_source,_id,_parent,_routing,_ttl")); + hasEntry("fields", "_source,_id,_parent,_routing,_ttl")); // But only versions before 1.0 force _source to be in the list searchRequest = new SearchRequest().source(new SearchSourceBuilder()); searchRequest.source().storedField("_id"); remoteVersion = Version.fromId(between(1000099, 2000099 - 1)); assertThat(initialSearch(searchRequest, query, remoteVersion).getParameters(), - hasEntry("fields", "_id,_parent,_routing,_ttl")); + hasEntry("fields", "_id,_parent,_routing,_ttl")); } public void testInitialSearchParamsMisc() { @@ -184,7 +161,7 @@ public void testInitialSearchParamsMisc() { fetchVersion = randomBoolean(); searchRequest.source().version(fetchVersion); } - + Map params = initialSearch(searchRequest, query, remoteVersion).getParameters(); if (scroll == null) { @@ -193,7 +170,12 @@ public void testInitialSearchParamsMisc() { assertScroll(remoteVersion, params, scroll); } assertThat(params, hasEntry("size", Integer.toString(size))); - assertThat(params, fetchVersion == null || fetchVersion == true ? hasEntry("version", null) : not(hasEntry("version", null))); + if (fetchVersion != null) { + assertThat(params, fetchVersion ? hasEntry("version", Boolean.TRUE.toString()) : + hasEntry("version", Boolean.FALSE.toString())); + } else { + assertThat(params, hasEntry("version", Boolean.FALSE.toString())); + } } private void assertScroll(Version remoteVersion, Map params, TimeValue requested) { @@ -220,22 +202,22 @@ public void testInitialSearchEntity() throws IOException { assertEquals(ContentType.APPLICATION_JSON.toString(), entity.getContentType().getValue()); if (remoteVersion.onOrAfter(Version.fromId(1000099))) { assertEquals("{\"query\":" + query + ",\"_source\":true}", - Streams.copyToString(new InputStreamReader(entity.getContent(), StandardCharsets.UTF_8))); + Streams.copyToString(new InputStreamReader(entity.getContent(), StandardCharsets.UTF_8))); } else { assertEquals("{\"query\":" + query + "}", - Streams.copyToString(new InputStreamReader(entity.getContent(), StandardCharsets.UTF_8))); + Streams.copyToString(new InputStreamReader(entity.getContent(), StandardCharsets.UTF_8))); } // Source filtering is included if set up - searchRequest.source().fetchSource(new String[] {"in1", "in2"}, new String[] {"out"}); + searchRequest.source().fetchSource(new String[]{"in1", "in2"}, new String[]{"out"}); entity = initialSearch(searchRequest, new BytesArray(query), remoteVersion).getEntity(); assertEquals(ContentType.APPLICATION_JSON.toString(), entity.getContentType().getValue()); assertEquals("{\"query\":" + query + ",\"_source\":{\"includes\":[\"in1\",\"in2\"],\"excludes\":[\"out\"]}}", - Streams.copyToString(new InputStreamReader(entity.getContent(), StandardCharsets.UTF_8))); + Streams.copyToString(new InputStreamReader(entity.getContent(), StandardCharsets.UTF_8))); // Invalid XContent fails RuntimeException e = expectThrows(RuntimeException.class, - () -> initialSearch(searchRequest, new BytesArray("{}, \"trailing\": {}"), remoteVersion)); + () -> initialSearch(searchRequest, new BytesArray("{}, \"trailing\": {}"), remoteVersion)); assertThat(e.getCause().getMessage(), containsString("Unexpected character (',' (code 44))")); e = expectThrows(RuntimeException.class, () -> initialSearch(searchRequest, new BytesArray("{"), remoteVersion)); assertThat(e.getCause().getMessage(), containsString("Unexpected end-of-input")); diff --git a/modules/reindex/src/test/java/org/elasticsearch/index/reindex/remote/RemoteResponseParsersTests.java b/modules/reindex/src/test/java/org/elasticsearch/index/reindex/remote/RemoteResponseParsersTests.java new file mode 100644 index 0000000000000..f5b406f779a9e --- /dev/null +++ b/modules/reindex/src/test/java/org/elasticsearch/index/reindex/remote/RemoteResponseParsersTests.java @@ -0,0 +1,52 @@ +/* + * 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.reindex.remote; + +import org.elasticsearch.action.search.ShardSearchFailure; +import org.elasticsearch.common.util.concurrent.EsRejectedExecutionException; +import org.elasticsearch.common.xcontent.ToXContent; +import org.elasticsearch.common.xcontent.XContentBuilder; +import org.elasticsearch.common.xcontent.XContentParser; +import org.elasticsearch.common.xcontent.XContentType; +import org.elasticsearch.index.reindex.ScrollableHitSource; +import org.elasticsearch.test.ESTestCase; +import org.hamcrest.Matchers; + +import java.io.IOException; + +import static org.elasticsearch.common.xcontent.XContentFactory.jsonBuilder; + +public class RemoteResponseParsersTests extends ESTestCase { + + /** + * Check that we can parse shard search failures without index information. + */ + public void testFailureWithoutIndex() throws IOException { + ShardSearchFailure failure = new ShardSearchFailure(new EsRejectedExecutionException("exhausted")); + XContentBuilder builder = jsonBuilder(); + failure.toXContent(builder, ToXContent.EMPTY_PARAMS); + try (XContentParser parser = createParser(builder)) { + ScrollableHitSource.SearchFailure parsed = RemoteResponseParsers.SEARCH_FAILURE_PARSER.parse(parser, XContentType.JSON); + assertNotNull(parsed.getReason()); + assertThat(parsed.getReason().getMessage(), Matchers.containsString("exhausted")); + assertThat(parsed.getReason(), Matchers.instanceOf(EsRejectedExecutionException.class)); + } + } +} diff --git a/modules/reindex/src/test/java/org/elasticsearch/index/reindex/remote/RemoteScrollableHitSourceTests.java b/modules/reindex/src/test/java/org/elasticsearch/index/reindex/remote/RemoteScrollableHitSourceTests.java index 844c6b8351993..0ab100a856fc1 100644 --- a/modules/reindex/src/test/java/org/elasticsearch/index/reindex/remote/RemoteScrollableHitSourceTests.java +++ b/modules/reindex/src/test/java/org/elasticsearch/index/reindex/remote/RemoteScrollableHitSourceTests.java @@ -40,7 +40,6 @@ import org.elasticsearch.ElasticsearchStatusException; import org.elasticsearch.Version; import org.elasticsearch.action.bulk.BackoffPolicy; -import org.elasticsearch.index.reindex.ScrollableHitSource.Response; import org.elasticsearch.action.search.SearchRequest; import org.elasticsearch.client.HeapBufferedAsyncResponseConsumer; import org.elasticsearch.client.RestClient; @@ -53,6 +52,7 @@ import org.elasticsearch.common.unit.TimeValue; import org.elasticsearch.common.util.concurrent.EsExecutors; import org.elasticsearch.common.util.concurrent.EsRejectedExecutionException; +import org.elasticsearch.index.reindex.ScrollableHitSource.Response; import org.elasticsearch.rest.RestStatus; import org.elasticsearch.search.builder.SearchSourceBuilder; import org.elasticsearch.test.ESTestCase; @@ -123,41 +123,20 @@ public void tearDown() throws Exception { } public void testLookupRemoteVersion() throws Exception { + assertLookupRemoteVersion(Version.fromString("0.20.5"), "main/0_20_5.json"); + assertLookupRemoteVersion(Version.fromString("0.90.13"), "main/0_90_13.json"); + assertLookupRemoteVersion(Version.fromString("1.7.5"), "main/1_7_5.json"); + assertLookupRemoteVersion(Version.fromId(2030399), "main/2_3_3.json"); + // assert for V_5_0_0 (no qualifier) since we no longer consider qualifier in Version since 7 + assertLookupRemoteVersion(Version.fromId(5000099), "main/5_0_0_alpha_3.json"); + // V_5_0_0 since we no longer consider qualifier in Version + assertLookupRemoteVersion(Version.fromId(5000099), "main/with_unknown_fields.json"); + } + + private void assertLookupRemoteVersion(Version expected, String s) throws Exception { AtomicBoolean called = new AtomicBoolean(); - sourceWithMockedRemoteCall(false, ContentType.APPLICATION_JSON, "main/0_20_5.json").lookupRemoteVersion(v -> { - assertEquals(Version.fromString("0.20.5"), v); - called.set(true); - }); - assertTrue(called.get()); - called.set(false); - sourceWithMockedRemoteCall(false, ContentType.APPLICATION_JSON, "main/0_90_13.json").lookupRemoteVersion(v -> { - assertEquals(Version.fromString("0.90.13"), v); - called.set(true); - }); - assertTrue(called.get()); - called.set(false); - sourceWithMockedRemoteCall(false, ContentType.APPLICATION_JSON, "main/1_7_5.json").lookupRemoteVersion(v -> { - assertEquals(Version.fromString("1.7.5"), v); - called.set(true); - }); - assertTrue(called.get()); - called.set(false); - sourceWithMockedRemoteCall(false, ContentType.APPLICATION_JSON, "main/2_3_3.json").lookupRemoteVersion(v -> { - assertEquals(Version.fromId(2030399), v); - called.set(true); - }); - assertTrue(called.get()); - called.set(false); - sourceWithMockedRemoteCall(false, ContentType.APPLICATION_JSON, "main/5_0_0_alpha_3.json").lookupRemoteVersion(v -> { - // assert for V_5_0_0 (no qualifier) since we no longer consider qualifier in Version since 7 - assertEquals(Version.fromId(5000099), v); - called.set(true); - }); - assertTrue(called.get()); - called.set(false); - sourceWithMockedRemoteCall(false, ContentType.APPLICATION_JSON, "main/with_unknown_fields.json").lookupRemoteVersion(v -> { - // V_5_0_0 since we no longer consider qualifier in Version - assertEquals(Version.fromId(5000099), v); + sourceWithMockedRemoteCall(false, ContentType.APPLICATION_JSON, s).lookupRemoteVersion(v -> { + assertEquals(expected, v); called.set(true); }); assertTrue(called.get()); diff --git a/modules/reindex/src/test/resources/rest-api-spec/test/delete_by_query/10_basic.yml b/modules/reindex/src/test/resources/rest-api-spec/test/delete_by_query/10_basic.yml index ded91a095f611..1763baebe0277 100644 --- a/modules/reindex/src/test/resources/rest-api-spec/test/delete_by_query/10_basic.yml +++ b/modules/reindex/src/test/resources/rest-api-spec/test/delete_by_query/10_basic.yml @@ -280,7 +280,11 @@ - match: {count: 1} --- -"Limit by size": +"Limit by size pre 7.3": + - skip: + version: "7.3.0 - " + reason: "7.3 should use max_docs or get deprecation warning" + - do: index: index: twitter @@ -318,6 +322,93 @@ - match: {count: 1} +--- +"Limit by max_docs in URL": + - skip: + version: " - 7.2.99" + reason: "max_docs introduced in 7.3.0" + + - do: + index: + index: twitter + id: 1 + body: { "user": "kimchy" } + - do: + index: + index: twitter + id: 2 + body: { "user": "kimchy" } + - do: + indices.refresh: {} + + - do: + delete_by_query: + index: twitter + max_docs: 1 + body: + query: + match_all: {} + + - match: {deleted: 1} + - match: {version_conflicts: 0} + - match: {batches: 1} + - match: {failures: []} + - match: {throttled_millis: 0} + - gte: { took: 0 } + + - do: + indices.refresh: {} + + - do: + count: + index: twitter + + - match: {count: 1} + +--- +"Limit by max_docs in body": + - skip: + version: " - 7.2.99" + reason: "max_docs introduced in 7.3.0" + + - do: + index: + index: twitter + id: 1 + body: { "user": "kimchy" } + - do: + index: + index: twitter + id: 2 + body: { "user": "kimchy" } + - do: + indices.refresh: {} + + - do: + delete_by_query: + index: twitter + body: + max_docs: 1 + query: + match: + user: "kimchy" + + - match: {deleted: 1} + - match: {version_conflicts: 0} + - match: {batches: 1} + - match: {failures: []} + - match: {throttled_millis: 0} + - gte: { took: 0 } + + - do: + indices.refresh: {} + + - do: + count: + index: twitter + + - match: {count: 1} + --- "Can override scroll_size": - do: diff --git a/modules/reindex/src/test/resources/rest-api-spec/test/delete_by_query/20_validation.yml b/modules/reindex/src/test/resources/rest-api-spec/test/delete_by_query/20_validation.yml index 3caa8c4b43405..45de10b370669 100644 --- a/modules/reindex/src/test/resources/rest-api-spec/test/delete_by_query/20_validation.yml +++ b/modules/reindex/src/test/resources/rest-api-spec/test/delete_by_query/20_validation.yml @@ -31,17 +31,21 @@ match_all: {} --- -"invalid size fails": +"invalid max_docs fails": + - skip: + version: " - 7.2.99" + reason: "max_docs introduced in 7.3.0" + - do: index: index: test id: 1 body: { "text": "test" } - do: - catch: /\[size\] parameter cannot be negative, found \[-4\]/ + catch: /\[max_docs\] parameter cannot be negative, found \[-4\]/ delete_by_query: index: test - size: -4 + max_docs: -4 body: query: match_all: {} diff --git a/modules/reindex/src/test/resources/rest-api-spec/test/reindex/20_validation.yml b/modules/reindex/src/test/resources/rest-api-spec/test/reindex/20_validation.yml index d45b14a1edc5f..44971c49c358a 100644 --- a/modules/reindex/src/test/resources/rest-api-spec/test/reindex/20_validation.yml +++ b/modules/reindex/src/test/resources/rest-api-spec/test/reindex/20_validation.yml @@ -95,21 +95,90 @@ conflicts: cat --- -"invalid size fails": +"specifying size fails": + - skip: + version: " - 7.99.99" + reason: "size supported until 8" + + - do: + index: + index: test + id: 1 + body: { "text": "test" } + + - do: + catch: /invalid parameter \[size\], use \[max_docs\] instead/ + reindex: + body: + source: + index: test + dest: + index: dest + size: 1 + +--- +"invalid max_docs in body fails": + - skip: + version: " - 7.2.99" + reason: "max_docs introduced in 7.3.0" + + - do: + index: + index: test + id: 1 + body: { "text": "test" } + - do: + catch: /\[max_docs\] parameter cannot be negative, found \[-4\]/ + reindex: + body: + source: + index: test + dest: + index: dest + max_docs: -4 + +--- +"invalid max_docs in URL fails": + - skip: + version: " - 7.2.99" + reason: "max_docs introduced in 7.3.0" + + - do: + index: + index: test + id: 1 + body: { "text": "test" } + - do: + catch: /\[max_docs\] parameter cannot be negative, found \[-4\]/ + reindex: + max_docs: -4 + body: + source: + index: test + dest: + index: dest + +--- +"inconsistent max_docs in body and max_docs in URL fails": + - skip: + version: " - 7.2.99" + reason: "max_docs introduced in 7.3.0" + - do: index: index: test id: 1 body: { "text": "test" } - do: - catch: /\[size\] parameter cannot be negative, found \[-4\]/ + catch: /\[max_docs\] set to two different values \[4\] and \[5\]/ reindex: + max_docs: 5 body: source: index: test dest: index: dest - size: -4 + max_docs: 4 --- "requests_per_second cannot be an empty string": diff --git a/modules/reindex/src/test/resources/rest-api-spec/test/reindex/30_search.yml b/modules/reindex/src/test/resources/rest-api-spec/test/reindex/30_search.yml index dff24e4413368..709b9c0d17340 100644 --- a/modules/reindex/src/test/resources/rest-api-spec/test/reindex/30_search.yml +++ b/modules/reindex/src/test/resources/rest-api-spec/test/reindex/30_search.yml @@ -32,7 +32,11 @@ - match: { hits.total: 1 } --- -"Sorting and size combined": +"Sorting and size combined pre 7.3": + - skip: + version: "7.3.0 - " + reason: "7.3 should use max_docs or get deprecation warning" + - do: index: index: test @@ -69,3 +73,81 @@ index: target q: order:1 - match: { hits.total: 1 } + +--- +"Sorting and max_docs in body combined": + - skip: + version: " - 7.2.99" + reason: "max_docs introduced in 7.3.0" + + - do: + index: + index: test + id: 1 + body: { "order": 1 } + - do: + index: + index: test + id: 2 + body: { "order": 2 } + - do: + indices.refresh: {} + + - do: + reindex: + refresh: true + body: + max_docs: 1 + source: + index: test + sort: order + dest: + index: target + + - do: + search: + rest_total_hits_as_int: true + index: target + - match: { hits.total: 1 } + + - do: + search: + rest_total_hits_as_int: true + index: target + q: order:1 + - match: { hits.total: 1 } + +--- +"max_docs in URL": + - skip: + version: " - 7.2.99" + reason: "max_docs introduced in 7.3.0" + + - do: + index: + index: test + id: 1 + body: { "value": 17 } + - do: + index: + index: test + id: 2 + body: { "value": 17 } + - do: + indices.refresh: {} + + - do: + reindex: + refresh: true + max_docs: 1 + body: + source: + index: test + dest: + index: target + + - do: + search: + rest_total_hits_as_int: true + index: target + - match: { hits.total: 1 } diff --git a/modules/reindex/src/test/resources/rest-api-spec/test/reindex/90_remote.yml b/modules/reindex/src/test/resources/rest-api-spec/test/reindex/90_remote.yml index 65d6438e75d28..8354fc0aaf322 100644 --- a/modules/reindex/src/test/resources/rest-api-spec/test/reindex/90_remote.yml +++ b/modules/reindex/src/test/resources/rest-api-spec/test/reindex/90_remote.yml @@ -218,7 +218,11 @@ - match: {indices.source.total.search.open_contexts: 0} --- -"Reindex from remote with size": +"Reindex from remote with max_docs": + - skip: + version: " - 7.2.99" + reason: "max_docs introduced in 7.3.0" + - do: index: index: source @@ -245,7 +249,7 @@ reindex: refresh: true body: - size: 1 + max_docs: 1 source: remote: host: http://${host} diff --git a/modules/reindex/src/test/resources/rest-api-spec/test/update_by_query/10_basic.yml b/modules/reindex/src/test/resources/rest-api-spec/test/update_by_query/10_basic.yml index 40119f22abb5b..2a3696a4005c7 100644 --- a/modules/reindex/src/test/resources/rest-api-spec/test/update_by_query/10_basic.yml +++ b/modules/reindex/src/test/resources/rest-api-spec/test/update_by_query/10_basic.yml @@ -218,7 +218,11 @@ - gte: { took: 0 } --- -"Limit by size": +"Limit by size pre 7.3": + - skip: + version: "7.3.0 - " + reason: "7.3 should use max_docs or get deprecation warning" + - do: index: index: twitter @@ -243,6 +247,70 @@ - match: {throttled_millis: 0} - gte: { took: 0 } +--- +"Limit by max_docs in URL": + - skip: + version: " - 7.2.99" + reason: "max_docs introduced in 7.3.0" + + - do: + index: + index: twitter + id: 1 + body: { "user": "kimchy" } + - do: + index: + index: twitter + id: 2 + body: { "user": "kimchy" } + - do: + indices.refresh: {} + + - do: + update_by_query: + index: twitter + max_docs: 1 + - match: {updated: 1} + - match: {version_conflicts: 0} + - match: {batches: 1} + - match: {failures: []} + - match: {throttled_millis: 0} + - gte: { took: 0 } + +--- +"Limit by max_docs in body": + - skip: + version: " - 7.2.99" + reason: "max_docs introduced in 7.3.0" + + - do: + index: + index: twitter + id: 1 + body: { "user": "kimchy" } + - do: + index: + index: twitter + id: 2 + body: { "user": "kimchy" } + - do: + indices.refresh: {} + + - do: + update_by_query: + index: twitter + body: + max_docs: 1 + query: + match: + user: "kimchy" + - match: {updated: 1} + - match: {version_conflicts: 0} + - match: {batches: 1} + - match: {failures: []} + - match: {throttled_millis: 0} + - gte: { took: 0 } + --- "Can override scroll_size": - do: diff --git a/modules/reindex/src/test/resources/rest-api-spec/test/update_by_query/20_validation.yml b/modules/reindex/src/test/resources/rest-api-spec/test/update_by_query/20_validation.yml index b6823f88c15fc..21644b3932984 100644 --- a/modules/reindex/src/test/resources/rest-api-spec/test/update_by_query/20_validation.yml +++ b/modules/reindex/src/test/resources/rest-api-spec/test/update_by_query/20_validation.yml @@ -12,17 +12,60 @@ conflicts: cat --- -"invalid size fails": +"invalid max_docs in URL fails": + - skip: + version: " - 7.2.99" + reason: "max_docs introduced in 7.3.0" + + - do: + index: + index: test + id: 1 + body: { "text": "test" } + - do: + catch: /\[max_docs\] parameter cannot be negative, found \[-4\]/ + update_by_query: + index: test + max_docs: -4 + +--- +"invalid max_docs in body fails": + - skip: + version: " - 7.2.99" + reason: "max_docs introduced in 7.3.0" + - do: index: index: test id: 1 body: { "text": "test" } - do: - catch: /\[size\] parameter cannot be negative, found \[-4\]/ + catch: /\[max_docs\] parameter cannot be negative, found \[-4\]/ update_by_query: index: test - size: -4 + body: + max_docs: -4 + +--- +"inconsistent max_docs in body and max_docs in URL fails": + - skip: + version: " - 7.2.99" + reason: "max_docs introduced in 7.3.0" + + - do: + index: + index: test + id: 1 + body: { "text": "test" } + - do: + catch: /\[max_docs\] set to two different values \[4\] and \[5\]/ + delete_by_query: + index: test + max_docs: 5 + body: + max_docs: 4 + query: + match_all: {} --- "invalid scroll_size fails": diff --git a/modules/repository-url/src/main/java/org/elasticsearch/common/blobstore/url/URLBlobContainer.java b/modules/repository-url/src/main/java/org/elasticsearch/common/blobstore/url/URLBlobContainer.java index 91676037043e3..f9406e454474f 100644 --- a/modules/repository-url/src/main/java/org/elasticsearch/common/blobstore/url/URLBlobContainer.java +++ b/modules/repository-url/src/main/java/org/elasticsearch/common/blobstore/url/URLBlobContainer.java @@ -20,6 +20,7 @@ package org.elasticsearch.common.blobstore.url; import org.elasticsearch.common.SuppressForbidden; +import org.elasticsearch.common.blobstore.BlobContainer; import org.elasticsearch.common.blobstore.BlobMetaData; import org.elasticsearch.common.blobstore.BlobPath; import org.elasticsearch.common.blobstore.support.AbstractBlobContainer; @@ -74,6 +75,11 @@ public Map listBlobs() throws IOException { throw new UnsupportedOperationException("URL repository doesn't support this operation"); } + @Override + public Map children() throws IOException { + throw new UnsupportedOperationException("URL repository doesn't support this operation"); + } + /** * This operation is not supported by URLBlobContainer */ @@ -90,6 +96,11 @@ public void deleteBlob(String blobName) throws IOException { throw new UnsupportedOperationException("URL repository is read only"); } + @Override + public void delete() { + throw new UnsupportedOperationException("URL repository is read only"); + } + /** * This operation is not supported by URLBlobContainer */ diff --git a/modules/repository-url/src/main/java/org/elasticsearch/repositories/url/URLRepository.java b/modules/repository-url/src/main/java/org/elasticsearch/repositories/url/URLRepository.java index 0ea2a1b72c574..a27b091cfc037 100644 --- a/modules/repository-url/src/main/java/org/elasticsearch/repositories/url/URLRepository.java +++ b/modules/repository-url/src/main/java/org/elasticsearch/repositories/url/URLRepository.java @@ -75,8 +75,6 @@ public class URLRepository extends BlobStoreRepository { private final Environment environment; - private final BlobPath basePath; - private final URL url; /** @@ -84,7 +82,7 @@ public class URLRepository extends BlobStoreRepository { */ public URLRepository(RepositoryMetaData metadata, Environment environment, NamedXContentRegistry namedXContentRegistry, ThreadPool threadPool) { - super(metadata, environment.settings(), namedXContentRegistry, threadPool); + super(metadata, environment.settings(), namedXContentRegistry, threadPool, BlobPath.cleanPath()); if (URL_SETTING.exists(metadata.settings()) == false && REPOSITORIES_URL_SETTING.exists(environment.settings()) == false) { throw new RepositoryException(metadata.name(), "missing url"); @@ -92,7 +90,6 @@ public URLRepository(RepositoryMetaData metadata, Environment environment, this.environment = environment; supportedProtocols = SUPPORTED_PROTOCOLS_SETTING.get(environment.settings()); urlWhiteList = ALLOWED_URLS_SETTING.get(environment.settings()).toArray(new URIPattern[]{}); - basePath = BlobPath.cleanPath(); url = URL_SETTING.exists(metadata.settings()) ? URL_SETTING.get(metadata.settings()) : REPOSITORIES_URL_SETTING.get(environment.settings()); } @@ -115,11 +112,6 @@ protected BlobStore getBlobStore() { return super.getBlobStore(); } - @Override - protected BlobPath basePath() { - return basePath; - } - /** * Makes sure that the url is white listed or if it points to the local file system it matches one on of the root path in path.repo */ diff --git a/modules/repository-url/src/test/java/org/elasticsearch/repositories/url/URLSnapshotRestoreTests.java b/modules/repository-url/src/test/java/org/elasticsearch/repositories/url/URLSnapshotRestoreTests.java index ab9268b081456..160333d5346ed 100644 --- a/modules/repository-url/src/test/java/org/elasticsearch/repositories/url/URLSnapshotRestoreTests.java +++ b/modules/repository-url/src/test/java/org/elasticsearch/repositories/url/URLSnapshotRestoreTests.java @@ -88,7 +88,7 @@ public void testUrlRepository() throws Exception { .prepareGetSnapshots("test-repo") .setSnapshots("test-snap") .get() - .getSnapshots() + .getSnapshots("test-repo") .get(0) .state(); assertThat(state, equalTo(SnapshotState.SUCCESS)); @@ -116,8 +116,8 @@ public void testUrlRepository() throws Exception { logger.info("--> list available shapshots"); GetSnapshotsResponse getSnapshotsResponse = client.admin().cluster().prepareGetSnapshots("url-repo").get(); - assertThat(getSnapshotsResponse.getSnapshots(), notNullValue()); - assertThat(getSnapshotsResponse.getSnapshots().size(), equalTo(1)); + assertThat(getSnapshotsResponse.getSnapshots("url-repo"), notNullValue()); + assertThat(getSnapshotsResponse.getSnapshots("url-repo").size(), equalTo(1)); logger.info("--> delete snapshot"); AcknowledgedResponse deleteSnapshotResponse = client.admin().cluster().prepareDeleteSnapshot("test-repo", "test-snap").get(); @@ -125,7 +125,7 @@ public void testUrlRepository() throws Exception { logger.info("--> list available shapshot again, no snapshots should be returned"); getSnapshotsResponse = client.admin().cluster().prepareGetSnapshots("url-repo").get(); - assertThat(getSnapshotsResponse.getSnapshots(), notNullValue()); - assertThat(getSnapshotsResponse.getSnapshots().size(), equalTo(0)); + assertThat(getSnapshotsResponse.getSnapshots("url-repo"), notNullValue()); + assertThat(getSnapshotsResponse.getSnapshots("url-repo").size(), equalTo(0)); } } diff --git a/modules/repository-url/src/test/resources/rest-api-spec/test/repository_url/10_basic.yml b/modules/repository-url/src/test/resources/rest-api-spec/test/repository_url/10_basic.yml index b932f0d53caad..e76669dc75e01 100644 --- a/modules/repository-url/src/test/resources/rest-api-spec/test/repository_url/10_basic.yml +++ b/modules/repository-url/src/test/resources/rest-api-spec/test/repository_url/10_basic.yml @@ -112,6 +112,9 @@ teardown: --- "Restore with repository-url using http://": + - skip: + version: " - 7.9.99" + reason: "8.0 changes get snapshots response format" # Ensure that the URL repository is registered - do: @@ -126,9 +129,9 @@ teardown: repository: repository-url snapshot: snapshot-one,snapshot-two - - is_true: snapshots - - match: { snapshots.0.state : SUCCESS } - - match: { snapshots.1.state : SUCCESS } + - is_true: responses.0.snapshots + - match: { responses.0.snapshots.0.state : SUCCESS } + - match: { responses.0.snapshots.1.state : SUCCESS } # Delete the index - do: @@ -174,6 +177,9 @@ teardown: --- "Restore with repository-url using file://": + - skip: + version: " - 7.9.99" + reason: "8.0 changes get snapshots response format" # Ensure that the URL repository is registered - do: @@ -188,9 +194,9 @@ teardown: repository: repository-file snapshot: snapshot-one,snapshot-two - - is_true: snapshots - - match: { snapshots.0.state : SUCCESS } - - match: { snapshots.1.state : SUCCESS } + - is_true: responses.0.snapshots + - match: { responses.0.snapshots.0.state : SUCCESS } + - match: { responses.0.snapshots.1.state : SUCCESS } # Delete the index - do: @@ -236,13 +242,18 @@ teardown: --- "Get a non existing snapshot": + - skip: + version: " - 7.9.99" + reason: "8.0 changes get snapshots response format" - do: - catch: /snapshot_missing_exception/ snapshot.get: repository: repository-url snapshot: missing + - is_true: responses.0.error + - match: { responses.0.error.type: snapshot_missing_exception } + --- "Delete a non existing snapshot": diff --git a/modules/transport-netty4/build.gradle b/modules/transport-netty4/build.gradle index d64e0aff7749d..49d9f3693fdc7 100644 --- a/modules/transport-netty4/build.gradle +++ b/modules/transport-netty4/build.gradle @@ -26,7 +26,6 @@ esplugin { description 'Netty 4 based transport implementation' classname 'org.elasticsearch.transport.Netty4Plugin' - hasClientJar = true } compileTestJava.options.compilerArgs << "-Xlint:-cast,-rawtypes,-unchecked" @@ -76,6 +75,10 @@ thirdPartyAudit { 'org.apache.commons.logging.Log', 'org.apache.commons.logging.LogFactory', + // from Log4j (deliberate, Netty will fallback to Log4j 2) + 'org.apache.log4j.Level', + 'org.apache.log4j.Logger', + // from io.netty.handler.ssl.OpenSslEngine (netty) 'io.netty.internal.tcnative.Buffer', 'io.netty.internal.tcnative.Library', diff --git a/modules/transport-netty4/src/main/java/org/elasticsearch/http/netty4/Netty4HttpServerTransport.java b/modules/transport-netty4/src/main/java/org/elasticsearch/http/netty4/Netty4HttpServerTransport.java index 356cfa0bbf99d..ad1dac038f4a9 100644 --- a/modules/transport-netty4/src/main/java/org/elasticsearch/http/netty4/Netty4HttpServerTransport.java +++ b/modules/transport-netty4/src/main/java/org/elasticsearch/http/netty4/Netty4HttpServerTransport.java @@ -23,8 +23,8 @@ import io.netty.channel.Channel; import io.netty.channel.ChannelFuture; import io.netty.channel.ChannelHandler; -import io.netty.channel.ChannelHandlerAdapter; import io.netty.channel.ChannelHandlerContext; +import io.netty.channel.ChannelInboundHandlerAdapter; import io.netty.channel.ChannelInitializer; import io.netty.channel.ChannelOption; import io.netty.channel.FixedRecvByteBufAllocator; @@ -34,7 +34,6 @@ import io.netty.handler.codec.ByteToMessageDecoder; import io.netty.handler.codec.http.HttpContentCompressor; import io.netty.handler.codec.http.HttpContentDecompressor; -import io.netty.handler.codec.http.HttpMethod; import io.netty.handler.codec.http.HttpObjectAggregator; import io.netty.handler.codec.http.HttpRequestDecoder; import io.netty.handler.codec.http.HttpResponseEncoder; @@ -44,12 +43,10 @@ import org.apache.logging.log4j.LogManager; import org.apache.logging.log4j.Logger; import org.elasticsearch.ExceptionsHelper; -import org.elasticsearch.common.Strings; import org.elasticsearch.common.network.NetworkService; import org.elasticsearch.common.settings.Setting; import org.elasticsearch.common.settings.Setting.Property; import org.elasticsearch.common.settings.Settings; -import org.elasticsearch.common.settings.SettingsException; import org.elasticsearch.common.unit.ByteSizeUnit; import org.elasticsearch.common.unit.ByteSizeValue; import org.elasticsearch.common.util.BigArrays; @@ -60,26 +57,14 @@ import org.elasticsearch.http.HttpHandlingSettings; import org.elasticsearch.http.HttpReadTimeoutException; import org.elasticsearch.http.HttpServerChannel; -import org.elasticsearch.http.netty4.cors.Netty4CorsConfig; -import org.elasticsearch.http.netty4.cors.Netty4CorsConfigBuilder; import org.elasticsearch.http.netty4.cors.Netty4CorsHandler; -import org.elasticsearch.rest.RestUtils; import org.elasticsearch.threadpool.ThreadPool; import org.elasticsearch.transport.netty4.Netty4Utils; import java.net.InetSocketAddress; -import java.util.Arrays; import java.util.concurrent.TimeUnit; -import java.util.regex.Pattern; -import java.util.regex.PatternSyntaxException; import static org.elasticsearch.common.util.concurrent.EsExecutors.daemonThreadFactory; -import static org.elasticsearch.http.HttpTransportSettings.SETTING_CORS_ALLOW_CREDENTIALS; -import static org.elasticsearch.http.HttpTransportSettings.SETTING_CORS_ALLOW_HEADERS; -import static org.elasticsearch.http.HttpTransportSettings.SETTING_CORS_ALLOW_METHODS; -import static org.elasticsearch.http.HttpTransportSettings.SETTING_CORS_ALLOW_ORIGIN; -import static org.elasticsearch.http.HttpTransportSettings.SETTING_CORS_ENABLED; -import static org.elasticsearch.http.HttpTransportSettings.SETTING_CORS_MAX_AGE; import static org.elasticsearch.http.HttpTransportSettings.SETTING_HTTP_MAX_CHUNK_SIZE; import static org.elasticsearch.http.HttpTransportSettings.SETTING_HTTP_MAX_CONTENT_LENGTH; import static org.elasticsearch.http.HttpTransportSettings.SETTING_HTTP_MAX_HEADER_SIZE; @@ -91,15 +76,10 @@ import static org.elasticsearch.http.HttpTransportSettings.SETTING_HTTP_TCP_REUSE_ADDRESS; import static org.elasticsearch.http.HttpTransportSettings.SETTING_HTTP_TCP_SEND_BUFFER_SIZE; import static org.elasticsearch.http.HttpTransportSettings.SETTING_PIPELINING_MAX_EVENTS; -import static org.elasticsearch.http.netty4.cors.Netty4CorsHandler.ANY_ORIGIN; public class Netty4HttpServerTransport extends AbstractHttpServerTransport { private static final Logger logger = LogManager.getLogger(Netty4HttpServerTransport.class); - static { - Netty4Utils.setup(); - } - /* * Size in bytes of an individual message received by io.netty.handler.codec.MessageAggregator which accumulates the content for an * HTTP request. This number is used for estimating the maximum number of allowed buffers before the MessageAggregator's internal @@ -160,8 +140,6 @@ public class Netty4HttpServerTransport extends AbstractHttpServerTransport { protected volatile ServerBootstrap serverBootstrap; - private final Netty4CorsConfig corsConfig; - public Netty4HttpServerTransport(Settings settings, NetworkService networkService, BigArrays bigArrays, ThreadPool threadPool, NamedXContentRegistry xContentRegistry, Dispatcher dispatcher) { super(settings, networkService, bigArrays, threadPool, xContentRegistry, dispatcher); @@ -180,8 +158,6 @@ public Netty4HttpServerTransport(Settings settings, NetworkService networkServic ByteSizeValue receivePredictor = SETTING_HTTP_NETTY_RECEIVE_PREDICTOR_SIZE.get(settings); recvByteBufAllocator = new FixedRecvByteBufAllocator(receivePredictor.bytesAsInt()); - this.corsConfig = buildCorsConfig(settings); - logger.debug("using max_chunk_size[{}], max_header_size[{}], max_initial_line_length[{}], max_content_length[{}], " + "receive_predictor[{}], max_composite_buffer_components[{}], pipelining_max_events[{}]", maxChunkSize, maxHeaderSize, maxInitialLineLength, maxContentLength, receivePredictor, maxCompositeBufferComponents, @@ -234,43 +210,6 @@ protected void doStart() { } } - // package private for testing - static Netty4CorsConfig buildCorsConfig(Settings settings) { - if (SETTING_CORS_ENABLED.get(settings) == false) { - return Netty4CorsConfigBuilder.forOrigins().disable().build(); - } - String origin = SETTING_CORS_ALLOW_ORIGIN.get(settings); - final Netty4CorsConfigBuilder builder; - if (Strings.isNullOrEmpty(origin)) { - builder = Netty4CorsConfigBuilder.forOrigins(); - } else if (origin.equals(ANY_ORIGIN)) { - builder = Netty4CorsConfigBuilder.forAnyOrigin(); - } else { - try { - Pattern p = RestUtils.checkCorsSettingForRegex(origin); - if (p == null) { - builder = Netty4CorsConfigBuilder.forOrigins(RestUtils.corsSettingAsArray(origin)); - } else { - builder = Netty4CorsConfigBuilder.forPattern(p); - } - } catch (PatternSyntaxException e) { - throw new SettingsException("Bad regex in [" + SETTING_CORS_ALLOW_ORIGIN.getKey() + "]: [" + origin + "]", e); - } - } - if (SETTING_CORS_ALLOW_CREDENTIALS.get(settings)) { - builder.allowCredentials(); - } - String[] strMethods = Strings.tokenizeToStringArray(SETTING_CORS_ALLOW_METHODS.get(settings), ","); - HttpMethod[] methods = Arrays.stream(strMethods) - .map(HttpMethod::valueOf) - .toArray(HttpMethod[]::new); - return builder.allowedRequestMethods(methods) - .maxAge(SETTING_CORS_MAX_AGE.get(settings)) - .allowedRequestHeaders(Strings.tokenizeToStringArray(SETTING_CORS_ALLOW_HEADERS.get(settings), ",")) - .shortCircuit() - .build(); - } - @Override protected HttpServerChannel bind(InetSocketAddress socketAddress) throws Exception { ChannelFuture future = serverBootstrap.bind(socketAddress).sync(); @@ -351,7 +290,7 @@ public void exceptionCaught(ChannelHandlerContext ctx, Throwable cause) throws E } @ChannelHandler.Sharable - private static class ServerChannelExceptionHandler extends ChannelHandlerAdapter { + private static class ServerChannelExceptionHandler extends ChannelInboundHandlerAdapter { private final Netty4HttpServerTransport transport; diff --git a/modules/transport-netty4/src/main/java/org/elasticsearch/http/netty4/cors/Netty4CorsConfig.java b/modules/transport-netty4/src/main/java/org/elasticsearch/http/netty4/cors/Netty4CorsConfig.java deleted file mode 100644 index 17e21ce468b4c..0000000000000 --- a/modules/transport-netty4/src/main/java/org/elasticsearch/http/netty4/cors/Netty4CorsConfig.java +++ /dev/null @@ -1,221 +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.http.netty4.cors; - -import io.netty.handler.codec.http.DefaultHttpHeaders; -import io.netty.handler.codec.http.EmptyHttpHeaders; -import io.netty.handler.codec.http.HttpHeaders; -import io.netty.handler.codec.http.HttpMethod; - -import java.util.Collections; -import java.util.LinkedHashSet; -import java.util.Map; -import java.util.Optional; -import java.util.Set; -import java.util.concurrent.Callable; -import java.util.regex.Pattern; - -/** - * Configuration for Cross-Origin Resource Sharing (CORS). - * - * This class was lifted from the Netty project: - * https://github.com/netty/netty - */ -public final class Netty4CorsConfig { - - private final Optional> origins; - private final Optional pattern; - private final boolean anyOrigin; - private final boolean enabled; - private final boolean allowCredentials; - private final long maxAge; - private final Set allowedRequestMethods; - private final Set allowedRequestHeaders; - private final Map> preflightHeaders; - private final boolean shortCircuit; - - Netty4CorsConfig(final Netty4CorsConfigBuilder builder) { - origins = builder.origins.map(s -> new LinkedHashSet<>(s)); - pattern = builder.pattern; - anyOrigin = builder.anyOrigin; - enabled = builder.enabled; - allowCredentials = builder.allowCredentials; - maxAge = builder.maxAge; - allowedRequestMethods = builder.requestMethods; - allowedRequestHeaders = builder.requestHeaders; - preflightHeaders = builder.preflightHeaders; - shortCircuit = builder.shortCircuit; - } - - /** - * Determines if support for CORS is enabled. - * - * @return {@code true} if support for CORS is enabled, false otherwise. - */ - public boolean isCorsSupportEnabled() { - return enabled; - } - - /** - * Determines whether a wildcard origin, '*', is supported. This also means that null origins are - * supported. - * - * @return {@code boolean} true if any origin is allowed. - */ - public boolean isAnyOriginSupported() { - return anyOrigin; - } - - /** - * Returns the set of allowed origins. - * - * @return {@code Set} the allowed origins. - */ - public Optional> origins() { - return origins; - } - - /** - * Returns whether the input origin is allowed by this configuration. - * - * @return {@code true} if the origin is allowed, otherwise {@code false} - */ - public boolean isOriginAllowed(final String origin) { - if (origins.isPresent()) { - return origins.get().contains(origin); - } else if (pattern.isPresent()) { - return pattern.get().matcher(origin).matches(); - } - return false; - } - - /** - * Determines if credentials are supported for CORS requests. - * - * By default credentials are not included in CORS requests but if isCredentialsAllowed returns - * true credentials will be added to CORS requests. Setting this value to true will set the - * CORS 'Access-Control-Allow-Credentials' response header to true. - * - * Please note that credentials support needs to be enabled on the client side as well. - * The client needs to opt-in to send credentials by calling: - *
    -     * xhr.withCredentials = true;
    -     * 
    - * The default value for 'withCredentials' is false in which case no credentials are sent. - * Setting this to true will included credentials in cross origin requests. - * - * @return {@code true} if credentials are supported. - */ - public boolean isCredentialsAllowed() { - return allowCredentials; - } - - /** - * Gets the maxAge setting. - * - * When making a preflight request the client has to perform two request with can be inefficient. - * This setting will set the CORS 'Access-Control-Max-Age' response header and enables the - * caching of the preflight response for the specified time. During this time no preflight - * request will be made. - * - * @return {@code long} the time in seconds that a preflight request may be cached. - */ - public long maxAge() { - return maxAge; - } - - /** - * Returns the allowed set of Request Methods. The Http methods that should be returned in the - * CORS 'Access-Control-Request-Method' response header. - * - * @return {@code Set} of {@link HttpMethod}s that represent the allowed Request Methods. - */ - public Set allowedRequestMethods() { - return Collections.unmodifiableSet(allowedRequestMethods); - } - - /** - * Returns the allowed set of Request Headers. - * - * The header names returned from this method will be used to set the CORS - * 'Access-Control-Allow-Headers' response header. - * - * @return {@code Set} of strings that represent the allowed Request Headers. - */ - public Set allowedRequestHeaders() { - return Collections.unmodifiableSet(allowedRequestHeaders); - } - - /** - * Returns HTTP response headers that should be added to a CORS preflight response. - * - * @return {@link HttpHeaders} the HTTP response headers to be added. - */ - public HttpHeaders preflightResponseHeaders() { - if (preflightHeaders.isEmpty()) { - return EmptyHttpHeaders.INSTANCE; - } - final HttpHeaders preflightHeaders = new DefaultHttpHeaders(); - for (Map.Entry> entry : this.preflightHeaders.entrySet()) { - final Object value = getValue(entry.getValue()); - if (value instanceof Iterable) { - preflightHeaders.add(entry.getKey().toString(), (Iterable) value); - } else { - preflightHeaders.add(entry.getKey().toString(), value); - } - } - return preflightHeaders; - } - - /** - * Determines whether a CORS request should be rejected if it's invalid before being - * further processing. - * - * CORS headers are set after a request is processed. This may not always be desired - * and this setting will check that the Origin is valid and if it is not valid no - * further processing will take place, and a error will be returned to the calling client. - * - * @return {@code true} if a CORS request should short-circuit upon receiving an invalid Origin header. - */ - public boolean isShortCircuit() { - return shortCircuit; - } - - private static T getValue(final Callable callable) { - try { - return callable.call(); - } catch (final Exception e) { - throw new IllegalStateException("Could not generate value for callable [" + callable + ']', e); - } - } - - @Override - public String toString() { - return "CorsConfig[enabled=" + enabled + - ", origins=" + origins + - ", anyOrigin=" + anyOrigin + - ", isCredentialsAllowed=" + allowCredentials + - ", maxAge=" + maxAge + - ", allowedRequestMethods=" + allowedRequestMethods + - ", allowedRequestHeaders=" + allowedRequestHeaders + - ", preflightHeaders=" + preflightHeaders + ']'; - } - -} diff --git a/modules/transport-netty4/src/main/java/org/elasticsearch/http/netty4/cors/Netty4CorsConfigBuilder.java b/modules/transport-netty4/src/main/java/org/elasticsearch/http/netty4/cors/Netty4CorsConfigBuilder.java deleted file mode 100644 index 3e87d948dabf4..0000000000000 --- a/modules/transport-netty4/src/main/java/org/elasticsearch/http/netty4/cors/Netty4CorsConfigBuilder.java +++ /dev/null @@ -1,265 +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.http.netty4.cors; - -import io.netty.handler.codec.http.HttpMethod; - -import java.util.Arrays; -import java.util.Date; -import java.util.HashMap; -import java.util.HashSet; -import java.util.LinkedHashSet; -import java.util.Map; -import java.util.Optional; -import java.util.Set; -import java.util.concurrent.Callable; -import java.util.regex.Pattern; - -/** - * Builder used to configure and build a {@link Netty4CorsConfig} instance. - * - * This class was lifted from the Netty project: - * https://github.com/netty/netty - */ -public final class Netty4CorsConfigBuilder { - - /** - * Creates a Builder instance with it's origin set to '*'. - * - * @return Builder to support method chaining. - */ - public static Netty4CorsConfigBuilder forAnyOrigin() { - return new Netty4CorsConfigBuilder(); - } - - /** - * Create a {@link Netty4CorsConfigBuilder} instance with the specified pattern origin. - * - * @param pattern the regular expression pattern to match incoming origins on. - * @return {@link Netty4CorsConfigBuilder} with the configured origin pattern. - */ - public static Netty4CorsConfigBuilder forPattern(final Pattern pattern) { - if (pattern == null) { - throw new IllegalArgumentException("CORS pattern cannot be null"); - } - return new Netty4CorsConfigBuilder(pattern); - } - - /** - * Creates a {@link Netty4CorsConfigBuilder} instance with the specified origins. - * - * @return {@link Netty4CorsConfigBuilder} to support method chaining. - */ - public static Netty4CorsConfigBuilder forOrigins(final String... origins) { - return new Netty4CorsConfigBuilder(origins); - } - - Optional> origins; - Optional pattern; - final boolean anyOrigin; - boolean enabled = true; - boolean allowCredentials; - long maxAge; - final Set requestMethods = new HashSet<>(); - final Set requestHeaders = new HashSet<>(); - final Map> preflightHeaders = new HashMap<>(); - boolean shortCircuit; - - /** - * Creates a new Builder instance with the origin passed in. - * - * @param origins the origin to be used for this builder. - */ - Netty4CorsConfigBuilder(final String... origins) { - this.origins = Optional.of(new LinkedHashSet<>(Arrays.asList(origins))); - pattern = Optional.empty(); - anyOrigin = false; - } - - /** - * Creates a new Builder instance allowing any origin, "*" which is the - * wildcard origin. - * - */ - Netty4CorsConfigBuilder() { - anyOrigin = true; - origins = Optional.empty(); - pattern = Optional.empty(); - } - - /** - * Creates a new Builder instance allowing any origin that matches the pattern. - * - * @param pattern the pattern to match against for incoming origins. - */ - Netty4CorsConfigBuilder(final Pattern pattern) { - this.pattern = Optional.of(pattern); - origins = Optional.empty(); - anyOrigin = false; - } - - /** - * Disables CORS support. - * - * @return {@link Netty4CorsConfigBuilder} to support method chaining. - */ - public Netty4CorsConfigBuilder disable() { - enabled = false; - return this; - } - - /** - * By default cookies are not included in CORS requests, but this method will enable cookies to - * be added to CORS requests. Calling this method will set the CORS 'Access-Control-Allow-Credentials' - * response header to true. - * - * Please note, that cookie support needs to be enabled on the client side as well. - * The client needs to opt-in to send cookies by calling: - *
    -     * xhr.withCredentials = true;
    -     * 
    - * The default value for 'withCredentials' is false in which case no cookies are sent. - * Setting this to true will included cookies in cross origin requests. - * - * @return {@link Netty4CorsConfigBuilder} to support method chaining. - */ - public Netty4CorsConfigBuilder allowCredentials() { - allowCredentials = true; - return this; - } - - /** - * When making a preflight request the client has to perform two request with can be inefficient. - * This setting will set the CORS 'Access-Control-Max-Age' response header and enables the - * caching of the preflight response for the specified time. During this time no preflight - * request will be made. - * - * @param max the maximum time, in seconds, that the preflight response may be cached. - * @return {@link Netty4CorsConfigBuilder} to support method chaining. - */ - public Netty4CorsConfigBuilder maxAge(final long max) { - maxAge = max; - return this; - } - - /** - * Specifies the allowed set of HTTP Request Methods that should be returned in the - * CORS 'Access-Control-Request-Method' response header. - * - * @param methods the {@link HttpMethod}s that should be allowed. - * @return {@link Netty4CorsConfigBuilder} to support method chaining. - */ - public Netty4CorsConfigBuilder allowedRequestMethods(final HttpMethod... methods) { - requestMethods.addAll(Arrays.asList(methods)); - return this; - } - - /** - * Specifies the if headers that should be returned in the CORS 'Access-Control-Allow-Headers' - * response header. - * - * If a client specifies headers on the request, for example by calling: - *
    -     * xhr.setRequestHeader('My-Custom-Header', "SomeValue");
    -     * 
    - * the server will receive the above header name in the 'Access-Control-Request-Headers' of the - * preflight request. The server will then decide if it allows this header to be sent for the - * real request (remember that a preflight is not the real request but a request asking the server - * if it allow a request). - * - * @param headers the headers to be added to the preflight 'Access-Control-Allow-Headers' response header. - * @return {@link Netty4CorsConfigBuilder} to support method chaining. - */ - public Netty4CorsConfigBuilder allowedRequestHeaders(final String... headers) { - requestHeaders.addAll(Arrays.asList(headers)); - return this; - } - - /** - * Specifies that a CORS request should be rejected if it's invalid before being - * further processing. - * - * CORS headers are set after a request is processed. This may not always be desired - * and this setting will check that the Origin is valid and if it is not valid no - * further processing will take place, and a error will be returned to the calling client. - * - * @return {@link Netty4CorsConfigBuilder} to support method chaining. - */ - public Netty4CorsConfigBuilder shortCircuit() { - shortCircuit = true; - return this; - } - - /** - * Builds a {@link Netty4CorsConfig} with settings specified by previous method calls. - * - * @return {@link Netty4CorsConfig} the configured CorsConfig instance. - */ - public Netty4CorsConfig build() { - if (preflightHeaders.isEmpty()) { - preflightHeaders.put("date", DateValueGenerator.INSTANCE); - preflightHeaders.put("content-length", new ConstantValueGenerator("0")); - } - return new Netty4CorsConfig(this); - } - - /** - * This class is used for preflight HTTP response values that do not need to be - * generated, but instead the value is "static" in that the same value will be returned - * for each call. - */ - private static final class ConstantValueGenerator implements Callable { - - private final Object value; - - /** - * Sole constructor. - * - * @param value the value that will be returned when the call method is invoked. - */ - private ConstantValueGenerator(final Object value) { - if (value == null) { - throw new IllegalArgumentException("value must not be null"); - } - this.value = value; - } - - @Override - public Object call() { - return value; - } - } - - /** - * This callable is used for the DATE preflight HTTP response HTTP header. - * It's value must be generated when the response is generated, hence will be - * different for every call. - */ - private static final class DateValueGenerator implements Callable { - - static final DateValueGenerator INSTANCE = new DateValueGenerator(); - - @Override - public Date call() throws Exception { - return new Date(); - } - } - -} diff --git a/modules/transport-netty4/src/main/java/org/elasticsearch/http/netty4/cors/Netty4CorsHandler.java b/modules/transport-netty4/src/main/java/org/elasticsearch/http/netty4/cors/Netty4CorsHandler.java index 5f7baffc86aa6..1855d7c8b757e 100644 --- a/modules/transport-netty4/src/main/java/org/elasticsearch/http/netty4/cors/Netty4CorsHandler.java +++ b/modules/transport-netty4/src/main/java/org/elasticsearch/http/netty4/cors/Netty4CorsHandler.java @@ -32,31 +32,32 @@ import io.netty.handler.codec.http.HttpResponse; import io.netty.handler.codec.http.HttpResponseStatus; import org.elasticsearch.common.Strings; +import org.elasticsearch.http.CorsHandler; import org.elasticsearch.http.netty4.Netty4HttpResponse; +import java.util.Date; import java.util.regex.Pattern; import java.util.stream.Collectors; /** * Handles Cross Origin Resource Sharing (CORS) requests. *

    - * This handler can be configured using a {@link Netty4CorsConfig}, please + * This handler can be configured using a {@link CorsHandler.Config}, please * refer to this class for details about the configuration options available. * - * This code was borrowed from Netty 4 and refactored to work for Elasticsearch's Netty 3 setup. */ public class Netty4CorsHandler extends ChannelDuplexHandler { public static final String ANY_ORIGIN = "*"; private static Pattern SCHEME_PATTERN = Pattern.compile("^https?://"); - private final Netty4CorsConfig config; + private final CorsHandler.Config config; private FullHttpRequest request; /** - * Creates a new instance with the specified {@link Netty4CorsConfig}. + * Creates a new instance with the specified {@link CorsHandler.Config}. */ - public Netty4CorsHandler(final Netty4CorsConfig config) { + public Netty4CorsHandler(final CorsHandler.Config config) { if (config == null) { throw new NullPointerException(); } @@ -76,7 +77,7 @@ public void channelRead(ChannelHandlerContext ctx, Object msg) throws Exception releaseRequest(); } } - if (config.isShortCircuit() && !validateOrigin()) { + if (!validateOrigin()) { try { forbidden(ctx, request); return; @@ -96,7 +97,7 @@ public void write(ChannelHandlerContext ctx, Object msg, ChannelPromise promise) ctx.write(response, promise); } - public static void setCorsResponseHeaders(HttpRequest request, HttpResponse resp, Netty4CorsConfig config) { + public static void setCorsResponseHeaders(HttpRequest request, HttpResponse resp, CorsHandler.Config config) { if (!config.isCorsSupportEnabled()) { return; } @@ -161,7 +162,8 @@ private static boolean isSameOrigin(final String origin, final String host) { * @param response the HttpResponse to which the preflight response headers should be added. */ private void setPreflightHeaders(final HttpResponse response) { - response.headers().add(config.preflightResponseHeaders()); + response.headers().add("date", new Date()); + response.headers().add("content-length", "0"); } private boolean setOrigin(final HttpResponse response) { diff --git a/modules/transport-netty4/src/main/java/org/elasticsearch/transport/Netty4Plugin.java b/modules/transport-netty4/src/main/java/org/elasticsearch/transport/Netty4Plugin.java index 609186fc3c30e..bcfd5e0b326d3 100644 --- a/modules/transport-netty4/src/main/java/org/elasticsearch/transport/Netty4Plugin.java +++ b/modules/transport-netty4/src/main/java/org/elasticsearch/transport/Netty4Plugin.java @@ -35,7 +35,6 @@ import org.elasticsearch.plugins.Plugin; import org.elasticsearch.threadpool.ThreadPool; import org.elasticsearch.transport.netty4.Netty4Transport; -import org.elasticsearch.transport.netty4.Netty4Utils; import java.util.Arrays; import java.util.Collections; @@ -45,10 +44,6 @@ public class Netty4Plugin extends Plugin implements NetworkPlugin { - static { - Netty4Utils.setup(); - } - public static final String NETTY_TRANSPORT_NAME = "netty4"; public static final String NETTY_HTTP_TRANSPORT_NAME = "netty4"; diff --git a/modules/transport-netty4/src/main/java/org/elasticsearch/transport/netty4/Netty4InternalESLogger.java b/modules/transport-netty4/src/main/java/org/elasticsearch/transport/netty4/Netty4InternalESLogger.java deleted file mode 100644 index 4eca1803b6381..0000000000000 --- a/modules/transport-netty4/src/main/java/org/elasticsearch/transport/netty4/Netty4InternalESLogger.java +++ /dev/null @@ -1,187 +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.transport.netty4; - -import io.netty.util.internal.logging.AbstractInternalLogger; -import org.apache.logging.log4j.LogManager; -import org.apache.logging.log4j.Logger; -import org.elasticsearch.common.SuppressLoggerChecks; - -@SuppressLoggerChecks(reason = "safely delegates to logger") -class Netty4InternalESLogger extends AbstractInternalLogger { - - private final Logger logger; - - Netty4InternalESLogger(final String name) { - super(name); - this.logger = LogManager.getLogger(name); - } - - @Override - public boolean isTraceEnabled() { - return logger.isTraceEnabled(); - } - - @Override - public void trace(String msg) { - logger.trace(msg); - } - - @Override - public void trace(String format, Object arg) { - logger.trace(format, arg); - } - - @Override - public void trace(String format, Object argA, Object argB) { - logger.trace(format, argA, argB); - } - - @Override - public void trace(String format, Object... arguments) { - logger.trace(format, arguments); - } - - @Override - public void trace(String msg, Throwable t) { - logger.trace(msg, t); - } - - @Override - public boolean isDebugEnabled() { - return logger.isDebugEnabled(); - } - - @Override - public void debug(String msg) { - logger.debug(msg); - } - - @Override - public void debug(String format, Object arg) { - logger.debug(format, arg); - } - - @Override - public void debug(String format, Object argA, Object argB) { - logger.debug(format, argA, argB); - } - - @Override - public void debug(String format, Object... arguments) { - logger.debug(format, arguments); - } - - @Override - public void debug(String msg, Throwable t) { - logger.debug(msg, t); - } - - @Override - public boolean isInfoEnabled() { - return logger.isInfoEnabled(); - } - - @Override - public void info(String msg) { - logger.info(msg); - } - - @Override - public void info(String format, Object arg) { - logger.info(format, arg); - } - - @Override - public void info(String format, Object argA, Object argB) { - logger.info(format, argA, argB); - } - - @Override - public void info(String format, Object... arguments) { - logger.info(format, arguments); - } - - @Override - public void info(String msg, Throwable t) { - logger.info(msg, t); - } - - @Override - public boolean isWarnEnabled() { - return logger.isWarnEnabled(); - } - - @Override - public void warn(String msg) { - logger.warn(msg); - } - - @Override - public void warn(String format, Object arg) { - logger.warn(format, arg); - } - - @Override - public void warn(String format, Object... arguments) { - logger.warn(format, arguments); - } - - @Override - public void warn(String format, Object argA, Object argB) { - logger.warn(format, argA, argB); - } - - @Override - public void warn(String msg, Throwable t) { - logger.warn(msg, t); - } - - @Override - public boolean isErrorEnabled() { - return logger.isErrorEnabled(); - } - - @Override - public void error(String msg) { - logger.error(msg); - } - - @Override - public void error(String format, Object arg) { - logger.error(format, arg); - } - - @Override - public void error(String format, Object argA, Object argB) { - logger.error(format, argA, argB); - } - - @Override - public void error(String format, Object... arguments) { - logger.error(format, arguments); - } - - @Override - public void error(String msg, Throwable t) { - logger.error(msg, t); - } - -} diff --git a/modules/transport-netty4/src/main/java/org/elasticsearch/transport/netty4/Netty4Transport.java b/modules/transport-netty4/src/main/java/org/elasticsearch/transport/netty4/Netty4Transport.java index 34fb2173143c8..5f29c51a1ce60 100644 --- a/modules/transport-netty4/src/main/java/org/elasticsearch/transport/netty4/Netty4Transport.java +++ b/modules/transport-netty4/src/main/java/org/elasticsearch/transport/netty4/Netty4Transport.java @@ -25,8 +25,8 @@ import io.netty.channel.Channel; import io.netty.channel.ChannelFuture; import io.netty.channel.ChannelHandler; -import io.netty.channel.ChannelHandlerAdapter; import io.netty.channel.ChannelHandlerContext; +import io.netty.channel.ChannelInboundHandlerAdapter; import io.netty.channel.ChannelInitializer; import io.netty.channel.ChannelOption; import io.netty.channel.FixedRecvByteBufAllocator; @@ -78,10 +78,6 @@ public class Netty4Transport extends TcpTransport { private static final Logger logger = LogManager.getLogger(Netty4Transport.class); - static { - Netty4Utils.setup(); - } - public static final Setting WORKER_COUNT = new Setting<>("transport.netty.worker_count", (s) -> Integer.toString(EsExecutors.numberOfProcessors(s) * 2), @@ -315,7 +311,7 @@ private void addClosedExceptionLogger(Channel channel) { } @ChannelHandler.Sharable - private class ServerChannelExceptionHandler extends ChannelHandlerAdapter { + private class ServerChannelExceptionHandler extends ChannelInboundHandlerAdapter { @Override public void exceptionCaught(ChannelHandlerContext ctx, Throwable cause) { diff --git a/modules/transport-netty4/src/main/java/org/elasticsearch/transport/netty4/Netty4Utils.java b/modules/transport-netty4/src/main/java/org/elasticsearch/transport/netty4/Netty4Utils.java index 35928b6c3c8a2..211a574dc980d 100644 --- a/modules/transport-netty4/src/main/java/org/elasticsearch/transport/netty4/Netty4Utils.java +++ b/modules/transport-netty4/src/main/java/org/elasticsearch/transport/netty4/Netty4Utils.java @@ -23,8 +23,6 @@ import io.netty.buffer.CompositeByteBuf; import io.netty.buffer.Unpooled; import io.netty.util.NettyRuntime; -import io.netty.util.internal.logging.InternalLogger; -import io.netty.util.internal.logging.InternalLoggerFactory; import org.apache.lucene.util.BytesRef; import org.apache.lucene.util.BytesRefIterator; import org.elasticsearch.common.Booleans; @@ -38,21 +36,6 @@ public class Netty4Utils { - static { - InternalLoggerFactory.setDefaultFactory(new InternalLoggerFactory() { - - @Override - public InternalLogger newInstance(final String name) { - return new Netty4InternalESLogger(name); - } - - }); - } - - public static void setup() { - - } - private static AtomicBoolean isAvailableProcessorsSet = new AtomicBoolean(); /** diff --git a/modules/transport-netty4/src/test/java/org/elasticsearch/http/netty4/Netty4CorsTests.java b/modules/transport-netty4/src/test/java/org/elasticsearch/http/netty4/Netty4CorsTests.java index 115e6735eb273..0ed380d68f75e 100644 --- a/modules/transport-netty4/src/test/java/org/elasticsearch/http/netty4/Netty4CorsTests.java +++ b/modules/transport-netty4/src/test/java/org/elasticsearch/http/netty4/Netty4CorsTests.java @@ -29,6 +29,7 @@ import io.netty.handler.codec.http.HttpVersion; import org.elasticsearch.common.bytes.BytesArray; import org.elasticsearch.common.settings.Settings; +import org.elasticsearch.http.CorsHandler; import org.elasticsearch.http.HttpTransportSettings; import org.elasticsearch.http.netty4.cors.Netty4CorsHandler; import org.elasticsearch.rest.RestStatus; @@ -140,7 +141,7 @@ private FullHttpResponse executeRequest(final Settings settings, final String or } httpRequest.headers().add(HttpHeaderNames.HOST, host); EmbeddedChannel embeddedChannel = new EmbeddedChannel(); - embeddedChannel.pipeline().addLast(new Netty4CorsHandler(Netty4HttpServerTransport.buildCorsConfig(settings))); + embeddedChannel.pipeline().addLast(new Netty4CorsHandler(CorsHandler.fromSettings(settings))); Netty4HttpRequest nettyRequest = new Netty4HttpRequest(httpRequest, 0); embeddedChannel.writeOutbound(nettyRequest.createResponse(RestStatus.OK, new BytesArray("content"))); return embeddedChannel.readOutbound(); diff --git a/modules/transport-netty4/src/test/java/org/elasticsearch/http/netty4/Netty4HttpRequestSizeLimitIT.java b/modules/transport-netty4/src/test/java/org/elasticsearch/http/netty4/Netty4HttpRequestSizeLimitIT.java index e45f6de92d5fe..eaec286c7b237 100644 --- a/modules/transport-netty4/src/test/java/org/elasticsearch/http/netty4/Netty4HttpRequestSizeLimitIT.java +++ b/modules/transport-netty4/src/test/java/org/elasticsearch/http/netty4/Netty4HttpRequestSizeLimitIT.java @@ -43,7 +43,7 @@ * This test checks that in-flight requests are limited on HTTP level and that requests that are excluded from limiting can pass. * * As the same setting is also used to limit in-flight requests on transport level, we avoid transport messages by forcing - * a single node "cluster". We also force test infrastructure to use the node client instead of the transport client for the same reason. + * a single node "cluster". */ @ClusterScope(scope = Scope.TEST, supportsDedicatedMasters = false, numClientNodes = 0, numDataNodes = 1) public class Netty4HttpRequestSizeLimitIT extends ESNetty4IntegTestCase { diff --git a/modules/transport-netty4/src/test/java/org/elasticsearch/http/netty4/Netty4HttpServerTransportTests.java b/modules/transport-netty4/src/test/java/org/elasticsearch/http/netty4/Netty4HttpServerTransportTests.java index bc4ebe5672ec7..d398890bf23f7 100644 --- a/modules/transport-netty4/src/test/java/org/elasticsearch/http/netty4/Netty4HttpServerTransportTests.java +++ b/modules/transport-netty4/src/test/java/org/elasticsearch/http/netty4/Netty4HttpServerTransportTests.java @@ -39,12 +39,10 @@ import io.netty.handler.codec.http.HttpUtil; import io.netty.handler.codec.http.HttpVersion; import org.elasticsearch.ElasticsearchException; -import org.elasticsearch.common.Strings; import org.elasticsearch.common.bytes.BytesArray; import org.elasticsearch.common.network.NetworkService; import org.elasticsearch.common.settings.Setting; import org.elasticsearch.common.settings.Settings; -import org.elasticsearch.common.settings.SettingsException; import org.elasticsearch.common.transport.TransportAddress; import org.elasticsearch.common.unit.ByteSizeValue; import org.elasticsearch.common.unit.TimeValue; @@ -55,7 +53,6 @@ import org.elasticsearch.http.HttpServerTransport; import org.elasticsearch.http.HttpTransportSettings; import org.elasticsearch.http.NullDispatcher; -import org.elasticsearch.http.netty4.cors.Netty4CorsConfig; import org.elasticsearch.indices.breaker.NoneCircuitBreakerService; import org.elasticsearch.rest.BytesRestResponse; import org.elasticsearch.rest.RestChannel; @@ -69,23 +66,11 @@ import java.io.IOException; import java.nio.charset.Charset; import java.nio.charset.StandardCharsets; -import java.util.Arrays; import java.util.Collections; -import java.util.HashSet; -import java.util.Set; import java.util.concurrent.CountDownLatch; import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicReference; -import java.util.regex.PatternSyntaxException; -import java.util.stream.Collectors; - -import static org.elasticsearch.common.Strings.collectionToDelimitedString; -import static org.elasticsearch.http.HttpTransportSettings.SETTING_CORS_ALLOW_CREDENTIALS; -import static org.elasticsearch.http.HttpTransportSettings.SETTING_CORS_ALLOW_HEADERS; -import static org.elasticsearch.http.HttpTransportSettings.SETTING_CORS_ALLOW_METHODS; -import static org.elasticsearch.http.HttpTransportSettings.SETTING_CORS_ALLOW_ORIGIN; -import static org.elasticsearch.http.HttpTransportSettings.SETTING_CORS_ENABLED; -import static org.elasticsearch.http.HttpTransportSettings.SETTING_CORS_MAX_AGE; + import static org.elasticsearch.rest.RestStatus.BAD_REQUEST; import static org.elasticsearch.rest.RestStatus.OK; import static org.hamcrest.Matchers.containsString; @@ -119,48 +104,6 @@ public void shutdown() throws Exception { bigArrays = null; } - public void testCorsConfig() { - final Set methods = new HashSet<>(Arrays.asList("get", "options", "post")); - final Set headers = new HashSet<>(Arrays.asList("Content-Type", "Content-Length")); - final String prefix = randomBoolean() ? " " : ""; // sometimes have a leading whitespace between comma delimited elements - final Settings settings = Settings.builder() - .put(SETTING_CORS_ENABLED.getKey(), true) - .put(SETTING_CORS_ALLOW_ORIGIN.getKey(), "*") - .put(SETTING_CORS_ALLOW_METHODS.getKey(), collectionToDelimitedString(methods, ",", prefix, "")) - .put(SETTING_CORS_ALLOW_HEADERS.getKey(), collectionToDelimitedString(headers, ",", prefix, "")) - .put(SETTING_CORS_ALLOW_CREDENTIALS.getKey(), true) - .build(); - final Netty4CorsConfig corsConfig = Netty4HttpServerTransport.buildCorsConfig(settings); - assertTrue(corsConfig.isAnyOriginSupported()); - assertEquals(headers, corsConfig.allowedRequestHeaders()); - assertEquals(methods, corsConfig.allowedRequestMethods().stream().map(HttpMethod::name).collect(Collectors.toSet())); - } - - public void testCorsConfigWithDefaults() { - final Set methods = Strings.commaDelimitedListToSet(SETTING_CORS_ALLOW_METHODS.getDefault(Settings.EMPTY)); - final Set headers = Strings.commaDelimitedListToSet(SETTING_CORS_ALLOW_HEADERS.getDefault(Settings.EMPTY)); - final long maxAge = SETTING_CORS_MAX_AGE.getDefault(Settings.EMPTY); - final Settings settings = Settings.builder().put(SETTING_CORS_ENABLED.getKey(), true).build(); - final Netty4CorsConfig corsConfig = Netty4HttpServerTransport.buildCorsConfig(settings); - assertFalse(corsConfig.isAnyOriginSupported()); - assertEquals(Collections.emptySet(), corsConfig.origins().get()); - assertEquals(headers, corsConfig.allowedRequestHeaders()); - assertEquals(methods, corsConfig.allowedRequestMethods().stream().map(HttpMethod::name).collect(Collectors.toSet())); - assertEquals(maxAge, corsConfig.maxAge()); - assertFalse(corsConfig.isCredentialsAllowed()); - } - - public void testCorsConfigWithBadRegex() { - final Settings settings = Settings.builder() - .put(SETTING_CORS_ENABLED.getKey(), true) - .put(SETTING_CORS_ALLOW_ORIGIN.getKey(), "/[*/") - .put(SETTING_CORS_ALLOW_CREDENTIALS.getKey(), true) - .build(); - SettingsException e = expectThrows(SettingsException.class, () -> Netty4HttpServerTransport.buildCorsConfig(settings)); - assertThat(e.getMessage(), containsString("Bad regex in [http.cors.allow-origin]: [/[*/]")); - assertThat(e.getCause(), instanceOf(PatternSyntaxException.class)); - } - /** * Test that {@link Netty4HttpServerTransport} supports the "Expect: 100-continue" HTTP header * @throws InterruptedException if the client communication with the server is interrupted diff --git a/modules/transport-netty4/src/test/java/org/elasticsearch/rest/discovery/Zen2RestApiIT.java b/modules/transport-netty4/src/test/java/org/elasticsearch/rest/discovery/Zen2RestApiIT.java index fcb8e75700d0c..664632cb527f8 100644 --- a/modules/transport-netty4/src/test/java/org/elasticsearch/rest/discovery/Zen2RestApiIT.java +++ b/modules/transport-netty4/src/test/java/org/elasticsearch/rest/discovery/Zen2RestApiIT.java @@ -47,7 +47,7 @@ // These tests are here today so they have access to a proper REST client. They cannot be in :server:integTest since the REST client needs a // proper transport implementation, and they cannot be REST tests today since they need to restart nodes. When #35599 and friends land we // should be able to move these tests to run against a proper cluster instead. TODO do this. -@ESIntegTestCase.ClusterScope(scope = ESIntegTestCase.Scope.TEST, numDataNodes = 0, autoMinMasterNodes = false) +@ESIntegTestCase.ClusterScope(scope = ESIntegTestCase.Scope.TEST, numDataNodes = 0, autoManageMasterNodes = false) public class Zen2RestApiIT extends ESNetty4IntegTestCase { @Override diff --git a/modules/transport-netty4/src/test/java/org/elasticsearch/transport/netty4/Netty4TransportMultiPortIntegrationIT.java b/modules/transport-netty4/src/test/java/org/elasticsearch/transport/netty4/Netty4TransportMultiPortIntegrationIT.java index 881fe73112c5a..47af90d2bf39e 100644 --- a/modules/transport-netty4/src/test/java/org/elasticsearch/transport/netty4/Netty4TransportMultiPortIntegrationIT.java +++ b/modules/transport-netty4/src/test/java/org/elasticsearch/transport/netty4/Netty4TransportMultiPortIntegrationIT.java @@ -19,24 +19,16 @@ package org.elasticsearch.transport.netty4; import org.elasticsearch.ESNetty4IntegTestCase; -import org.elasticsearch.action.admin.cluster.health.ClusterHealthResponse; import org.elasticsearch.action.admin.cluster.node.info.NodeInfo; import org.elasticsearch.action.admin.cluster.node.info.NodesInfoResponse; -import org.elasticsearch.client.transport.TransportClient; -import org.elasticsearch.cluster.health.ClusterHealthStatus; import org.elasticsearch.common.network.NetworkAddress; -import org.elasticsearch.common.network.NetworkModule; import org.elasticsearch.common.settings.Settings; import org.elasticsearch.common.transport.BoundTransportAddress; import org.elasticsearch.common.transport.TransportAddress; -import org.elasticsearch.env.Environment; import org.elasticsearch.test.ESIntegTestCase.ClusterScope; import org.elasticsearch.test.ESIntegTestCase.Scope; import org.elasticsearch.test.junit.annotations.Network; -import org.elasticsearch.transport.MockTransportClient; -import org.elasticsearch.transport.Netty4Plugin; -import java.net.InetAddress; import java.util.Locale; import static org.hamcrest.Matchers.allOf; @@ -69,22 +61,6 @@ protected Settings nodeSettings(int nodeOrdinal) { return builder.build(); } - public void testThatTransportClientCanConnect() throws Exception { - Settings settings = Settings.builder() - .put("cluster.name", internalCluster().getClusterName()) - .put(NetworkModule.TRANSPORT_TYPE_KEY, Netty4Plugin.NETTY_TRANSPORT_NAME) - .put(Environment.PATH_HOME_SETTING.getKey(), createTempDir().toString()) - .build(); - // we have to test all the ports that the data node might be bound to - try (TransportClient transportClient = new MockTransportClient(settings, Netty4Plugin.class)) { - for (int i = 0; i <= 10; i++) { - transportClient.addTransportAddress(new TransportAddress(InetAddress.getByName("127.0.0.1"), randomPort + i)); - } - ClusterHealthResponse response = transportClient.admin().cluster().prepareHealth().get(); - assertThat(response.getStatus(), is(ClusterHealthStatus.GREEN)); - } - } - @Network public void testThatInfosAreExposed() throws Exception { NodesInfoResponse response = client().admin().cluster().prepareNodesInfo().clear().setTransport(true).get(); diff --git a/plugins/analysis-icu/build.gradle b/plugins/analysis-icu/build.gradle index 90132e2c58fcd..f818959e706de 100644 --- a/plugins/analysis-icu/build.gradle +++ b/plugins/analysis-icu/build.gradle @@ -22,7 +22,6 @@ import de.thetaphi.forbiddenapis.gradle.CheckForbiddenApis esplugin { description 'The ICU Analysis plugin integrates Lucene ICU module into elasticsearch, adding ICU relates analysis components.' classname 'org.elasticsearch.plugin.analysis.icu.AnalysisICUPlugin' - hasClientJar = true } tasks.withType(CheckForbiddenApis) { diff --git a/plugins/analysis-icu/src/test/java/org/elasticsearch/index/mapper/ICUCollationKeywordFieldMapperIT.java b/plugins/analysis-icu/src/test/java/org/elasticsearch/index/mapper/ICUCollationKeywordFieldMapperIT.java index 3bcefe0cf5680..8a55d3424c1ff 100644 --- a/plugins/analysis-icu/src/test/java/org/elasticsearch/index/mapper/ICUCollationKeywordFieldMapperIT.java +++ b/plugins/analysis-icu/src/test/java/org/elasticsearch/index/mapper/ICUCollationKeywordFieldMapperIT.java @@ -18,15 +18,10 @@ */ package org.elasticsearch.index.mapper; -import static org.elasticsearch.common.xcontent.XContentFactory.jsonBuilder; -import static org.elasticsearch.test.hamcrest.ElasticsearchAssertions.assertAcked; -import static org.elasticsearch.test.hamcrest.ElasticsearchAssertions.assertHitCount; -import static org.elasticsearch.test.hamcrest.ElasticsearchAssertions.assertNoFailures; -import static org.elasticsearch.test.hamcrest.ElasticsearchAssertions.assertOrderedSearchHits; - import com.ibm.icu.text.Collator; import com.ibm.icu.text.RuleBasedCollator; import com.ibm.icu.util.ULocale; + import org.elasticsearch.action.search.SearchRequest; import org.elasticsearch.action.search.SearchResponse; import org.elasticsearch.common.xcontent.XContentBuilder; @@ -43,6 +38,12 @@ import java.util.Collection; import java.util.Collections; +import static org.elasticsearch.common.xcontent.XContentFactory.jsonBuilder; +import static org.elasticsearch.test.hamcrest.ElasticsearchAssertions.assertAcked; +import static org.elasticsearch.test.hamcrest.ElasticsearchAssertions.assertHitCount; +import static org.elasticsearch.test.hamcrest.ElasticsearchAssertions.assertNoFailures; +import static org.elasticsearch.test.hamcrest.ElasticsearchAssertions.assertOrderedSearchHits; + public class ICUCollationKeywordFieldMapperIT extends ESIntegTestCase { @Override @@ -82,7 +83,6 @@ public void testBasicUsage() throws Exception { // searching for either of the terms should return both results since they collate to the same value SearchRequest request = new SearchRequest() .indices(index) - .types(type) .source(new SearchSourceBuilder() .fetchSource(false) .query(QueryBuilders.termQuery("collate", randomBoolean() ? equivalent[0] : equivalent[1])) @@ -122,7 +122,6 @@ public void testMultipleValues() throws Exception { // using sort mode = max, values B and C will be used for the sort SearchRequest request = new SearchRequest() .indices(index) - .types(type) .source(new SearchSourceBuilder() .fetchSource(false) .query(QueryBuilders.termQuery("collate", "a")) @@ -139,7 +138,6 @@ public void testMultipleValues() throws Exception { // same thing, using different sort mode that will use a for both docs request = new SearchRequest() .indices(index) - .types(type) .source(new SearchSourceBuilder() .fetchSource(false) .query(QueryBuilders.termQuery("collate", "a")) @@ -183,7 +181,6 @@ public void testNormalization() throws Exception { // searching for either of the terms should return both results since they collate to the same value SearchRequest request = new SearchRequest() .indices(index) - .types(type) .source(new SearchSourceBuilder() .fetchSource(false) .query(QueryBuilders.termQuery("collate", randomBoolean() ? equivalent[0] : equivalent[1])) @@ -225,7 +222,6 @@ public void testSecondaryStrength() throws Exception { SearchRequest request = new SearchRequest() .indices(index) - .types(type) .source(new SearchSourceBuilder() .fetchSource(false) .query(QueryBuilders.termQuery("collate", randomBoolean() ? equivalent[0] : equivalent[1])) @@ -268,7 +264,6 @@ public void testIgnorePunctuation() throws Exception { SearchRequest request = new SearchRequest() .indices(index) - .types(type) .source(new SearchSourceBuilder() .fetchSource(false) .query(QueryBuilders.termQuery("collate", randomBoolean() ? equivalent[0] : equivalent[1])) @@ -312,7 +307,6 @@ public void testIgnoreWhitespace() throws Exception { SearchRequest request = new SearchRequest() .indices(index) - .types(type) .source(new SearchSourceBuilder() .fetchSource(false) .sort("collate", SortOrder.ASC) @@ -352,7 +346,6 @@ public void testNumerics() throws Exception { SearchRequest request = new SearchRequest() .indices(index) - .types(type) .source(new SearchSourceBuilder() .fetchSource(false) .sort("collate", SortOrder.ASC) @@ -394,7 +387,6 @@ public void testIgnoreAccentsButNotCase() throws Exception { SearchRequest request = new SearchRequest() .indices(index) - .types(type) .source(new SearchSourceBuilder() .fetchSource(false) .sort("collate", SortOrder.ASC) @@ -435,7 +427,6 @@ public void testUpperCaseFirst() throws Exception { SearchRequest request = new SearchRequest() .indices(index) - .types(type) .source(new SearchSourceBuilder() .fetchSource(false) .sort("collate", SortOrder.ASC) @@ -487,7 +478,6 @@ public void testCustomRules() throws Exception { SearchRequest request = new SearchRequest() .indices(index) - .types(type) .source(new SearchSourceBuilder() .fetchSource(false) .query(QueryBuilders.termQuery("collate", randomBoolean() ? equivalent[0] : equivalent[1])) diff --git a/plugins/analysis-smartcn/src/main/java/org/elasticsearch/index/analysis/SmartChineseStopTokenFilterFactory.java b/plugins/analysis-smartcn/src/main/java/org/elasticsearch/index/analysis/SmartChineseStopTokenFilterFactory.java new file mode 100644 index 0000000000000..b528481811394 --- /dev/null +++ b/plugins/analysis-smartcn/src/main/java/org/elasticsearch/index/analysis/SmartChineseStopTokenFilterFactory.java @@ -0,0 +1,61 @@ +/* + * 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.analysis; + +import org.apache.lucene.analysis.CharArraySet; +import org.apache.lucene.analysis.StopFilter; +import org.apache.lucene.analysis.TokenStream; +import org.apache.lucene.analysis.cn.smart.SmartChineseAnalyzer; +import org.apache.lucene.search.suggest.analyzing.SuggestStopFilter; +import org.elasticsearch.common.settings.Settings; +import org.elasticsearch.env.Environment; +import org.elasticsearch.index.IndexSettings; +import java.util.Map; +import java.util.Set; + +import static java.util.Collections.singletonMap; + +public class SmartChineseStopTokenFilterFactory extends AbstractTokenFilterFactory { + private static final Map> NAMED_STOP_WORDS = singletonMap("_smartcn_", SmartChineseAnalyzer.getDefaultStopSet()); + + private final CharArraySet stopWords; + + private final boolean ignoreCase; + + private final boolean removeTrailing; + + public SmartChineseStopTokenFilterFactory(IndexSettings indexSettings, Environment env, String name, Settings settings) { + super(indexSettings, name, settings); + this.ignoreCase = settings.getAsBoolean("ignore_case", false); + this.removeTrailing = settings.getAsBoolean("remove_trailing", true); + this.stopWords = Analysis.parseWords(env, settings, "stopwords", + SmartChineseAnalyzer.getDefaultStopSet(), NAMED_STOP_WORDS, ignoreCase); + } + + @Override + public TokenStream create(TokenStream tokenStream) { + if (removeTrailing) { + return new StopFilter(tokenStream, stopWords); + } else { + return new SuggestStopFilter(tokenStream, stopWords); + } + } + +} diff --git a/plugins/analysis-smartcn/src/main/java/org/elasticsearch/plugin/analysis/smartcn/AnalysisSmartChinesePlugin.java b/plugins/analysis-smartcn/src/main/java/org/elasticsearch/plugin/analysis/smartcn/AnalysisSmartChinesePlugin.java index b11a157c149d6..61d2d75359a95 100644 --- a/plugins/analysis-smartcn/src/main/java/org/elasticsearch/plugin/analysis/smartcn/AnalysisSmartChinesePlugin.java +++ b/plugins/analysis-smartcn/src/main/java/org/elasticsearch/plugin/analysis/smartcn/AnalysisSmartChinesePlugin.java @@ -23,6 +23,7 @@ import org.elasticsearch.index.analysis.AnalyzerProvider; import org.elasticsearch.index.analysis.SmartChineseAnalyzerProvider; import org.elasticsearch.index.analysis.SmartChineseNoOpTokenFilterFactory; +import org.elasticsearch.index.analysis.SmartChineseStopTokenFilterFactory; import org.elasticsearch.index.analysis.SmartChineseTokenizerTokenizerFactory; import org.elasticsearch.index.analysis.TokenFilterFactory; import org.elasticsearch.index.analysis.TokenizerFactory; @@ -38,15 +39,18 @@ public class AnalysisSmartChinesePlugin extends Plugin implements AnalysisPlugin { @Override public Map> getTokenFilters() { - // This is a noop token filter; it's here for backwards compat before we had "smartcn_tokenizer" - return singletonMap("smartcn_word", SmartChineseNoOpTokenFilterFactory::new); + Map> tokenFilters = new HashMap<>(); + tokenFilters.put("smartcn_stop", SmartChineseStopTokenFilterFactory::new); + // TODO: deprecate and remove, this is a noop token filter; it's here for backwards compat before we had "smartcn_tokenizer" + tokenFilters.put("smartcn_word", SmartChineseNoOpTokenFilterFactory::new); + return tokenFilters; } @Override public Map> getTokenizers() { Map> extra = new HashMap<>(); extra.put("smartcn_tokenizer", SmartChineseTokenizerTokenizerFactory::new); - // This is an alias to "smartcn_tokenizer"; it's here for backwards compat + // TODO: deprecate and remove, this is an alias to "smartcn_tokenizer"; it's here for backwards compat extra.put("smartcn_sentence", SmartChineseTokenizerTokenizerFactory::new); return extra; } diff --git a/plugins/analysis-stempel/src/main/java/org/elasticsearch/index/analysis/pl/PolishStopTokenFilterFactory.java b/plugins/analysis-stempel/src/main/java/org/elasticsearch/index/analysis/pl/PolishStopTokenFilterFactory.java new file mode 100644 index 0000000000000..32897ad29d7b4 --- /dev/null +++ b/plugins/analysis-stempel/src/main/java/org/elasticsearch/index/analysis/pl/PolishStopTokenFilterFactory.java @@ -0,0 +1,73 @@ +/* + * 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.analysis.pl; + + +import org.apache.lucene.analysis.CharArraySet; +import org.apache.lucene.analysis.StopFilter; +import org.apache.lucene.analysis.TokenStream; +import org.apache.lucene.analysis.pl.PolishAnalyzer; +import org.apache.lucene.search.suggest.analyzing.SuggestStopFilter; +import org.elasticsearch.common.settings.Settings; +import org.elasticsearch.env.Environment; +import org.elasticsearch.index.IndexSettings; +import org.elasticsearch.index.analysis.AbstractTokenFilterFactory; +import org.elasticsearch.index.analysis.Analysis; + +import java.util.Map; +import java.util.Set; + +import static java.util.Collections.singletonMap; + +public class PolishStopTokenFilterFactory extends AbstractTokenFilterFactory { + private static final Map> NAMED_STOP_WORDS = singletonMap("_polish_", PolishAnalyzer.getDefaultStopSet()); + + private final CharArraySet stopWords; + + private final boolean ignoreCase; + + private final boolean removeTrailing; + + public PolishStopTokenFilterFactory(IndexSettings indexSettings, Environment env, String name, Settings settings) { + super(indexSettings, name, settings); + this.ignoreCase = settings.getAsBoolean("ignore_case", false); + this.removeTrailing = settings.getAsBoolean("remove_trailing", true); + this.stopWords = Analysis.parseWords(env, settings, "stopwords", + PolishAnalyzer.getDefaultStopSet(), NAMED_STOP_WORDS, ignoreCase); + } + + @Override + public TokenStream create(TokenStream tokenStream) { + if (removeTrailing) { + return new StopFilter(tokenStream, stopWords); + } else { + return new SuggestStopFilter(tokenStream, stopWords); + } + } + + public Set stopWords() { + return stopWords; + } + + public boolean ignoreCase() { + return ignoreCase; + } + +} diff --git a/plugins/analysis-stempel/src/main/java/org/elasticsearch/plugin/analysis/stempel/AnalysisStempelPlugin.java b/plugins/analysis-stempel/src/main/java/org/elasticsearch/plugin/analysis/stempel/AnalysisStempelPlugin.java index 98dd9634fb961..a523d7dcaa0ce 100644 --- a/plugins/analysis-stempel/src/main/java/org/elasticsearch/plugin/analysis/stempel/AnalysisStempelPlugin.java +++ b/plugins/analysis-stempel/src/main/java/org/elasticsearch/plugin/analysis/stempel/AnalysisStempelPlugin.java @@ -24,6 +24,7 @@ import org.elasticsearch.index.analysis.TokenFilterFactory; import org.elasticsearch.index.analysis.pl.PolishAnalyzerProvider; import org.elasticsearch.index.analysis.pl.PolishStemTokenFilterFactory; +import org.elasticsearch.index.analysis.pl.PolishStopTokenFilterFactory; import org.elasticsearch.indices.analysis.AnalysisModule.AnalysisProvider; import org.elasticsearch.plugins.AnalysisPlugin; import org.elasticsearch.plugins.Plugin; @@ -35,7 +36,8 @@ public class AnalysisStempelPlugin extends Plugin implements AnalysisPlugin { @Override public Map> getTokenFilters() { - return singletonMap("polish_stem", PolishStemTokenFilterFactory::new); + return Map.of("polish_stem", PolishStemTokenFilterFactory::new, + "polish_stop", PolishStopTokenFilterFactory::new); } @Override diff --git a/plugins/discovery-azure-classic/build.gradle b/plugins/discovery-azure-classic/build.gradle index cb024d8ac00d7..8a9f207f0b467 100644 --- a/plugins/discovery-azure-classic/build.gradle +++ b/plugins/discovery-azure-classic/build.gradle @@ -35,6 +35,7 @@ dependencies { compile "org.apache.httpcomponents:httpclient:${versions.httpclient}" compile "org.apache.httpcomponents:httpcore:${versions.httpcore}" compile "commons-logging:commons-logging:${versions.commonslogging}" + compile "org.apache.logging.log4j:log4j-1.2-api:${versions.log4j}" compile "commons-codec:commons-codec:${versions.commonscodec}" compile "commons-lang:commons-lang:2.6" compile "commons-io:commons-io:2.4" diff --git a/server/licenses/log4j-1.2-api-2.11.1.jar.sha1 b/plugins/discovery-azure-classic/licenses/log4j-1.2-api-2.11.1.jar.sha1 similarity index 100% rename from server/licenses/log4j-1.2-api-2.11.1.jar.sha1 rename to plugins/discovery-azure-classic/licenses/log4j-1.2-api-2.11.1.jar.sha1 diff --git a/server/licenses/log4j-LICENSE.txt b/plugins/discovery-azure-classic/licenses/log4j-LICENSE.txt similarity index 100% rename from server/licenses/log4j-LICENSE.txt rename to plugins/discovery-azure-classic/licenses/log4j-LICENSE.txt diff --git a/server/licenses/log4j-NOTICE.txt b/plugins/discovery-azure-classic/licenses/log4j-NOTICE.txt similarity index 100% rename from server/licenses/log4j-NOTICE.txt rename to plugins/discovery-azure-classic/licenses/log4j-NOTICE.txt diff --git a/plugins/discovery-azure-classic/src/test/java/org/elasticsearch/discovery/azure/classic/AzureDiscoveryClusterFormationTests.java b/plugins/discovery-azure-classic/src/test/java/org/elasticsearch/discovery/azure/classic/AzureDiscoveryClusterFormationTests.java index 82347ca2bf1e4..c5b310c14817a 100644 --- a/plugins/discovery-azure-classic/src/test/java/org/elasticsearch/discovery/azure/classic/AzureDiscoveryClusterFormationTests.java +++ b/plugins/discovery-azure-classic/src/test/java/org/elasticsearch/discovery/azure/classic/AzureDiscoveryClusterFormationTests.java @@ -279,7 +279,8 @@ private static String getProtocol() { return "TLSv1.2"; } else { JavaVersion full = - AccessController.doPrivileged((PrivilegedAction) () -> JavaVersion.parse(System.getProperty("java.version"))); + AccessController.doPrivileged( + (PrivilegedAction) () -> JavaVersion.parse(System.getProperty("java.specification.version"))); if (full.compareTo(JavaVersion.parse("12.0.1")) < 0) { return "TLSv1.2"; } @@ -289,13 +290,14 @@ private static String getProtocol() { @AfterClass public static void stopHttpd() throws IOException { - for (int i = 0; i < internalCluster().size(); i++) { + try { // shut them all down otherwise we get spammed with connection refused exceptions - internalCluster().stopRandomDataNode(); + internalCluster().close(); + } finally { + httpsServer.stop(0); + httpsServer = null; + logDir = null; } - httpsServer.stop(0); - httpsServer = null; - logDir = null; } public void testJoin() throws ExecutionException, InterruptedException { diff --git a/plugins/discovery-ec2/build.gradle b/plugins/discovery-ec2/build.gradle index 50dc6ac5d85b5..6935de286b237 100644 --- a/plugins/discovery-ec2/build.gradle +++ b/plugins/discovery-ec2/build.gradle @@ -23,7 +23,7 @@ esplugin { } versions << [ - 'aws': '1.11.505' + 'aws': '1.11.562' ] dependencies { @@ -32,6 +32,7 @@ dependencies { compile "org.apache.httpcomponents:httpclient:${versions.httpclient}" compile "org.apache.httpcomponents:httpcore:${versions.httpcore}" compile "commons-logging:commons-logging:${versions.commonslogging}" + compile "org.apache.logging.log4j:log4j-1.2-api:${versions.log4j}" compile "commons-codec:commons-codec:${versions.commonscodec}" compile 'com.fasterxml.jackson.core:jackson-databind:2.8.11.3' compile "com.fasterxml.jackson.core:jackson-annotations:${versions.jackson}" diff --git a/plugins/discovery-ec2/licenses/aws-java-sdk-core-1.11.505.jar.sha1 b/plugins/discovery-ec2/licenses/aws-java-sdk-core-1.11.505.jar.sha1 deleted file mode 100644 index add5db290e8a8..0000000000000 --- a/plugins/discovery-ec2/licenses/aws-java-sdk-core-1.11.505.jar.sha1 +++ /dev/null @@ -1 +0,0 @@ -d19328c227b2b5ad81d137361ebc9cbcd0396465 \ No newline at end of file diff --git a/plugins/discovery-ec2/licenses/aws-java-sdk-core-1.11.562.jar.sha1 b/plugins/discovery-ec2/licenses/aws-java-sdk-core-1.11.562.jar.sha1 new file mode 100644 index 0000000000000..ed8ded6a3608c --- /dev/null +++ b/plugins/discovery-ec2/licenses/aws-java-sdk-core-1.11.562.jar.sha1 @@ -0,0 +1 @@ +b5fc47ec1b5afe180f5ebb4eda755acdca7a20ae \ No newline at end of file diff --git a/plugins/discovery-ec2/licenses/aws-java-sdk-ec2-1.11.505.jar.sha1 b/plugins/discovery-ec2/licenses/aws-java-sdk-ec2-1.11.505.jar.sha1 deleted file mode 100644 index 857f0888de3aa..0000000000000 --- a/plugins/discovery-ec2/licenses/aws-java-sdk-ec2-1.11.505.jar.sha1 +++ /dev/null @@ -1 +0,0 @@ -b669b3c90ea9bf73734ab26f0cb30c5c66addf55 \ No newline at end of file diff --git a/plugins/discovery-ec2/licenses/aws-java-sdk-ec2-1.11.562.jar.sha1 b/plugins/discovery-ec2/licenses/aws-java-sdk-ec2-1.11.562.jar.sha1 new file mode 100644 index 0000000000000..040d28de70b29 --- /dev/null +++ b/plugins/discovery-ec2/licenses/aws-java-sdk-ec2-1.11.562.jar.sha1 @@ -0,0 +1 @@ +0211a055fb3e036033af4b1ca25ada0574a756ec \ No newline at end of file diff --git a/plugins/discovery-ec2/licenses/log4j-1.2-api-2.11.1.jar.sha1 b/plugins/discovery-ec2/licenses/log4j-1.2-api-2.11.1.jar.sha1 new file mode 100644 index 0000000000000..575d75dbda8c5 --- /dev/null +++ b/plugins/discovery-ec2/licenses/log4j-1.2-api-2.11.1.jar.sha1 @@ -0,0 +1 @@ +3aba3398fe064a3eab4331f88161c7480e848418 \ No newline at end of file diff --git a/plugins/discovery-ec2/licenses/log4j-LICENSE.txt b/plugins/discovery-ec2/licenses/log4j-LICENSE.txt new file mode 100644 index 0000000000000..6279e5206de13 --- /dev/null +++ b/plugins/discovery-ec2/licenses/log4j-LICENSE.txt @@ -0,0 +1,202 @@ + + Apache License + Version 2.0, January 2004 + http://www.apache.org/licenses/ + + TERMS AND CONDITIONS FOR USE, REPRODUCTION, AND DISTRIBUTION + + 1. Definitions. + + "License" shall mean the terms and conditions for use, reproduction, + and distribution as defined by Sections 1 through 9 of this document. + + "Licensor" shall mean the copyright owner or entity authorized by + the copyright owner that is granting the License. + + "Legal Entity" shall mean the union of the acting entity and all + other entities that control, are controlled by, or are under common + control with that entity. For the purposes of this definition, + "control" means (i) the power, direct or indirect, to cause the + direction or management of such entity, whether by contract or + otherwise, or (ii) ownership of fifty percent (50%) or more of the + outstanding shares, or (iii) beneficial ownership of such entity. + + "You" (or "Your") shall mean an individual or Legal Entity + exercising permissions granted by this License. + + "Source" form shall mean the preferred form for making modifications, + including but not limited to software source code, documentation + source, and configuration files. + + "Object" form shall mean any form resulting from mechanical + transformation or translation of a Source form, including but + not limited to compiled object code, generated documentation, + and conversions to other media types. + + "Work" shall mean the work of authorship, whether in Source or + Object form, made available under the License, as indicated by a + copyright notice that is included in or attached to the work + (an example is provided in the Appendix below). + + "Derivative Works" shall mean any work, whether in Source or Object + form, that is based on (or derived from) the Work and for which the + editorial revisions, annotations, elaborations, or other modifications + represent, as a whole, an original work of authorship. For the purposes + of this License, Derivative Works shall not include works that remain + separable from, or merely link (or bind by name) to the interfaces of, + the Work and Derivative Works thereof. + + "Contribution" shall mean any work of authorship, including + the original version of the Work and any modifications or additions + to that Work or Derivative Works thereof, that is intentionally + submitted to Licensor for inclusion in the Work by the copyright owner + or by an individual or Legal Entity authorized to submit on behalf of + the copyright owner. For the purposes of this definition, "submitted" + means any form of electronic, verbal, or written communication sent + to the Licensor or its representatives, including but not limited to + communication on electronic mailing lists, source code control systems, + and issue tracking systems that are managed by, or on behalf of, the + Licensor for the purpose of discussing and improving the Work, but + excluding communication that is conspicuously marked or otherwise + designated in writing by the copyright owner as "Not a Contribution." + + "Contributor" shall mean Licensor and any individual or Legal Entity + on behalf of whom a Contribution has been received by Licensor and + subsequently incorporated within the Work. + + 2. Grant of Copyright License. Subject to the terms and conditions of + this License, each Contributor hereby grants to You a perpetual, + worldwide, non-exclusive, no-charge, royalty-free, irrevocable + copyright license to reproduce, prepare Derivative Works of, + publicly display, publicly perform, sublicense, and distribute the + Work and such Derivative Works in Source or Object form. + + 3. Grant of Patent License. Subject to the terms and conditions of + this License, each Contributor hereby grants to You a perpetual, + worldwide, non-exclusive, no-charge, royalty-free, irrevocable + (except as stated in this section) patent license to make, have made, + use, offer to sell, sell, import, and otherwise transfer the Work, + where such license applies only to those patent claims licensable + by such Contributor that are necessarily infringed by their + Contribution(s) alone or by combination of their Contribution(s) + with the Work to which such Contribution(s) was submitted. If You + institute patent litigation against any entity (including a + cross-claim or counterclaim in a lawsuit) alleging that the Work + or a Contribution incorporated within the Work constitutes direct + or contributory patent infringement, then any patent licenses + granted to You under this License for that Work shall terminate + as of the date such litigation is filed. + + 4. Redistribution. You may reproduce and distribute copies of the + Work or Derivative Works thereof in any medium, with or without + modifications, and in Source or Object form, provided that You + meet the following conditions: + + (a) You must give any other recipients of the Work or + Derivative Works a copy of this License; and + + (b) You must cause any modified files to carry prominent notices + stating that You changed the files; and + + (c) You must retain, in the Source form of any Derivative Works + that You distribute, all copyright, patent, trademark, and + attribution notices from the Source form of the Work, + excluding those notices that do not pertain to any part of + the Derivative Works; and + + (d) If the Work includes a "NOTICE" text file as part of its + distribution, then any Derivative Works that You distribute must + include a readable copy of the attribution notices contained + within such NOTICE file, excluding those notices that do not + pertain to any part of the Derivative Works, in at least one + of the following places: within a NOTICE text file distributed + as part of the Derivative Works; within the Source form or + documentation, if provided along with the Derivative Works; or, + within a display generated by the Derivative Works, if and + wherever such third-party notices normally appear. The contents + of the NOTICE file are for informational purposes only and + do not modify the License. You may add Your own attribution + notices within Derivative Works that You distribute, alongside + or as an addendum to the NOTICE text from the Work, provided + that such additional attribution notices cannot be construed + as modifying the License. + + You may add Your own copyright statement to Your modifications and + may provide additional or different license terms and conditions + for use, reproduction, or distribution of Your modifications, or + for any such Derivative Works as a whole, provided Your use, + reproduction, and distribution of the Work otherwise complies with + the conditions stated in this License. + + 5. Submission of Contributions. Unless You explicitly state otherwise, + any Contribution intentionally submitted for inclusion in the Work + by You to the Licensor shall be under the terms and conditions of + this License, without any additional terms or conditions. + Notwithstanding the above, nothing herein shall supersede or modify + the terms of any separate license agreement you may have executed + with Licensor regarding such Contributions. + + 6. Trademarks. This License does not grant permission to use the trade + names, trademarks, service marks, or product names of the Licensor, + except as required for reasonable and customary use in describing the + origin of the Work and reproducing the content of the NOTICE file. + + 7. Disclaimer of Warranty. Unless required by applicable law or + agreed to in writing, Licensor provides the Work (and each + Contributor provides its Contributions) on an "AS IS" BASIS, + WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or + implied, including, without limitation, any warranties or conditions + of TITLE, NON-INFRINGEMENT, MERCHANTABILITY, or FITNESS FOR A + PARTICULAR PURPOSE. You are solely responsible for determining the + appropriateness of using or redistributing the Work and assume any + risks associated with Your exercise of permissions under this License. + + 8. Limitation of Liability. In no event and under no legal theory, + whether in tort (including negligence), contract, or otherwise, + unless required by applicable law (such as deliberate and grossly + negligent acts) or agreed to in writing, shall any Contributor be + liable to You for damages, including any direct, indirect, special, + incidental, or consequential damages of any character arising as a + result of this License or out of the use or inability to use the + Work (including but not limited to damages for loss of goodwill, + work stoppage, computer failure or malfunction, or any and all + other commercial damages or losses), even if such Contributor + has been advised of the possibility of such damages. + + 9. Accepting Warranty or Additional Liability. While redistributing + the Work or Derivative Works thereof, You may choose to offer, + and charge a fee for, acceptance of support, warranty, indemnity, + or other liability obligations and/or rights consistent with this + License. However, in accepting such obligations, You may act only + on Your own behalf and on Your sole responsibility, not on behalf + of any other Contributor, and only if You agree to indemnify, + defend, and hold each Contributor harmless for any liability + incurred by, or claims asserted against, such Contributor by reason + of your accepting any such warranty or additional liability. + + END OF TERMS AND CONDITIONS + + APPENDIX: How to apply the Apache License to your work. + + To apply the Apache License to your work, attach the following + boilerplate notice, with the fields enclosed by brackets "[]" + replaced with your own identifying information. (Don't include + the brackets!) The text should be enclosed in the appropriate + comment syntax for the file format. We also recommend that a + file or class name and description of purpose be included on the + same "printed page" as the copyright notice for easier + identification within third-party archives. + + Copyright 1999-2005 The Apache Software Foundation + + Licensed 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. diff --git a/plugins/discovery-ec2/licenses/log4j-NOTICE.txt b/plugins/discovery-ec2/licenses/log4j-NOTICE.txt new file mode 100644 index 0000000000000..0375732360047 --- /dev/null +++ b/plugins/discovery-ec2/licenses/log4j-NOTICE.txt @@ -0,0 +1,5 @@ +Apache log4j +Copyright 2007 The Apache Software Foundation + +This product includes software developed at +The Apache Software Foundation (http://www.apache.org/). \ No newline at end of file diff --git a/plugins/discovery-ec2/qa/amazon-ec2/build.gradle b/plugins/discovery-ec2/qa/amazon-ec2/build.gradle index 8397549f384fe..218c9aca2920a 100644 --- a/plugins/discovery-ec2/qa/amazon-ec2/build.gradle +++ b/plugins/discovery-ec2/qa/amazon-ec2/build.gradle @@ -22,6 +22,8 @@ import org.elasticsearch.gradle.MavenFilteringHack import org.elasticsearch.gradle.test.AntFixture import org.elasticsearch.gradle.test.RestIntegTestTask +import static org.elasticsearch.gradle.PropertyNormalization.IGNORE_VALUE + apply plugin: 'elasticsearch.standalone-rest-test' apply plugin: 'elasticsearch.rest-test' @@ -74,9 +76,9 @@ integTest.enabled = false setting 'discovery.seed_providers', 'ec2' setting 'network.host', '_ec2_' - setting 'discovery.ec2.endpoint', { "http://${-> fixture.addressAndPort}" } + setting 'discovery.ec2.endpoint', { "http://${-> fixture.addressAndPort}" }, IGNORE_VALUE - systemProperty "com.amazonaws.sdk.ec2MetadataServiceEndpointOverride", { "http://${-> fixture.addressAndPort}" } + systemProperty "com.amazonaws.sdk.ec2MetadataServiceEndpointOverride", { "http://${-> fixture.addressAndPort}" }, IGNORE_VALUE } } @@ -103,7 +105,7 @@ ec2FixtureContainerCredentials.env 'ACTIVATE_CONTAINER_CREDENTIALS', true testClusters.integTestContainerCredentials { environment 'AWS_CONTAINER_CREDENTIALS_FULL_URI', - { "http://${-> tasks.findByName("ec2FixtureContainerCredentials").addressAndPort}/ecs_credentials_endpoint" } + { "http://${-> tasks.findByName("ec2FixtureContainerCredentials").addressAndPort}/ecs_credentials_endpoint" }, IGNORE_VALUE } // Extra config for InstanceProfile diff --git a/plugins/discovery-ec2/src/test/java/org/elasticsearch/discovery/ec2/AmazonEC2Mock.java b/plugins/discovery-ec2/src/test/java/org/elasticsearch/discovery/ec2/AmazonEC2Mock.java index 3135769df5f46..040472723fbc0 100644 --- a/plugins/discovery-ec2/src/test/java/org/elasticsearch/discovery/ec2/AmazonEC2Mock.java +++ b/plugins/discovery-ec2/src/test/java/org/elasticsearch/discovery/ec2/AmazonEC2Mock.java @@ -20,679 +20,17 @@ package org.elasticsearch.discovery.ec2; import com.amazonaws.AmazonClientException; -import com.amazonaws.AmazonWebServiceRequest; import com.amazonaws.ClientConfiguration; -import com.amazonaws.ResponseMetadata; import com.amazonaws.auth.AWSCredentialsProvider; -import com.amazonaws.regions.Region; -import com.amazonaws.services.ec2.AmazonEC2; -import com.amazonaws.services.ec2.model.AcceptReservedInstancesExchangeQuoteRequest; -import com.amazonaws.services.ec2.model.AcceptReservedInstancesExchangeQuoteResult; -import com.amazonaws.services.ec2.model.AcceptTransitGatewayVpcAttachmentRequest; -import com.amazonaws.services.ec2.model.AcceptTransitGatewayVpcAttachmentResult; -import com.amazonaws.services.ec2.model.AcceptVpcEndpointConnectionsRequest; -import com.amazonaws.services.ec2.model.AcceptVpcEndpointConnectionsResult; -import com.amazonaws.services.ec2.model.AcceptVpcPeeringConnectionRequest; -import com.amazonaws.services.ec2.model.AcceptVpcPeeringConnectionResult; -import com.amazonaws.services.ec2.model.AdvertiseByoipCidrRequest; -import com.amazonaws.services.ec2.model.AdvertiseByoipCidrResult; -import com.amazonaws.services.ec2.model.AllocateAddressRequest; -import com.amazonaws.services.ec2.model.AllocateAddressResult; -import com.amazonaws.services.ec2.model.AllocateHostsRequest; -import com.amazonaws.services.ec2.model.AllocateHostsResult; -import com.amazonaws.services.ec2.model.ApplySecurityGroupsToClientVpnTargetNetworkRequest; -import com.amazonaws.services.ec2.model.ApplySecurityGroupsToClientVpnTargetNetworkResult; -import com.amazonaws.services.ec2.model.AssignIpv6AddressesRequest; -import com.amazonaws.services.ec2.model.AssignIpv6AddressesResult; -import com.amazonaws.services.ec2.model.AssignPrivateIpAddressesRequest; -import com.amazonaws.services.ec2.model.AssignPrivateIpAddressesResult; -import com.amazonaws.services.ec2.model.AssociateAddressRequest; -import com.amazonaws.services.ec2.model.AssociateAddressResult; -import com.amazonaws.services.ec2.model.AssociateClientVpnTargetNetworkRequest; -import com.amazonaws.services.ec2.model.AssociateClientVpnTargetNetworkResult; -import com.amazonaws.services.ec2.model.AssociateDhcpOptionsRequest; -import com.amazonaws.services.ec2.model.AssociateDhcpOptionsResult; -import com.amazonaws.services.ec2.model.AssociateIamInstanceProfileRequest; -import com.amazonaws.services.ec2.model.AssociateIamInstanceProfileResult; -import com.amazonaws.services.ec2.model.AssociateRouteTableRequest; -import com.amazonaws.services.ec2.model.AssociateRouteTableResult; -import com.amazonaws.services.ec2.model.AssociateSubnetCidrBlockRequest; -import com.amazonaws.services.ec2.model.AssociateSubnetCidrBlockResult; -import com.amazonaws.services.ec2.model.AssociateTransitGatewayRouteTableRequest; -import com.amazonaws.services.ec2.model.AssociateTransitGatewayRouteTableResult; -import com.amazonaws.services.ec2.model.AssociateVpcCidrBlockRequest; -import com.amazonaws.services.ec2.model.AssociateVpcCidrBlockResult; -import com.amazonaws.services.ec2.model.AttachClassicLinkVpcRequest; -import com.amazonaws.services.ec2.model.AttachClassicLinkVpcResult; -import com.amazonaws.services.ec2.model.AttachInternetGatewayRequest; -import com.amazonaws.services.ec2.model.AttachInternetGatewayResult; -import com.amazonaws.services.ec2.model.AttachNetworkInterfaceRequest; -import com.amazonaws.services.ec2.model.AttachNetworkInterfaceResult; -import com.amazonaws.services.ec2.model.AttachVolumeRequest; -import com.amazonaws.services.ec2.model.AttachVolumeResult; -import com.amazonaws.services.ec2.model.AttachVpnGatewayRequest; -import com.amazonaws.services.ec2.model.AttachVpnGatewayResult; -import com.amazonaws.services.ec2.model.AuthorizeClientVpnIngressRequest; -import com.amazonaws.services.ec2.model.AuthorizeClientVpnIngressResult; -import com.amazonaws.services.ec2.model.AuthorizeSecurityGroupEgressRequest; -import com.amazonaws.services.ec2.model.AuthorizeSecurityGroupEgressResult; -import com.amazonaws.services.ec2.model.AuthorizeSecurityGroupIngressRequest; -import com.amazonaws.services.ec2.model.AuthorizeSecurityGroupIngressResult; -import com.amazonaws.services.ec2.model.BundleInstanceRequest; -import com.amazonaws.services.ec2.model.BundleInstanceResult; -import com.amazonaws.services.ec2.model.CancelBundleTaskRequest; -import com.amazonaws.services.ec2.model.CancelBundleTaskResult; -import com.amazonaws.services.ec2.model.CancelCapacityReservationRequest; -import com.amazonaws.services.ec2.model.CancelCapacityReservationResult; -import com.amazonaws.services.ec2.model.CancelConversionTaskRequest; -import com.amazonaws.services.ec2.model.CancelConversionTaskResult; -import com.amazonaws.services.ec2.model.CancelExportTaskRequest; -import com.amazonaws.services.ec2.model.CancelExportTaskResult; -import com.amazonaws.services.ec2.model.CancelImportTaskRequest; -import com.amazonaws.services.ec2.model.CancelImportTaskResult; -import com.amazonaws.services.ec2.model.CancelReservedInstancesListingRequest; -import com.amazonaws.services.ec2.model.CancelReservedInstancesListingResult; -import com.amazonaws.services.ec2.model.CancelSpotFleetRequestsRequest; -import com.amazonaws.services.ec2.model.CancelSpotFleetRequestsResult; -import com.amazonaws.services.ec2.model.CancelSpotInstanceRequestsRequest; -import com.amazonaws.services.ec2.model.CancelSpotInstanceRequestsResult; -import com.amazonaws.services.ec2.model.ConfirmProductInstanceRequest; -import com.amazonaws.services.ec2.model.ConfirmProductInstanceResult; -import com.amazonaws.services.ec2.model.CopyFpgaImageRequest; -import com.amazonaws.services.ec2.model.CopyFpgaImageResult; -import com.amazonaws.services.ec2.model.CopyImageRequest; -import com.amazonaws.services.ec2.model.CopyImageResult; -import com.amazonaws.services.ec2.model.CopySnapshotRequest; -import com.amazonaws.services.ec2.model.CopySnapshotResult; -import com.amazonaws.services.ec2.model.CreateCapacityReservationRequest; -import com.amazonaws.services.ec2.model.CreateCapacityReservationResult; -import com.amazonaws.services.ec2.model.CreateClientVpnEndpointRequest; -import com.amazonaws.services.ec2.model.CreateClientVpnEndpointResult; -import com.amazonaws.services.ec2.model.CreateClientVpnRouteRequest; -import com.amazonaws.services.ec2.model.CreateClientVpnRouteResult; -import com.amazonaws.services.ec2.model.CreateCustomerGatewayRequest; -import com.amazonaws.services.ec2.model.CreateCustomerGatewayResult; -import com.amazonaws.services.ec2.model.CreateDefaultSubnetRequest; -import com.amazonaws.services.ec2.model.CreateDefaultSubnetResult; -import com.amazonaws.services.ec2.model.CreateDefaultVpcRequest; -import com.amazonaws.services.ec2.model.CreateDefaultVpcResult; -import com.amazonaws.services.ec2.model.CreateDhcpOptionsRequest; -import com.amazonaws.services.ec2.model.CreateDhcpOptionsResult; -import com.amazonaws.services.ec2.model.CreateEgressOnlyInternetGatewayRequest; -import com.amazonaws.services.ec2.model.CreateEgressOnlyInternetGatewayResult; -import com.amazonaws.services.ec2.model.CreateFleetRequest; -import com.amazonaws.services.ec2.model.CreateFleetResult; -import com.amazonaws.services.ec2.model.CreateFlowLogsRequest; -import com.amazonaws.services.ec2.model.CreateFlowLogsResult; -import com.amazonaws.services.ec2.model.CreateFpgaImageRequest; -import com.amazonaws.services.ec2.model.CreateFpgaImageResult; -import com.amazonaws.services.ec2.model.CreateImageRequest; -import com.amazonaws.services.ec2.model.CreateImageResult; -import com.amazonaws.services.ec2.model.CreateInstanceExportTaskRequest; -import com.amazonaws.services.ec2.model.CreateInstanceExportTaskResult; -import com.amazonaws.services.ec2.model.CreateInternetGatewayRequest; -import com.amazonaws.services.ec2.model.CreateInternetGatewayResult; -import com.amazonaws.services.ec2.model.CreateKeyPairRequest; -import com.amazonaws.services.ec2.model.CreateKeyPairResult; -import com.amazonaws.services.ec2.model.CreateLaunchTemplateRequest; -import com.amazonaws.services.ec2.model.CreateLaunchTemplateResult; -import com.amazonaws.services.ec2.model.CreateLaunchTemplateVersionRequest; -import com.amazonaws.services.ec2.model.CreateLaunchTemplateVersionResult; -import com.amazonaws.services.ec2.model.CreateNatGatewayRequest; -import com.amazonaws.services.ec2.model.CreateNatGatewayResult; -import com.amazonaws.services.ec2.model.CreateNetworkAclEntryRequest; -import com.amazonaws.services.ec2.model.CreateNetworkAclEntryResult; -import com.amazonaws.services.ec2.model.CreateNetworkAclRequest; -import com.amazonaws.services.ec2.model.CreateNetworkAclResult; -import com.amazonaws.services.ec2.model.CreateNetworkInterfacePermissionRequest; -import com.amazonaws.services.ec2.model.CreateNetworkInterfacePermissionResult; -import com.amazonaws.services.ec2.model.CreateNetworkInterfaceRequest; -import com.amazonaws.services.ec2.model.CreateNetworkInterfaceResult; -import com.amazonaws.services.ec2.model.CreatePlacementGroupRequest; -import com.amazonaws.services.ec2.model.CreatePlacementGroupResult; -import com.amazonaws.services.ec2.model.CreateReservedInstancesListingRequest; -import com.amazonaws.services.ec2.model.CreateReservedInstancesListingResult; -import com.amazonaws.services.ec2.model.CreateRouteRequest; -import com.amazonaws.services.ec2.model.CreateRouteResult; -import com.amazonaws.services.ec2.model.CreateRouteTableRequest; -import com.amazonaws.services.ec2.model.CreateRouteTableResult; -import com.amazonaws.services.ec2.model.CreateSecurityGroupRequest; -import com.amazonaws.services.ec2.model.CreateSecurityGroupResult; -import com.amazonaws.services.ec2.model.CreateSnapshotRequest; -import com.amazonaws.services.ec2.model.CreateSnapshotResult; -import com.amazonaws.services.ec2.model.CreateSpotDatafeedSubscriptionRequest; -import com.amazonaws.services.ec2.model.CreateSpotDatafeedSubscriptionResult; -import com.amazonaws.services.ec2.model.CreateSubnetRequest; -import com.amazonaws.services.ec2.model.CreateSubnetResult; -import com.amazonaws.services.ec2.model.CreateTagsRequest; -import com.amazonaws.services.ec2.model.CreateTagsResult; -import com.amazonaws.services.ec2.model.CreateTransitGatewayRequest; -import com.amazonaws.services.ec2.model.CreateTransitGatewayResult; -import com.amazonaws.services.ec2.model.CreateTransitGatewayRouteRequest; -import com.amazonaws.services.ec2.model.CreateTransitGatewayRouteResult; -import com.amazonaws.services.ec2.model.CreateTransitGatewayRouteTableRequest; -import com.amazonaws.services.ec2.model.CreateTransitGatewayRouteTableResult; -import com.amazonaws.services.ec2.model.CreateTransitGatewayVpcAttachmentRequest; -import com.amazonaws.services.ec2.model.CreateTransitGatewayVpcAttachmentResult; -import com.amazonaws.services.ec2.model.CreateVolumeRequest; -import com.amazonaws.services.ec2.model.CreateVolumeResult; -import com.amazonaws.services.ec2.model.CreateVpcEndpointConnectionNotificationRequest; -import com.amazonaws.services.ec2.model.CreateVpcEndpointConnectionNotificationResult; -import com.amazonaws.services.ec2.model.CreateVpcEndpointRequest; -import com.amazonaws.services.ec2.model.CreateVpcEndpointResult; -import com.amazonaws.services.ec2.model.CreateVpcEndpointServiceConfigurationRequest; -import com.amazonaws.services.ec2.model.CreateVpcEndpointServiceConfigurationResult; -import com.amazonaws.services.ec2.model.CreateVpcPeeringConnectionRequest; -import com.amazonaws.services.ec2.model.CreateVpcPeeringConnectionResult; -import com.amazonaws.services.ec2.model.CreateVpcRequest; -import com.amazonaws.services.ec2.model.CreateVpcResult; -import com.amazonaws.services.ec2.model.CreateVpnConnectionRequest; -import com.amazonaws.services.ec2.model.CreateVpnConnectionResult; -import com.amazonaws.services.ec2.model.CreateVpnConnectionRouteRequest; -import com.amazonaws.services.ec2.model.CreateVpnConnectionRouteResult; -import com.amazonaws.services.ec2.model.CreateVpnGatewayRequest; -import com.amazonaws.services.ec2.model.CreateVpnGatewayResult; -import com.amazonaws.services.ec2.model.DeleteClientVpnEndpointRequest; -import com.amazonaws.services.ec2.model.DeleteClientVpnEndpointResult; -import com.amazonaws.services.ec2.model.DeleteClientVpnRouteRequest; -import com.amazonaws.services.ec2.model.DeleteClientVpnRouteResult; -import com.amazonaws.services.ec2.model.DeleteCustomerGatewayRequest; -import com.amazonaws.services.ec2.model.DeleteCustomerGatewayResult; -import com.amazonaws.services.ec2.model.DeleteDhcpOptionsRequest; -import com.amazonaws.services.ec2.model.DeleteDhcpOptionsResult; -import com.amazonaws.services.ec2.model.DeleteEgressOnlyInternetGatewayRequest; -import com.amazonaws.services.ec2.model.DeleteEgressOnlyInternetGatewayResult; -import com.amazonaws.services.ec2.model.DeleteFleetsRequest; -import com.amazonaws.services.ec2.model.DeleteFleetsResult; -import com.amazonaws.services.ec2.model.DeleteFlowLogsRequest; -import com.amazonaws.services.ec2.model.DeleteFlowLogsResult; -import com.amazonaws.services.ec2.model.DeleteFpgaImageRequest; -import com.amazonaws.services.ec2.model.DeleteFpgaImageResult; -import com.amazonaws.services.ec2.model.DeleteInternetGatewayRequest; -import com.amazonaws.services.ec2.model.DeleteInternetGatewayResult; -import com.amazonaws.services.ec2.model.DeleteKeyPairRequest; -import com.amazonaws.services.ec2.model.DeleteKeyPairResult; -import com.amazonaws.services.ec2.model.DeleteLaunchTemplateRequest; -import com.amazonaws.services.ec2.model.DeleteLaunchTemplateResult; -import com.amazonaws.services.ec2.model.DeleteLaunchTemplateVersionsRequest; -import com.amazonaws.services.ec2.model.DeleteLaunchTemplateVersionsResult; -import com.amazonaws.services.ec2.model.DeleteNatGatewayRequest; -import com.amazonaws.services.ec2.model.DeleteNatGatewayResult; -import com.amazonaws.services.ec2.model.DeleteNetworkAclEntryRequest; -import com.amazonaws.services.ec2.model.DeleteNetworkAclEntryResult; -import com.amazonaws.services.ec2.model.DeleteNetworkAclRequest; -import com.amazonaws.services.ec2.model.DeleteNetworkAclResult; -import com.amazonaws.services.ec2.model.DeleteNetworkInterfacePermissionRequest; -import com.amazonaws.services.ec2.model.DeleteNetworkInterfacePermissionResult; -import com.amazonaws.services.ec2.model.DeleteNetworkInterfaceRequest; -import com.amazonaws.services.ec2.model.DeleteNetworkInterfaceResult; -import com.amazonaws.services.ec2.model.DeletePlacementGroupRequest; -import com.amazonaws.services.ec2.model.DeletePlacementGroupResult; -import com.amazonaws.services.ec2.model.DeleteRouteRequest; -import com.amazonaws.services.ec2.model.DeleteRouteResult; -import com.amazonaws.services.ec2.model.DeleteRouteTableRequest; -import com.amazonaws.services.ec2.model.DeleteRouteTableResult; -import com.amazonaws.services.ec2.model.DeleteSecurityGroupRequest; -import com.amazonaws.services.ec2.model.DeleteSecurityGroupResult; -import com.amazonaws.services.ec2.model.DeleteSnapshotRequest; -import com.amazonaws.services.ec2.model.DeleteSnapshotResult; -import com.amazonaws.services.ec2.model.DeleteSpotDatafeedSubscriptionRequest; -import com.amazonaws.services.ec2.model.DeleteSpotDatafeedSubscriptionResult; -import com.amazonaws.services.ec2.model.DeleteSubnetRequest; -import com.amazonaws.services.ec2.model.DeleteSubnetResult; -import com.amazonaws.services.ec2.model.DeleteTagsRequest; -import com.amazonaws.services.ec2.model.DeleteTagsResult; -import com.amazonaws.services.ec2.model.DeleteTransitGatewayRequest; -import com.amazonaws.services.ec2.model.DeleteTransitGatewayResult; -import com.amazonaws.services.ec2.model.DeleteTransitGatewayRouteRequest; -import com.amazonaws.services.ec2.model.DeleteTransitGatewayRouteResult; -import com.amazonaws.services.ec2.model.DeleteTransitGatewayRouteTableRequest; -import com.amazonaws.services.ec2.model.DeleteTransitGatewayRouteTableResult; -import com.amazonaws.services.ec2.model.DeleteTransitGatewayVpcAttachmentRequest; -import com.amazonaws.services.ec2.model.DeleteTransitGatewayVpcAttachmentResult; -import com.amazonaws.services.ec2.model.DeleteVolumeRequest; -import com.amazonaws.services.ec2.model.DeleteVolumeResult; -import com.amazonaws.services.ec2.model.DeleteVpcEndpointConnectionNotificationsRequest; -import com.amazonaws.services.ec2.model.DeleteVpcEndpointConnectionNotificationsResult; -import com.amazonaws.services.ec2.model.DeleteVpcEndpointServiceConfigurationsRequest; -import com.amazonaws.services.ec2.model.DeleteVpcEndpointServiceConfigurationsResult; -import com.amazonaws.services.ec2.model.DeleteVpcEndpointsRequest; -import com.amazonaws.services.ec2.model.DeleteVpcEndpointsResult; -import com.amazonaws.services.ec2.model.DeleteVpcPeeringConnectionRequest; -import com.amazonaws.services.ec2.model.DeleteVpcPeeringConnectionResult; -import com.amazonaws.services.ec2.model.DeleteVpcRequest; -import com.amazonaws.services.ec2.model.DeleteVpcResult; -import com.amazonaws.services.ec2.model.DeleteVpnConnectionRequest; -import com.amazonaws.services.ec2.model.DeleteVpnConnectionResult; -import com.amazonaws.services.ec2.model.DeleteVpnConnectionRouteRequest; -import com.amazonaws.services.ec2.model.DeleteVpnConnectionRouteResult; -import com.amazonaws.services.ec2.model.DeleteVpnGatewayRequest; -import com.amazonaws.services.ec2.model.DeleteVpnGatewayResult; -import com.amazonaws.services.ec2.model.DeprovisionByoipCidrRequest; -import com.amazonaws.services.ec2.model.DeprovisionByoipCidrResult; -import com.amazonaws.services.ec2.model.DeregisterImageRequest; -import com.amazonaws.services.ec2.model.DeregisterImageResult; -import com.amazonaws.services.ec2.model.DescribeAccountAttributesRequest; -import com.amazonaws.services.ec2.model.DescribeAccountAttributesResult; -import com.amazonaws.services.ec2.model.DescribeAddressesRequest; -import com.amazonaws.services.ec2.model.DescribeAddressesResult; -import com.amazonaws.services.ec2.model.DescribeAggregateIdFormatRequest; -import com.amazonaws.services.ec2.model.DescribeAggregateIdFormatResult; -import com.amazonaws.services.ec2.model.DescribeAvailabilityZonesRequest; -import com.amazonaws.services.ec2.model.DescribeAvailabilityZonesResult; -import com.amazonaws.services.ec2.model.DescribeBundleTasksRequest; -import com.amazonaws.services.ec2.model.DescribeBundleTasksResult; -import com.amazonaws.services.ec2.model.DescribeByoipCidrsRequest; -import com.amazonaws.services.ec2.model.DescribeByoipCidrsResult; -import com.amazonaws.services.ec2.model.DescribeCapacityReservationsRequest; -import com.amazonaws.services.ec2.model.DescribeCapacityReservationsResult; -import com.amazonaws.services.ec2.model.DescribeClassicLinkInstancesRequest; -import com.amazonaws.services.ec2.model.DescribeClassicLinkInstancesResult; -import com.amazonaws.services.ec2.model.DescribeClientVpnAuthorizationRulesRequest; -import com.amazonaws.services.ec2.model.DescribeClientVpnAuthorizationRulesResult; -import com.amazonaws.services.ec2.model.DescribeClientVpnConnectionsRequest; -import com.amazonaws.services.ec2.model.DescribeClientVpnConnectionsResult; -import com.amazonaws.services.ec2.model.DescribeClientVpnEndpointsRequest; -import com.amazonaws.services.ec2.model.DescribeClientVpnEndpointsResult; -import com.amazonaws.services.ec2.model.DescribeClientVpnRoutesRequest; -import com.amazonaws.services.ec2.model.DescribeClientVpnRoutesResult; -import com.amazonaws.services.ec2.model.DescribeClientVpnTargetNetworksRequest; -import com.amazonaws.services.ec2.model.DescribeClientVpnTargetNetworksResult; -import com.amazonaws.services.ec2.model.DescribeConversionTasksRequest; -import com.amazonaws.services.ec2.model.DescribeConversionTasksResult; -import com.amazonaws.services.ec2.model.DescribeCustomerGatewaysRequest; -import com.amazonaws.services.ec2.model.DescribeCustomerGatewaysResult; -import com.amazonaws.services.ec2.model.DescribeDhcpOptionsRequest; -import com.amazonaws.services.ec2.model.DescribeDhcpOptionsResult; -import com.amazonaws.services.ec2.model.DescribeEgressOnlyInternetGatewaysRequest; -import com.amazonaws.services.ec2.model.DescribeEgressOnlyInternetGatewaysResult; -import com.amazonaws.services.ec2.model.DescribeElasticGpusRequest; -import com.amazonaws.services.ec2.model.DescribeElasticGpusResult; -import com.amazonaws.services.ec2.model.DescribeExportTasksRequest; -import com.amazonaws.services.ec2.model.DescribeExportTasksResult; -import com.amazonaws.services.ec2.model.DescribeFleetHistoryRequest; -import com.amazonaws.services.ec2.model.DescribeFleetHistoryResult; -import com.amazonaws.services.ec2.model.DescribeFleetInstancesRequest; -import com.amazonaws.services.ec2.model.DescribeFleetInstancesResult; -import com.amazonaws.services.ec2.model.DescribeFleetsRequest; -import com.amazonaws.services.ec2.model.DescribeFleetsResult; -import com.amazonaws.services.ec2.model.DescribeFlowLogsRequest; -import com.amazonaws.services.ec2.model.DescribeFlowLogsResult; -import com.amazonaws.services.ec2.model.DescribeFpgaImageAttributeRequest; -import com.amazonaws.services.ec2.model.DescribeFpgaImageAttributeResult; -import com.amazonaws.services.ec2.model.DescribeFpgaImagesRequest; -import com.amazonaws.services.ec2.model.DescribeFpgaImagesResult; -import com.amazonaws.services.ec2.model.DescribeHostReservationOfferingsRequest; -import com.amazonaws.services.ec2.model.DescribeHostReservationOfferingsResult; -import com.amazonaws.services.ec2.model.DescribeHostReservationsRequest; -import com.amazonaws.services.ec2.model.DescribeHostReservationsResult; -import com.amazonaws.services.ec2.model.DescribeHostsRequest; -import com.amazonaws.services.ec2.model.DescribeHostsResult; -import com.amazonaws.services.ec2.model.DescribeIamInstanceProfileAssociationsRequest; -import com.amazonaws.services.ec2.model.DescribeIamInstanceProfileAssociationsResult; -import com.amazonaws.services.ec2.model.DescribeIdFormatRequest; -import com.amazonaws.services.ec2.model.DescribeIdFormatResult; -import com.amazonaws.services.ec2.model.DescribeIdentityIdFormatRequest; -import com.amazonaws.services.ec2.model.DescribeIdentityIdFormatResult; -import com.amazonaws.services.ec2.model.DescribeImageAttributeRequest; -import com.amazonaws.services.ec2.model.DescribeImageAttributeResult; -import com.amazonaws.services.ec2.model.DescribeImagesRequest; -import com.amazonaws.services.ec2.model.DescribeImagesResult; -import com.amazonaws.services.ec2.model.DescribeImportImageTasksRequest; -import com.amazonaws.services.ec2.model.DescribeImportImageTasksResult; -import com.amazonaws.services.ec2.model.DescribeImportSnapshotTasksRequest; -import com.amazonaws.services.ec2.model.DescribeImportSnapshotTasksResult; -import com.amazonaws.services.ec2.model.DescribeInstanceAttributeRequest; -import com.amazonaws.services.ec2.model.DescribeInstanceAttributeResult; -import com.amazonaws.services.ec2.model.DescribeInstanceCreditSpecificationsRequest; -import com.amazonaws.services.ec2.model.DescribeInstanceCreditSpecificationsResult; -import com.amazonaws.services.ec2.model.DescribeInstanceStatusRequest; -import com.amazonaws.services.ec2.model.DescribeInstanceStatusResult; +import com.amazonaws.services.ec2.AbstractAmazonEC2; import com.amazonaws.services.ec2.model.DescribeInstancesRequest; import com.amazonaws.services.ec2.model.DescribeInstancesResult; -import com.amazonaws.services.ec2.model.DescribeInternetGatewaysRequest; -import com.amazonaws.services.ec2.model.DescribeInternetGatewaysResult; -import com.amazonaws.services.ec2.model.DescribeKeyPairsRequest; -import com.amazonaws.services.ec2.model.DescribeKeyPairsResult; -import com.amazonaws.services.ec2.model.DescribeLaunchTemplateVersionsRequest; -import com.amazonaws.services.ec2.model.DescribeLaunchTemplateVersionsResult; -import com.amazonaws.services.ec2.model.DescribeLaunchTemplatesRequest; -import com.amazonaws.services.ec2.model.DescribeLaunchTemplatesResult; -import com.amazonaws.services.ec2.model.DescribeMovingAddressesRequest; -import com.amazonaws.services.ec2.model.DescribeMovingAddressesResult; -import com.amazonaws.services.ec2.model.DescribeNatGatewaysRequest; -import com.amazonaws.services.ec2.model.DescribeNatGatewaysResult; -import com.amazonaws.services.ec2.model.DescribeNetworkAclsRequest; -import com.amazonaws.services.ec2.model.DescribeNetworkAclsResult; -import com.amazonaws.services.ec2.model.DescribeNetworkInterfaceAttributeRequest; -import com.amazonaws.services.ec2.model.DescribeNetworkInterfaceAttributeResult; -import com.amazonaws.services.ec2.model.DescribeNetworkInterfacePermissionsRequest; -import com.amazonaws.services.ec2.model.DescribeNetworkInterfacePermissionsResult; -import com.amazonaws.services.ec2.model.DescribeNetworkInterfacesRequest; -import com.amazonaws.services.ec2.model.DescribeNetworkInterfacesResult; -import com.amazonaws.services.ec2.model.DescribePlacementGroupsRequest; -import com.amazonaws.services.ec2.model.DescribePlacementGroupsResult; -import com.amazonaws.services.ec2.model.DescribePrefixListsRequest; -import com.amazonaws.services.ec2.model.DescribePrefixListsResult; -import com.amazonaws.services.ec2.model.DescribePrincipalIdFormatRequest; -import com.amazonaws.services.ec2.model.DescribePrincipalIdFormatResult; -import com.amazonaws.services.ec2.model.DescribePublicIpv4PoolsRequest; -import com.amazonaws.services.ec2.model.DescribePublicIpv4PoolsResult; -import com.amazonaws.services.ec2.model.DescribeRegionsRequest; -import com.amazonaws.services.ec2.model.DescribeRegionsResult; -import com.amazonaws.services.ec2.model.DescribeReservedInstancesListingsRequest; -import com.amazonaws.services.ec2.model.DescribeReservedInstancesListingsResult; -import com.amazonaws.services.ec2.model.DescribeReservedInstancesModificationsRequest; -import com.amazonaws.services.ec2.model.DescribeReservedInstancesModificationsResult; -import com.amazonaws.services.ec2.model.DescribeReservedInstancesOfferingsRequest; -import com.amazonaws.services.ec2.model.DescribeReservedInstancesOfferingsResult; -import com.amazonaws.services.ec2.model.DescribeReservedInstancesRequest; -import com.amazonaws.services.ec2.model.DescribeReservedInstancesResult; -import com.amazonaws.services.ec2.model.DescribeRouteTablesRequest; -import com.amazonaws.services.ec2.model.DescribeRouteTablesResult; -import com.amazonaws.services.ec2.model.DescribeScheduledInstanceAvailabilityRequest; -import com.amazonaws.services.ec2.model.DescribeScheduledInstanceAvailabilityResult; -import com.amazonaws.services.ec2.model.DescribeScheduledInstancesRequest; -import com.amazonaws.services.ec2.model.DescribeScheduledInstancesResult; -import com.amazonaws.services.ec2.model.DescribeSecurityGroupReferencesRequest; -import com.amazonaws.services.ec2.model.DescribeSecurityGroupReferencesResult; -import com.amazonaws.services.ec2.model.DescribeSecurityGroupsRequest; -import com.amazonaws.services.ec2.model.DescribeSecurityGroupsResult; -import com.amazonaws.services.ec2.model.DescribeSnapshotAttributeRequest; -import com.amazonaws.services.ec2.model.DescribeSnapshotAttributeResult; -import com.amazonaws.services.ec2.model.DescribeSnapshotsRequest; -import com.amazonaws.services.ec2.model.DescribeSnapshotsResult; -import com.amazonaws.services.ec2.model.DescribeSpotDatafeedSubscriptionRequest; -import com.amazonaws.services.ec2.model.DescribeSpotDatafeedSubscriptionResult; -import com.amazonaws.services.ec2.model.DescribeSpotFleetInstancesRequest; -import com.amazonaws.services.ec2.model.DescribeSpotFleetInstancesResult; -import com.amazonaws.services.ec2.model.DescribeSpotFleetRequestHistoryRequest; -import com.amazonaws.services.ec2.model.DescribeSpotFleetRequestHistoryResult; -import com.amazonaws.services.ec2.model.DescribeSpotFleetRequestsRequest; -import com.amazonaws.services.ec2.model.DescribeSpotFleetRequestsResult; -import com.amazonaws.services.ec2.model.DescribeSpotInstanceRequestsRequest; -import com.amazonaws.services.ec2.model.DescribeSpotInstanceRequestsResult; -import com.amazonaws.services.ec2.model.DescribeSpotPriceHistoryRequest; -import com.amazonaws.services.ec2.model.DescribeSpotPriceHistoryResult; -import com.amazonaws.services.ec2.model.DescribeStaleSecurityGroupsRequest; -import com.amazonaws.services.ec2.model.DescribeStaleSecurityGroupsResult; -import com.amazonaws.services.ec2.model.DescribeSubnetsRequest; -import com.amazonaws.services.ec2.model.DescribeSubnetsResult; -import com.amazonaws.services.ec2.model.DescribeTagsRequest; -import com.amazonaws.services.ec2.model.DescribeTagsResult; -import com.amazonaws.services.ec2.model.DescribeTransitGatewayAttachmentsRequest; -import com.amazonaws.services.ec2.model.DescribeTransitGatewayAttachmentsResult; -import com.amazonaws.services.ec2.model.DescribeTransitGatewayRouteTablesRequest; -import com.amazonaws.services.ec2.model.DescribeTransitGatewayRouteTablesResult; -import com.amazonaws.services.ec2.model.DescribeTransitGatewayVpcAttachmentsRequest; -import com.amazonaws.services.ec2.model.DescribeTransitGatewayVpcAttachmentsResult; -import com.amazonaws.services.ec2.model.DescribeTransitGatewaysRequest; -import com.amazonaws.services.ec2.model.DescribeTransitGatewaysResult; -import com.amazonaws.services.ec2.model.DescribeVolumeAttributeRequest; -import com.amazonaws.services.ec2.model.DescribeVolumeAttributeResult; -import com.amazonaws.services.ec2.model.DescribeVolumeStatusRequest; -import com.amazonaws.services.ec2.model.DescribeVolumeStatusResult; -import com.amazonaws.services.ec2.model.DescribeVolumesModificationsRequest; -import com.amazonaws.services.ec2.model.DescribeVolumesModificationsResult; -import com.amazonaws.services.ec2.model.DescribeVolumesRequest; -import com.amazonaws.services.ec2.model.DescribeVolumesResult; -import com.amazonaws.services.ec2.model.DescribeVpcAttributeRequest; -import com.amazonaws.services.ec2.model.DescribeVpcAttributeResult; -import com.amazonaws.services.ec2.model.DescribeVpcClassicLinkDnsSupportRequest; -import com.amazonaws.services.ec2.model.DescribeVpcClassicLinkDnsSupportResult; -import com.amazonaws.services.ec2.model.DescribeVpcClassicLinkRequest; -import com.amazonaws.services.ec2.model.DescribeVpcClassicLinkResult; -import com.amazonaws.services.ec2.model.DescribeVpcEndpointConnectionNotificationsRequest; -import com.amazonaws.services.ec2.model.DescribeVpcEndpointConnectionNotificationsResult; -import com.amazonaws.services.ec2.model.DescribeVpcEndpointConnectionsRequest; -import com.amazonaws.services.ec2.model.DescribeVpcEndpointConnectionsResult; -import com.amazonaws.services.ec2.model.DescribeVpcEndpointServiceConfigurationsRequest; -import com.amazonaws.services.ec2.model.DescribeVpcEndpointServiceConfigurationsResult; -import com.amazonaws.services.ec2.model.DescribeVpcEndpointServicePermissionsRequest; -import com.amazonaws.services.ec2.model.DescribeVpcEndpointServicePermissionsResult; -import com.amazonaws.services.ec2.model.DescribeVpcEndpointServicesRequest; -import com.amazonaws.services.ec2.model.DescribeVpcEndpointServicesResult; -import com.amazonaws.services.ec2.model.DescribeVpcEndpointsRequest; -import com.amazonaws.services.ec2.model.DescribeVpcEndpointsResult; -import com.amazonaws.services.ec2.model.DescribeVpcPeeringConnectionsRequest; -import com.amazonaws.services.ec2.model.DescribeVpcPeeringConnectionsResult; -import com.amazonaws.services.ec2.model.DescribeVpcsRequest; -import com.amazonaws.services.ec2.model.DescribeVpcsResult; -import com.amazonaws.services.ec2.model.DescribeVpnConnectionsRequest; -import com.amazonaws.services.ec2.model.DescribeVpnConnectionsResult; -import com.amazonaws.services.ec2.model.DescribeVpnGatewaysRequest; -import com.amazonaws.services.ec2.model.DescribeVpnGatewaysResult; -import com.amazonaws.services.ec2.model.DetachClassicLinkVpcRequest; -import com.amazonaws.services.ec2.model.DetachClassicLinkVpcResult; -import com.amazonaws.services.ec2.model.DetachInternetGatewayRequest; -import com.amazonaws.services.ec2.model.DetachInternetGatewayResult; -import com.amazonaws.services.ec2.model.DetachNetworkInterfaceRequest; -import com.amazonaws.services.ec2.model.DetachNetworkInterfaceResult; -import com.amazonaws.services.ec2.model.DetachVolumeRequest; -import com.amazonaws.services.ec2.model.DetachVolumeResult; -import com.amazonaws.services.ec2.model.DetachVpnGatewayRequest; -import com.amazonaws.services.ec2.model.DetachVpnGatewayResult; -import com.amazonaws.services.ec2.model.DisableTransitGatewayRouteTablePropagationRequest; -import com.amazonaws.services.ec2.model.DisableTransitGatewayRouteTablePropagationResult; -import com.amazonaws.services.ec2.model.DisableVgwRoutePropagationRequest; -import com.amazonaws.services.ec2.model.DisableVgwRoutePropagationResult; -import com.amazonaws.services.ec2.model.DisableVpcClassicLinkDnsSupportRequest; -import com.amazonaws.services.ec2.model.DisableVpcClassicLinkDnsSupportResult; -import com.amazonaws.services.ec2.model.DisableVpcClassicLinkRequest; -import com.amazonaws.services.ec2.model.DisableVpcClassicLinkResult; -import com.amazonaws.services.ec2.model.DisassociateAddressRequest; -import com.amazonaws.services.ec2.model.DisassociateAddressResult; -import com.amazonaws.services.ec2.model.DisassociateClientVpnTargetNetworkRequest; -import com.amazonaws.services.ec2.model.DisassociateClientVpnTargetNetworkResult; -import com.amazonaws.services.ec2.model.DisassociateIamInstanceProfileRequest; -import com.amazonaws.services.ec2.model.DisassociateIamInstanceProfileResult; -import com.amazonaws.services.ec2.model.DisassociateRouteTableRequest; -import com.amazonaws.services.ec2.model.DisassociateRouteTableResult; -import com.amazonaws.services.ec2.model.DisassociateSubnetCidrBlockRequest; -import com.amazonaws.services.ec2.model.DisassociateSubnetCidrBlockResult; -import com.amazonaws.services.ec2.model.DisassociateTransitGatewayRouteTableRequest; -import com.amazonaws.services.ec2.model.DisassociateTransitGatewayRouteTableResult; -import com.amazonaws.services.ec2.model.DisassociateVpcCidrBlockRequest; -import com.amazonaws.services.ec2.model.DisassociateVpcCidrBlockResult; -import com.amazonaws.services.ec2.model.DryRunResult; -import com.amazonaws.services.ec2.model.DryRunSupportedRequest; -import com.amazonaws.services.ec2.model.EnableTransitGatewayRouteTablePropagationRequest; -import com.amazonaws.services.ec2.model.EnableTransitGatewayRouteTablePropagationResult; -import com.amazonaws.services.ec2.model.EnableVgwRoutePropagationRequest; -import com.amazonaws.services.ec2.model.EnableVgwRoutePropagationResult; -import com.amazonaws.services.ec2.model.EnableVolumeIORequest; -import com.amazonaws.services.ec2.model.EnableVolumeIOResult; -import com.amazonaws.services.ec2.model.EnableVpcClassicLinkDnsSupportRequest; -import com.amazonaws.services.ec2.model.EnableVpcClassicLinkDnsSupportResult; -import com.amazonaws.services.ec2.model.EnableVpcClassicLinkRequest; -import com.amazonaws.services.ec2.model.EnableVpcClassicLinkResult; -import com.amazonaws.services.ec2.model.ExportClientVpnClientCertificateRevocationListRequest; -import com.amazonaws.services.ec2.model.ExportClientVpnClientCertificateRevocationListResult; -import com.amazonaws.services.ec2.model.ExportClientVpnClientConfigurationRequest; -import com.amazonaws.services.ec2.model.ExportClientVpnClientConfigurationResult; -import com.amazonaws.services.ec2.model.ExportTransitGatewayRoutesRequest; -import com.amazonaws.services.ec2.model.ExportTransitGatewayRoutesResult; import com.amazonaws.services.ec2.model.Filter; -import com.amazonaws.services.ec2.model.GetConsoleOutputRequest; -import com.amazonaws.services.ec2.model.GetConsoleOutputResult; -import com.amazonaws.services.ec2.model.GetConsoleScreenshotRequest; -import com.amazonaws.services.ec2.model.GetConsoleScreenshotResult; -import com.amazonaws.services.ec2.model.GetHostReservationPurchasePreviewRequest; -import com.amazonaws.services.ec2.model.GetHostReservationPurchasePreviewResult; -import com.amazonaws.services.ec2.model.GetLaunchTemplateDataRequest; -import com.amazonaws.services.ec2.model.GetLaunchTemplateDataResult; -import com.amazonaws.services.ec2.model.GetPasswordDataRequest; -import com.amazonaws.services.ec2.model.GetPasswordDataResult; -import com.amazonaws.services.ec2.model.GetReservedInstancesExchangeQuoteRequest; -import com.amazonaws.services.ec2.model.GetReservedInstancesExchangeQuoteResult; -import com.amazonaws.services.ec2.model.GetTransitGatewayAttachmentPropagationsRequest; -import com.amazonaws.services.ec2.model.GetTransitGatewayAttachmentPropagationsResult; -import com.amazonaws.services.ec2.model.GetTransitGatewayRouteTableAssociationsRequest; -import com.amazonaws.services.ec2.model.GetTransitGatewayRouteTableAssociationsResult; -import com.amazonaws.services.ec2.model.GetTransitGatewayRouteTablePropagationsRequest; -import com.amazonaws.services.ec2.model.GetTransitGatewayRouteTablePropagationsResult; -import com.amazonaws.services.ec2.model.ImportClientVpnClientCertificateRevocationListRequest; -import com.amazonaws.services.ec2.model.ImportClientVpnClientCertificateRevocationListResult; -import com.amazonaws.services.ec2.model.ImportImageRequest; -import com.amazonaws.services.ec2.model.ImportImageResult; -import com.amazonaws.services.ec2.model.ImportInstanceRequest; -import com.amazonaws.services.ec2.model.ImportInstanceResult; -import com.amazonaws.services.ec2.model.ImportKeyPairRequest; -import com.amazonaws.services.ec2.model.ImportKeyPairResult; -import com.amazonaws.services.ec2.model.ImportSnapshotRequest; -import com.amazonaws.services.ec2.model.ImportSnapshotResult; -import com.amazonaws.services.ec2.model.ImportVolumeRequest; -import com.amazonaws.services.ec2.model.ImportVolumeResult; import com.amazonaws.services.ec2.model.Instance; import com.amazonaws.services.ec2.model.InstanceState; import com.amazonaws.services.ec2.model.InstanceStateName; -import com.amazonaws.services.ec2.model.ModifyCapacityReservationRequest; -import com.amazonaws.services.ec2.model.ModifyCapacityReservationResult; -import com.amazonaws.services.ec2.model.ModifyClientVpnEndpointRequest; -import com.amazonaws.services.ec2.model.ModifyClientVpnEndpointResult; -import com.amazonaws.services.ec2.model.ModifyFleetRequest; -import com.amazonaws.services.ec2.model.ModifyFleetResult; -import com.amazonaws.services.ec2.model.ModifyFpgaImageAttributeRequest; -import com.amazonaws.services.ec2.model.ModifyFpgaImageAttributeResult; -import com.amazonaws.services.ec2.model.ModifyHostsRequest; -import com.amazonaws.services.ec2.model.ModifyHostsResult; -import com.amazonaws.services.ec2.model.ModifyIdFormatRequest; -import com.amazonaws.services.ec2.model.ModifyIdFormatResult; -import com.amazonaws.services.ec2.model.ModifyIdentityIdFormatRequest; -import com.amazonaws.services.ec2.model.ModifyIdentityIdFormatResult; -import com.amazonaws.services.ec2.model.ModifyImageAttributeRequest; -import com.amazonaws.services.ec2.model.ModifyImageAttributeResult; -import com.amazonaws.services.ec2.model.ModifyInstanceAttributeRequest; -import com.amazonaws.services.ec2.model.ModifyInstanceAttributeResult; -import com.amazonaws.services.ec2.model.ModifyInstanceCapacityReservationAttributesRequest; -import com.amazonaws.services.ec2.model.ModifyInstanceCapacityReservationAttributesResult; -import com.amazonaws.services.ec2.model.ModifyInstanceCreditSpecificationRequest; -import com.amazonaws.services.ec2.model.ModifyInstanceCreditSpecificationResult; -import com.amazonaws.services.ec2.model.ModifyInstancePlacementRequest; -import com.amazonaws.services.ec2.model.ModifyInstancePlacementResult; -import com.amazonaws.services.ec2.model.ModifyLaunchTemplateRequest; -import com.amazonaws.services.ec2.model.ModifyLaunchTemplateResult; -import com.amazonaws.services.ec2.model.ModifyNetworkInterfaceAttributeRequest; -import com.amazonaws.services.ec2.model.ModifyNetworkInterfaceAttributeResult; -import com.amazonaws.services.ec2.model.ModifyReservedInstancesRequest; -import com.amazonaws.services.ec2.model.ModifyReservedInstancesResult; -import com.amazonaws.services.ec2.model.ModifySnapshotAttributeRequest; -import com.amazonaws.services.ec2.model.ModifySnapshotAttributeResult; -import com.amazonaws.services.ec2.model.ModifySpotFleetRequestRequest; -import com.amazonaws.services.ec2.model.ModifySpotFleetRequestResult; -import com.amazonaws.services.ec2.model.ModifySubnetAttributeRequest; -import com.amazonaws.services.ec2.model.ModifySubnetAttributeResult; -import com.amazonaws.services.ec2.model.ModifyTransitGatewayVpcAttachmentRequest; -import com.amazonaws.services.ec2.model.ModifyTransitGatewayVpcAttachmentResult; -import com.amazonaws.services.ec2.model.ModifyVolumeAttributeRequest; -import com.amazonaws.services.ec2.model.ModifyVolumeAttributeResult; -import com.amazonaws.services.ec2.model.ModifyVolumeRequest; -import com.amazonaws.services.ec2.model.ModifyVolumeResult; -import com.amazonaws.services.ec2.model.ModifyVpcAttributeRequest; -import com.amazonaws.services.ec2.model.ModifyVpcAttributeResult; -import com.amazonaws.services.ec2.model.ModifyVpcEndpointConnectionNotificationRequest; -import com.amazonaws.services.ec2.model.ModifyVpcEndpointConnectionNotificationResult; -import com.amazonaws.services.ec2.model.ModifyVpcEndpointRequest; -import com.amazonaws.services.ec2.model.ModifyVpcEndpointResult; -import com.amazonaws.services.ec2.model.ModifyVpcEndpointServiceConfigurationRequest; -import com.amazonaws.services.ec2.model.ModifyVpcEndpointServiceConfigurationResult; -import com.amazonaws.services.ec2.model.ModifyVpcEndpointServicePermissionsRequest; -import com.amazonaws.services.ec2.model.ModifyVpcEndpointServicePermissionsResult; -import com.amazonaws.services.ec2.model.ModifyVpcPeeringConnectionOptionsRequest; -import com.amazonaws.services.ec2.model.ModifyVpcPeeringConnectionOptionsResult; -import com.amazonaws.services.ec2.model.ModifyVpcTenancyRequest; -import com.amazonaws.services.ec2.model.ModifyVpcTenancyResult; -import com.amazonaws.services.ec2.model.MonitorInstancesRequest; -import com.amazonaws.services.ec2.model.MonitorInstancesResult; -import com.amazonaws.services.ec2.model.MoveAddressToVpcRequest; -import com.amazonaws.services.ec2.model.MoveAddressToVpcResult; -import com.amazonaws.services.ec2.model.ProvisionByoipCidrRequest; -import com.amazonaws.services.ec2.model.ProvisionByoipCidrResult; -import com.amazonaws.services.ec2.model.PurchaseHostReservationRequest; -import com.amazonaws.services.ec2.model.PurchaseHostReservationResult; -import com.amazonaws.services.ec2.model.PurchaseReservedInstancesOfferingRequest; -import com.amazonaws.services.ec2.model.PurchaseReservedInstancesOfferingResult; -import com.amazonaws.services.ec2.model.PurchaseScheduledInstancesRequest; -import com.amazonaws.services.ec2.model.PurchaseScheduledInstancesResult; -import com.amazonaws.services.ec2.model.RebootInstancesRequest; -import com.amazonaws.services.ec2.model.RebootInstancesResult; -import com.amazonaws.services.ec2.model.RegisterImageRequest; -import com.amazonaws.services.ec2.model.RegisterImageResult; -import com.amazonaws.services.ec2.model.RejectTransitGatewayVpcAttachmentRequest; -import com.amazonaws.services.ec2.model.RejectTransitGatewayVpcAttachmentResult; -import com.amazonaws.services.ec2.model.RejectVpcEndpointConnectionsRequest; -import com.amazonaws.services.ec2.model.RejectVpcEndpointConnectionsResult; -import com.amazonaws.services.ec2.model.RejectVpcPeeringConnectionRequest; -import com.amazonaws.services.ec2.model.RejectVpcPeeringConnectionResult; -import com.amazonaws.services.ec2.model.ReleaseAddressRequest; -import com.amazonaws.services.ec2.model.ReleaseAddressResult; -import com.amazonaws.services.ec2.model.ReleaseHostsRequest; -import com.amazonaws.services.ec2.model.ReleaseHostsResult; -import com.amazonaws.services.ec2.model.ReplaceIamInstanceProfileAssociationRequest; -import com.amazonaws.services.ec2.model.ReplaceIamInstanceProfileAssociationResult; -import com.amazonaws.services.ec2.model.ReplaceNetworkAclAssociationRequest; -import com.amazonaws.services.ec2.model.ReplaceNetworkAclAssociationResult; -import com.amazonaws.services.ec2.model.ReplaceNetworkAclEntryRequest; -import com.amazonaws.services.ec2.model.ReplaceNetworkAclEntryResult; -import com.amazonaws.services.ec2.model.ReplaceRouteRequest; -import com.amazonaws.services.ec2.model.ReplaceRouteResult; -import com.amazonaws.services.ec2.model.ReplaceRouteTableAssociationRequest; -import com.amazonaws.services.ec2.model.ReplaceRouteTableAssociationResult; -import com.amazonaws.services.ec2.model.ReplaceTransitGatewayRouteRequest; -import com.amazonaws.services.ec2.model.ReplaceTransitGatewayRouteResult; -import com.amazonaws.services.ec2.model.ReportInstanceStatusRequest; -import com.amazonaws.services.ec2.model.ReportInstanceStatusResult; -import com.amazonaws.services.ec2.model.RequestSpotFleetRequest; -import com.amazonaws.services.ec2.model.RequestSpotFleetResult; -import com.amazonaws.services.ec2.model.RequestSpotInstancesRequest; -import com.amazonaws.services.ec2.model.RequestSpotInstancesResult; import com.amazonaws.services.ec2.model.Reservation; -import com.amazonaws.services.ec2.model.ResetFpgaImageAttributeRequest; -import com.amazonaws.services.ec2.model.ResetFpgaImageAttributeResult; -import com.amazonaws.services.ec2.model.ResetImageAttributeRequest; -import com.amazonaws.services.ec2.model.ResetImageAttributeResult; -import com.amazonaws.services.ec2.model.ResetInstanceAttributeRequest; -import com.amazonaws.services.ec2.model.ResetInstanceAttributeResult; -import com.amazonaws.services.ec2.model.ResetNetworkInterfaceAttributeRequest; -import com.amazonaws.services.ec2.model.ResetNetworkInterfaceAttributeResult; -import com.amazonaws.services.ec2.model.ResetSnapshotAttributeRequest; -import com.amazonaws.services.ec2.model.ResetSnapshotAttributeResult; -import com.amazonaws.services.ec2.model.RestoreAddressToClassicRequest; -import com.amazonaws.services.ec2.model.RestoreAddressToClassicResult; -import com.amazonaws.services.ec2.model.RevokeClientVpnIngressRequest; -import com.amazonaws.services.ec2.model.RevokeClientVpnIngressResult; -import com.amazonaws.services.ec2.model.RevokeSecurityGroupEgressRequest; -import com.amazonaws.services.ec2.model.RevokeSecurityGroupEgressResult; -import com.amazonaws.services.ec2.model.RevokeSecurityGroupIngressRequest; -import com.amazonaws.services.ec2.model.RevokeSecurityGroupIngressResult; -import com.amazonaws.services.ec2.model.RunInstancesRequest; -import com.amazonaws.services.ec2.model.RunInstancesResult; -import com.amazonaws.services.ec2.model.RunScheduledInstancesRequest; -import com.amazonaws.services.ec2.model.RunScheduledInstancesResult; -import com.amazonaws.services.ec2.model.SearchTransitGatewayRoutesRequest; -import com.amazonaws.services.ec2.model.SearchTransitGatewayRoutesResult; -import com.amazonaws.services.ec2.model.StartInstancesRequest; -import com.amazonaws.services.ec2.model.StartInstancesResult; -import com.amazonaws.services.ec2.model.StopInstancesRequest; -import com.amazonaws.services.ec2.model.StopInstancesResult; import com.amazonaws.services.ec2.model.Tag; -import com.amazonaws.services.ec2.model.TerminateClientVpnConnectionsRequest; -import com.amazonaws.services.ec2.model.TerminateClientVpnConnectionsResult; -import com.amazonaws.services.ec2.model.TerminateInstancesRequest; -import com.amazonaws.services.ec2.model.TerminateInstancesResult; -import com.amazonaws.services.ec2.model.UnassignIpv6AddressesRequest; -import com.amazonaws.services.ec2.model.UnassignIpv6AddressesResult; -import com.amazonaws.services.ec2.model.UnassignPrivateIpAddressesRequest; -import com.amazonaws.services.ec2.model.UnassignPrivateIpAddressesResult; -import com.amazonaws.services.ec2.model.UnmonitorInstancesRequest; -import com.amazonaws.services.ec2.model.UnmonitorInstancesResult; -import com.amazonaws.services.ec2.model.UpdateSecurityGroupRuleDescriptionsEgressRequest; -import com.amazonaws.services.ec2.model.UpdateSecurityGroupRuleDescriptionsEgressResult; -import com.amazonaws.services.ec2.model.UpdateSecurityGroupRuleDescriptionsIngressRequest; -import com.amazonaws.services.ec2.model.UpdateSecurityGroupRuleDescriptionsIngressResult; -import com.amazonaws.services.ec2.model.WithdrawByoipCidrRequest; -import com.amazonaws.services.ec2.model.WithdrawByoipCidrResult; -import com.amazonaws.services.ec2.waiters.AmazonEC2Waiters; import org.apache.logging.log4j.LogManager; import org.apache.logging.log4j.Logger; @@ -704,7 +42,7 @@ import java.util.regex.Matcher; import java.util.regex.Pattern; -public class AmazonEC2Mock implements AmazonEC2 { +public class AmazonEC2Mock extends AbstractAmazonEC2 { private static final Logger logger = LogManager.getLogger(AmazonEC2Mock.class); @@ -830,2250 +168,12 @@ public DescribeInstancesResult describeInstances(DescribeInstancesRequest descri ); } - // Not implemented methods in Mock - @Override public void setEndpoint(String endpoint) throws IllegalArgumentException { this.endpoint = endpoint; } - @Override - public void setRegion(Region region) throws IllegalArgumentException { - throw new UnsupportedOperationException("Not supported in mock"); - } - - @Override - public AcceptReservedInstancesExchangeQuoteResult acceptReservedInstancesExchangeQuote( - AcceptReservedInstancesExchangeQuoteRequest acceptReservedInstancesExchangeQuoteRequest) - throws AmazonClientException { - throw new UnsupportedOperationException("Not supported in mock"); - } - - @Override - public AcceptTransitGatewayVpcAttachmentResult acceptTransitGatewayVpcAttachment( - AcceptTransitGatewayVpcAttachmentRequest acceptTransitGatewayVpcAttachmentRequest) { - throw new UnsupportedOperationException("Not supported in mock"); - } - - @Override - public AcceptVpcEndpointConnectionsResult acceptVpcEndpointConnections( - AcceptVpcEndpointConnectionsRequest acceptVpcEndpointConnectionsRequest) { - throw new UnsupportedOperationException("Not supported in mock"); - } - - @Override - public RebootInstancesResult rebootInstances(RebootInstancesRequest rebootInstancesRequest) - throws AmazonClientException { - throw new UnsupportedOperationException("Not supported in mock"); - } - - @Override - public DescribeReservedInstancesResult describeReservedInstances( - DescribeReservedInstancesRequest describeReservedInstancesRequest) - throws AmazonClientException { - throw new UnsupportedOperationException("Not supported in mock"); - } - - @Override - public CreateFlowLogsResult createFlowLogs(CreateFlowLogsRequest createFlowLogsRequest) - throws AmazonClientException { - throw new UnsupportedOperationException("Not supported in mock"); - } - - @Override - public DescribeAvailabilityZonesResult describeAvailabilityZones(DescribeAvailabilityZonesRequest describeAvailabilityZonesRequest) - throws AmazonClientException { - throw new UnsupportedOperationException("Not supported in mock"); - } - - @Override - public RestoreAddressToClassicResult restoreAddressToClassic(RestoreAddressToClassicRequest restoreAddressToClassicRequest) - throws AmazonClientException { - throw new UnsupportedOperationException("Not supported in mock"); - } - - @Override - public RevokeClientVpnIngressResult revokeClientVpnIngress(RevokeClientVpnIngressRequest revokeClientVpnIngressRequest) { - throw new UnsupportedOperationException("Not supported in mock"); - } - - @Override - public DetachVolumeResult detachVolume(DetachVolumeRequest detachVolumeRequest) throws AmazonClientException { - throw new UnsupportedOperationException("Not supported in mock"); - } - - @Override - public DeleteKeyPairResult deleteKeyPair(DeleteKeyPairRequest deleteKeyPairRequest) - throws AmazonClientException { - throw new UnsupportedOperationException("Not supported in mock"); - } - - @Override - public DeleteLaunchTemplateResult deleteLaunchTemplate(DeleteLaunchTemplateRequest deleteLaunchTemplateRequest) { - throw new UnsupportedOperationException("Not supported in mock"); - } - - @Override - public DeleteLaunchTemplateVersionsResult deleteLaunchTemplateVersions( - DeleteLaunchTemplateVersionsRequest deleteLaunchTemplateVersionsRequest) { - throw new UnsupportedOperationException("Not supported in mock"); - } - - @Override - public DeleteNatGatewayResult deleteNatGateway(DeleteNatGatewayRequest deleteNatGatewayRequest) { - throw new UnsupportedOperationException("Not supported in mock"); - } - - @Override - public UnmonitorInstancesResult unmonitorInstances(UnmonitorInstancesRequest unmonitorInstancesRequest) - throws AmazonClientException { - throw new UnsupportedOperationException("Not supported in mock"); - } - - @Override - public UpdateSecurityGroupRuleDescriptionsIngressResult updateSecurityGroupRuleDescriptionsIngress( - UpdateSecurityGroupRuleDescriptionsIngressRequest updateSecurityGroupRuleDescriptionsIngressRequest) - throws AmazonClientException { - throw new UnsupportedOperationException("Not supported in mock"); - } - - @Override - public WithdrawByoipCidrResult withdrawByoipCidr(WithdrawByoipCidrRequest withdrawByoipCidrRequest) { - throw new UnsupportedOperationException("Not supported in mock"); - } - - @Override - public UpdateSecurityGroupRuleDescriptionsEgressResult updateSecurityGroupRuleDescriptionsEgress( - UpdateSecurityGroupRuleDescriptionsEgressRequest updateSecurityGroupRuleDescriptionsEgressRequest) - throws AmazonClientException { - throw new UnsupportedOperationException("Not supported in mock"); - } - - @Override - public AttachVpnGatewayResult attachVpnGateway(AttachVpnGatewayRequest attachVpnGatewayRequest) - throws AmazonClientException { - throw new UnsupportedOperationException("Not supported in mock"); - } - - @Override - public AuthorizeClientVpnIngressResult authorizeClientVpnIngress(AuthorizeClientVpnIngressRequest authorizeClientVpnIngressRequest) { - throw new UnsupportedOperationException("Not supported in mock"); - } - - @Override - public CreateImageResult createImage(CreateImageRequest createImageRequest) throws AmazonClientException { - throw new UnsupportedOperationException("Not supported in mock"); - } - - @Override - public DeleteSecurityGroupResult deleteSecurityGroup(DeleteSecurityGroupRequest deleteSecurityGroupRequest) - throws AmazonClientException { - throw new UnsupportedOperationException("Not supported in mock"); - } - - @Override - public CreateInstanceExportTaskResult createInstanceExportTask(CreateInstanceExportTaskRequest createInstanceExportTaskRequest) - throws AmazonClientException { - throw new UnsupportedOperationException("Not supported in mock"); - } - - @Override - public AuthorizeSecurityGroupEgressResult authorizeSecurityGroupEgress( - AuthorizeSecurityGroupEgressRequest authorizeSecurityGroupEgressRequest) throws AmazonClientException { - throw new UnsupportedOperationException("Not supported in mock"); - } - - @Override - public AssociateDhcpOptionsResult associateDhcpOptions(AssociateDhcpOptionsRequest associateDhcpOptionsRequest) - throws AmazonClientException { - throw new UnsupportedOperationException("Not supported in mock"); - } - - @Override - public GetPasswordDataResult getPasswordData(GetPasswordDataRequest getPasswordDataRequest) - throws AmazonClientException { - throw new UnsupportedOperationException("Not supported in mock"); - } - - @Override - public GetReservedInstancesExchangeQuoteResult getReservedInstancesExchangeQuote( - GetReservedInstancesExchangeQuoteRequest getReservedInstancesExchangeQuoteRequest) - throws AmazonClientException { - throw new UnsupportedOperationException("Not supported in mock"); - } - - @Override - public GetTransitGatewayAttachmentPropagationsResult getTransitGatewayAttachmentPropagations( - GetTransitGatewayAttachmentPropagationsRequest getTransitGatewayAttachmentPropagationsRequest) { - throw new UnsupportedOperationException("Not supported in mock"); - } - - @Override - public GetTransitGatewayRouteTableAssociationsResult getTransitGatewayRouteTableAssociations( - GetTransitGatewayRouteTableAssociationsRequest getTransitGatewayRouteTableAssociationsRequest) { - throw new UnsupportedOperationException("Not supported in mock"); - } - - @Override - public GetTransitGatewayRouteTablePropagationsResult getTransitGatewayRouteTablePropagations( - GetTransitGatewayRouteTablePropagationsRequest getTransitGatewayRouteTablePropagationsRequest) { - throw new UnsupportedOperationException("Not supported in mock"); - } - - @Override - public ImportClientVpnClientCertificateRevocationListResult importClientVpnClientCertificateRevocationList( - ImportClientVpnClientCertificateRevocationListRequest importClientVpnClientCertificateRevocationListRequest) { - throw new UnsupportedOperationException("Not supported in mock"); - } - - @Override - public StopInstancesResult stopInstances(StopInstancesRequest stopInstancesRequest) - throws AmazonClientException { - throw new UnsupportedOperationException("Not supported in mock"); - } - - @Override - public TerminateClientVpnConnectionsResult terminateClientVpnConnections( - TerminateClientVpnConnectionsRequest terminateClientVpnConnectionsRequest) { - throw new UnsupportedOperationException("Not supported in mock"); - } - - @Override - public ImportKeyPairResult importKeyPair(ImportKeyPairRequest importKeyPairRequest) - throws AmazonClientException { - throw new UnsupportedOperationException("Not supported in mock"); - } - - @Override - public DeleteNetworkInterfaceResult deleteNetworkInterface(DeleteNetworkInterfaceRequest deleteNetworkInterfaceRequest) - throws AmazonClientException { - throw new UnsupportedOperationException("Not supported in mock"); - } - - @Override - public ModifyVpcAttributeResult modifyVpcAttribute(ModifyVpcAttributeRequest modifyVpcAttributeRequest) - throws AmazonClientException { - throw new UnsupportedOperationException("Not supported in mock"); - } - - @Override - public DescribeSpotFleetInstancesResult describeSpotFleetInstances(DescribeSpotFleetInstancesRequest describeSpotFleetInstancesRequest) - throws AmazonClientException { - throw new UnsupportedOperationException("Not supported in mock"); - } - - @Override - public CreateSecurityGroupResult createSecurityGroup(CreateSecurityGroupRequest createSecurityGroupRequest) - throws AmazonClientException { - throw new UnsupportedOperationException("Not supported in mock"); - } - - @Override - public DescribeSpotPriceHistoryResult describeSpotPriceHistory(DescribeSpotPriceHistoryRequest describeSpotPriceHistoryRequest) - throws AmazonClientException { - throw new UnsupportedOperationException("Not supported in mock"); - } - - @Override - public DescribeNetworkInterfacesResult describeNetworkInterfaces(DescribeNetworkInterfacesRequest describeNetworkInterfacesRequest) - throws AmazonClientException { - throw new UnsupportedOperationException("Not supported in mock"); - } - - @Override - public DescribeNetworkInterfacePermissionsResult describeNetworkInterfacePermissions( - DescribeNetworkInterfacePermissionsRequest describeNetworkInterfacePermissionsRequest) - throws AmazonClientException { - throw new UnsupportedOperationException("Not supported in mock"); - } - - @Override - public DescribeRegionsResult describeRegions(DescribeRegionsRequest describeRegionsRequest) - throws AmazonClientException { - throw new UnsupportedOperationException("Not supported in mock"); - } - - @Override - public CreateDhcpOptionsResult createDhcpOptions(CreateDhcpOptionsRequest createDhcpOptionsRequest) - throws AmazonClientException { - throw new UnsupportedOperationException("Not supported in mock"); - } - - @Override - public CreateReservedInstancesListingResult createReservedInstancesListing( - CreateReservedInstancesListingRequest createReservedInstancesListingRequest) - throws AmazonClientException { - throw new UnsupportedOperationException("Not supported in mock"); - } - - @Override - public DeleteVpcEndpointsResult deleteVpcEndpoints(DeleteVpcEndpointsRequest deleteVpcEndpointsRequest) - throws AmazonClientException { - throw new UnsupportedOperationException("Not supported in mock"); - } - - @Override - public ResetSnapshotAttributeResult resetSnapshotAttribute(ResetSnapshotAttributeRequest resetSnapshotAttributeRequest) - throws AmazonClientException { - throw new UnsupportedOperationException("Not supported in mock"); - } - - @Override - public DeleteRouteResult deleteRoute(DeleteRouteRequest deleteRouteRequest) throws AmazonClientException { - throw new UnsupportedOperationException("Not supported in mock"); - } - - @Override - public DescribeInternetGatewaysResult describeInternetGateways(DescribeInternetGatewaysRequest describeInternetGatewaysRequest) - throws AmazonClientException { - throw new UnsupportedOperationException("Not supported in mock"); - } - - @Override - public ImportVolumeResult importVolume(ImportVolumeRequest importVolumeRequest) throws AmazonClientException { - throw new UnsupportedOperationException("Not supported in mock"); - } - - @Override - public ModifyCapacityReservationResult modifyCapacityReservation(ModifyCapacityReservationRequest modifyCapacityReservationRequest) { - throw new UnsupportedOperationException("Not supported in mock"); - } - - @Override - public ModifyClientVpnEndpointResult modifyClientVpnEndpoint(ModifyClientVpnEndpointRequest modifyClientVpnEndpointRequest) { - throw new UnsupportedOperationException("Not supported in mock"); - } - - @Override - public ModifyFleetResult modifyFleet(ModifyFleetRequest modifyFleetRequest) { - throw new UnsupportedOperationException("Not supported in mock"); - } - - @Override - public ModifyFpgaImageAttributeResult modifyFpgaImageAttribute(ModifyFpgaImageAttributeRequest modifyFpgaImageAttributeRequest) { - throw new UnsupportedOperationException("Not supported in mock"); - } - - @Override - public ModifyHostsResult modifyHosts(ModifyHostsRequest modifyHostsRequest) { - throw new UnsupportedOperationException("Not supported in mock"); - } - - @Override - public ModifyIdFormatResult modifyIdFormat(ModifyIdFormatRequest modifyIdFormatRequest) { - throw new UnsupportedOperationException("Not supported in mock"); - } - - @Override - public DescribeSecurityGroupsResult describeSecurityGroups(DescribeSecurityGroupsRequest describeSecurityGroupsRequest) - throws AmazonClientException { - throw new UnsupportedOperationException("Not supported in mock"); - } - - @Override - public DescribeStaleSecurityGroupsResult describeStaleSecurityGroups( - DescribeStaleSecurityGroupsRequest describeStaleSecurityGroupsRequest) - throws AmazonClientException { - throw new UnsupportedOperationException("Not supported in mock"); - } - - @Override - public DescribeSecurityGroupReferencesResult describeSecurityGroupReferences( - DescribeSecurityGroupReferencesRequest describeSecurityGroupReferencesRequest) - throws AmazonClientException { - throw new UnsupportedOperationException("Not supported in mock"); - } - - @Override - public RejectVpcPeeringConnectionResult rejectVpcPeeringConnection( - RejectVpcPeeringConnectionRequest rejectVpcPeeringConnectionRequest) - throws AmazonClientException { - throw new UnsupportedOperationException("Not supported in mock"); - } - - @Override - public ModifyVpcPeeringConnectionOptionsResult modifyVpcPeeringConnectionOptions( - ModifyVpcPeeringConnectionOptionsRequest modifyVpcPeeringConnectionOptionsRequest) - throws AmazonClientException { - throw new UnsupportedOperationException("Not supported in mock"); - } - - @Override - public ModifyVpcTenancyResult modifyVpcTenancy(ModifyVpcTenancyRequest modifyVpcTenancyRequest) { - throw new UnsupportedOperationException("Not supported in mock"); - } - - @Override - public DeleteFlowLogsResult deleteFlowLogs(DeleteFlowLogsRequest deleteFlowLogsRequest) - throws AmazonClientException { - throw new UnsupportedOperationException("Not supported in mock"); - } - - @Override - public DeleteFpgaImageResult deleteFpgaImage(DeleteFpgaImageRequest deleteFpgaImageRequest) { - throw new UnsupportedOperationException("Not supported in mock"); - } - - @Override - public DetachVpnGatewayResult detachVpnGateway(DetachVpnGatewayRequest detachVpnGatewayRequest) - throws AmazonClientException { - throw new UnsupportedOperationException("Not supported in mock"); - } - - @Override - public DisableTransitGatewayRouteTablePropagationResult disableTransitGatewayRouteTablePropagation( - DisableTransitGatewayRouteTablePropagationRequest disableTransitGatewayRouteTablePropagationRequest) { - throw new UnsupportedOperationException("Not supported in mock"); - } - - @Override - public DeregisterImageResult deregisterImage(DeregisterImageRequest deregisterImageRequest) - throws AmazonClientException { - throw new UnsupportedOperationException("Not supported in mock"); - } - - @Override - public DescribeSpotDatafeedSubscriptionResult describeSpotDatafeedSubscription( - DescribeSpotDatafeedSubscriptionRequest describeSpotDatafeedSubscriptionRequest) - throws AmazonClientException { - throw new UnsupportedOperationException("Not supported in mock"); - } - - @Override - public DeleteTagsResult deleteTags(DeleteTagsRequest deleteTagsRequest) throws AmazonClientException { - throw new UnsupportedOperationException("Not supported in mock"); - } - - @Override - public DeleteTransitGatewayResult deleteTransitGateway(DeleteTransitGatewayRequest deleteTransitGatewayRequest) { - throw new UnsupportedOperationException("Not supported in mock"); - } - - @Override - public DeleteTransitGatewayRouteResult deleteTransitGatewayRoute(DeleteTransitGatewayRouteRequest deleteTransitGatewayRouteRequest) { - throw new UnsupportedOperationException("Not supported in mock"); - } - - @Override - public DeleteTransitGatewayRouteTableResult deleteTransitGatewayRouteTable( - DeleteTransitGatewayRouteTableRequest deleteTransitGatewayRouteTableRequest) { - throw new UnsupportedOperationException("Not supported in mock"); - } - - @Override - public DeleteTransitGatewayVpcAttachmentResult deleteTransitGatewayVpcAttachment( - DeleteTransitGatewayVpcAttachmentRequest deleteTransitGatewayVpcAttachmentRequest) { - throw new UnsupportedOperationException("Not supported in mock"); - } - - @Override - public DeleteSubnetResult deleteSubnet(DeleteSubnetRequest deleteSubnetRequest) - throws AmazonClientException { - throw new UnsupportedOperationException("Not supported in mock"); - } - - @Override - public DescribeAccountAttributesResult describeAccountAttributes(DescribeAccountAttributesRequest describeAccountAttributesRequest) - throws AmazonClientException { - throw new UnsupportedOperationException("Not supported in mock"); - } - - @Override - public AttachClassicLinkVpcResult attachClassicLinkVpc(AttachClassicLinkVpcRequest attachClassicLinkVpcRequest) - throws AmazonClientException { - throw new UnsupportedOperationException("Not supported in mock"); - } - - @Override - public CreateVpnGatewayResult createVpnGateway(CreateVpnGatewayRequest createVpnGatewayRequest) - throws AmazonClientException { - throw new UnsupportedOperationException("Not supported in mock"); - } - - @Override - public DeleteClientVpnEndpointResult deleteClientVpnEndpoint(DeleteClientVpnEndpointRequest deleteClientVpnEndpointRequest) { - throw new UnsupportedOperationException("Not supported in mock"); - } - - @Override - public DeleteClientVpnRouteResult deleteClientVpnRoute(DeleteClientVpnRouteRequest deleteClientVpnRouteRequest) { - throw new UnsupportedOperationException("Not supported in mock"); - } - - @Override - public EnableVolumeIOResult enableVolumeIO(EnableVolumeIORequest enableVolumeIORequest) - throws AmazonClientException { - throw new UnsupportedOperationException("Not supported in mock"); - } - - @Override - public MoveAddressToVpcResult moveAddressToVpc(MoveAddressToVpcRequest moveAddressToVpcRequest) - throws AmazonClientException { - throw new UnsupportedOperationException("Not supported in mock"); - } - - @Override - public ProvisionByoipCidrResult provisionByoipCidr(ProvisionByoipCidrRequest provisionByoipCidrRequest) { - throw new UnsupportedOperationException("Not supported in mock"); - } - - @Override - public DeleteVpnGatewayResult deleteVpnGateway(DeleteVpnGatewayRequest deleteVpnGatewayRequest) - throws AmazonClientException { - throw new UnsupportedOperationException("Not supported in mock"); - } - - @Override - public DeprovisionByoipCidrResult deprovisionByoipCidr(DeprovisionByoipCidrRequest deprovisionByoipCidrRequest) { - throw new UnsupportedOperationException("Not supported in mock"); - } - - @Override - public AttachVolumeResult attachVolume(AttachVolumeRequest attachVolumeRequest) throws AmazonClientException { - throw new UnsupportedOperationException("Not supported in mock"); - } - - @Override - public DescribeVolumeStatusResult describeVolumeStatus(DescribeVolumeStatusRequest describeVolumeStatusRequest) - throws AmazonClientException { - throw new UnsupportedOperationException("Not supported in mock"); - } - - @Override - public DescribeVolumesModificationsResult describeVolumesModifications( - DescribeVolumesModificationsRequest describeVolumesModificationsRequest) - throws AmazonClientException { - throw new UnsupportedOperationException("Not supported in mock"); - } - - @Override - public DescribeImportSnapshotTasksResult describeImportSnapshotTasks( - DescribeImportSnapshotTasksRequest describeImportSnapshotTasksRequest) - throws AmazonClientException { - throw new UnsupportedOperationException("Not supported in mock"); - } - - @Override - public DescribeVpnConnectionsResult describeVpnConnections(DescribeVpnConnectionsRequest describeVpnConnectionsRequest) - throws AmazonClientException { - throw new UnsupportedOperationException("Not supported in mock"); - } - - @Override - public ResetImageAttributeResult resetImageAttribute(ResetImageAttributeRequest resetImageAttributeRequest) - throws AmazonClientException { - throw new UnsupportedOperationException("Not supported in mock"); - } - - @Override - public EnableVgwRoutePropagationResult enableVgwRoutePropagation(EnableVgwRoutePropagationRequest enableVgwRoutePropagationRequest) - throws AmazonClientException { - throw new UnsupportedOperationException("Not supported in mock"); - } - - @Override - public CreateSnapshotResult createSnapshot(CreateSnapshotRequest createSnapshotRequest) - throws AmazonClientException { - throw new UnsupportedOperationException("Not supported in mock"); - } - - @Override - public DeleteVolumeResult deleteVolume(DeleteVolumeRequest deleteVolumeRequest) throws AmazonClientException { - throw new UnsupportedOperationException("Not supported in mock"); - } - - @Override - public CreateNetworkInterfaceResult createNetworkInterface(CreateNetworkInterfaceRequest createNetworkInterfaceRequest) - throws AmazonClientException { - throw new UnsupportedOperationException("Not supported in mock"); - } - - @Override - public ModifyReservedInstancesResult modifyReservedInstances(ModifyReservedInstancesRequest modifyReservedInstancesRequest) - throws AmazonClientException { - throw new UnsupportedOperationException("Not supported in mock"); - } - - @Override - public CancelSpotFleetRequestsResult cancelSpotFleetRequests(CancelSpotFleetRequestsRequest cancelSpotFleetRequestsRequest) - throws AmazonClientException { - throw new UnsupportedOperationException("Not supported in mock"); - } - - @Override - public UnassignPrivateIpAddressesResult unassignPrivateIpAddresses(UnassignPrivateIpAddressesRequest unassignPrivateIpAddressesRequest) - throws AmazonClientException { - throw new UnsupportedOperationException("Not supported in mock"); - } - - @Override - public UnassignIpv6AddressesResult unassignIpv6Addresses(UnassignIpv6AddressesRequest unassignIpv6AddressesRequest) - throws AmazonClientException { - throw new UnsupportedOperationException("Not supported in mock"); - } - - @Override - public DescribeVpcsResult describeVpcs(DescribeVpcsRequest describeVpcsRequest) throws AmazonClientException { - throw new UnsupportedOperationException("Not supported in mock"); - } - - @Override - public CancelConversionTaskResult cancelConversionTask(CancelConversionTaskRequest cancelConversionTaskRequest) - throws AmazonClientException { - throw new UnsupportedOperationException("Not supported in mock"); - } - - @Override - public AssociateAddressResult associateAddress(AssociateAddressRequest associateAddressRequest) - throws AmazonClientException { - throw new UnsupportedOperationException("Not supported in mock"); - } - - @Override - public AssociateClientVpnTargetNetworkResult associateClientVpnTargetNetwork( - AssociateClientVpnTargetNetworkRequest associateClientVpnTargetNetworkRequest) { - throw new UnsupportedOperationException("Not supported in mock"); - } - - @Override - public AssociateIamInstanceProfileResult associateIamInstanceProfile(AssociateIamInstanceProfileRequest associateIamInstanceRequest) - throws AmazonClientException { - throw new UnsupportedOperationException("Not supported in mock"); - } - - @Override - public AssociateVpcCidrBlockResult associateVpcCidrBlock(AssociateVpcCidrBlockRequest associateVpcCidrBlockRequest) - throws AmazonClientException { - throw new UnsupportedOperationException("Not supported in mock"); - } - - @Override - public AssociateSubnetCidrBlockResult associateSubnetCidrBlock(AssociateSubnetCidrBlockRequest associateSubnetCidrBlockRequest) - throws AmazonClientException { - throw new UnsupportedOperationException("Not supported in mock"); - } - - @Override - public AssociateTransitGatewayRouteTableResult associateTransitGatewayRouteTable( - AssociateTransitGatewayRouteTableRequest associateTransitGatewayRouteTableRequest) { - throw new UnsupportedOperationException("Not supported in mock"); - } - - @Override - public DeleteCustomerGatewayResult deleteCustomerGateway(DeleteCustomerGatewayRequest deleteCustomerGatewayRequest) - throws AmazonClientException { - throw new UnsupportedOperationException("Not supported in mock"); - } - - @Override - public CreateNetworkAclEntryResult createNetworkAclEntry(CreateNetworkAclEntryRequest createNetworkAclEntryRequest) - throws AmazonClientException { - throw new UnsupportedOperationException("Not supported in mock"); - } - - @Override - public AcceptVpcPeeringConnectionResult acceptVpcPeeringConnection(AcceptVpcPeeringConnectionRequest acceptVpcPeeringConnectionRequest) - throws AmazonClientException { - throw new UnsupportedOperationException("Not supported in mock"); - } - - @Override - public DescribeExportTasksResult describeExportTasks(DescribeExportTasksRequest describeExportTasksRequest) - throws AmazonClientException { - throw new UnsupportedOperationException("Not supported in mock"); - } - - @Override - public DescribeElasticGpusResult describeElasticGpus(DescribeElasticGpusRequest describeElasticGpusRequest) - throws AmazonClientException { - throw new UnsupportedOperationException("Not supported in mock"); - } - - @Override - public DescribeFpgaImagesResult describeFpgaImages(DescribeFpgaImagesRequest describeFpgaImagesRequest) - throws AmazonClientException { - throw new UnsupportedOperationException("Not supported in mock"); - } - - @Override - public DescribeHostReservationOfferingsResult describeHostReservationOfferings( - DescribeHostReservationOfferingsRequest describeHostReservationOfferingsRequest) - throws AmazonClientException { - throw new UnsupportedOperationException("Not supported in mock"); - } - - @Override - public DescribeHostReservationsResult describeHostReservations(DescribeHostReservationsRequest describeHostReservationsRequest) - throws AmazonClientException { - throw new UnsupportedOperationException("Not supported in mock"); - } - - @Override - public DescribeIdentityIdFormatResult describeIdentityIdFormat(DescribeIdentityIdFormatRequest describeIdentityIdFormatRequest) - throws AmazonClientException { - throw new UnsupportedOperationException("Not supported in mock"); - } - - @Override - public DetachInternetGatewayResult detachInternetGateway(DetachInternetGatewayRequest detachInternetGatewayRequest) - throws AmazonClientException { - throw new UnsupportedOperationException("Not supported in mock"); - } - - @Override - public CreateVpcPeeringConnectionResult createVpcPeeringConnection(CreateVpcPeeringConnectionRequest createVpcPeeringConnectionRequest) - throws AmazonClientException { - throw new UnsupportedOperationException("Not supported in mock"); - } - - @Override - public CreateRouteTableResult createRouteTable(CreateRouteTableRequest createRouteTableRequest) - throws AmazonClientException { - throw new UnsupportedOperationException("Not supported in mock"); - } - - @Override - public CancelImportTaskResult cancelImportTask(CancelImportTaskRequest cancelImportTaskRequest) - throws AmazonClientException { - throw new UnsupportedOperationException("Not supported in mock"); - } - - @Override - public DescribeVolumesResult describeVolumes(DescribeVolumesRequest describeVolumesRequest) - throws AmazonClientException { - throw new UnsupportedOperationException("Not supported in mock"); - } - - @Override - public DescribeReservedInstancesListingsResult describeReservedInstancesListings( - DescribeReservedInstancesListingsRequest describeReservedInstancesListingsRequest) - throws AmazonClientException { - throw new UnsupportedOperationException("Not supported in mock"); - } - - @Override - public ReportInstanceStatusResult reportInstanceStatus(ReportInstanceStatusRequest reportInstanceStatusRequest) - throws AmazonClientException { - throw new UnsupportedOperationException("Not supported in mock"); - } - - @Override - public DescribeRouteTablesResult describeRouteTables(DescribeRouteTablesRequest describeRouteTablesRequest) - throws AmazonClientException { - throw new UnsupportedOperationException("Not supported in mock"); - } - - @Override - public DescribeDhcpOptionsResult describeDhcpOptions(DescribeDhcpOptionsRequest describeDhcpOptionsRequest) - throws AmazonClientException { - throw new UnsupportedOperationException("Not supported in mock"); - } - - @Override - public DescribeEgressOnlyInternetGatewaysResult describeEgressOnlyInternetGateways( - DescribeEgressOnlyInternetGatewaysRequest describeEgressOnlyInternetGatewaysRequest) - throws AmazonClientException { - throw new UnsupportedOperationException("Not supported in mock"); - } - - @Override - public MonitorInstancesResult monitorInstances(MonitorInstancesRequest monitorInstancesRequest) - throws AmazonClientException { - throw new UnsupportedOperationException("Not supported in mock"); - } - - @Override - public DescribePrefixListsResult describePrefixLists(DescribePrefixListsRequest describePrefixListsRequest) - throws AmazonClientException { - throw new UnsupportedOperationException("Not supported in mock"); - } - - @Override - public RequestSpotFleetResult requestSpotFleet(RequestSpotFleetRequest requestSpotFleetRequest) - throws AmazonClientException { - throw new UnsupportedOperationException("Not supported in mock"); - } - - @Override - public DescribeImportImageTasksResult describeImportImageTasks(DescribeImportImageTasksRequest describeImportImageTasksRequest) - throws AmazonClientException { - throw new UnsupportedOperationException("Not supported in mock"); - } - - @Override - public DescribeNetworkAclsResult describeNetworkAcls(DescribeNetworkAclsRequest describeNetworkAclsRequest) - throws AmazonClientException { - throw new UnsupportedOperationException("Not supported in mock"); - } - - @Override - public DescribeBundleTasksResult describeBundleTasks(DescribeBundleTasksRequest describeBundleTasksRequest) - throws AmazonClientException { - throw new UnsupportedOperationException("Not supported in mock"); - } - - @Override - public ImportInstanceResult importInstance(ImportInstanceRequest importInstanceRequest) - throws AmazonClientException { - throw new UnsupportedOperationException("Not supported in mock"); - } - - @Override - public DeleteVpcPeeringConnectionResult deleteVpcPeeringConnection(DeleteVpcPeeringConnectionRequest deleteVpcPeeringConnectionRequest) - throws AmazonClientException { - throw new UnsupportedOperationException("Not supported in mock"); - } - - @Override - public GetConsoleOutputResult getConsoleOutput(GetConsoleOutputRequest getConsoleOutputRequest) - throws AmazonClientException { - throw new UnsupportedOperationException("Not supported in mock"); - } - - @Override - public GetConsoleScreenshotResult getConsoleScreenshot(GetConsoleScreenshotRequest getConsoleScreenshotRequest) - throws AmazonClientException { - throw new UnsupportedOperationException("Not supported in mock"); - } - - @Override - public GetHostReservationPurchasePreviewResult getHostReservationPurchasePreview( - GetHostReservationPurchasePreviewRequest getHostReservationPurchasePreviewRequest) - throws AmazonClientException { - throw new UnsupportedOperationException("Not supported in mock"); - } - - @Override - public GetLaunchTemplateDataResult getLaunchTemplateData(GetLaunchTemplateDataRequest getLaunchTemplateDataRequest) { - throw new UnsupportedOperationException("Not supported in mock"); - } - - @Override - public CreateInternetGatewayResult createInternetGateway(CreateInternetGatewayRequest createInternetGatewayRequest) - throws AmazonClientException { - throw new UnsupportedOperationException("Not supported in mock"); - } - - @Override - public DeleteVpnConnectionRouteResult deleteVpnConnectionRoute(DeleteVpnConnectionRouteRequest deleteVpnConnectionRouteRequest) - throws AmazonClientException { - throw new UnsupportedOperationException("Not supported in mock"); - } - - @Override - public DetachNetworkInterfaceResult detachNetworkInterface(DetachNetworkInterfaceRequest detachNetworkInterfaceRequest) - throws AmazonClientException { - throw new UnsupportedOperationException("Not supported in mock"); - } - - @Override - public ModifyImageAttributeResult modifyImageAttribute(ModifyImageAttributeRequest modifyImageAttributeRequest) - throws AmazonClientException { - throw new UnsupportedOperationException("Not supported in mock"); - } - - @Override - public CreateCustomerGatewayResult createCustomerGateway(CreateCustomerGatewayRequest createCustomerGatewayRequest) - throws AmazonClientException { - throw new UnsupportedOperationException("Not supported in mock"); - } - - @Override - public CreateDefaultSubnetResult createDefaultSubnet(CreateDefaultSubnetRequest createDefaultSubnetRequest) { - throw new UnsupportedOperationException("Not supported in mock"); - } - - @Override - public CreateEgressOnlyInternetGatewayResult createEgressOnlyInternetGateway( - CreateEgressOnlyInternetGatewayRequest createEgressOnlyInternetGatewayRequest) - throws AmazonClientException { - throw new UnsupportedOperationException("Not supported in mock"); - } - - @Override - public CreateFleetResult createFleet(CreateFleetRequest createFleetRequest) { - throw new UnsupportedOperationException("Not supported in mock"); - } - - @Override - public CreateFpgaImageResult createFpgaImage(CreateFpgaImageRequest createFpgaImageRequest) - throws AmazonClientException { - throw new UnsupportedOperationException("Not supported in mock"); - } - - @Override - public CreateNetworkInterfacePermissionResult createNetworkInterfacePermission( - CreateNetworkInterfacePermissionRequest createNetworkInterfacePermissionRequest) - throws AmazonClientException { - throw new UnsupportedOperationException("Not supported in mock"); - } - - @Override - public CreateDefaultVpcResult createDefaultVpc(CreateDefaultVpcRequest createDefaultVpcRequest) - throws AmazonClientException { - throw new UnsupportedOperationException("Not supported in mock"); - } - - @Override - public CreateSpotDatafeedSubscriptionResult createSpotDatafeedSubscription( - CreateSpotDatafeedSubscriptionRequest createSpotDatafeedSubscriptionRequest) - throws AmazonClientException { - throw new UnsupportedOperationException("Not supported in mock"); - } - - @Override - public AttachInternetGatewayResult attachInternetGateway(AttachInternetGatewayRequest attachInternetGatewayRequest) - throws AmazonClientException { - throw new UnsupportedOperationException("Not supported in mock"); - } - - @Override - public DeleteVpnConnectionResult deleteVpnConnection(DeleteVpnConnectionRequest deleteVpnConnectionRequest) - throws AmazonClientException { - throw new UnsupportedOperationException("Not supported in mock"); - } - - @Override - public DescribeMovingAddressesResult describeMovingAddresses(DescribeMovingAddressesRequest describeMovingAddressesRequest) - throws AmazonClientException { - throw new UnsupportedOperationException("Not supported in mock"); - } - - @Override - public DescribeConversionTasksResult describeConversionTasks(DescribeConversionTasksRequest describeConversionTasksRequest) - throws AmazonClientException { - throw new UnsupportedOperationException("Not supported in mock"); - } - - @Override - public CreateVpnConnectionResult createVpnConnection(CreateVpnConnectionRequest createVpnConnectionRequest) - throws AmazonClientException { - throw new UnsupportedOperationException("Not supported in mock"); - } - - @Override - public ImportImageResult importImage(ImportImageRequest importImageRequest) throws AmazonClientException { - throw new UnsupportedOperationException("Not supported in mock"); - } - - @Override - public DisableVpcClassicLinkResult disableVpcClassicLink(DisableVpcClassicLinkRequest disableVpcClassicLinkRequest) - throws AmazonClientException { - throw new UnsupportedOperationException("Not supported in mock"); - } - - @Override - public DisableVpcClassicLinkDnsSupportResult disableVpcClassicLinkDnsSupport( - DisableVpcClassicLinkDnsSupportRequest disableVpcClassicLinkDnsSupportRequest) { - throw new UnsupportedOperationException("Not supported in mock"); - } - - @Override - public DescribeInstanceAttributeResult describeInstanceAttribute(DescribeInstanceAttributeRequest describeInstanceAttributeRequest) - throws AmazonClientException { - throw new UnsupportedOperationException("Not supported in mock"); - } - - @Override - public DescribeInstanceCreditSpecificationsResult describeInstanceCreditSpecifications( - DescribeInstanceCreditSpecificationsRequest describeInstanceCreditSpecificationsRequest) { - throw new UnsupportedOperationException("Not supported in mock"); - } - - @Override - public DescribeFlowLogsResult describeFlowLogs(DescribeFlowLogsRequest describeFlowLogsRequest) - throws AmazonClientException { - throw new UnsupportedOperationException("Not supported in mock"); - } - - @Override - public DescribeVpcPeeringConnectionsResult describeVpcPeeringConnections( - DescribeVpcPeeringConnectionsRequest describeVpcPeeringConnectionsRequest) - throws AmazonClientException { - throw new UnsupportedOperationException("Not supported in mock"); - } - - @Override - public DescribePlacementGroupsResult describePlacementGroups(DescribePlacementGroupsRequest describePlacementGroupsRequest) - throws AmazonClientException { - throw new UnsupportedOperationException("Not supported in mock"); - } - - @Override - public RunInstancesResult runInstances(RunInstancesRequest runInstancesRequest) throws AmazonClientException { - throw new UnsupportedOperationException("Not supported in mock"); - } - - @Override - public RunScheduledInstancesResult runScheduledInstances(RunScheduledInstancesRequest runScheduledInstancesRequest) { - throw new UnsupportedOperationException("Not supported in mock"); - } - - @Override - public SearchTransitGatewayRoutesResult searchTransitGatewayRoutes( - SearchTransitGatewayRoutesRequest searchTransitGatewayRoutesRequest) { - throw new UnsupportedOperationException("Not supported in mock"); - } - - @Override - public DescribeSubnetsResult describeSubnets(DescribeSubnetsRequest describeSubnetsRequest) - throws AmazonClientException { - throw new UnsupportedOperationException("Not supported in mock"); - } - - @Override - public AssociateRouteTableResult associateRouteTable(AssociateRouteTableRequest associateRouteTableRequest) - throws AmazonClientException { - throw new UnsupportedOperationException("Not supported in mock"); - } - - @Override - public ModifyVolumeAttributeResult modifyVolumeAttribute(ModifyVolumeAttributeRequest modifyVolumeAttributeRequest) - throws AmazonClientException { - throw new UnsupportedOperationException("Not supported in mock"); - } - - @Override - public DeleteNetworkAclResult deleteNetworkAcl(DeleteNetworkAclRequest deleteNetworkAclRequest) - throws AmazonClientException { - throw new UnsupportedOperationException("Not supported in mock"); - } - - @Override - public DescribeImagesResult describeImages(DescribeImagesRequest describeImagesRequest) - throws AmazonClientException { - throw new UnsupportedOperationException("Not supported in mock"); - } - - @Override - public StartInstancesResult startInstances(StartInstancesRequest startInstancesRequest) - throws AmazonClientException { - throw new UnsupportedOperationException("Not supported in mock"); - } - - @Override - public ModifyInstanceAttributeResult modifyInstanceAttribute(ModifyInstanceAttributeRequest modifyInstanceAttributeRequest) - throws AmazonClientException { - throw new UnsupportedOperationException("Not supported in mock"); - } - - @Override - public ModifyInstanceCapacityReservationAttributesResult modifyInstanceCapacityReservationAttributes( - ModifyInstanceCapacityReservationAttributesRequest modifyInstanceCapacityReservationAttributesRequest) { - throw new UnsupportedOperationException("Not supported in mock"); - } - - @Override - public ModifyInstanceCreditSpecificationResult modifyInstanceCreditSpecification( - ModifyInstanceCreditSpecificationRequest modifyInstanceCreditSpecificationRequest) { - throw new UnsupportedOperationException("Not supported in mock"); - } - - @Override - public ModifyInstancePlacementResult modifyInstancePlacement(ModifyInstancePlacementRequest modifyInstancePlacementRequest) { - throw new UnsupportedOperationException("Not supported in mock"); - } - - @Override - public ModifyLaunchTemplateResult modifyLaunchTemplate(ModifyLaunchTemplateRequest modifyLaunchTemplateRequest) { - throw new UnsupportedOperationException("Not supported in mock"); - } - - @Override - public ModifyIdentityIdFormatResult modifyIdentityIdFormat(ModifyIdentityIdFormatRequest modifyIdentityIdFormatRequest) { - throw new UnsupportedOperationException("Not supported in mock"); - } - - @Override - public CancelReservedInstancesListingResult cancelReservedInstancesListing( - CancelReservedInstancesListingRequest cancelReservedInstancesListingRequest) - throws AmazonClientException { - throw new UnsupportedOperationException("Not supported in mock"); - } - - @Override - public DeleteDhcpOptionsResult deleteDhcpOptions(DeleteDhcpOptionsRequest deleteDhcpOptionsRequest) - throws AmazonClientException { - throw new UnsupportedOperationException("Not supported in mock"); - } - - @Override - public DeleteEgressOnlyInternetGatewayResult deleteEgressOnlyInternetGateway( - DeleteEgressOnlyInternetGatewayRequest deleteEgressOnlyInternetGatewayRequest) - throws AmazonClientException { - throw new UnsupportedOperationException("Not supported in mock"); - } - - @Override - public DeleteFleetsResult deleteFleets(DeleteFleetsRequest deleteFleetsRequest) { - throw new UnsupportedOperationException("Not supported in mock"); - } - - @Override - public DeleteNetworkInterfacePermissionResult deleteNetworkInterfacePermission( - DeleteNetworkInterfacePermissionRequest deleteNetworkInterfacePermissionRequest) - throws AmazonClientException { - throw new UnsupportedOperationException("Not supported in mock"); - } - - @Override - public AuthorizeSecurityGroupIngressResult authorizeSecurityGroupIngress( - AuthorizeSecurityGroupIngressRequest authorizeSecurityGroupIngressRequest) - throws AmazonClientException { - throw new UnsupportedOperationException("Not supported in mock"); - } - - @Override - public DescribeSpotInstanceRequestsResult describeSpotInstanceRequests( - DescribeSpotInstanceRequestsRequest describeSpotInstanceRequestsRequest) throws AmazonClientException { - throw new UnsupportedOperationException("Not supported in mock"); - } - - @Override - public CreateVpcResult createVpc(CreateVpcRequest createVpcRequest) throws AmazonClientException { - throw new UnsupportedOperationException("Not supported in mock"); - } - - @Override - public DescribeCustomerGatewaysResult describeCustomerGateways(DescribeCustomerGatewaysRequest describeCustomerGatewaysRequest) - throws AmazonClientException { - throw new UnsupportedOperationException("Not supported in mock"); - } - - @Override - public CancelExportTaskResult cancelExportTask(CancelExportTaskRequest cancelExportTaskRequest) - throws AmazonClientException { - throw new UnsupportedOperationException("Not supported in mock"); - } - - @Override - public CreateRouteResult createRoute(CreateRouteRequest createRouteRequest) throws AmazonClientException { - throw new UnsupportedOperationException("Not supported in mock"); - } - - @Override - public CreateVpcEndpointResult createVpcEndpoint(CreateVpcEndpointRequest createVpcEndpointRequest) - throws AmazonClientException { - throw new UnsupportedOperationException("Not supported in mock"); - } - - @Override - public CreateVpcEndpointConnectionNotificationResult createVpcEndpointConnectionNotification( - CreateVpcEndpointConnectionNotificationRequest createVpcEndpointConnectionNotificationRequest) { - throw new UnsupportedOperationException("Not supported in mock"); - } - - @Override - public CreateVpcEndpointServiceConfigurationResult createVpcEndpointServiceConfiguration( - CreateVpcEndpointServiceConfigurationRequest createVpcEndpointServiceConfigurationRequest) { - throw new UnsupportedOperationException("Not supported in mock"); - } - - @Override - public CopyImageResult copyImage(CopyImageRequest copyImageRequest) throws AmazonClientException { - throw new UnsupportedOperationException("Not supported in mock"); - } - - @Override - public DescribeVpcClassicLinkResult describeVpcClassicLink(DescribeVpcClassicLinkRequest describeVpcClassicLinkRequest) - throws AmazonClientException { - throw new UnsupportedOperationException("Not supported in mock"); - } - - @Override - public ModifyNetworkInterfaceAttributeResult modifyNetworkInterfaceAttribute( - ModifyNetworkInterfaceAttributeRequest modifyNetworkInterfaceAttributeRequest) - throws AmazonClientException { - throw new UnsupportedOperationException("Not supported in mock"); - } - - @Override - public DeleteRouteTableResult deleteRouteTable(DeleteRouteTableRequest deleteRouteTableRequest) - throws AmazonClientException { - throw new UnsupportedOperationException("Not supported in mock"); - } - - @Override - public DescribeNetworkInterfaceAttributeResult describeNetworkInterfaceAttribute( - DescribeNetworkInterfaceAttributeRequest describeNetworkInterfaceAttributeRequest) - throws AmazonClientException { - throw new UnsupportedOperationException("Not supported in mock"); - } - - @Override - public DescribeClassicLinkInstancesResult describeClassicLinkInstances( - DescribeClassicLinkInstancesRequest describeClassicLinkInstancesRequest) throws AmazonClientException { - throw new UnsupportedOperationException("Not supported in mock"); - } - - @Override - public RequestSpotInstancesResult requestSpotInstances(RequestSpotInstancesRequest requestSpotInstancesRequest) - throws AmazonClientException { - throw new UnsupportedOperationException("Not supported in mock"); - } - - @Override - public ResetFpgaImageAttributeResult resetFpgaImageAttribute(ResetFpgaImageAttributeRequest resetFpgaImageAttributeRequest) { - throw new UnsupportedOperationException("Not supported in mock"); - } - - @Override - public CreateTagsResult createTags(CreateTagsRequest createTagsRequest) throws AmazonClientException { - throw new UnsupportedOperationException("Not supported in mock"); - } - - @Override - public CreateTransitGatewayResult createTransitGateway(CreateTransitGatewayRequest createTransitGatewayRequest) { - throw new UnsupportedOperationException("Not supported in mock"); - } - - @Override - public CreateTransitGatewayRouteResult createTransitGatewayRoute(CreateTransitGatewayRouteRequest createTransitGatewayRouteRequest) { - throw new UnsupportedOperationException("Not supported in mock"); - } - - @Override - public CreateTransitGatewayRouteTableResult createTransitGatewayRouteTable( - CreateTransitGatewayRouteTableRequest createTransitGatewayRouteTableRequest) { - throw new UnsupportedOperationException("Not supported in mock"); - } - - @Override - public CreateTransitGatewayVpcAttachmentResult createTransitGatewayVpcAttachment( - CreateTransitGatewayVpcAttachmentRequest createTransitGatewayVpcAttachmentRequest) { - throw new UnsupportedOperationException("Not supported in mock"); - } - - @Override - public DescribeVolumeAttributeResult describeVolumeAttribute(DescribeVolumeAttributeRequest describeVolumeAttributeRequest) - throws AmazonClientException { - throw new UnsupportedOperationException("Not supported in mock"); - } - - @Override - public AttachNetworkInterfaceResult attachNetworkInterface(AttachNetworkInterfaceRequest attachNetworkInterfaceRequest) - throws AmazonClientException { - throw new UnsupportedOperationException("Not supported in mock"); - } - - @Override - public ReplaceRouteResult replaceRoute(ReplaceRouteRequest replaceRouteRequest) throws AmazonClientException { - throw new UnsupportedOperationException("Not supported in mock"); - } - - @Override - public DescribeTagsResult describeTags(DescribeTagsRequest describeTagsRequest) throws AmazonClientException { - throw new UnsupportedOperationException("Not supported in mock"); - } - - @Override - public CancelBundleTaskResult cancelBundleTask(CancelBundleTaskRequest cancelBundleTaskRequest) - throws AmazonClientException { - throw new UnsupportedOperationException("Not supported in mock"); - } - - @Override - public CancelCapacityReservationResult cancelCapacityReservation(CancelCapacityReservationRequest cancelCapacityReservationRequest) { - throw new UnsupportedOperationException("Not supported in mock"); - } - - @Override - public DisableVgwRoutePropagationResult disableVgwRoutePropagation(DisableVgwRoutePropagationRequest disableVgwRoutePropagationRequest) - throws AmazonClientException { - throw new UnsupportedOperationException("Not supported in mock"); - } - - @Override - public ImportSnapshotResult importSnapshot(ImportSnapshotRequest importSnapshotRequest) - throws AmazonClientException { - throw new UnsupportedOperationException("Not supported in mock"); - } - - @Override - public CancelSpotInstanceRequestsResult cancelSpotInstanceRequests(CancelSpotInstanceRequestsRequest cancelSpotInstanceRequestsRequest) - throws AmazonClientException { - throw new UnsupportedOperationException("Not supported in mock"); - } - - @Override - public DescribeSpotFleetRequestsResult describeSpotFleetRequests(DescribeSpotFleetRequestsRequest describeSpotFleetRequestsRequest) - throws AmazonClientException { - throw new UnsupportedOperationException("Not supported in mock"); - } - - @Override - public PurchaseReservedInstancesOfferingResult purchaseReservedInstancesOffering( - PurchaseReservedInstancesOfferingRequest purchaseReservedInstancesOfferingRequest) - throws AmazonClientException { - throw new UnsupportedOperationException("Not supported in mock"); - } - - @Override - public PurchaseScheduledInstancesResult purchaseScheduledInstances( - PurchaseScheduledInstancesRequest purchaseScheduledInstancesRequest) { - throw new UnsupportedOperationException("Not supported in mock"); - } - - @Override - public PurchaseHostReservationResult purchaseHostReservation(PurchaseHostReservationRequest purchaseHostReservationRequest) { - throw new UnsupportedOperationException("Not supported in mock"); - } - - @Override - public ModifySnapshotAttributeResult modifySnapshotAttribute(ModifySnapshotAttributeRequest modifySnapshotAttributeRequest) - throws AmazonClientException { - throw new UnsupportedOperationException("Not supported in mock"); - } - - @Override - public DescribeReservedInstancesModificationsResult describeReservedInstancesModifications( - DescribeReservedInstancesModificationsRequest describeReservedInstancesModificationsRequest) - throws AmazonClientException { - throw new UnsupportedOperationException("Not supported in mock"); - } - - @Override - public TerminateInstancesResult terminateInstances(TerminateInstancesRequest terminateInstancesRequest) - throws AmazonClientException { - throw new UnsupportedOperationException("Not supported in mock"); - } - - @Override - public ModifyVpcEndpointResult modifyVpcEndpoint(ModifyVpcEndpointRequest modifyVpcEndpointRequest) - throws AmazonClientException { - throw new UnsupportedOperationException("Not supported in mock"); - } - - @Override - public ModifyVpcEndpointConnectionNotificationResult modifyVpcEndpointConnectionNotification( - ModifyVpcEndpointConnectionNotificationRequest modifyVpcEndpointConnectionNotificationRequest) { - throw new UnsupportedOperationException("Not supported in mock"); - } - - @Override - public ModifyVpcEndpointServiceConfigurationResult modifyVpcEndpointServiceConfiguration( - ModifyVpcEndpointServiceConfigurationRequest modifyVpcEndpointServiceConfigurationRequest) { - throw new UnsupportedOperationException("Not supported in mock"); - } - - @Override - public ModifyVpcEndpointServicePermissionsResult modifyVpcEndpointServicePermissions( - ModifyVpcEndpointServicePermissionsRequest modifyVpcEndpointServicePermissionsRequest) { - throw new UnsupportedOperationException("Not supported in mock"); - } - - @Override - public DeleteSpotDatafeedSubscriptionResult deleteSpotDatafeedSubscription( - DeleteSpotDatafeedSubscriptionRequest deleteSpotDatafeedSubscriptionRequest) - throws AmazonClientException { - throw new UnsupportedOperationException("Not supported in mock"); - } - - @Override - public DeleteInternetGatewayResult deleteInternetGateway(DeleteInternetGatewayRequest deleteInternetGatewayRequest) - throws AmazonClientException { - throw new UnsupportedOperationException("Not supported in mock"); - } - - @Override - public DescribeSnapshotAttributeResult describeSnapshotAttribute(DescribeSnapshotAttributeRequest describeSnapshotAttributeRequest) - throws AmazonClientException { - throw new UnsupportedOperationException("Not supported in mock"); - } - - @Override - public ReplaceRouteTableAssociationResult replaceRouteTableAssociation( - ReplaceRouteTableAssociationRequest replaceRouteTableAssociationRequest) throws AmazonClientException { - throw new UnsupportedOperationException("Not supported in mock"); - } - - @Override - public ReplaceTransitGatewayRouteResult replaceTransitGatewayRoute( - ReplaceTransitGatewayRouteRequest replaceTransitGatewayRouteRequest) { - throw new UnsupportedOperationException("Not supported in mock"); - } - - @Override - public DescribeAddressesResult describeAddresses(DescribeAddressesRequest describeAddressesRequest) - throws AmazonClientException { - throw new UnsupportedOperationException("Not supported in mock"); - } - - @Override - public DescribeImageAttributeResult describeImageAttribute(DescribeImageAttributeRequest describeImageAttributeRequest) - throws AmazonClientException { - throw new UnsupportedOperationException("Not supported in mock"); - } - - @Override - public DescribeKeyPairsResult describeKeyPairs(DescribeKeyPairsRequest describeKeyPairsRequest) - throws AmazonClientException { - throw new UnsupportedOperationException("Not supported in mock"); - } - - @Override - public ConfirmProductInstanceResult confirmProductInstance(ConfirmProductInstanceRequest confirmProductInstanceRequest) - throws AmazonClientException { - throw new UnsupportedOperationException("Not supported in mock"); - } - - @Override - public CopyFpgaImageResult copyFpgaImage(CopyFpgaImageRequest copyFpgaImageRequest) { - throw new UnsupportedOperationException("Not supported in mock"); - } - - @Override - public DisassociateRouteTableResult disassociateRouteTable(DisassociateRouteTableRequest disassociateRouteTableRequest) - throws AmazonClientException { - throw new UnsupportedOperationException("Not supported in mock"); - } - - @Override - public DisassociateIamInstanceProfileResult disassociateIamInstanceProfile( - DisassociateIamInstanceProfileRequest disassociateIamInstanceProfileRequest) - throws AmazonClientException { - throw new UnsupportedOperationException("Not supported in mock"); - } - - @Override - public DisassociateVpcCidrBlockResult disassociateVpcCidrBlock(DisassociateVpcCidrBlockRequest disassociateVpcCidrBlockRequest) - throws AmazonClientException { - throw new UnsupportedOperationException("Not supported in mock"); - } - - @Override - public EnableTransitGatewayRouteTablePropagationResult enableTransitGatewayRouteTablePropagation( - EnableTransitGatewayRouteTablePropagationRequest enableTransitGatewayRouteTablePropagationRequest) { - throw new UnsupportedOperationException("Not supported in mock"); - } - - @Override - public DisassociateSubnetCidrBlockResult disassociateSubnetCidrBlock( - DisassociateSubnetCidrBlockRequest disassociateSubnetCidrBlockRequest) throws AmazonClientException { - throw new UnsupportedOperationException("Not supported in mock"); - } - - @Override - public DisassociateTransitGatewayRouteTableResult disassociateTransitGatewayRouteTable( - DisassociateTransitGatewayRouteTableRequest disassociateTransitGatewayRouteTableRequest) { - throw new UnsupportedOperationException("Not supported in mock"); - } - - @Override - public DescribeVpcAttributeResult describeVpcAttribute(DescribeVpcAttributeRequest describeVpcAttributeRequest) - throws AmazonClientException { - throw new UnsupportedOperationException("Not supported in mock"); - } - - @Override - public RevokeSecurityGroupEgressResult revokeSecurityGroupEgress(RevokeSecurityGroupEgressRequest revokeSecurityGroupEgressRequest) - throws AmazonClientException { - throw new UnsupportedOperationException("Not supported in mock"); - } - - @Override - public DeleteNetworkAclEntryResult deleteNetworkAclEntry(DeleteNetworkAclEntryRequest deleteNetworkAclEntryRequest) - throws AmazonClientException { - throw new UnsupportedOperationException("Not supported in mock"); - } - - @Override - public CreateVolumeResult createVolume(CreateVolumeRequest createVolumeRequest) throws AmazonClientException { - throw new UnsupportedOperationException("Not supported in mock"); - } - - @Override - public ModifyVolumeResult modifyVolume(ModifyVolumeRequest modifyVolumeRequest) throws AmazonClientException { - throw new UnsupportedOperationException("Not supported in mock"); - } - - @Override - public DescribeInstanceStatusResult describeInstanceStatus(DescribeInstanceStatusRequest describeInstanceStatusRequest) - throws AmazonClientException { - throw new UnsupportedOperationException("Not supported in mock"); - } - - @Override - public DescribeVpnGatewaysResult describeVpnGateways(DescribeVpnGatewaysRequest describeVpnGatewaysRequest) - throws AmazonClientException { - throw new UnsupportedOperationException("Not supported in mock"); - } - - @Override - public CreateSubnetResult createSubnet(CreateSubnetRequest createSubnetRequest) throws AmazonClientException { - throw new UnsupportedOperationException("Not supported in mock"); - } - - @Override - public DescribeReservedInstancesOfferingsResult describeReservedInstancesOfferings( - DescribeReservedInstancesOfferingsRequest describeReservedInstancesOfferingsRequest) - throws AmazonClientException { - throw new UnsupportedOperationException("Not supported in mock"); - } - - @Override - public AssignPrivateIpAddressesResult assignPrivateIpAddresses(AssignPrivateIpAddressesRequest assignPrivateIpAddressesRequest) - throws AmazonClientException { - throw new UnsupportedOperationException("Not supported in mock"); - } - - @Override - public AssignIpv6AddressesResult assignIpv6Addresses(AssignIpv6AddressesRequest assignIpv6AddressesRequest) - throws AmazonClientException { - throw new UnsupportedOperationException("Not supported in mock"); - } - - @Override - public DescribeSpotFleetRequestHistoryResult describeSpotFleetRequestHistory( - DescribeSpotFleetRequestHistoryRequest describeSpotFleetRequestHistoryRequest) - throws AmazonClientException { - throw new UnsupportedOperationException("Not supported in mock"); - } - - @Override - public DeleteSnapshotResult deleteSnapshot(DeleteSnapshotRequest deleteSnapshotRequest) - throws AmazonClientException { - throw new UnsupportedOperationException("Not supported in mock"); - } - - @Override - public ReplaceNetworkAclAssociationResult replaceNetworkAclAssociation( - ReplaceNetworkAclAssociationRequest replaceNetworkAclAssociationRequest) - throws AmazonClientException { - throw new UnsupportedOperationException("Not supported in mock"); - } - - @Override - public DisassociateAddressResult disassociateAddress(DisassociateAddressRequest disassociateAddressRequest) - throws AmazonClientException { - throw new UnsupportedOperationException("Not supported in mock"); - } - - @Override - public DisassociateClientVpnTargetNetworkResult disassociateClientVpnTargetNetwork( - DisassociateClientVpnTargetNetworkRequest disassociateClientVpnTargetNetworkRequest) { - throw new UnsupportedOperationException("Not supported in mock"); - } - - @Override - public CreatePlacementGroupResult createPlacementGroup(CreatePlacementGroupRequest createPlacementGroupRequest) - throws AmazonClientException { - throw new UnsupportedOperationException("Not supported in mock"); - } - - @Override - public BundleInstanceResult bundleInstance(BundleInstanceRequest bundleInstanceRequest) - throws AmazonClientException { - throw new UnsupportedOperationException("Not supported in mock"); - } - - @Override - public DeletePlacementGroupResult deletePlacementGroup(DeletePlacementGroupRequest deletePlacementGroupRequest) - throws AmazonClientException { - throw new UnsupportedOperationException("Not supported in mock"); - } - - @Override - public ModifySubnetAttributeResult modifySubnetAttribute(ModifySubnetAttributeRequest modifySubnetAttributeRequest) - throws AmazonClientException { - throw new UnsupportedOperationException("Not supported in mock"); - } - - @Override - public ModifyTransitGatewayVpcAttachmentResult modifyTransitGatewayVpcAttachment( - ModifyTransitGatewayVpcAttachmentRequest modifyTransitGatewayVpcAttachmentRequest) { - throw new UnsupportedOperationException("Not supported in mock"); - } - - @Override - public DeleteVpcResult deleteVpc(DeleteVpcRequest deleteVpcRequest) throws AmazonClientException { - throw new UnsupportedOperationException("Not supported in mock"); - } - - @Override - public DeleteVpcEndpointConnectionNotificationsResult deleteVpcEndpointConnectionNotifications( - DeleteVpcEndpointConnectionNotificationsRequest deleteVpcEndpointConnectionNotificationsRequest) { - throw new UnsupportedOperationException("Not supported in mock"); - } - - @Override - public DeleteVpcEndpointServiceConfigurationsResult deleteVpcEndpointServiceConfigurations( - DeleteVpcEndpointServiceConfigurationsRequest deleteVpcEndpointServiceConfigurationsRequest) { - throw new UnsupportedOperationException("Not supported in mock"); - } - - @Override - public CopySnapshotResult copySnapshot(CopySnapshotRequest copySnapshotRequest) throws AmazonClientException { - throw new UnsupportedOperationException("Not supported in mock"); - } - - @Override - public CreateCapacityReservationResult createCapacityReservation(CreateCapacityReservationRequest createCapacityReservationRequest) { - throw new UnsupportedOperationException("Not supported in mock"); - } - - @Override - public CreateClientVpnEndpointResult createClientVpnEndpoint(CreateClientVpnEndpointRequest createClientVpnEndpointRequest) { - throw new UnsupportedOperationException("Not supported in mock"); - } - - @Override - public CreateClientVpnRouteResult createClientVpnRoute(CreateClientVpnRouteRequest createClientVpnRouteRequest) { - throw new UnsupportedOperationException("Not supported in mock"); - } - - @Override - public DescribeVpcEndpointServicesResult describeVpcEndpointServices( - DescribeVpcEndpointServicesRequest describeVpcEndpointServicesRequest) - throws AmazonClientException { - throw new UnsupportedOperationException("Not supported in mock"); - } - - @Override - public AllocateAddressResult allocateAddress(AllocateAddressRequest allocateAddressRequest) - throws AmazonClientException { - throw new UnsupportedOperationException("Not supported in mock"); - } - - @Override - public ReleaseAddressResult releaseAddress(ReleaseAddressRequest releaseAddressRequest) - throws AmazonClientException { - throw new UnsupportedOperationException("Not supported in mock"); - } - - @Override - public ReleaseHostsResult releaseHosts(ReleaseHostsRequest releaseHostsRequest) { - throw new UnsupportedOperationException("Not supported in mock"); - } - - @Override - public ReplaceIamInstanceProfileAssociationResult replaceIamInstanceProfileAssociation( - ReplaceIamInstanceProfileAssociationRequest replaceIamInstanceProfileAssociationRequest) { - throw new UnsupportedOperationException("Not supported in mock"); - } - - @Override - public ResetInstanceAttributeResult resetInstanceAttribute(ResetInstanceAttributeRequest resetInstanceAttributeRequest) - throws AmazonClientException { - throw new UnsupportedOperationException("Not supported in mock"); - } - - @Override - public CreateKeyPairResult createKeyPair(CreateKeyPairRequest createKeyPairRequest) - throws AmazonClientException { - throw new UnsupportedOperationException("Not supported in mock"); - } - - @Override - public CreateLaunchTemplateResult createLaunchTemplate(CreateLaunchTemplateRequest createLaunchTemplateRequest) { - throw new UnsupportedOperationException("Not supported in mock"); - } - - @Override - public CreateLaunchTemplateVersionResult createLaunchTemplateVersion( - CreateLaunchTemplateVersionRequest createLaunchTemplateVersionRequest) { - throw new UnsupportedOperationException("Not supported in mock"); - } - - @Override - public CreateNatGatewayResult createNatGateway(CreateNatGatewayRequest createNatGatewayRequest) { - throw new UnsupportedOperationException("Not supported in mock"); - } - - @Override - public ReplaceNetworkAclEntryResult replaceNetworkAclEntry(ReplaceNetworkAclEntryRequest replaceNetworkAclEntryRequest) - throws AmazonClientException { - throw new UnsupportedOperationException("Not supported in mock"); - } - - @Override - public DescribeSnapshotsResult describeSnapshots(DescribeSnapshotsRequest describeSnapshotsRequest) - throws AmazonClientException { - throw new UnsupportedOperationException("Not supported in mock"); - } - - @Override - public CreateNetworkAclResult createNetworkAcl(CreateNetworkAclRequest createNetworkAclRequest) - throws AmazonClientException { - throw new UnsupportedOperationException("Not supported in mock"); - } - - @Override - public RegisterImageResult registerImage(RegisterImageRequest registerImageRequest) - throws AmazonClientException { - throw new UnsupportedOperationException("Not supported in mock"); - } - - @Override - public RejectTransitGatewayVpcAttachmentResult rejectTransitGatewayVpcAttachment( - RejectTransitGatewayVpcAttachmentRequest rejectTransitGatewayVpcAttachmentRequest) { - throw new UnsupportedOperationException("Not supported in mock"); - } - - @Override - public RejectVpcEndpointConnectionsResult rejectVpcEndpointConnections( - RejectVpcEndpointConnectionsRequest rejectVpcEndpointConnectionsRequest) { - throw new UnsupportedOperationException("Not supported in mock"); - } - - @Override - public ResetNetworkInterfaceAttributeResult resetNetworkInterfaceAttribute( - ResetNetworkInterfaceAttributeRequest resetNetworkInterfaceAttributeRequest) - throws AmazonClientException { - throw new UnsupportedOperationException("Not supported in mock"); - } - - @Override - public EnableVpcClassicLinkResult enableVpcClassicLink(EnableVpcClassicLinkRequest enableVpcClassicLinkRequest) - throws AmazonClientException { - throw new UnsupportedOperationException("Not supported in mock"); - } - - @Override - public EnableVpcClassicLinkDnsSupportResult enableVpcClassicLinkDnsSupport( - EnableVpcClassicLinkDnsSupportRequest enableVpcClassicLinkDnsSupportRequest) { - throw new UnsupportedOperationException("Not supported in mock"); - } - - @Override - public ExportClientVpnClientCertificateRevocationListResult exportClientVpnClientCertificateRevocationList( - ExportClientVpnClientCertificateRevocationListRequest exportClientVpnClientCertificateRevocationListRequest) { - throw new UnsupportedOperationException("Not supported in mock"); - } - - @Override - public ExportClientVpnClientConfigurationResult exportClientVpnClientConfiguration( - ExportClientVpnClientConfigurationRequest exportClientVpnClientConfigurationRequest) { - throw new UnsupportedOperationException("Not supported in mock"); - } - - @Override - public ExportTransitGatewayRoutesResult exportTransitGatewayRoutes( - ExportTransitGatewayRoutesRequest exportTransitGatewayRoutesRequest) { - throw new UnsupportedOperationException("Not supported in mock"); - } - - @Override - public CreateVpnConnectionRouteResult createVpnConnectionRoute(CreateVpnConnectionRouteRequest createVpnConnectionRouteRequest) - throws AmazonClientException { - throw new UnsupportedOperationException("Not supported in mock"); - } - - @Override - public DescribeVpcEndpointsResult describeVpcEndpoints(DescribeVpcEndpointsRequest describeVpcEndpointsRequest) - throws AmazonClientException { - throw new UnsupportedOperationException("Not supported in mock"); - } - - @Override - public DetachClassicLinkVpcResult detachClassicLinkVpc(DetachClassicLinkVpcRequest detachClassicLinkVpcRequest) - throws AmazonClientException { - throw new UnsupportedOperationException("Not supported in mock"); - } - - @Override - public DescribeReservedInstancesResult describeReservedInstances() throws AmazonClientException { - throw new UnsupportedOperationException("Not supported in mock"); - } - - @Override - public DescribeAvailabilityZonesResult describeAvailabilityZones() throws AmazonClientException { - throw new UnsupportedOperationException("Not supported in mock"); - } - - @Override - public DescribeSpotPriceHistoryResult describeSpotPriceHistory() throws AmazonClientException { - throw new UnsupportedOperationException("Not supported in mock"); - } - - @Override - public DescribeNetworkInterfacesResult describeNetworkInterfaces() throws AmazonClientException { - throw new UnsupportedOperationException("Not supported in mock"); - } - - @Override - public DescribeRegionsResult describeRegions() throws AmazonClientException { - throw new UnsupportedOperationException("Not supported in mock"); - } - - @Override - public DescribeInternetGatewaysResult describeInternetGateways() throws AmazonClientException { - throw new UnsupportedOperationException("Not supported in mock"); - } - - @Override - public DescribeSecurityGroupsResult describeSecurityGroups() throws AmazonClientException { - throw new UnsupportedOperationException("Not supported in mock"); - } - - @Override - public DescribeSpotDatafeedSubscriptionResult describeSpotDatafeedSubscription() throws AmazonClientException { - throw new UnsupportedOperationException("Not supported in mock"); - } - - @Override - public DescribeAccountAttributesResult describeAccountAttributes() throws AmazonClientException { - throw new UnsupportedOperationException("Not supported in mock"); - } - - @Override - public DescribeVolumeStatusResult describeVolumeStatus() throws AmazonClientException { - throw new UnsupportedOperationException("Not supported in mock"); - } - - @Override - public DescribeImportSnapshotTasksResult describeImportSnapshotTasks() throws AmazonClientException { - throw new UnsupportedOperationException("Not supported in mock"); - } - - @Override - public DescribeVpnConnectionsResult describeVpnConnections() throws AmazonClientException { - throw new UnsupportedOperationException("Not supported in mock"); - } - - @Override - public DescribeVpcsResult describeVpcs() throws AmazonClientException { - throw new UnsupportedOperationException("Not supported in mock"); - } - - @Override - public AcceptVpcPeeringConnectionResult acceptVpcPeeringConnection() throws AmazonClientException { - throw new UnsupportedOperationException("Not supported in mock"); - } - - @Override - public AdvertiseByoipCidrResult advertiseByoipCidr(AdvertiseByoipCidrRequest advertiseByoipCidrRequest) { - throw new UnsupportedOperationException("Not supported in mock"); - } - - @Override - public DescribeExportTasksResult describeExportTasks() throws AmazonClientException { - throw new UnsupportedOperationException("Not supported in mock"); - } - - @Override - public DescribeFleetHistoryResult describeFleetHistory(DescribeFleetHistoryRequest describeFleetHistoryRequest) { - throw new UnsupportedOperationException("Not supported in mock"); - } - - @Override - public DescribeFleetInstancesResult describeFleetInstances(DescribeFleetInstancesRequest describeFleetInstancesRequest) { - throw new UnsupportedOperationException("Not supported in mock"); - } - - @Override - public DescribeFleetsResult describeFleets(DescribeFleetsRequest describeFleetsRequest) { - throw new UnsupportedOperationException("Not supported in mock"); - } - - @Override - public CreateVpcPeeringConnectionResult createVpcPeeringConnection() throws AmazonClientException { - throw new UnsupportedOperationException("Not supported in mock"); - } - - @Override - public CancelImportTaskResult cancelImportTask() throws AmazonClientException { - throw new UnsupportedOperationException("Not supported in mock"); - } - - @Override - public DescribeVolumesResult describeVolumes() throws AmazonClientException { - throw new UnsupportedOperationException("Not supported in mock"); - } - - @Override - public DescribeReservedInstancesListingsResult describeReservedInstancesListings() - throws AmazonClientException { - throw new UnsupportedOperationException("Not supported in mock"); - } - - @Override - public DescribeRouteTablesResult describeRouteTables() throws AmazonClientException { - throw new UnsupportedOperationException("Not supported in mock"); - } - - @Override - public DescribeScheduledInstanceAvailabilityResult describeScheduledInstanceAvailability( - DescribeScheduledInstanceAvailabilityRequest describeScheduledInstanceAvailabilityRequest) { - throw new UnsupportedOperationException("Not supported in mock"); - } - - @Override - public DescribeScheduledInstancesResult describeScheduledInstances( - DescribeScheduledInstancesRequest describeScheduledInstancesRequest) { - throw new UnsupportedOperationException("Not supported in mock"); - } - - @Override - public DescribeDhcpOptionsResult describeDhcpOptions() throws AmazonClientException { - throw new UnsupportedOperationException("Not supported in mock"); - } - - @Override - public DescribePrefixListsResult describePrefixLists() throws AmazonClientException { - throw new UnsupportedOperationException("Not supported in mock"); - } - - @Override - public DescribePrincipalIdFormatResult describePrincipalIdFormat(DescribePrincipalIdFormatRequest describePrincipalIdFormatRequest) { - throw new UnsupportedOperationException("Not supported in mock"); - } - - @Override - public DescribePublicIpv4PoolsResult describePublicIpv4Pools(DescribePublicIpv4PoolsRequest describePublicIpv4PoolsRequest) { - throw new UnsupportedOperationException("Not supported in mock"); - } - - @Override - public DescribeImportImageTasksResult describeImportImageTasks() throws AmazonClientException { - throw new UnsupportedOperationException("Not supported in mock"); - } - - @Override - public DescribeNetworkAclsResult describeNetworkAcls() throws AmazonClientException { - throw new UnsupportedOperationException("Not supported in mock"); - } - - @Override - public DescribeBundleTasksResult describeBundleTasks() throws AmazonClientException { - throw new UnsupportedOperationException("Not supported in mock"); - } - - @Override - public DescribeByoipCidrsResult describeByoipCidrs(DescribeByoipCidrsRequest describeByoipCidrsRequest) { - throw new UnsupportedOperationException("Not supported in mock"); - } - - @Override - public DescribeCapacityReservationsResult describeCapacityReservations( - DescribeCapacityReservationsRequest describeCapacityReservationsRequest) { - throw new UnsupportedOperationException("Not supported in mock"); - } - - @Override - public RevokeSecurityGroupIngressResult revokeSecurityGroupIngress(RevokeSecurityGroupIngressRequest revokeSecurityGroupIngressRequest) - throws AmazonClientException { - throw new UnsupportedOperationException("Not supported in mock"); - } - - @Override - public RevokeSecurityGroupIngressResult revokeSecurityGroupIngress() throws AmazonClientException { - throw new UnsupportedOperationException("Not supported in mock"); - } - - @Override - public CreateInternetGatewayResult createInternetGateway() throws AmazonClientException { - throw new UnsupportedOperationException("Not supported in mock"); - } - - @Override - public DescribeMovingAddressesResult describeMovingAddresses() throws AmazonClientException { - throw new UnsupportedOperationException("Not supported in mock"); - } - - @Override - public DescribeNatGatewaysResult describeNatGateways(DescribeNatGatewaysRequest describeNatGatewaysRequest) { - throw new UnsupportedOperationException("Not supported in mock"); - } - - @Override - public DescribeConversionTasksResult describeConversionTasks() throws AmazonClientException { - throw new UnsupportedOperationException("Not supported in mock"); - } - - @Override - public ImportImageResult importImage() throws AmazonClientException { - throw new UnsupportedOperationException("Not supported in mock"); - } - - @Override - public DescribeFlowLogsResult describeFlowLogs() throws AmazonClientException { - throw new UnsupportedOperationException("Not supported in mock"); - } - - @Override - public DescribeFpgaImageAttributeResult describeFpgaImageAttribute( - DescribeFpgaImageAttributeRequest describeFpgaImageAttributeRequest) { - throw new UnsupportedOperationException("Not supported in mock"); - } - - @Override - public DescribeHostsResult describeHosts(DescribeHostsRequest describeHostsRequest) { - throw new UnsupportedOperationException("Not supported in mock"); - } - - @Override - public DescribeHostsResult describeHosts() { - throw new UnsupportedOperationException("Not supported in mock"); - } - - @Override - public DescribeIamInstanceProfileAssociationsResult describeIamInstanceProfileAssociations( - DescribeIamInstanceProfileAssociationsRequest describeIamInstanceProfileAssociationsRequest) - throws AmazonClientException { - throw new UnsupportedOperationException("Not supported in mock"); - } - - @Override - public DescribeIdFormatResult describeIdFormat(DescribeIdFormatRequest describeIdFormatRequest) { - throw new UnsupportedOperationException("Not supported in mock"); - } - - @Override - public DescribeIdFormatResult describeIdFormat() { - throw new UnsupportedOperationException("Not supported in mock"); - } - - @Override - public DescribeVpcPeeringConnectionsResult describeVpcPeeringConnections() throws AmazonClientException { - throw new UnsupportedOperationException("Not supported in mock"); - } - - @Override - public DescribePlacementGroupsResult describePlacementGroups() throws AmazonClientException { - throw new UnsupportedOperationException("Not supported in mock"); - } - - @Override - public DescribeSubnetsResult describeSubnets() throws AmazonClientException { - throw new UnsupportedOperationException("Not supported in mock"); - } - - @Override - public DescribeInstancesResult describeInstances() throws AmazonClientException { - throw new UnsupportedOperationException("Not supported in mock"); - } - - @Override - public DescribeImagesResult describeImages() throws AmazonClientException { - throw new UnsupportedOperationException("Not supported in mock"); - } - - @Override - public DescribeSpotInstanceRequestsResult describeSpotInstanceRequests() throws AmazonClientException { - throw new UnsupportedOperationException("Not supported in mock"); - } - - @Override - public DescribeCustomerGatewaysResult describeCustomerGateways() throws AmazonClientException { - throw new UnsupportedOperationException("Not supported in mock"); - } - - @Override - public DescribeVpcClassicLinkResult describeVpcClassicLink() throws AmazonClientException { - throw new UnsupportedOperationException("Not supported in mock"); - } - - @Override - public DescribeVpcClassicLinkDnsSupportResult describeVpcClassicLinkDnsSupport( - DescribeVpcClassicLinkDnsSupportRequest describeVpcClassicLinkDnsSupportRequest) { - throw new UnsupportedOperationException("Not supported in mock"); - } - - @Override - public DescribeVpcEndpointConnectionNotificationsResult describeVpcEndpointConnectionNotifications( - DescribeVpcEndpointConnectionNotificationsRequest describeVpcEndpointConnectionNotificationsRequest) { - throw new UnsupportedOperationException("Not supported in mock"); - } - - @Override - public DescribeVpcEndpointConnectionsResult describeVpcEndpointConnections( - DescribeVpcEndpointConnectionsRequest describeVpcEndpointConnectionsRequest) { - throw new UnsupportedOperationException("Not supported in mock"); - } - - @Override - public DescribeVpcEndpointServiceConfigurationsResult describeVpcEndpointServiceConfigurations( - DescribeVpcEndpointServiceConfigurationsRequest describeVpcEndpointServiceConfigurationsRequest) { - throw new UnsupportedOperationException("Not supported in mock"); - } - - @Override - public DescribeVpcEndpointServicePermissionsResult describeVpcEndpointServicePermissions( - DescribeVpcEndpointServicePermissionsRequest describeVpcEndpointServicePermissionsRequest) { - throw new UnsupportedOperationException("Not supported in mock"); - } - - @Override - public DescribeClassicLinkInstancesResult describeClassicLinkInstances() throws AmazonClientException { - throw new UnsupportedOperationException("Not supported in mock"); - } - - @Override - public DescribeClientVpnAuthorizationRulesResult describeClientVpnAuthorizationRules( - DescribeClientVpnAuthorizationRulesRequest describeClientVpnAuthorizationRulesRequest) { - throw new UnsupportedOperationException("Not supported in mock"); - } - - @Override - public DescribeClientVpnConnectionsResult describeClientVpnConnections( - DescribeClientVpnConnectionsRequest describeClientVpnConnectionsRequest) { - throw new UnsupportedOperationException("Not supported in mock"); - } - - @Override - public DescribeClientVpnEndpointsResult describeClientVpnEndpoints( - DescribeClientVpnEndpointsRequest describeClientVpnEndpointsRequest) { - throw new UnsupportedOperationException("Not supported in mock"); - } - - @Override - public DescribeClientVpnRoutesResult describeClientVpnRoutes( - DescribeClientVpnRoutesRequest describeClientVpnRoutesRequest) { - throw new UnsupportedOperationException("Not supported in mock"); - } - - @Override - public DescribeClientVpnTargetNetworksResult describeClientVpnTargetNetworks( - DescribeClientVpnTargetNetworksRequest describeClientVpnTargetNetworksRequest) { - throw new UnsupportedOperationException("Not supported in mock"); - } - - @Override - public DescribeTagsResult describeTags() throws AmazonClientException { - throw new UnsupportedOperationException("Not supported in mock"); - } - - @Override - public DescribeTransitGatewayAttachmentsResult describeTransitGatewayAttachments( - DescribeTransitGatewayAttachmentsRequest describeTransitGatewayAttachmentsRequest) { - throw new UnsupportedOperationException("Not supported in mock"); - } - - @Override - public DescribeTransitGatewayRouteTablesResult describeTransitGatewayRouteTables( - DescribeTransitGatewayRouteTablesRequest describeTransitGatewayRouteTablesRequest) { - throw new UnsupportedOperationException("Not supported in mock"); - } - - @Override - public DescribeTransitGatewayVpcAttachmentsResult describeTransitGatewayVpcAttachments( - DescribeTransitGatewayVpcAttachmentsRequest describeTransitGatewayVpcAttachmentsRequest) { - throw new UnsupportedOperationException("Not supported in mock"); - } - - @Override - public DescribeTransitGatewaysResult describeTransitGateways(DescribeTransitGatewaysRequest describeTransitGatewaysRequest) { - throw new UnsupportedOperationException("Not supported in mock"); - } - - @Override - public ImportSnapshotResult importSnapshot() throws AmazonClientException { - throw new UnsupportedOperationException("Not supported in mock"); - } - - @Override - public DescribeSpotFleetRequestsResult describeSpotFleetRequests() throws AmazonClientException { - throw new UnsupportedOperationException("Not supported in mock"); - } - - @Override - public DescribeReservedInstancesModificationsResult describeReservedInstancesModifications() - throws AmazonClientException { - throw new UnsupportedOperationException("Not supported in mock"); - } - - @Override - public DeleteSpotDatafeedSubscriptionResult deleteSpotDatafeedSubscription() throws AmazonClientException { - throw new UnsupportedOperationException("Not supported in mock"); - } - - @Override - public DescribeAddressesResult describeAddresses() throws AmazonClientException { - throw new UnsupportedOperationException("Not supported in mock"); - } - - @Override - public DescribeAggregateIdFormatResult describeAggregateIdFormat(DescribeAggregateIdFormatRequest describeAggregateIdFormatRequest) { - throw new UnsupportedOperationException("Not supported in mock"); - } - - @Override - public DescribeKeyPairsResult describeKeyPairs() throws AmazonClientException { - throw new UnsupportedOperationException("Not supported in mock"); - } - - @Override - public DescribeLaunchTemplateVersionsResult describeLaunchTemplateVersions( - DescribeLaunchTemplateVersionsRequest describeLaunchTemplateVersionsRequest) { - throw new UnsupportedOperationException("Not supported in mock"); - } - - @Override - public DescribeLaunchTemplatesResult describeLaunchTemplates(DescribeLaunchTemplatesRequest describeLaunchTemplatesRequest) { - throw new UnsupportedOperationException("Not supported in mock"); - } - - @Override - public DescribeInstanceStatusResult describeInstanceStatus() throws AmazonClientException { - throw new UnsupportedOperationException("Not supported in mock"); - } - - @Override - public DescribeVpnGatewaysResult describeVpnGateways() throws AmazonClientException { - throw new UnsupportedOperationException("Not supported in mock"); - } - - @Override - public DescribeReservedInstancesOfferingsResult describeReservedInstancesOfferings() - throws AmazonClientException { - throw new UnsupportedOperationException("Not supported in mock"); - } - - @Override - public DescribeVpcEndpointServicesResult describeVpcEndpointServices() throws AmazonClientException { - throw new UnsupportedOperationException("Not supported in mock"); - } - - @Override - public AllocateAddressResult allocateAddress() throws AmazonClientException { - throw new UnsupportedOperationException("Not supported in mock"); - } - - @Override - public AllocateHostsResult allocateHosts(AllocateHostsRequest allocateHostsRequest) { - throw new UnsupportedOperationException("Not supported in mock"); - } - - @Override - public ApplySecurityGroupsToClientVpnTargetNetworkResult applySecurityGroupsToClientVpnTargetNetwork( - ApplySecurityGroupsToClientVpnTargetNetworkRequest applySecurityGroupsToClientVpnTargetNetworkRequest) { - throw new UnsupportedOperationException("Not supported in mock"); - } - - @Override - public DescribeSnapshotsResult describeSnapshots() throws AmazonClientException { - throw new UnsupportedOperationException("Not supported in mock"); - } - - @Override - public DescribeVpcEndpointsResult describeVpcEndpoints() throws AmazonClientException { - throw new UnsupportedOperationException("Not supported in mock"); - } - - @Override - public DryRunResult dryRun(DryRunSupportedRequest request) - throws AmazonClientException { - throw new UnsupportedOperationException("Not supported in mock"); - } - @Override public void shutdown() { } - - @Override - public AmazonEC2Waiters waiters() { - throw new UnsupportedOperationException("Not supported in mock"); - } - - @Override - public ResponseMetadata getCachedResponseMetadata(AmazonWebServiceRequest request) { - throw new UnsupportedOperationException("Not supported in mock"); - } - - @Override - public ModifySpotFleetRequestResult modifySpotFleetRequest(ModifySpotFleetRequestRequest modifySpotFleetRequestRequest) - throws AmazonClientException { - throw new UnsupportedOperationException("Not supported in mock"); - } } diff --git a/plugins/discovery-gce/build.gradle b/plugins/discovery-gce/build.gradle index 697cc3780a1fd..25baa4b17ce4a 100644 --- a/plugins/discovery-gce/build.gradle +++ b/plugins/discovery-gce/build.gradle @@ -17,6 +17,7 @@ dependencies { compile "org.apache.httpcomponents:httpclient:${versions.httpclient}" compile "org.apache.httpcomponents:httpcore:${versions.httpcore}" compile "commons-logging:commons-logging:${versions.commonslogging}" + compile "org.apache.logging.log4j:log4j-1.2-api:${versions.log4j}" compile "commons-codec:commons-codec:${versions.commonscodec}" } diff --git a/plugins/discovery-gce/licenses/log4j-1.2-api-2.11.1.jar.sha1 b/plugins/discovery-gce/licenses/log4j-1.2-api-2.11.1.jar.sha1 new file mode 100644 index 0000000000000..575d75dbda8c5 --- /dev/null +++ b/plugins/discovery-gce/licenses/log4j-1.2-api-2.11.1.jar.sha1 @@ -0,0 +1 @@ +3aba3398fe064a3eab4331f88161c7480e848418 \ No newline at end of file diff --git a/plugins/discovery-gce/licenses/log4j-LICENSE.txt b/plugins/discovery-gce/licenses/log4j-LICENSE.txt new file mode 100644 index 0000000000000..6279e5206de13 --- /dev/null +++ b/plugins/discovery-gce/licenses/log4j-LICENSE.txt @@ -0,0 +1,202 @@ + + Apache License + Version 2.0, January 2004 + http://www.apache.org/licenses/ + + TERMS AND CONDITIONS FOR USE, REPRODUCTION, AND DISTRIBUTION + + 1. Definitions. + + "License" shall mean the terms and conditions for use, reproduction, + and distribution as defined by Sections 1 through 9 of this document. + + "Licensor" shall mean the copyright owner or entity authorized by + the copyright owner that is granting the License. + + "Legal Entity" shall mean the union of the acting entity and all + other entities that control, are controlled by, or are under common + control with that entity. For the purposes of this definition, + "control" means (i) the power, direct or indirect, to cause the + direction or management of such entity, whether by contract or + otherwise, or (ii) ownership of fifty percent (50%) or more of the + outstanding shares, or (iii) beneficial ownership of such entity. + + "You" (or "Your") shall mean an individual or Legal Entity + exercising permissions granted by this License. + + "Source" form shall mean the preferred form for making modifications, + including but not limited to software source code, documentation + source, and configuration files. + + "Object" form shall mean any form resulting from mechanical + transformation or translation of a Source form, including but + not limited to compiled object code, generated documentation, + and conversions to other media types. + + "Work" shall mean the work of authorship, whether in Source or + Object form, made available under the License, as indicated by a + copyright notice that is included in or attached to the work + (an example is provided in the Appendix below). + + "Derivative Works" shall mean any work, whether in Source or Object + form, that is based on (or derived from) the Work and for which the + editorial revisions, annotations, elaborations, or other modifications + represent, as a whole, an original work of authorship. For the purposes + of this License, Derivative Works shall not include works that remain + separable from, or merely link (or bind by name) to the interfaces of, + the Work and Derivative Works thereof. + + "Contribution" shall mean any work of authorship, including + the original version of the Work and any modifications or additions + to that Work or Derivative Works thereof, that is intentionally + submitted to Licensor for inclusion in the Work by the copyright owner + or by an individual or Legal Entity authorized to submit on behalf of + the copyright owner. For the purposes of this definition, "submitted" + means any form of electronic, verbal, or written communication sent + to the Licensor or its representatives, including but not limited to + communication on electronic mailing lists, source code control systems, + and issue tracking systems that are managed by, or on behalf of, the + Licensor for the purpose of discussing and improving the Work, but + excluding communication that is conspicuously marked or otherwise + designated in writing by the copyright owner as "Not a Contribution." + + "Contributor" shall mean Licensor and any individual or Legal Entity + on behalf of whom a Contribution has been received by Licensor and + subsequently incorporated within the Work. + + 2. Grant of Copyright License. Subject to the terms and conditions of + this License, each Contributor hereby grants to You a perpetual, + worldwide, non-exclusive, no-charge, royalty-free, irrevocable + copyright license to reproduce, prepare Derivative Works of, + publicly display, publicly perform, sublicense, and distribute the + Work and such Derivative Works in Source or Object form. + + 3. Grant of Patent License. Subject to the terms and conditions of + this License, each Contributor hereby grants to You a perpetual, + worldwide, non-exclusive, no-charge, royalty-free, irrevocable + (except as stated in this section) patent license to make, have made, + use, offer to sell, sell, import, and otherwise transfer the Work, + where such license applies only to those patent claims licensable + by such Contributor that are necessarily infringed by their + Contribution(s) alone or by combination of their Contribution(s) + with the Work to which such Contribution(s) was submitted. If You + institute patent litigation against any entity (including a + cross-claim or counterclaim in a lawsuit) alleging that the Work + or a Contribution incorporated within the Work constitutes direct + or contributory patent infringement, then any patent licenses + granted to You under this License for that Work shall terminate + as of the date such litigation is filed. + + 4. Redistribution. You may reproduce and distribute copies of the + Work or Derivative Works thereof in any medium, with or without + modifications, and in Source or Object form, provided that You + meet the following conditions: + + (a) You must give any other recipients of the Work or + Derivative Works a copy of this License; and + + (b) You must cause any modified files to carry prominent notices + stating that You changed the files; and + + (c) You must retain, in the Source form of any Derivative Works + that You distribute, all copyright, patent, trademark, and + attribution notices from the Source form of the Work, + excluding those notices that do not pertain to any part of + the Derivative Works; and + + (d) If the Work includes a "NOTICE" text file as part of its + distribution, then any Derivative Works that You distribute must + include a readable copy of the attribution notices contained + within such NOTICE file, excluding those notices that do not + pertain to any part of the Derivative Works, in at least one + of the following places: within a NOTICE text file distributed + as part of the Derivative Works; within the Source form or + documentation, if provided along with the Derivative Works; or, + within a display generated by the Derivative Works, if and + wherever such third-party notices normally appear. The contents + of the NOTICE file are for informational purposes only and + do not modify the License. You may add Your own attribution + notices within Derivative Works that You distribute, alongside + or as an addendum to the NOTICE text from the Work, provided + that such additional attribution notices cannot be construed + as modifying the License. + + You may add Your own copyright statement to Your modifications and + may provide additional or different license terms and conditions + for use, reproduction, or distribution of Your modifications, or + for any such Derivative Works as a whole, provided Your use, + reproduction, and distribution of the Work otherwise complies with + the conditions stated in this License. + + 5. Submission of Contributions. Unless You explicitly state otherwise, + any Contribution intentionally submitted for inclusion in the Work + by You to the Licensor shall be under the terms and conditions of + this License, without any additional terms or conditions. + Notwithstanding the above, nothing herein shall supersede or modify + the terms of any separate license agreement you may have executed + with Licensor regarding such Contributions. + + 6. Trademarks. This License does not grant permission to use the trade + names, trademarks, service marks, or product names of the Licensor, + except as required for reasonable and customary use in describing the + origin of the Work and reproducing the content of the NOTICE file. + + 7. Disclaimer of Warranty. Unless required by applicable law or + agreed to in writing, Licensor provides the Work (and each + Contributor provides its Contributions) on an "AS IS" BASIS, + WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or + implied, including, without limitation, any warranties or conditions + of TITLE, NON-INFRINGEMENT, MERCHANTABILITY, or FITNESS FOR A + PARTICULAR PURPOSE. You are solely responsible for determining the + appropriateness of using or redistributing the Work and assume any + risks associated with Your exercise of permissions under this License. + + 8. Limitation of Liability. In no event and under no legal theory, + whether in tort (including negligence), contract, or otherwise, + unless required by applicable law (such as deliberate and grossly + negligent acts) or agreed to in writing, shall any Contributor be + liable to You for damages, including any direct, indirect, special, + incidental, or consequential damages of any character arising as a + result of this License or out of the use or inability to use the + Work (including but not limited to damages for loss of goodwill, + work stoppage, computer failure or malfunction, or any and all + other commercial damages or losses), even if such Contributor + has been advised of the possibility of such damages. + + 9. Accepting Warranty or Additional Liability. While redistributing + the Work or Derivative Works thereof, You may choose to offer, + and charge a fee for, acceptance of support, warranty, indemnity, + or other liability obligations and/or rights consistent with this + License. However, in accepting such obligations, You may act only + on Your own behalf and on Your sole responsibility, not on behalf + of any other Contributor, and only if You agree to indemnify, + defend, and hold each Contributor harmless for any liability + incurred by, or claims asserted against, such Contributor by reason + of your accepting any such warranty or additional liability. + + END OF TERMS AND CONDITIONS + + APPENDIX: How to apply the Apache License to your work. + + To apply the Apache License to your work, attach the following + boilerplate notice, with the fields enclosed by brackets "[]" + replaced with your own identifying information. (Don't include + the brackets!) The text should be enclosed in the appropriate + comment syntax for the file format. We also recommend that a + file or class name and description of purpose be included on the + same "printed page" as the copyright notice for easier + identification within third-party archives. + + Copyright 1999-2005 The Apache Software Foundation + + Licensed 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. diff --git a/plugins/discovery-gce/licenses/log4j-NOTICE.txt b/plugins/discovery-gce/licenses/log4j-NOTICE.txt new file mode 100644 index 0000000000000..0375732360047 --- /dev/null +++ b/plugins/discovery-gce/licenses/log4j-NOTICE.txt @@ -0,0 +1,5 @@ +Apache log4j +Copyright 2007 The Apache Software Foundation + +This product includes software developed at +The Apache Software Foundation (http://www.apache.org/). \ No newline at end of file diff --git a/plugins/discovery-gce/qa/gce/build.gradle b/plugins/discovery-gce/qa/gce/build.gradle index 16a65c762f6f7..b0231ff7f3c90 100644 --- a/plugins/discovery-gce/qa/gce/build.gradle +++ b/plugins/discovery-gce/qa/gce/build.gradle @@ -17,9 +17,12 @@ * under the License. */ + import org.elasticsearch.gradle.MavenFilteringHack import org.elasticsearch.gradle.test.AntFixture +import static org.elasticsearch.gradle.PropertyNormalization.IGNORE_VALUE + apply plugin: 'elasticsearch.standalone-rest-test' apply plugin: 'elasticsearch.rest-test' @@ -54,13 +57,13 @@ testClusters.integTest { numberOfNodes = gceNumberOfNodes plugin file(project(':plugins:discovery-gce').bundlePlugin.archiveFile) // use gce fixture for Auth calls instead of http://metadata.google.internal - environment 'GCE_METADATA_HOST', { "http://${gceFixture.addressAndPort}" } + environment 'GCE_METADATA_HOST', { "http://${gceFixture.addressAndPort}" }, IGNORE_VALUE // allows to configure hidden settings (`cloud.gce.host` and `cloud.gce.root_url`) systemProperty 'es.allow_reroute_gce_settings', 'true' setting 'discovery.seed_providers', 'gce' // use gce fixture for metadata server calls instead of http://metadata.google.internal - setting 'cloud.gce.host', { "http://${gceFixture.addressAndPort}" } + setting 'cloud.gce.host', { "http://${gceFixture.addressAndPort}" }, IGNORE_VALUE // use gce fixture for API calls instead of https://www.googleapis.com - setting 'cloud.gce.root_url', { "http://${gceFixture.addressAndPort}" } + setting 'cloud.gce.root_url', { "http://${gceFixture.addressAndPort}" }, IGNORE_VALUE } diff --git a/plugins/examples/build.gradle b/plugins/examples/build.gradle index 2b9f3c6433d8f..a251c1bbbe85d 100644 --- a/plugins/examples/build.gradle +++ b/plugins/examples/build.gradle @@ -7,3 +7,22 @@ gradle.projectsEvaluated { } } } + +configure(project('painless-whitelist')) { + configurations.all { + resolutionStrategy.dependencySubstitution { + substitute module('org.elasticsearch.plugin:elasticsearch-scripting-painless-spi') with project(':modules:lang-painless:spi') + substitute module('org.elasticsearch.test:logger-usage') with project(':test:logger-usage') + } + } +} + +configure(project('security-authorization-engine')) { + configurations.all { + resolutionStrategy.dependencySubstitution { + substitute module('org.elasticsearch.plugin:x-pack-core') with project(':x-pack:plugin:core') + substitute module('org.elasticsearch.client:elasticsearch-rest-high-level-client') with project(':client:rest-high-level') + substitute module('org.elasticsearch.test:logger-usage') with project(':test:logger-usage') + } + } +} \ No newline at end of file diff --git a/plugins/examples/custom-suggester/src/main/java/org/elasticsearch/example/customsuggester/CustomSuggester.java b/plugins/examples/custom-suggester/src/main/java/org/elasticsearch/example/customsuggester/CustomSuggester.java index b6a5b5e8f842f..9220206903eb5 100644 --- a/plugins/examples/custom-suggester/src/main/java/org/elasticsearch/example/customsuggester/CustomSuggester.java +++ b/plugins/examples/custom-suggester/src/main/java/org/elasticsearch/example/customsuggester/CustomSuggester.java @@ -25,6 +25,7 @@ import org.elasticsearch.search.suggest.Suggest; import org.elasticsearch.search.suggest.Suggester; +import java.io.IOException; import java.util.Locale; public class CustomSuggester extends Suggester { @@ -35,15 +36,12 @@ public Suggest.Suggestion { + public static final String NAME = "example_annotation"; - public AliasesExistRequestBuilder(ElasticsearchClient client, AliasesExistAction action, String... aliases) { - super(client, action, aliases); + public int category; + public String message; + + public ExamplePainlessAnnotation(int category, String message) { + this.category = category; + this.message = message; + } + + public int getCategory() { + return category; + } + + public String getMessage() { + return message; } } diff --git a/plugins/examples/painless-whitelist/src/main/java/org/elasticsearch/example/painlesswhitelist/ExampleWhitelistAnnotationParser.java b/plugins/examples/painless-whitelist/src/main/java/org/elasticsearch/example/painlesswhitelist/ExampleWhitelistAnnotationParser.java new file mode 100644 index 0000000000000..b0a5edd29cffc --- /dev/null +++ b/plugins/examples/painless-whitelist/src/main/java/org/elasticsearch/example/painlesswhitelist/ExampleWhitelistAnnotationParser.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.example.painlesswhitelist; + +import org.elasticsearch.painless.spi.annotation.WhitelistAnnotationParser; + +import java.util.Map; + +public class ExampleWhitelistAnnotationParser implements WhitelistAnnotationParser { + + public static final ExampleWhitelistAnnotationParser INSTANCE = new ExampleWhitelistAnnotationParser(); + + private ExampleWhitelistAnnotationParser() { + + } + + @Override + public Object parse(Map arguments) { + if (arguments.size() != 2) { + throw new IllegalArgumentException("expected exactly two arguments"); + } + + String categoryString = arguments.get("category"); + + if (categoryString == null) { + throw new IllegalArgumentException("expected category argument"); + } + + int category; + + try { + category = Integer.parseInt(categoryString); + } catch (NumberFormatException nfe) { + throw new IllegalArgumentException("expected category as an int, found [" + categoryString + "]", nfe); + } + + String message = arguments.get("message"); + + if (categoryString == null) { + throw new IllegalArgumentException("expected message argument"); + } + + return new ExamplePainlessAnnotation(category, message); + } +} diff --git a/plugins/examples/painless-whitelist/src/main/java/org/elasticsearch/example/painlesswhitelist/ExampleWhitelistExtension.java b/plugins/examples/painless-whitelist/src/main/java/org/elasticsearch/example/painlesswhitelist/ExampleWhitelistExtension.java index d0b03708a08b3..2ad4cb3d4de97 100644 --- a/plugins/examples/painless-whitelist/src/main/java/org/elasticsearch/example/painlesswhitelist/ExampleWhitelistExtension.java +++ b/plugins/examples/painless-whitelist/src/main/java/org/elasticsearch/example/painlesswhitelist/ExampleWhitelistExtension.java @@ -23,30 +23,34 @@ import org.elasticsearch.painless.spi.Whitelist; import org.elasticsearch.painless.spi.WhitelistInstanceBinding; import org.elasticsearch.painless.spi.WhitelistLoader; +import org.elasticsearch.painless.spi.annotation.WhitelistAnnotationParser; import org.elasticsearch.script.FieldScript; import org.elasticsearch.script.ScriptContext; import java.util.Arrays; import java.util.Collections; +import java.util.HashMap; import java.util.List; import java.util.Map; /** An extension of painless which adds a whitelist. */ public class ExampleWhitelistExtension implements PainlessExtension { - private static final Whitelist WHITELIST = - WhitelistLoader.loadFromResourceFiles(ExampleWhitelistExtension.class, "example_whitelist.txt"); - @Override public Map, List> getContextWhitelists() { + Map parsers = new HashMap<>(WhitelistAnnotationParser.BASE_ANNOTATION_PARSERS); + parsers.put(ExamplePainlessAnnotation.NAME, ExampleWhitelistAnnotationParser.INSTANCE); + Whitelist classWhitelist = + WhitelistLoader.loadFromResourceFiles(ExampleWhitelistExtension.class, parsers, "example_whitelist.txt"); + ExampleWhitelistedInstance ewi = new ExampleWhitelistedInstance(1); WhitelistInstanceBinding addValue = new WhitelistInstanceBinding("example addValue", ewi, - "addValue", "int", Collections.singletonList("int")); + "addValue", "int", Collections.singletonList("int"), Collections.emptyList()); WhitelistInstanceBinding getValue = new WhitelistInstanceBinding("example getValue", ewi, - "getValue", "int", Collections.emptyList()); + "getValue", "int", Collections.emptyList(), Collections.emptyList()); Whitelist instanceWhitelist = new Whitelist(ewi.getClass().getClassLoader(), Collections.emptyList(), Collections.emptyList(), Collections.emptyList(), Arrays.asList(addValue, getValue)); - return Collections.singletonMap(FieldScript.CONTEXT, Arrays.asList(WHITELIST, instanceWhitelist)); + return Collections.singletonMap(FieldScript.CONTEXT, Arrays.asList(classWhitelist, instanceWhitelist)); } } diff --git a/plugins/examples/painless-whitelist/src/main/java/org/elasticsearch/example/painlesswhitelist/ExampleWhitelistedClass.java b/plugins/examples/painless-whitelist/src/main/java/org/elasticsearch/example/painlesswhitelist/ExampleWhitelistedClass.java index 14f15b383d0c8..bb0e71f3dc870 100644 --- a/plugins/examples/painless-whitelist/src/main/java/org/elasticsearch/example/painlesswhitelist/ExampleWhitelistedClass.java +++ b/plugins/examples/painless-whitelist/src/main/java/org/elasticsearch/example/painlesswhitelist/ExampleWhitelistedClass.java @@ -54,4 +54,9 @@ public static void staticMethod() { public static int toInt(String x) { return Integer.parseInt(x); } + + // example method to attach annotations in whitelist + public void annotate() { + // some logic here + } } diff --git a/plugins/examples/painless-whitelist/src/main/resources/org/elasticsearch/example/painlesswhitelist/example_whitelist.txt b/plugins/examples/painless-whitelist/src/main/resources/org/elasticsearch/example/painlesswhitelist/example_whitelist.txt index 99e5521aebf36..9746b968d4df1 100644 --- a/plugins/examples/painless-whitelist/src/main/resources/org/elasticsearch/example/painlesswhitelist/example_whitelist.txt +++ b/plugins/examples/painless-whitelist/src/main/resources/org/elasticsearch/example/painlesswhitelist/example_whitelist.txt @@ -33,6 +33,9 @@ class org.elasticsearch.example.painlesswhitelist.ExampleWhitelistedClass { # getter and setter for private member int getPrivateMemberAccessor() void setPrivateMemberAccessor(int) + + # annotation + void annotate() @example_annotation[category="1",message="example annotation"] } class java.lang.String { diff --git a/plugins/examples/rest-handler/src/test/resources/rest-api-spec/api/cat.example.json b/plugins/examples/rest-handler/src/test/resources/rest-api-spec/api/cat.example.json index e42617eef7bf7..2aa5edd4d5ae1 100644 --- a/plugins/examples/rest-handler/src/test/resources/rest-api-spec/api/cat.example.json +++ b/plugins/examples/rest-handler/src/test/resources/rest-api-spec/api/cat.example.json @@ -1,9 +1,9 @@ { "cat.example": { "documentation": "", + "stability" : "stable", "methods": ["GET"], "url": { - "path": "/_cat/example", "paths": ["/_cat/example"], "parts": {}, "params": { diff --git a/plugins/mapper-annotated-text/src/test/java/org/elasticsearch/index/mapper/annotatedtext/AnnotatedTextFieldMapperTests.java b/plugins/mapper-annotated-text/src/test/java/org/elasticsearch/index/mapper/annotatedtext/AnnotatedTextFieldMapperTests.java index aa0dbe07c8305..969d91c2d6b5d 100644 --- a/plugins/mapper-annotated-text/src/test/java/org/elasticsearch/index/mapper/annotatedtext/AnnotatedTextFieldMapperTests.java +++ b/plugins/mapper-annotated-text/src/test/java/org/elasticsearch/index/mapper/annotatedtext/AnnotatedTextFieldMapperTests.java @@ -226,7 +226,7 @@ public void testAgainstTermVectorsAPI() throws IOException { } bulk.get(); - TermVectorsRequest request = new TermVectorsRequest("test", "type", "0").termStatistics(true); + TermVectorsRequest request = new TermVectorsRequest("test", "0").termStatistics(true); IndicesService indicesService = getInstanceFromNode(IndicesService.class); IndexService test = indicesService.indexService(resolveIndex("test")); diff --git a/plugins/repository-azure/build.gradle b/plugins/repository-azure/build.gradle index a7c1af412d949..47ee57d3a3e78 100644 --- a/plugins/repository-azure/build.gradle +++ b/plugins/repository-azure/build.gradle @@ -71,3 +71,26 @@ testClusters { keystore 'azure.client.integration_test.key', 'azure_key' } } + +String azureAccount = System.getenv("azure_storage_account") +String azureKey = System.getenv("azure_storage_key") +String azureContainer = System.getenv("azure_storage_container") +String azureBasePath = System.getenv("azure_storage_base_path") +String azureSasToken = System.getenv("azure_storage_sas_token") + +test { + exclude '**/AzureStorageCleanupThirdPartyTests.class' +} + +task thirdPartyTest(type: Test) { + include '**/AzureStorageCleanupThirdPartyTests.class' + systemProperty 'test.azure.account', azureAccount ? azureAccount : "" + systemProperty 'test.azure.key', azureKey ? azureKey : "" + systemProperty 'test.azure.sas_token', azureSasToken ? azureSasToken : "" + systemProperty 'test.azure.container', azureContainer ? azureContainer : "" + systemProperty 'test.azure.base', azureBasePath ? azureBasePath : "" +} + +if (azureAccount || azureKey || azureContainer || azureBasePath || azureSasToken) { + check.dependsOn(thirdPartyTest) +} diff --git a/plugins/repository-azure/qa/microsoft-azure-storage/build.gradle b/plugins/repository-azure/qa/microsoft-azure-storage/build.gradle index 8d8aaebac9f3f..5f96fd9dc305b 100644 --- a/plugins/repository-azure/qa/microsoft-azure-storage/build.gradle +++ b/plugins/repository-azure/qa/microsoft-azure-storage/build.gradle @@ -17,9 +17,13 @@ * under the License. */ + import org.elasticsearch.gradle.MavenFilteringHack import org.elasticsearch.gradle.test.AntFixture +import static org.elasticsearch.gradle.PropertyNormalization.DEFAULT +import static org.elasticsearch.gradle.PropertyNormalization.IGNORE_VALUE + apply plugin: 'elasticsearch.standalone-rest-test' apply plugin: 'elasticsearch.rest-test' @@ -29,12 +33,14 @@ String azureAccount = System.getenv("azure_storage_account") String azureKey = System.getenv("azure_storage_key") String azureContainer = System.getenv("azure_storage_container") String azureBasePath = System.getenv("azure_storage_base_path") +String azureSasToken = System.getenv("azure_storage_sas_token") -if (!azureAccount && !azureKey && !azureContainer && !azureBasePath) { +if (!azureAccount && !azureKey && !azureContainer && !azureBasePath && !azureSasToken) { azureAccount = 'azure_integration_test_account' azureKey = 'YXp1cmVfaW50ZWdyYXRpb25fdGVzdF9rZXk=' // The key is "azure_integration_test_key" encoded using base64 azureContainer = 'container_test' azureBasePath = 'integration_test' + azureSasToken = '' useFixture = true } @@ -63,14 +69,23 @@ integTest { testClusters.integTest { plugin file(project(':plugins:repository-azure').bundlePlugin.archiveFile) keystore 'azure.client.integration_test.account', azureAccount - keystore 'azure.client.integration_test.key', azureKey + if (azureKey != null && azureKey.isEmpty() == false) { + println "Using access key in external service tests." + keystore 'azure.client.integration_test.key', azureKey + } + if (azureSasToken != null && azureSasToken.isEmpty() == false) { + println "Using SAS token in external service tests." + keystore 'azure.client.integration_test.sas_token', azureSasToken + } if (useFixture) { tasks.integTest.dependsOn azureStorageFixture // Use a closure on the string to delay evaluation until tests are executed. The endpoint_suffix is used // in a hacky way to change the protocol and endpoint. We must fix that. setting 'azure.client.integration_test.endpoint_suffix', - { "ignored;DefaultEndpointsProtocol=http;BlobEndpoint=http://${azureStorageFixture.addressAndPort }" } + { "ignored;DefaultEndpointsProtocol=http;BlobEndpoint=http://${azureStorageFixture.addressAndPort }" }, IGNORE_VALUE + String firstPartOfSeed = project.rootProject.testSeed.tokenize(':').get(0) + setting 'thread_pool.repository_azure.max', (Math.abs(Long.parseUnsignedLong(firstPartOfSeed, 16) % 10) + 1).toString(), System.getProperty('ignore.tests.seed') == null ? DEFAULT : IGNORE_VALUE } else { println "Using an external service to test the repository-azure plugin" } diff --git a/plugins/repository-azure/qa/microsoft-azure-storage/src/test/resources/rest-api-spec/test/repository_azure/10_repository.yml b/plugins/repository-azure/qa/microsoft-azure-storage/src/test/resources/rest-api-spec/test/repository_azure/10_repository.yml index 92866190959e6..fade1f9f1e67d 100644 --- a/plugins/repository-azure/qa/microsoft-azure-storage/src/test/resources/rest-api-spec/test/repository_azure/10_repository.yml +++ b/plugins/repository-azure/qa/microsoft-azure-storage/src/test/resources/rest-api-spec/test/repository_azure/10_repository.yml @@ -15,6 +15,9 @@ setup: --- "Snapshot/Restore with repository-azure": + - skip: + version: " - 7.9.99" + reason: "8.0 changes get snapshots response format" # Get repository - do: @@ -121,9 +124,9 @@ setup: repository: repository snapshot: snapshot-one,snapshot-two - - is_true: snapshots - - match: { snapshots.0.state : SUCCESS } - - match: { snapshots.1.state : SUCCESS } + - is_true: responses.0.snapshots + - match: { responses.0.snapshots.0.state: SUCCESS } + - match: { responses.0.snapshots.1.state: SUCCESS } # Delete the index - do: @@ -203,13 +206,18 @@ setup: --- "Get a non existing snapshot": + - skip: + version: " - 7.9.99" + reason: "8.0 changes get snapshots response format" - do: - catch: /snapshot_missing_exception/ snapshot.get: repository: repository snapshot: missing + - is_true: responses.0.error + - match: { responses.0.error.type: snapshot_missing_exception } + --- "Delete a non existing snapshot": diff --git a/plugins/repository-azure/src/main/java/org/elasticsearch/repositories/azure/AzureBlobContainer.java b/plugins/repository-azure/src/main/java/org/elasticsearch/repositories/azure/AzureBlobContainer.java index 002907a0a7eb1..12113542dee44 100644 --- a/plugins/repository-azure/src/main/java/org/elasticsearch/repositories/azure/AzureBlobContainer.java +++ b/plugins/repository-azure/src/main/java/org/elasticsearch/repositories/azure/AzureBlobContainer.java @@ -23,29 +23,38 @@ import com.microsoft.azure.storage.StorageException; import org.apache.logging.log4j.LogManager; import org.apache.logging.log4j.Logger; +import org.elasticsearch.action.ActionListener; +import org.elasticsearch.action.ActionRunnable; +import org.elasticsearch.action.support.GroupedActionListener; +import org.elasticsearch.action.support.PlainActionFuture; import org.elasticsearch.common.Nullable; +import org.elasticsearch.common.blobstore.BlobContainer; import org.elasticsearch.common.blobstore.BlobMetaData; import org.elasticsearch.common.blobstore.BlobPath; import org.elasticsearch.common.blobstore.support.AbstractBlobContainer; +import org.elasticsearch.threadpool.ThreadPool; import java.io.IOException; import java.io.InputStream; import java.net.HttpURLConnection; import java.net.URISyntaxException; import java.nio.file.NoSuchFileException; +import java.util.List; import java.util.Map; +import java.util.concurrent.ExecutorService; public class AzureBlobContainer extends AbstractBlobContainer { private final Logger logger = LogManager.getLogger(AzureBlobContainer.class); private final AzureBlobStore blobStore; - + private final ThreadPool threadPool; private final String keyPath; - public AzureBlobContainer(BlobPath path, AzureBlobStore blobStore) { + AzureBlobContainer(BlobPath path, AzureBlobStore blobStore, ThreadPool threadPool) { super(path); this.blobStore = blobStore; this.keyPath = path.buildAsString(); + this.threadPool = threadPool; } @Override @@ -117,6 +126,43 @@ public void deleteBlob(String blobName) throws IOException { } } + @Override + public void delete() throws IOException { + try { + blobStore.deleteBlobDirectory(keyPath, threadPool.executor(AzureRepositoryPlugin.REPOSITORY_THREAD_POOL_NAME)); + } catch (URISyntaxException | StorageException e) { + throw new IOException(e); + } + } + + @Override + public void deleteBlobsIgnoringIfNotExists(List blobNames) throws IOException { + final PlainActionFuture result = PlainActionFuture.newFuture(); + if (blobNames.isEmpty()) { + result.onResponse(null); + } else { + final GroupedActionListener listener = + new GroupedActionListener<>(ActionListener.map(result, v -> null), blobNames.size()); + final ExecutorService executor = threadPool.executor(AzureRepositoryPlugin.REPOSITORY_THREAD_POOL_NAME); + // Executing deletes in parallel since Azure SDK 8 is using blocking IO while Azure does not provide a bulk delete API endpoint + // TODO: Upgrade to newer non-blocking Azure SDK 11 and execute delete requests in parallel that way. + for (String blobName : blobNames) { + executor.execute(new ActionRunnable<>(listener) { + @Override + protected void doRun() throws IOException { + deleteBlobIgnoringIfNotExists(blobName); + listener.onResponse(null); + } + }); + } + } + try { + result.actionGet(); + } catch (Exception e) { + throw new IOException("Exception during bulk delete", e); + } + } + @Override public Map listBlobsByPrefix(@Nullable String prefix) throws IOException { logger.trace("listBlobsByPrefix({})", prefix); @@ -135,6 +181,16 @@ public Map listBlobs() throws IOException { return listBlobsByPrefix(null); } + @Override + public Map children() throws IOException { + final BlobPath path = path(); + try { + return blobStore.children(path); + } catch (URISyntaxException | StorageException e) { + throw new IOException("Failed to list children in path [" + path.buildAsString() + "].", e); + } + } + protected String buildKey(String blobName) { return keyPath + (blobName == null ? "" : blobName); } diff --git a/plugins/repository-azure/src/main/java/org/elasticsearch/repositories/azure/AzureBlobStore.java b/plugins/repository-azure/src/main/java/org/elasticsearch/repositories/azure/AzureBlobStore.java index 7eeadc7f6475b..a7d9bb93a5125 100644 --- a/plugins/repository-azure/src/main/java/org/elasticsearch/repositories/azure/AzureBlobStore.java +++ b/plugins/repository-azure/src/main/java/org/elasticsearch/repositories/azure/AzureBlobStore.java @@ -28,27 +28,34 @@ import org.elasticsearch.common.blobstore.BlobPath; import org.elasticsearch.common.blobstore.BlobStore; import org.elasticsearch.repositories.azure.AzureRepository.Repository; +import org.elasticsearch.threadpool.ThreadPool; import java.io.IOException; import java.io.InputStream; import java.net.URISyntaxException; import java.nio.file.FileAlreadyExistsException; +import java.util.Collections; import java.util.Map; +import java.util.concurrent.Executor; +import java.util.function.Function; +import java.util.stream.Collectors; import static java.util.Collections.emptyMap; public class AzureBlobStore implements BlobStore { private final AzureStorageService service; + private final ThreadPool threadPool; private final String clientName; private final String container; private final LocationMode locationMode; - public AzureBlobStore(RepositoryMetaData metadata, AzureStorageService service) { + public AzureBlobStore(RepositoryMetaData metadata, AzureStorageService service, ThreadPool threadPool) { this.container = Repository.CONTAINER_SETTING.get(metadata.settings()); this.clientName = Repository.CLIENT_NAME.get(metadata.settings()); this.service = service; + this.threadPool = threadPool; // locationMode is set per repository, not per client this.locationMode = Repository.LOCATION_MODE_SETTING.get(metadata.settings()); final Map prevSettings = this.service.refreshAndClearCache(emptyMap()); @@ -70,17 +77,13 @@ public LocationMode getLocationMode() { @Override public BlobContainer blobContainer(BlobPath path) { - return new AzureBlobContainer(path, this); + return new AzureBlobContainer(path, this, threadPool); } @Override public void close() { } - public boolean containerExist() throws URISyntaxException, StorageException { - return service.doesContainerExist(clientName, container); - } - public boolean blobExists(String blob) throws URISyntaxException, StorageException { return service.blobExists(clientName, container, blob); } @@ -89,6 +92,10 @@ public void deleteBlob(String blob) throws URISyntaxException, StorageException service.deleteBlob(clientName, container, blob); } + public void deleteBlobDirectory(String path, Executor executor) throws URISyntaxException, StorageException, IOException { + service.deleteBlobDirectory(clientName, container, path, executor); + } + public InputStream getInputStream(String blob) throws URISyntaxException, StorageException, IOException { return service.getInputStream(clientName, container, blob); } @@ -98,6 +105,11 @@ public Map listBlobsByPrefix(String keyPath, String prefix return service.listBlobsByPrefix(clientName, container, keyPath, prefix); } + public Map children(BlobPath path) throws URISyntaxException, StorageException { + return Collections.unmodifiableMap(service.children(clientName, container, path).stream().collect( + Collectors.toMap(Function.identity(), name -> new AzureBlobContainer(path.add(name), this, threadPool)))); + } + public void writeBlob(String blobName, InputStream inputStream, long blobSize, boolean failIfAlreadyExists) throws URISyntaxException, StorageException, FileAlreadyExistsException { service.writeBlob(this.clientName, container, blobName, inputStream, blobSize, failIfAlreadyExists); diff --git a/plugins/repository-azure/src/main/java/org/elasticsearch/repositories/azure/AzureRepository.java b/plugins/repository-azure/src/main/java/org/elasticsearch/repositories/azure/AzureRepository.java index 7c3520918fc58..08b3bb3b45897 100644 --- a/plugins/repository-azure/src/main/java/org/elasticsearch/repositories/azure/AzureRepository.java +++ b/plugins/repository-azure/src/main/java/org/elasticsearch/repositories/azure/AzureRepository.java @@ -20,11 +20,9 @@ package org.elasticsearch.repositories.azure; import com.microsoft.azure.storage.LocationMode; -import com.microsoft.azure.storage.StorageException; import org.apache.logging.log4j.LogManager; import org.apache.logging.log4j.Logger; import org.apache.logging.log4j.message.ParameterizedMessage; -import org.elasticsearch.cluster.metadata.MetaData; import org.elasticsearch.cluster.metadata.RepositoryMetaData; import org.elasticsearch.common.Strings; import org.elasticsearch.common.blobstore.BlobPath; @@ -34,14 +32,9 @@ import org.elasticsearch.common.unit.ByteSizeValue; import org.elasticsearch.common.xcontent.NamedXContentRegistry; import org.elasticsearch.env.Environment; -import org.elasticsearch.repositories.IndexId; import org.elasticsearch.repositories.blobstore.BlobStoreRepository; -import org.elasticsearch.snapshots.SnapshotCreationException; -import org.elasticsearch.snapshots.SnapshotId; import org.elasticsearch.threadpool.ThreadPool; -import java.net.URISyntaxException; -import java.util.List; import java.util.Locale; import java.util.function.Function; @@ -79,17 +72,27 @@ public static final class Repository { public static final Setting READONLY_SETTING = Setting.boolSetting("readonly", false, Property.NodeScope); } - private final BlobPath basePath; private final ByteSizeValue chunkSize; private final AzureStorageService storageService; private final boolean readonly; public AzureRepository(RepositoryMetaData metadata, Environment environment, NamedXContentRegistry namedXContentRegistry, AzureStorageService storageService, ThreadPool threadPool) { - super(metadata, environment.settings(), namedXContentRegistry, threadPool); + super(metadata, environment.settings(), namedXContentRegistry, threadPool, buildBasePath(metadata)); this.chunkSize = Repository.CHUNK_SIZE_SETTING.get(metadata.settings()); this.storageService = storageService; + // If the user explicitly did not define a readonly value, we set it by ourselves depending on the location mode setting. + // For secondary_only setting, the repository should be read only + final LocationMode locationMode = Repository.LOCATION_MODE_SETTING.get(metadata.settings()); + if (Repository.READONLY_SETTING.exists(metadata.settings())) { + this.readonly = Repository.READONLY_SETTING.get(metadata.settings()); + } else { + this.readonly = locationMode == LocationMode.SECONDARY_ONLY; + } + } + + private static BlobPath buildBasePath(RepositoryMetaData metadata) { final String basePath = Strings.trimLeadingCharacter(Repository.BASE_PATH_SETTING.get(metadata.settings()), '/'); if (Strings.hasLength(basePath)) { // Remove starting / if any @@ -97,18 +100,9 @@ public AzureRepository(RepositoryMetaData metadata, Environment environment, Nam for(final String elem : basePath.split("/")) { path = path.add(elem); } - this.basePath = path; + return path; } else { - this.basePath = BlobPath.cleanPath(); - } - - // If the user explicitly did not define a readonly value, we set it by ourselves depending on the location mode setting. - // For secondary_only setting, the repository should be read only - final LocationMode locationMode = Repository.LOCATION_MODE_SETTING.get(metadata.settings()); - if (Repository.READONLY_SETTING.exists(metadata.settings())) { - this.readonly = Repository.READONLY_SETTING.get(metadata.settings()); - } else { - this.readonly = locationMode == LocationMode.SECONDARY_ONLY; + return BlobPath.cleanPath(); } } @@ -119,38 +113,19 @@ protected BlobStore getBlobStore() { @Override protected AzureBlobStore createBlobStore() { - final AzureBlobStore blobStore = new AzureBlobStore(metadata, storageService); + final AzureBlobStore blobStore = new AzureBlobStore(metadata, storageService, threadPool); logger.debug(() -> new ParameterizedMessage( "using container [{}], chunk_size [{}], compress [{}], base_path [{}]", - blobStore, chunkSize, isCompress(), basePath)); + blobStore, chunkSize, isCompress(), basePath())); return blobStore; } - @Override - protected BlobPath basePath() { - return basePath; - } - @Override protected ByteSizeValue chunkSize() { return chunkSize; } - @Override - public void initializeSnapshot(SnapshotId snapshotId, List indices, MetaData clusterMetadata) { - try { - final AzureBlobStore blobStore = (AzureBlobStore) blobStore(); - if (blobStore.containerExist() == false) { - throw new IllegalArgumentException("The bucket [" + blobStore + "] does not exist. Please create it before " - + " creating an azure snapshot repository backed by it."); - } - } catch (URISyntaxException | StorageException e) { - throw new SnapshotCreationException(metadata.name(), snapshotId, e); - } - super.initializeSnapshot(snapshotId, indices, clusterMetadata); - } - @Override public boolean isReadOnly() { return readonly; diff --git a/plugins/repository-azure/src/main/java/org/elasticsearch/repositories/azure/AzureRepositoryPlugin.java b/plugins/repository-azure/src/main/java/org/elasticsearch/repositories/azure/AzureRepositoryPlugin.java index ab48cf1314ec5..809ba9d515834 100644 --- a/plugins/repository-azure/src/main/java/org/elasticsearch/repositories/azure/AzureRepositoryPlugin.java +++ b/plugins/repository-azure/src/main/java/org/elasticsearch/repositories/azure/AzureRepositoryPlugin.java @@ -22,12 +22,15 @@ import org.elasticsearch.common.settings.Setting; import org.elasticsearch.common.settings.Settings; import org.elasticsearch.common.settings.SettingsException; +import org.elasticsearch.common.unit.TimeValue; import org.elasticsearch.common.xcontent.NamedXContentRegistry; import org.elasticsearch.env.Environment; import org.elasticsearch.plugins.Plugin; import org.elasticsearch.plugins.ReloadablePlugin; import org.elasticsearch.plugins.RepositoryPlugin; import org.elasticsearch.repositories.Repository; +import org.elasticsearch.threadpool.ExecutorBuilder; +import org.elasticsearch.threadpool.ScalingExecutorBuilder; import org.elasticsearch.threadpool.ThreadPool; import java.util.Arrays; @@ -40,6 +43,8 @@ */ public class AzureRepositoryPlugin extends Plugin implements RepositoryPlugin, ReloadablePlugin { + public static final String REPOSITORY_THREAD_POOL_NAME = "repository_azure"; + // protected for testing final AzureStorageService azureStoreService; @@ -60,6 +65,7 @@ public List> getSettings() { return Arrays.asList( AzureStorageSettings.ACCOUNT_SETTING, AzureStorageSettings.KEY_SETTING, + AzureStorageSettings.SAS_TOKEN_SETTING, AzureStorageSettings.ENDPOINT_SUFFIX_SETTING, AzureStorageSettings.TIMEOUT_SETTING, AzureStorageSettings.MAX_RETRIES_SETTING, @@ -69,6 +75,15 @@ public List> getSettings() { ); } + @Override + public List> getExecutorBuilders(Settings settings) { + return Collections.singletonList(executorBuilder()); + } + + public static ExecutorBuilder executorBuilder() { + return new ScalingExecutorBuilder(REPOSITORY_THREAD_POOL_NAME, 0, 32, TimeValue.timeValueSeconds(30L)); + } + @Override public void reload(Settings settings) { // secure settings should be readable diff --git a/plugins/repository-azure/src/main/java/org/elasticsearch/repositories/azure/AzureStorageService.java b/plugins/repository-azure/src/main/java/org/elasticsearch/repositories/azure/AzureStorageService.java index 89a78fd8045ee..f4ee7b9dbcad9 100644 --- a/plugins/repository-azure/src/main/java/org/elasticsearch/repositories/azure/AzureStorageService.java +++ b/plugins/repository-azure/src/main/java/org/elasticsearch/repositories/azure/AzureStorageService.java @@ -29,21 +29,26 @@ import com.microsoft.azure.storage.blob.BlobInputStream; import com.microsoft.azure.storage.blob.BlobListingDetails; import com.microsoft.azure.storage.blob.BlobProperties; +import com.microsoft.azure.storage.blob.CloudBlob; import com.microsoft.azure.storage.blob.CloudBlobClient; import com.microsoft.azure.storage.blob.CloudBlobContainer; +import com.microsoft.azure.storage.blob.CloudBlobDirectory; import com.microsoft.azure.storage.blob.CloudBlockBlob; import com.microsoft.azure.storage.blob.DeleteSnapshotsOption; import com.microsoft.azure.storage.blob.ListBlobItem; import org.apache.logging.log4j.LogManager; import org.apache.logging.log4j.Logger; import org.apache.logging.log4j.message.ParameterizedMessage; +import org.elasticsearch.action.support.PlainActionFuture; import org.elasticsearch.common.blobstore.BlobMetaData; +import org.elasticsearch.common.blobstore.BlobPath; import org.elasticsearch.common.blobstore.support.PlainBlobMetaData; import org.elasticsearch.common.collect.Tuple; import org.elasticsearch.common.settings.Settings; import org.elasticsearch.common.settings.SettingsException; import org.elasticsearch.common.unit.ByteSizeUnit; import org.elasticsearch.common.unit.ByteSizeValue; +import org.elasticsearch.common.util.concurrent.AbstractRunnable; import java.io.IOException; import java.io.InputStream; @@ -52,9 +57,16 @@ import java.net.URISyntaxException; import java.nio.file.FileAlreadyExistsException; import java.security.InvalidKeyException; +import java.util.ArrayList; +import java.util.Collection; +import java.util.Collections; import java.util.EnumSet; import java.util.HashMap; +import java.util.HashSet; import java.util.Map; +import java.util.Set; +import java.util.concurrent.Executor; +import java.util.concurrent.atomic.AtomicLong; import java.util.function.Supplier; import static java.util.Collections.emptyMap; @@ -116,7 +128,7 @@ private static CloudBlobClient buildClient(AzureStorageSettings azureStorageSett } private static CloudBlobClient createClient(AzureStorageSettings azureStorageSettings) throws InvalidKeyException, URISyntaxException { - final String connectionString = azureStorageSettings.buildConnectionString(); + final String connectionString = azureStorageSettings.getConnectString(); return CloudStorageAccount.parse(connectionString).createCloudBlobClient(); } @@ -140,12 +152,6 @@ public Map refreshAndClearCache(Map> client = client(account); - final CloudBlobContainer blobContainer = client.v1().getContainerReference(container); - return SocketAccess.doPrivilegedException(() -> blobContainer.exists(null, null, client.v2().get())); - } - /** * Extract the blob name from a URI like https://myservice.azure.net/container/path/to/myfile * It should remove the container part (first part of the path) and gives path/to/myfile @@ -186,6 +192,50 @@ public void deleteBlob(String account, String container, String blob) throws URI }); } + void deleteBlobDirectory(String account, String container, String path, Executor executor) + throws URISyntaxException, StorageException, IOException { + final Tuple> client = client(account); + final CloudBlobContainer blobContainer = client.v1().getContainerReference(container); + final Collection exceptions = Collections.synchronizedList(new ArrayList<>()); + final AtomicLong outstanding = new AtomicLong(1L); + final PlainActionFuture result = PlainActionFuture.newFuture(); + SocketAccess.doPrivilegedVoidException(() -> { + for (final ListBlobItem blobItem : blobContainer.listBlobs(path, true)) { + // uri.getPath is of the form /container/keyPath.* and we want to strip off the /container/ + // this requires 1 + container.length() + 1, with each 1 corresponding to one of the / + final String blobPath = blobItem.getUri().getPath().substring(1 + container.length() + 1); + outstanding.incrementAndGet(); + executor.execute(new AbstractRunnable() { + @Override + protected void doRun() throws Exception { + deleteBlob(account, container, blobPath); + } + + @Override + public void onFailure(Exception e) { + exceptions.add(e); + } + + @Override + public void onAfter() { + if (outstanding.decrementAndGet() == 0) { + result.onResponse(null); + } + } + }); + } + }); + if (outstanding.decrementAndGet() == 0) { + result.onResponse(null); + } + result.actionGet(); + if (exceptions.isEmpty() == false) { + final IOException ex = new IOException("Deleting directory [" + path + "] failed"); + exceptions.forEach(ex::addSuppressed); + throw ex; + } + } + public InputStream getInputStream(String account, String container, String blob) throws URISyntaxException, StorageException, IOException { final Tuple> client = client(account); @@ -214,15 +264,40 @@ public Map listBlobsByPrefix(String account, String contai // uri.getPath is of the form /container/keyPath.* and we want to strip off the /container/ // this requires 1 + container.length() + 1, with each 1 corresponding to one of the / final String blobPath = uri.getPath().substring(1 + container.length() + 1); - final BlobProperties properties = ((CloudBlockBlob) blobItem).getProperties(); - final String name = blobPath.substring(keyPath.length()); - logger.trace(() -> new ParameterizedMessage("blob url [{}], name [{}], size [{}]", uri, name, properties.getLength())); - blobsBuilder.put(name, new PlainBlobMetaData(name, properties.getLength())); + if (blobItem instanceof CloudBlob) { + final BlobProperties properties = ((CloudBlob) blobItem).getProperties(); + final String name = blobPath.substring(keyPath.length()); + logger.trace(() -> new ParameterizedMessage("blob url [{}], name [{}], size [{}]", uri, name, properties.getLength())); + blobsBuilder.put(name, new PlainBlobMetaData(name, properties.getLength())); + } } }); return Map.copyOf(blobsBuilder); } + public Set children(String account, String container, BlobPath path) throws URISyntaxException, StorageException { + final var blobsBuilder = new HashSet(); + final Tuple> client = client(account); + final CloudBlobContainer blobContainer = client.v1().getContainerReference(container); + final String keyPath = path.buildAsString(); + final EnumSet enumBlobListingDetails = EnumSet.of(BlobListingDetails.METADATA); + + SocketAccess.doPrivilegedVoidException(() -> { + for (ListBlobItem blobItem : blobContainer.listBlobs(keyPath, false, enumBlobListingDetails, null, client.v2().get())) { + if (blobItem instanceof CloudBlobDirectory) { + final URI uri = blobItem.getUri(); + logger.trace(() -> new ParameterizedMessage("blob url [{}]", uri)); + // uri.getPath is of the form /container/keyPath.* and we want to strip off the /container/ + // this requires 1 + container.length() + 1, with each 1 corresponding to one of the /. + // Lastly, we add the length of keyPath to the offset to strip this container's path. + final String uriPath = uri.getPath(); + blobsBuilder.add(uriPath.substring(1 + container.length() + 1 + keyPath.length(), uriPath.length() - 1)); + } + } + }); + return Set.copyOf(blobsBuilder); + } + public void writeBlob(String account, String container, String blobName, InputStream inputStream, long blobSize, boolean failIfAlreadyExists) throws URISyntaxException, StorageException, FileAlreadyExistsException { diff --git a/plugins/repository-azure/src/main/java/org/elasticsearch/repositories/azure/AzureStorageSettings.java b/plugins/repository-azure/src/main/java/org/elasticsearch/repositories/azure/AzureStorageSettings.java index e57d855cb0ee5..d87e48542d1c4 100644 --- a/plugins/repository-azure/src/main/java/org/elasticsearch/repositories/azure/AzureStorageSettings.java +++ b/plugins/repository-azure/src/main/java/org/elasticsearch/repositories/azure/AzureStorageSettings.java @@ -21,6 +21,7 @@ import com.microsoft.azure.storage.LocationMode; import com.microsoft.azure.storage.RetryPolicy; +import org.elasticsearch.common.Nullable; import org.elasticsearch.common.Strings; import org.elasticsearch.common.settings.SecureSetting; import org.elasticsearch.common.settings.SecureString; @@ -53,6 +54,10 @@ final class AzureStorageSettings { public static final AffixSetting KEY_SETTING = Setting.affixKeySetting(AZURE_CLIENT_PREFIX_KEY, "key", key -> SecureSetting.secureString(key, null)); + /** Azure SAS token */ + public static final AffixSetting SAS_TOKEN_SETTING = Setting.affixKeySetting(AZURE_CLIENT_PREFIX_KEY, "sas_token", + key -> SecureSetting.secureString(key, null)); + /** max_retries: Number of retries in case of Azure errors. Defaults to 3 (RetryPolicy.DEFAULT_CLIENT_RETRY_COUNT). */ public static final Setting MAX_RETRIES_SETTING = Setting.affixKeySetting(AZURE_CLIENT_PREFIX_KEY, "max_retries", @@ -82,7 +87,7 @@ final class AzureStorageSettings { PROXY_HOST_SETTING); private final String account; - private final String key; + private final String connectString; private final String endpointSuffix; private final TimeValue timeout; private final int maxRetries; @@ -90,10 +95,10 @@ final class AzureStorageSettings { private final LocationMode locationMode; // copy-constructor - private AzureStorageSettings(String account, String key, String endpointSuffix, TimeValue timeout, int maxRetries, Proxy proxy, - LocationMode locationMode) { + private AzureStorageSettings(String account, String connectString, String endpointSuffix, TimeValue timeout, int maxRetries, + Proxy proxy, LocationMode locationMode) { this.account = account; - this.key = key; + this.connectString = connectString; this.endpointSuffix = endpointSuffix; this.timeout = timeout; this.maxRetries = maxRetries; @@ -101,10 +106,10 @@ private AzureStorageSettings(String account, String key, String endpointSuffix, this.locationMode = locationMode; } - AzureStorageSettings(String account, String key, String endpointSuffix, TimeValue timeout, int maxRetries, - Proxy.Type proxyType, String proxyHost, Integer proxyPort) { + private AzureStorageSettings(String account, String key, String sasToken, String endpointSuffix, TimeValue timeout, int maxRetries, + Proxy.Type proxyType, String proxyHost, Integer proxyPort) { this.account = account; - this.key = key; + this.connectString = buildConnectString(account, key, sasToken, endpointSuffix); this.endpointSuffix = endpointSuffix; this.timeout = timeout; this.maxRetries = maxRetries; @@ -145,13 +150,26 @@ public Proxy getProxy() { return proxy; } - public String buildConnectionString() { + public String getConnectString() { + return connectString; + } + + private static String buildConnectString(String account, @Nullable String key, @Nullable String sasToken, String endpointSuffix) { + final boolean hasSasToken = Strings.hasText(sasToken); + final boolean hasKey = Strings.hasText(key); + if (hasSasToken == false && hasKey == false) { + throw new SettingsException("Neither a secret key nor a shared access token was set."); + } + if (hasSasToken && hasKey) { + throw new SettingsException("Both a secret as well as a shared access token were set."); + } final StringBuilder connectionStringBuilder = new StringBuilder(); - connectionStringBuilder.append("DefaultEndpointsProtocol=https") - .append(";AccountName=") - .append(account) - .append(";AccountKey=") - .append(key); + connectionStringBuilder.append("DefaultEndpointsProtocol=https").append(";AccountName=").append(account); + if (hasKey) { + connectionStringBuilder.append(";AccountKey=").append(key); + } else { + connectionStringBuilder.append(";SharedAccessSignature=").append(sasToken); + } if (Strings.hasText(endpointSuffix)) { connectionStringBuilder.append(";EndpointSuffix=").append(endpointSuffix); } @@ -166,7 +184,6 @@ public LocationMode getLocationMode() { public String toString() { final StringBuilder sb = new StringBuilder("AzureStorageSettings{"); sb.append("account='").append(account).append('\''); - sb.append(", key='").append(key).append('\''); sb.append(", timeout=").append(timeout); sb.append(", endpointSuffix='").append(endpointSuffix).append('\''); sb.append(", maxRetries=").append(maxRetries); @@ -201,8 +218,9 @@ public static Map load(Settings settings) { /** Parse settings for a single client. */ private static AzureStorageSettings getClientSettings(Settings settings, String clientName) { try (SecureString account = getConfigValue(settings, clientName, ACCOUNT_SETTING); - SecureString key = getConfigValue(settings, clientName, KEY_SETTING)) { - return new AzureStorageSettings(account.toString(), key.toString(), + SecureString key = getConfigValue(settings, clientName, KEY_SETTING); + SecureString sasToken = getConfigValue(settings, clientName, SAS_TOKEN_SETTING)) { + return new AzureStorageSettings(account.toString(), key.toString(), sasToken.toString(), getValue(settings, clientName, ENDPOINT_SUFFIX_SETTING), getValue(settings, clientName, TIMEOUT_SETTING), getValue(settings, clientName, MAX_RETRIES_SETTING), @@ -228,10 +246,9 @@ static Map overrideLocationMode(Map(); for (final Map.Entry entry : clientsSettings.entrySet()) { - final AzureStorageSettings azureSettings = new AzureStorageSettings(entry.getValue().account, entry.getValue().key, - entry.getValue().endpointSuffix, entry.getValue().timeout, entry.getValue().maxRetries, entry.getValue().proxy, - locationMode); - map.put(entry.getKey(), azureSettings); + map.put(entry.getKey(), + new AzureStorageSettings(entry.getValue().account, entry.getValue().connectString, entry.getValue().endpointSuffix, + entry.getValue().timeout, entry.getValue().maxRetries, entry.getValue().proxy, locationMode)); } return Map.copyOf(map); } diff --git a/plugins/repository-azure/src/test/java/org/elasticsearch/repositories/azure/AzureBlobStoreContainerTests.java b/plugins/repository-azure/src/test/java/org/elasticsearch/repositories/azure/AzureBlobStoreContainerTests.java index 13cc487a1c122..07d0a1e18d3bd 100644 --- a/plugins/repository-azure/src/test/java/org/elasticsearch/repositories/azure/AzureBlobStoreContainerTests.java +++ b/plugins/repository-azure/src/test/java/org/elasticsearch/repositories/azure/AzureBlobStoreContainerTests.java @@ -23,13 +23,31 @@ import org.elasticsearch.common.blobstore.BlobStore; import org.elasticsearch.common.settings.Settings; import org.elasticsearch.repositories.ESBlobStoreContainerTestCase; +import org.elasticsearch.threadpool.TestThreadPool; +import org.elasticsearch.threadpool.ThreadPool; +import java.util.concurrent.TimeUnit; public class AzureBlobStoreContainerTests extends ESBlobStoreContainerTestCase { + + private ThreadPool threadPool; + + @Override + public void setUp() throws Exception { + super.setUp(); + threadPool = new TestThreadPool("AzureBlobStoreTests", AzureRepositoryPlugin.executorBuilder()); + } + + @Override + public void tearDown() throws Exception { + super.tearDown(); + ThreadPool.terminate(threadPool, 10L, TimeUnit.SECONDS); + } + @Override protected BlobStore newBlobStore() { RepositoryMetaData repositoryMetaData = new RepositoryMetaData("azure", "ittest", Settings.EMPTY); AzureStorageServiceMock client = new AzureStorageServiceMock(); - return new AzureBlobStore(repositoryMetaData, client); + return new AzureBlobStore(repositoryMetaData, client, threadPool); } } diff --git a/plugins/repository-azure/src/test/java/org/elasticsearch/repositories/azure/AzureBlobStoreTests.java b/plugins/repository-azure/src/test/java/org/elasticsearch/repositories/azure/AzureBlobStoreTests.java index 67d30fda05b69..74bfcb784aed0 100644 --- a/plugins/repository-azure/src/test/java/org/elasticsearch/repositories/azure/AzureBlobStoreTests.java +++ b/plugins/repository-azure/src/test/java/org/elasticsearch/repositories/azure/AzureBlobStoreTests.java @@ -22,13 +22,31 @@ import org.elasticsearch.common.blobstore.BlobStore; import org.elasticsearch.common.settings.Settings; import org.elasticsearch.repositories.ESBlobStoreTestCase; +import org.elasticsearch.threadpool.TestThreadPool; +import org.elasticsearch.threadpool.ThreadPool; + +import java.util.concurrent.TimeUnit; public class AzureBlobStoreTests extends ESBlobStoreTestCase { + private ThreadPool threadPool; + + @Override + public void setUp() throws Exception { + super.setUp(); + threadPool = new TestThreadPool("AzureBlobStoreTests", AzureRepositoryPlugin.executorBuilder()); + } + + @Override + public void tearDown() throws Exception { + super.tearDown(); + ThreadPool.terminate(threadPool, 10L, TimeUnit.SECONDS); + } + @Override protected BlobStore newBlobStore() { RepositoryMetaData repositoryMetaData = new RepositoryMetaData("azure", "ittest", Settings.EMPTY); AzureStorageServiceMock client = new AzureStorageServiceMock(); - return new AzureBlobStore(repositoryMetaData, client); + return new AzureBlobStore(repositoryMetaData, client, threadPool); } } diff --git a/plugins/repository-azure/src/test/java/org/elasticsearch/repositories/azure/AzureStorageCleanupThirdPartyTests.java b/plugins/repository-azure/src/test/java/org/elasticsearch/repositories/azure/AzureStorageCleanupThirdPartyTests.java new file mode 100644 index 0000000000000..1c5c2dd39fae6 --- /dev/null +++ b/plugins/repository-azure/src/test/java/org/elasticsearch/repositories/azure/AzureStorageCleanupThirdPartyTests.java @@ -0,0 +1,75 @@ +/* + * 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.repositories.azure; + +import org.elasticsearch.action.support.master.AcknowledgedResponse; +import org.elasticsearch.common.Strings; +import org.elasticsearch.common.settings.MockSecureSettings; +import org.elasticsearch.common.settings.SecureSettings; +import org.elasticsearch.common.settings.Settings; +import org.elasticsearch.plugins.Plugin; +import org.elasticsearch.repositories.AbstractThirdPartyRepositoryTestCase; + +import java.util.Collection; + +import static org.hamcrest.Matchers.blankOrNullString; +import static org.hamcrest.Matchers.equalTo; +import static org.hamcrest.Matchers.not; + +public class AzureStorageCleanupThirdPartyTests extends AbstractThirdPartyRepositoryTestCase { + + @Override + protected Collection> getPlugins() { + return pluginList(AzureRepositoryPlugin.class); + } + + @Override + protected SecureSettings credentials() { + assertThat(System.getProperty("test.azure.account"), not(blankOrNullString())); + final boolean hasSasToken = Strings.hasText(System.getProperty("test.azure.sas_token")); + if (hasSasToken == false) { + assertThat(System.getProperty("test.azure.key"), not(blankOrNullString())); + } else { + assertThat(System.getProperty("test.azure.key"), blankOrNullString()); + } + assertThat(System.getProperty("test.azure.container"), not(blankOrNullString())); + assertThat(System.getProperty("test.azure.base"), not(blankOrNullString())); + + MockSecureSettings secureSettings = new MockSecureSettings(); + secureSettings.setString("azure.client.default.account", System.getProperty("test.azure.account")); + if (hasSasToken) { + secureSettings.setString("azure.client.default.sas_token", System.getProperty("test.azure.sas_token")); + } else { + secureSettings.setString("azure.client.default.key", System.getProperty("test.azure.key")); + } + return secureSettings; + } + + @Override + protected void createRepository(String repoName) { + AcknowledgedResponse putRepositoryResponse = client().admin().cluster().preparePutRepository(repoName) + .setType("azure") + .setSettings(Settings.builder() + .put("container", System.getProperty("test.azure.container")) + .put("base_path", System.getProperty("test.azure.base")) + ).get(); + assertThat(putRepositoryResponse.isAcknowledged(), equalTo(true)); + } +} diff --git a/plugins/repository-azure/src/test/java/org/elasticsearch/repositories/azure/AzureStorageServiceMock.java b/plugins/repository-azure/src/test/java/org/elasticsearch/repositories/azure/AzureStorageServiceMock.java index 17502a1d1f982..2217b6743fb2e 100644 --- a/plugins/repository-azure/src/test/java/org/elasticsearch/repositories/azure/AzureStorageServiceMock.java +++ b/plugins/repository-azure/src/test/java/org/elasticsearch/repositories/azure/AzureStorageServiceMock.java @@ -55,11 +55,6 @@ public AzureStorageServiceMock() { super(Settings.EMPTY); } - @Override - public boolean doesContainerExist(String account, String container) { - return true; - } - @Override public boolean blobExists(String account, String container, String blob) { return blobs.containsKey(blob); diff --git a/plugins/repository-azure/src/test/java/org/elasticsearch/repositories/azure/AzureStorageServiceTests.java b/plugins/repository-azure/src/test/java/org/elasticsearch/repositories/azure/AzureStorageServiceTests.java index f7b49bd24adf6..128e0e0a2140e 100644 --- a/plugins/repository-azure/src/test/java/org/elasticsearch/repositories/azure/AzureStorageServiceTests.java +++ b/plugins/repository-azure/src/test/java/org/elasticsearch/repositories/azure/AzureStorageServiceTests.java @@ -158,15 +158,21 @@ public void testReinitClientWrongSettings() throws IOException { secureSettings2.setString("azure.client.azure1.account", "myaccount1"); // missing key final Settings settings2 = Settings.builder().setSecureSettings(secureSettings2).build(); + final MockSecureSettings secureSettings3 = new MockSecureSettings(); + secureSettings3.setString("azure.client.azure1.account", "myaccount3"); + secureSettings3.setString("azure.client.azure1.key", encodeKey("mykey33")); + secureSettings3.setString("azure.client.azure1.sas_token", encodeKey("mysasToken33")); + final Settings settings3 = Settings.builder().setSecureSettings(secureSettings3).build(); try (AzureRepositoryPlugin plugin = pluginWithSettingsValidation(settings1)) { final AzureStorageService azureStorageService = plugin.azureStoreService; final CloudBlobClient client11 = azureStorageService.client("azure1").v1(); assertThat(client11.getEndpoint().toString(), equalTo("https://myaccount1.blob.core.windows.net")); - plugin.reload(settings2); + final SettingsException e1 = expectThrows(SettingsException.class, () -> plugin.reload(settings2)); + assertThat(e1.getMessage(), is("Neither a secret key nor a shared access token was set.")); + final SettingsException e2 = expectThrows(SettingsException.class, () -> plugin.reload(settings3)); + assertThat(e2.getMessage(), is("Both a secret as well as a shared access token were set.")); // existing client untouched assertThat(client11.getEndpoint().toString(), equalTo("https://myaccount1.blob.core.windows.net")); - final SettingsException e = expectThrows(SettingsException.class, () -> azureStorageService.client("azure1")); - assertThat(e.getMessage(), is("Invalid azure client settings with name [azure1]")); } } diff --git a/plugins/repository-gcs/build.gradle b/plugins/repository-gcs/build.gradle index e5af9081ca189..f16a82fc5587f 100644 --- a/plugins/repository-gcs/build.gradle +++ b/plugins/repository-gcs/build.gradle @@ -1,3 +1,5 @@ +import java.nio.file.Files + /* * Licensed to Elasticsearch under one or more contributor * license agreements. See the NOTICE file distributed with @@ -23,38 +25,35 @@ esplugin { } dependencies { - compile 'com.google.cloud:google-cloud-storage:1.59.0' - compile 'com.google.cloud:google-cloud-core:1.59.0' - compile 'com.google.guava:guava:20.0' - compile "joda-time:joda-time:${versions.joda}" - compile 'com.google.http-client:google-http-client:1.24.1' + compile 'com.google.cloud:google-cloud-storage:1.77.0' + compile 'com.google.cloud:google-cloud-core:1.77.0' + compile 'com.google.guava:guava:26.0-jre' + compile 'com.google.http-client:google-http-client:1.30.1' compile "org.apache.httpcomponents:httpclient:${versions.httpclient}" compile "org.apache.httpcomponents:httpcore:${versions.httpcore}" compile "commons-logging:commons-logging:${versions.commonslogging}" + compile "org.apache.logging.log4j:log4j-1.2-api:${versions.log4j}" compile "commons-codec:commons-codec:${versions.commonscodec}" - compile 'com.google.api:api-common:1.7.0' - compile 'com.google.api:gax:1.30.0' + compile 'com.google.api:api-common:1.8.1' + compile 'com.google.api:gax:1.45.0' compile 'org.threeten:threetenbp:1.3.3' - compile 'com.google.protobuf:protobuf-java-util:3.6.0' - compile 'com.google.protobuf:protobuf-java:3.6.0' + compile 'com.google.protobuf:protobuf-java-util:3.7.1' + compile 'com.google.protobuf:protobuf-java:3.7.1' compile 'com.google.code.gson:gson:2.7' - compile 'com.google.api.grpc:proto-google-common-protos:1.12.0' + compile 'com.google.api.grpc:proto-google-common-protos:1.16.0' compile 'com.google.api.grpc:proto-google-iam-v1:0.12.0' - compile 'com.google.cloud:google-cloud-core-http:1.59.0' - compile 'com.google.auth:google-auth-library-credentials:0.10.0' - compile 'com.google.auth:google-auth-library-oauth2-http:0.10.0' - compile 'com.google.oauth-client:google-oauth-client:1.24.1' - compile 'com.google.api-client:google-api-client:1.24.1' - compile 'com.google.http-client:google-http-client-appengine:1.24.1' - compile 'com.google.http-client:google-http-client-jackson:1.24.1' - compile 'org.codehaus.jackson:jackson-core-asl:1.9.11' - compile 'com.google.http-client:google-http-client-jackson2:1.24.1' - compile "com.fasterxml.jackson.core:jackson-core:${versions.jackson}" - compile 'com.google.api:gax-httpjson:0.47.0' - compile 'io.opencensus:opencensus-api:0.15.0' + compile 'com.google.cloud:google-cloud-core-http:1.77.0' + compile 'com.google.auth:google-auth-library-credentials:0.16.1' + compile 'com.google.auth:google-auth-library-oauth2-http:0.16.1' + compile 'com.google.oauth-client:google-oauth-client:1.28.0' + compile 'com.google.api-client:google-api-client:1.28.0' + compile 'com.google.http-client:google-http-client-appengine:1.29.2' + compile 'com.google.http-client:google-http-client-jackson2:1.29.2' + compile 'com.google.api:gax-httpjson:0.62.0' compile 'io.grpc:grpc-context:1.12.0' - compile 'io.opencensus:opencensus-contrib-http-util:0.15.0' - compile 'com.google.apis:google-api-services-storage:v1-rev135-1.24.1' + compile 'io.opencensus:opencensus-api:0.18.0' + compile 'io.opencensus:opencensus-contrib-http-util:0.18.0' + compile 'com.google.apis:google-api-services-storage:v1-rev20190426-1.28.0' } dependencyLicenses { @@ -62,7 +61,6 @@ dependencyLicenses { mapping from: /google-auth-.*/, to: 'google-auth' mapping from: /google-http-.*/, to: 'google-http' mapping from: /opencensus.*/, to: 'opencensus' - mapping from: /jackson-.*/, to: 'jackson' mapping from: /http.*/, to: 'httpclient' mapping from: /protobuf.*/, to: 'protobuf' mapping from: /proto-google.*/, to: 'proto-google' @@ -78,6 +76,10 @@ thirdPartyAudit { 'com.google.common.cache.Striped64', 'com.google.common.cache.Striped64$1', 'com.google.common.cache.Striped64$Cell', + 'com.google.common.hash.Striped64', + 'com.google.common.hash.Striped64$1', + 'com.google.common.hash.Striped64$Cell', + 'com.google.common.hash.LittleEndianByteArray$UnsafeByteArray', 'com.google.common.hash.LittleEndianByteArray$UnsafeByteArray$1', 'com.google.common.hash.LittleEndianByteArray$UnsafeByteArray$2', 'com.google.common.hash.LittleEndianByteArray$UnsafeByteArray$3', @@ -122,3 +124,22 @@ check { // also execute the QA tests when testing the plugin dependsOn 'qa:google-cloud-storage:check' } + +String gcsServiceAccount = System.getenv("google_storage_service_account") +String gcsBucket = System.getenv("google_storage_bucket") +String gcsBasePath = System.getenv("google_storage_base_path") + +test { + exclude '**/GoogleCloudStorageThirdPartyTests.class' +} + +task thirdPartyTest(type: Test) { + include '**/GoogleCloudStorageThirdPartyTests.class' + systemProperty 'test.google.account', gcsServiceAccount ? Base64.encoder.encodeToString(Files.readAllBytes(file(gcsServiceAccount).toPath())) : "" + systemProperty 'test.google.bucket', gcsBucket ? gcsBucket : "" + systemProperty 'test.google.base', gcsBasePath ? gcsBasePath : "/" +} + +if (gcsServiceAccount || gcsBucket || gcsBasePath) { + check.dependsOn(thirdPartyTest) +} \ No newline at end of file diff --git a/plugins/repository-gcs/licenses/api-common-1.7.0.jar.sha1 b/plugins/repository-gcs/licenses/api-common-1.7.0.jar.sha1 deleted file mode 100644 index 67291b658e5c5..0000000000000 --- a/plugins/repository-gcs/licenses/api-common-1.7.0.jar.sha1 +++ /dev/null @@ -1 +0,0 @@ -ea59fb8b2450999345035dec8a6f472543391766 \ No newline at end of file diff --git a/plugins/repository-gcs/licenses/api-common-1.8.1.jar.sha1 b/plugins/repository-gcs/licenses/api-common-1.8.1.jar.sha1 new file mode 100644 index 0000000000000..7a1c114c6c0fc --- /dev/null +++ b/plugins/repository-gcs/licenses/api-common-1.8.1.jar.sha1 @@ -0,0 +1 @@ +e89befb19b08ad84b262b2f226ab79aefcaa9d7f \ No newline at end of file diff --git a/plugins/repository-gcs/licenses/gax-1.30.0.jar.sha1 b/plugins/repository-gcs/licenses/gax-1.30.0.jar.sha1 deleted file mode 100644 index d6d2bb20ed840..0000000000000 --- a/plugins/repository-gcs/licenses/gax-1.30.0.jar.sha1 +++ /dev/null @@ -1 +0,0 @@ -58fa2feb11b092be0a6ebe705a28736f12374230 \ No newline at end of file diff --git a/plugins/repository-gcs/licenses/gax-1.45.0.jar.sha1 b/plugins/repository-gcs/licenses/gax-1.45.0.jar.sha1 new file mode 100644 index 0000000000000..8c0cbc659aa60 --- /dev/null +++ b/plugins/repository-gcs/licenses/gax-1.45.0.jar.sha1 @@ -0,0 +1 @@ +2ade3e3502f9d14e3731347a82ea02372094211f \ No newline at end of file diff --git a/plugins/repository-gcs/licenses/gax-httpjson-0.47.0.jar.sha1 b/plugins/repository-gcs/licenses/gax-httpjson-0.47.0.jar.sha1 deleted file mode 100644 index fdc722d1520d6..0000000000000 --- a/plugins/repository-gcs/licenses/gax-httpjson-0.47.0.jar.sha1 +++ /dev/null @@ -1 +0,0 @@ -d096f3142eb3adbf877588d1044895d148d9efcb \ No newline at end of file diff --git a/plugins/repository-gcs/licenses/gax-httpjson-0.62.0.jar.sha1 b/plugins/repository-gcs/licenses/gax-httpjson-0.62.0.jar.sha1 new file mode 100644 index 0000000000000..161ca85ccfc0c --- /dev/null +++ b/plugins/repository-gcs/licenses/gax-httpjson-0.62.0.jar.sha1 @@ -0,0 +1 @@ +05a1a4736acd1c4f30304be953532be6aecdc2c9 \ No newline at end of file diff --git a/plugins/repository-gcs/licenses/google-api-client-1.24.1.jar.sha1 b/plugins/repository-gcs/licenses/google-api-client-1.24.1.jar.sha1 deleted file mode 100644 index 27dafe58a0182..0000000000000 --- a/plugins/repository-gcs/licenses/google-api-client-1.24.1.jar.sha1 +++ /dev/null @@ -1 +0,0 @@ -37de23fb9b8b077de4ecec3192d98e752b0e5d72 \ No newline at end of file diff --git a/plugins/repository-gcs/licenses/google-api-client-1.28.0.jar.sha1 b/plugins/repository-gcs/licenses/google-api-client-1.28.0.jar.sha1 new file mode 100644 index 0000000000000..c9b0efd4cdcbf --- /dev/null +++ b/plugins/repository-gcs/licenses/google-api-client-1.28.0.jar.sha1 @@ -0,0 +1 @@ +8fe155d766ed22480939e3a9db428151e0264d9e \ No newline at end of file diff --git a/plugins/repository-gcs/licenses/google-api-services-storage-v1-rev135-1.24.1.jar.sha1 b/plugins/repository-gcs/licenses/google-api-services-storage-v1-rev135-1.24.1.jar.sha1 deleted file mode 100644 index e3042ee6ea07e..0000000000000 --- a/plugins/repository-gcs/licenses/google-api-services-storage-v1-rev135-1.24.1.jar.sha1 +++ /dev/null @@ -1 +0,0 @@ -28d3d391dfc7e7e7951760708ad2f48cecacf38f \ No newline at end of file diff --git a/plugins/repository-gcs/licenses/google-api-services-storage-v1-rev20190426-1.28.0.jar.sha1 b/plugins/repository-gcs/licenses/google-api-services-storage-v1-rev20190426-1.28.0.jar.sha1 new file mode 100644 index 0000000000000..84c68e6ffd528 --- /dev/null +++ b/plugins/repository-gcs/licenses/google-api-services-storage-v1-rev20190426-1.28.0.jar.sha1 @@ -0,0 +1 @@ +34dd008901f382507a572f5242d0e5c5ea4ad713 \ No newline at end of file diff --git a/plugins/repository-gcs/licenses/google-auth-library-credentials-0.10.0.jar.sha1 b/plugins/repository-gcs/licenses/google-auth-library-credentials-0.10.0.jar.sha1 deleted file mode 100644 index c8258d69326b8..0000000000000 --- a/plugins/repository-gcs/licenses/google-auth-library-credentials-0.10.0.jar.sha1 +++ /dev/null @@ -1 +0,0 @@ -f981288bd84fe6d140ed70d1d8dbe994a64fa3cc \ No newline at end of file diff --git a/plugins/repository-gcs/licenses/google-auth-library-credentials-0.16.1.jar.sha1 b/plugins/repository-gcs/licenses/google-auth-library-credentials-0.16.1.jar.sha1 new file mode 100644 index 0000000000000..6527ebec6da50 --- /dev/null +++ b/plugins/repository-gcs/licenses/google-auth-library-credentials-0.16.1.jar.sha1 @@ -0,0 +1 @@ +9a15387cc0438ac3f3e625b6050cf39f4e981e13 \ No newline at end of file diff --git a/plugins/repository-gcs/licenses/google-auth-library-oauth2-http-0.10.0.jar.sha1 b/plugins/repository-gcs/licenses/google-auth-library-oauth2-http-0.10.0.jar.sha1 deleted file mode 100644 index f55ef7c9c2150..0000000000000 --- a/plugins/repository-gcs/licenses/google-auth-library-oauth2-http-0.10.0.jar.sha1 +++ /dev/null @@ -1 +0,0 @@ -c079a62086121973a23d90f54e2b8c13050fa39d \ No newline at end of file diff --git a/plugins/repository-gcs/licenses/google-auth-library-oauth2-http-0.16.1.jar.sha1 b/plugins/repository-gcs/licenses/google-auth-library-oauth2-http-0.16.1.jar.sha1 new file mode 100644 index 0000000000000..e6467acf8f595 --- /dev/null +++ b/plugins/repository-gcs/licenses/google-auth-library-oauth2-http-0.16.1.jar.sha1 @@ -0,0 +1 @@ +3407d434678faef3439a7012efa336e751ddc623 \ No newline at end of file diff --git a/plugins/repository-gcs/licenses/google-cloud-core-1.59.0.jar.sha1 b/plugins/repository-gcs/licenses/google-cloud-core-1.59.0.jar.sha1 deleted file mode 100644 index 20e3b0c782dfd..0000000000000 --- a/plugins/repository-gcs/licenses/google-cloud-core-1.59.0.jar.sha1 +++ /dev/null @@ -1 +0,0 @@ -f2d0c00917660b244da514f82cba96f7697f2c82 \ No newline at end of file diff --git a/plugins/repository-gcs/licenses/google-cloud-core-1.77.0.jar.sha1 b/plugins/repository-gcs/licenses/google-cloud-core-1.77.0.jar.sha1 new file mode 100644 index 0000000000000..d16477c5bd68f --- /dev/null +++ b/plugins/repository-gcs/licenses/google-cloud-core-1.77.0.jar.sha1 @@ -0,0 +1 @@ +7cd83a789fde368a999c1793c6297e7b4e56b2ac \ No newline at end of file diff --git a/plugins/repository-gcs/licenses/google-cloud-core-http-1.59.0.jar.sha1 b/plugins/repository-gcs/licenses/google-cloud-core-http-1.59.0.jar.sha1 deleted file mode 100644 index ab4c7b7dca9c5..0000000000000 --- a/plugins/repository-gcs/licenses/google-cloud-core-http-1.59.0.jar.sha1 +++ /dev/null @@ -1 +0,0 @@ -e2a094ec3e8acb15b99f2d4bd42ac9bbc7d9f33e \ No newline at end of file diff --git a/plugins/repository-gcs/licenses/google-cloud-core-http-1.77.0.jar.sha1 b/plugins/repository-gcs/licenses/google-cloud-core-http-1.77.0.jar.sha1 new file mode 100644 index 0000000000000..7efc3167589eb --- /dev/null +++ b/plugins/repository-gcs/licenses/google-cloud-core-http-1.77.0.jar.sha1 @@ -0,0 +1 @@ +e16acbc935a7762ba9b220860ae45c2c67d17d8c \ No newline at end of file diff --git a/plugins/repository-gcs/licenses/google-cloud-storage-1.59.0.jar.sha1 b/plugins/repository-gcs/licenses/google-cloud-storage-1.59.0.jar.sha1 deleted file mode 100644 index 0f5a8633bd028..0000000000000 --- a/plugins/repository-gcs/licenses/google-cloud-storage-1.59.0.jar.sha1 +++ /dev/null @@ -1 +0,0 @@ -23dc0edf739ff1fb5a91fbddd7bd1f2cbfe0f827 \ No newline at end of file diff --git a/plugins/repository-gcs/licenses/google-cloud-storage-1.77.0.jar.sha1 b/plugins/repository-gcs/licenses/google-cloud-storage-1.77.0.jar.sha1 new file mode 100644 index 0000000000000..de15e888520d9 --- /dev/null +++ b/plugins/repository-gcs/licenses/google-cloud-storage-1.77.0.jar.sha1 @@ -0,0 +1 @@ +e368e1a8bbc0d0a4354f4e5eec076f38f6966050 \ No newline at end of file diff --git a/plugins/repository-gcs/licenses/google-http-client-1.24.1.jar.sha1 b/plugins/repository-gcs/licenses/google-http-client-1.24.1.jar.sha1 deleted file mode 100644 index 46b99f23e470a..0000000000000 --- a/plugins/repository-gcs/licenses/google-http-client-1.24.1.jar.sha1 +++ /dev/null @@ -1 +0,0 @@ -396eac8d3fb1332675f82b208f48a469d64f3b4a \ No newline at end of file diff --git a/plugins/repository-gcs/licenses/google-http-client-1.30.1.jar.sha1 b/plugins/repository-gcs/licenses/google-http-client-1.30.1.jar.sha1 new file mode 100644 index 0000000000000..85323c108f99e --- /dev/null +++ b/plugins/repository-gcs/licenses/google-http-client-1.30.1.jar.sha1 @@ -0,0 +1 @@ +573aacbda8feb0d43f7056291fbce5496f42a6aa \ No newline at end of file diff --git a/plugins/repository-gcs/licenses/google-http-client-appengine-1.24.1.jar.sha1 b/plugins/repository-gcs/licenses/google-http-client-appengine-1.24.1.jar.sha1 deleted file mode 100644 index e39f63fe33ae3..0000000000000 --- a/plugins/repository-gcs/licenses/google-http-client-appengine-1.24.1.jar.sha1 +++ /dev/null @@ -1 +0,0 @@ -8535031ae10bf6a196e68f25e10c0d6382699cb6 \ No newline at end of file diff --git a/plugins/repository-gcs/licenses/google-http-client-appengine-1.29.2.jar.sha1 b/plugins/repository-gcs/licenses/google-http-client-appengine-1.29.2.jar.sha1 new file mode 100644 index 0000000000000..6973b62e9281c --- /dev/null +++ b/plugins/repository-gcs/licenses/google-http-client-appengine-1.29.2.jar.sha1 @@ -0,0 +1 @@ +d93f4d1d8c2496d75221e53173e4c503b7096a4d \ No newline at end of file diff --git a/plugins/repository-gcs/licenses/google-http-client-jackson-1.24.1.jar.sha1 b/plugins/repository-gcs/licenses/google-http-client-jackson-1.24.1.jar.sha1 deleted file mode 100644 index f6b9694abaa6c..0000000000000 --- a/plugins/repository-gcs/licenses/google-http-client-jackson-1.24.1.jar.sha1 +++ /dev/null @@ -1 +0,0 @@ -02c88e77c14effdda76f02a0eac968de74e0bd4e \ No newline at end of file diff --git a/plugins/repository-gcs/licenses/google-http-client-jackson2-1.24.1.jar.sha1 b/plugins/repository-gcs/licenses/google-http-client-jackson2-1.24.1.jar.sha1 deleted file mode 100644 index 634b7d9198c8e..0000000000000 --- a/plugins/repository-gcs/licenses/google-http-client-jackson2-1.24.1.jar.sha1 +++ /dev/null @@ -1 +0,0 @@ -2ad1dffd8a450055e68d8004fe003033b751d761 \ No newline at end of file diff --git a/plugins/repository-gcs/licenses/google-http-client-jackson2-1.29.2.jar.sha1 b/plugins/repository-gcs/licenses/google-http-client-jackson2-1.29.2.jar.sha1 new file mode 100644 index 0000000000000..5a44d18d2aa1b --- /dev/null +++ b/plugins/repository-gcs/licenses/google-http-client-jackson2-1.29.2.jar.sha1 @@ -0,0 +1 @@ +d67891f5a438e1f339387a09628e0ab0af8b612a \ No newline at end of file diff --git a/plugins/repository-gcs/licenses/google-oauth-client-1.24.1.jar.sha1 b/plugins/repository-gcs/licenses/google-oauth-client-1.24.1.jar.sha1 deleted file mode 100644 index 2d89939674a51..0000000000000 --- a/plugins/repository-gcs/licenses/google-oauth-client-1.24.1.jar.sha1 +++ /dev/null @@ -1 +0,0 @@ -7b0e0218b96808868c23a7d0b40566a713931d9f \ No newline at end of file diff --git a/plugins/repository-gcs/licenses/google-oauth-client-1.28.0.jar.sha1 b/plugins/repository-gcs/licenses/google-oauth-client-1.28.0.jar.sha1 new file mode 100644 index 0000000000000..474df6e026570 --- /dev/null +++ b/plugins/repository-gcs/licenses/google-oauth-client-1.28.0.jar.sha1 @@ -0,0 +1 @@ +9a9e5d0c33b663d6475c96ce79b2949545a113af \ No newline at end of file diff --git a/plugins/repository-gcs/licenses/guava-20.0.jar.sha1 b/plugins/repository-gcs/licenses/guava-20.0.jar.sha1 deleted file mode 100644 index 7b6ae09060b29..0000000000000 --- a/plugins/repository-gcs/licenses/guava-20.0.jar.sha1 +++ /dev/null @@ -1 +0,0 @@ -89507701249388e1ed5ddcf8c41f4ce1be7831ef \ No newline at end of file diff --git a/plugins/repository-gcs/licenses/guava-26.0-jre.jar.sha1 b/plugins/repository-gcs/licenses/guava-26.0-jre.jar.sha1 new file mode 100644 index 0000000000000..63d05007650ce --- /dev/null +++ b/plugins/repository-gcs/licenses/guava-26.0-jre.jar.sha1 @@ -0,0 +1 @@ +6a806eff209f36f635f943e16d97491f00f6bfab \ No newline at end of file diff --git a/plugins/repository-gcs/licenses/jackson-LICENSE b/plugins/repository-gcs/licenses/jackson-LICENSE deleted file mode 100644 index f5f45d26a49d6..0000000000000 --- a/plugins/repository-gcs/licenses/jackson-LICENSE +++ /dev/null @@ -1,8 +0,0 @@ -This copy of Jackson JSON processor streaming parser/generator is licensed under the -Apache (Software) License, version 2.0 ("the License"). -See the License for details about distribution rights, and the -specific rights regarding derivate works. - -You may obtain a copy of the License at: - -http://www.apache.org/licenses/LICENSE-2.0 diff --git a/plugins/repository-gcs/licenses/jackson-NOTICE b/plugins/repository-gcs/licenses/jackson-NOTICE deleted file mode 100644 index 4c976b7b4cc58..0000000000000 --- a/plugins/repository-gcs/licenses/jackson-NOTICE +++ /dev/null @@ -1,20 +0,0 @@ -# Jackson JSON processor - -Jackson is a high-performance, Free/Open Source JSON processing library. -It was originally written by Tatu Saloranta (tatu.saloranta@iki.fi), and has -been in development since 2007. -It is currently developed by a community of developers, as well as supported -commercially by FasterXML.com. - -## Licensing - -Jackson core and extension components may licensed under different licenses. -To find the details that apply to this artifact see the accompanying LICENSE file. -For more information, including possible other licensing options, contact -FasterXML.com (http://fasterxml.com). - -## Credits - -A list of contributors may be found from CREDITS file, which is included -in some artifacts (usually source distributions); but is always available -from the source code management (SCM) system project uses. diff --git a/plugins/repository-gcs/licenses/jackson-core-asl-1.9.11.jar.sha1 b/plugins/repository-gcs/licenses/jackson-core-asl-1.9.11.jar.sha1 deleted file mode 100644 index ed70030899aa0..0000000000000 --- a/plugins/repository-gcs/licenses/jackson-core-asl-1.9.11.jar.sha1 +++ /dev/null @@ -1 +0,0 @@ -e32303ef8bd18a5c9272780d49b81c95e05ddf43 \ No newline at end of file diff --git a/plugins/repository-gcs/licenses/log4j-1.2-api-2.11.1.jar.sha1 b/plugins/repository-gcs/licenses/log4j-1.2-api-2.11.1.jar.sha1 new file mode 100644 index 0000000000000..575d75dbda8c5 --- /dev/null +++ b/plugins/repository-gcs/licenses/log4j-1.2-api-2.11.1.jar.sha1 @@ -0,0 +1 @@ +3aba3398fe064a3eab4331f88161c7480e848418 \ No newline at end of file diff --git a/plugins/repository-gcs/licenses/log4j-LICENSE.txt b/plugins/repository-gcs/licenses/log4j-LICENSE.txt new file mode 100644 index 0000000000000..6279e5206de13 --- /dev/null +++ b/plugins/repository-gcs/licenses/log4j-LICENSE.txt @@ -0,0 +1,202 @@ + + Apache License + Version 2.0, January 2004 + http://www.apache.org/licenses/ + + TERMS AND CONDITIONS FOR USE, REPRODUCTION, AND DISTRIBUTION + + 1. Definitions. + + "License" shall mean the terms and conditions for use, reproduction, + and distribution as defined by Sections 1 through 9 of this document. + + "Licensor" shall mean the copyright owner or entity authorized by + the copyright owner that is granting the License. + + "Legal Entity" shall mean the union of the acting entity and all + other entities that control, are controlled by, or are under common + control with that entity. For the purposes of this definition, + "control" means (i) the power, direct or indirect, to cause the + direction or management of such entity, whether by contract or + otherwise, or (ii) ownership of fifty percent (50%) or more of the + outstanding shares, or (iii) beneficial ownership of such entity. + + "You" (or "Your") shall mean an individual or Legal Entity + exercising permissions granted by this License. + + "Source" form shall mean the preferred form for making modifications, + including but not limited to software source code, documentation + source, and configuration files. + + "Object" form shall mean any form resulting from mechanical + transformation or translation of a Source form, including but + not limited to compiled object code, generated documentation, + and conversions to other media types. + + "Work" shall mean the work of authorship, whether in Source or + Object form, made available under the License, as indicated by a + copyright notice that is included in or attached to the work + (an example is provided in the Appendix below). + + "Derivative Works" shall mean any work, whether in Source or Object + form, that is based on (or derived from) the Work and for which the + editorial revisions, annotations, elaborations, or other modifications + represent, as a whole, an original work of authorship. For the purposes + of this License, Derivative Works shall not include works that remain + separable from, or merely link (or bind by name) to the interfaces of, + the Work and Derivative Works thereof. + + "Contribution" shall mean any work of authorship, including + the original version of the Work and any modifications or additions + to that Work or Derivative Works thereof, that is intentionally + submitted to Licensor for inclusion in the Work by the copyright owner + or by an individual or Legal Entity authorized to submit on behalf of + the copyright owner. For the purposes of this definition, "submitted" + means any form of electronic, verbal, or written communication sent + to the Licensor or its representatives, including but not limited to + communication on electronic mailing lists, source code control systems, + and issue tracking systems that are managed by, or on behalf of, the + Licensor for the purpose of discussing and improving the Work, but + excluding communication that is conspicuously marked or otherwise + designated in writing by the copyright owner as "Not a Contribution." + + "Contributor" shall mean Licensor and any individual or Legal Entity + on behalf of whom a Contribution has been received by Licensor and + subsequently incorporated within the Work. + + 2. Grant of Copyright License. Subject to the terms and conditions of + this License, each Contributor hereby grants to You a perpetual, + worldwide, non-exclusive, no-charge, royalty-free, irrevocable + copyright license to reproduce, prepare Derivative Works of, + publicly display, publicly perform, sublicense, and distribute the + Work and such Derivative Works in Source or Object form. + + 3. Grant of Patent License. Subject to the terms and conditions of + this License, each Contributor hereby grants to You a perpetual, + worldwide, non-exclusive, no-charge, royalty-free, irrevocable + (except as stated in this section) patent license to make, have made, + use, offer to sell, sell, import, and otherwise transfer the Work, + where such license applies only to those patent claims licensable + by such Contributor that are necessarily infringed by their + Contribution(s) alone or by combination of their Contribution(s) + with the Work to which such Contribution(s) was submitted. If You + institute patent litigation against any entity (including a + cross-claim or counterclaim in a lawsuit) alleging that the Work + or a Contribution incorporated within the Work constitutes direct + or contributory patent infringement, then any patent licenses + granted to You under this License for that Work shall terminate + as of the date such litigation is filed. + + 4. Redistribution. You may reproduce and distribute copies of the + Work or Derivative Works thereof in any medium, with or without + modifications, and in Source or Object form, provided that You + meet the following conditions: + + (a) You must give any other recipients of the Work or + Derivative Works a copy of this License; and + + (b) You must cause any modified files to carry prominent notices + stating that You changed the files; and + + (c) You must retain, in the Source form of any Derivative Works + that You distribute, all copyright, patent, trademark, and + attribution notices from the Source form of the Work, + excluding those notices that do not pertain to any part of + the Derivative Works; and + + (d) If the Work includes a "NOTICE" text file as part of its + distribution, then any Derivative Works that You distribute must + include a readable copy of the attribution notices contained + within such NOTICE file, excluding those notices that do not + pertain to any part of the Derivative Works, in at least one + of the following places: within a NOTICE text file distributed + as part of the Derivative Works; within the Source form or + documentation, if provided along with the Derivative Works; or, + within a display generated by the Derivative Works, if and + wherever such third-party notices normally appear. The contents + of the NOTICE file are for informational purposes only and + do not modify the License. You may add Your own attribution + notices within Derivative Works that You distribute, alongside + or as an addendum to the NOTICE text from the Work, provided + that such additional attribution notices cannot be construed + as modifying the License. + + You may add Your own copyright statement to Your modifications and + may provide additional or different license terms and conditions + for use, reproduction, or distribution of Your modifications, or + for any such Derivative Works as a whole, provided Your use, + reproduction, and distribution of the Work otherwise complies with + the conditions stated in this License. + + 5. Submission of Contributions. Unless You explicitly state otherwise, + any Contribution intentionally submitted for inclusion in the Work + by You to the Licensor shall be under the terms and conditions of + this License, without any additional terms or conditions. + Notwithstanding the above, nothing herein shall supersede or modify + the terms of any separate license agreement you may have executed + with Licensor regarding such Contributions. + + 6. Trademarks. This License does not grant permission to use the trade + names, trademarks, service marks, or product names of the Licensor, + except as required for reasonable and customary use in describing the + origin of the Work and reproducing the content of the NOTICE file. + + 7. Disclaimer of Warranty. Unless required by applicable law or + agreed to in writing, Licensor provides the Work (and each + Contributor provides its Contributions) on an "AS IS" BASIS, + WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or + implied, including, without limitation, any warranties or conditions + of TITLE, NON-INFRINGEMENT, MERCHANTABILITY, or FITNESS FOR A + PARTICULAR PURPOSE. You are solely responsible for determining the + appropriateness of using or redistributing the Work and assume any + risks associated with Your exercise of permissions under this License. + + 8. Limitation of Liability. In no event and under no legal theory, + whether in tort (including negligence), contract, or otherwise, + unless required by applicable law (such as deliberate and grossly + negligent acts) or agreed to in writing, shall any Contributor be + liable to You for damages, including any direct, indirect, special, + incidental, or consequential damages of any character arising as a + result of this License or out of the use or inability to use the + Work (including but not limited to damages for loss of goodwill, + work stoppage, computer failure or malfunction, or any and all + other commercial damages or losses), even if such Contributor + has been advised of the possibility of such damages. + + 9. Accepting Warranty or Additional Liability. While redistributing + the Work or Derivative Works thereof, You may choose to offer, + and charge a fee for, acceptance of support, warranty, indemnity, + or other liability obligations and/or rights consistent with this + License. However, in accepting such obligations, You may act only + on Your own behalf and on Your sole responsibility, not on behalf + of any other Contributor, and only if You agree to indemnify, + defend, and hold each Contributor harmless for any liability + incurred by, or claims asserted against, such Contributor by reason + of your accepting any such warranty or additional liability. + + END OF TERMS AND CONDITIONS + + APPENDIX: How to apply the Apache License to your work. + + To apply the Apache License to your work, attach the following + boilerplate notice, with the fields enclosed by brackets "[]" + replaced with your own identifying information. (Don't include + the brackets!) The text should be enclosed in the appropriate + comment syntax for the file format. We also recommend that a + file or class name and description of purpose be included on the + same "printed page" as the copyright notice for easier + identification within third-party archives. + + Copyright 1999-2005 The Apache Software Foundation + + Licensed 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. diff --git a/plugins/repository-gcs/licenses/log4j-NOTICE.txt b/plugins/repository-gcs/licenses/log4j-NOTICE.txt new file mode 100644 index 0000000000000..0375732360047 --- /dev/null +++ b/plugins/repository-gcs/licenses/log4j-NOTICE.txt @@ -0,0 +1,5 @@ +Apache log4j +Copyright 2007 The Apache Software Foundation + +This product includes software developed at +The Apache Software Foundation (http://www.apache.org/). \ No newline at end of file diff --git a/plugins/repository-gcs/licenses/opencensus-api-0.15.0.jar.sha1 b/plugins/repository-gcs/licenses/opencensus-api-0.15.0.jar.sha1 deleted file mode 100644 index e200e2e24a7df..0000000000000 --- a/plugins/repository-gcs/licenses/opencensus-api-0.15.0.jar.sha1 +++ /dev/null @@ -1 +0,0 @@ -9a098392b287d7924660837f4eba0ce252013683 \ No newline at end of file diff --git a/plugins/repository-gcs/licenses/opencensus-api-0.18.0.jar.sha1 b/plugins/repository-gcs/licenses/opencensus-api-0.18.0.jar.sha1 new file mode 100644 index 0000000000000..8b95ab4e4c49c --- /dev/null +++ b/plugins/repository-gcs/licenses/opencensus-api-0.18.0.jar.sha1 @@ -0,0 +1 @@ +b89a8f8dfd1e1e0d68d83c82a855624814b19a6e \ No newline at end of file diff --git a/plugins/repository-gcs/licenses/opencensus-contrib-http-util-0.15.0.jar.sha1 b/plugins/repository-gcs/licenses/opencensus-contrib-http-util-0.15.0.jar.sha1 deleted file mode 100644 index b642e1ebebd59..0000000000000 --- a/plugins/repository-gcs/licenses/opencensus-contrib-http-util-0.15.0.jar.sha1 +++ /dev/null @@ -1 +0,0 @@ -d88690591669d9b5ba6d91d9eac7736e58ccf3da \ No newline at end of file diff --git a/plugins/repository-gcs/licenses/opencensus-contrib-http-util-0.18.0.jar.sha1 b/plugins/repository-gcs/licenses/opencensus-contrib-http-util-0.18.0.jar.sha1 new file mode 100644 index 0000000000000..1757e00591110 --- /dev/null +++ b/plugins/repository-gcs/licenses/opencensus-contrib-http-util-0.18.0.jar.sha1 @@ -0,0 +1 @@ +76a37e4a931d5801a9e25b0c0353e5f37c4d1e8e \ No newline at end of file diff --git a/plugins/repository-gcs/licenses/proto-google-common-protos-1.12.0.jar.sha1 b/plugins/repository-gcs/licenses/proto-google-common-protos-1.12.0.jar.sha1 deleted file mode 100644 index 47f3c178a68c6..0000000000000 --- a/plugins/repository-gcs/licenses/proto-google-common-protos-1.12.0.jar.sha1 +++ /dev/null @@ -1 +0,0 @@ -1140cc74df039deb044ed0e320035e674dc13062 \ No newline at end of file diff --git a/plugins/repository-gcs/licenses/proto-google-common-protos-1.16.0.jar.sha1 b/plugins/repository-gcs/licenses/proto-google-common-protos-1.16.0.jar.sha1 new file mode 100644 index 0000000000000..7762b7a3ebdc3 --- /dev/null +++ b/plugins/repository-gcs/licenses/proto-google-common-protos-1.16.0.jar.sha1 @@ -0,0 +1 @@ +2c5f022ea3b8e8df6a619c4cd8faf9af86022daa \ No newline at end of file diff --git a/plugins/repository-gcs/licenses/protobuf-java-3.6.0.jar.sha1 b/plugins/repository-gcs/licenses/protobuf-java-3.6.0.jar.sha1 deleted file mode 100644 index 050ebd44c9282..0000000000000 --- a/plugins/repository-gcs/licenses/protobuf-java-3.6.0.jar.sha1 +++ /dev/null @@ -1 +0,0 @@ -5333f7e422744d76840c08a106e28e519fbe3acd \ No newline at end of file diff --git a/plugins/repository-gcs/licenses/protobuf-java-3.7.1.jar.sha1 b/plugins/repository-gcs/licenses/protobuf-java-3.7.1.jar.sha1 new file mode 100644 index 0000000000000..51d30a4c18568 --- /dev/null +++ b/plugins/repository-gcs/licenses/protobuf-java-3.7.1.jar.sha1 @@ -0,0 +1 @@ +0bce1b6dc9e4531169542ab37a1c8641bcaa8afb \ No newline at end of file diff --git a/plugins/repository-gcs/licenses/protobuf-java-util-3.6.0.jar.sha1 b/plugins/repository-gcs/licenses/protobuf-java-util-3.6.0.jar.sha1 deleted file mode 100644 index cc85974499a65..0000000000000 --- a/plugins/repository-gcs/licenses/protobuf-java-util-3.6.0.jar.sha1 +++ /dev/null @@ -1 +0,0 @@ -3680d0042d4fe0b95ada844ff24da0698a7f0773 \ No newline at end of file diff --git a/plugins/repository-gcs/licenses/protobuf-java-util-3.7.1.jar.sha1 b/plugins/repository-gcs/licenses/protobuf-java-util-3.7.1.jar.sha1 new file mode 100644 index 0000000000000..d08f6be735b84 --- /dev/null +++ b/plugins/repository-gcs/licenses/protobuf-java-util-3.7.1.jar.sha1 @@ -0,0 +1 @@ +45dc95896cfad26397675fdabef7b032d6db4bb6 \ No newline at end of file diff --git a/plugins/repository-gcs/qa/google-cloud-storage/build.gradle b/plugins/repository-gcs/qa/google-cloud-storage/build.gradle index 4f201a812f2b2..d2411c7577692 100644 --- a/plugins/repository-gcs/qa/google-cloud-storage/build.gradle +++ b/plugins/repository-gcs/qa/google-cloud-storage/build.gradle @@ -17,12 +17,15 @@ * under the License. */ + import org.elasticsearch.gradle.MavenFilteringHack import org.elasticsearch.gradle.test.AntFixture import java.security.KeyPair import java.security.KeyPairGenerator +import static org.elasticsearch.gradle.PropertyNormalization.IGNORE_VALUE + apply plugin: 'elasticsearch.standalone-rest-test' apply plugin: 'elasticsearch.rest-test' @@ -90,13 +93,13 @@ integTest { testClusters.integTest { plugin file(project(':plugins:repository-gcs').bundlePlugin.archiveFile) - keystore 'gcs.client.integration_test.credentials_file', serviceAccountFile + keystore 'gcs.client.integration_test.credentials_file', serviceAccountFile, IGNORE_VALUE if (useFixture) { tasks.integTest.dependsOn createServiceAccountFile, googleCloudStorageFixture /* Use a closure on the string to delay evaluation until tests are executed */ - setting 'gcs.client.integration_test.endpoint', { "http://${googleCloudStorageFixture.addressAndPort}" } - setting 'gcs.client.integration_test.token_uri', { "http://${googleCloudStorageFixture.addressAndPort}/o/oauth2/token" } + setting 'gcs.client.integration_test.endpoint', { "http://${googleCloudStorageFixture.addressAndPort}" }, IGNORE_VALUE + setting 'gcs.client.integration_test.token_uri', { "http://${googleCloudStorageFixture.addressAndPort}/o/oauth2/token" }, IGNORE_VALUE } else { println "Using an external service to test the repository-gcs plugin" } diff --git a/plugins/repository-gcs/qa/google-cloud-storage/src/test/resources/rest-api-spec/test/repository_gcs/10_repository.yml b/plugins/repository-gcs/qa/google-cloud-storage/src/test/resources/rest-api-spec/test/repository_gcs/10_repository.yml index ac649229001db..553b6a3e14e50 100644 --- a/plugins/repository-gcs/qa/google-cloud-storage/src/test/resources/rest-api-spec/test/repository_gcs/10_repository.yml +++ b/plugins/repository-gcs/qa/google-cloud-storage/src/test/resources/rest-api-spec/test/repository_gcs/10_repository.yml @@ -28,6 +28,9 @@ setup: --- "Snapshot/Restore with repository-gcs": + - skip: + version: " - 7.9.99" + reason: "8.0 changes get snapshots response format" # Get repository - do: @@ -134,9 +137,9 @@ setup: repository: repository snapshot: snapshot-one,snapshot-two - - is_true: snapshots - - match: { snapshots.0.state : SUCCESS } - - match: { snapshots.1.state : SUCCESS } + - is_true: responses.0.snapshots + - match: { responses.0.snapshots.0.state : SUCCESS } + - match: { responses.0.snapshots.1.state : SUCCESS } # Delete the index - do: @@ -213,13 +216,18 @@ setup: --- "Get a non existing snapshot": + - skip: + version: " - 7.9.99" + reason: "8.0 changes get snapshots response format" - do: - catch: /snapshot_missing_exception/ snapshot.get: repository: repository snapshot: missing + - is_true: responses.0.error + - match: { responses.0.error.type: snapshot_missing_exception } + --- "Delete a non existing snapshot": diff --git a/plugins/repository-gcs/src/main/java/org/elasticsearch/repositories/gcs/GoogleCloudStorageBlobContainer.java b/plugins/repository-gcs/src/main/java/org/elasticsearch/repositories/gcs/GoogleCloudStorageBlobContainer.java index fb81a5c90039f..75d4ad92fbf8e 100644 --- a/plugins/repository-gcs/src/main/java/org/elasticsearch/repositories/gcs/GoogleCloudStorageBlobContainer.java +++ b/plugins/repository-gcs/src/main/java/org/elasticsearch/repositories/gcs/GoogleCloudStorageBlobContainer.java @@ -19,6 +19,7 @@ package org.elasticsearch.repositories.gcs; +import org.elasticsearch.common.blobstore.BlobContainer; import org.elasticsearch.common.blobstore.BlobMetaData; import org.elasticsearch.common.blobstore.BlobPath; import org.elasticsearch.common.blobstore.BlobStoreException; @@ -55,6 +56,11 @@ public Map listBlobs() throws IOException { return blobStore.listBlobs(path); } + @Override + public Map children() throws IOException { + return blobStore.listChildren(path()); + } + @Override public Map listBlobsByPrefix(String prefix) throws IOException { return blobStore.listBlobsByPrefix(path, prefix); @@ -80,6 +86,11 @@ public void deleteBlob(String blobName) throws IOException { blobStore.deleteBlob(buildKey(blobName)); } + @Override + public void delete() throws IOException { + blobStore.deleteDirectory(path().buildAsString()); + } + @Override public void deleteBlobsIgnoringIfNotExists(List blobNames) throws IOException { blobStore.deleteBlobsIgnoringIfNotExists(blobNames.stream().map(this::buildKey).collect(Collectors.toList())); diff --git a/plugins/repository-gcs/src/main/java/org/elasticsearch/repositories/gcs/GoogleCloudStorageBlobStore.java b/plugins/repository-gcs/src/main/java/org/elasticsearch/repositories/gcs/GoogleCloudStorageBlobStore.java index 4214e5d408210..2477fd2962bde 100644 --- a/plugins/repository-gcs/src/main/java/org/elasticsearch/repositories/gcs/GoogleCloudStorageBlobStore.java +++ b/plugins/repository-gcs/src/main/java/org/elasticsearch/repositories/gcs/GoogleCloudStorageBlobStore.java @@ -19,6 +19,7 @@ package org.elasticsearch.repositories.gcs; +import com.google.api.gax.paging.Page; import com.google.cloud.BatchResult; import com.google.cloud.ReadChannel; import com.google.cloud.WriteChannel; @@ -30,7 +31,6 @@ import com.google.cloud.storage.Storage.BlobListOption; import com.google.cloud.storage.StorageBatch; import com.google.cloud.storage.StorageException; - import org.elasticsearch.common.SuppressForbidden; import org.elasticsearch.common.blobstore.BlobContainer; import org.elasticsearch.common.blobstore.BlobMetaData; @@ -50,11 +50,11 @@ import java.nio.channels.WritableByteChannel; import java.nio.file.FileAlreadyExistsException; import java.nio.file.NoSuchFileException; -import java.util.Map; import java.util.ArrayList; import java.util.Collection; import java.util.Collections; import java.util.List; +import java.util.Map; import java.util.concurrent.atomic.AtomicReference; import java.util.stream.Collectors; @@ -132,13 +132,34 @@ Map listBlobs(String path) throws IOException { Map listBlobsByPrefix(String path, String prefix) throws IOException { final String pathPrefix = buildKey(path, prefix); final MapBuilder mapBuilder = MapBuilder.newMapBuilder(); - SocketAccess.doPrivilegedVoidIOException(() -> { - client().get(bucketName).list(BlobListOption.prefix(pathPrefix)).iterateAll().forEach(blob -> { - assert blob.getName().startsWith(path); - final String suffixName = blob.getName().substring(path.length()); - mapBuilder.put(suffixName, new PlainBlobMetaData(suffixName, blob.getSize())); - }); - }); + SocketAccess.doPrivilegedVoidIOException( + () -> client().get(bucketName).list(BlobListOption.currentDirectory(), BlobListOption.prefix(pathPrefix)).iterateAll().forEach( + blob -> { + assert blob.getName().startsWith(path); + if (blob.isDirectory() == false) { + final String suffixName = blob.getName().substring(path.length()); + mapBuilder.put(suffixName, new PlainBlobMetaData(suffixName, blob.getSize())); + } + })); + return mapBuilder.immutableMap(); + } + + Map listChildren(BlobPath path) throws IOException { + final String pathStr = path.buildAsString(); + final MapBuilder mapBuilder = MapBuilder.newMapBuilder(); + SocketAccess.doPrivilegedVoidIOException + (() -> client().get(bucketName).list(BlobListOption.currentDirectory(), BlobListOption.prefix(pathStr)).iterateAll().forEach( + blob -> { + if (blob.isDirectory()) { + assert blob.getName().startsWith(pathStr); + assert blob.getName().endsWith("/"); + // Strip path prefix and trailing slash + final String suffixName = blob.getName().substring(pathStr.length(), blob.getName().length() - 1); + if (suffixName.isEmpty() == false) { + mapBuilder.put(suffixName, new GoogleCloudStorageBlobContainer(path.add(suffixName), this)); + } + } + })); return mapBuilder.immutableMap(); } @@ -286,6 +307,23 @@ void deleteBlob(String blobName) throws IOException { } } + /** + * Deletes the given path and all its children. + * + * @param pathStr Name of path to delete + */ + void deleteDirectory(String pathStr) throws IOException { + SocketAccess.doPrivilegedVoidIOException(() -> { + Page page = client().get(bucketName).list(BlobListOption.prefix(pathStr)); + do { + final Collection blobsToDelete = new ArrayList<>(); + page.getValues().forEach(b -> blobsToDelete.add(b.getName())); + deleteBlobsIgnoringIfNotExists(blobsToDelete); + page = page.getNextPage(); + } while (page != null); + }); + } + /** * Deletes multiple blobs from the specific bucket using a batch request * @@ -295,11 +333,6 @@ void deleteBlobsIgnoringIfNotExists(Collection blobNames) throws IOExcep if (blobNames.isEmpty()) { return; } - // for a single op submit a simple delete instead of a batch of size 1 - if (blobNames.size() == 1) { - deleteBlob(blobNames.iterator().next()); - return; - } final List blobIdsToDelete = blobNames.stream().map(blob -> BlobId.of(bucketName, blob)).collect(Collectors.toList()); final List failedBlobs = Collections.synchronizedList(new ArrayList<>()); final StorageException e = SocketAccess.doPrivilegedIOException(() -> { diff --git a/plugins/repository-gcs/src/main/java/org/elasticsearch/repositories/gcs/GoogleCloudStorageRepository.java b/plugins/repository-gcs/src/main/java/org/elasticsearch/repositories/gcs/GoogleCloudStorageRepository.java index 804fafd5e855e..6382a537c4682 100644 --- a/plugins/repository-gcs/src/main/java/org/elasticsearch/repositories/gcs/GoogleCloudStorageRepository.java +++ b/plugins/repository-gcs/src/main/java/org/elasticsearch/repositories/gcs/GoogleCloudStorageRepository.java @@ -57,7 +57,6 @@ class GoogleCloudStorageRepository extends BlobStoreRepository { static final Setting CLIENT_NAME = new Setting<>("client", "default", Function.identity()); private final GoogleCloudStorageService storageService; - private final BlobPath basePath; private final ByteSizeValue chunkSize; private final String bucket; private final String clientName; @@ -65,24 +64,27 @@ class GoogleCloudStorageRepository extends BlobStoreRepository { GoogleCloudStorageRepository(RepositoryMetaData metadata, Environment environment, NamedXContentRegistry namedXContentRegistry, GoogleCloudStorageService storageService, ThreadPool threadPool) { - super(metadata, environment.settings(), namedXContentRegistry, threadPool); + super(metadata, environment.settings(), namedXContentRegistry, threadPool, buildBasePath(metadata)); this.storageService = storageService; + this.chunkSize = getSetting(CHUNK_SIZE, metadata); + this.bucket = getSetting(BUCKET, metadata); + this.clientName = CLIENT_NAME.get(metadata.settings()); + logger.debug( + "using bucket [{}], base_path [{}], chunk_size [{}], compress [{}]", bucket, basePath(), chunkSize, isCompress()); + } + + private static BlobPath buildBasePath(RepositoryMetaData metadata) { String basePath = BASE_PATH.get(metadata.settings()); if (Strings.hasLength(basePath)) { BlobPath path = new BlobPath(); for (String elem : basePath.split("/")) { path = path.add(elem); } - this.basePath = path; + return path; } else { - this.basePath = BlobPath.cleanPath(); + return BlobPath.cleanPath(); } - - this.chunkSize = getSetting(CHUNK_SIZE, metadata); - this.bucket = getSetting(BUCKET, metadata); - this.clientName = CLIENT_NAME.get(metadata.settings()); - logger.debug("using bucket [{}], base_path [{}], chunk_size [{}], compress [{}]", bucket, basePath, chunkSize, isCompress()); } @Override @@ -90,11 +92,6 @@ protected GoogleCloudStorageBlobStore createBlobStore() { return new GoogleCloudStorageBlobStore(bucket, clientName, storageService); } - @Override - protected BlobPath basePath() { - return basePath; - } - @Override protected ByteSizeValue chunkSize() { return chunkSize; diff --git a/plugins/repository-gcs/src/main/java/org/elasticsearch/repositories/gcs/GoogleCloudStorageService.java b/plugins/repository-gcs/src/main/java/org/elasticsearch/repositories/gcs/GoogleCloudStorageService.java index f30b8b47d82b8..970296eff95f4 100644 --- a/plugins/repository-gcs/src/main/java/org/elasticsearch/repositories/gcs/GoogleCloudStorageService.java +++ b/plugins/repository-gcs/src/main/java/org/elasticsearch/repositories/gcs/GoogleCloudStorageService.java @@ -21,7 +21,6 @@ import com.google.api.client.googleapis.GoogleUtils; import com.google.api.client.http.HttpTransport; -import com.google.api.client.http.javanet.DefaultConnectionFactory; import com.google.api.client.http.javanet.NetHttpTransport; import com.google.auth.oauth2.ServiceAccountCredentials; import com.google.cloud.http.HttpTransportOptions; @@ -36,10 +35,7 @@ import org.elasticsearch.common.util.LazyInitializable; import java.io.IOException; -import java.net.HttpURLConnection; import java.net.URI; -import java.net.URISyntaxException; -import java.net.URL; import java.util.Map; import java.util.concurrent.atomic.AtomicReference; import java.util.stream.Collectors; @@ -106,10 +102,16 @@ public Storage client(final String clientName) throws IOException { * @return a new client storage instance that can be used to manage objects * (blobs) */ - private Storage createClient(final String clientName, final GoogleCloudStorageClientSettings clientSettings) throws IOException { + private static Storage createClient(String clientName, GoogleCloudStorageClientSettings clientSettings) throws IOException { logger.debug(() -> new ParameterizedMessage("creating GCS client with client_name [{}], endpoint [{}]", clientName, clientSettings.getHost())); - final HttpTransport httpTransport = SocketAccess.doPrivilegedIOException(() -> createHttpTransport(clientSettings.getHost())); + final HttpTransport httpTransport = SocketAccess.doPrivilegedIOException(() -> { + final NetHttpTransport.Builder builder = new NetHttpTransport.Builder(); + // requires java.lang.RuntimePermission "setFactory" + // Pin the TLS trust certificates. + builder.trustCertificates(GoogleUtils.getCertificateTrustStore()); + return builder.build(); + }); final HttpTransportOptions httpTransportOptions = HttpTransportOptions.newBuilder() .setConnectTimeout(toTimeout(clientSettings.getConnectTimeout())) .setReadTimeout(toTimeout(clientSettings.getReadTimeout())) @@ -147,54 +149,6 @@ private Storage createClient(final String clientName, final GoogleCloudStorageCl return storageOptionsBuilder.build().getService(); } - /** - * Pins the TLS trust certificates and, more importantly, overrides connection - * URLs in the case of a custom endpoint setting because some connections don't - * fully honor this setting (bugs in the SDK). The default connection factory - * opens a new connection for each request. This is required for the storage - * instance to be thread-safe. - **/ - private static HttpTransport createHttpTransport(final String endpoint) throws Exception { - final NetHttpTransport.Builder builder = new NetHttpTransport.Builder(); - // requires java.lang.RuntimePermission "setFactory" - builder.trustCertificates(GoogleUtils.getCertificateTrustStore()); - if (Strings.hasLength(endpoint)) { - final URL endpointUrl = URI.create(endpoint).toURL(); - // it is crucial to open a connection for each URL (see {@code - // DefaultConnectionFactory#openConnection}) instead of reusing connections, - // because the storage instance has to be thread-safe as it is cached. - builder.setConnectionFactory(new DefaultConnectionFactory() { - @Override - public HttpURLConnection openConnection(final URL originalUrl) throws IOException { - // test if the URL is built correctly, ie following the `host` setting - if (originalUrl.getHost().equals(endpointUrl.getHost()) && originalUrl.getPort() == endpointUrl.getPort() - && originalUrl.getProtocol().equals(endpointUrl.getProtocol())) { - return super.openConnection(originalUrl); - } - // override connection URLs because some don't follow the config. See - // https://github.com/GoogleCloudPlatform/google-cloud-java/issues/3254 and - // https://github.com/GoogleCloudPlatform/google-cloud-java/issues/3255 - URI originalUri; - try { - originalUri = originalUrl.toURI(); - } catch (final URISyntaxException e) { - throw new RuntimeException(e); - } - String overridePath = "/"; - if (originalUri.getRawPath() != null) { - overridePath = originalUri.getRawPath(); - } - if (originalUri.getRawQuery() != null) { - overridePath += "?" + originalUri.getRawQuery(); - } - return super.openConnection( - new URL(endpointUrl.getProtocol(), endpointUrl.getHost(), endpointUrl.getPort(), overridePath)); - } - }); - } - return builder.build(); - } - /** * Converts timeout values from the settings to a timeout value for the Google * Cloud SDK diff --git a/plugins/repository-gcs/src/test/java/org/elasticsearch/repositories/gcs/GoogleCloudStorageThirdPartyTests.java b/plugins/repository-gcs/src/test/java/org/elasticsearch/repositories/gcs/GoogleCloudStorageThirdPartyTests.java new file mode 100644 index 0000000000000..06eb63ddd22f0 --- /dev/null +++ b/plugins/repository-gcs/src/test/java/org/elasticsearch/repositories/gcs/GoogleCloudStorageThirdPartyTests.java @@ -0,0 +1,64 @@ +/* + * 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.repositories.gcs; + +import org.elasticsearch.action.support.master.AcknowledgedResponse; +import org.elasticsearch.common.settings.MockSecureSettings; +import org.elasticsearch.common.settings.SecureSettings; +import org.elasticsearch.common.settings.Settings; +import org.elasticsearch.plugins.Plugin; +import org.elasticsearch.repositories.AbstractThirdPartyRepositoryTestCase; + +import java.util.Base64; +import java.util.Collection; + +import static org.hamcrest.Matchers.blankOrNullString; +import static org.hamcrest.Matchers.equalTo; +import static org.hamcrest.Matchers.not; + +public class GoogleCloudStorageThirdPartyTests extends AbstractThirdPartyRepositoryTestCase { + + @Override + protected Collection> getPlugins() { + return pluginList(GoogleCloudStoragePlugin.class); + } + + @Override + protected SecureSettings credentials() { + assertThat(System.getProperty("test.google.account"), not(blankOrNullString())); + assertThat(System.getProperty("test.google.bucket"), not(blankOrNullString())); + + MockSecureSettings secureSettings = new MockSecureSettings(); + secureSettings.setFile("gcs.client.default.credentials_file", + Base64.getDecoder().decode(System.getProperty("test.google.account"))); + return secureSettings; + } + + @Override + protected void createRepository(final String repoName) { + AcknowledgedResponse putRepositoryResponse = client().admin().cluster().preparePutRepository("test-repo") + .setType("gcs") + .setSettings(Settings.builder() + .put("bucket", System.getProperty("test.google.bucket")) + .put("base_path", System.getProperty("test.google.base", "/")) + ).get(); + assertThat(putRepositoryResponse.isAcknowledged(), equalTo(true)); + } +} diff --git a/plugins/repository-gcs/src/test/java/org/elasticsearch/repositories/gcs/MockStorage.java b/plugins/repository-gcs/src/test/java/org/elasticsearch/repositories/gcs/MockStorage.java index eddf2a9f78082..ca6ca60e41e95 100644 --- a/plugins/repository-gcs/src/test/java/org/elasticsearch/repositories/gcs/MockStorage.java +++ b/plugins/repository-gcs/src/test/java/org/elasticsearch/repositories/gcs/MockStorage.java @@ -281,6 +281,11 @@ public void close() { return null; } + @Override + public WriteChannel writer(URL signedURL) { + return null; + } + // Everything below this line is not implemented. @Override @@ -288,6 +293,11 @@ public CopyWriter copy(CopyRequest copyRequest) { return null; } + @Override + public Blob create(BlobInfo blobInfo, byte[] content, int offset, int length, BlobTargetOption... options) { + return null; + } + @Override public Bucket create(BucketInfo bucketInfo, BucketTargetOption... options) { return null; diff --git a/plugins/repository-hdfs/build.gradle b/plugins/repository-hdfs/build.gradle index 3d6dcd29d6d19..3060867b01b08 100644 --- a/plugins/repository-hdfs/build.gradle +++ b/plugins/repository-hdfs/build.gradle @@ -24,6 +24,8 @@ import java.nio.file.Files import java.nio.file.Path import java.nio.file.Paths +import static org.elasticsearch.gradle.PropertyNormalization.IGNORE_VALUE + apply plugin: 'elasticsearch.test.fixtures' esplugin { @@ -52,6 +54,7 @@ dependencies { compile 'com.google.guava:guava:11.0.2' compile 'com.google.protobuf:protobuf-java:2.5.0' compile 'commons-logging:commons-logging:1.1.3' + compile "org.apache.logging.log4j:log4j-1.2-api:${versions.log4j}" compile 'commons-cli:commons-cli:1.2' compile "commons-codec:commons-codec:${versions.commonscodec}" compile 'commons-collections:commons-collections:3.2.2' @@ -65,7 +68,14 @@ dependencies { hdfsFixture project(':test:fixtures:hdfs-fixture') // Set the keytab files in the classpath so that we can access them from test code without the security manager // freaking out. - testRuntime fileTree(dir: project(':test:fixtures:krb5kdc-fixture').ext.krb5Keytabs("hdfs","hdfs_hdfs.build.elastic.co.keytab").parent, include: ['*.keytab']) + testRuntime files(project(':test:fixtures:krb5kdc-fixture').ext.krb5Keytabs("hdfs","hdfs_hdfs.build.elastic.co.keytab").parent) +} + +normalization { + runtimeClasspath { + // ignore generated keytab files for the purposes of build avoidance + ignore '*.keytab' + } } dependencyLicenses { @@ -86,7 +96,7 @@ for (String fixtureName : ['hdfsFixture', 'haHdfsFixture', 'secureHdfsFixture', // the hdfs.MiniHDFS fixture writes the ports file when // it's ready, so we can just wait for the file to exist return fixture.portsFile.exists() - } + } final List miniHDFSArgs = [] @@ -115,20 +125,29 @@ for (String fixtureName : ['hdfsFixture', 'haHdfsFixture', 'secureHdfsFixture', } } +Set disabledIntegTestTaskNames = ['integTestSecure', 'integTestSecureHa'] + for (String integTestTaskName : ['integTestHa', 'integTestSecure', 'integTestSecureHa']) { task "${integTestTaskName}"(type: RestIntegTestTask) { description = "Runs rest tests against an elasticsearch cluster with HDFS." dependsOn(project.bundlePlugin) + + if (disabledIntegTestTaskNames.contains(integTestTaskName)) { + enabled = false; + } + runner { if (integTestTaskName.contains("Secure")) { - dependsOn secureHdfsFixture - systemProperty "test.krb5.principal.es", "elasticsearch@${realm}" - systemProperty "test.krb5.principal.hdfs", "hdfs/hdfs.build.elastic.co@${realm}" - jvmArgs "-Djava.security.krb5.conf=${krb5conf}" - systemProperty ( - "test.krb5.keytab.hdfs", - project(':test:fixtures:krb5kdc-fixture').ext.krb5Keytabs("hdfs","hdfs_hdfs.build.elastic.co.keytab") - ) + if (disabledIntegTestTaskNames.contains(integTestTaskName) == false) { + dependsOn secureHdfsFixture + nonInputProperties.systemProperty "test.krb5.principal.es", "elasticsearch@${realm}" + nonInputProperties.systemProperty "test.krb5.principal.hdfs", "hdfs/hdfs.build.elastic.co@${realm}" + jvmArgs "-Djava.security.krb5.conf=${krb5conf}" + nonInputProperties.systemProperty ( + "test.krb5.keytab.hdfs", + project(':test:fixtures:krb5kdc-fixture').ext.krb5Keytabs("hdfs","hdfs_hdfs.build.elastic.co.keytab") + ) + } } } } @@ -139,7 +158,7 @@ for (String integTestTaskName : ['integTestHa', 'integTestSecure', 'integTestSec systemProperty "java.security.krb5.conf", krb5conf extraConfigFile( "repository-hdfs/krb5.keytab", - file("${project(':test:fixtures:krb5kdc-fixture').ext.krb5Keytabs("hdfs", "elasticsearch.keytab")}") + file("${project(':test:fixtures:krb5kdc-fixture').ext.krb5Keytabs("hdfs", "elasticsearch.keytab")}"), IGNORE_VALUE ) } } diff --git a/plugins/repository-hdfs/licenses/log4j-1.2-api-2.11.1.jar.sha1 b/plugins/repository-hdfs/licenses/log4j-1.2-api-2.11.1.jar.sha1 new file mode 100644 index 0000000000000..575d75dbda8c5 --- /dev/null +++ b/plugins/repository-hdfs/licenses/log4j-1.2-api-2.11.1.jar.sha1 @@ -0,0 +1 @@ +3aba3398fe064a3eab4331f88161c7480e848418 \ No newline at end of file diff --git a/plugins/repository-hdfs/licenses/log4j-LICENSE.txt b/plugins/repository-hdfs/licenses/log4j-LICENSE.txt new file mode 100644 index 0000000000000..6279e5206de13 --- /dev/null +++ b/plugins/repository-hdfs/licenses/log4j-LICENSE.txt @@ -0,0 +1,202 @@ + + Apache License + Version 2.0, January 2004 + http://www.apache.org/licenses/ + + TERMS AND CONDITIONS FOR USE, REPRODUCTION, AND DISTRIBUTION + + 1. Definitions. + + "License" shall mean the terms and conditions for use, reproduction, + and distribution as defined by Sections 1 through 9 of this document. + + "Licensor" shall mean the copyright owner or entity authorized by + the copyright owner that is granting the License. + + "Legal Entity" shall mean the union of the acting entity and all + other entities that control, are controlled by, or are under common + control with that entity. For the purposes of this definition, + "control" means (i) the power, direct or indirect, to cause the + direction or management of such entity, whether by contract or + otherwise, or (ii) ownership of fifty percent (50%) or more of the + outstanding shares, or (iii) beneficial ownership of such entity. + + "You" (or "Your") shall mean an individual or Legal Entity + exercising permissions granted by this License. + + "Source" form shall mean the preferred form for making modifications, + including but not limited to software source code, documentation + source, and configuration files. + + "Object" form shall mean any form resulting from mechanical + transformation or translation of a Source form, including but + not limited to compiled object code, generated documentation, + and conversions to other media types. + + "Work" shall mean the work of authorship, whether in Source or + Object form, made available under the License, as indicated by a + copyright notice that is included in or attached to the work + (an example is provided in the Appendix below). + + "Derivative Works" shall mean any work, whether in Source or Object + form, that is based on (or derived from) the Work and for which the + editorial revisions, annotations, elaborations, or other modifications + represent, as a whole, an original work of authorship. For the purposes + of this License, Derivative Works shall not include works that remain + separable from, or merely link (or bind by name) to the interfaces of, + the Work and Derivative Works thereof. + + "Contribution" shall mean any work of authorship, including + the original version of the Work and any modifications or additions + to that Work or Derivative Works thereof, that is intentionally + submitted to Licensor for inclusion in the Work by the copyright owner + or by an individual or Legal Entity authorized to submit on behalf of + the copyright owner. For the purposes of this definition, "submitted" + means any form of electronic, verbal, or written communication sent + to the Licensor or its representatives, including but not limited to + communication on electronic mailing lists, source code control systems, + and issue tracking systems that are managed by, or on behalf of, the + Licensor for the purpose of discussing and improving the Work, but + excluding communication that is conspicuously marked or otherwise + designated in writing by the copyright owner as "Not a Contribution." + + "Contributor" shall mean Licensor and any individual or Legal Entity + on behalf of whom a Contribution has been received by Licensor and + subsequently incorporated within the Work. + + 2. Grant of Copyright License. Subject to the terms and conditions of + this License, each Contributor hereby grants to You a perpetual, + worldwide, non-exclusive, no-charge, royalty-free, irrevocable + copyright license to reproduce, prepare Derivative Works of, + publicly display, publicly perform, sublicense, and distribute the + Work and such Derivative Works in Source or Object form. + + 3. Grant of Patent License. Subject to the terms and conditions of + this License, each Contributor hereby grants to You a perpetual, + worldwide, non-exclusive, no-charge, royalty-free, irrevocable + (except as stated in this section) patent license to make, have made, + use, offer to sell, sell, import, and otherwise transfer the Work, + where such license applies only to those patent claims licensable + by such Contributor that are necessarily infringed by their + Contribution(s) alone or by combination of their Contribution(s) + with the Work to which such Contribution(s) was submitted. If You + institute patent litigation against any entity (including a + cross-claim or counterclaim in a lawsuit) alleging that the Work + or a Contribution incorporated within the Work constitutes direct + or contributory patent infringement, then any patent licenses + granted to You under this License for that Work shall terminate + as of the date such litigation is filed. + + 4. Redistribution. You may reproduce and distribute copies of the + Work or Derivative Works thereof in any medium, with or without + modifications, and in Source or Object form, provided that You + meet the following conditions: + + (a) You must give any other recipients of the Work or + Derivative Works a copy of this License; and + + (b) You must cause any modified files to carry prominent notices + stating that You changed the files; and + + (c) You must retain, in the Source form of any Derivative Works + that You distribute, all copyright, patent, trademark, and + attribution notices from the Source form of the Work, + excluding those notices that do not pertain to any part of + the Derivative Works; and + + (d) If the Work includes a "NOTICE" text file as part of its + distribution, then any Derivative Works that You distribute must + include a readable copy of the attribution notices contained + within such NOTICE file, excluding those notices that do not + pertain to any part of the Derivative Works, in at least one + of the following places: within a NOTICE text file distributed + as part of the Derivative Works; within the Source form or + documentation, if provided along with the Derivative Works; or, + within a display generated by the Derivative Works, if and + wherever such third-party notices normally appear. The contents + of the NOTICE file are for informational purposes only and + do not modify the License. You may add Your own attribution + notices within Derivative Works that You distribute, alongside + or as an addendum to the NOTICE text from the Work, provided + that such additional attribution notices cannot be construed + as modifying the License. + + You may add Your own copyright statement to Your modifications and + may provide additional or different license terms and conditions + for use, reproduction, or distribution of Your modifications, or + for any such Derivative Works as a whole, provided Your use, + reproduction, and distribution of the Work otherwise complies with + the conditions stated in this License. + + 5. Submission of Contributions. Unless You explicitly state otherwise, + any Contribution intentionally submitted for inclusion in the Work + by You to the Licensor shall be under the terms and conditions of + this License, without any additional terms or conditions. + Notwithstanding the above, nothing herein shall supersede or modify + the terms of any separate license agreement you may have executed + with Licensor regarding such Contributions. + + 6. Trademarks. This License does not grant permission to use the trade + names, trademarks, service marks, or product names of the Licensor, + except as required for reasonable and customary use in describing the + origin of the Work and reproducing the content of the NOTICE file. + + 7. Disclaimer of Warranty. Unless required by applicable law or + agreed to in writing, Licensor provides the Work (and each + Contributor provides its Contributions) on an "AS IS" BASIS, + WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or + implied, including, without limitation, any warranties or conditions + of TITLE, NON-INFRINGEMENT, MERCHANTABILITY, or FITNESS FOR A + PARTICULAR PURPOSE. You are solely responsible for determining the + appropriateness of using or redistributing the Work and assume any + risks associated with Your exercise of permissions under this License. + + 8. Limitation of Liability. In no event and under no legal theory, + whether in tort (including negligence), contract, or otherwise, + unless required by applicable law (such as deliberate and grossly + negligent acts) or agreed to in writing, shall any Contributor be + liable to You for damages, including any direct, indirect, special, + incidental, or consequential damages of any character arising as a + result of this License or out of the use or inability to use the + Work (including but not limited to damages for loss of goodwill, + work stoppage, computer failure or malfunction, or any and all + other commercial damages or losses), even if such Contributor + has been advised of the possibility of such damages. + + 9. Accepting Warranty or Additional Liability. While redistributing + the Work or Derivative Works thereof, You may choose to offer, + and charge a fee for, acceptance of support, warranty, indemnity, + or other liability obligations and/or rights consistent with this + License. However, in accepting such obligations, You may act only + on Your own behalf and on Your sole responsibility, not on behalf + of any other Contributor, and only if You agree to indemnify, + defend, and hold each Contributor harmless for any liability + incurred by, or claims asserted against, such Contributor by reason + of your accepting any such warranty or additional liability. + + END OF TERMS AND CONDITIONS + + APPENDIX: How to apply the Apache License to your work. + + To apply the Apache License to your work, attach the following + boilerplate notice, with the fields enclosed by brackets "[]" + replaced with your own identifying information. (Don't include + the brackets!) The text should be enclosed in the appropriate + comment syntax for the file format. We also recommend that a + file or class name and description of purpose be included on the + same "printed page" as the copyright notice for easier + identification within third-party archives. + + Copyright 1999-2005 The Apache Software Foundation + + Licensed 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. diff --git a/plugins/repository-hdfs/licenses/log4j-NOTICE.txt b/plugins/repository-hdfs/licenses/log4j-NOTICE.txt new file mode 100644 index 0000000000000..0375732360047 --- /dev/null +++ b/plugins/repository-hdfs/licenses/log4j-NOTICE.txt @@ -0,0 +1,5 @@ +Apache log4j +Copyright 2007 The Apache Software Foundation + +This product includes software developed at +The Apache Software Foundation (http://www.apache.org/). \ No newline at end of file diff --git a/plugins/repository-hdfs/src/main/java/org/elasticsearch/repositories/hdfs/HdfsBlobContainer.java b/plugins/repository-hdfs/src/main/java/org/elasticsearch/repositories/hdfs/HdfsBlobContainer.java index e9b45a9b52e70..b050645f9952c 100644 --- a/plugins/repository-hdfs/src/main/java/org/elasticsearch/repositories/hdfs/HdfsBlobContainer.java +++ b/plugins/repository-hdfs/src/main/java/org/elasticsearch/repositories/hdfs/HdfsBlobContainer.java @@ -25,6 +25,7 @@ import org.apache.hadoop.fs.Options.CreateOpts; import org.apache.hadoop.fs.Path; import org.elasticsearch.common.Nullable; +import org.elasticsearch.common.blobstore.BlobContainer; import org.elasticsearch.common.blobstore.BlobMetaData; import org.elasticsearch.common.blobstore.BlobPath; import org.elasticsearch.common.blobstore.fs.FsBlobContainer; @@ -77,6 +78,11 @@ public void deleteBlob(String blobName) throws IOException { } } + @Override + public void delete() throws IOException { + store.execute(fileContext -> fileContext.delete(path, true)); + } + @Override public InputStream readBlob(String blobName) throws IOException { // FSDataInputStream does buffering internally @@ -137,11 +143,13 @@ public void writeBlobAtomic(String blobName, InputStream inputStream, long blobS @Override public Map listBlobsByPrefix(@Nullable final String prefix) throws IOException { - FileStatus[] files = store.execute(fileContext -> (fileContext.util().listStatus(path, - path -> prefix == null || path.getName().startsWith(prefix)))); - Map map = new LinkedHashMap(); + FileStatus[] files = store.execute(fileContext -> fileContext.util().listStatus(path, + path -> prefix == null || path.getName().startsWith(prefix))); + Map map = new LinkedHashMap<>(); for (FileStatus file : files) { - map.put(file.getPath().getName(), new PlainBlobMetaData(file.getPath().getName(), file.getLen())); + if (file.isFile()) { + map.put(file.getPath().getName(), new PlainBlobMetaData(file.getPath().getName(), file.getLen())); + } } return Collections.unmodifiableMap(map); } @@ -151,6 +159,19 @@ public Map listBlobs() throws IOException { return listBlobsByPrefix(null); } + @Override + public Map children() throws IOException { + FileStatus[] files = store.execute(fileContext -> fileContext.util().listStatus(path)); + Map map = new LinkedHashMap<>(); + for (FileStatus file : files) { + if (file.isDirectory()) { + final String name = file.getPath().getName(); + map.put(name, new HdfsBlobContainer(path().add(name), store, new Path(path, name), bufferSize, securityContext)); + } + } + return Collections.unmodifiableMap(map); + } + /** * Exists to wrap underlying InputStream methods that might make socket connections in * doPrivileged blocks. This is due to the way that hdfs client libraries might open diff --git a/plugins/repository-hdfs/src/main/java/org/elasticsearch/repositories/hdfs/HdfsRepository.java b/plugins/repository-hdfs/src/main/java/org/elasticsearch/repositories/hdfs/HdfsRepository.java index b614753d83883..b51f843162a74 100644 --- a/plugins/repository-hdfs/src/main/java/org/elasticsearch/repositories/hdfs/HdfsRepository.java +++ b/plugins/repository-hdfs/src/main/java/org/elasticsearch/repositories/hdfs/HdfsRepository.java @@ -59,7 +59,6 @@ public final class HdfsRepository extends BlobStoreRepository { private final Environment environment; private final ByteSizeValue chunkSize; - private final BlobPath basePath = BlobPath.cleanPath(); private final URI uri; private final String pathSetting; @@ -69,7 +68,7 @@ public final class HdfsRepository extends BlobStoreRepository { public HdfsRepository(RepositoryMetaData metadata, Environment environment, NamedXContentRegistry namedXContentRegistry, ThreadPool threadPool) { - super(metadata, environment.settings(), namedXContentRegistry, threadPool); + super(metadata, environment.settings(), namedXContentRegistry, threadPool, BlobPath.cleanPath()); this.environment = environment; this.chunkSize = metadata.settings().getAsBytesSize("chunk_size", null); @@ -233,11 +232,6 @@ protected HdfsBlobStore createBlobStore() { return blobStore; } - @Override - protected BlobPath basePath() { - return basePath; - } - @Override protected ByteSizeValue chunkSize() { return chunkSize; diff --git a/plugins/repository-hdfs/src/test/java/org/elasticsearch/repositories/hdfs/HdfsRepositoryTests.java b/plugins/repository-hdfs/src/test/java/org/elasticsearch/repositories/hdfs/HdfsRepositoryTests.java new file mode 100644 index 0000000000000..e34f290a8e299 --- /dev/null +++ b/plugins/repository-hdfs/src/test/java/org/elasticsearch/repositories/hdfs/HdfsRepositoryTests.java @@ -0,0 +1,61 @@ +/* + * 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.repositories.hdfs; + +import com.carrotsearch.randomizedtesting.annotations.ThreadLeakFilters; +import org.elasticsearch.action.support.master.AcknowledgedResponse; +import org.elasticsearch.bootstrap.JavaVersion; +import org.elasticsearch.common.settings.MockSecureSettings; +import org.elasticsearch.common.settings.SecureSettings; +import org.elasticsearch.common.settings.Settings; +import org.elasticsearch.plugins.Plugin; +import org.elasticsearch.repositories.AbstractThirdPartyRepositoryTestCase; + +import java.util.Collection; + +import static org.hamcrest.Matchers.equalTo; + +@ThreadLeakFilters(filters = HdfsClientThreadLeakFilter.class) +public class HdfsRepositoryTests extends AbstractThirdPartyRepositoryTestCase { + + @Override + protected Collection> getPlugins() { + return pluginList(HdfsPlugin.class); + } + + @Override + protected SecureSettings credentials() { + return new MockSecureSettings(); + } + + @Override + protected void createRepository(String repoName) { + assumeFalse("https://github.com/elastic/elasticsearch/issues/31498", JavaVersion.current().equals(JavaVersion.parse("11"))); + AcknowledgedResponse putRepositoryResponse = client().admin().cluster().preparePutRepository(repoName) + .setType("hdfs") + .setSettings(Settings.builder() + .put("uri", "hdfs:///") + .put("conf.fs.AbstractFileSystem.hdfs.impl", TestingFs.class.getName()) + .put("path", "foo") + .put("chunk_size", randomIntBetween(100, 1000) + "k") + .put("compress", randomBoolean()) + ).get(); + assertThat(putRepositoryResponse.isAcknowledged(), equalTo(true)); + } +} diff --git a/plugins/repository-hdfs/src/test/java/org/elasticsearch/repositories/hdfs/HdfsTests.java b/plugins/repository-hdfs/src/test/java/org/elasticsearch/repositories/hdfs/HdfsTests.java index 88454188da588..41423d25b9f9f 100644 --- a/plugins/repository-hdfs/src/test/java/org/elasticsearch/repositories/hdfs/HdfsTests.java +++ b/plugins/repository-hdfs/src/test/java/org/elasticsearch/repositories/hdfs/HdfsTests.java @@ -27,9 +27,13 @@ import org.elasticsearch.cluster.ClusterState; import org.elasticsearch.common.settings.Settings; import org.elasticsearch.plugins.Plugin; +import org.elasticsearch.repositories.RepositoriesService; import org.elasticsearch.repositories.RepositoryException; +import org.elasticsearch.repositories.blobstore.BlobStoreRepository; +import org.elasticsearch.repositories.blobstore.BlobStoreTestUtil; import org.elasticsearch.snapshots.SnapshotState; import org.elasticsearch.test.ESSingleNodeTestCase; +import org.elasticsearch.threadpool.ThreadPool; import java.util.Collection; @@ -91,7 +95,7 @@ public void testSimpleWorkflow() { .prepareGetSnapshots("test-repo") .setSnapshots("test-snap") .get() - .getSnapshots() + .getSnapshots("test-repo") .get(0) .state(), equalTo(SnapshotState.SUCCESS)); @@ -145,6 +149,9 @@ public void testSimpleWorkflow() { ClusterState clusterState = client.admin().cluster().prepareState().get().getState(); assertThat(clusterState.getMetaData().hasIndex("test-idx-1"), equalTo(true)); assertThat(clusterState.getMetaData().hasIndex("test-idx-2"), equalTo(false)); + final BlobStoreRepository repo = + (BlobStoreRepository) getInstanceFromNode(RepositoriesService.class).repository("test-repo"); + BlobStoreTestUtil.assertConsistency(repo, repo.threadPool().executor(ThreadPool.Names.GENERIC)); } public void testMissingUri() { diff --git a/plugins/repository-hdfs/src/test/resources/rest-api-spec/test/hdfs_repository/30_snapshot_get.yml b/plugins/repository-hdfs/src/test/resources/rest-api-spec/test/hdfs_repository/30_snapshot_get.yml index f38f4783b195b..6ed93e07160b2 100644 --- a/plugins/repository-hdfs/src/test/resources/rest-api-spec/test/hdfs_repository/30_snapshot_get.yml +++ b/plugins/repository-hdfs/src/test/resources/rest-api-spec/test/hdfs_repository/30_snapshot_get.yml @@ -46,8 +46,8 @@ repository: test_snapshot_get_repository snapshot: test_snapshot_get - - length: { snapshots: 1 } - - match: { snapshots.0.snapshot : test_snapshot_get } + - length: { responses.0.snapshots: 1 } + - match: { responses.0.snapshots.0.snapshot : test_snapshot_get } # List snapshot info - do: @@ -55,8 +55,8 @@ repository: test_snapshot_get_repository snapshot: "*" - - length: { snapshots: 1 } - - match: { snapshots.0.snapshot : test_snapshot_get } + - length: { responses.0.snapshots: 1 } + - match: { responses.0.snapshots.0.snapshot : test_snapshot_get } # Remove our snapshot - do: diff --git a/plugins/repository-hdfs/src/test/resources/rest-api-spec/test/hdfs_repository/30_snapshot_readonly.yml b/plugins/repository-hdfs/src/test/resources/rest-api-spec/test/hdfs_repository/30_snapshot_readonly.yml index c2a37964e70a7..dda910ae36c26 100644 --- a/plugins/repository-hdfs/src/test/resources/rest-api-spec/test/hdfs_repository/30_snapshot_readonly.yml +++ b/plugins/repository-hdfs/src/test/resources/rest-api-spec/test/hdfs_repository/30_snapshot_readonly.yml @@ -21,7 +21,7 @@ repository: test_snapshot_repository_ro snapshot: "_all" - - length: { snapshots: 1 } + - length: { responses.0.snapshots: 1 } # Remove our repository - do: diff --git a/plugins/repository-s3/build.gradle b/plugins/repository-s3/build.gradle index 60a4e97cfa493..b6bf60c1c92e5 100644 --- a/plugins/repository-s3/build.gradle +++ b/plugins/repository-s3/build.gradle @@ -1,8 +1,9 @@ -import org.elasticsearch.gradle.BuildPlugin import org.elasticsearch.gradle.MavenFilteringHack import org.elasticsearch.gradle.test.AntFixture import org.elasticsearch.gradle.test.RestIntegTestTask +import static org.elasticsearch.gradle.PropertyNormalization.IGNORE_VALUE + /* * Licensed to Elasticsearch under one or more contributor * license agreements. See the NOTICE file distributed with @@ -28,17 +29,17 @@ esplugin { } versions << [ - 'aws': '1.11.505' + 'aws': '1.11.562' ] dependencies { compile "com.amazonaws:aws-java-sdk-s3:${versions.aws}" - compile "com.amazonaws:aws-java-sdk-kms:${versions.aws}" compile "com.amazonaws:aws-java-sdk-core:${versions.aws}" compile "com.amazonaws:jmespath-java:${versions.aws}" compile "org.apache.httpcomponents:httpclient:${versions.httpclient}" compile "org.apache.httpcomponents:httpcore:${versions.httpcore}" compile "commons-logging:commons-logging:${versions.commonslogging}" + compile "org.apache.logging.log4j:log4j-1.2-api:${versions.log4j}" compile "commons-codec:commons-codec:${versions.commonscodec}" compile "com.fasterxml.jackson.core:jackson-core:${versions.jackson}" compile 'com.fasterxml.jackson.core:jackson-databind:2.8.11.3' @@ -75,6 +76,7 @@ test { // these are tested explicitly in separate test tasks exclude '**/*CredentialsTests.class' exclude '**/S3BlobStoreRepositoryTests.class' + exclude '**/S3RepositoryThirdPartyTests.class' } boolean useFixture = false @@ -134,6 +136,14 @@ if (!s3EC2Bucket && !s3EC2BasePath && !s3ECSBucket && !s3ECSBasePath) { throw new IllegalArgumentException("not all options specified to run EC2/ECS tests are present") } +task thirdPartyTest(type: Test) { + include '**/S3RepositoryThirdPartyTests.class' + systemProperty 'test.s3.account', s3PermanentAccessKey + systemProperty 'test.s3.key', s3PermanentSecretKey + systemProperty 'test.s3.bucket', s3PermanentBucket + systemProperty 'test.s3.base', s3PermanentBasePath +} + if (useFixture) { apply plugin: 'elasticsearch.test.fixtures' task writeDockerFile { @@ -151,6 +161,31 @@ if (useFixture) { dependsOn(writeDockerFile) } + def minioAddress = { + int minioPort = postProcessFixture.ext."test.fixtures.minio-fixture.tcp.9000" + assert minioPort > 0 + 'http://127.0.0.1:' + minioPort + } + + File minioAddressFile = new File(project.buildDir, 'generated-resources/s3Fixture.address') + + normalization { + runtimeClasspath { + // ignore generated address file for the purposes of build avoidance + ignore 's3Fixture.address' + } + } + + thirdPartyTest { + dependsOn tasks.bundlePlugin, tasks.postProcessFixture + outputs.file(minioAddressFile) + doFirst { + file(minioAddressFile).text = "${ -> minioAddress.call() }" + } + // TODO: this could be a nonInputProperties.systemProperty so we don't need a file + systemProperty 'test.s3.endpoint', minioAddressFile.name + } + task integTestMinio(type: RestIntegTestTask) { description = "Runs REST tests using the Minio repository." dependsOn tasks.bundlePlugin, tasks.postProcessFixture @@ -164,16 +199,11 @@ if (useFixture) { } } check.dependsOn(integTestMinio) - BuildPlugin.requireDocker(tasks.integTestMinio) testClusters.integTestMinio { keystore 's3.client.integration_test_permanent.access_key', s3PermanentAccessKey keystore 's3.client.integration_test_permanent.secret_key', s3PermanentSecretKey - setting 's3.client.integration_test_permanent.endpoint', { - int minioPort = postProcessFixture.ext."test.fixtures.minio-fixture.tcp.9000" - assert minioPort > 0 - return 'http://127.0.0.1:' + minioPort - } + setting 's3.client.integration_test_permanent.endpoint', minioAddress, IGNORE_VALUE plugin file(tasks.bundlePlugin.archiveFile) } @@ -191,6 +221,8 @@ if (useFixture) { } } +check.dependsOn(thirdPartyTest) + File parentFixtures = new File(project.buildDir, "fixtures") File s3FixtureFile = new File(parentFixtures, 's3Fixture.properties') @@ -251,12 +283,12 @@ testClusters.integTest { keystore 's3.client.integration_test_temporary.session_token', s3TemporarySessionToken if (useFixture) { - setting 's3.client.integration_test_permanent.endpoint', { "http://${s3Fixture.addressAndPort}" } - setting 's3.client.integration_test_temporary.endpoint', { "http://${s3Fixture.addressAndPort}" } - setting 's3.client.integration_test_ec2.endpoint', { "http://${s3Fixture.addressAndPort}" } + setting 's3.client.integration_test_permanent.endpoint', { "http://${s3Fixture.addressAndPort}" }, IGNORE_VALUE + setting 's3.client.integration_test_temporary.endpoint', { "http://${s3Fixture.addressAndPort}" }, IGNORE_VALUE + setting 's3.client.integration_test_ec2.endpoint', { "http://${s3Fixture.addressAndPort}" }, IGNORE_VALUE // to redirect InstanceProfileCredentialsProvider to custom auth point - systemProperty "com.amazonaws.sdk.ec2MetadataServiceEndpointOverride", { "http://${s3Fixture.addressAndPort}" } + systemProperty "com.amazonaws.sdk.ec2MetadataServiceEndpointOverride", { "http://${s3Fixture.addressAndPort}" }, IGNORE_VALUE } else { println "Using an external service to test the repository-s3 plugin" } @@ -278,9 +310,9 @@ if (useFixture) { check.dependsOn(integTestECS) testClusters.integTestECS { - setting 's3.client.integration_test_ecs.endpoint', { "http://${s3Fixture.addressAndPort}" } + setting 's3.client.integration_test_ecs.endpoint', { "http://${s3Fixture.addressAndPort}" }, IGNORE_VALUE plugin file(tasks.bundlePlugin.archiveFile) - environment 'AWS_CONTAINER_CREDENTIALS_FULL_URI', { "http://${s3Fixture.addressAndPort}/ecs_credentials_endpoint" } + environment 'AWS_CONTAINER_CREDENTIALS_FULL_URI', { "http://${s3Fixture.addressAndPort}/ecs_credentials_endpoint" }, IGNORE_VALUE } } diff --git a/plugins/repository-s3/licenses/aws-java-sdk-core-1.11.505.jar.sha1 b/plugins/repository-s3/licenses/aws-java-sdk-core-1.11.505.jar.sha1 deleted file mode 100644 index add5db290e8a8..0000000000000 --- a/plugins/repository-s3/licenses/aws-java-sdk-core-1.11.505.jar.sha1 +++ /dev/null @@ -1 +0,0 @@ -d19328c227b2b5ad81d137361ebc9cbcd0396465 \ No newline at end of file diff --git a/plugins/repository-s3/licenses/aws-java-sdk-core-1.11.562.jar.sha1 b/plugins/repository-s3/licenses/aws-java-sdk-core-1.11.562.jar.sha1 new file mode 100644 index 0000000000000..ed8ded6a3608c --- /dev/null +++ b/plugins/repository-s3/licenses/aws-java-sdk-core-1.11.562.jar.sha1 @@ -0,0 +1 @@ +b5fc47ec1b5afe180f5ebb4eda755acdca7a20ae \ No newline at end of file diff --git a/plugins/repository-s3/licenses/aws-java-sdk-kms-1.11.505.jar.sha1 b/plugins/repository-s3/licenses/aws-java-sdk-kms-1.11.505.jar.sha1 deleted file mode 100644 index ab2bfdbc1897b..0000000000000 --- a/plugins/repository-s3/licenses/aws-java-sdk-kms-1.11.505.jar.sha1 +++ /dev/null @@ -1 +0,0 @@ -2a219919090a6cadd7e119c899c90343ad9c0077 \ No newline at end of file diff --git a/plugins/repository-s3/licenses/aws-java-sdk-s3-1.11.505.jar.sha1 b/plugins/repository-s3/licenses/aws-java-sdk-s3-1.11.505.jar.sha1 deleted file mode 100644 index 04c4cc213c690..0000000000000 --- a/plugins/repository-s3/licenses/aws-java-sdk-s3-1.11.505.jar.sha1 +++ /dev/null @@ -1 +0,0 @@ -b4cf82765b04a579609314ab7f296a9a0ddae1cf \ No newline at end of file diff --git a/plugins/repository-s3/licenses/aws-java-sdk-s3-1.11.562.jar.sha1 b/plugins/repository-s3/licenses/aws-java-sdk-s3-1.11.562.jar.sha1 new file mode 100644 index 0000000000000..8e852fe9b275f --- /dev/null +++ b/plugins/repository-s3/licenses/aws-java-sdk-s3-1.11.562.jar.sha1 @@ -0,0 +1 @@ +1712c878f7e9483ceac1eb2356a9457a3c8df03e \ No newline at end of file diff --git a/plugins/repository-s3/licenses/jmespath-java-1.11.505.jar.sha1 b/plugins/repository-s3/licenses/jmespath-java-1.11.505.jar.sha1 deleted file mode 100644 index 803d4fe85d77c..0000000000000 --- a/plugins/repository-s3/licenses/jmespath-java-1.11.505.jar.sha1 +++ /dev/null @@ -1 +0,0 @@ -067234d307b210097e247a49f08875e0cd3f2b95 \ No newline at end of file diff --git a/plugins/repository-s3/licenses/jmespath-java-1.11.562.jar.sha1 b/plugins/repository-s3/licenses/jmespath-java-1.11.562.jar.sha1 new file mode 100644 index 0000000000000..8e2d0e1935a3f --- /dev/null +++ b/plugins/repository-s3/licenses/jmespath-java-1.11.562.jar.sha1 @@ -0,0 +1 @@ +1147ed0ad1f2c5a16b8271e38e3cda5cd488c8ae \ No newline at end of file diff --git a/plugins/repository-s3/licenses/log4j-1.2-api-2.11.1.jar.sha1 b/plugins/repository-s3/licenses/log4j-1.2-api-2.11.1.jar.sha1 new file mode 100644 index 0000000000000..575d75dbda8c5 --- /dev/null +++ b/plugins/repository-s3/licenses/log4j-1.2-api-2.11.1.jar.sha1 @@ -0,0 +1 @@ +3aba3398fe064a3eab4331f88161c7480e848418 \ No newline at end of file diff --git a/plugins/repository-s3/licenses/log4j-LICENSE.txt b/plugins/repository-s3/licenses/log4j-LICENSE.txt new file mode 100644 index 0000000000000..6279e5206de13 --- /dev/null +++ b/plugins/repository-s3/licenses/log4j-LICENSE.txt @@ -0,0 +1,202 @@ + + Apache License + Version 2.0, January 2004 + http://www.apache.org/licenses/ + + TERMS AND CONDITIONS FOR USE, REPRODUCTION, AND DISTRIBUTION + + 1. Definitions. + + "License" shall mean the terms and conditions for use, reproduction, + and distribution as defined by Sections 1 through 9 of this document. + + "Licensor" shall mean the copyright owner or entity authorized by + the copyright owner that is granting the License. + + "Legal Entity" shall mean the union of the acting entity and all + other entities that control, are controlled by, or are under common + control with that entity. For the purposes of this definition, + "control" means (i) the power, direct or indirect, to cause the + direction or management of such entity, whether by contract or + otherwise, or (ii) ownership of fifty percent (50%) or more of the + outstanding shares, or (iii) beneficial ownership of such entity. + + "You" (or "Your") shall mean an individual or Legal Entity + exercising permissions granted by this License. + + "Source" form shall mean the preferred form for making modifications, + including but not limited to software source code, documentation + source, and configuration files. + + "Object" form shall mean any form resulting from mechanical + transformation or translation of a Source form, including but + not limited to compiled object code, generated documentation, + and conversions to other media types. + + "Work" shall mean the work of authorship, whether in Source or + Object form, made available under the License, as indicated by a + copyright notice that is included in or attached to the work + (an example is provided in the Appendix below). + + "Derivative Works" shall mean any work, whether in Source or Object + form, that is based on (or derived from) the Work and for which the + editorial revisions, annotations, elaborations, or other modifications + represent, as a whole, an original work of authorship. For the purposes + of this License, Derivative Works shall not include works that remain + separable from, or merely link (or bind by name) to the interfaces of, + the Work and Derivative Works thereof. + + "Contribution" shall mean any work of authorship, including + the original version of the Work and any modifications or additions + to that Work or Derivative Works thereof, that is intentionally + submitted to Licensor for inclusion in the Work by the copyright owner + or by an individual or Legal Entity authorized to submit on behalf of + the copyright owner. For the purposes of this definition, "submitted" + means any form of electronic, verbal, or written communication sent + to the Licensor or its representatives, including but not limited to + communication on electronic mailing lists, source code control systems, + and issue tracking systems that are managed by, or on behalf of, the + Licensor for the purpose of discussing and improving the Work, but + excluding communication that is conspicuously marked or otherwise + designated in writing by the copyright owner as "Not a Contribution." + + "Contributor" shall mean Licensor and any individual or Legal Entity + on behalf of whom a Contribution has been received by Licensor and + subsequently incorporated within the Work. + + 2. Grant of Copyright License. Subject to the terms and conditions of + this License, each Contributor hereby grants to You a perpetual, + worldwide, non-exclusive, no-charge, royalty-free, irrevocable + copyright license to reproduce, prepare Derivative Works of, + publicly display, publicly perform, sublicense, and distribute the + Work and such Derivative Works in Source or Object form. + + 3. Grant of Patent License. Subject to the terms and conditions of + this License, each Contributor hereby grants to You a perpetual, + worldwide, non-exclusive, no-charge, royalty-free, irrevocable + (except as stated in this section) patent license to make, have made, + use, offer to sell, sell, import, and otherwise transfer the Work, + where such license applies only to those patent claims licensable + by such Contributor that are necessarily infringed by their + Contribution(s) alone or by combination of their Contribution(s) + with the Work to which such Contribution(s) was submitted. If You + institute patent litigation against any entity (including a + cross-claim or counterclaim in a lawsuit) alleging that the Work + or a Contribution incorporated within the Work constitutes direct + or contributory patent infringement, then any patent licenses + granted to You under this License for that Work shall terminate + as of the date such litigation is filed. + + 4. Redistribution. You may reproduce and distribute copies of the + Work or Derivative Works thereof in any medium, with or without + modifications, and in Source or Object form, provided that You + meet the following conditions: + + (a) You must give any other recipients of the Work or + Derivative Works a copy of this License; and + + (b) You must cause any modified files to carry prominent notices + stating that You changed the files; and + + (c) You must retain, in the Source form of any Derivative Works + that You distribute, all copyright, patent, trademark, and + attribution notices from the Source form of the Work, + excluding those notices that do not pertain to any part of + the Derivative Works; and + + (d) If the Work includes a "NOTICE" text file as part of its + distribution, then any Derivative Works that You distribute must + include a readable copy of the attribution notices contained + within such NOTICE file, excluding those notices that do not + pertain to any part of the Derivative Works, in at least one + of the following places: within a NOTICE text file distributed + as part of the Derivative Works; within the Source form or + documentation, if provided along with the Derivative Works; or, + within a display generated by the Derivative Works, if and + wherever such third-party notices normally appear. The contents + of the NOTICE file are for informational purposes only and + do not modify the License. You may add Your own attribution + notices within Derivative Works that You distribute, alongside + or as an addendum to the NOTICE text from the Work, provided + that such additional attribution notices cannot be construed + as modifying the License. + + You may add Your own copyright statement to Your modifications and + may provide additional or different license terms and conditions + for use, reproduction, or distribution of Your modifications, or + for any such Derivative Works as a whole, provided Your use, + reproduction, and distribution of the Work otherwise complies with + the conditions stated in this License. + + 5. Submission of Contributions. Unless You explicitly state otherwise, + any Contribution intentionally submitted for inclusion in the Work + by You to the Licensor shall be under the terms and conditions of + this License, without any additional terms or conditions. + Notwithstanding the above, nothing herein shall supersede or modify + the terms of any separate license agreement you may have executed + with Licensor regarding such Contributions. + + 6. Trademarks. This License does not grant permission to use the trade + names, trademarks, service marks, or product names of the Licensor, + except as required for reasonable and customary use in describing the + origin of the Work and reproducing the content of the NOTICE file. + + 7. Disclaimer of Warranty. Unless required by applicable law or + agreed to in writing, Licensor provides the Work (and each + Contributor provides its Contributions) on an "AS IS" BASIS, + WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or + implied, including, without limitation, any warranties or conditions + of TITLE, NON-INFRINGEMENT, MERCHANTABILITY, or FITNESS FOR A + PARTICULAR PURPOSE. You are solely responsible for determining the + appropriateness of using or redistributing the Work and assume any + risks associated with Your exercise of permissions under this License. + + 8. Limitation of Liability. In no event and under no legal theory, + whether in tort (including negligence), contract, or otherwise, + unless required by applicable law (such as deliberate and grossly + negligent acts) or agreed to in writing, shall any Contributor be + liable to You for damages, including any direct, indirect, special, + incidental, or consequential damages of any character arising as a + result of this License or out of the use or inability to use the + Work (including but not limited to damages for loss of goodwill, + work stoppage, computer failure or malfunction, or any and all + other commercial damages or losses), even if such Contributor + has been advised of the possibility of such damages. + + 9. Accepting Warranty or Additional Liability. While redistributing + the Work or Derivative Works thereof, You may choose to offer, + and charge a fee for, acceptance of support, warranty, indemnity, + or other liability obligations and/or rights consistent with this + License. However, in accepting such obligations, You may act only + on Your own behalf and on Your sole responsibility, not on behalf + of any other Contributor, and only if You agree to indemnify, + defend, and hold each Contributor harmless for any liability + incurred by, or claims asserted against, such Contributor by reason + of your accepting any such warranty or additional liability. + + END OF TERMS AND CONDITIONS + + APPENDIX: How to apply the Apache License to your work. + + To apply the Apache License to your work, attach the following + boilerplate notice, with the fields enclosed by brackets "[]" + replaced with your own identifying information. (Don't include + the brackets!) The text should be enclosed in the appropriate + comment syntax for the file format. We also recommend that a + file or class name and description of purpose be included on the + same "printed page" as the copyright notice for easier + identification within third-party archives. + + Copyright 1999-2005 The Apache Software Foundation + + Licensed 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. diff --git a/plugins/repository-s3/licenses/log4j-NOTICE.txt b/plugins/repository-s3/licenses/log4j-NOTICE.txt new file mode 100644 index 0000000000000..0375732360047 --- /dev/null +++ b/plugins/repository-s3/licenses/log4j-NOTICE.txt @@ -0,0 +1,5 @@ +Apache log4j +Copyright 2007 The Apache Software Foundation + +This product includes software developed at +The Apache Software Foundation (http://www.apache.org/). \ No newline at end of file diff --git a/plugins/repository-s3/src/main/java/org/elasticsearch/repositories/s3/S3BlobContainer.java b/plugins/repository-s3/src/main/java/org/elasticsearch/repositories/s3/S3BlobContainer.java index aabdd8b52e8e1..c2e1f3de7f0dd 100644 --- a/plugins/repository-s3/src/main/java/org/elasticsearch/repositories/s3/S3BlobContainer.java +++ b/plugins/repository-s3/src/main/java/org/elasticsearch/repositories/s3/S3BlobContainer.java @@ -25,6 +25,7 @@ import com.amazonaws.services.s3.model.CompleteMultipartUploadRequest; import com.amazonaws.services.s3.model.DeleteObjectsRequest; import com.amazonaws.services.s3.model.InitiateMultipartUploadRequest; +import com.amazonaws.services.s3.model.ListObjectsRequest; import com.amazonaws.services.s3.model.MultiObjectDeleteException; import com.amazonaws.services.s3.model.ObjectListing; import com.amazonaws.services.s3.model.ObjectMetadata; @@ -38,6 +39,7 @@ import org.elasticsearch.ExceptionsHelper; import org.elasticsearch.common.Nullable; import org.elasticsearch.common.Strings; +import org.elasticsearch.common.blobstore.BlobContainer; import org.elasticsearch.common.blobstore.BlobMetaData; import org.elasticsearch.common.blobstore.BlobPath; import org.elasticsearch.common.blobstore.BlobStoreException; @@ -50,6 +52,7 @@ import java.io.InputStream; import java.nio.file.NoSuchFileException; import java.util.ArrayList; +import java.util.HashSet; import java.util.List; import java.util.Map; import java.util.Set; @@ -127,12 +130,53 @@ public void deleteBlob(String blobName) throws IOException { deleteBlobIgnoringIfNotExists(blobName); } + @Override + public void delete() throws IOException { + try (AmazonS3Reference clientReference = blobStore.clientReference()) { + ObjectListing prevListing = null; + while (true) { + ObjectListing list; + if (prevListing != null) { + final ObjectListing finalPrevListing = prevListing; + list = SocketAccess.doPrivileged(() -> clientReference.client().listNextBatchOfObjects(finalPrevListing)); + } else { + final ListObjectsRequest listObjectsRequest = new ListObjectsRequest(); + listObjectsRequest.setBucketName(blobStore.bucket()); + listObjectsRequest.setPrefix(keyPath); + list = SocketAccess.doPrivileged(() -> clientReference.client().listObjects(listObjectsRequest)); + } + final List blobsToDelete = + list.getObjectSummaries().stream().map(S3ObjectSummary::getKey).collect(Collectors.toList()); + if (list.isTruncated()) { + doDeleteBlobs(blobsToDelete, false); + prevListing = list; + } else { + final List lastBlobsToDelete = new ArrayList<>(blobsToDelete); + lastBlobsToDelete.add(keyPath); + doDeleteBlobs(lastBlobsToDelete, false); + break; + } + } + } catch (final AmazonClientException e) { + throw new IOException("Exception when deleting blob container [" + keyPath + "]", e); + } + } + @Override public void deleteBlobsIgnoringIfNotExists(List blobNames) throws IOException { + doDeleteBlobs(blobNames, true); + } + + private void doDeleteBlobs(List blobNames, boolean relative) throws IOException { if (blobNames.isEmpty()) { return; } - final Set outstanding = blobNames.stream().map(this::buildKey).collect(Collectors.toSet()); + final Set outstanding; + if (relative) { + outstanding = blobNames.stream().map(this::buildKey).collect(Collectors.toSet()); + } else { + outstanding = new HashSet<>(blobNames); + } try (AmazonS3Reference clientReference = blobStore.clientReference()) { // S3 API only allows 1k blobs per delete so we split up the given blobs into requests of max. 1k deletes final List deleteRequests = new ArrayList<>(); @@ -203,12 +247,15 @@ public Map listBlobsByPrefix(@Nullable String blobNamePref final ObjectListing finalPrevListing = prevListing; list = SocketAccess.doPrivileged(() -> clientReference.client().listNextBatchOfObjects(finalPrevListing)); } else { + final ListObjectsRequest listObjectsRequest = new ListObjectsRequest(); + listObjectsRequest.setBucketName(blobStore.bucket()); + listObjectsRequest.setDelimiter("/"); if (blobNamePrefix != null) { - list = SocketAccess.doPrivileged(() -> clientReference.client().listObjects(blobStore.bucket(), - buildKey(blobNamePrefix))); + listObjectsRequest.setPrefix(buildKey(blobNamePrefix)); } else { - list = SocketAccess.doPrivileged(() -> clientReference.client().listObjects(blobStore.bucket(), keyPath)); + listObjectsRequest.setPrefix(keyPath); } + list = SocketAccess.doPrivileged(() -> clientReference.client().listObjects(listObjectsRequest)); } for (final S3ObjectSummary summary : list.getObjectSummaries()) { final String name = summary.getKey().substring(keyPath.length()); @@ -231,6 +278,52 @@ public Map listBlobs() throws IOException { return listBlobsByPrefix(null); } + @Override + public Map children() throws IOException { + try (AmazonS3Reference clientReference = blobStore.clientReference()) { + ObjectListing prevListing = null; + final var entries = new ArrayList>(); + while (true) { + ObjectListing list; + if (prevListing != null) { + final ObjectListing finalPrevListing = prevListing; + list = SocketAccess.doPrivileged(() -> clientReference.client().listNextBatchOfObjects(finalPrevListing)); + } else { + final ListObjectsRequest listObjectsRequest = new ListObjectsRequest(); + listObjectsRequest.setBucketName(blobStore.bucket()); + listObjectsRequest.setPrefix(keyPath); + listObjectsRequest.setDelimiter("/"); + list = SocketAccess.doPrivileged(() -> clientReference.client().listObjects(listObjectsRequest)); + } + for (final String summary : list.getCommonPrefixes()) { + final String name = summary.substring(keyPath.length()); + if (name.isEmpty() == false) { + // Stripping the trailing slash off of the common prefix + final String last = name.substring(0, name.length() - 1); + final BlobPath path = path().add(last); + entries.add(entry(last, blobStore.blobContainer(path))); + } + } + assert list.getObjectSummaries().stream().noneMatch(s -> { + for (String commonPrefix : list.getCommonPrefixes()) { + if (s.getKey().substring(keyPath.length()).startsWith(commonPrefix)) { + return true; + } + } + return false; + }) : "Response contained children for listed common prefixes."; + if (list.isTruncated()) { + prevListing = list; + } else { + break; + } + } + return Maps.ofEntries(entries); + } catch (final AmazonClientException e) { + throw new IOException("Exception when listing children of [" + path().buildAsString() + ']', e); + } + } + private String buildKey(String blobName) { return keyPath + blobName; } diff --git a/plugins/repository-s3/src/main/java/org/elasticsearch/repositories/s3/S3Repository.java b/plugins/repository-s3/src/main/java/org/elasticsearch/repositories/s3/S3Repository.java index a91e7bf663f92..50b6e3265f9d3 100644 --- a/plugins/repository-s3/src/main/java/org/elasticsearch/repositories/s3/S3Repository.java +++ b/plugins/repository-s3/src/main/java/org/elasticsearch/repositories/s3/S3Repository.java @@ -150,8 +150,6 @@ class S3Repository extends BlobStoreRepository { private final ByteSizeValue chunkSize; - private final BlobPath basePath; - private final boolean serverSideEncryption; private final String storageClass; @@ -165,7 +163,7 @@ class S3Repository extends BlobStoreRepository { final Settings settings, final NamedXContentRegistry namedXContentRegistry, final S3Service service, final ThreadPool threadPool) { - super(metadata, settings, namedXContentRegistry, threadPool); + super(metadata, settings, namedXContentRegistry, threadPool, buildBasePath(metadata)); this.service = service; // Parse and validate the user's S3 Storage Class setting @@ -183,13 +181,6 @@ class S3Repository extends BlobStoreRepository { ") can't be lower than " + BUFFER_SIZE_SETTING.getKey() + " (" + bufferSize + ")."); } - final String basePath = BASE_PATH_SETTING.get(metadata.settings()); - if (Strings.hasLength(basePath)) { - this.basePath = new BlobPath().add(basePath); - } else { - this.basePath = BlobPath.cleanPath(); - } - this.serverSideEncryption = SERVER_SIDE_ENCRYPTION_SETTING.get(metadata.settings()); this.storageClass = STORAGE_CLASS_SETTING.get(metadata.settings()); @@ -211,15 +202,18 @@ class S3Repository extends BlobStoreRepository { storageClass); } - @Override - protected S3BlobStore createBlobStore() { - return new S3BlobStore(service, bucket, serverSideEncryption, bufferSize, cannedACL, storageClass, metadata); + private static BlobPath buildBasePath(RepositoryMetaData metadata) { + final String basePath = BASE_PATH_SETTING.get(metadata.settings()); + if (Strings.hasLength(basePath)) { + return new BlobPath().add(basePath); + } else { + return BlobPath.cleanPath(); + } } - // only use for testing @Override - protected BlobStore blobStore() { - return super.blobStore(); + protected S3BlobStore createBlobStore() { + return new S3BlobStore(service, bucket, serverSideEncryption, bufferSize, cannedACL, storageClass, metadata); } // only use for testing @@ -228,11 +222,6 @@ protected BlobStore getBlobStore() { return super.getBlobStore(); } - @Override - protected BlobPath basePath() { - return basePath; - } - @Override protected ByteSizeValue chunkSize() { return chunkSize; diff --git a/plugins/repository-s3/src/test/java/org/elasticsearch/repositories/s3/S3RepositoryThirdPartyTests.java b/plugins/repository-s3/src/test/java/org/elasticsearch/repositories/s3/S3RepositoryThirdPartyTests.java new file mode 100644 index 0000000000000..7e7ac8d430062 --- /dev/null +++ b/plugins/repository-s3/src/test/java/org/elasticsearch/repositories/s3/S3RepositoryThirdPartyTests.java @@ -0,0 +1,114 @@ +/* + * 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.repositories.s3; + +import org.elasticsearch.action.support.master.AcknowledgedResponse; +import org.elasticsearch.common.blobstore.BlobMetaData; +import org.elasticsearch.common.blobstore.BlobPath; +import org.elasticsearch.common.settings.MockSecureSettings; +import org.elasticsearch.common.settings.SecureSettings; +import org.elasticsearch.common.settings.Settings; +import org.elasticsearch.plugins.Plugin; +import org.elasticsearch.repositories.AbstractThirdPartyRepositoryTestCase; +import org.elasticsearch.repositories.blobstore.BlobStoreRepository; +import org.elasticsearch.test.StreamsUtils; + +import java.io.IOException; +import java.util.Collection; +import java.util.concurrent.Executor; +import java.util.Map; +import java.util.concurrent.TimeUnit; + +import static org.hamcrest.Matchers.blankOrNullString; +import static org.hamcrest.Matchers.equalTo; +import static org.hamcrest.Matchers.not; + +public class S3RepositoryThirdPartyTests extends AbstractThirdPartyRepositoryTestCase { + + @Override + protected Collection> getPlugins() { + return pluginList(S3RepositoryPlugin.class); + } + + @Override + protected SecureSettings credentials() { + assertThat(System.getProperty("test.s3.account"), not(blankOrNullString())); + assertThat(System.getProperty("test.s3.key"), not(blankOrNullString())); + assertThat(System.getProperty("test.s3.bucket"), not(blankOrNullString())); + + MockSecureSettings secureSettings = new MockSecureSettings(); + secureSettings.setString("s3.client.default.access_key", System.getProperty("test.s3.account")); + secureSettings.setString("s3.client.default.secret_key", System.getProperty("test.s3.key")); + return secureSettings; + } + + @Override + protected void createRepository(String repoName) { + Settings.Builder settings = Settings.builder() + .put("bucket", System.getProperty("test.s3.bucket")) + .put("base_path", System.getProperty("test.s3.base", "testpath")); + final String endpointPath = System.getProperty("test.s3.endpoint"); + if (endpointPath != null) { + try { + settings = settings.put("endpoint", StreamsUtils.copyToStringFromClasspath("/" + endpointPath)); + } catch (IOException e) { + throw new AssertionError(e); + } + } + AcknowledgedResponse putRepositoryResponse = client().admin().cluster().preparePutRepository("test-repo") + .setType("s3") + .setSettings(settings).get(); + assertThat(putRepositoryResponse.isAcknowledged(), equalTo(true)); + } + + @Override + protected boolean assertCorruptionVisible(BlobStoreRepository repo, Executor genericExec) throws Exception { + // S3 is only eventually consistent for the list operations used by this assertions so we retry for 10 minutes assuming that + // listing operations will become consistent within these 10 minutes. + assertBusy(() -> assertTrue(super.assertCorruptionVisible(repo, genericExec)), 10L, TimeUnit.MINUTES); + return true; + } + + @Override + protected void assertConsistentRepository(BlobStoreRepository repo, Executor executor) throws Exception { + // S3 is only eventually consistent for the list operations used by this assertions so we retry for 10 minutes assuming that + // listing operations will become consistent within these 10 minutes. + assertBusy(() -> super.assertConsistentRepository(repo, executor), 10L, TimeUnit.MINUTES); + } + + protected void assertBlobsByPrefix(BlobPath path, String prefix, Map blobs) throws Exception { + // AWS S3 is eventually consistent so we retry for 10 minutes assuming a list operation will never take longer than that + // to become consistent. + assertBusy(() -> super.assertBlobsByPrefix(path, prefix, blobs), 10L, TimeUnit.MINUTES); + } + + @Override + protected void assertChildren(BlobPath path, Collection children) throws Exception { + // AWS S3 is eventually consistent so we retry for 10 minutes assuming a list operation will never take longer than that + // to become consistent. + assertBusy(() -> super.assertChildren(path, children), 10L, TimeUnit.MINUTES); + } + + @Override + protected void assertDeleted(BlobPath path, String name) throws Exception { + // AWS S3 is eventually consistent so we retry for 10 minutes assuming a list operation will never take longer than that + // to become consistent. + assertBusy(() -> super.assertDeleted(path, name), 10L, TimeUnit.MINUTES); + } +} diff --git a/plugins/repository-s3/src/test/resources/rest-api-spec/test/repository_s3/20_repository_permanent_credentials.yml b/plugins/repository-s3/src/test/resources/rest-api-spec/test/repository_s3/20_repository_permanent_credentials.yml index e6c94f8c408d9..d319bf8984a97 100644 --- a/plugins/repository-s3/src/test/resources/rest-api-spec/test/repository_s3/20_repository_permanent_credentials.yml +++ b/plugins/repository-s3/src/test/resources/rest-api-spec/test/repository_s3/20_repository_permanent_credentials.yml @@ -31,7 +31,6 @@ setup: --- "Try to create repository with broken endpoint override and named client": - # Register repository with broken endpoint setting - do: catch: /repository_verification_exception/ @@ -108,6 +107,9 @@ setup: --- "Snapshot and Restore with repository-s3 using permanent credentials": + - skip: + version: " - 7.9.99" + reason: "8.0 changes get snapshots response format" # Get repository - do: @@ -215,9 +217,9 @@ setup: repository: repository_permanent snapshot: snapshot-one,snapshot-two - - is_true: snapshots - - match: { snapshots.0.state : SUCCESS } - - match: { snapshots.1.state : SUCCESS } + - is_true: responses.0.snapshots + - match: { responses.0.snapshots.0.state : SUCCESS } + - match: { responses.0.snapshots.1.state : SUCCESS } # Delete the index - do: @@ -322,13 +324,18 @@ setup: --- "Get a non existing snapshot": + - skip: + version: " - 7.9.99" + reason: "8.0 changes get snapshots response format" - do: - catch: /snapshot_missing_exception/ snapshot.get: repository: repository_permanent snapshot: missing + - is_true: responses.0.error + - match: { responses.0.error.type: snapshot_missing_exception } + --- "Delete a non existing snapshot": diff --git a/plugins/repository-s3/src/test/resources/rest-api-spec/test/repository_s3/30_repository_temporary_credentials.yml b/plugins/repository-s3/src/test/resources/rest-api-spec/test/repository_s3/30_repository_temporary_credentials.yml index d5bdcd9c4f203..3ad6c3959634b 100644 --- a/plugins/repository-s3/src/test/resources/rest-api-spec/test/repository_s3/30_repository_temporary_credentials.yml +++ b/plugins/repository-s3/src/test/resources/rest-api-spec/test/repository_s3/30_repository_temporary_credentials.yml @@ -18,6 +18,9 @@ setup: --- "Snapshot and Restore with repository-s3 using temporary credentials": + - skip: + version: " - 7.9.99" + reason: "8.0 changes get snapshots response format" # Get repository - do: @@ -122,9 +125,9 @@ setup: repository: repository_temporary snapshot: snapshot-one,snapshot-two - - is_true: snapshots - - match: { snapshots.0.state : SUCCESS } - - match: { snapshots.1.state : SUCCESS } + - is_true: responses.0.snapshots + - match: { responses.0.snapshots.0.state : SUCCESS } + - match: { responses.0.snapshots.1.state : SUCCESS } # Delete the index - do: @@ -229,13 +232,18 @@ setup: --- "Get a non existing snapshot": + - skip: + version: " - 7.9.99" + reason: "8.0 changes get snapshots response format" - do: - catch: /snapshot_missing_exception/ snapshot.get: repository: repository_temporary snapshot: missing + - is_true: responses.0.error + - match: { responses.0.error.type: snapshot_missing_exception } + --- "Delete a non existing snapshot": diff --git a/plugins/repository-s3/src/test/resources/rest-api-spec/test/repository_s3/40_repository_ec2_credentials.yml b/plugins/repository-s3/src/test/resources/rest-api-spec/test/repository_s3/40_repository_ec2_credentials.yml index 829ae6197659c..fa1d3fc10fb13 100644 --- a/plugins/repository-s3/src/test/resources/rest-api-spec/test/repository_s3/40_repository_ec2_credentials.yml +++ b/plugins/repository-s3/src/test/resources/rest-api-spec/test/repository_s3/40_repository_ec2_credentials.yml @@ -18,6 +18,9 @@ setup: --- "Snapshot and Restore with repository-s3 using ec2 credentials": + - skip: + version: " - 7.9.99" + reason: "8.0 changes get snapshots response format" # Get repository - do: @@ -122,9 +125,9 @@ setup: repository: repository_ec2 snapshot: snapshot-one,snapshot-two - - is_true: snapshots - - match: { snapshots.0.state : SUCCESS } - - match: { snapshots.1.state : SUCCESS } + - is_true: responses.0.snapshots + - match: { responses.0.snapshots.0.state : SUCCESS } + - match: { responses.0.snapshots.1.state : SUCCESS } # Delete the index - do: @@ -229,13 +232,18 @@ setup: --- "Get a non existing snapshot": + - skip: + version: " - 7.9.99" + reason: "8.0 changes get snapshots response format" - do: - catch: /snapshot_missing_exception/ snapshot.get: repository: repository_ec2 snapshot: missing + - is_true: responses.0.error + - match: { responses.0.error.type: snapshot_missing_exception } + --- "Delete a non existing snapshot": diff --git a/plugins/repository-s3/src/test/resources/rest-api-spec/test/repository_s3/50_repository_ecs_credentials.yml b/plugins/repository-s3/src/test/resources/rest-api-spec/test/repository_s3/50_repository_ecs_credentials.yml index c59d3a32badc7..99736fb25ff24 100644 --- a/plugins/repository-s3/src/test/resources/rest-api-spec/test/repository_s3/50_repository_ecs_credentials.yml +++ b/plugins/repository-s3/src/test/resources/rest-api-spec/test/repository_s3/50_repository_ecs_credentials.yml @@ -18,6 +18,9 @@ setup: --- "Snapshot and Restore with repository-s3 using ecs credentials": + - skip: + version: " - 7.9.99" + reason: "8.0 changes get snapshots response format" # Get repository - do: @@ -122,9 +125,9 @@ setup: repository: repository_ecs snapshot: snapshot-one,snapshot-two - - is_true: snapshots - - match: { snapshots.0.state : SUCCESS } - - match: { snapshots.1.state : SUCCESS } + - is_true: responses.0.snapshots + - match: { responses.0.snapshots.0.state : SUCCESS } + - match: { responses.0.snapshots.1.state : SUCCESS } # Delete the index - do: @@ -229,13 +232,18 @@ setup: --- "Get a non existing snapshot": + - skip: + version: " - 7.9.99" + reason: "8.0 changes get snapshots response format" - do: - catch: /snapshot_missing_exception/ snapshot.get: repository: repository_ecs snapshot: missing + - is_true: responses.0.error + - match: { responses.0.error.type: snapshot_missing_exception } + --- "Delete a non existing snapshot": diff --git a/plugins/store-smb/src/main/java/org/elasticsearch/index/store/smbmmapfs/SmbMmapFsDirectoryFactory.java b/plugins/store-smb/src/main/java/org/elasticsearch/index/store/smbmmapfs/SmbMmapFsDirectoryFactory.java index 13b6f9401abc5..ea604c7957027 100644 --- a/plugins/store-smb/src/main/java/org/elasticsearch/index/store/smbmmapfs/SmbMmapFsDirectoryFactory.java +++ b/plugins/store-smb/src/main/java/org/elasticsearch/index/store/smbmmapfs/SmbMmapFsDirectoryFactory.java @@ -22,17 +22,20 @@ import org.apache.lucene.store.Directory; import org.apache.lucene.store.LockFactory; import org.apache.lucene.store.MMapDirectory; +import org.elasticsearch.index.IndexModule; import org.elasticsearch.index.IndexSettings; import org.elasticsearch.index.store.FsDirectoryFactory; import org.elasticsearch.index.store.SmbDirectoryWrapper; import java.io.IOException; import java.nio.file.Path; +import java.util.HashSet; public final class SmbMmapFsDirectoryFactory extends FsDirectoryFactory { @Override protected Directory newFSDirectory(Path location, LockFactory lockFactory, IndexSettings indexSettings) throws IOException { - return new SmbDirectoryWrapper(new MMapDirectory(location, lockFactory)); + return new SmbDirectoryWrapper(setPreload(new MMapDirectory(location, lockFactory), lockFactory, new HashSet<>( + indexSettings.getValue(IndexModule.INDEX_STORE_PRE_LOAD_SETTING)))); } } diff --git a/plugins/transport-nio/build.gradle b/plugins/transport-nio/build.gradle index 7800ff6951a89..bf9b42d0558b2 100644 --- a/plugins/transport-nio/build.gradle +++ b/plugins/transport-nio/build.gradle @@ -21,11 +21,10 @@ apply plugin: "nebula.maven-scm" esplugin { description 'The nio transport.' classname 'org.elasticsearch.transport.nio.NioTransportPlugin' - hasClientJar = true } dependencies { - compile "org.elasticsearch:elasticsearch-nio:${version}" + compile project(':libs:elasticsearch-nio') // network stack compile "io.netty:netty-buffer:${versions.netty}" @@ -53,6 +52,10 @@ thirdPartyAudit { 'org.apache.commons.logging.Log', 'org.apache.commons.logging.LogFactory', + // from Log4j (deliberate, Netty will fallback to Log4j 2) + 'org.apache.log4j.Level', + 'org.apache.log4j.Logger', + // from io.netty.handler.ssl.util.BouncyCastleSelfSignedCertGenerator (netty) 'org.bouncycastle.cert.X509v3CertificateBuilder', 'org.bouncycastle.cert.jcajce.JcaX509CertificateConverter', 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 7a4fbfe42aefa..3c3406b0eda45 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 @@ -31,14 +31,14 @@ import io.netty.handler.codec.http.HttpResponseEncoder; import org.elasticsearch.ExceptionsHelper; import org.elasticsearch.common.unit.TimeValue; +import org.elasticsearch.http.CorsHandler; import org.elasticsearch.http.HttpHandlingSettings; import org.elasticsearch.http.HttpPipelinedRequest; import org.elasticsearch.http.HttpReadTimeoutException; -import org.elasticsearch.http.nio.cors.NioCorsConfig; import org.elasticsearch.http.nio.cors.NioCorsHandler; import org.elasticsearch.nio.FlushOperation; import org.elasticsearch.nio.InboundChannelBuffer; -import org.elasticsearch.nio.ReadWriteHandler; +import org.elasticsearch.nio.NioChannelHandler; import org.elasticsearch.nio.SocketChannelContext; import org.elasticsearch.nio.TaskScheduler; import org.elasticsearch.nio.WriteOperation; @@ -50,7 +50,7 @@ import java.util.function.BiConsumer; import java.util.function.LongSupplier; -public class HttpReadWriteHandler implements ReadWriteHandler { +public class HttpReadWriteHandler implements NioChannelHandler { private final NettyAdaptor adaptor; private final NioHttpChannel nioHttpChannel; @@ -63,7 +63,7 @@ public class HttpReadWriteHandler implements ReadWriteHandler { private int inFlightRequests = 0; public HttpReadWriteHandler(NioHttpChannel nioHttpChannel, NioHttpServerTransport transport, HttpHandlingSettings settings, - NioCorsConfig corsConfig, TaskScheduler taskScheduler, LongSupplier nanoClock) { + CorsHandler.Config corsConfig, TaskScheduler taskScheduler, LongSupplier nanoClock) { this.nioHttpChannel = nioHttpChannel; this.transport = transport; this.taskScheduler = taskScheduler; @@ -140,6 +140,11 @@ public List pollFlushOperations() { return copiedOperations; } + @Override + public boolean closeNow() { + return false; + } + @Override public void close() throws IOException { try { diff --git a/plugins/transport-nio/src/main/java/org/elasticsearch/http/nio/NioHttpServerTransport.java b/plugins/transport-nio/src/main/java/org/elasticsearch/http/nio/NioHttpServerTransport.java index fa0f3e9572c98..99a0d87e19c91 100644 --- a/plugins/transport-nio/src/main/java/org/elasticsearch/http/nio/NioHttpServerTransport.java +++ b/plugins/transport-nio/src/main/java/org/elasticsearch/http/nio/NioHttpServerTransport.java @@ -19,14 +19,11 @@ package org.elasticsearch.http.nio; -import io.netty.handler.codec.http.HttpMethod; import org.apache.logging.log4j.LogManager; import org.apache.logging.log4j.Logger; import org.elasticsearch.ElasticsearchException; -import org.elasticsearch.common.Strings; import org.elasticsearch.common.network.NetworkService; import org.elasticsearch.common.settings.Settings; -import org.elasticsearch.common.settings.SettingsException; import org.elasticsearch.common.unit.ByteSizeValue; import org.elasticsearch.common.util.BigArrays; import org.elasticsearch.common.util.PageCacheRecycler; @@ -34,8 +31,6 @@ import org.elasticsearch.http.AbstractHttpServerTransport; import org.elasticsearch.http.HttpChannel; import org.elasticsearch.http.HttpServerChannel; -import org.elasticsearch.http.nio.cors.NioCorsConfig; -import org.elasticsearch.http.nio.cors.NioCorsConfigBuilder; import org.elasticsearch.nio.BytesChannelContext; import org.elasticsearch.nio.ChannelFactory; import org.elasticsearch.nio.InboundChannelBuffer; @@ -44,7 +39,6 @@ import org.elasticsearch.nio.NioSocketChannel; import org.elasticsearch.nio.ServerChannelContext; import org.elasticsearch.nio.SocketChannelContext; -import org.elasticsearch.rest.RestUtils; import org.elasticsearch.threadpool.ThreadPool; import org.elasticsearch.transport.nio.NioGroupFactory; import org.elasticsearch.transport.nio.PageAllocator; @@ -53,17 +47,8 @@ import java.net.InetSocketAddress; import java.nio.channels.ServerSocketChannel; import java.nio.channels.SocketChannel; -import java.util.Arrays; import java.util.function.Consumer; -import java.util.regex.Pattern; -import java.util.regex.PatternSyntaxException; - -import static org.elasticsearch.http.HttpTransportSettings.SETTING_CORS_ALLOW_CREDENTIALS; -import static org.elasticsearch.http.HttpTransportSettings.SETTING_CORS_ALLOW_HEADERS; -import static org.elasticsearch.http.HttpTransportSettings.SETTING_CORS_ALLOW_METHODS; -import static org.elasticsearch.http.HttpTransportSettings.SETTING_CORS_ALLOW_ORIGIN; -import static org.elasticsearch.http.HttpTransportSettings.SETTING_CORS_ENABLED; -import static org.elasticsearch.http.HttpTransportSettings.SETTING_CORS_MAX_AGE; + import static org.elasticsearch.http.HttpTransportSettings.SETTING_HTTP_MAX_CHUNK_SIZE; import static org.elasticsearch.http.HttpTransportSettings.SETTING_HTTP_MAX_HEADER_SIZE; import static org.elasticsearch.http.HttpTransportSettings.SETTING_HTTP_MAX_INITIAL_LINE_LENGTH; @@ -73,12 +58,10 @@ import static org.elasticsearch.http.HttpTransportSettings.SETTING_HTTP_TCP_REUSE_ADDRESS; import static org.elasticsearch.http.HttpTransportSettings.SETTING_HTTP_TCP_SEND_BUFFER_SIZE; import static org.elasticsearch.http.HttpTransportSettings.SETTING_PIPELINING_MAX_EVENTS; -import static org.elasticsearch.http.nio.cors.NioCorsHandler.ANY_ORIGIN; public class NioHttpServerTransport extends AbstractHttpServerTransport { private static final Logger logger = LogManager.getLogger(NioHttpServerTransport.class); - protected final NioCorsConfig corsConfig; protected final PageAllocator pageAllocator; private final NioGroupFactory nioGroupFactory; @@ -102,7 +85,6 @@ public NioHttpServerTransport(Settings settings, NetworkService networkService, ByteSizeValue maxHeaderSize = SETTING_HTTP_MAX_HEADER_SIZE.get(settings); ByteSizeValue maxInitialLineLength = SETTING_HTTP_MAX_INITIAL_LINE_LENGTH.get(settings); int pipeliningMaxEvents = SETTING_PIPELINING_MAX_EVENTS.get(settings); - this.corsConfig = buildCorsConfig(settings); this.tcpNoDelay = SETTING_HTTP_TCP_NO_DELAY.get(settings); this.tcpKeepAlive = SETTING_HTTP_TCP_KEEP_ALIVE.get(settings); @@ -155,42 +137,6 @@ protected ChannelFactory channelFactory() return new HttpChannelFactory(); } - static NioCorsConfig buildCorsConfig(Settings settings) { - if (SETTING_CORS_ENABLED.get(settings) == false) { - return NioCorsConfigBuilder.forOrigins().disable().build(); - } - String origin = SETTING_CORS_ALLOW_ORIGIN.get(settings); - final NioCorsConfigBuilder builder; - if (Strings.isNullOrEmpty(origin)) { - builder = NioCorsConfigBuilder.forOrigins(); - } else if (origin.equals(ANY_ORIGIN)) { - builder = NioCorsConfigBuilder.forAnyOrigin(); - } else { - try { - Pattern p = RestUtils.checkCorsSettingForRegex(origin); - if (p == null) { - builder = NioCorsConfigBuilder.forOrigins(RestUtils.corsSettingAsArray(origin)); - } else { - builder = NioCorsConfigBuilder.forPattern(p); - } - } catch (PatternSyntaxException e) { - throw new SettingsException("Bad regex in [" + SETTING_CORS_ALLOW_ORIGIN.getKey() + "]: [" + origin + "]", e); - } - } - if (SETTING_CORS_ALLOW_CREDENTIALS.get(settings)) { - builder.allowCredentials(); - } - String[] strMethods = Strings.tokenizeToStringArray(SETTING_CORS_ALLOW_METHODS.get(settings), ","); - HttpMethod[] methods = Arrays.stream(strMethods) - .map(HttpMethod::valueOf) - .toArray(HttpMethod[]::new); - return builder.allowedRequestMethods(methods) - .maxAge(SETTING_CORS_MAX_AGE.get(settings)) - .allowedRequestHeaders(Strings.tokenizeToStringArray(SETTING_CORS_ALLOW_HEADERS.get(settings), ",")) - .shortCircuit() - .build(); - } - protected void acceptChannel(NioSocketChannel socketChannel) { super.serverAcceptedChannel((HttpChannel) socketChannel); } diff --git a/plugins/transport-nio/src/main/java/org/elasticsearch/http/nio/cors/NioCorsConfig.java b/plugins/transport-nio/src/main/java/org/elasticsearch/http/nio/cors/NioCorsConfig.java deleted file mode 100644 index 1ffffdf0d315d..0000000000000 --- a/plugins/transport-nio/src/main/java/org/elasticsearch/http/nio/cors/NioCorsConfig.java +++ /dev/null @@ -1,221 +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.http.nio.cors; - -import io.netty.handler.codec.http.DefaultHttpHeaders; -import io.netty.handler.codec.http.EmptyHttpHeaders; -import io.netty.handler.codec.http.HttpHeaders; -import io.netty.handler.codec.http.HttpMethod; - -import java.util.Collections; -import java.util.LinkedHashSet; -import java.util.Map; -import java.util.Optional; -import java.util.Set; -import java.util.concurrent.Callable; -import java.util.regex.Pattern; - -/** - * Configuration for Cross-Origin Resource Sharing (CORS). - * - * This class was lifted from the Netty project: - * https://github.com/netty/netty - */ -public final class NioCorsConfig { - - private final Optional> origins; - private final Optional pattern; - private final boolean anyOrigin; - private final boolean enabled; - private final boolean allowCredentials; - private final long maxAge; - private final Set allowedRequestMethods; - private final Set allowedRequestHeaders; - private final Map> preflightHeaders; - private final boolean shortCircuit; - - NioCorsConfig(final NioCorsConfigBuilder builder) { - origins = builder.origins.map(s -> new LinkedHashSet<>(s)); - pattern = builder.pattern; - anyOrigin = builder.anyOrigin; - enabled = builder.enabled; - allowCredentials = builder.allowCredentials; - maxAge = builder.maxAge; - allowedRequestMethods = builder.requestMethods; - allowedRequestHeaders = builder.requestHeaders; - preflightHeaders = builder.preflightHeaders; - shortCircuit = builder.shortCircuit; - } - - /** - * Determines if support for CORS is enabled. - * - * @return {@code true} if support for CORS is enabled, false otherwise. - */ - public boolean isCorsSupportEnabled() { - return enabled; - } - - /** - * Determines whether a wildcard origin, '*', is supported. This also means that null origins are - * supported. - * - * @return {@code boolean} true if any origin is allowed. - */ - public boolean isAnyOriginSupported() { - return anyOrigin; - } - - /** - * Returns the set of allowed origins. - * - * @return {@code Set} the allowed origins. - */ - public Optional> origins() { - return origins; - } - - /** - * Returns whether the input origin is allowed by this configuration. - * - * @return {@code true} if the origin is allowed, otherwise {@code false} - */ - public boolean isOriginAllowed(final String origin) { - if (origins.isPresent()) { - return origins.get().contains(origin); - } else if (pattern.isPresent()) { - return pattern.get().matcher(origin).matches(); - } - return false; - } - - /** - * Determines if credentials are supported for CORS requests. - * - * By default credentials are not included in CORS requests but if isCredentialsAllowed returns - * true credentials will be added to CORS requests. Setting this value to true will set the - * CORS 'Access-Control-Allow-Credentials' response header to true. - * - * Please note that credentials support needs to be enabled on the client side as well. - * The client needs to opt-in to send credentials by calling: - *

    -     * xhr.withCredentials = true;
    -     * 
    - * The default value for 'withCredentials' is false in which case no credentials are sent. - * Setting this to true will included cookies in cross origin requests. - * - * @return {@code true} if credentials are supported. - */ - public boolean isCredentialsAllowed() { - return allowCredentials; - } - - /** - * Gets the maxAge setting. - * - * When making a preflight request the client has to perform two request with can be inefficient. - * This setting will set the CORS 'Access-Control-Max-Age' response header and enables the - * caching of the preflight response for the specified time. During this time no preflight - * request will be made. - * - * @return {@code long} the time in seconds that a preflight request may be cached. - */ - public long maxAge() { - return maxAge; - } - - /** - * Returns the allowed set of Request Methods. The Http methods that should be returned in the - * CORS 'Access-Control-Request-Method' response header. - * - * @return {@code Set} of {@link HttpMethod}s that represent the allowed Request Methods. - */ - public Set allowedRequestMethods() { - return Collections.unmodifiableSet(allowedRequestMethods); - } - - /** - * Returns the allowed set of Request Headers. - * - * The header names returned from this method will be used to set the CORS - * 'Access-Control-Allow-Headers' response header. - * - * @return {@code Set} of strings that represent the allowed Request Headers. - */ - public Set allowedRequestHeaders() { - return Collections.unmodifiableSet(allowedRequestHeaders); - } - - /** - * Returns HTTP response headers that should be added to a CORS preflight response. - * - * @return {@link HttpHeaders} the HTTP response headers to be added. - */ - public HttpHeaders preflightResponseHeaders() { - if (preflightHeaders.isEmpty()) { - return EmptyHttpHeaders.INSTANCE; - } - final HttpHeaders preflightHeaders = new DefaultHttpHeaders(); - for (Map.Entry> entry : this.preflightHeaders.entrySet()) { - final Object value = getValue(entry.getValue()); - if (value instanceof Iterable) { - preflightHeaders.add(entry.getKey().toString(), (Iterable) value); - } else { - preflightHeaders.add(entry.getKey().toString(), value); - } - } - return preflightHeaders; - } - - /** - * Determines whether a CORS request should be rejected if it's invalid before being - * further processing. - * - * CORS headers are set after a request is processed. This may not always be desired - * and this setting will check that the Origin is valid and if it is not valid no - * further processing will take place, and a error will be returned to the calling client. - * - * @return {@code true} if a CORS request should short-circuit upon receiving an invalid Origin header. - */ - public boolean isShortCircuit() { - return shortCircuit; - } - - private static T getValue(final Callable callable) { - try { - return callable.call(); - } catch (final Exception e) { - throw new IllegalStateException("Could not generate value for callable [" + callable + ']', e); - } - } - - @Override - public String toString() { - return "CorsConfig[enabled=" + enabled + - ", origins=" + origins + - ", anyOrigin=" + anyOrigin + - ", isCredentialsAllowed=" + allowCredentials + - ", maxAge=" + maxAge + - ", allowedRequestMethods=" + allowedRequestMethods + - ", allowedRequestHeaders=" + allowedRequestHeaders + - ", preflightHeaders=" + preflightHeaders + ']'; - } - -} diff --git a/plugins/transport-nio/src/main/java/org/elasticsearch/http/nio/cors/NioCorsConfigBuilder.java b/plugins/transport-nio/src/main/java/org/elasticsearch/http/nio/cors/NioCorsConfigBuilder.java deleted file mode 100644 index 62eda913b0ac7..0000000000000 --- a/plugins/transport-nio/src/main/java/org/elasticsearch/http/nio/cors/NioCorsConfigBuilder.java +++ /dev/null @@ -1,265 +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.http.nio.cors; - -import io.netty.handler.codec.http.HttpMethod; - -import java.util.Arrays; -import java.util.Date; -import java.util.HashMap; -import java.util.HashSet; -import java.util.LinkedHashSet; -import java.util.Map; -import java.util.Optional; -import java.util.Set; -import java.util.concurrent.Callable; -import java.util.regex.Pattern; - -/** - * Builder used to configure and build a {@link NioCorsConfig} instance. - * - * This class was lifted from the Netty project: - * https://github.com/netty/netty - */ -public final class NioCorsConfigBuilder { - - /** - * Creates a Builder instance with it's origin set to '*'. - * - * @return Builder to support method chaining. - */ - public static NioCorsConfigBuilder forAnyOrigin() { - return new NioCorsConfigBuilder(); - } - - /** - * Create a {@link NioCorsConfigBuilder} instance with the specified pattern origin. - * - * @param pattern the regular expression pattern to match incoming origins on. - * @return {@link NioCorsConfigBuilder} with the configured origin pattern. - */ - public static NioCorsConfigBuilder forPattern(final Pattern pattern) { - if (pattern == null) { - throw new IllegalArgumentException("CORS pattern cannot be null"); - } - return new NioCorsConfigBuilder(pattern); - } - - /** - * Creates a {@link NioCorsConfigBuilder} instance with the specified origins. - * - * @return {@link NioCorsConfigBuilder} to support method chaining. - */ - public static NioCorsConfigBuilder forOrigins(final String... origins) { - return new NioCorsConfigBuilder(origins); - } - - Optional> origins; - Optional pattern; - final boolean anyOrigin; - boolean enabled = true; - boolean allowCredentials; - long maxAge; - final Set requestMethods = new HashSet<>(); - final Set requestHeaders = new HashSet<>(); - final Map> preflightHeaders = new HashMap<>(); - boolean shortCircuit; - - /** - * Creates a new Builder instance with the origin passed in. - * - * @param origins the origin to be used for this builder. - */ - NioCorsConfigBuilder(final String... origins) { - this.origins = Optional.of(new LinkedHashSet<>(Arrays.asList(origins))); - pattern = Optional.empty(); - anyOrigin = false; - } - - /** - * Creates a new Builder instance allowing any origin, "*" which is the - * wildcard origin. - * - */ - NioCorsConfigBuilder() { - anyOrigin = true; - origins = Optional.empty(); - pattern = Optional.empty(); - } - - /** - * Creates a new Builder instance allowing any origin that matches the pattern. - * - * @param pattern the pattern to match against for incoming origins. - */ - NioCorsConfigBuilder(final Pattern pattern) { - this.pattern = Optional.of(pattern); - origins = Optional.empty(); - anyOrigin = false; - } - - /** - * Disables CORS support. - * - * @return {@link NioCorsConfigBuilder} to support method chaining. - */ - public NioCorsConfigBuilder disable() { - enabled = false; - return this; - } - - /** - * By default cookies are not included in CORS requests, but this method will enable cookies to - * be added to CORS requests. Calling this method will set the CORS 'Access-Control-Allow-Credentials' - * response header to true. - * - * Please note, that cookie support needs to be enabled on the client side as well. - * The client needs to opt-in to send cookies by calling: - *
    -     * xhr.withCredentials = true;
    -     * 
    - * The default value for 'withCredentials' is false in which case no cookies are sent. - * Setting this to true will included cookies in cross origin requests. - * - * @return {@link NioCorsConfigBuilder} to support method chaining. - */ - public NioCorsConfigBuilder allowCredentials() { - allowCredentials = true; - return this; - } - - /** - * When making a preflight request the client has to perform two request with can be inefficient. - * This setting will set the CORS 'Access-Control-Max-Age' response header and enables the - * caching of the preflight response for the specified time. During this time no preflight - * request will be made. - * - * @param max the maximum time, in seconds, that the preflight response may be cached. - * @return {@link NioCorsConfigBuilder} to support method chaining. - */ - public NioCorsConfigBuilder maxAge(final long max) { - maxAge = max; - return this; - } - - /** - * Specifies the allowed set of HTTP Request Methods that should be returned in the - * CORS 'Access-Control-Request-Method' response header. - * - * @param methods the {@link HttpMethod}s that should be allowed. - * @return {@link NioCorsConfigBuilder} to support method chaining. - */ - public NioCorsConfigBuilder allowedRequestMethods(final HttpMethod... methods) { - requestMethods.addAll(Arrays.asList(methods)); - return this; - } - - /** - * Specifies the if headers that should be returned in the CORS 'Access-Control-Allow-Headers' - * response header. - * - * If a client specifies headers on the request, for example by calling: - *
    -     * xhr.setRequestHeader('My-Custom-Header', "SomeValue");
    -     * 
    - * the server will receive the above header name in the 'Access-Control-Request-Headers' of the - * preflight request. The server will then decide if it allows this header to be sent for the - * real request (remember that a preflight is not the real request but a request asking the server - * if it allow a request). - * - * @param headers the headers to be added to the preflight 'Access-Control-Allow-Headers' response header. - * @return {@link NioCorsConfigBuilder} to support method chaining. - */ - public NioCorsConfigBuilder allowedRequestHeaders(final String... headers) { - requestHeaders.addAll(Arrays.asList(headers)); - return this; - } - - /** - * Specifies that a CORS request should be rejected if it's invalid before being - * further processing. - * - * CORS headers are set after a request is processed. This may not always be desired - * and this setting will check that the Origin is valid and if it is not valid no - * further processing will take place, and a error will be returned to the calling client. - * - * @return {@link NioCorsConfigBuilder} to support method chaining. - */ - public NioCorsConfigBuilder shortCircuit() { - shortCircuit = true; - return this; - } - - /** - * Builds a {@link NioCorsConfig} with settings specified by previous method calls. - * - * @return {@link NioCorsConfig} the configured CorsConfig instance. - */ - public NioCorsConfig build() { - if (preflightHeaders.isEmpty()) { - preflightHeaders.put("date", DateValueGenerator.INSTANCE); - preflightHeaders.put("content-length", new ConstantValueGenerator("0")); - } - return new NioCorsConfig(this); - } - - /** - * This class is used for preflight HTTP response values that do not need to be - * generated, but instead the value is "static" in that the same value will be returned - * for each call. - */ - private static final class ConstantValueGenerator implements Callable { - - private final Object value; - - /** - * Sole constructor. - * - * @param value the value that will be returned when the call method is invoked. - */ - private ConstantValueGenerator(final Object value) { - if (value == null) { - throw new IllegalArgumentException("value must not be null"); - } - this.value = value; - } - - @Override - public Object call() { - return value; - } - } - - /** - * This callable is used for the DATE preflight HTTP response HTTP header. - * It's value must be generated when the response is generated, hence will be - * different for every call. - */ - private static final class DateValueGenerator implements Callable { - - static final DateValueGenerator INSTANCE = new DateValueGenerator(); - - @Override - public Date call() throws Exception { - return new Date(); - } - } - -} diff --git a/plugins/transport-nio/src/main/java/org/elasticsearch/http/nio/cors/NioCorsHandler.java b/plugins/transport-nio/src/main/java/org/elasticsearch/http/nio/cors/NioCorsHandler.java index dfb531992f8ef..bdeffdfa1c715 100644 --- a/plugins/transport-nio/src/main/java/org/elasticsearch/http/nio/cors/NioCorsHandler.java +++ b/plugins/transport-nio/src/main/java/org/elasticsearch/http/nio/cors/NioCorsHandler.java @@ -32,15 +32,17 @@ import io.netty.handler.codec.http.HttpResponse; import io.netty.handler.codec.http.HttpResponseStatus; import org.elasticsearch.common.Strings; +import org.elasticsearch.http.CorsHandler; import org.elasticsearch.http.nio.NioHttpResponse; +import java.util.Date; import java.util.regex.Pattern; import java.util.stream.Collectors; /** * Handles Cross Origin Resource Sharing (CORS) requests. *

    - * This handler can be configured using a {@link NioCorsConfig}, please + * This handler can be configured using a {@link CorsHandler.Config}, please * refer to this class for details about the configuration options available. * * This code was borrowed from Netty 4 and refactored to work for Elasticsearch's Netty 3 setup. @@ -50,13 +52,13 @@ public class NioCorsHandler extends ChannelDuplexHandler { public static final String ANY_ORIGIN = "*"; private static Pattern SCHEME_PATTERN = Pattern.compile("^https?://"); - private final NioCorsConfig config; + private final CorsHandler.Config config; private FullHttpRequest request; /** - * Creates a new instance with the specified {@link NioCorsConfig}. + * Creates a new instance with the specified {@link CorsHandler.Config}. */ - public NioCorsHandler(final NioCorsConfig config) { + public NioCorsHandler(final CorsHandler.Config config) { if (config == null) { throw new NullPointerException(); } @@ -76,7 +78,7 @@ public void channelRead(ChannelHandlerContext ctx, Object msg) throws Exception releaseRequest(); } } - if (config.isShortCircuit() && !validateOrigin()) { + if (!validateOrigin()) { try { forbidden(ctx, request); return; @@ -96,7 +98,7 @@ public void write(ChannelHandlerContext ctx, Object msg, ChannelPromise promise) ctx.write(response, promise); } - public static void setCorsResponseHeaders(HttpRequest request, HttpResponse resp, NioCorsConfig config) { + public static void setCorsResponseHeaders(HttpRequest request, HttpResponse resp, CorsHandler.Config config) { if (!config.isCorsSupportEnabled()) { return; } @@ -161,7 +163,8 @@ private static boolean isSameOrigin(final String origin, final String host) { * @param response the HttpResponse to which the preflight response headers should be added. */ private void setPreflightHeaders(final HttpResponse response) { - response.headers().add(config.preflightResponseHeaders()); + response.headers().add("date", new Date()); + response.headers().add("content-length", "0"); } private boolean setOrigin(final HttpResponse response) { diff --git a/plugins/transport-nio/src/test/java/org/elasticsearch/http/nio/HttpReadWriteHandlerTests.java b/plugins/transport-nio/src/test/java/org/elasticsearch/http/nio/HttpReadWriteHandlerTests.java index 93a846ea90f81..9ee66341a487a 100644 --- a/plugins/transport-nio/src/test/java/org/elasticsearch/http/nio/HttpReadWriteHandlerTests.java +++ b/plugins/transport-nio/src/test/java/org/elasticsearch/http/nio/HttpReadWriteHandlerTests.java @@ -37,14 +37,13 @@ import org.elasticsearch.common.settings.Settings; import org.elasticsearch.common.unit.ByteSizeValue; import org.elasticsearch.common.unit.TimeValue; +import org.elasticsearch.http.CorsHandler; import org.elasticsearch.http.HttpChannel; import org.elasticsearch.http.HttpHandlingSettings; import org.elasticsearch.http.HttpReadTimeoutException; import org.elasticsearch.http.HttpRequest; import org.elasticsearch.http.HttpResponse; import org.elasticsearch.http.HttpTransportSettings; -import org.elasticsearch.http.nio.cors.NioCorsConfig; -import org.elasticsearch.http.nio.cors.NioCorsConfigBuilder; import org.elasticsearch.http.nio.cors.NioCorsHandler; import org.elasticsearch.nio.FlushOperation; import org.elasticsearch.nio.InboundChannelBuffer; @@ -98,7 +97,7 @@ public void setMocks() { channel = mock(NioHttpChannel.class); taskScheduler = mock(TaskScheduler.class); - NioCorsConfig corsConfig = NioCorsConfigBuilder.forAnyOrigin().build(); + CorsHandler.Config corsConfig = CorsHandler.disabled(); handler = new HttpReadWriteHandler(channel, transport, httpHandlingSettings, corsConfig, taskScheduler, System::nanoTime); handler.channelRegistered(); } @@ -329,7 +328,7 @@ public void testReadTimeout() throws IOException { Settings settings = Settings.builder().put(SETTING_HTTP_READ_TIMEOUT.getKey(), timeValue).build(); HttpHandlingSettings httpHandlingSettings = HttpHandlingSettings.fromSettings(settings); - NioCorsConfig corsConfig = NioCorsConfigBuilder.forAnyOrigin().build(); + CorsHandler.Config corsConfig = CorsHandler.disabled(); TaskScheduler taskScheduler = new TaskScheduler(); Iterator timeValues = Arrays.asList(0, 2, 4, 6, 8).iterator(); @@ -378,7 +377,7 @@ private static NioHttpResponse emptyGetResponse(int sequenceNumber) { private FullHttpResponse executeCorsRequest(final Settings settings, final String originValue, final String host) throws IOException { HttpHandlingSettings httpSettings = HttpHandlingSettings.fromSettings(settings); - NioCorsConfig corsConfig = NioHttpServerTransport.buildCorsConfig(settings); + CorsHandler.Config corsConfig = CorsHandler.fromSettings(settings); HttpReadWriteHandler handler = new HttpReadWriteHandler(channel, transport, httpSettings, corsConfig, taskScheduler, System::nanoTime); handler.channelRegistered(); diff --git a/plugins/transport-nio/src/test/java/org/elasticsearch/http/nio/NioHttpClient.java b/plugins/transport-nio/src/test/java/org/elasticsearch/http/nio/NioHttpClient.java index 634ea7b44af74..ed55007f3ba61 100644 --- a/plugins/transport-nio/src/test/java/org/elasticsearch/http/nio/NioHttpClient.java +++ b/plugins/transport-nio/src/test/java/org/elasticsearch/http/nio/NioHttpClient.java @@ -49,7 +49,7 @@ import org.elasticsearch.nio.NioSelector; import org.elasticsearch.nio.NioServerSocketChannel; import org.elasticsearch.nio.NioSocketChannel; -import org.elasticsearch.nio.ReadWriteHandler; +import org.elasticsearch.nio.NioChannelHandler; import org.elasticsearch.nio.SocketChannelContext; import org.elasticsearch.nio.WriteOperation; import org.elasticsearch.tasks.Task; @@ -207,7 +207,7 @@ public NioServerSocketChannel createServerChannel(NioSelector selector, ServerSo } } - private static class HttpClientHandler implements ReadWriteHandler { + private static class HttpClientHandler implements NioChannelHandler { private final NettyAdaptor adaptor; private final CountDownLatch latch; @@ -277,6 +277,11 @@ public int consumeReads(InboundChannelBuffer channelBuffer) throws IOException { return bytesConsumed; } + @Override + public boolean closeNow() { + return false; + } + @Override public void close() throws IOException { try { diff --git a/plugins/transport-nio/src/test/java/org/elasticsearch/http/nio/NioHttpServerTransportTests.java b/plugins/transport-nio/src/test/java/org/elasticsearch/http/nio/NioHttpServerTransportTests.java index 0b470fda00a6b..bc3208eb29809 100644 --- a/plugins/transport-nio/src/test/java/org/elasticsearch/http/nio/NioHttpServerTransportTests.java +++ b/plugins/transport-nio/src/test/java/org/elasticsearch/http/nio/NioHttpServerTransportTests.java @@ -32,12 +32,10 @@ import io.netty.handler.codec.http.HttpUtil; import io.netty.handler.codec.http.HttpVersion; import org.elasticsearch.ElasticsearchException; -import org.elasticsearch.common.Strings; import org.elasticsearch.common.bytes.BytesArray; import org.elasticsearch.common.network.NetworkService; import org.elasticsearch.common.settings.Setting; import org.elasticsearch.common.settings.Settings; -import org.elasticsearch.common.settings.SettingsException; import org.elasticsearch.common.transport.TransportAddress; import org.elasticsearch.common.unit.ByteSizeValue; import org.elasticsearch.common.unit.TimeValue; @@ -48,7 +46,6 @@ import org.elasticsearch.http.HttpServerTransport; import org.elasticsearch.http.HttpTransportSettings; import org.elasticsearch.http.NullDispatcher; -import org.elasticsearch.http.nio.cors.NioCorsConfig; import org.elasticsearch.indices.breaker.NoneCircuitBreakerService; import org.elasticsearch.nio.NioSocketChannel; import org.elasticsearch.rest.BytesRestResponse; @@ -64,22 +61,11 @@ import java.io.IOException; import java.nio.charset.Charset; import java.nio.charset.StandardCharsets; -import java.util.Arrays; import java.util.Collections; -import java.util.HashSet; -import java.util.Set; import java.util.concurrent.CountDownLatch; import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicReference; -import java.util.regex.PatternSyntaxException; -import java.util.stream.Collectors; - -import static org.elasticsearch.http.HttpTransportSettings.SETTING_CORS_ALLOW_CREDENTIALS; -import static org.elasticsearch.http.HttpTransportSettings.SETTING_CORS_ALLOW_HEADERS; -import static org.elasticsearch.http.HttpTransportSettings.SETTING_CORS_ALLOW_METHODS; -import static org.elasticsearch.http.HttpTransportSettings.SETTING_CORS_ALLOW_ORIGIN; -import static org.elasticsearch.http.HttpTransportSettings.SETTING_CORS_ENABLED; -import static org.elasticsearch.http.HttpTransportSettings.SETTING_CORS_MAX_AGE; + import static org.elasticsearch.rest.RestStatus.BAD_REQUEST; import static org.elasticsearch.rest.RestStatus.OK; import static org.hamcrest.Matchers.containsString; @@ -115,48 +101,6 @@ public void shutdown() throws Exception { bigArrays = null; } - public void testCorsConfig() { - final Set methods = new HashSet<>(Arrays.asList("get", "options", "post")); - final Set headers = new HashSet<>(Arrays.asList("Content-Type", "Content-Length")); - final String prefix = randomBoolean() ? " " : ""; // sometimes have a leading whitespace between comma delimited elements - final Settings settings = Settings.builder() - .put(SETTING_CORS_ENABLED.getKey(), true) - .put(SETTING_CORS_ALLOW_ORIGIN.getKey(), "*") - .put(SETTING_CORS_ALLOW_METHODS.getKey(), Strings.collectionToDelimitedString(methods, ",", prefix, "")) - .put(SETTING_CORS_ALLOW_HEADERS.getKey(), Strings.collectionToDelimitedString(headers, ",", prefix, "")) - .put(SETTING_CORS_ALLOW_CREDENTIALS.getKey(), true) - .build(); - final NioCorsConfig corsConfig = NioHttpServerTransport.buildCorsConfig(settings); - assertTrue(corsConfig.isAnyOriginSupported()); - assertEquals(headers, corsConfig.allowedRequestHeaders()); - assertEquals(methods, corsConfig.allowedRequestMethods().stream().map(HttpMethod::name).collect(Collectors.toSet())); - } - - public void testCorsConfigWithDefaults() { - final Set methods = Strings.commaDelimitedListToSet(SETTING_CORS_ALLOW_METHODS.getDefault(Settings.EMPTY)); - final Set headers = Strings.commaDelimitedListToSet(SETTING_CORS_ALLOW_HEADERS.getDefault(Settings.EMPTY)); - final long maxAge = SETTING_CORS_MAX_AGE.getDefault(Settings.EMPTY); - final Settings settings = Settings.builder().put(SETTING_CORS_ENABLED.getKey(), true).build(); - final NioCorsConfig corsConfig = NioHttpServerTransport.buildCorsConfig(settings); - assertFalse(corsConfig.isAnyOriginSupported()); - assertEquals(Collections.emptySet(), corsConfig.origins().get()); - assertEquals(headers, corsConfig.allowedRequestHeaders()); - assertEquals(methods, corsConfig.allowedRequestMethods().stream().map(HttpMethod::name).collect(Collectors.toSet())); - assertEquals(maxAge, corsConfig.maxAge()); - assertFalse(corsConfig.isCredentialsAllowed()); - } - - public void testCorsConfigWithBadRegex() { - final Settings settings = Settings.builder() - .put(SETTING_CORS_ENABLED.getKey(), true) - .put(SETTING_CORS_ALLOW_ORIGIN.getKey(), "/[*/") - .put(SETTING_CORS_ALLOW_CREDENTIALS.getKey(), true) - .build(); - SettingsException e = expectThrows(SettingsException.class, () -> NioHttpServerTransport.buildCorsConfig(settings)); - assertThat(e.getMessage(), containsString("Bad regex in [http.cors.allow-origin]: [/[*/]")); - assertThat(e.getCause(), instanceOf(PatternSyntaxException.class)); - } - /** * Test that {@link NioHttpServerTransport} supports the "Expect: 100-continue" HTTP header * @throws InterruptedException if the client communication with the server is interrupted diff --git a/qa/build.gradle b/qa/build.gradle index f1727f115155d..9266a09b25735 100644 --- a/qa/build.gradle +++ b/qa/build.gradle @@ -1,25 +1,20 @@ import org.elasticsearch.gradle.test.RestIntegTestTask +import org.elasticsearch.gradle.testclusters.TestClustersPlugin subprojects { Project subproj -> subproj.tasks.withType(RestIntegTestTask) { - subproj.extensions.configure("${it.name}Cluster") { cluster -> - cluster.distribution = System.getProperty('tests.distribution', 'oss') - if (cluster.distribution == 'default') { - /* - * Add Elastic's repositories so we can resolve older versions of the - * default distribution. Those aren't in maven central. - */ - repositories { - maven { - name "elastic" - url "https://artifacts.elastic.co/maven" - } - maven { - name "elastic-snapshots" - url "https://snapshots.elastic.co/maven" - } - } + if (subproj.extensions.findByName("${it.name}Cluster")) { + subproj.extensions.configure("${it.name}Cluster") { cluster -> + cluster.distribution = System.getProperty('tests.distribution', 'oss') + } + } + } + plugins.withType(TestClustersPlugin).whenPluginAdded { + afterEvaluate { + // We need to delay this so it's not overwritten in RestIntegTestTask + testClusters.all { + distribution = System.getProperty('tests.distribution', 'oss').toUpperCase() } } } diff --git a/qa/ccs-unavailable-clusters/build.gradle b/qa/ccs-unavailable-clusters/build.gradle index c1f2bc9627108..749623b26c329 100644 --- a/qa/ccs-unavailable-clusters/build.gradle +++ b/qa/ccs-unavailable-clusters/build.gradle @@ -16,10 +16,11 @@ * specific language governing permissions and limitations * under the License. */ +apply plugin: 'elasticsearch.testclusters' apply plugin: 'elasticsearch.standalone-rest-test' apply plugin: 'elasticsearch.rest-test' apply plugin: 'elasticsearch.test-with-dependencies' dependencies { - testCompile "org.elasticsearch.client:elasticsearch-rest-high-level-client:${version}" + testCompile project(":client:rest-high-level") } diff --git a/qa/die-with-dignity/build.gradle b/qa/die-with-dignity/build.gradle index 140df6e283ab8..a40f6366e6515 100644 --- a/qa/die-with-dignity/build.gradle +++ b/qa/die-with-dignity/build.gradle @@ -17,6 +17,7 @@ * under the License. */ +apply plugin: 'elasticsearch.testclusters' apply plugin: 'elasticsearch.esplugin' esplugin { @@ -24,14 +25,17 @@ esplugin { classname 'org.elasticsearch.DieWithDignityPlugin' } -integTestRunner { +integTest.runner { systemProperty 'tests.security.manager', 'false' systemProperty 'tests.system_call_filter', 'false' - nonInputProperties.systemProperty 'pidfile', "${-> integTest.getNodes().get(0).pidFile}" - nonInputProperties.systemProperty 'log', "${-> integTest.getNodes().get(0).homeDir}/logs/${-> integTest.getNodes().get(0).clusterName}_server.json" + nonInputProperties.systemProperty 'log', "${-> testClusters.integTest.singleNode().getServerLog()}" systemProperty 'runtime.java.home', "${project.runtimeJavaHome}" } +testClusters.integTest { + systemProperty "die.with.dignity.test", "whatever" +} + test.enabled = false check.dependsOn integTest diff --git a/qa/die-with-dignity/src/main/java/org/elasticsearch/DieWithDignityPlugin.java b/qa/die-with-dignity/src/main/java/org/elasticsearch/DieWithDignityPlugin.java index ed1e3d3879a5d..8027eeb8948d0 100644 --- a/qa/die-with-dignity/src/main/java/org/elasticsearch/DieWithDignityPlugin.java +++ b/qa/die-with-dignity/src/main/java/org/elasticsearch/DieWithDignityPlugin.java @@ -36,6 +36,10 @@ public class DieWithDignityPlugin extends Plugin implements ActionPlugin { + public DieWithDignityPlugin() { + assert System.getProperty("die.with.dignity.test") != null : "test should pass the `die.with.dignity.test` property"; + } + @Override public List getRestHandlers( final Settings settings, diff --git a/qa/die-with-dignity/src/test/java/org/elasticsearch/qa/die_with_dignity/DieWithDignityIT.java b/qa/die-with-dignity/src/test/java/org/elasticsearch/qa/die_with_dignity/DieWithDignityIT.java index 16398b380cfe1..c6350f92ae78f 100644 --- a/qa/die-with-dignity/src/test/java/org/elasticsearch/qa/die_with_dignity/DieWithDignityIT.java +++ b/qa/die-with-dignity/src/test/java/org/elasticsearch/qa/die_with_dignity/DieWithDignityIT.java @@ -19,16 +19,10 @@ package org.elasticsearch.qa.die_with_dignity; -import org.apache.http.ConnectionClosedException; -import org.apache.lucene.util.Constants; -import org.elasticsearch.cli.Terminal; import org.elasticsearch.client.Request; import org.elasticsearch.common.io.PathUtils; -import org.elasticsearch.common.logging.JsonLogLine; -import org.elasticsearch.common.logging.JsonLogsStream; +import org.elasticsearch.common.settings.Settings; import org.elasticsearch.test.rest.ESRestTestCase; -import org.hamcrest.Matcher; -import org.hamcrest.Matchers; import java.io.BufferedReader; import java.io.IOException; @@ -38,82 +32,55 @@ import java.nio.file.Path; import java.util.Iterator; import java.util.List; -import java.util.stream.Stream; import static org.hamcrest.Matchers.containsString; -import static org.hamcrest.Matchers.either; -import static org.hamcrest.Matchers.equalTo; -import static org.hamcrest.Matchers.hasItem; -import static org.hamcrest.Matchers.hasSize; -import static org.hamcrest.Matchers.hasToString; -import static org.hamcrest.Matchers.instanceOf; import static org.hamcrest.Matchers.not; public class DieWithDignityIT extends ESRestTestCase { public void testDieWithDignity() throws Exception { - // deleting the PID file prevents stopping the cluster from failing since it occurs if and only if the PID file exists - final Path pidFile = PathUtils.get(System.getProperty("pidfile")); - final List pidFileLines = Files.readAllLines(pidFile); - assertThat(pidFileLines, hasSize(1)); - final int pid = Integer.parseInt(pidFileLines.get(0)); - Files.delete(pidFile); - IOException e = expectThrows(IOException.class, - () -> client().performRequest(new Request("GET", "/_die_with_dignity"))); - Matcher failureMatcher = instanceOf(ConnectionClosedException.class); - if (Constants.WINDOWS) { - /* - * If the other side closes the connection while we're waiting to fill our buffer - * we can get IOException with the message below. It seems to only come up on - * Windows and it *feels* like it could be a ConnectionClosedException but - * upstream does not consider this a bug: - * https://issues.apache.org/jira/browse/HTTPASYNC-134 - * - * So we catch it here and consider it "ok". - */ - failureMatcher = either(failureMatcher) - .or(hasToString(containsString("An existing connection was forcibly closed by the remote host"))); - } - assertThat(e, failureMatcher); + expectThrows( + IOException.class, + () -> client().performRequest(new Request("GET", "/_die_with_dignity")) + ); // the Elasticsearch process should die and disappear from the output of jps assertBusy(() -> { final String jpsPath = PathUtils.get(System.getProperty("runtime.java.home"), "bin/jps").toString(); - final Process process = new ProcessBuilder().command(jpsPath).start(); - assertThat(process.waitFor(), equalTo(0)); + final Process process = new ProcessBuilder().command(jpsPath, "-v").start(); + try (InputStream is = process.getInputStream(); BufferedReader in = new BufferedReader(new InputStreamReader(is, "UTF-8"))) { String line; while ((line = in.readLine()) != null) { - final int currentPid = Integer.parseInt(line.split("\\s+")[0]); - assertThat(line, pid, not(equalTo(currentPid))); + assertThat(line, line, not(containsString("-Ddie.with.dignity.test"))); } } }); + // parse the logs and ensure that Elasticsearch died with the expected cause + final List lines = Files.readAllLines(PathUtils.get(System.getProperty("log"))); + + final Iterator it = lines.iterator(); + + boolean fatalError = false; + boolean fatalErrorInThreadExiting = false; try { - // parse the logs and ensure that Elasticsearch died with the expected cause - Path path = PathUtils.get(System.getProperty("log")); - try (Stream stream = JsonLogsStream.from(path)) { - final Iterator it = stream.iterator(); - - boolean fatalError = false; - boolean fatalErrorInThreadExiting = false; - - while (it.hasNext() && (fatalError == false || fatalErrorInThreadExiting == false)) { - final JsonLogLine line = it.next(); - if (isFatalError(line)) { - fatalError = true; - } else if (isFatalErrorInThreadExiting(line) || isWarnExceptionReceived(line)) { - fatalErrorInThreadExiting = true; - assertThat(line.stacktrace(), - hasItem(Matchers.containsString("java.lang.OutOfMemoryError: die with dignity"))); - } + while (it.hasNext() && (fatalError == false || fatalErrorInThreadExiting == false)) { + final String line = it.next(); + if (line.matches(".*ERROR.*o\\.e\\.ExceptionsHelper.*integTest-0.*fatal error.*")) { + fatalError = true; + } else if (line.matches(".*ERROR.*o\\.e\\.b\\.ElasticsearchUncaughtExceptionHandler.*integTest-0.*" + + "fatal error in thread \\[Thread-\\d+\\], exiting.*")) { + fatalErrorInThreadExiting = true; + assertTrue(it.hasNext()); + assertThat(it.next(), containsString("java.lang.OutOfMemoryError: die with dignity")); } - - assertTrue(fatalError); - assertTrue(fatalErrorInThreadExiting); } + + assertTrue(fatalError); + assertTrue(fatalErrorInThreadExiting); + } catch (AssertionError ae) { Path path = PathUtils.get(System.getProperty("log")); debugLogs(path); @@ -121,38 +88,26 @@ public void testDieWithDignity() throws Exception { } } - private boolean isWarnExceptionReceived(JsonLogLine line) { - return line.level().equals("WARN") - && line.component().equals("o.e.h.AbstractHttpServerTransport") - && line.nodeName().equals("node-0") - && line.message().contains("caught exception while handling client http traffic"); - } - private void debugLogs(Path path) throws IOException { try (BufferedReader reader = Files.newBufferedReader(path)) { - Terminal terminal = Terminal.DEFAULT; - reader.lines().forEach(line -> terminal.println(line)); + reader.lines().forEach(line -> logger.info(line)); } } - private boolean isFatalErrorInThreadExiting(JsonLogLine line) { - return line.level().equals("ERROR") - && line.component().equals("o.e.b.ElasticsearchUncaughtExceptionHandler") - && line.nodeName().equals("node-0") - && line.message().matches("fatal error in thread \\[Thread-\\d+\\], exiting$"); - } - - private boolean isFatalError(JsonLogLine line) { - return line.level().equals("ERROR") - && line.component().equals("o.e.ExceptionsHelper") - && line.nodeName().equals("node-0") - && line.message().contains("fatal error"); - } - @Override protected boolean preserveClusterUponCompletion() { // as the cluster is dead its state can not be wiped successfully so we have to bypass wiping the cluster return true; } + @Override + protected final Settings restClientSettings() { + return Settings.builder().put(super.restClientSettings()) + // increase the timeout here to 90 seconds to handle long waits for a green + // cluster health. the waits for green need to be longer than a minute to + // account for delayed shards + .put(ESRestTestCase.CLIENT_SOCKET_TIMEOUT, "1s") + .build(); + } + } diff --git a/qa/evil-tests/build.gradle b/qa/evil-tests/build.gradle index 2f9239e5c2f22..38c1b3e1a9aa9 100644 --- a/qa/evil-tests/build.gradle +++ b/qa/evil-tests/build.gradle @@ -23,6 +23,7 @@ * threads, etc. */ +apply plugin: 'elasticsearch.testclusters' apply plugin: 'elasticsearch.standalone-test' dependencies { diff --git a/qa/evil-tests/src/test/java/org/elasticsearch/env/NodeEnvironmentEvilTests.java b/qa/evil-tests/src/test/java/org/elasticsearch/env/NodeEnvironmentEvilTests.java index 44d3c2a88a55b..49e30ac4b5ed3 100644 --- a/qa/evil-tests/src/test/java/org/elasticsearch/env/NodeEnvironmentEvilTests.java +++ b/qa/evil-tests/src/test/java/org/elasticsearch/env/NodeEnvironmentEvilTests.java @@ -51,10 +51,11 @@ public void testMissingWritePermission() throws IOException { Settings build = Settings.builder() .put(Environment.PATH_HOME_SETTING.getKey(), createTempDir().toAbsolutePath().toString()) .putList(Environment.PATH_DATA_SETTING.getKey(), tempPaths).build(); - IOException exception = expectThrows(IOException.class, () -> { + IllegalStateException exception = expectThrows(IllegalStateException.class, () -> { new NodeEnvironment(build, TestEnvironment.newEnvironment(build)); }); - assertTrue(exception.getMessage(), exception.getMessage().startsWith(path.toString())); + assertTrue(exception.getCause().getCause().getMessage(), + exception.getCause().getCause().getMessage().startsWith(path.toString())); } } @@ -62,7 +63,7 @@ public void testMissingWritePermissionOnIndex() throws IOException { assumeTrue("posix filesystem", isPosix); final String[] tempPaths = tmpPaths(); Path path = PathUtils.get(randomFrom(tempPaths)); - Path fooIndex = path.resolve("nodes").resolve("0").resolve(NodeEnvironment.INDICES_FOLDER) + Path fooIndex = path.resolve(NodeEnvironment.INDICES_FOLDER) .resolve("foo"); Files.createDirectories(fooIndex); try (PosixPermissionsResetter attr = new PosixPermissionsResetter(fooIndex)) { @@ -82,7 +83,7 @@ public void testMissingWritePermissionOnShard() throws IOException { assumeTrue("posix filesystem", isPosix); final String[] tempPaths = tmpPaths(); Path path = PathUtils.get(randomFrom(tempPaths)); - Path fooIndex = path.resolve("nodes").resolve("0").resolve(NodeEnvironment.INDICES_FOLDER) + Path fooIndex = path.resolve(NodeEnvironment.INDICES_FOLDER) .resolve("foo"); Path fooShard = fooIndex.resolve("0"); Path fooShardIndex = fooShard.resolve("index"); diff --git a/qa/full-cluster-restart/src/test/java/org/elasticsearch/upgrades/FullClusterRestartIT.java b/qa/full-cluster-restart/src/test/java/org/elasticsearch/upgrades/FullClusterRestartIT.java index 50eee32810adc..350ef19ccc6c5 100644 --- a/qa/full-cluster-restart/src/test/java/org/elasticsearch/upgrades/FullClusterRestartIT.java +++ b/qa/full-cluster-restart/src/test/java/org/elasticsearch/upgrades/FullClusterRestartIT.java @@ -917,7 +917,7 @@ public void testSoftDeletes() throws Exception { mappingsAndSettings.startObject("settings"); mappingsAndSettings.field("number_of_shards", 1); mappingsAndSettings.field("number_of_replicas", 1); - if (getOldClusterVersion().onOrAfter(Version.V_6_5_0) && randomBoolean()) { + if (randomBoolean()) { mappingsAndSettings.field("soft_deletes.enabled", true); } mappingsAndSettings.endObject(); @@ -961,7 +961,6 @@ public void testSoftDeletes() throws Exception { * This test creates an index in the old cluster and then closes it. When the cluster is fully restarted in a newer version, * it verifies that the index exists and is replicated if the old version supports replication. */ - @AwaitsFix(bugUrl = "https://github.com/elastic/elasticsearch/issues/39576") public void testClosedIndices() throws Exception { if (isRunningAgainstOldCluster()) { createIndex(index, Settings.builder() @@ -1049,19 +1048,26 @@ private void assertClosedIndex(final String index, final boolean checkRoutingTab } } + @SuppressWarnings("unchecked") private void checkSnapshot(final String snapshotName, final int count, final Version tookOnVersion) throws IOException { // Check the snapshot metadata, especially the version Request listSnapshotRequest = new Request("GET", "/_snapshot/repo/" + snapshotName); - Map listSnapshotResponse = entityAsMap(client().performRequest(listSnapshotRequest)); - assertEquals(singletonList(snapshotName), XContentMapValues.extractValue("snapshots.snapshot", listSnapshotResponse)); - assertEquals(singletonList("SUCCESS"), XContentMapValues.extractValue("snapshots.state", listSnapshotResponse)); - assertEquals(singletonList(tookOnVersion.toString()), XContentMapValues.extractValue("snapshots.version", listSnapshotResponse)); + Map responseMap = entityAsMap(client().performRequest(listSnapshotRequest)); + Map snapResponse; + if (responseMap.get("responses") != null) { + snapResponse = (Map) ((List) responseMap.get("responses")).get(0); + } else { + snapResponse = responseMap; + } + + assertEquals(singletonList(snapshotName), XContentMapValues.extractValue("snapshots.snapshot", snapResponse)); + assertEquals(singletonList("SUCCESS"), XContentMapValues.extractValue("snapshots.state", snapResponse)); + assertEquals(singletonList(tookOnVersion.toString()), XContentMapValues.extractValue("snapshots.version", snapResponse)); // Remove the routing setting and template so we can test restoring them. Request clearRoutingFromSettings = new Request("PUT", "/_cluster/settings"); clearRoutingFromSettings.setJsonEntity("{\"persistent\":{\"cluster.routing.allocation.exclude.test_attr\": null}}"); client().performRequest(clearRoutingFromSettings); - client().performRequest(new Request("DELETE", "/_template/test_template")); // Restore diff --git a/qa/full-cluster-restart/src/test/java/org/elasticsearch/upgrades/QueryBuilderBWCIT.java b/qa/full-cluster-restart/src/test/java/org/elasticsearch/upgrades/QueryBuilderBWCIT.java index 861a774ee17d2..2c62facbfc7cc 100644 --- a/qa/full-cluster-restart/src/test/java/org/elasticsearch/upgrades/QueryBuilderBWCIT.java +++ b/qa/full-cluster-restart/src/test/java/org/elasticsearch/upgrades/QueryBuilderBWCIT.java @@ -188,7 +188,7 @@ public void testQueryBuilderBWC() throws Exception { assertEquals(201, rsp.getStatusLine().getStatusCode()); } } else { - NamedWriteableRegistry registry = new NamedWriteableRegistry(new SearchModule(Settings.EMPTY, false, + NamedWriteableRegistry registry = new NamedWriteableRegistry(new SearchModule(Settings.EMPTY, Collections.emptyList()).getNamedWriteables()); for (int i = 0; i < CANDIDATES.size(); i++) { diff --git a/qa/logging-config/build.gradle b/qa/logging-config/build.gradle index 544614ef26958..438079b9942fe 100644 --- a/qa/logging-config/build.gradle +++ b/qa/logging-config/build.gradle @@ -17,22 +17,22 @@ * under the License. */ - +apply plugin: 'elasticsearch.testclusters' apply plugin: 'elasticsearch.standalone-rest-test' apply plugin: 'elasticsearch.rest-test' apply plugin: 'elasticsearch.standalone-test' -integTestCluster { +testClusters.integTest { /** * Provide a custom log4j configuration where layout is an old style pattern and confirm that Elasticsearch * can successfully startup. */ - extraConfigFile 'log4j2.properties', 'custom-log4j2.properties' + extraConfigFile 'log4j2.properties', file('custom-log4j2.properties') } -integTestRunner { +integTest.runner { nonInputProperties.systemProperty 'tests.logfile', - "${ -> integTest.nodes[0].homeDir}/logs/${ -> integTest.nodes[0].clusterName }_server.log" + "${ -> testClusters.integTest.singleNode().getServerLog().absolutePath.replaceAll(".json", ".log")}" } test { diff --git a/qa/logging-config/src/test/java/org/elasticsearch/common/logging/JsonLoggerTests.java b/qa/logging-config/src/test/java/org/elasticsearch/common/logging/JsonLoggerTests.java index bbb20737c4708..2416eb02bfd4e 100644 --- a/qa/logging-config/src/test/java/org/elasticsearch/common/logging/JsonLoggerTests.java +++ b/qa/logging-config/src/test/java/org/elasticsearch/common/logging/JsonLoggerTests.java @@ -19,7 +19,7 @@ package org.elasticsearch.common.logging; -import org.apache.log4j.Level; +import org.apache.logging.log4j.Level; import org.apache.logging.log4j.LogManager; import org.apache.logging.log4j.Logger; import org.apache.logging.log4j.core.LoggerContext; @@ -67,7 +67,6 @@ public void tearDown() throws Exception { super.tearDown(); } - @SuppressWarnings("unchecked") public void testJsonLayout() throws IOException { final Logger testLogger = LogManager.getLogger("test"); @@ -90,7 +89,6 @@ public void testJsonLayout() throws IOException { } } - @SuppressWarnings("unchecked") public void testPrefixLoggerInJson() throws IOException { Logger shardIdLogger = Loggers.getLogger("shardIdLogger", ShardId.fromString("[indexName][123]")); shardIdLogger.info("This is an info message with a shardId"); diff --git a/qa/logging-config/src/test/java/org/elasticsearch/qa/custom_logging/CustomLoggingConfigIT.java b/qa/logging-config/src/test/java/org/elasticsearch/qa/custom_logging/CustomLoggingConfigIT.java index 2e6b32d79fc41..6ab9a00c1e9f5 100644 --- a/qa/logging-config/src/test/java/org/elasticsearch/qa/custom_logging/CustomLoggingConfigIT.java +++ b/qa/logging-config/src/test/java/org/elasticsearch/qa/custom_logging/CustomLoggingConfigIT.java @@ -40,7 +40,7 @@ * The intention is to confirm that users can still run their Elasticsearch instances with previous configurations. */ public class CustomLoggingConfigIT extends ESRestTestCase { - private static final String NODE_STARTED = ".*node-0.*cluster.uuid.*node.id.*recovered.*cluster_state.*"; + private static final String NODE_STARTED = ".*integTest-0.*cluster.uuid.*node.id.*recovered.*cluster_state.*"; public void testSuccessfulStartupWithCustomConfig() throws Exception { assertBusy(() -> { diff --git a/qa/multi-cluster-search/build.gradle b/qa/multi-cluster-search/build.gradle index bca12be6754ab..1913c86fc9c92 100644 --- a/qa/multi-cluster-search/build.gradle +++ b/qa/multi-cluster-search/build.gradle @@ -19,42 +19,43 @@ import org.elasticsearch.gradle.test.RestIntegTestTask +apply plugin: 'elasticsearch.testclusters' apply plugin: 'elasticsearch.standalone-test' dependencies { - testCompile "org.elasticsearch.client:elasticsearch-rest-high-level-client:${version}" + testCompile project(":client:rest-high-level") } -task remoteClusterTest(type: RestIntegTestTask) { +task 'remote-cluster'(type: RestIntegTestTask) { mustRunAfter(precommit) + runner { + systemProperty 'tests.rest.suite', 'remote_cluster' + } } -remoteClusterTestCluster { - numNodes = 2 - clusterName = 'remote-cluster' - setting 'cluster.remote.connect', false +testClusters.'remote-cluster' { + numberOfNodes = 2 + setting 'cluster.remote.connect', 'false' } -remoteClusterTestRunner { - systemProperty 'tests.rest.suite', 'remote_cluster' +task mixedClusterTest(type: RestIntegTestTask) { + useCluster testClusters.'remote-cluster' + runner { + dependsOn 'remote-cluster' + systemProperty 'tests.rest.suite', 'multi_cluster' + } } -task mixedClusterTest(type: RestIntegTestTask) {} - -mixedClusterTestCluster { - dependsOn remoteClusterTestRunner - setting 'cluster.remote.my_remote_cluster.seeds', "\"${-> remoteClusterTest.nodes.get(0).transportUri()}\"" - setting 'cluster.remote.connections_per_cluster', 1 - setting 'cluster.remote.connect', true +testClusters.mixedClusterTest { + setting 'cluster.remote.my_remote_cluster.seeds', + { "\"${testClusters.'remote-cluster'.getAllTransportPortURI().get(0)}\"" } + setting 'cluster.remote.connections_per_cluster', '1' + setting 'cluster.remote.connect', 'true' } -mixedClusterTestRunner { - systemProperty 'tests.rest.suite', 'multi_cluster' - finalizedBy 'remoteClusterTestCluster#node0.stop','remoteClusterTestCluster#node1.stop' -} task integTest { - dependsOn = [mixedClusterTest] + dependsOn mixedClusterTest } test.enabled = false // no unit tests for multi-cluster-search, only integration tests diff --git a/qa/rolling-upgrade/src/test/java/org/elasticsearch/upgrades/RecoveryIT.java b/qa/rolling-upgrade/src/test/java/org/elasticsearch/upgrades/RecoveryIT.java index 32ddc77113bc8..602a31288ef46 100644 --- a/qa/rolling-upgrade/src/test/java/org/elasticsearch/upgrades/RecoveryIT.java +++ b/qa/rolling-upgrade/src/test/java/org/elasticsearch/upgrades/RecoveryIT.java @@ -356,7 +356,7 @@ public void testRecoveryWithSoftDeletes() throws Exception { // before timing out .put(INDEX_DELAYED_NODE_LEFT_TIMEOUT_SETTING.getKey(), "100ms") .put(SETTING_ALLOCATION_MAX_RETRY.getKey(), "0"); // fail faster - if (getNodeId(v -> v.onOrAfter(Version.V_6_5_0)) != null && randomBoolean()) { + if (randomBoolean()) { settings.put(IndexSettings.INDEX_SOFT_DELETES_SETTING.getKey(), true); } createIndex(index, settings.build()); @@ -567,4 +567,22 @@ private void ensureGlobalCheckpointSynced(String index) throws Exception { }); }, 60, TimeUnit.SECONDS); } + + /** Ensure that we can always execute update requests regardless of the version of cluster */ + public void testUpdateDoc() throws Exception { + final String index = "test_update_doc"; + if (CLUSTER_TYPE == ClusterType.OLD) { + Settings.Builder settings = Settings.builder() + .put(IndexMetaData.INDEX_NUMBER_OF_SHARDS_SETTING.getKey(), 1) + .put(IndexMetaData.INDEX_NUMBER_OF_REPLICAS_SETTING.getKey(), 2); + createIndex(index, settings.build()); + } + ensureGreen(index); + indexDocs(index, 0, 10); + for (int i = 0; i < 10; i++) { + Request update = new Request("POST", index + "/_update/" + i); + update.setJsonEntity("{\"doc\": {\"f\": " + randomNonNegativeLong() + "}}"); + client().performRequest(update); + } + } } diff --git a/qa/smoke-test-client/build.gradle b/qa/smoke-test-client/build.gradle deleted file mode 100644 index 637913083b48c..0000000000000 --- a/qa/smoke-test-client/build.gradle +++ /dev/null @@ -1,52 +0,0 @@ -import org.elasticsearch.gradle.test.RestIntegTestTask - -/* - * 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. - */ - -apply plugin: 'elasticsearch.standalone-rest-test' -apply plugin: 'elasticsearch.rest-test' - -// TODO: this test works, but it isn't really a rest test...should we have another plugin for "non rest test that just needs N clusters?" - -dependencies { - testCompile project(path: ':client:transport', configuration: 'runtime') // randomly swapped in as a transport -} - -task singleNodeIntegTest(type: RestIntegTestTask) { - mustRunAfter(precommit) -} - -singleNodeIntegTestCluster { - numNodes = 1 - setting 'discovery.type', 'single-node' -} - -integTestCluster.dependsOn(singleNodeIntegTestRunner, 'singleNodeIntegTestCluster#stop') - -check.dependsOn(integTest) - - -testingConventions { - naming.clear() - naming { - IT { - baseClass 'org.elasticsearch.smoketest.ESSmokeClientTestCase' - } - } -} \ No newline at end of file diff --git a/qa/smoke-test-client/src/test/java/org/elasticsearch/smoketest/ESSmokeClientTestCase.java b/qa/smoke-test-client/src/test/java/org/elasticsearch/smoketest/ESSmokeClientTestCase.java deleted file mode 100644 index 00460a8f004d0..0000000000000 --- a/qa/smoke-test-client/src/test/java/org/elasticsearch/smoketest/ESSmokeClientTestCase.java +++ /dev/null @@ -1,170 +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.smoketest; - -import org.apache.logging.log4j.LogManager; -import org.apache.logging.log4j.Logger; -import org.apache.lucene.util.LuceneTestCase; -import org.elasticsearch.action.admin.cluster.health.ClusterHealthResponse; -import org.elasticsearch.client.Client; -import org.elasticsearch.client.transport.TransportClient; -import org.elasticsearch.common.network.NetworkModule; -import org.elasticsearch.common.settings.Settings; -import org.elasticsearch.common.transport.TransportAddress; -import org.elasticsearch.env.Environment; -import org.elasticsearch.plugins.Plugin; -import org.elasticsearch.transport.client.PreBuiltTransportClient; -import org.elasticsearch.transport.nio.MockNioTransportPlugin; -import org.junit.After; -import org.junit.AfterClass; -import org.junit.Before; -import org.junit.BeforeClass; - -import java.io.IOException; -import java.net.InetAddress; -import java.net.InetSocketAddress; -import java.net.URL; -import java.nio.file.Path; -import java.util.Collection; -import java.util.Collections; -import java.util.Locale; -import java.util.concurrent.atomic.AtomicInteger; - -import static com.carrotsearch.randomizedtesting.RandomizedTest.randomAsciiOfLength; -import static org.hamcrest.Matchers.notNullValue; - -/** - * An abstract base class to run integration tests against an Elasticsearch cluster running outside of the test process. - *

    - * You can define a list of transport addresses from where you can reach your cluster by setting "tests.cluster" system - * property. It defaults to "localhost:9300". If you run this from `gradle integTest` then it will start the clsuter for - * you and set up the property. - *

    - * If you want to debug this module from your IDE, then start an external cluster by yourself, maybe with `gradle run`, - * then run JUnit. If you changed the default port, set "-Dtests.cluster=localhost:PORT" when running your test. - */ -@LuceneTestCase.SuppressSysoutChecks(bugUrl = "we log a lot on purpose") -public abstract class ESSmokeClientTestCase extends LuceneTestCase { - - /** - * Key used to eventually switch to using an external cluster and provide its transport addresses - */ - public static final String TESTS_CLUSTER = "tests.cluster"; - - protected static final Logger logger = LogManager.getLogger(ESSmokeClientTestCase.class); - - private static final AtomicInteger counter = new AtomicInteger(); - private static Client client; - private static String clusterAddresses; - protected String index; - - private static Client startClient(Path tempDir, TransportAddress... transportAddresses) { - Settings.Builder builder = Settings.builder() - .put("node.name", "qa_smoke_client_" + counter.getAndIncrement()) - .put("client.transport.ignore_cluster_name", true) - .put(Environment.PATH_HOME_SETTING.getKey(), tempDir); - final Collection> plugins; - String transportKey = MockNioTransportPlugin.MOCK_NIO_TRANSPORT_NAME; - Class transportPlugin = MockNioTransportPlugin.class; - if (random().nextBoolean()) { - builder.put(NetworkModule.TRANSPORT_TYPE_KEY, transportKey); - plugins = Collections.singleton(transportPlugin); - } else { - plugins = Collections.emptyList(); - } - TransportClient client = new PreBuiltTransportClient(builder.build(), plugins).addTransportAddresses(transportAddresses); - - logger.info("--> Elasticsearch Java TransportClient started"); - - Exception clientException = null; - try { - ClusterHealthResponse health = client.admin().cluster().prepareHealth().get(); - logger.info("--> connected to [{}] cluster which is running [{}] node(s).", - health.getClusterName(), health.getNumberOfNodes()); - } catch (Exception e) { - clientException = e; - } - - assumeNoException("Sounds like your cluster is not running at " + clusterAddresses, clientException); - - return client; - } - - private static Client startClient() throws IOException { - String[] stringAddresses = clusterAddresses.split(","); - TransportAddress[] transportAddresses = new TransportAddress[stringAddresses.length]; - int i = 0; - for (String stringAddress : stringAddresses) { - URL url = new URL("http://" + stringAddress); - InetAddress inetAddress = InetAddress.getByName(url.getHost()); - transportAddresses[i++] = new TransportAddress(new InetSocketAddress(inetAddress, url.getPort())); - } - return startClient(createTempDir(), transportAddresses); - } - - public static Client getClient() { - if (client == null) { - try { - client = startClient(); - } catch (IOException e) { - logger.error("can not start the client", e); - } - assertThat(client, notNullValue()); - } - return client; - } - - @BeforeClass - public static void initializeSettings() { - clusterAddresses = System.getProperty(TESTS_CLUSTER); - if (clusterAddresses == null || clusterAddresses.isEmpty()) { - fail("Must specify " + TESTS_CLUSTER + " for smoke client test"); - } - } - - @AfterClass - public static void stopTransportClient() { - if (client != null) { - client.close(); - client = null; - } - } - - @Before - public void defineIndexName() { - doClean(); - index = "qa-smoke-test-client-" + randomAsciiOfLength(10).toLowerCase(Locale.getDefault()); - } - - @After - public void cleanIndex() { - doClean(); - } - - private void doClean() { - if (client != null) { - try { - client.admin().indices().prepareDelete(index).get(); - } catch (Exception e) { - // We ignore this cleanup exception - } - } - } -} diff --git a/qa/smoke-test-client/src/test/java/org/elasticsearch/smoketest/SmokeTestClientIT.java b/qa/smoke-test-client/src/test/java/org/elasticsearch/smoketest/SmokeTestClientIT.java deleted file mode 100644 index 53753b06b8aa9..0000000000000 --- a/qa/smoke-test-client/src/test/java/org/elasticsearch/smoketest/SmokeTestClientIT.java +++ /dev/null @@ -1,73 +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.smoketest; - -import org.elasticsearch.action.admin.cluster.health.ClusterHealthResponse; -import org.elasticsearch.action.search.SearchResponse; -import org.elasticsearch.client.Client; - -import static org.hamcrest.CoreMatchers.is; -import static org.hamcrest.Matchers.greaterThan; - -public class SmokeTestClientIT extends ESSmokeClientTestCase { - - /** - * Check that we are connected to a cluster named "elasticsearch". - */ - public void testSimpleClient() { - final Client client = getClient(); - - // START SNIPPET: java-doc-admin-cluster-health - final ClusterHealthResponse health = - client.admin().cluster().prepareHealth().setWaitForYellowStatus().get(); - final String clusterName = health.getClusterName(); - final int numberOfNodes = health.getNumberOfNodes(); - // END SNIPPET: java-doc-admin-cluster-health - assertThat( - "cluster [" + clusterName + "] should have at least 1 node", - numberOfNodes, - greaterThan(0)); - } - - /** - * Create an index and index some docs - */ - public void testPutDocument() { - final Client client = getClient(); - - // START SNIPPET: java-doc-index-doc-simple - client.prepareIndex(index, "doc", "1") // Index, Type, Id - .setSource("foo", "bar") // Simple document: { "foo" : "bar" } - .get(); // Execute and wait for the result - // END SNIPPET: java-doc-index-doc-simple - - // START SNIPPET: java-doc-admin-indices-refresh - // Prepare a refresh action on a given index, execute and wait for the result - client.admin().indices().prepareRefresh(index).get(); - // END SNIPPET: java-doc-admin-indices-refresh - - // START SNIPPET: java-doc-search-simple - final SearchResponse searchResponse = client.prepareSearch(index).get(); - assertThat(searchResponse.getHits().getTotalHits().value, is(1L)); - // END SNIPPET: java-doc-search-simple - } - -} - diff --git a/qa/smoke-test-http/build.gradle b/qa/smoke-test-http/build.gradle index ef1a97fc7abaf..90fc9e3950147 100644 --- a/qa/smoke-test-http/build.gradle +++ b/qa/smoke-test-http/build.gradle @@ -17,6 +17,7 @@ * under the License. */ +apply plugin: 'elasticsearch.testclusters' apply plugin: 'elasticsearch.standalone-rest-test' apply plugin: 'elasticsearch.rest-test' apply plugin: 'elasticsearch.test-with-dependencies' @@ -26,7 +27,7 @@ dependencies { testCompile project(path: ':plugins:transport-nio', configuration: 'runtime') // for http } -integTestRunner { +integTest.runner { /* * We have to disable setting the number of available processors as tests in the same JVM randomize processors and will step on each * other if we allow them to set the number of available processors as it's set-once in Netty. diff --git a/qa/smoke-test-ingest-disabled/build.gradle b/qa/smoke-test-ingest-disabled/build.gradle index cf3ca9a713f96..a0abc8b129d1c 100644 --- a/qa/smoke-test-ingest-disabled/build.gradle +++ b/qa/smoke-test-ingest-disabled/build.gradle @@ -17,6 +17,7 @@ * under the License. */ +apply plugin: 'elasticsearch.testclusters' apply plugin: 'elasticsearch.standalone-rest-test' apply plugin: 'elasticsearch.rest-test' @@ -24,6 +25,6 @@ dependencies { testCompile project(path: ':modules:ingest-common', configuration: 'runtime') } -integTestCluster { +testClusters.integTest { setting 'node.ingest', 'false' } diff --git a/qa/smoke-test-ingest-with-all-dependencies/build.gradle b/qa/smoke-test-ingest-with-all-dependencies/build.gradle index 9267f90cd7e0a..9f5c40ac93798 100644 --- a/qa/smoke-test-ingest-with-all-dependencies/build.gradle +++ b/qa/smoke-test-ingest-with-all-dependencies/build.gradle @@ -17,6 +17,7 @@ * under the License. */ +apply plugin: 'elasticsearch.testclusters' apply plugin: 'elasticsearch.standalone-rest-test' apply plugin: 'elasticsearch.rest-test' diff --git a/qa/smoke-test-multinode/build.gradle b/qa/smoke-test-multinode/build.gradle index cd64262fcdce1..25964a871e052 100644 --- a/qa/smoke-test-multinode/build.gradle +++ b/qa/smoke-test-multinode/build.gradle @@ -17,6 +17,7 @@ * under the License. */ +apply plugin: 'elasticsearch.testclusters' apply plugin: 'elasticsearch.standalone-rest-test' apply plugin: 'elasticsearch.rest-test' @@ -24,12 +25,18 @@ integTest { includePackaged = true } -integTestCluster { - numNodes = 2 +File repo = file("$buildDir/testclusters/repo") +testClusters.integTest { + numberOfNodes = 2 + setting 'path.repo', repo.absolutePath } -integTestRunner { - if ('default'.equals(integTestCluster.distribution)) { +integTest.runner { + doFirst { + project.delete(repo) + repo.mkdirs() + } + if ('default'.equalsIgnoreCase(System.getProperty('tests.distribution', 'oss'))) { systemProperty 'tests.rest.blacklist', [ 'cat.templates/10_basic/No templates', 'cat.templates/10_basic/Sort templates', diff --git a/qa/smoke-test-plugins/build.gradle b/qa/smoke-test-plugins/build.gradle index 602dfa2d6ea4f..bdd5e74184e83 100644 --- a/qa/smoke-test-plugins/build.gradle +++ b/qa/smoke-test-plugins/build.gradle @@ -19,15 +19,18 @@ import org.elasticsearch.gradle.MavenFilteringHack +apply plugin: 'elasticsearch.testclusters' apply plugin: 'elasticsearch.standalone-rest-test' apply plugin: 'elasticsearch.rest-test' -ext.pluginsCount = 0 -project(':plugins').getChildProjects().each { pluginName, pluginProject -> - integTestCluster { - plugin pluginProject.path +int pluginsCount = 0 + +testClusters.integTest { + project(':plugins').getChildProjects().each { pluginName, pluginProject -> + plugin file(pluginProject.tasks.bundlePlugin.archiveFile) + tasks.integTest.dependsOn pluginProject.tasks.bundlePlugin + pluginsCount += 1 } - pluginsCount += 1 } assert pluginsCount > 0 diff --git a/qa/unconfigured-node-name/build.gradle b/qa/unconfigured-node-name/build.gradle index 3b0faa10a7e33..56805f37beae3 100644 --- a/qa/unconfigured-node-name/build.gradle +++ b/qa/unconfigured-node-name/build.gradle @@ -17,18 +17,15 @@ * under the License. */ +apply plugin: 'elasticsearch.testclusters' apply plugin: 'elasticsearch.standalone-rest-test' apply plugin: 'elasticsearch.rest-test' -integTestCluster { - setting 'node.name', null - // Run with no discovery configuration at all, demonstrating that a node in its - // "out-of-the-box" configuration can automatically bootstrap a cluster - autoSetInitialMasterNodes = false - autoSetHostsProvider = false +testClusters.integTest { + nameCustomization = { null } } -integTestRunner { +integTest.runner { nonInputProperties.systemProperty 'tests.logfile', - "${ -> integTest.nodes[0].homeDir}/logs/${ -> integTest.nodes[0].clusterName }_server.json" + "${ -> testClusters.integTest.singleNode().getServerLog() }" } diff --git a/qa/vagrant/build.gradle b/qa/vagrant/build.gradle index 7c342436dd3ca..f5cfcdda03cc6 100644 --- a/qa/vagrant/build.gradle +++ b/qa/vagrant/build.gradle @@ -35,7 +35,7 @@ dependencies { compile "commons-codec:commons-codec:${versions.commonscodec}" compile "commons-logging:commons-logging:${versions.commonslogging}" - compile project(':libs:core') + compile project(':libs:elasticsearch-core') // pulls in the jar built by this project and its dependencies packagingTest project(path: project.path, configuration: 'runtime') diff --git a/qa/wildfly/build.gradle b/qa/wildfly/build.gradle index bcb55079b8269..ed2e4a098b642 100644 --- a/qa/wildfly/build.gradle +++ b/qa/wildfly/build.gradle @@ -26,6 +26,7 @@ import java.util.stream.Stream */ apply plugin: 'war' +apply plugin: 'elasticsearch.testclusters' apply plugin: 'elasticsearch.build' apply plugin: 'elasticsearch.rest-test' @@ -70,9 +71,9 @@ dependencies { compile "com.fasterxml.jackson.module:jackson-module-jaxb-annotations:${versions.jackson}" compile "org.apache.logging.log4j:log4j-api:${versions.log4j}" compile "org.apache.logging.log4j:log4j-core:${versions.log4j}" - compile project(path: ':client:rest-high-level') + compile project(':client:rest-high-level') wildfly "org.jboss:wildfly:${wildflyVersion}@zip" - testCompile "org.elasticsearch.test:framework:${VersionProperties.elasticsearch}" + testCompile project(':test:framework') } task unzipWildfly(type: Sync) { @@ -88,12 +89,13 @@ task deploy(type: Copy) { task writeElasticsearchProperties { onlyIf { !Os.isFamily(Os.FAMILY_WINDOWS) } - dependsOn 'integTestCluster#wait', deploy + useCluster testClusters.integTest + dependsOn deploy doLast { final File elasticsearchProperties = file("${wildflyInstall}/standalone/configuration/elasticsearch.properties") elasticsearchProperties.write( [ - "http.uri=${-> integTest.getNodes().get(0).httpUri()}" + "http.uri=${-> testClusters.integTest.getAllHttpSocketURI().get(0)}" ].join("\n")) } } @@ -210,6 +212,7 @@ if (!Os.isFamily(Os.FAMILY_WINDOWS)) { integTestRunner.finalizedBy(stopWildfly) } else { integTest.enabled = false + testingConventions.enabled = false } check.dependsOn(integTest) diff --git a/qa/wildfly/src/test/java/org/elasticsearch/wildfly/WildflyIT.java b/qa/wildfly/src/test/java/org/elasticsearch/wildfly/WildflyIT.java index 28e11f021a1c7..16e0227f270de 100644 --- a/qa/wildfly/src/test/java/org/elasticsearch/wildfly/WildflyIT.java +++ b/qa/wildfly/src/test/java/org/elasticsearch/wildfly/WildflyIT.java @@ -27,7 +27,8 @@ import org.apache.http.impl.client.CloseableHttpClient; import org.apache.http.impl.client.HttpClientBuilder; import org.apache.http.util.EntityUtils; -import org.apache.log4j.Logger; +import org.apache.logging.log4j.LogManager; +import org.apache.logging.log4j.Logger; import org.apache.lucene.util.LuceneTestCase; import org.apache.lucene.util.TestRuleLimitSysouts; import org.elasticsearch.cluster.ClusterModule; @@ -53,9 +54,9 @@ @TestRuleLimitSysouts.Limit(bytes = 14000) public class WildflyIT extends LuceneTestCase { - private Logger logger = Logger.getLogger(WildflyIT.class); + private Logger logger = LogManager.getLogger(WildflyIT.class); - public void testTransportClient() throws URISyntaxException, IOException { + public void testRestClient() throws URISyntaxException, IOException { try (CloseableHttpClient client = HttpClientBuilder.create().build()) { final String str = String.format( Locale.ROOT, diff --git a/rest-api-spec/README.markdown b/rest-api-spec/README.markdown index 56b5e9b553b83..241433d8c5cc4 100644 --- a/rest-api-spec/README.markdown +++ b/rest-api-spec/README.markdown @@ -10,9 +10,9 @@ Example for the ["Create Index"](http://www.elastic.co/guide/en/elasticsearch/re { "indices.create": { "documentation": "http://www.elastic.co/guide/en/elasticsearch/reference/master/indices-create-index.html", + "stability": "stable", "methods": ["PUT", "POST"], "url": { - "path": "/{index}", "paths": ["/{index}"], "parts": { "index": { @@ -39,13 +39,105 @@ The specification contains: * The _name_ of the API (`indices.create`), which usually corresponds to the client calls * Link to the documentation at +* `stability` indicating the state of the API, has to be declared explicitly or YAML tests will fail + * `experimental` highly likely to break in the near future (minor/path), no bwc guarantees. + Possibly removed in the future. + * `beta` less likely to break or be removed but still reserve the right to do so + * `stable` No backwards breaking changes in a minor * List of HTTP methods for the endpoint * URL specification: path, parts, parameters * Whether body is allowed for the endpoint or not and its description +**NOTE** +If an API is stable but it response should be treated as an arbitrary map of key values please notate this as followed + +```json +{ + "api.name": { + "stability" : "stable", + "response": { + "treat_json_as_key_value" : true + } + } +} +``` + + The `methods` and `url.paths` elements list all possible HTTP methods and URLs for the endpoint; it is the responsibility of the developer to use this information for a sensible API on the target platform. +## Backwards compatibility + +The specification follows the same backward compatibility guarantees as Elasticsearch. + +- Within a Major, additions only. + - If an item has been documented wrong it should be deprecated instead as removing these might break downstream clients. +- Major version change, may deprecate pieces or simply remove them given enough deprecation time. + +## Deprecations + +The spec allows for deprecations of: + +#### Entire API: + +```json +{ + "api" : { + "documentation": "...", + "deprecated" : { + "version" : "7.0.0", + "description" : "Reason API is being deprecated" + }, + } +} +``` + +#### Specific paths: + +```json +{ + "api": { + "documentation": "", + "url": { + "paths": ["/_monitoring/bulk"], + "deprecated_paths" : [ + { + "version" : "7.0.0", + "path" : "/_monitoring/{type}/bulk", + "description" : "Specifying types in urls has been deprecated" + } + ] + } + } +} +``` + +Here `paths` describes the preferred paths and `deprecated_paths` indicates `paths` that will still work but are now +deprecated. + +#### Parameters + +```json +{ + "api": { + "documentation": "", + "methods": ["GET"], + "url": { + "params": { + "stored_fields": { + "type": "list", + "description" : "", + "deprecated" : { + "version" : "7.0.0", + "description" : "Reason parameter is being deprecated" + } + } + } + } + } +} +``` + ## License This software is licensed under the Apache License, version 2 ("ALv2"). diff --git a/rest-api-spec/src/main/resources/rest-api-spec/api/bulk.json b/rest-api-spec/src/main/resources/rest-api-spec/api/bulk.json index a07f362bef064..e02aa1b0d611a 100644 --- a/rest-api-spec/src/main/resources/rest-api-spec/api/bulk.json +++ b/rest-api-spec/src/main/resources/rest-api-spec/api/bulk.json @@ -1,9 +1,9 @@ { "bulk": { "documentation": "http://www.elastic.co/guide/en/elasticsearch/reference/master/docs-bulk.html", + "stability": "stable", "methods": ["POST", "PUT"], "url": { - "path": "/_bulk", "paths": ["/_bulk", "/{index}/_bulk", "/{index}/{type}/_bulk"], "parts": { "index": { diff --git a/rest-api-spec/src/main/resources/rest-api-spec/api/cat.aliases.json b/rest-api-spec/src/main/resources/rest-api-spec/api/cat.aliases.json index d30c6ace6f723..2c89c1dcdd15d 100644 --- a/rest-api-spec/src/main/resources/rest-api-spec/api/cat.aliases.json +++ b/rest-api-spec/src/main/resources/rest-api-spec/api/cat.aliases.json @@ -1,9 +1,9 @@ { "cat.aliases": { "documentation": "http://www.elastic.co/guide/en/elasticsearch/reference/master/cat-alias.html", + "stability": "stable", "methods": ["GET"], "url": { - "path": "/_cat/aliases", "paths": ["/_cat/aliases", "/_cat/aliases/{name}"], "parts": { "name": { diff --git a/rest-api-spec/src/main/resources/rest-api-spec/api/cat.allocation.json b/rest-api-spec/src/main/resources/rest-api-spec/api/cat.allocation.json index 7c826890607e4..711a894094277 100644 --- a/rest-api-spec/src/main/resources/rest-api-spec/api/cat.allocation.json +++ b/rest-api-spec/src/main/resources/rest-api-spec/api/cat.allocation.json @@ -1,9 +1,9 @@ { "cat.allocation": { "documentation": "http://www.elastic.co/guide/en/elasticsearch/reference/master/cat-allocation.html", + "stability": "stable", "methods": ["GET"], "url": { - "path": "/_cat/allocation", "paths": ["/_cat/allocation", "/_cat/allocation/{node_id}"], "parts": { "node_id": { diff --git a/rest-api-spec/src/main/resources/rest-api-spec/api/cat.count.json b/rest-api-spec/src/main/resources/rest-api-spec/api/cat.count.json index 4311d9a3be12d..7317e4866a255 100644 --- a/rest-api-spec/src/main/resources/rest-api-spec/api/cat.count.json +++ b/rest-api-spec/src/main/resources/rest-api-spec/api/cat.count.json @@ -1,9 +1,9 @@ { "cat.count": { "documentation": "http://www.elastic.co/guide/en/elasticsearch/reference/master/cat-count.html", + "stability": "stable", "methods": ["GET"], "url": { - "path": "/_cat/count", "paths": ["/_cat/count", "/_cat/count/{index}"], "parts": { "index": { diff --git a/rest-api-spec/src/main/resources/rest-api-spec/api/cat.fielddata.json b/rest-api-spec/src/main/resources/rest-api-spec/api/cat.fielddata.json index 88c7eee126294..acba8e58668cb 100644 --- a/rest-api-spec/src/main/resources/rest-api-spec/api/cat.fielddata.json +++ b/rest-api-spec/src/main/resources/rest-api-spec/api/cat.fielddata.json @@ -1,9 +1,9 @@ { "cat.fielddata": { "documentation": "http://www.elastic.co/guide/en/elasticsearch/reference/master/cat-fielddata.html", + "stability": "stable", "methods": ["GET"], "url": { - "path": "/_cat/fielddata", "paths": ["/_cat/fielddata", "/_cat/fielddata/{fields}"], "parts": { "fields": { diff --git a/rest-api-spec/src/main/resources/rest-api-spec/api/cat.health.json b/rest-api-spec/src/main/resources/rest-api-spec/api/cat.health.json index e858e83b465ed..36be2f1afedd8 100644 --- a/rest-api-spec/src/main/resources/rest-api-spec/api/cat.health.json +++ b/rest-api-spec/src/main/resources/rest-api-spec/api/cat.health.json @@ -1,9 +1,9 @@ { "cat.health": { "documentation": "http://www.elastic.co/guide/en/elasticsearch/reference/master/cat-health.html", + "stability": "stable", "methods": ["GET"], "url": { - "path": "/_cat/health", "paths": ["/_cat/health"], "parts": { }, diff --git a/rest-api-spec/src/main/resources/rest-api-spec/api/cat.help.json b/rest-api-spec/src/main/resources/rest-api-spec/api/cat.help.json index e893fb1b0447e..16a935517aa23 100644 --- a/rest-api-spec/src/main/resources/rest-api-spec/api/cat.help.json +++ b/rest-api-spec/src/main/resources/rest-api-spec/api/cat.help.json @@ -1,9 +1,9 @@ { "cat.help": { "documentation": "http://www.elastic.co/guide/en/elasticsearch/reference/master/cat.html", + "stability": "stable", "methods": ["GET"], "url": { - "path": "/_cat", "paths": ["/_cat"], "parts": { }, diff --git a/rest-api-spec/src/main/resources/rest-api-spec/api/cat.indices.json b/rest-api-spec/src/main/resources/rest-api-spec/api/cat.indices.json index 24961a57cf331..3246b8f73ea21 100644 --- a/rest-api-spec/src/main/resources/rest-api-spec/api/cat.indices.json +++ b/rest-api-spec/src/main/resources/rest-api-spec/api/cat.indices.json @@ -1,9 +1,9 @@ { "cat.indices": { "documentation": "http://www.elastic.co/guide/en/elasticsearch/reference/master/cat-indices.html", + "stability": "stable", "methods": ["GET"], "url": { - "path": "/_cat/indices", "paths": ["/_cat/indices", "/_cat/indices/{index}"], "parts": { "index": { diff --git a/rest-api-spec/src/main/resources/rest-api-spec/api/cat.master.json b/rest-api-spec/src/main/resources/rest-api-spec/api/cat.master.json index ab87b2adb7e72..a97d3a5ca2bdd 100644 --- a/rest-api-spec/src/main/resources/rest-api-spec/api/cat.master.json +++ b/rest-api-spec/src/main/resources/rest-api-spec/api/cat.master.json @@ -1,9 +1,9 @@ { "cat.master": { "documentation": "http://www.elastic.co/guide/en/elasticsearch/reference/master/cat-master.html", + "stability": "stable", "methods": ["GET"], "url": { - "path": "/_cat/master", "paths": ["/_cat/master"], "parts": { }, diff --git a/rest-api-spec/src/main/resources/rest-api-spec/api/cat.nodeattrs.json b/rest-api-spec/src/main/resources/rest-api-spec/api/cat.nodeattrs.json index 3d8a4a77a7704..6a9ee3c424a57 100644 --- a/rest-api-spec/src/main/resources/rest-api-spec/api/cat.nodeattrs.json +++ b/rest-api-spec/src/main/resources/rest-api-spec/api/cat.nodeattrs.json @@ -1,9 +1,9 @@ { "cat.nodeattrs": { "documentation": "http://www.elastic.co/guide/en/elasticsearch/reference/master/cat-nodeattrs.html", + "stability": "stable", "methods": ["GET"], "url": { - "path": "/_cat/nodeattrs", "paths": ["/_cat/nodeattrs"], "parts": { }, diff --git a/rest-api-spec/src/main/resources/rest-api-spec/api/cat.nodes.json b/rest-api-spec/src/main/resources/rest-api-spec/api/cat.nodes.json index 1b3c1266a63a0..9e17bf14a42dc 100644 --- a/rest-api-spec/src/main/resources/rest-api-spec/api/cat.nodes.json +++ b/rest-api-spec/src/main/resources/rest-api-spec/api/cat.nodes.json @@ -1,9 +1,9 @@ { "cat.nodes": { "documentation": "http://www.elastic.co/guide/en/elasticsearch/reference/master/cat-nodes.html", + "stability": "stable", "methods": ["GET"], "url": { - "path": "/_cat/nodes", "paths": ["/_cat/nodes"], "parts": { }, diff --git a/rest-api-spec/src/main/resources/rest-api-spec/api/cat.pending_tasks.json b/rest-api-spec/src/main/resources/rest-api-spec/api/cat.pending_tasks.json index 983b82482ae40..0fc951c424590 100644 --- a/rest-api-spec/src/main/resources/rest-api-spec/api/cat.pending_tasks.json +++ b/rest-api-spec/src/main/resources/rest-api-spec/api/cat.pending_tasks.json @@ -1,9 +1,9 @@ { "cat.pending_tasks": { "documentation": "http://www.elastic.co/guide/en/elasticsearch/reference/master/cat-pending-tasks.html", + "stability": "stable", "methods": ["GET"], "url": { - "path": "/_cat/pending_tasks", "paths": ["/_cat/pending_tasks"], "parts": { }, diff --git a/rest-api-spec/src/main/resources/rest-api-spec/api/cat.plugins.json b/rest-api-spec/src/main/resources/rest-api-spec/api/cat.plugins.json index 93c7feababa99..3217ec0ce62f6 100644 --- a/rest-api-spec/src/main/resources/rest-api-spec/api/cat.plugins.json +++ b/rest-api-spec/src/main/resources/rest-api-spec/api/cat.plugins.json @@ -1,9 +1,9 @@ { "cat.plugins": { "documentation": "http://www.elastic.co/guide/en/elasticsearch/reference/master/cat-plugins.html", + "stability": "stable", "methods": ["GET"], "url": { - "path": "/_cat/plugins", "paths": ["/_cat/plugins"], "params": { "format": { diff --git a/rest-api-spec/src/main/resources/rest-api-spec/api/cat.recovery.json b/rest-api-spec/src/main/resources/rest-api-spec/api/cat.recovery.json index 42f91cedfdd6c..c592d8c23db40 100644 --- a/rest-api-spec/src/main/resources/rest-api-spec/api/cat.recovery.json +++ b/rest-api-spec/src/main/resources/rest-api-spec/api/cat.recovery.json @@ -1,9 +1,9 @@ { "cat.recovery": { "documentation": "http://www.elastic.co/guide/en/elasticsearch/reference/master/cat-recovery.html", + "stability": "stable", "methods": ["GET"], "url": { - "path": "/_cat/recovery", "paths": ["/_cat/recovery", "/_cat/recovery/{index}"], "parts": { "index": { diff --git a/rest-api-spec/src/main/resources/rest-api-spec/api/cat.repositories.json b/rest-api-spec/src/main/resources/rest-api-spec/api/cat.repositories.json index c640a568fde0b..0b006fc023cf4 100644 --- a/rest-api-spec/src/main/resources/rest-api-spec/api/cat.repositories.json +++ b/rest-api-spec/src/main/resources/rest-api-spec/api/cat.repositories.json @@ -1,9 +1,9 @@ { "cat.repositories": { "documentation": "http://www.elastic.co/guide/en/elasticsearch/reference/master/cat-repositories.html", + "stability": "stable", "methods": ["GET"], "url": { - "path": "/_cat/repositories", "paths": ["/_cat/repositories"], "parts": { }, diff --git a/rest-api-spec/src/main/resources/rest-api-spec/api/cat.segments.json b/rest-api-spec/src/main/resources/rest-api-spec/api/cat.segments.json index 3306b2f753b2a..869b179a6cc7a 100644 --- a/rest-api-spec/src/main/resources/rest-api-spec/api/cat.segments.json +++ b/rest-api-spec/src/main/resources/rest-api-spec/api/cat.segments.json @@ -1,9 +1,9 @@ { "cat.segments": { "documentation": "http://www.elastic.co/guide/en/elasticsearch/reference/master/cat-segments.html", + "stability": "stable", "methods": ["GET"], "url": { - "path": "/_cat/segments", "paths": ["/_cat/segments", "/_cat/segments/{index}"], "parts": { "index": { diff --git a/rest-api-spec/src/main/resources/rest-api-spec/api/cat.shards.json b/rest-api-spec/src/main/resources/rest-api-spec/api/cat.shards.json index 2ad714e7225d7..abf96f666e474 100644 --- a/rest-api-spec/src/main/resources/rest-api-spec/api/cat.shards.json +++ b/rest-api-spec/src/main/resources/rest-api-spec/api/cat.shards.json @@ -1,9 +1,9 @@ { "cat.shards": { "documentation": "http://www.elastic.co/guide/en/elasticsearch/reference/master/cat-shards.html", + "stability": "stable", "methods": ["GET"], "url": { - "path": "/_cat/shards", "paths": ["/_cat/shards", "/_cat/shards/{index}"], "parts": { "index": { diff --git a/rest-api-spec/src/main/resources/rest-api-spec/api/cat.snapshots.json b/rest-api-spec/src/main/resources/rest-api-spec/api/cat.snapshots.json index eec22e2e0412d..1087537114c14 100644 --- a/rest-api-spec/src/main/resources/rest-api-spec/api/cat.snapshots.json +++ b/rest-api-spec/src/main/resources/rest-api-spec/api/cat.snapshots.json @@ -1,9 +1,9 @@ { "cat.snapshots": { "documentation": "http://www.elastic.co/guide/en/elasticsearch/reference/master/cat-snapshots.html", + "stability": "stable", "methods": ["GET"], "url": { - "path": "/_cat/snapshots", "paths": [ "/_cat/snapshots", "/_cat/snapshots/{repository}"], diff --git a/rest-api-spec/src/main/resources/rest-api-spec/api/cat.tasks.json b/rest-api-spec/src/main/resources/rest-api-spec/api/cat.tasks.json index 1958843e601c7..25b10a9147969 100644 --- a/rest-api-spec/src/main/resources/rest-api-spec/api/cat.tasks.json +++ b/rest-api-spec/src/main/resources/rest-api-spec/api/cat.tasks.json @@ -1,9 +1,9 @@ { "cat.tasks": { "documentation": "http://www.elastic.co/guide/en/elasticsearch/reference/master/tasks.html", + "stability": "stable", "methods": ["GET"], "url": { - "path": "/_cat/tasks", "paths": ["/_cat/tasks"], "parts": { }, diff --git a/rest-api-spec/src/main/resources/rest-api-spec/api/cat.templates.json b/rest-api-spec/src/main/resources/rest-api-spec/api/cat.templates.json index f0757c2d652dc..5bd6fd8b10273 100644 --- a/rest-api-spec/src/main/resources/rest-api-spec/api/cat.templates.json +++ b/rest-api-spec/src/main/resources/rest-api-spec/api/cat.templates.json @@ -1,9 +1,9 @@ { "cat.templates": { "documentation": "http://www.elastic.co/guide/en/elasticsearch/reference/master/cat-templates.html", + "stability": "stable", "methods": ["GET"], "url": { - "path": "/_cat/templates", "paths": ["/_cat/templates", "/_cat/templates/{name}"], "parts": { "name": { diff --git a/rest-api-spec/src/main/resources/rest-api-spec/api/cat.thread_pool.json b/rest-api-spec/src/main/resources/rest-api-spec/api/cat.thread_pool.json index 70a11000bbe41..056a1421d719f 100644 --- a/rest-api-spec/src/main/resources/rest-api-spec/api/cat.thread_pool.json +++ b/rest-api-spec/src/main/resources/rest-api-spec/api/cat.thread_pool.json @@ -1,9 +1,9 @@ { "cat.thread_pool": { "documentation": "http://www.elastic.co/guide/en/elasticsearch/reference/master/cat-thread-pool.html", + "stability": "stable", "methods": ["GET"], "url": { - "path": "/_cat/thread_pool", "paths": ["/_cat/thread_pool","/_cat/thread_pool/{thread_pool_patterns}"], "parts": { "thread_pool_patterns": { diff --git a/rest-api-spec/src/main/resources/rest-api-spec/api/clear_scroll.json b/rest-api-spec/src/main/resources/rest-api-spec/api/clear_scroll.json index b2b0343230c66..eb01710fd8415 100644 --- a/rest-api-spec/src/main/resources/rest-api-spec/api/clear_scroll.json +++ b/rest-api-spec/src/main/resources/rest-api-spec/api/clear_scroll.json @@ -1,9 +1,9 @@ { "clear_scroll": { "documentation": "http://www.elastic.co/guide/en/elasticsearch/reference/master/search-request-scroll.html", + "stability": "stable", "methods": ["DELETE"], "url": { - "path": "/_search/scroll", "paths": [ "/_search/scroll"], "deprecated_paths" : [ { diff --git a/rest-api-spec/src/main/resources/rest-api-spec/api/cluster.allocation_explain.json b/rest-api-spec/src/main/resources/rest-api-spec/api/cluster.allocation_explain.json index 26b24cfb69760..7af79af09ea9b 100644 --- a/rest-api-spec/src/main/resources/rest-api-spec/api/cluster.allocation_explain.json +++ b/rest-api-spec/src/main/resources/rest-api-spec/api/cluster.allocation_explain.json @@ -1,9 +1,9 @@ { "cluster.allocation_explain": { "documentation": "http://www.elastic.co/guide/en/elasticsearch/reference/master/cluster-allocation-explain.html", + "stability": "stable", "methods": ["GET", "POST"], "url": { - "path": "/_cluster/allocation/explain", "paths": ["/_cluster/allocation/explain"], "parts": {}, "params": { diff --git a/rest-api-spec/src/main/resources/rest-api-spec/api/cluster.get_settings.json b/rest-api-spec/src/main/resources/rest-api-spec/api/cluster.get_settings.json index baba4e3436abc..40c5a8d9f0eae 100644 --- a/rest-api-spec/src/main/resources/rest-api-spec/api/cluster.get_settings.json +++ b/rest-api-spec/src/main/resources/rest-api-spec/api/cluster.get_settings.json @@ -1,9 +1,9 @@ { "cluster.get_settings": { "documentation": "http://www.elastic.co/guide/en/elasticsearch/reference/master/cluster-update-settings.html", + "stability": "stable", "methods": ["GET"], "url": { - "path": "/_cluster/settings", "paths": ["/_cluster/settings"], "parts": {}, "params": { diff --git a/rest-api-spec/src/main/resources/rest-api-spec/api/cluster.health.json b/rest-api-spec/src/main/resources/rest-api-spec/api/cluster.health.json index 49d0ef5c83aee..1ea7c37943931 100644 --- a/rest-api-spec/src/main/resources/rest-api-spec/api/cluster.health.json +++ b/rest-api-spec/src/main/resources/rest-api-spec/api/cluster.health.json @@ -1,9 +1,9 @@ { "cluster.health": { "documentation": "http://www.elastic.co/guide/en/elasticsearch/reference/master/cluster-health.html", + "stability": "stable", "methods": ["GET"], "url": { - "path": "/_cluster/health", "paths": ["/_cluster/health", "/_cluster/health/{index}"], "parts": { "index": { diff --git a/rest-api-spec/src/main/resources/rest-api-spec/api/cluster.pending_tasks.json b/rest-api-spec/src/main/resources/rest-api-spec/api/cluster.pending_tasks.json index fb5e1609ff39d..7b8b2dbcedf1d 100644 --- a/rest-api-spec/src/main/resources/rest-api-spec/api/cluster.pending_tasks.json +++ b/rest-api-spec/src/main/resources/rest-api-spec/api/cluster.pending_tasks.json @@ -1,9 +1,9 @@ { "cluster.pending_tasks": { "documentation": "http://www.elastic.co/guide/en/elasticsearch/reference/master/cluster-pending.html", + "stability": "stable", "methods": ["GET"], "url": { - "path": "/_cluster/pending_tasks", "paths": ["/_cluster/pending_tasks"], "parts": { }, diff --git a/rest-api-spec/src/main/resources/rest-api-spec/api/cluster.put_settings.json b/rest-api-spec/src/main/resources/rest-api-spec/api/cluster.put_settings.json index 5fcf03102836a..4681640f4b12f 100644 --- a/rest-api-spec/src/main/resources/rest-api-spec/api/cluster.put_settings.json +++ b/rest-api-spec/src/main/resources/rest-api-spec/api/cluster.put_settings.json @@ -1,9 +1,9 @@ { "cluster.put_settings": { "documentation": "http://www.elastic.co/guide/en/elasticsearch/reference/master/cluster-update-settings.html", + "stability": "stable", "methods": ["PUT"], "url": { - "path": "/_cluster/settings", "paths": ["/_cluster/settings"], "parts": {}, "params": { diff --git a/rest-api-spec/src/main/resources/rest-api-spec/api/cluster.remote_info.json b/rest-api-spec/src/main/resources/rest-api-spec/api/cluster.remote_info.json index 3237853154493..4fdb89b891056 100644 --- a/rest-api-spec/src/main/resources/rest-api-spec/api/cluster.remote_info.json +++ b/rest-api-spec/src/main/resources/rest-api-spec/api/cluster.remote_info.json @@ -1,9 +1,9 @@ { "cluster.remote_info": { "documentation": "http://www.elastic.co/guide/en/elasticsearch/reference/master/cluster-remote-info.html", + "stability": "stable", "methods": ["GET"], "url": { - "path": "/_remote/info", "paths": ["/_remote/info"], "params": {} }, diff --git a/rest-api-spec/src/main/resources/rest-api-spec/api/cluster.reroute.json b/rest-api-spec/src/main/resources/rest-api-spec/api/cluster.reroute.json index 8bb85ca087ae8..c7d8ece64d8eb 100644 --- a/rest-api-spec/src/main/resources/rest-api-spec/api/cluster.reroute.json +++ b/rest-api-spec/src/main/resources/rest-api-spec/api/cluster.reroute.json @@ -1,9 +1,9 @@ { "cluster.reroute": { "documentation": "http://www.elastic.co/guide/en/elasticsearch/reference/master/cluster-reroute.html", + "stability": "stable", "methods": ["POST"], "url": { - "path": "/_cluster/reroute", "paths": ["/_cluster/reroute"], "parts": { }, diff --git a/rest-api-spec/src/main/resources/rest-api-spec/api/cluster.state.json b/rest-api-spec/src/main/resources/rest-api-spec/api/cluster.state.json index fec3f45000497..cd7f40cd86017 100644 --- a/rest-api-spec/src/main/resources/rest-api-spec/api/cluster.state.json +++ b/rest-api-spec/src/main/resources/rest-api-spec/api/cluster.state.json @@ -1,9 +1,9 @@ { "cluster.state": { "documentation": "http://www.elastic.co/guide/en/elasticsearch/reference/master/cluster-state.html", + "stability" : "stable", "methods": ["GET"], "url": { - "path": "/_cluster/state", "paths": [ "/_cluster/state", "/_cluster/state/{metric}", @@ -57,6 +57,9 @@ } } }, - "body": null + "body": null, + "response": { + "treat_json_as_key_value" : true + } } } diff --git a/rest-api-spec/src/main/resources/rest-api-spec/api/cluster.stats.json b/rest-api-spec/src/main/resources/rest-api-spec/api/cluster.stats.json index 36400ed2f7c4c..192cf7dcdf5ee 100644 --- a/rest-api-spec/src/main/resources/rest-api-spec/api/cluster.stats.json +++ b/rest-api-spec/src/main/resources/rest-api-spec/api/cluster.stats.json @@ -1,9 +1,9 @@ { "cluster.stats": { "documentation": "http://www.elastic.co/guide/en/elasticsearch/reference/master/cluster-stats.html", + "stability": "stable", "methods": ["GET"], "url": { - "path": "/_cluster/stats", "paths": ["/_cluster/stats", "/_cluster/stats/nodes/{node_id}"], "parts": { "node_id": { diff --git a/rest-api-spec/src/main/resources/rest-api-spec/api/count.json b/rest-api-spec/src/main/resources/rest-api-spec/api/count.json index b933091b9a416..e115a0ae06fb4 100644 --- a/rest-api-spec/src/main/resources/rest-api-spec/api/count.json +++ b/rest-api-spec/src/main/resources/rest-api-spec/api/count.json @@ -1,17 +1,10 @@ { "count": { "documentation": "http://www.elastic.co/guide/en/elasticsearch/reference/master/search-count.html", + "stability": "stable", "methods": ["POST", "GET"], "url": { - "path": "/_count", "paths": ["/_count", "/{index}/_count"], - "deprecated_paths" : [ - { - "version" : "7.0.0", - "path" : "/{index}/{type}/_count", - "description" : "Specifying types in urls has been deprecated" - } - ], "parts": { "index": { "type" : "list", diff --git a/rest-api-spec/src/main/resources/rest-api-spec/api/create.json b/rest-api-spec/src/main/resources/rest-api-spec/api/create.json index 65fcf02807ba1..d0e2375d0014b 100644 --- a/rest-api-spec/src/main/resources/rest-api-spec/api/create.json +++ b/rest-api-spec/src/main/resources/rest-api-spec/api/create.json @@ -1,9 +1,9 @@ { "create": { "documentation": "http://www.elastic.co/guide/en/elasticsearch/reference/master/docs-index_.html", + "stability": "stable", "methods": ["PUT","POST"], "url": { - "path": "/{index}/_create/{id}", "paths": ["/{index}/_create/{id}"], "deprecated_paths" : [ { diff --git a/rest-api-spec/src/main/resources/rest-api-spec/api/delete.json b/rest-api-spec/src/main/resources/rest-api-spec/api/delete.json index 0152374028832..27c7e62ddaad0 100644 --- a/rest-api-spec/src/main/resources/rest-api-spec/api/delete.json +++ b/rest-api-spec/src/main/resources/rest-api-spec/api/delete.json @@ -1,9 +1,9 @@ { "delete": { "documentation": "http://www.elastic.co/guide/en/elasticsearch/reference/master/docs-delete.html", + "stability": "stable", "methods": ["DELETE"], "url": { - "path": "/{index}/_doc/{id}", "paths": ["/{index}/_doc/{id}"], "deprecated_paths" : [ { diff --git a/rest-api-spec/src/main/resources/rest-api-spec/api/delete_by_query.json b/rest-api-spec/src/main/resources/rest-api-spec/api/delete_by_query.json index aa1d8b4be4f26..d4aadea8b3d80 100644 --- a/rest-api-spec/src/main/resources/rest-api-spec/api/delete_by_query.json +++ b/rest-api-spec/src/main/resources/rest-api-spec/api/delete_by_query.json @@ -1,9 +1,9 @@ { "delete_by_query": { "documentation": "https://www.elastic.co/guide/en/elasticsearch/reference/master/docs-delete-by-query.html", + "stability": "stable", "methods": ["POST"], "url": { - "path": "/{index}/_delete_by_query", "paths": ["/{index}/_delete_by_query"], "comment": "most things below this are just copied from search.json", "parts": { @@ -86,9 +86,9 @@ "type" : "time", "description" : "Explicit timeout for each search request. Defaults to no timeout." }, - "size": { + "max_docs": { "type" : "number", - "description" : "Number of hits to return (default: 10)" + "description" : "Maximum number of documents to process (default: all documents)" }, "sort": { "type" : "list", diff --git a/rest-api-spec/src/main/resources/rest-api-spec/api/delete_by_query_rethrottle.json b/rest-api-spec/src/main/resources/rest-api-spec/api/delete_by_query_rethrottle.json index f49af01cfc3b7..1104af560239d 100644 --- a/rest-api-spec/src/main/resources/rest-api-spec/api/delete_by_query_rethrottle.json +++ b/rest-api-spec/src/main/resources/rest-api-spec/api/delete_by_query_rethrottle.json @@ -1,9 +1,9 @@ { "delete_by_query_rethrottle": { "documentation": "https://www.elastic.co/guide/en/elasticsearch/reference/current/docs-delete-by-query.html", + "stability": "stable", "methods": ["POST"], "url": { - "path": "/_delete_by_query/{task_id}/_rethrottle", "paths": ["/_delete_by_query/{task_id}/_rethrottle"], "parts": { "task_id": { diff --git a/rest-api-spec/src/main/resources/rest-api-spec/api/delete_script.json b/rest-api-spec/src/main/resources/rest-api-spec/api/delete_script.json index 83bb690cc0428..d407aa0bfe694 100644 --- a/rest-api-spec/src/main/resources/rest-api-spec/api/delete_script.json +++ b/rest-api-spec/src/main/resources/rest-api-spec/api/delete_script.json @@ -1,9 +1,9 @@ { "delete_script": { "documentation": "http://www.elastic.co/guide/en/elasticsearch/reference/master/modules-scripting.html", + "stability": "stable", "methods": ["DELETE"], "url": { - "path": "/_scripts/{id}", "paths": [ "/_scripts/{id}" ], "parts": { "id": { diff --git a/rest-api-spec/src/main/resources/rest-api-spec/api/exists.json b/rest-api-spec/src/main/resources/rest-api-spec/api/exists.json index 2a451344521e4..35112389aa392 100644 --- a/rest-api-spec/src/main/resources/rest-api-spec/api/exists.json +++ b/rest-api-spec/src/main/resources/rest-api-spec/api/exists.json @@ -1,9 +1,9 @@ { "exists": { "documentation": "http://www.elastic.co/guide/en/elasticsearch/reference/master/docs-get.html", + "stability": "stable", "methods": ["HEAD"], "url": { - "path": "/{index}/_doc/{id}", "paths": ["/{index}/_doc/{id}"], "deprecated_paths" : [ { diff --git a/rest-api-spec/src/main/resources/rest-api-spec/api/exists_source.json b/rest-api-spec/src/main/resources/rest-api-spec/api/exists_source.json index 30e56141ec001..7a34094100d02 100644 --- a/rest-api-spec/src/main/resources/rest-api-spec/api/exists_source.json +++ b/rest-api-spec/src/main/resources/rest-api-spec/api/exists_source.json @@ -1,9 +1,9 @@ { "exists_source": { "documentation": "http://www.elastic.co/guide/en/elasticsearch/reference/master/docs-get.html", + "stability": "stable", "methods": ["HEAD"], "url": { - "path": "/{index}/_source/{id}", "paths": ["/{index}/_source/{id}"], "deprecated_paths" : [ { diff --git a/rest-api-spec/src/main/resources/rest-api-spec/api/explain.json b/rest-api-spec/src/main/resources/rest-api-spec/api/explain.json index 203ef23c9cc10..11f32801d4d19 100644 --- a/rest-api-spec/src/main/resources/rest-api-spec/api/explain.json +++ b/rest-api-spec/src/main/resources/rest-api-spec/api/explain.json @@ -1,9 +1,9 @@ { "explain": { "documentation": "http://www.elastic.co/guide/en/elasticsearch/reference/master/search-explain.html", + "stability": "stable", "methods": ["GET", "POST"], "url": { - "path": "/{index}/_explain/{id}", "paths": ["/{index}/_explain/{id}"], "deprecated_paths" : [ { diff --git a/rest-api-spec/src/main/resources/rest-api-spec/api/field_caps.json b/rest-api-spec/src/main/resources/rest-api-spec/api/field_caps.json index ebb520fd433b9..d41785615090b 100644 --- a/rest-api-spec/src/main/resources/rest-api-spec/api/field_caps.json +++ b/rest-api-spec/src/main/resources/rest-api-spec/api/field_caps.json @@ -1,9 +1,9 @@ { "field_caps": { "documentation": "http://www.elastic.co/guide/en/elasticsearch/reference/master/search-field-caps.html", + "stability": "stable", "methods": ["GET", "POST"], "url": { - "path": "/_field_caps", "paths": [ "/_field_caps", "/{index}/_field_caps" diff --git a/rest-api-spec/src/main/resources/rest-api-spec/api/get.json b/rest-api-spec/src/main/resources/rest-api-spec/api/get.json index f4e0fdd5f90ef..f02ca3e4a4304 100644 --- a/rest-api-spec/src/main/resources/rest-api-spec/api/get.json +++ b/rest-api-spec/src/main/resources/rest-api-spec/api/get.json @@ -1,9 +1,9 @@ { "get": { "documentation": "http://www.elastic.co/guide/en/elasticsearch/reference/master/docs-get.html", + "stability": "stable", "methods": ["GET"], "url": { - "path": "/{index}/_doc/{id}", "paths": ["/{index}/_doc/{id}"], "deprecated_paths" : [ { diff --git a/rest-api-spec/src/main/resources/rest-api-spec/api/get_script.json b/rest-api-spec/src/main/resources/rest-api-spec/api/get_script.json index 0b2d6c5a5b9c2..6beaf8ae85a6c 100644 --- a/rest-api-spec/src/main/resources/rest-api-spec/api/get_script.json +++ b/rest-api-spec/src/main/resources/rest-api-spec/api/get_script.json @@ -1,9 +1,9 @@ { "get_script": { "documentation": "http://www.elastic.co/guide/en/elasticsearch/reference/master/modules-scripting.html", + "stability": "stable", "methods": ["GET"], "url": { - "path": "/_scripts/{id}", "paths": [ "/_scripts/{id}" ], "parts": { "id": { diff --git a/rest-api-spec/src/main/resources/rest-api-spec/api/get_source.json b/rest-api-spec/src/main/resources/rest-api-spec/api/get_source.json index d6f6964aa7c36..0c7414f818411 100644 --- a/rest-api-spec/src/main/resources/rest-api-spec/api/get_source.json +++ b/rest-api-spec/src/main/resources/rest-api-spec/api/get_source.json @@ -1,9 +1,9 @@ { "get_source": { "documentation": "http://www.elastic.co/guide/en/elasticsearch/reference/master/docs-get.html", + "stability": "stable", "methods": ["GET"], "url": { - "path": "/{index}/_source/{id}", "paths": ["/{index}/_source/{id}"], "deprecated_paths" : [ { diff --git a/rest-api-spec/src/main/resources/rest-api-spec/api/index.json b/rest-api-spec/src/main/resources/rest-api-spec/api/index.json index 438032980a3c5..d85e0d709944c 100644 --- a/rest-api-spec/src/main/resources/rest-api-spec/api/index.json +++ b/rest-api-spec/src/main/resources/rest-api-spec/api/index.json @@ -1,9 +1,9 @@ { "index": { "documentation": "http://www.elastic.co/guide/en/elasticsearch/reference/master/docs-index_.html", + "stability": "stable", "methods": ["POST", "PUT"], "url": { - "path": "/{index}/_doc", "paths": ["/{index}/_doc/{id}", "/{index}/_doc"], "deprecated_paths" : [ { diff --git a/rest-api-spec/src/main/resources/rest-api-spec/api/indices.analyze.json b/rest-api-spec/src/main/resources/rest-api-spec/api/indices.analyze.json index c340ccd2f37c2..be119620d6cdf 100644 --- a/rest-api-spec/src/main/resources/rest-api-spec/api/indices.analyze.json +++ b/rest-api-spec/src/main/resources/rest-api-spec/api/indices.analyze.json @@ -1,9 +1,9 @@ { "indices.analyze": { "documentation": "http://www.elastic.co/guide/en/elasticsearch/reference/master/indices-analyze.html", + "stability": "stable", "methods": ["GET", "POST"], "url": { - "path": "/_analyze", "paths": ["/_analyze", "/{index}/_analyze"], "parts": { "index": { diff --git a/rest-api-spec/src/main/resources/rest-api-spec/api/indices.clear_cache.json b/rest-api-spec/src/main/resources/rest-api-spec/api/indices.clear_cache.json index 7e62371dd674d..9e591275927a8 100644 --- a/rest-api-spec/src/main/resources/rest-api-spec/api/indices.clear_cache.json +++ b/rest-api-spec/src/main/resources/rest-api-spec/api/indices.clear_cache.json @@ -1,9 +1,9 @@ { "indices.clear_cache": { "documentation": "http://www.elastic.co/guide/en/elasticsearch/reference/master/indices-clearcache.html", + "stability": "stable", "methods": ["POST"], "url": { - "path": "/_cache/clear", "paths": ["/_cache/clear", "/{index}/_cache/clear"], "parts": { "index": { diff --git a/rest-api-spec/src/main/resources/rest-api-spec/api/indices.close.json b/rest-api-spec/src/main/resources/rest-api-spec/api/indices.close.json index 55fd245f26c91..187cce0d07796 100644 --- a/rest-api-spec/src/main/resources/rest-api-spec/api/indices.close.json +++ b/rest-api-spec/src/main/resources/rest-api-spec/api/indices.close.json @@ -1,9 +1,9 @@ { "indices.close": { "documentation": "http://www.elastic.co/guide/en/elasticsearch/reference/master/indices-open-close.html", + "stability": "stable", "methods": ["POST"], "url": { - "path": "/{index}/_close", "paths": ["/{index}/_close"], "parts": { "index": { diff --git a/rest-api-spec/src/main/resources/rest-api-spec/api/indices.create.json b/rest-api-spec/src/main/resources/rest-api-spec/api/indices.create.json index d4a16e576e1b9..dfe3236c9c053 100644 --- a/rest-api-spec/src/main/resources/rest-api-spec/api/indices.create.json +++ b/rest-api-spec/src/main/resources/rest-api-spec/api/indices.create.json @@ -1,9 +1,9 @@ { "indices.create": { "documentation": "http://www.elastic.co/guide/en/elasticsearch/reference/master/indices-create-index.html", + "stability": "stable", "methods": ["PUT"], "url": { - "path": "/{index}", "paths": ["/{index}"], "parts": { "index": { diff --git a/rest-api-spec/src/main/resources/rest-api-spec/api/indices.delete.json b/rest-api-spec/src/main/resources/rest-api-spec/api/indices.delete.json index c391242e9de15..4c2b7eaaa2fb2 100644 --- a/rest-api-spec/src/main/resources/rest-api-spec/api/indices.delete.json +++ b/rest-api-spec/src/main/resources/rest-api-spec/api/indices.delete.json @@ -1,9 +1,9 @@ { "indices.delete": { "documentation": "http://www.elastic.co/guide/en/elasticsearch/reference/master/indices-delete-index.html", + "stability": "stable", "methods": ["DELETE"], "url": { - "path": "/{index}", "paths": ["/{index}"], "parts": { "index": { diff --git a/rest-api-spec/src/main/resources/rest-api-spec/api/indices.delete_alias.json b/rest-api-spec/src/main/resources/rest-api-spec/api/indices.delete_alias.json index 30a32fbfef871..c6e4ec8295443 100644 --- a/rest-api-spec/src/main/resources/rest-api-spec/api/indices.delete_alias.json +++ b/rest-api-spec/src/main/resources/rest-api-spec/api/indices.delete_alias.json @@ -1,9 +1,9 @@ { "indices.delete_alias": { "documentation": "http://www.elastic.co/guide/en/elasticsearch/reference/master/indices-aliases.html", + "stability": "stable", "methods": ["DELETE"], "url": { - "path": "/{index}/_alias/{name}", "paths": ["/{index}/_alias/{name}", "/{index}/_aliases/{name}"], "parts": { "index": { diff --git a/rest-api-spec/src/main/resources/rest-api-spec/api/indices.delete_template.json b/rest-api-spec/src/main/resources/rest-api-spec/api/indices.delete_template.json index b311c9bbda82a..0b3758953b3fb 100644 --- a/rest-api-spec/src/main/resources/rest-api-spec/api/indices.delete_template.json +++ b/rest-api-spec/src/main/resources/rest-api-spec/api/indices.delete_template.json @@ -1,9 +1,9 @@ { "indices.delete_template": { "documentation": "http://www.elastic.co/guide/en/elasticsearch/reference/master/indices-templates.html", + "stability": "stable", "methods": ["DELETE"], "url": { - "path": "/_template/{name}", "paths": ["/_template/{name}"], "parts": { "name": { diff --git a/rest-api-spec/src/main/resources/rest-api-spec/api/indices.exists.json b/rest-api-spec/src/main/resources/rest-api-spec/api/indices.exists.json index 7f04f00b40def..b84f4d2cacbf3 100644 --- a/rest-api-spec/src/main/resources/rest-api-spec/api/indices.exists.json +++ b/rest-api-spec/src/main/resources/rest-api-spec/api/indices.exists.json @@ -1,9 +1,9 @@ { "indices.exists": { "documentation": "http://www.elastic.co/guide/en/elasticsearch/reference/master/indices-exists.html", + "stability": "stable", "methods": [ "HEAD" ], "url": { - "path": "/{index}", "paths": [ "/{index}" ], "parts": { "index": { diff --git a/rest-api-spec/src/main/resources/rest-api-spec/api/indices.exists_alias.json b/rest-api-spec/src/main/resources/rest-api-spec/api/indices.exists_alias.json index aea20b2b634d0..d95c423b66db5 100644 --- a/rest-api-spec/src/main/resources/rest-api-spec/api/indices.exists_alias.json +++ b/rest-api-spec/src/main/resources/rest-api-spec/api/indices.exists_alias.json @@ -1,9 +1,9 @@ { "indices.exists_alias": { "documentation": "http://www.elastic.co/guide/en/elasticsearch/reference/master/indices-aliases.html", + "stability": "stable", "methods": ["HEAD"], "url": { - "path": "/_alias/{name}", "paths": ["/_alias/{name}", "/{index}/_alias/{name}"], "parts": { "index": { diff --git a/rest-api-spec/src/main/resources/rest-api-spec/api/indices.exists_template.json b/rest-api-spec/src/main/resources/rest-api-spec/api/indices.exists_template.json index 3fb9d1e207e1e..8140c5748fb4d 100644 --- a/rest-api-spec/src/main/resources/rest-api-spec/api/indices.exists_template.json +++ b/rest-api-spec/src/main/resources/rest-api-spec/api/indices.exists_template.json @@ -1,9 +1,9 @@ { "indices.exists_template": { "documentation": "http://www.elastic.co/guide/en/elasticsearch/reference/master/indices-templates.html", + "stability": "stable", "methods": ["HEAD"], "url": { - "path": "/_template/{name}", "paths": [ "/_template/{name}" ], "parts": { "name": { diff --git a/rest-api-spec/src/main/resources/rest-api-spec/api/indices.exists_type.json b/rest-api-spec/src/main/resources/rest-api-spec/api/indices.exists_type.json index a788202c93ae2..10b69b2647f82 100644 --- a/rest-api-spec/src/main/resources/rest-api-spec/api/indices.exists_type.json +++ b/rest-api-spec/src/main/resources/rest-api-spec/api/indices.exists_type.json @@ -1,13 +1,13 @@ { "indices.exists_type": { "documentation": "http://www.elastic.co/guide/en/elasticsearch/reference/master/indices-types-exists.html", + "stability": "stable", "methods": ["HEAD"], "deprecated" : { "version" : "7.0.0", "description" : "Types are being removed from elasticsearch and therefor this API is on the way out. Read more here: https://www.elastic.co/guide/en/elasticsearch/reference/master/removal-of-types.html" }, "url": { - "path": "/{index}/_mapping/{type}", "paths": ["/{index}/_mapping/{type}"], "parts": { "index": { diff --git a/rest-api-spec/src/main/resources/rest-api-spec/api/indices.flush.json b/rest-api-spec/src/main/resources/rest-api-spec/api/indices.flush.json index 77d9e037163fa..7c6c9cb32cd7d 100644 --- a/rest-api-spec/src/main/resources/rest-api-spec/api/indices.flush.json +++ b/rest-api-spec/src/main/resources/rest-api-spec/api/indices.flush.json @@ -1,9 +1,9 @@ { "indices.flush": { "documentation": "http://www.elastic.co/guide/en/elasticsearch/reference/master/indices-flush.html", + "stability": "stable", "methods": ["POST", "GET"], "url": { - "path": "/_flush", "paths": ["/_flush", "/{index}/_flush"], "parts": { "index": { diff --git a/rest-api-spec/src/main/resources/rest-api-spec/api/indices.flush_synced.json b/rest-api-spec/src/main/resources/rest-api-spec/api/indices.flush_synced.json index 08488eae8648b..1b9ab1bb5eff5 100644 --- a/rest-api-spec/src/main/resources/rest-api-spec/api/indices.flush_synced.json +++ b/rest-api-spec/src/main/resources/rest-api-spec/api/indices.flush_synced.json @@ -1,9 +1,9 @@ { "indices.flush_synced": { "documentation": "http://www.elastic.co/guide/en/elasticsearch/reference/master/indices-synced-flush.html", + "stability": "stable", "methods": ["POST", "GET"], "url": { - "path": "/_flush/synced", "paths": [ "/_flush/synced", "/{index}/_flush/synced" diff --git a/rest-api-spec/src/main/resources/rest-api-spec/api/indices.forcemerge.json b/rest-api-spec/src/main/resources/rest-api-spec/api/indices.forcemerge.json index d87ce2a4451d6..c75bf0e47ef96 100644 --- a/rest-api-spec/src/main/resources/rest-api-spec/api/indices.forcemerge.json +++ b/rest-api-spec/src/main/resources/rest-api-spec/api/indices.forcemerge.json @@ -1,9 +1,9 @@ { "indices.forcemerge": { "documentation": "http://www.elastic.co/guide/en/elasticsearch/reference/master/indices-forcemerge.html", + "stability": "stable", "methods": ["POST"], "url": { - "path": "/_forcemerge", "paths": ["/_forcemerge", "/{index}/_forcemerge"], "parts": { "index": { diff --git a/rest-api-spec/src/main/resources/rest-api-spec/api/indices.get.json b/rest-api-spec/src/main/resources/rest-api-spec/api/indices.get.json index 76e6ed00feb26..a0e947504b3a0 100644 --- a/rest-api-spec/src/main/resources/rest-api-spec/api/indices.get.json +++ b/rest-api-spec/src/main/resources/rest-api-spec/api/indices.get.json @@ -1,9 +1,9 @@ { "indices.get":{ "documentation":"http://www.elastic.co/guide/en/elasticsearch/reference/master/indices-get-index.html", + "stability": "stable", "methods":[ "GET" ], "url":{ - "path":"/{index}", "paths":[ "/{index}" ], "parts":{ "index":{ diff --git a/rest-api-spec/src/main/resources/rest-api-spec/api/indices.get_alias.json b/rest-api-spec/src/main/resources/rest-api-spec/api/indices.get_alias.json index b68d7c527fcdc..db819f446ad1d 100644 --- a/rest-api-spec/src/main/resources/rest-api-spec/api/indices.get_alias.json +++ b/rest-api-spec/src/main/resources/rest-api-spec/api/indices.get_alias.json @@ -1,9 +1,9 @@ { "indices.get_alias": { "documentation": "http://www.elastic.co/guide/en/elasticsearch/reference/master/indices-aliases.html", + "stability": "stable", "methods": ["GET"], "url": { - "path": "/_alias/", "paths": [ "/_alias", "/_alias/{name}", "/{index}/_alias/{name}", "/{index}/_alias"], "parts": { "index": { diff --git a/rest-api-spec/src/main/resources/rest-api-spec/api/indices.get_field_mapping.json b/rest-api-spec/src/main/resources/rest-api-spec/api/indices.get_field_mapping.json index 305cb700ceae9..83a199cbdc045 100644 --- a/rest-api-spec/src/main/resources/rest-api-spec/api/indices.get_field_mapping.json +++ b/rest-api-spec/src/main/resources/rest-api-spec/api/indices.get_field_mapping.json @@ -1,9 +1,9 @@ { "indices.get_field_mapping": { "documentation": "http://www.elastic.co/guide/en/elasticsearch/reference/master/indices-get-field-mapping.html", + "stability": "stable", "methods": ["GET"], "url": { - "path": "/_mapping/field/{fields}", "paths": ["/_mapping/field/{fields}", "/{index}/_mapping/field/{fields}"], "deprecated_paths" : [ { diff --git a/rest-api-spec/src/main/resources/rest-api-spec/api/indices.get_mapping.json b/rest-api-spec/src/main/resources/rest-api-spec/api/indices.get_mapping.json index abfc3d3867e9a..ad55adc18933d 100644 --- a/rest-api-spec/src/main/resources/rest-api-spec/api/indices.get_mapping.json +++ b/rest-api-spec/src/main/resources/rest-api-spec/api/indices.get_mapping.json @@ -1,9 +1,9 @@ { "indices.get_mapping": { "documentation": "http://www.elastic.co/guide/en/elasticsearch/reference/master/indices-get-mapping.html", + "stability": "stable", "methods": ["GET"], "url": { - "path": "/_mapping", "paths": ["/_mapping", "/{index}/_mapping"], "deprecated_paths" : [ { diff --git a/rest-api-spec/src/main/resources/rest-api-spec/api/indices.get_settings.json b/rest-api-spec/src/main/resources/rest-api-spec/api/indices.get_settings.json index ed22cc837d6a8..d86f2b0a3b810 100644 --- a/rest-api-spec/src/main/resources/rest-api-spec/api/indices.get_settings.json +++ b/rest-api-spec/src/main/resources/rest-api-spec/api/indices.get_settings.json @@ -1,9 +1,9 @@ { "indices.get_settings": { "documentation": "http://www.elastic.co/guide/en/elasticsearch/reference/master/indices-get-settings.html", + "stability": "stable", "methods": ["GET"], "url": { - "path": "/_settings", "paths": ["/_settings", "/{index}/_settings", "/{index}/_settings/{name}", "/_settings/{name}"], "parts": { "index": { diff --git a/rest-api-spec/src/main/resources/rest-api-spec/api/indices.get_template.json b/rest-api-spec/src/main/resources/rest-api-spec/api/indices.get_template.json index e2aae3b7444aa..f81fc337dd4ac 100644 --- a/rest-api-spec/src/main/resources/rest-api-spec/api/indices.get_template.json +++ b/rest-api-spec/src/main/resources/rest-api-spec/api/indices.get_template.json @@ -1,9 +1,9 @@ { "indices.get_template": { "documentation": "http://www.elastic.co/guide/en/elasticsearch/reference/master/indices-templates.html", + "stability": "stable", "methods": ["GET"], "url": { - "path": "/_template/{name}", "paths": [ "/_template", "/_template/{name}" diff --git a/rest-api-spec/src/main/resources/rest-api-spec/api/indices.get_upgrade.json b/rest-api-spec/src/main/resources/rest-api-spec/api/indices.get_upgrade.json index 3c114a450eef2..4770fba4e9c4a 100644 --- a/rest-api-spec/src/main/resources/rest-api-spec/api/indices.get_upgrade.json +++ b/rest-api-spec/src/main/resources/rest-api-spec/api/indices.get_upgrade.json @@ -1,9 +1,9 @@ { "indices.get_upgrade": { "documentation": "http://www.elastic.co/guide/en/elasticsearch/reference/master/indices-upgrade.html", + "stability": "stable", "methods": ["GET"], "url": { - "path": "/_upgrade", "paths": ["/_upgrade", "/{index}/_upgrade"], "parts": { "index": { diff --git a/rest-api-spec/src/main/resources/rest-api-spec/api/indices.open.json b/rest-api-spec/src/main/resources/rest-api-spec/api/indices.open.json index 86c39988e181f..36cc75e264935 100644 --- a/rest-api-spec/src/main/resources/rest-api-spec/api/indices.open.json +++ b/rest-api-spec/src/main/resources/rest-api-spec/api/indices.open.json @@ -1,9 +1,9 @@ { "indices.open": { "documentation": "http://www.elastic.co/guide/en/elasticsearch/reference/master/indices-open-close.html", + "stability": "stable", "methods": ["POST"], "url": { - "path": "/{index}/_open", "paths": ["/{index}/_open"], "parts": { "index": { diff --git a/rest-api-spec/src/main/resources/rest-api-spec/api/indices.put_alias.json b/rest-api-spec/src/main/resources/rest-api-spec/api/indices.put_alias.json index da53d5ac441e1..8cbcfcdd25e53 100644 --- a/rest-api-spec/src/main/resources/rest-api-spec/api/indices.put_alias.json +++ b/rest-api-spec/src/main/resources/rest-api-spec/api/indices.put_alias.json @@ -1,9 +1,9 @@ { "indices.put_alias": { "documentation": "http://www.elastic.co/guide/en/elasticsearch/reference/master/indices-aliases.html", + "stability": "stable", "methods": ["PUT", "POST"], "url": { - "path": "/{index}/_alias/{name}", "paths": ["/{index}/_alias/{name}", "/{index}/_aliases/{name}"], "parts": { "index": { diff --git a/rest-api-spec/src/main/resources/rest-api-spec/api/indices.put_mapping.json b/rest-api-spec/src/main/resources/rest-api-spec/api/indices.put_mapping.json index c46e6b15549cc..064c532917c74 100644 --- a/rest-api-spec/src/main/resources/rest-api-spec/api/indices.put_mapping.json +++ b/rest-api-spec/src/main/resources/rest-api-spec/api/indices.put_mapping.json @@ -1,9 +1,9 @@ { "indices.put_mapping": { "documentation": "http://www.elastic.co/guide/en/elasticsearch/reference/master/indices-put-mapping.html", + "stability": "stable", "methods": ["PUT", "POST"], "url": { - "path": "{index}/_mapping", "paths": ["{index}/_mapping"], "deprecated_paths" : [ { diff --git a/rest-api-spec/src/main/resources/rest-api-spec/api/indices.put_settings.json b/rest-api-spec/src/main/resources/rest-api-spec/api/indices.put_settings.json index 3055cb8e32e2e..3ef98b8b0a54c 100644 --- a/rest-api-spec/src/main/resources/rest-api-spec/api/indices.put_settings.json +++ b/rest-api-spec/src/main/resources/rest-api-spec/api/indices.put_settings.json @@ -1,9 +1,9 @@ { "indices.put_settings": { "documentation": "http://www.elastic.co/guide/en/elasticsearch/reference/master/indices-update-settings.html", + "stability": "stable", "methods": ["PUT"], "url": { - "path": "/_settings", "paths": ["/_settings", "/{index}/_settings"], "parts": { "index": { diff --git a/rest-api-spec/src/main/resources/rest-api-spec/api/indices.put_template.json b/rest-api-spec/src/main/resources/rest-api-spec/api/indices.put_template.json index 65aa9506ff9f1..58235afa281f2 100644 --- a/rest-api-spec/src/main/resources/rest-api-spec/api/indices.put_template.json +++ b/rest-api-spec/src/main/resources/rest-api-spec/api/indices.put_template.json @@ -1,9 +1,9 @@ { "indices.put_template": { "documentation": "http://www.elastic.co/guide/en/elasticsearch/reference/master/indices-templates.html", + "stability": "stable", "methods": ["PUT", "POST"], "url": { - "path": "/_template/{name}", "paths": ["/_template/{name}"], "parts": { "name": { diff --git a/rest-api-spec/src/main/resources/rest-api-spec/api/indices.recovery.json b/rest-api-spec/src/main/resources/rest-api-spec/api/indices.recovery.json index 80b7f1a000986..6c3b574d23b0d 100644 --- a/rest-api-spec/src/main/resources/rest-api-spec/api/indices.recovery.json +++ b/rest-api-spec/src/main/resources/rest-api-spec/api/indices.recovery.json @@ -1,9 +1,9 @@ { "indices.recovery" : { "documentation": "http://www.elastic.co/guide/en/elasticsearch/reference/master/indices-recovery.html", + "stability": "stable", "methods": ["GET"], "url": { - "path": "/_recovery", "paths": ["/_recovery", "/{index}/_recovery"], "parts": { "index": { diff --git a/rest-api-spec/src/main/resources/rest-api-spec/api/indices.refresh.json b/rest-api-spec/src/main/resources/rest-api-spec/api/indices.refresh.json index a32974d017f22..157aae6c9178e 100644 --- a/rest-api-spec/src/main/resources/rest-api-spec/api/indices.refresh.json +++ b/rest-api-spec/src/main/resources/rest-api-spec/api/indices.refresh.json @@ -1,9 +1,9 @@ { "indices.refresh": { "documentation": "http://www.elastic.co/guide/en/elasticsearch/reference/master/indices-refresh.html", + "stability": "stable", "methods": ["POST", "GET"], "url": { - "path": "/_refresh", "paths": ["/_refresh", "/{index}/_refresh"], "parts": { "index": { diff --git a/rest-api-spec/src/main/resources/rest-api-spec/api/indices.rollover.json b/rest-api-spec/src/main/resources/rest-api-spec/api/indices.rollover.json index 7bf1513969fb3..3d5addef6b5cb 100644 --- a/rest-api-spec/src/main/resources/rest-api-spec/api/indices.rollover.json +++ b/rest-api-spec/src/main/resources/rest-api-spec/api/indices.rollover.json @@ -1,9 +1,9 @@ { "indices.rollover": { "documentation": "http://www.elastic.co/guide/en/elasticsearch/reference/master/indices-rollover-index.html", + "stability": "stable", "methods": ["POST"], "url": { - "path": "/{alias}/_rollover", "paths": ["/{alias}/_rollover", "/{alias}/_rollover/{new_index}"], "parts": { "alias": { diff --git a/rest-api-spec/src/main/resources/rest-api-spec/api/indices.segments.json b/rest-api-spec/src/main/resources/rest-api-spec/api/indices.segments.json index 3ecf1b23aaea2..a39d492b5a40c 100644 --- a/rest-api-spec/src/main/resources/rest-api-spec/api/indices.segments.json +++ b/rest-api-spec/src/main/resources/rest-api-spec/api/indices.segments.json @@ -1,9 +1,9 @@ { "indices.segments": { "documentation": "http://www.elastic.co/guide/en/elasticsearch/reference/master/indices-segments.html", + "stability": "stable", "methods": ["GET"], "url": { - "path": "/_segments", "paths": ["/_segments", "/{index}/_segments"], "parts": { "index": { diff --git a/rest-api-spec/src/main/resources/rest-api-spec/api/indices.shard_stores.json b/rest-api-spec/src/main/resources/rest-api-spec/api/indices.shard_stores.json index ecdaa75a3a9e4..ff64c40f6674e 100644 --- a/rest-api-spec/src/main/resources/rest-api-spec/api/indices.shard_stores.json +++ b/rest-api-spec/src/main/resources/rest-api-spec/api/indices.shard_stores.json @@ -1,9 +1,9 @@ { "indices.shard_stores": { "documentation": "http://www.elastic.co/guide/en/elasticsearch/reference/master/indices-shards-stores.html", + "stability": "stable", "methods": ["GET"], "url": { - "path": "/_shard_stores", "paths": ["/_shard_stores", "/{index}/_shard_stores"], "parts": { "index": { diff --git a/rest-api-spec/src/main/resources/rest-api-spec/api/indices.shrink.json b/rest-api-spec/src/main/resources/rest-api-spec/api/indices.shrink.json index 5ef943eacba6c..111444b7ebca8 100644 --- a/rest-api-spec/src/main/resources/rest-api-spec/api/indices.shrink.json +++ b/rest-api-spec/src/main/resources/rest-api-spec/api/indices.shrink.json @@ -1,9 +1,9 @@ { "indices.shrink": { "documentation": "http://www.elastic.co/guide/en/elasticsearch/reference/master/indices-shrink-index.html", + "stability": "stable", "methods": ["PUT", "POST"], "url": { - "path": "/{index}/_shrink/{target}", "paths": ["/{index}/_shrink/{target}"], "parts": { "index": { diff --git a/rest-api-spec/src/main/resources/rest-api-spec/api/indices.split.json b/rest-api-spec/src/main/resources/rest-api-spec/api/indices.split.json index a79fa7b708269..10830b4bb9504 100644 --- a/rest-api-spec/src/main/resources/rest-api-spec/api/indices.split.json +++ b/rest-api-spec/src/main/resources/rest-api-spec/api/indices.split.json @@ -1,9 +1,9 @@ { "indices.split": { "documentation": "http://www.elastic.co/guide/en/elasticsearch/reference/master/indices-split-index.html", + "stability": "stable", "methods": ["PUT", "POST"], "url": { - "path": "/{index}/_split/{target}", "paths": ["/{index}/_split/{target}"], "parts": { "index": { diff --git a/rest-api-spec/src/main/resources/rest-api-spec/api/indices.stats.json b/rest-api-spec/src/main/resources/rest-api-spec/api/indices.stats.json index c86a2c1147a9b..3be4d9387245b 100644 --- a/rest-api-spec/src/main/resources/rest-api-spec/api/indices.stats.json +++ b/rest-api-spec/src/main/resources/rest-api-spec/api/indices.stats.json @@ -1,9 +1,9 @@ { "indices.stats": { "documentation": "http://www.elastic.co/guide/en/elasticsearch/reference/master/indices-stats.html", + "stability": "stable", "methods": ["GET"], "url": { - "path": "/_stats", "paths": [ "/_stats", "/_stats/{metric}", diff --git a/rest-api-spec/src/main/resources/rest-api-spec/api/indices.update_aliases.json b/rest-api-spec/src/main/resources/rest-api-spec/api/indices.update_aliases.json index 30c369e410a25..8c75adfe68bf0 100644 --- a/rest-api-spec/src/main/resources/rest-api-spec/api/indices.update_aliases.json +++ b/rest-api-spec/src/main/resources/rest-api-spec/api/indices.update_aliases.json @@ -1,9 +1,9 @@ { "indices.update_aliases": { "documentation": "http://www.elastic.co/guide/en/elasticsearch/reference/master/indices-aliases.html", + "stability": "stable", "methods": ["POST"], "url": { - "path": "/_aliases", "paths": ["/_aliases"], "parts": { }, diff --git a/rest-api-spec/src/main/resources/rest-api-spec/api/indices.upgrade.json b/rest-api-spec/src/main/resources/rest-api-spec/api/indices.upgrade.json index 1e2413ee723cc..ce408a4e1e509 100644 --- a/rest-api-spec/src/main/resources/rest-api-spec/api/indices.upgrade.json +++ b/rest-api-spec/src/main/resources/rest-api-spec/api/indices.upgrade.json @@ -1,9 +1,9 @@ { "indices.upgrade": { "documentation": "http://www.elastic.co/guide/en/elasticsearch/reference/master/indices-upgrade.html", + "stability": "stable", "methods": ["POST"], "url": { - "path": "/_upgrade", "paths": ["/_upgrade", "/{index}/_upgrade"], "parts": { "index": { diff --git a/rest-api-spec/src/main/resources/rest-api-spec/api/indices.validate_query.json b/rest-api-spec/src/main/resources/rest-api-spec/api/indices.validate_query.json index 6f1395b1582a0..a4cf1d0e387c8 100644 --- a/rest-api-spec/src/main/resources/rest-api-spec/api/indices.validate_query.json +++ b/rest-api-spec/src/main/resources/rest-api-spec/api/indices.validate_query.json @@ -1,9 +1,9 @@ { "indices.validate_query": { "documentation": "http://www.elastic.co/guide/en/elasticsearch/reference/master/search-validate.html", + "stability": "stable", "methods": ["GET", "POST"], "url": { - "path": "/_validate/query", "paths": ["/_validate/query", "/{index}/_validate/query"], "deprecated_paths" : [ { diff --git a/rest-api-spec/src/main/resources/rest-api-spec/api/info.json b/rest-api-spec/src/main/resources/rest-api-spec/api/info.json index 63754eb7f7c3c..c1fec52fc8ca4 100644 --- a/rest-api-spec/src/main/resources/rest-api-spec/api/info.json +++ b/rest-api-spec/src/main/resources/rest-api-spec/api/info.json @@ -1,9 +1,9 @@ { "info": { "documentation": "http://www.elastic.co/guide/", + "stability": "stable", "methods": ["GET"], "url": { - "path": "/", "paths": ["/"], "parts": { }, diff --git a/rest-api-spec/src/main/resources/rest-api-spec/api/ingest.delete_pipeline.json b/rest-api-spec/src/main/resources/rest-api-spec/api/ingest.delete_pipeline.json index c3b51de862097..f1b3168941814 100644 --- a/rest-api-spec/src/main/resources/rest-api-spec/api/ingest.delete_pipeline.json +++ b/rest-api-spec/src/main/resources/rest-api-spec/api/ingest.delete_pipeline.json @@ -1,9 +1,9 @@ { "ingest.delete_pipeline": { "documentation": "https://www.elastic.co/guide/en/elasticsearch/reference/master/delete-pipeline-api.html", + "stability": "stable", "methods": [ "DELETE" ], "url": { - "path": "/_ingest/pipeline/{id}", "paths": [ "/_ingest/pipeline/{id}" ], "parts": { "id": { diff --git a/rest-api-spec/src/main/resources/rest-api-spec/api/ingest.get_pipeline.json b/rest-api-spec/src/main/resources/rest-api-spec/api/ingest.get_pipeline.json index 16a07e072b771..e91226c1859f9 100644 --- a/rest-api-spec/src/main/resources/rest-api-spec/api/ingest.get_pipeline.json +++ b/rest-api-spec/src/main/resources/rest-api-spec/api/ingest.get_pipeline.json @@ -1,9 +1,9 @@ { "ingest.get_pipeline": { "documentation": "https://www.elastic.co/guide/en/elasticsearch/reference/master/get-pipeline-api.html", + "stability": "stable", "methods": [ "GET" ], "url": { - "path": "/_ingest/pipeline/{id}", "paths": [ "/_ingest/pipeline", "/_ingest/pipeline/{id}" ], "parts": { "id": { diff --git a/rest-api-spec/src/main/resources/rest-api-spec/api/ingest.processor_grok.json b/rest-api-spec/src/main/resources/rest-api-spec/api/ingest.processor_grok.json index bf40be853e2d1..cb98ef30df948 100644 --- a/rest-api-spec/src/main/resources/rest-api-spec/api/ingest.processor_grok.json +++ b/rest-api-spec/src/main/resources/rest-api-spec/api/ingest.processor_grok.json @@ -1,9 +1,9 @@ { "ingest.processor_grok": { "documentation": "https://www.elastic.co/guide/en/elasticsearch/reference/master/grok-processor.html#grok-processor-rest-get", + "stability": "stable", "methods": [ "GET" ], "url": { - "path": "/_ingest/processor/grok", "paths": ["/_ingest/processor/grok"], "parts": { }, diff --git a/rest-api-spec/src/main/resources/rest-api-spec/api/ingest.put_pipeline.json b/rest-api-spec/src/main/resources/rest-api-spec/api/ingest.put_pipeline.json index 1ea77901d8dbd..166b2b9d4190e 100644 --- a/rest-api-spec/src/main/resources/rest-api-spec/api/ingest.put_pipeline.json +++ b/rest-api-spec/src/main/resources/rest-api-spec/api/ingest.put_pipeline.json @@ -1,9 +1,9 @@ { "ingest.put_pipeline": { "documentation": "https://www.elastic.co/guide/en/elasticsearch/reference/master/put-pipeline-api.html", + "stability": "stable", "methods": [ "PUT" ], "url": { - "path": "/_ingest/pipeline/{id}", "paths": [ "/_ingest/pipeline/{id}" ], "parts": { "id": { diff --git a/rest-api-spec/src/main/resources/rest-api-spec/api/ingest.simulate.json b/rest-api-spec/src/main/resources/rest-api-spec/api/ingest.simulate.json index c16008ad6b659..5010c02424e5b 100644 --- a/rest-api-spec/src/main/resources/rest-api-spec/api/ingest.simulate.json +++ b/rest-api-spec/src/main/resources/rest-api-spec/api/ingest.simulate.json @@ -1,9 +1,9 @@ { "ingest.simulate": { "documentation": "https://www.elastic.co/guide/en/elasticsearch/reference/master/simulate-pipeline-api.html", + "stability": "stable", "methods": [ "GET", "POST" ], "url": { - "path": "/_ingest/pipeline/_simulate", "paths": [ "/_ingest/pipeline/_simulate", "/_ingest/pipeline/{id}/_simulate" ], "parts": { "id": { diff --git a/rest-api-spec/src/main/resources/rest-api-spec/api/mget.json b/rest-api-spec/src/main/resources/rest-api-spec/api/mget.json index c04fb34beac18..bd8e258c2e206 100644 --- a/rest-api-spec/src/main/resources/rest-api-spec/api/mget.json +++ b/rest-api-spec/src/main/resources/rest-api-spec/api/mget.json @@ -1,9 +1,9 @@ { "mget": { "documentation": "http://www.elastic.co/guide/en/elasticsearch/reference/master/docs-multi-get.html", + "stability": "stable", "methods": ["GET", "POST"], "url": { - "path": "/_mget", "paths": ["/_mget", "/{index}/_mget"], "deprecated_paths" : [ { diff --git a/rest-api-spec/src/main/resources/rest-api-spec/api/msearch.json b/rest-api-spec/src/main/resources/rest-api-spec/api/msearch.json index 73780a1835893..6f3e0c5e9d3bb 100644 --- a/rest-api-spec/src/main/resources/rest-api-spec/api/msearch.json +++ b/rest-api-spec/src/main/resources/rest-api-spec/api/msearch.json @@ -1,9 +1,9 @@ { "msearch": { "documentation": "http://www.elastic.co/guide/en/elasticsearch/reference/master/search-multi-search.html", + "stability": "stable", "methods": ["GET", "POST"], "url": { - "path": "/_msearch", "paths": ["/_msearch", "/{index}/_msearch"], "parts": { "index": { diff --git a/rest-api-spec/src/main/resources/rest-api-spec/api/msearch_template.json b/rest-api-spec/src/main/resources/rest-api-spec/api/msearch_template.json index 7bc957d98ff29..412f4a508f433 100644 --- a/rest-api-spec/src/main/resources/rest-api-spec/api/msearch_template.json +++ b/rest-api-spec/src/main/resources/rest-api-spec/api/msearch_template.json @@ -1,9 +1,9 @@ { "msearch_template": { "documentation": "http://www.elastic.co/guide/en/elasticsearch/reference/current/search-multi-search.html", + "stability": "stable", "methods": ["GET", "POST"], "url": { - "path": "/_msearch/template", "paths": ["/_msearch/template", "/{index}/_msearch/template"], "parts": { "index": { diff --git a/rest-api-spec/src/main/resources/rest-api-spec/api/mtermvectors.json b/rest-api-spec/src/main/resources/rest-api-spec/api/mtermvectors.json index aaff8e73259cf..7b722a5bebfa5 100644 --- a/rest-api-spec/src/main/resources/rest-api-spec/api/mtermvectors.json +++ b/rest-api-spec/src/main/resources/rest-api-spec/api/mtermvectors.json @@ -1,9 +1,9 @@ { "mtermvectors" : { "documentation" : "http://www.elastic.co/guide/en/elasticsearch/reference/master/docs-multi-termvectors.html", + "stability": "stable", "methods" : ["GET", "POST"], "url" : { - "path" : "/_mtermvectors", "paths" : ["/_mtermvectors", "/{index}/_mtermvectors"], "parts" : { "index" : { diff --git a/rest-api-spec/src/main/resources/rest-api-spec/api/nodes.hot_threads.json b/rest-api-spec/src/main/resources/rest-api-spec/api/nodes.hot_threads.json index 95df4f53aa579..b94fbaaaedf88 100644 --- a/rest-api-spec/src/main/resources/rest-api-spec/api/nodes.hot_threads.json +++ b/rest-api-spec/src/main/resources/rest-api-spec/api/nodes.hot_threads.json @@ -1,9 +1,9 @@ { "nodes.hot_threads": { "documentation": "http://www.elastic.co/guide/en/elasticsearch/reference/master/cluster-nodes-hot-threads.html", + "stability": "stable", "methods": ["GET"], "url": { - "path": "/_nodes/hot_threads", "paths": ["/_nodes/hot_threads", "/_nodes/{node_id}/hot_threads"], "deprecated_paths" : [ { diff --git a/rest-api-spec/src/main/resources/rest-api-spec/api/nodes.info.json b/rest-api-spec/src/main/resources/rest-api-spec/api/nodes.info.json index c86d35dc1a95a..ca6637d81dcb4 100644 --- a/rest-api-spec/src/main/resources/rest-api-spec/api/nodes.info.json +++ b/rest-api-spec/src/main/resources/rest-api-spec/api/nodes.info.json @@ -1,9 +1,9 @@ { "nodes.info": { "documentation": "http://www.elastic.co/guide/en/elasticsearch/reference/master/cluster-nodes-info.html", + "stability": "stable", "methods": ["GET"], "url": { - "path": "/_nodes", "paths": ["/_nodes", "/_nodes/{node_id}", "/_nodes/{metric}", "/_nodes/{node_id}/{metric}"], "parts": { "node_id": { diff --git a/rest-api-spec/src/main/resources/rest-api-spec/api/nodes.reload_secure_settings.json b/rest-api-spec/src/main/resources/rest-api-spec/api/nodes.reload_secure_settings.json index 0a566df35bbc2..e40e66cb266bb 100644 --- a/rest-api-spec/src/main/resources/rest-api-spec/api/nodes.reload_secure_settings.json +++ b/rest-api-spec/src/main/resources/rest-api-spec/api/nodes.reload_secure_settings.json @@ -1,9 +1,9 @@ { "nodes.reload_secure_settings": { "documentation": "https://www.elastic.co/guide/en/elasticsearch/reference/master/secure-settings.html#reloadable-secure-settings", + "stability": "stable", "methods": ["POST"], "url": { - "path": "/_nodes/reload_secure_settings", "paths": ["/_nodes/reload_secure_settings", "/_nodes/{node_id}/reload_secure_settings"], "parts": { "node_id": { diff --git a/rest-api-spec/src/main/resources/rest-api-spec/api/nodes.stats.json b/rest-api-spec/src/main/resources/rest-api-spec/api/nodes.stats.json index 7b426d03918b2..50af7b6d478fc 100644 --- a/rest-api-spec/src/main/resources/rest-api-spec/api/nodes.stats.json +++ b/rest-api-spec/src/main/resources/rest-api-spec/api/nodes.stats.json @@ -1,9 +1,9 @@ { "nodes.stats": { "documentation": "http://www.elastic.co/guide/en/elasticsearch/reference/master/cluster-nodes-stats.html", + "stability": "stable", "methods": ["GET"], "url": { - "path": "/_nodes/stats", "paths": [ "/_nodes/stats", "/_nodes/{node_id}/stats", diff --git a/rest-api-spec/src/main/resources/rest-api-spec/api/nodes.usage.json b/rest-api-spec/src/main/resources/rest-api-spec/api/nodes.usage.json index 97c3f201f6cdf..6b70af58c08ea 100644 --- a/rest-api-spec/src/main/resources/rest-api-spec/api/nodes.usage.json +++ b/rest-api-spec/src/main/resources/rest-api-spec/api/nodes.usage.json @@ -1,9 +1,9 @@ { "nodes.usage": { "documentation": "http://www.elastic.co/guide/en/elasticsearch/reference/master/cluster-nodes-usage.html", + "stability": "stable", "methods": ["GET"], "url": { - "path": "/_nodes/usage", "paths": [ "/_nodes/usage", "/_nodes/{node_id}/usage", diff --git a/rest-api-spec/src/main/resources/rest-api-spec/api/ping.json b/rest-api-spec/src/main/resources/rest-api-spec/api/ping.json index 41310bb30d494..d49dd78721659 100644 --- a/rest-api-spec/src/main/resources/rest-api-spec/api/ping.json +++ b/rest-api-spec/src/main/resources/rest-api-spec/api/ping.json @@ -1,9 +1,9 @@ { "ping": { "documentation": "http://www.elastic.co/guide/", + "stability": "stable", "methods": ["HEAD"], "url": { - "path": "/", "paths": ["/"], "parts": { }, diff --git a/rest-api-spec/src/main/resources/rest-api-spec/api/put_script.json b/rest-api-spec/src/main/resources/rest-api-spec/api/put_script.json index 34bd4f63c285e..5de2bf9bd306f 100644 --- a/rest-api-spec/src/main/resources/rest-api-spec/api/put_script.json +++ b/rest-api-spec/src/main/resources/rest-api-spec/api/put_script.json @@ -1,9 +1,9 @@ { "put_script": { "documentation": "http://www.elastic.co/guide/en/elasticsearch/reference/master/modules-scripting.html", + "stability": "stable", "methods": ["PUT", "POST"], "url": { - "path": "/_scripts/{id}", "paths": [ "/_scripts/{id}", "/_scripts/{id}/{context}" ], "parts": { "id": { diff --git a/rest-api-spec/src/main/resources/rest-api-spec/api/rank_eval.json b/rest-api-spec/src/main/resources/rest-api-spec/api/rank_eval.json index 5c9cebf741135..8bb2212d7a68a 100644 --- a/rest-api-spec/src/main/resources/rest-api-spec/api/rank_eval.json +++ b/rest-api-spec/src/main/resources/rest-api-spec/api/rank_eval.json @@ -1,9 +1,9 @@ { "rank_eval": { "documentation": "https://www.elastic.co/guide/en/elasticsearch/reference/master/search-rank-eval.html", + "stability" : "experimental", "methods": ["GET", "POST"], "url": { - "path": "/_rank_eval", "paths": ["/_rank_eval", "/{index}/_rank_eval"], "parts": { "index": { diff --git a/rest-api-spec/src/main/resources/rest-api-spec/api/reindex.json b/rest-api-spec/src/main/resources/rest-api-spec/api/reindex.json index e85eadb5bc41e..e4305bdd7a484 100644 --- a/rest-api-spec/src/main/resources/rest-api-spec/api/reindex.json +++ b/rest-api-spec/src/main/resources/rest-api-spec/api/reindex.json @@ -1,9 +1,9 @@ { "reindex": { "documentation": "https://www.elastic.co/guide/en/elasticsearch/reference/master/docs-reindex.html", + "stability": "stable", "methods": ["POST"], "url": { - "path": "/_reindex", "paths": ["/_reindex"], "parts": {}, "params": { @@ -39,6 +39,10 @@ "type": "number", "default": 1, "description": "The number of slices this task should be divided into. Defaults to 1 meaning the task isn't sliced into subtasks." + }, + "max_docs": { + "type" : "number", + "description" : "Maximum number of documents to process (default: all documents)" } } }, diff --git a/rest-api-spec/src/main/resources/rest-api-spec/api/reindex_rethrottle.json b/rest-api-spec/src/main/resources/rest-api-spec/api/reindex_rethrottle.json index 2763eb8983fd5..a6957c5f7c71a 100644 --- a/rest-api-spec/src/main/resources/rest-api-spec/api/reindex_rethrottle.json +++ b/rest-api-spec/src/main/resources/rest-api-spec/api/reindex_rethrottle.json @@ -1,9 +1,9 @@ { "reindex_rethrottle": { "documentation": "https://www.elastic.co/guide/en/elasticsearch/reference/master/docs-reindex.html", + "stability": "stable", "methods": ["POST"], "url": { - "path": "/_reindex/{task_id}/_rethrottle", "paths": ["/_reindex/{task_id}/_rethrottle"], "parts": { "task_id": { diff --git a/rest-api-spec/src/main/resources/rest-api-spec/api/render_search_template.json b/rest-api-spec/src/main/resources/rest-api-spec/api/render_search_template.json index 8f27c12b819da..fa3af422fee17 100644 --- a/rest-api-spec/src/main/resources/rest-api-spec/api/render_search_template.json +++ b/rest-api-spec/src/main/resources/rest-api-spec/api/render_search_template.json @@ -1,9 +1,9 @@ { "render_search_template": { "documentation": "http://www.elasticsearch.org/guide/en/elasticsearch/reference/master/search-template.html", + "stability": "stable", "methods": ["GET", "POST"], "url": { - "path": "/_render/template", "paths": [ "/_render/template", "/_render/template/{id}" ], "parts": { "id": { diff --git a/rest-api-spec/src/main/resources/rest-api-spec/api/scripts_painless_execute.json b/rest-api-spec/src/main/resources/rest-api-spec/api/scripts_painless_execute.json index c02627cfd874c..3cb9c18ef2103 100644 --- a/rest-api-spec/src/main/resources/rest-api-spec/api/scripts_painless_execute.json +++ b/rest-api-spec/src/main/resources/rest-api-spec/api/scripts_painless_execute.json @@ -1,9 +1,9 @@ { "scripts_painless_execute": { "documentation": "https://www.elastic.co/guide/en/elasticsearch/painless/master/painless-execute-api.html", + "stability" : "experimental", "methods": ["GET", "POST"], "url": { - "path": "/_scripts/painless/_execute", "paths": ["/_scripts/painless/_execute"], "parts": { }, diff --git a/rest-api-spec/src/main/resources/rest-api-spec/api/scroll.json b/rest-api-spec/src/main/resources/rest-api-spec/api/scroll.json index 31e02a172fd0e..88851b2db1e34 100644 --- a/rest-api-spec/src/main/resources/rest-api-spec/api/scroll.json +++ b/rest-api-spec/src/main/resources/rest-api-spec/api/scroll.json @@ -1,9 +1,9 @@ { "scroll": { "documentation": "http://www.elastic.co/guide/en/elasticsearch/reference/master/search-request-scroll.html", + "stability": "stable", "methods": ["GET", "POST"], "url": { - "path": "/_search/scroll", "paths": ["/_search/scroll"], "deprecated_paths" : [ { diff --git a/rest-api-spec/src/main/resources/rest-api-spec/api/search.json b/rest-api-spec/src/main/resources/rest-api-spec/api/search.json index 75444eb66767b..30f5c6747f3a8 100644 --- a/rest-api-spec/src/main/resources/rest-api-spec/api/search.json +++ b/rest-api-spec/src/main/resources/rest-api-spec/api/search.json @@ -1,9 +1,9 @@ { "search": { "documentation": "http://www.elastic.co/guide/en/elasticsearch/reference/master/search-search.html", + "stability": "stable", "methods": ["GET", "POST"], "url": { - "path": "/_search", "paths": ["/_search", "/{index}/_search"], "parts": { "index": { diff --git a/rest-api-spec/src/main/resources/rest-api-spec/api/search_shards.json b/rest-api-spec/src/main/resources/rest-api-spec/api/search_shards.json index b3de107b79787..082603209925e 100644 --- a/rest-api-spec/src/main/resources/rest-api-spec/api/search_shards.json +++ b/rest-api-spec/src/main/resources/rest-api-spec/api/search_shards.json @@ -1,9 +1,9 @@ { "search_shards": { "documentation": "http://www.elastic.co/guide/en/elasticsearch/reference/master/search-shards.html", + "stability": "stable", "methods": ["GET", "POST"], "url": { - "path": "/{index}/_search_shards", "paths": ["/_search_shards", "/{index}/_search_shards"], "parts": { "index": { diff --git a/rest-api-spec/src/main/resources/rest-api-spec/api/search_template.json b/rest-api-spec/src/main/resources/rest-api-spec/api/search_template.json index 81a293d4a9b34..a6777d0628126 100644 --- a/rest-api-spec/src/main/resources/rest-api-spec/api/search_template.json +++ b/rest-api-spec/src/main/resources/rest-api-spec/api/search_template.json @@ -1,9 +1,9 @@ { "search_template": { "documentation": "http://www.elastic.co/guide/en/elasticsearch/reference/current/search-template.html", + "stability": "stable", "methods": ["GET", "POST"], "url": { - "path": "/_search/template", "paths": ["/_search/template", "/{index}/_search/template"], "parts": { "index": { diff --git a/rest-api-spec/src/main/resources/rest-api-spec/api/snapshot.create.json b/rest-api-spec/src/main/resources/rest-api-spec/api/snapshot.create.json index 29ae2206c8568..66be1eeff675c 100644 --- a/rest-api-spec/src/main/resources/rest-api-spec/api/snapshot.create.json +++ b/rest-api-spec/src/main/resources/rest-api-spec/api/snapshot.create.json @@ -1,9 +1,9 @@ { "snapshot.create": { "documentation": "http://www.elastic.co/guide/en/elasticsearch/reference/master/modules-snapshots.html", + "stability": "stable", "methods": ["PUT", "POST"], "url": { - "path": "/_snapshot/{repository}/{snapshot}", "paths": ["/_snapshot/{repository}/{snapshot}"], "parts": { "repository": { diff --git a/rest-api-spec/src/main/resources/rest-api-spec/api/snapshot.create_repository.json b/rest-api-spec/src/main/resources/rest-api-spec/api/snapshot.create_repository.json index 7a73f6abc3c28..830eb2952886e 100644 --- a/rest-api-spec/src/main/resources/rest-api-spec/api/snapshot.create_repository.json +++ b/rest-api-spec/src/main/resources/rest-api-spec/api/snapshot.create_repository.json @@ -1,9 +1,9 @@ { "snapshot.create_repository": { "documentation": "http://www.elastic.co/guide/en/elasticsearch/reference/master/modules-snapshots.html", + "stability": "stable", "methods": ["PUT", "POST"], "url": { - "path": "/_snapshot/{repository}", "paths": ["/_snapshot/{repository}"], "parts": { "repository": { diff --git a/rest-api-spec/src/main/resources/rest-api-spec/api/snapshot.delete.json b/rest-api-spec/src/main/resources/rest-api-spec/api/snapshot.delete.json index 6668289f442a6..c04dc5410ea10 100644 --- a/rest-api-spec/src/main/resources/rest-api-spec/api/snapshot.delete.json +++ b/rest-api-spec/src/main/resources/rest-api-spec/api/snapshot.delete.json @@ -1,9 +1,9 @@ { "snapshot.delete": { "documentation": "http://www.elastic.co/guide/en/elasticsearch/reference/master/modules-snapshots.html", + "stability": "stable", "methods": ["DELETE"], "url": { - "path": "/_snapshot/{repository}/{snapshot}", "paths": ["/_snapshot/{repository}/{snapshot}"], "parts": { "repository": { diff --git a/rest-api-spec/src/main/resources/rest-api-spec/api/snapshot.delete_repository.json b/rest-api-spec/src/main/resources/rest-api-spec/api/snapshot.delete_repository.json index 4f0c43b8f0317..1912423978173 100644 --- a/rest-api-spec/src/main/resources/rest-api-spec/api/snapshot.delete_repository.json +++ b/rest-api-spec/src/main/resources/rest-api-spec/api/snapshot.delete_repository.json @@ -1,9 +1,9 @@ { "snapshot.delete_repository": { "documentation": "http://www.elastic.co/guide/en/elasticsearch/reference/master/modules-snapshots.html", + "stability": "stable", "methods": ["DELETE"], "url": { - "path": "/_snapshot/{repository}", "paths": ["/_snapshot/{repository}"], "parts": { "repository": { diff --git a/rest-api-spec/src/main/resources/rest-api-spec/api/snapshot.get.json b/rest-api-spec/src/main/resources/rest-api-spec/api/snapshot.get.json index 02f5259bc2ec8..b23a3fbc0281d 100644 --- a/rest-api-spec/src/main/resources/rest-api-spec/api/snapshot.get.json +++ b/rest-api-spec/src/main/resources/rest-api-spec/api/snapshot.get.json @@ -1,9 +1,9 @@ { "snapshot.get": { "documentation": "http://www.elastic.co/guide/en/elasticsearch/reference/master/modules-snapshots.html", + "stability": "stable", "methods": ["GET"], "url": { - "path": "/_snapshot/{repository}/{snapshot}", "paths": ["/_snapshot/{repository}/{snapshot}"], "parts": { "repository": { diff --git a/rest-api-spec/src/main/resources/rest-api-spec/api/snapshot.get_repository.json b/rest-api-spec/src/main/resources/rest-api-spec/api/snapshot.get_repository.json index bf6a660146173..9aa13758097c4 100644 --- a/rest-api-spec/src/main/resources/rest-api-spec/api/snapshot.get_repository.json +++ b/rest-api-spec/src/main/resources/rest-api-spec/api/snapshot.get_repository.json @@ -1,9 +1,9 @@ { "snapshot.get_repository": { "documentation": "http://www.elastic.co/guide/en/elasticsearch/reference/master/modules-snapshots.html", + "stability": "stable", "methods": ["GET"], "url": { - "path": "/_snapshot", "paths": ["/_snapshot", "/_snapshot/{repository}"], "parts": { "repository": { diff --git a/rest-api-spec/src/main/resources/rest-api-spec/api/snapshot.restore.json b/rest-api-spec/src/main/resources/rest-api-spec/api/snapshot.restore.json index bdd265799ce4b..6107dd1663a37 100644 --- a/rest-api-spec/src/main/resources/rest-api-spec/api/snapshot.restore.json +++ b/rest-api-spec/src/main/resources/rest-api-spec/api/snapshot.restore.json @@ -1,9 +1,9 @@ { "snapshot.restore": { "documentation": "http://www.elastic.co/guide/en/elasticsearch/reference/master/modules-snapshots.html", + "stability": "stable", "methods": ["POST"], "url": { - "path": "/_snapshot/{repository}/{snapshot}/_restore", "paths": ["/_snapshot/{repository}/{snapshot}/_restore"], "parts": { "repository": { diff --git a/rest-api-spec/src/main/resources/rest-api-spec/api/snapshot.status.json b/rest-api-spec/src/main/resources/rest-api-spec/api/snapshot.status.json index cba488de793bc..e76ce7893951e 100644 --- a/rest-api-spec/src/main/resources/rest-api-spec/api/snapshot.status.json +++ b/rest-api-spec/src/main/resources/rest-api-spec/api/snapshot.status.json @@ -1,9 +1,9 @@ { "snapshot.status": { "documentation": "http://www.elastic.co/guide/en/elasticsearch/reference/master/modules-snapshots.html", + "stability": "stable", "methods": ["GET"], "url": { - "path": "/_snapshot/_status", "paths": ["/_snapshot/_status", "/_snapshot/{repository}/_status", "/_snapshot/{repository}/{snapshot}/_status"], "parts": { "repository": { diff --git a/rest-api-spec/src/main/resources/rest-api-spec/api/snapshot.verify_repository.json b/rest-api-spec/src/main/resources/rest-api-spec/api/snapshot.verify_repository.json index 0d9a97c10877b..c875991b21a56 100644 --- a/rest-api-spec/src/main/resources/rest-api-spec/api/snapshot.verify_repository.json +++ b/rest-api-spec/src/main/resources/rest-api-spec/api/snapshot.verify_repository.json @@ -1,9 +1,9 @@ { "snapshot.verify_repository": { "documentation": "http://www.elastic.co/guide/en/elasticsearch/reference/master/modules-snapshots.html", + "stability": "stable", "methods": ["POST"], "url": { - "path": "/_snapshot/{repository}/_verify", "paths": ["/_snapshot/{repository}/_verify"], "parts": { "repository": { diff --git a/rest-api-spec/src/main/resources/rest-api-spec/api/tasks.cancel.json b/rest-api-spec/src/main/resources/rest-api-spec/api/tasks.cancel.json index 244ca986f9b7f..4ee287ee6454b 100644 --- a/rest-api-spec/src/main/resources/rest-api-spec/api/tasks.cancel.json +++ b/rest-api-spec/src/main/resources/rest-api-spec/api/tasks.cancel.json @@ -1,9 +1,9 @@ { "tasks.cancel": { "documentation": "http://www.elastic.co/guide/en/elasticsearch/reference/master/tasks.html", + "stability": "stable", "methods": ["POST"], "url": { - "path": "/_tasks", "paths": ["/_tasks/_cancel", "/_tasks/{task_id}/_cancel"], "parts": { "task_id": { diff --git a/rest-api-spec/src/main/resources/rest-api-spec/api/tasks.get.json b/rest-api-spec/src/main/resources/rest-api-spec/api/tasks.get.json index 3fefdb8f965ff..63702f7c592c6 100644 --- a/rest-api-spec/src/main/resources/rest-api-spec/api/tasks.get.json +++ b/rest-api-spec/src/main/resources/rest-api-spec/api/tasks.get.json @@ -1,9 +1,9 @@ { "tasks.get": { "documentation": "http://www.elastic.co/guide/en/elasticsearch/reference/master/tasks.html", + "stability": "stable", "methods": ["GET"], "url": { - "path": "/_tasks/{task_id}", "paths": ["/_tasks/{task_id}"], "parts": { "task_id": { diff --git a/rest-api-spec/src/main/resources/rest-api-spec/api/tasks.list.json b/rest-api-spec/src/main/resources/rest-api-spec/api/tasks.list.json index f16967eafeb02..8bb31723e0be4 100644 --- a/rest-api-spec/src/main/resources/rest-api-spec/api/tasks.list.json +++ b/rest-api-spec/src/main/resources/rest-api-spec/api/tasks.list.json @@ -1,9 +1,9 @@ { "tasks.list": { "documentation": "http://www.elastic.co/guide/en/elasticsearch/reference/master/tasks.html", + "stability": "stable", "methods": ["GET"], "url": { - "path": "/_tasks", "paths": ["/_tasks"], "parts": {}, "params": { diff --git a/rest-api-spec/src/main/resources/rest-api-spec/api/termvectors.json b/rest-api-spec/src/main/resources/rest-api-spec/api/termvectors.json index bbbdc7c87ad0b..2bfacaf676c67 100644 --- a/rest-api-spec/src/main/resources/rest-api-spec/api/termvectors.json +++ b/rest-api-spec/src/main/resources/rest-api-spec/api/termvectors.json @@ -1,9 +1,9 @@ { "termvectors" : { "documentation" : "http://www.elastic.co/guide/en/elasticsearch/reference/master/docs-termvectors.html", + "stability": "stable", "methods" : ["GET", "POST"], "url" : { - "path" : "/{index}/_termvectors/{id}", "paths" : ["/{index}/_termvectors/{id}", "/{index}/_termvectors"], "parts" : { "index" : { diff --git a/rest-api-spec/src/main/resources/rest-api-spec/api/update.json b/rest-api-spec/src/main/resources/rest-api-spec/api/update.json index 02435190674cf..4b850002a3337 100644 --- a/rest-api-spec/src/main/resources/rest-api-spec/api/update.json +++ b/rest-api-spec/src/main/resources/rest-api-spec/api/update.json @@ -1,9 +1,9 @@ { "update": { "documentation": "http://www.elastic.co/guide/en/elasticsearch/reference/master/docs-update.html", + "stability": "stable", "methods": ["POST"], "url": { - "path": "/{index}/_update/{id}", "paths": ["/{index}/_update/{id}"], "deprecated_paths" : [ { diff --git a/rest-api-spec/src/main/resources/rest-api-spec/api/update_by_query.json b/rest-api-spec/src/main/resources/rest-api-spec/api/update_by_query.json index 10b6e6d87f277..4048b4a55d962 100644 --- a/rest-api-spec/src/main/resources/rest-api-spec/api/update_by_query.json +++ b/rest-api-spec/src/main/resources/rest-api-spec/api/update_by_query.json @@ -1,9 +1,9 @@ { "update_by_query": { "documentation": "https://www.elastic.co/guide/en/elasticsearch/reference/master/docs-update-by-query.html", + "stability": "stable", "methods": ["POST"], "url": { - "path": "/{index}/_update_by_query", "paths": ["/{index}/_update_by_query"], "comment": "most things below this are just copied from search.json", "parts": { @@ -90,9 +90,9 @@ "type" : "time", "description" : "Explicit timeout for each search request. Defaults to no timeout." }, - "size": { + "max_docs": { "type" : "number", - "description" : "Number of hits to return (default: 10)" + "description" : "Maximum number of documents to process (default: all documents)" }, "sort": { "type" : "list", diff --git a/rest-api-spec/src/main/resources/rest-api-spec/api/update_by_query_rethrottle.json b/rest-api-spec/src/main/resources/rest-api-spec/api/update_by_query_rethrottle.json index 9ec2540b43006..e79f2902329a6 100644 --- a/rest-api-spec/src/main/resources/rest-api-spec/api/update_by_query_rethrottle.json +++ b/rest-api-spec/src/main/resources/rest-api-spec/api/update_by_query_rethrottle.json @@ -1,9 +1,9 @@ { "update_by_query_rethrottle": { "documentation": "https://www.elastic.co/guide/en/elasticsearch/reference/current/docs-update-by-query.html", + "stability": "stable", "methods": ["POST"], "url": { - "path": "/_update_by_query/{task_id}/_rethrottle", "paths": ["/_update_by_query/{task_id}/_rethrottle"], "parts": { "task_id": { diff --git a/rest-api-spec/src/main/resources/rest-api-spec/test/cat.snapshots/10_basic.yml b/rest-api-spec/src/main/resources/rest-api-spec/test/cat.snapshots/10_basic.yml index 6e03ceb98c716..4f425f4817638 100644 --- a/rest-api-spec/src/main/resources/rest-api-spec/test/cat.snapshots/10_basic.yml +++ b/rest-api-spec/src/main/resources/rest-api-spec/test/cat.snapshots/10_basic.yml @@ -1,5 +1,9 @@ --- "Help": + - skip: + version: " - 7.9.99" + reason: Repository field added in 8.0 + - do: cat.snapshots: help: true @@ -7,6 +11,7 @@ - match: $body: | /^ id .+ \n + repository .+ \n status .+ \n start_epoch .+ \n start_time .+ \n @@ -21,6 +26,9 @@ $/ --- "Test cat snapshots output": + - skip: + version: " - 7.9.99" + reason: Repository field added in 8.0 - do: snapshot.create_repository: @@ -74,6 +82,6 @@ - match: $body: | - /^ snap1\s+ SUCCESS\s+ \d+\s+ \d\d\:\d\d\:\d\d\s+ \d+\s+ \d\d\:\d\d\:\d\d\s+ \S+\s+ 2\s+ 2\s+ 0\s+ 2\s*\n - snap2\s+ SUCCESS\s+ \d+\s+ \d\d\:\d\d\:\d\d\s+ \d+\s+ \d\d\:\d\d\:\d\d\s+ \S+\s+ 2\s+ 2\s+ 0\s+ 2\s*\n + /^ snap1\s+ test_cat_snapshots_1\s+ SUCCESS\s+ \d+\s+ \d\d\:\d\d\:\d\d\s+ \d+\s+ \d\d\:\d\d\:\d\d\s+ \S+\s+ 2\s+ 2\s+ 0\s+ 2\s*\n + snap2\s+ test_cat_snapshots_1\s+ SUCCESS\s+ \d+\s+ \d\d\:\d\d\:\d\d\s+ \d+\s+ \d\d\:\d\d\:\d\d\s+ \S+\s+ 2\s+ 2\s+ 0\s+ 2\s*\n $/ diff --git a/rest-api-spec/src/main/resources/rest-api-spec/test/count/11_basic_with_types.yml b/rest-api-spec/src/main/resources/rest-api-spec/test/count/11_basic_with_types.yml deleted file mode 100644 index 48cfc610b435e..0000000000000 --- a/rest-api-spec/src/main/resources/rest-api-spec/test/count/11_basic_with_types.yml +++ /dev/null @@ -1,66 +0,0 @@ -setup: - - do: - indices.create: - index: test - - do: - index: - index: test - type: test - id: 1 - body: { foo: bar } - - - do: - indices.refresh: - index: [test] - ---- -"count with body": - - do: - count: - index: test - type: test - body: - query: - match: - foo: bar - - - match: {count : 1} - - - do: - count: - index: test - body: - query: - match: - foo: test - - - match: {count : 0} - ---- -"count with empty body": -# empty body should default to match_all query - - do: - count: - index: test - type: test - body: { } - - - match: {count : 1} - - - do: - count: - index: test - type: test - - - match: {count : 1} - ---- -"count body without query element": - - do: - catch: bad_request - count: - index: test - type: test - body: - match: - foo: bar diff --git a/rest-api-spec/src/main/resources/rest-api-spec/test/indices.analyze/10_analyze.yml b/rest-api-spec/src/main/resources/rest-api-spec/test/indices.analyze/10_analyze.yml index 8062a96f3e6bc..a852e6d3beef1 100644 --- a/rest-api-spec/src/main/resources/rest-api-spec/test/indices.analyze/10_analyze.yml +++ b/rest-api-spec/src/main/resources/rest-api-spec/test/indices.analyze/10_analyze.yml @@ -72,5 +72,4 @@ - match: { detail.tokenizer.tokens.0.token: foo } - match: { detail.tokenizer.tokens.1.token: bar } - match: { detail.tokenizer.tokens.2.token: buzz } - - match: { detail.tokenfilters.0.name: "_anonymous_tokenfilter" } - match: { detail.tokenfilters.0.tokens.0.token: bar } diff --git a/rest-api-spec/src/main/resources/rest-api-spec/test/indices.recovery/10_basic.yml b/rest-api-spec/src/main/resources/rest-api-spec/test/indices.recovery/10_basic.yml index f227e076aa9c6..4806601cec263 100644 --- a/rest-api-spec/src/main/resources/rest-api-spec/test/indices.recovery/10_basic.yml +++ b/rest-api-spec/src/main/resources/rest-api-spec/test/indices.recovery/10_basic.yml @@ -130,3 +130,28 @@ index: [v*] - match: { $body: {} } +--- +"Indices recovery test with detailed parameter": + - skip: + version: " - 7.2.99" + reason: bug with detailed parameter fixed in 7.3 + + - do: + indices.create: + index: test_3 + body: + settings: + index: + number_of_replicas: 0 + + - do: + cluster.health: + wait_for_status: green + + - do: + indices.recovery: + index: [test_3] + human: true + detailed: true + + - match: { test_3.shards.0.index.files.details: [] } diff --git a/rest-api-spec/src/main/resources/rest-api-spec/test/indices.stats/20_translog.yml b/rest-api-spec/src/main/resources/rest-api-spec/test/indices.stats/20_translog.yml index 83b9c429bbdbf..016bd5912d3a3 100644 --- a/rest-api-spec/src/main/resources/rest-api-spec/test/indices.stats/20_translog.yml +++ b/rest-api-spec/src/main/resources/rest-api-spec/test/indices.stats/20_translog.yml @@ -77,3 +77,47 @@ setup: indices.stats: metric: [ translog ] - gte: { indices.test.primaries.translog.earliest_last_modified_age: 0 } + +--- +"Translog stats on closed indices": + - skip: + version: " - 7.2.99" + reason: "closed indices have translog stats starting version 7.3.0" + + - do: + index: + index: test + id: 1 + body: { "foo": "bar" } + + - do: + index: + index: test + id: 2 + body: { "foo": "bar" } + + - do: + index: + index: test + id: 3 + body: { "foo": "bar" } + + - do: + indices.stats: + metric: [ translog ] + - match: { indices.test.primaries.translog.operations: 3 } + - match: { indices.test.primaries.translog.uncommitted_operations: 3 } + + - do: + indices.close: + index: test + wait_for_active_shards: 1 + - is_true: acknowledged + + - do: + indices.stats: + metric: [ translog ] + expand_wildcards: all + forbid_closed_indices: false + - match: { indices.test.primaries.translog.operations: 3 } + - match: { indices.test.primaries.translog.uncommitted_operations: 0 } diff --git a/rest-api-spec/src/main/resources/rest-api-spec/test/mlt/20_docs.yml b/rest-api-spec/src/main/resources/rest-api-spec/test/mlt/20_docs.yml index bb1b25a0dcb40..50eb344d99048 100644 --- a/rest-api-spec/src/main/resources/rest-api-spec/test/mlt/20_docs.yml +++ b/rest-api-spec/src/main/resources/rest-api-spec/test/mlt/20_docs.yml @@ -41,12 +41,10 @@ like: - _index: test_1 - _type: _doc doc: foo: bar - _index: test_1 - _type: _doc _id: 2 - _id: 3 diff --git a/rest-api-spec/src/main/resources/rest-api-spec/test/mlt/30_unlike.yml b/rest-api-spec/src/main/resources/rest-api-spec/test/mlt/30_unlike.yml index abea4c8fbe57a..a0f96eb6b2d1f 100644 --- a/rest-api-spec/src/main/resources/rest-api-spec/test/mlt/30_unlike.yml +++ b/rest-api-spec/src/main/resources/rest-api-spec/test/mlt/30_unlike.yml @@ -40,11 +40,9 @@ more_like_this: like: _index: test_1 - _type: _doc _id: 1 unlike: _index: test_1 - _type: _doc _id: 3 include: true min_doc_freq: 0 diff --git a/rest-api-spec/src/main/resources/rest-api-spec/test/msearch/12_basic_with_types.yml b/rest-api-spec/src/main/resources/rest-api-spec/test/msearch/12_basic_with_types.yml deleted file mode 100644 index 64e88de404ab7..0000000000000 --- a/rest-api-spec/src/main/resources/rest-api-spec/test/msearch/12_basic_with_types.yml +++ /dev/null @@ -1,97 +0,0 @@ ---- -setup: - - - do: - index: - index: index_1 - type: test - id: 1 - body: { foo: bar } - - - do: - index: - index: index_1 - type: test - id: 2 - body: { foo: baz } - - - do: - index: - index: index_1 - type: test - id: 3 - body: { foo: foo } - - - do: - index: - index: index_2 - type: test - id: 1 - body: { foo: foo } - - - do: - indices.refresh: {} - ---- -"Basic multi-search": - - - do: - msearch: - rest_total_hits_as_int: true - body: - - index: index_* - - query: - match: {foo: foo} - - index: index_2 - - query: - match_all: {} - - index: index_1 - - query: - match: {foo: foo} - - index: index_3 - - query: - match_all: {} - - type: test - - query: - match_all: {} - - - match: { responses.0.hits.total: 2 } - - match: { responses.1.hits.total: 1 } - - match: { responses.2.hits.total: 1 } - - match: { responses.3.error.root_cause.0.type: index_not_found_exception } - - match: { responses.3.error.root_cause.0.reason: "/no.such.index/" } - - match: { responses.3.error.root_cause.0.index: index_3 } - - match: { responses.4.hits.total: 4 } - ---- -"Least impact smoke test": -# only passing these parameters to make sure they are consumed - - do: - msearch: - rest_total_hits_as_int: true - max_concurrent_shard_requests: 1 - max_concurrent_searches: 1 - body: - - index: index_* - - query: - match: {foo: foo} - - index: index_2 - - query: - match_all: {} - - index: index_1 - - query: - match: {foo: foo} - - index: index_3 - - query: - match_all: {} - - type: test - - query: - match_all: {} - - - match: { responses.0.hits.total: 2 } - - match: { responses.1.hits.total: 1 } - - match: { responses.2.hits.total: 1 } - - match: { responses.3.error.root_cause.0.type: index_not_found_exception } - - match: { responses.3.error.root_cause.0.reason: "/no.such.index/" } - - match: { responses.3.error.root_cause.0.index: index_3 } - - match: { responses.4.hits.total: 4 } diff --git a/rest-api-spec/src/main/resources/rest-api-spec/test/mtermvectors/20_deprecated.yml b/rest-api-spec/src/main/resources/rest-api-spec/test/mtermvectors/20_deprecated.yml index db8e566f4b613..8c058deff841c 100644 --- a/rest-api-spec/src/main/resources/rest-api-spec/test/mtermvectors/20_deprecated.yml +++ b/rest-api-spec/src/main/resources/rest-api-spec/test/mtermvectors/20_deprecated.yml @@ -40,7 +40,6 @@ "docs": - "_index" : "testidx" - "_type" : "_doc" "_id" : "testing_document" "version" : 1 "_version_type" : "external" diff --git a/rest-api-spec/src/main/resources/rest-api-spec/test/search.aggregation/200_top_hits_metric.yml b/rest-api-spec/src/main/resources/rest-api-spec/test/search.aggregation/200_top_hits_metric.yml index cde56fa41e3d9..76274e9034d62 100644 --- a/rest-api-spec/src/main/resources/rest-api-spec/test/search.aggregation/200_top_hits_metric.yml +++ b/rest-api-spec/src/main/resources/rest-api-spec/test/search.aggregation/200_top_hits_metric.yml @@ -92,16 +92,19 @@ setup: aggs: users: top_hits: - sort: "users.last.keyword" + sort: + users.last.keyword: + nested: + path: users seq_no_primary_term: true - match: { hits.total: 2 } - length: { aggregations.groups.buckets.0.users.hits.hits: 2 } - - match: { aggregations.groups.buckets.0.users.hits.hits.0._id: "1" } + - match: { aggregations.groups.buckets.0.users.hits.hits.0._id: "2" } - match: { aggregations.groups.buckets.0.users.hits.hits.0._index: my-index } - gte: { aggregations.groups.buckets.0.users.hits.hits.0._seq_no: 0 } - gte: { aggregations.groups.buckets.0.users.hits.hits.0._primary_term: 1 } - - match: { aggregations.groups.buckets.0.users.hits.hits.1._id: "2" } + - match: { aggregations.groups.buckets.0.users.hits.hits.1._id: "1" } - match: { aggregations.groups.buckets.0.users.hits.hits.1._index: my-index } - gte: { aggregations.groups.buckets.0.users.hits.hits.1._seq_no: 0 } - gte: { aggregations.groups.buckets.0.users.hits.hits.1._primary_term: 1 } diff --git a/rest-api-spec/src/main/resources/rest-api-spec/test/search.aggregation/280_rare_terms.yml b/rest-api-spec/src/main/resources/rest-api-spec/test/search.aggregation/280_rare_terms.yml new file mode 100644 index 0000000000000..a82caddd9cfd4 --- /dev/null +++ b/rest-api-spec/src/main/resources/rest-api-spec/test/search.aggregation/280_rare_terms.yml @@ -0,0 +1,316 @@ +setup: + - skip: + version: " - 7.2.99" + reason: RareTerms added in 7.3.0 + - do: + indices.create: + index: test_1 + body: + settings: + number_of_replicas: 0 + mappings: + properties: + str: + type: keyword + ip: + type: ip + boolean: + type: boolean + integer: + type: long + number: + type: long + date: + type: date + + + - do: + cluster.health: + wait_for_status: green + +--- +"Basic test": + - do: + index: + index: test_1 + id: 1 + body: { "str" : "abc" } + + - do: + index: + index: test_1 + id: 2 + body: { "str": "abc" } + + - do: + index: + index: test_1 + id: 3 + body: { "str": "bcd" } + + - do: + indices.refresh: {} + + - do: + search: + body: { "size" : 0, "aggs" : { "str_terms" : { "rare_terms" : { "field" : "str", "max_doc_count" : 1 } } } } + + - match: { hits.total.value: 3 } + - length: { aggregations.str_terms.buckets: 1 } + - match: { aggregations.str_terms.buckets.0.key: "bcd" } + - is_false: aggregations.str_terms.buckets.0.key_as_string + - match: { aggregations.str_terms.buckets.0.doc_count: 1 } + +--- +"IP test": + - do: + index: + index: test_1 + id: 1 + body: { "ip": "::1" } + + - do: + index: + index: test_1 + id: 2 + body: { "ip": "127.0.0.1" } + + - do: + index: + index: test_1 + id: 3 + body: { "ip": "::1" } + + - do: + indices.refresh: {} + + - do: + search: + body: { "size" : 0, "aggs" : { "ip_terms" : { "rare_terms" : { "field" : "ip" } } } } + + - match: { hits.total.value: 3 } + - length: { aggregations.ip_terms.buckets: 1 } + - match: { aggregations.ip_terms.buckets.0.key: "127.0.0.1" } + - is_false: aggregations.ip_terms.buckets.0.key_as_string + - match: { aggregations.ip_terms.buckets.0.doc_count: 1 } + + - do: + search: + body: { "size" : 0, "aggs" : { "ip_terms" : { "rare_terms" : { "field" : "ip", "include" : [ "127.0.0.1" ] } } } } + + - match: { hits.total.value: 3 } + - length: { aggregations.ip_terms.buckets: 1 } + - match: { aggregations.ip_terms.buckets.0.key: "127.0.0.1" } + - is_false: aggregations.ip_terms.buckets.0.key_as_string + - match: { aggregations.ip_terms.buckets.0.doc_count: 1 } + + - do: + search: + body: { "size" : 0, "aggs" : { "ip_terms" : { "rare_terms" : { "field" : "ip", "exclude" : [ "127.0.0.1" ] } } } } + + - match: { hits.total.value: 3 } + - length: { aggregations.ip_terms.buckets: 0 } + + - do: + catch: request + search: + index: test_1 + body: { "size" : 0, "aggs" : { "ip_terms" : { "rare_terms" : { "field" : "ip", "exclude" : "127.*" } } } } + + + +--- +"Boolean test": + - do: + index: + index: test_1 + id: 1 + body: { "boolean": true } + + - do: + index: + index: test_1 + id: 2 + body: { "boolean": false } + + - do: + index: + index: test_1 + id: 3 + body: { "boolean": true } + + - do: + indices.refresh: {} + + - do: + search: + body: { "size" : 0, "aggs" : { "boolean_terms" : { "rare_terms" : { "field" : "boolean" } } } } + + - match: { hits.total.value: 3 } + - length: { aggregations.boolean_terms.buckets: 1 } + - match: { aggregations.boolean_terms.buckets.0.key: 0 } + - match: { aggregations.boolean_terms.buckets.0.key_as_string: "false" } + - match: { aggregations.boolean_terms.buckets.0.doc_count: 1 } + +--- +"Integer test": + - do: + index: + index: test_1 + id: 1 + body: { "integer": 1234 } + + - do: + index: + index: test_1 + id: 2 + body: { "integer": 5678 } + + - do: + index: + index: test_1 + id: 3 + body: { "integer": 1234 } + + - do: + indices.refresh: {} + + - do: + search: + body: { "size" : 0, "aggs" : { "integer_terms" : { "rare_terms" : { "field" : "integer" } } } } + + - match: { hits.total.value: 3 } + + - length: { aggregations.integer_terms.buckets: 1 } + + - match: { aggregations.integer_terms.buckets.0.key: 5678 } + - is_false: aggregations.integer_terms.buckets.0.key_as_string + - match: { aggregations.integer_terms.buckets.0.doc_count: 1 } + +--- +"Date test": + - do: + index: + index: test_1 + id: 1 + body: { "date": "2016-05-03" } + + - do: + index: + index: test_1 + id: 2 + body: { "date": "2014-09-01" } + + - do: + index: + index: test_1 + id: 3 + body: { "date": "2016-05-03" } + + - do: + indices.refresh: {} + + - do: + search: + body: { "size" : 0, "aggs" : { "date_terms" : { "rare_terms" : { "field" : "date" } } } } + + - match: { hits.total.value: 3 } + + - length: { aggregations.date_terms.buckets: 1 } + - match: { aggregations.date_terms.buckets.0.key: 1409529600000 } + - match: { aggregations.date_terms.buckets.0.key_as_string: "2014-09-01T00:00:00.000Z" } + - match: { aggregations.date_terms.buckets.0.doc_count: 1 } + + - do: + search: + body: { "size" : 0, "aggs" : { "date_terms" : { "rare_terms" : { "field" : "date", "include" : [ "2014-09-01" ] } } } } + + - match: { hits.total.value: 3 } + - length: { aggregations.date_terms.buckets: 1 } + - match: { aggregations.date_terms.buckets.0.key_as_string: "2014-09-01T00:00:00.000Z" } + - match: { aggregations.date_terms.buckets.0.doc_count: 1 } + + - do: + search: + body: { "size" : 0, "aggs" : { "date_terms" : { "rare_terms" : { "field" : "date", "exclude" : [ "2014-09-01" ] } } } } + + - match: { hits.total.value: 3 } + - length: { aggregations.date_terms.buckets: 0 } + +--- +"Unmapped strings": + + - do: + index: + index: test_1 + id: 1 + body: {} + + - do: + indices.refresh: {} + + - do: + search: + body: { "size" : 0, "aggs" : { "string_terms" : { "rare_terms" : { "field" : "unmapped_string"} } } } + + - match: { hits.total.value: 1 } + - length: { aggregations.string_terms.buckets: 0 } + +--- +"Unmapped booleans": + + - do: + index: + index: test_1 + id: 1 + body: {} + + - do: + indices.refresh: {} + + - do: + search: + body: { "size" : 0, "aggs" : { "boolean_terms" : { "rare_terms" : { "field" : "unmapped_boolean" } } } } + + - match: { hits.total.value: 1 } + - length: { aggregations.boolean_terms.buckets: 0 } + +--- +"Unmapped dates": + + - do: + index: + index: test_1 + id: 1 + body: {} + + - do: + indices.refresh: {} + + - do: + search: + body: { "size" : 0, "aggs" : { "date_terms" : { "rare_terms" : { "field" : "unmapped_date"} } } } + + - match: { hits.total.value: 1 } + - length: { aggregations.date_terms.buckets: 0 } + +--- +"Unmapped longs": + + - do: + index: + index: test_1 + id: 1 + body: {} + + - do: + indices.refresh: {} + + - do: + search: + body: { "size" : 0, "aggs" : { "long_terms" : { "rare_terms" : { "field" : "unmapped_long", "value_type" : "long" } } } } + + - match: { hits.total.value: 1 } + - length: { aggregations.long_terms.buckets: 0 } + + diff --git a/rest-api-spec/src/main/resources/rest-api-spec/test/search/230_interval_query.yml b/rest-api-spec/src/main/resources/rest-api-spec/test/search/230_interval_query.yml index 46bf2cada8e4d..82aa0883008a8 100644 --- a/rest-api-spec/src/main/resources/rest-api-spec/test/search/230_interval_query.yml +++ b/rest-api-spec/src/main/resources/rest-api-spec/test/search/230_interval_query.yml @@ -384,3 +384,43 @@ setup: - match: { hits.total.value: 1 } - match: { hits.hits.0._id: "4" } +--- +"Test prefix": + - skip: + version: " - 8.0.0" + reason: "TODO: change to 7.3 in backport" + - do: + search: + index: test + body: + query: + intervals: + text: + all_of: + intervals: + - match: + query: cold + - prefix: + prefix: out + - match: { hits.total.value: 3 } + +--- +"Test wildcard": + - skip: + version: " - 8.0.0" + reason: "TODO: change to 7.3 in backport" + - do: + search: + index: test + body: + query: + intervals: + text: + all_of: + intervals: + - match: + query: cold + - wildcard: + pattern: out?ide + - match: { hits.total.value: 3 } + diff --git a/rest-api-spec/src/main/resources/rest-api-spec/test/search/310_match_bool_prefix.yml b/rest-api-spec/src/main/resources/rest-api-spec/test/search/310_match_bool_prefix.yml index aa6a5158b4795..f92b0ffda80e3 100644 --- a/rest-api-spec/src/main/resources/rest-api-spec/test/search/310_match_bool_prefix.yml +++ b/rest-api-spec/src/main/resources/rest-api-spec/test/search/310_match_bool_prefix.yml @@ -345,19 +345,3 @@ setup: type: bool_prefix fields: [ "my_field1", "my_field2" ] slop: 1 - ---- -"multi_match multiple fields with cutoff_frequency throws exception": - - - do: - catch: /\[cutoff_frequency\] not allowed for type \[bool_prefix\]/ - search: - rest_total_hits_as_int: true - index: test - body: - query: - multi_match: - query: "brown" - type: bool_prefix - fields: [ "my_field1", "my_field2" ] - cutoff_frequency: 0.001 diff --git a/rest-api-spec/src/main/resources/rest-api-spec/test/snapshot.get/10_basic.yml b/rest-api-spec/src/main/resources/rest-api-spec/test/snapshot.get/10_basic.yml index e2b7279f1cd68..219c5a70f331e 100644 --- a/rest-api-spec/src/main/resources/rest-api-spec/test/snapshot.get/10_basic.yml +++ b/rest-api-spec/src/main/resources/rest-api-spec/test/snapshot.get/10_basic.yml @@ -11,6 +11,9 @@ setup: --- "Get snapshot info": + - skip: + version: " - 7.9.99" + reason: "8.0 changes get snapshots response format" - do: indices.create: @@ -31,8 +34,8 @@ setup: repository: test_repo_get_1 snapshot: test_snapshot - - is_true: snapshots - - is_true: snapshots.0.failures + - is_true: responses.0.snapshots + - is_true: responses.0.snapshots.0.failures - do: snapshot.delete: @@ -41,15 +44,23 @@ setup: --- "Get missing snapshot info throws an exception": + - skip: + version: " - 7.9.99" + reason: "8.0 changes get snapshots response format" - do: - catch: /snapshot_missing_exception.+ is missing/ snapshot.get: repository: test_repo_get_1 snapshot: test_nonexistent_snapshot + - is_true: responses.0.error + - match: { responses.0.error.type: snapshot_missing_exception } + --- "Get missing snapshot info succeeds when ignore_unavailable is true": + - skip: + version: " - 7.9.99" + reason: "8.0 changes get snapshots response format" - do: snapshot.get: @@ -57,10 +68,14 @@ setup: snapshot: test_nonexistent_snapshot ignore_unavailable: true - - is_true: snapshots + - is_true: responses.0.snapshots --- "Get snapshot info when verbose is false": + - skip: + version: " - 7.9.99" + reason: "8.0 changes get snapshots response format" + - do: indices.create: index: test_index @@ -81,12 +96,13 @@ setup: snapshot: test_snapshot verbose: false - - is_true: snapshots - - match: { snapshots.0.snapshot: test_snapshot } - - match: { snapshots.0.state: SUCCESS } - - is_false: snapshots.0.failures - - is_false: snapshots.0.shards - - is_false: snapshots.0.version + - is_true: responses.0.snapshots + - match: { responses.0.snapshots.0.snapshot: test_snapshot } + - match: { responses.0.snapshots.0.state: SUCCESS } + - is_false: responses.0.snapshots.0.failures + - is_false: responses.0.snapshots.0.shards + - is_false: responses.0.snapshots.0.version + - is_false: responses.0.snapshots.0._meta - do: snapshot.delete: @@ -95,6 +111,9 @@ setup: --- "Get snapshot info contains include_global_state": + - skip: + version: " - 7.9.99" + reason: "8.0 changes get snapshots response format" - do: indices.create: @@ -117,10 +136,10 @@ setup: repository: test_repo_get_1 snapshot: test_snapshot_with_include_global_state - - is_true: snapshots - - match: { snapshots.0.snapshot: test_snapshot_with_include_global_state } - - match: { snapshots.0.state: SUCCESS } - - match: { snapshots.0.include_global_state: true } + - is_true: responses.0.snapshots + - match: { responses.0.snapshots.0.snapshot: test_snapshot_with_include_global_state } + - match: { responses.0.snapshots.0.state: SUCCESS } + - match: { responses.0.snapshots.0.include_global_state: true } - do: snapshot.delete: @@ -140,12 +159,50 @@ setup: repository: test_repo_get_1 snapshot: test_snapshot_without_include_global_state - - is_true: snapshots - - match: { snapshots.0.snapshot: test_snapshot_without_include_global_state } - - match: { snapshots.0.state: SUCCESS } - - match: { snapshots.0.include_global_state: false } + - is_true: responses.0.snapshots + - match: { responses.0.snapshots.0.snapshot: test_snapshot_without_include_global_state } + - match: { responses.0.snapshots.0.state: SUCCESS } + - match: { responses.0.snapshots.0.include_global_state: false } - do: snapshot.delete: repository: test_repo_get_1 snapshot: test_snapshot_without_include_global_state + +--- +"Get snapshot info with metadata": + - skip: + version: " - 7.9.99" + reason: "8.0 changes get snapshots response format" + + - do: + indices.create: + index: test_index + body: + settings: + number_of_shards: 1 + number_of_replicas: 0 + + - do: + snapshot.create: + repository: test_repo_get_1 + snapshot: test_snapshot_with_metadata + wait_for_completion: true + body: | + { "metadata": {"taken_by": "test", "foo": {"bar": "baz"}} } + + - do: + snapshot.get: + repository: test_repo_get_1 + snapshot: test_snapshot_with_metadata + + - is_true: responses.0.snapshots + - match: { responses.0.snapshots.0.snapshot: test_snapshot_with_metadata } + - match: { responses.0.snapshots.0.state: SUCCESS } + - match: { responses.0.snapshots.0.metadata.taken_by: test } + - match: { responses.0.snapshots.0.metadata.foo.bar: baz } + + - do: + snapshot.delete: + repository: test_repo_get_1 + snapshot: test_snapshot_with_metadata diff --git a/rest-api-spec/src/main/resources/rest-api-spec/test/termvectors/20_issue7121.yml b/rest-api-spec/src/main/resources/rest-api-spec/test/termvectors/20_issue7121.yml index eb7d335c28c60..e3ba834de074f 100644 --- a/rest-api-spec/src/main/resources/rest-api-spec/test/termvectors/20_issue7121.yml +++ b/rest-api-spec/src/main/resources/rest-api-spec/test/termvectors/20_issue7121.yml @@ -33,6 +33,5 @@ realtime: false - match: { _index: "testidx" } - - match: { _type: "_doc" } - match: { _id: "1" } - is_false: found diff --git a/server/build.gradle b/server/build.gradle index 391fdf46469f0..33fab57bc0626 100644 --- a/server/build.gradle +++ b/server/build.gradle @@ -74,13 +74,13 @@ if (!isEclipse && !isIdea) { dependencies { - compile "org.elasticsearch:elasticsearch-core:${version}" - compile "org.elasticsearch:elasticsearch-secure-sm:${version}" - compile "org.elasticsearch:elasticsearch-x-content:${version}" - compile "org.elasticsearch:elasticsearch-geo:${version}" + compile project(':libs:elasticsearch-core') + compile project(':libs:elasticsearch-secure-sm') + compile project(':libs:elasticsearch-x-content') + compile project(":libs:elasticsearch-geo") - compileOnly project(':libs:plugin-classloader') - testRuntime project(':libs:plugin-classloader') + compileOnly project(':libs:elasticsearch-plugin-classloader') + testRuntime project(':libs:elasticsearch-plugin-classloader') // lucene compile "org.apache.lucene:lucene-core:${versions.lucene}" @@ -100,8 +100,8 @@ dependencies { compile "org.apache.lucene:lucene-suggest:${versions.lucene}" // utilities - compile "org.elasticsearch:elasticsearch-cli:${version}" - compile 'com.carrotsearch:hppc:0.7.1' + compile project(":libs:elasticsearch-cli") + compile 'com.carrotsearch:hppc:0.8.1' // time handling, remove with java 8 time compile "joda-time:joda-time:${versions.joda}" @@ -118,8 +118,6 @@ dependencies { // logging compile "org.apache.logging.log4j:log4j-api:${versions.log4j}" compile "org.apache.logging.log4j:log4j-core:${versions.log4j}", optional - // to bridge dependencies that are still on Log4j 1 to Log4j 2 - compile "org.apache.logging.log4j:log4j-1.2-api:${versions.log4j}", optional // repackaged jna with native bits linked against all elastic supported platforms compile "org.elasticsearch:jna:${versions.jna}" @@ -129,9 +127,9 @@ dependencies { } if (isEclipse == false || project.path == ":server-tests") { - testCompile("org.elasticsearch.test:framework:${version}") { + testCompile(project(":test:framework")) { // tests use the locally compiled version of server - exclude group: 'org.elasticsearch', module: 'elasticsearch' + exclude group: 'org.elasticsearch', module: 'server' } } } diff --git a/server/licenses/hppc-0.7.1.jar.sha1 b/server/licenses/hppc-0.7.1.jar.sha1 deleted file mode 100644 index aa191a6c93b99..0000000000000 --- a/server/licenses/hppc-0.7.1.jar.sha1 +++ /dev/null @@ -1 +0,0 @@ -8b5057f74ea378c0150a1860874a3ebdcb713767 \ No newline at end of file diff --git a/server/licenses/hppc-0.8.1.jar.sha1 b/server/licenses/hppc-0.8.1.jar.sha1 new file mode 100644 index 0000000000000..47684ed023210 --- /dev/null +++ b/server/licenses/hppc-0.8.1.jar.sha1 @@ -0,0 +1 @@ +ffc7ba8f289428b9508ab484b8001dea944ae603 \ No newline at end of file diff --git a/server/src/main/java/org/apache/lucene/queries/BlendedTermQuery.java b/server/src/main/java/org/apache/lucene/queries/BlendedTermQuery.java index c696d476bbb43..5f00631ad6028 100644 --- a/server/src/main/java/org/apache/lucene/queries/BlendedTermQuery.java +++ b/server/src/main/java/org/apache/lucene/queries/BlendedTermQuery.java @@ -22,11 +22,8 @@ import org.apache.lucene.index.IndexReaderContext; import org.apache.lucene.index.LeafReaderContext; import org.apache.lucene.index.Term; -import org.apache.lucene.index.TermStates; import org.apache.lucene.index.TermState; -import org.apache.lucene.search.BooleanClause; -import org.apache.lucene.search.BooleanClause.Occur; -import org.apache.lucene.search.BooleanQuery; +import org.apache.lucene.index.TermStates; import org.apache.lucene.search.BoostQuery; import org.apache.lucene.search.DisjunctionMaxQuery; import org.apache.lucene.search.Query; @@ -278,45 +275,6 @@ public int hashCode() { return Objects.hash(classHash(), Arrays.hashCode(equalsTerms())); } - public static BlendedTermQuery commonTermsBlendedQuery(Term[] terms, final float[] boosts, final float maxTermFrequency) { - return new BlendedTermQuery(terms, boosts) { - @Override - protected Query topLevelQuery(Term[] terms, TermStates[] ctx, int[] docFreqs, int maxDoc) { - BooleanQuery.Builder highBuilder = new BooleanQuery.Builder(); - BooleanQuery.Builder lowBuilder = new BooleanQuery.Builder(); - for (int i = 0; i < terms.length; i++) { - Query query = new TermQuery(terms[i], ctx[i]); - if (boosts != null && boosts[i] != 1f) { - query = new BoostQuery(query, boosts[i]); - } - if ((maxTermFrequency >= 1f && docFreqs[i] > maxTermFrequency) - || (docFreqs[i] > (int) Math.ceil(maxTermFrequency - * maxDoc))) { - highBuilder.add(query, BooleanClause.Occur.SHOULD); - } else { - lowBuilder.add(query, BooleanClause.Occur.SHOULD); - } - } - BooleanQuery high = highBuilder.build(); - BooleanQuery low = lowBuilder.build(); - if (low.clauses().isEmpty()) { - BooleanQuery.Builder queryBuilder = new BooleanQuery.Builder(); - for (BooleanClause booleanClause : high) { - queryBuilder.add(booleanClause.getQuery(), Occur.MUST); - } - return queryBuilder.build(); - } else if (high.clauses().isEmpty()) { - return low; - } else { - return new BooleanQuery.Builder() - .add(high, BooleanClause.Occur.SHOULD) - .add(low, BooleanClause.Occur.MUST) - .build(); - } - } - }; - } - public static BlendedTermQuery dismaxBlendedQuery(Term[] terms, final float tieBreakerMultiplier) { return dismaxBlendedQuery(terms, null, tieBreakerMultiplier); } diff --git a/server/src/main/java/org/apache/lucene/queries/ExtendedCommonTermsQuery.java b/server/src/main/java/org/apache/lucene/queries/ExtendedCommonTermsQuery.java deleted file mode 100644 index 249b7fa83b50b..0000000000000 --- a/server/src/main/java/org/apache/lucene/queries/ExtendedCommonTermsQuery.java +++ /dev/null @@ -1,76 +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.apache.lucene.queries; - -import org.apache.lucene.search.BooleanClause.Occur; -import org.elasticsearch.common.lucene.search.Queries; - -/** - * Extended version of {@link CommonTermsQuery} that allows to pass in a - * {@code minimumNumberShouldMatch} specification that uses the actual num of high frequent terms - * to calculate the minimum matching terms. - */ -public class ExtendedCommonTermsQuery extends CommonTermsQuery { - - public ExtendedCommonTermsQuery(Occur highFreqOccur, Occur lowFreqOccur, float maxTermFrequency) { - super(highFreqOccur, lowFreqOccur, maxTermFrequency); - } - - private String lowFreqMinNumShouldMatchSpec; - private String highFreqMinNumShouldMatchSpec; - - @Override - protected int calcLowFreqMinimumNumberShouldMatch(int numOptional) { - return calcMinimumNumberShouldMatch(lowFreqMinNumShouldMatchSpec, numOptional); - } - - protected int calcMinimumNumberShouldMatch(String spec, int numOptional) { - if (spec == null) { - return 0; - } - return Queries.calculateMinShouldMatch(numOptional, spec); - } - - @Override - protected int calcHighFreqMinimumNumberShouldMatch(int numOptional) { - return calcMinimumNumberShouldMatch(highFreqMinNumShouldMatchSpec, numOptional); - } - - public void setHighFreqMinimumNumberShouldMatch(String spec) { - this.highFreqMinNumShouldMatchSpec = spec; - } - - public String getHighFreqMinimumNumberShouldMatchSpec() { - return highFreqMinNumShouldMatchSpec; - } - - public void setLowFreqMinimumNumberShouldMatch(String spec) { - this.lowFreqMinNumShouldMatchSpec = spec; - } - - public String getLowFreqMinimumNumberShouldMatchSpec() { - return lowFreqMinNumShouldMatchSpec; - } - - public float getMaxTermFrequency() { - return this.maxTermFrequency; - } - -} diff --git a/server/src/main/java/org/apache/lucene/util/CombinedBitSet.java b/server/src/main/java/org/apache/lucene/util/CombinedBitSet.java new file mode 100644 index 0000000000000..cb1bd819ab24a --- /dev/null +++ b/server/src/main/java/org/apache/lucene/util/CombinedBitSet.java @@ -0,0 +1,117 @@ +/* + * 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.apache.lucene.util; + +import org.apache.lucene.search.DocIdSetIterator; + +/** + * A {@link BitSet} implementation that combines two instances of {@link BitSet} and {@link Bits} + * to provide a single merged view. + */ +public final class CombinedBitSet extends BitSet implements Bits { + private final BitSet first; + private final Bits second; + private final int length; + + public CombinedBitSet(BitSet first, Bits second) { + this.first = first; + this.second = second; + this.length = first.length(); + } + + public BitSet getFirst() { + return first; + } + + /** + * This implementation is slow and requires to iterate over all bits to compute + * the intersection. Use {@link #approximateCardinality()} for + * a fast approximation. + */ + @Override + public int cardinality() { + int card = 0; + for (int i = 0; i < length; i++) { + card += get(i) ? 1 : 0; + } + return card; + } + + @Override + public int approximateCardinality() { + return first.cardinality(); + } + + @Override + public int prevSetBit(int index) { + assert index >= 0 && index < length : "index=" + index + ", numBits=" + length(); + int prev = first.prevSetBit(index); + while (prev != -1 && second.get(prev) == false) { + if (prev == 0) { + return -1; + } + prev = first.prevSetBit(prev-1); + } + return prev; + } + + @Override + public int nextSetBit(int index) { + assert index >= 0 && index < length : "index=" + index + " numBits=" + length(); + int next = first.nextSetBit(index); + while (next != DocIdSetIterator.NO_MORE_DOCS && second.get(next) == false) { + if (next == length() - 1) { + return DocIdSetIterator.NO_MORE_DOCS; + } + next = first.nextSetBit(next+1); + } + return next; + } + + @Override + public long ramBytesUsed() { + return first.ramBytesUsed(); + } + + @Override + public boolean get(int index) { + return first.get(index) && second.get(index); + } + + @Override + public int length() { + return length; + } + + @Override + public void set(int i) { + throw new UnsupportedOperationException("not implemented"); + } + + @Override + public void clear(int i) { + throw new UnsupportedOperationException("not implemented"); + } + + @Override + public void clear(int startIndex, int endIndex) { + throw new UnsupportedOperationException("not implemented"); + } +} diff --git a/server/src/main/java/org/elasticsearch/ElasticsearchException.java b/server/src/main/java/org/elasticsearch/ElasticsearchException.java index 260b443a6a557..51662414e0d07 100644 --- a/server/src/main/java/org/elasticsearch/ElasticsearchException.java +++ b/server/src/main/java/org/elasticsearch/ElasticsearchException.java @@ -1027,7 +1027,7 @@ private enum ElasticsearchExceptionHandle { org.elasticsearch.index.shard.ShardNotInPrimaryModeException.class, org.elasticsearch.index.shard.ShardNotInPrimaryModeException::new, 155, - Version.V_6_8_1); + UNKNOWN_VERSION_ADDED); final Class exceptionClass; final CheckedFunction constructor; diff --git a/server/src/main/java/org/elasticsearch/ExceptionsHelper.java b/server/src/main/java/org/elasticsearch/ExceptionsHelper.java index 48461ffe30d4b..56d2cdbc087eb 100644 --- a/server/src/main/java/org/elasticsearch/ExceptionsHelper.java +++ b/server/src/main/java/org/elasticsearch/ExceptionsHelper.java @@ -38,12 +38,14 @@ import java.util.Arrays; import java.util.Collections; import java.util.HashSet; +import java.util.IdentityHashMap; import java.util.LinkedList; import java.util.List; import java.util.Objects; import java.util.Optional; import java.util.Queue; import java.util.Set; +import java.util.function.Predicate; import java.util.stream.Collectors; public final class ExceptionsHelper { @@ -185,22 +187,14 @@ public static T useOrSuppress(T first, T second) { * @return Corruption indicating exception if one is found, otherwise {@code null} */ public static IOException unwrapCorruption(Throwable t) { - if (t != null) { - do { - for (Class clazz : CORRUPTION_EXCEPTIONS) { - if (clazz.isInstance(t)) { - return (IOException) t; - } - } - for (Throwable suppressed : t.getSuppressed()) { - IOException corruptionException = unwrapCorruption(suppressed); - if (corruptionException != null) { - return corruptionException; - } + return t == null ? null : ExceptionsHelper.unwrapCausesAndSuppressed(t, cause -> { + for (Class clazz : CORRUPTION_EXCEPTIONS) { + if (clazz.isInstance(cause)) { + return true; } - } while ((t = t.getCause()) != null); - } - return null; + } + return false; + }).orElse(null); } /** @@ -213,7 +207,11 @@ public static IOException unwrapCorruption(Throwable t) { */ public static Throwable unwrap(Throwable t, Class... clazzes) { if (t != null) { + final Set seen = Collections.newSetFromMap(new IdentityHashMap<>()); do { + if (seen.add(t) == false) { + return null; + } for (Class clazz : clazzes) { if (clazz.isInstance(t)) { return t; @@ -246,33 +244,22 @@ public static boolean reThrowIfNotNull(@Nullable Throwable e) { return true; } - static final int MAX_ITERATIONS = 1024; - - /** - * Unwrap the specified throwable looking for any suppressed errors or errors as a root cause of the specified throwable. - * - * @param cause the root throwable - * @return an optional error if one is found suppressed or a root cause in the tree rooted at the specified throwable - */ - public static Optional maybeError(final Throwable cause, final Logger logger) { - // early terminate if the cause is already an error - if (cause instanceof Error) { - return Optional.of((Error) cause); + @SuppressWarnings("unchecked") + private static Optional unwrapCausesAndSuppressed(Throwable cause, Predicate predicate) { + if (predicate.test(cause)) { + return Optional.of((T) cause); } final Queue queue = new LinkedList<>(); queue.add(cause); - int iterations = 0; + final Set seen = Collections.newSetFromMap(new IdentityHashMap<>()); while (queue.isEmpty() == false) { - iterations++; - // this is a guard against deeply nested or circular chains of exceptions - if (iterations > MAX_ITERATIONS) { - logger.warn("giving up looking for fatal errors", cause); - break; - } final Throwable current = queue.remove(); - if (current instanceof Error) { - return Optional.of((Error) current); + if (seen.add(current) == false) { + continue; + } + if (predicate.test(current)) { + return Optional.of((T) current); } Collections.addAll(queue, current.getSuppressed()); if (current.getCause() != null) { @@ -283,21 +270,24 @@ public static Optional maybeError(final Throwable cause, final Logger log } /** - * See {@link #maybeError(Throwable, Logger)}. Uses the class-local logger. + * Unwrap the specified throwable looking for any suppressed errors or errors as a root cause of the specified throwable. + * + * @param cause the root throwable + * @return an optional error if one is found suppressed or a root cause in the tree rooted at the specified throwable */ public static Optional maybeError(final Throwable cause) { - return maybeError(cause, logger); + return unwrapCausesAndSuppressed(cause, t -> t instanceof Error); } /** * If the specified cause is an unrecoverable error, this method will rethrow the cause on a separate thread so that it can not be * caught and bubbles up to the uncaught exception handler. Note that the cause tree is examined for any {@link Error}. See - * {@link #maybeError(Throwable, Logger)} for the semantics. + * {@link #maybeError(Throwable)} for the semantics. * * @param throwable the throwable to possibly throw on another thread */ public static void maybeDieOnAnotherThread(final Throwable throwable) { - ExceptionsHelper.maybeError(throwable, logger).ifPresent(error -> { + ExceptionsHelper.maybeError(throwable).ifPresent(error -> { /* * Here be dragons. We want to rethrow this so that it bubbles up to the uncaught exception handler. Yet, sometimes the stack * contains statements that catch any throwable (e.g., Netty, and the JDK futures framework). This means that a rethrow here diff --git a/server/src/main/java/org/elasticsearch/Version.java b/server/src/main/java/org/elasticsearch/Version.java index c685d39c7562f..152446cb700be 100644 --- a/server/src/main/java/org/elasticsearch/Version.java +++ b/server/src/main/java/org/elasticsearch/Version.java @@ -22,6 +22,7 @@ import org.elasticsearch.cluster.metadata.IndexMetaData; import org.elasticsearch.common.Strings; import org.elasticsearch.common.SuppressForbidden; +import org.elasticsearch.common.collect.ImmutableOpenIntMap; import org.elasticsearch.common.io.stream.StreamInput; import org.elasticsearch.common.io.stream.StreamOutput; import org.elasticsearch.common.settings.Settings; @@ -46,54 +47,53 @@ public class Version implements Comparable, ToXContentFragment { */ public static final int V_EMPTY_ID = 0; public static final Version V_EMPTY = new Version(V_EMPTY_ID, org.apache.lucene.util.Version.LATEST); - public static final int V_6_4_0_ID = 6040099; - public static final Version V_6_4_0 = new Version(V_6_4_0_ID, org.apache.lucene.util.Version.LUCENE_7_4_0); - public static final int V_6_4_1_ID = 6040199; - public static final Version V_6_4_1 = new Version(V_6_4_1_ID, org.apache.lucene.util.Version.LUCENE_7_4_0); - public static final int V_6_4_2_ID = 6040299; - public static final Version V_6_4_2 = new Version(V_6_4_2_ID, org.apache.lucene.util.Version.LUCENE_7_4_0); - public static final int V_6_4_3_ID = 6040399; - public static final Version V_6_4_3 = new Version(V_6_4_3_ID, org.apache.lucene.util.Version.LUCENE_7_4_0); - public static final int V_6_5_0_ID = 6050099; - public static final Version V_6_5_0 = new Version(V_6_5_0_ID, org.apache.lucene.util.Version.LUCENE_7_5_0); - public static final int V_6_5_1_ID = 6050199; - public static final Version V_6_5_1 = new Version(V_6_5_1_ID, org.apache.lucene.util.Version.LUCENE_7_5_0); - public static final int V_6_5_2_ID = 6050299; - public static final Version V_6_5_2 = new Version(V_6_5_2_ID, org.apache.lucene.util.Version.LUCENE_7_5_0); - public static final int V_6_5_3_ID = 6050399; - public static final Version V_6_5_3 = new Version(V_6_5_3_ID, org.apache.lucene.util.Version.LUCENE_7_5_0); - public static final int V_6_5_4_ID = 6050499; - public static final Version V_6_5_4 = new Version(V_6_5_4_ID, org.apache.lucene.util.Version.LUCENE_7_5_0); - public static final int V_6_6_0_ID = 6060099; - public static final Version V_6_6_0 = new Version(V_6_6_0_ID, org.apache.lucene.util.Version.LUCENE_7_6_0); - public static final int V_6_6_1_ID = 6060199; - public static final Version V_6_6_1 = new Version(V_6_6_1_ID, org.apache.lucene.util.Version.LUCENE_7_6_0); - public static final int V_6_6_2_ID = 6060299; - public static final Version V_6_6_2 = new Version(V_6_6_2_ID, org.apache.lucene.util.Version.LUCENE_7_6_0); - public static final int V_6_8_0_ID = 6080099; - public static final Version V_6_8_0 = new Version(V_6_8_0_ID, org.apache.lucene.util.Version.LUCENE_7_7_0); - public static final int V_6_8_1_ID = 6080199; - public static final Version V_6_8_1 = new Version(V_6_8_1_ID, org.apache.lucene.util.Version.LUCENE_7_7_0); - public static final int V_7_0_0_ID = 7000099; - public static final Version V_7_0_0 = new Version(V_7_0_0_ID, org.apache.lucene.util.Version.LUCENE_8_0_0); - public static final int V_7_0_1_ID = 7000199; - public static final Version V_7_0_1 = new Version(V_7_0_1_ID, org.apache.lucene.util.Version.LUCENE_8_0_0); - public static final int V_7_1_0_ID = 7010099; - public static final Version V_7_1_0 = new Version(V_7_1_0_ID, org.apache.lucene.util.Version.LUCENE_8_0_0); - public static final int V_7_1_1_ID = 7010199; - public static final Version V_7_1_1 = new Version(V_7_1_1_ID, org.apache.lucene.util.Version.LUCENE_8_0_0); - public static final int V_7_2_0_ID = 7020099; - public static final Version V_7_2_0 = new Version(V_7_2_0_ID, org.apache.lucene.util.Version.LUCENE_8_0_0); - public static final int V_7_3_0_ID = 7030099; - public static final Version V_7_3_0 = new Version(V_7_3_0_ID, org.apache.lucene.util.Version.LUCENE_8_1_0); - public static final int V_8_0_0_ID = 8000099; - public static final Version V_8_0_0 = new Version(V_8_0_0_ID, org.apache.lucene.util.Version.LUCENE_8_1_0); + public static final Version V_7_0_0 = new Version(7000099, org.apache.lucene.util.Version.LUCENE_8_0_0); + public static final Version V_7_0_1 = new Version(7000199, org.apache.lucene.util.Version.LUCENE_8_0_0); + public static final Version V_7_1_0 = new Version(7010099, org.apache.lucene.util.Version.LUCENE_8_0_0); + public static final Version V_7_1_1 = new Version(7010199, org.apache.lucene.util.Version.LUCENE_8_0_0); + public static final Version V_7_2_0 = new Version(7020099, org.apache.lucene.util.Version.LUCENE_8_0_0); + public static final Version V_7_2_1 = new Version(7020199, org.apache.lucene.util.Version.LUCENE_8_0_0); + public static final Version V_7_3_0 = new Version(7030099, org.apache.lucene.util.Version.LUCENE_8_1_0); + public static final Version V_7_4_0 = new Version(7040099, org.apache.lucene.util.Version.LUCENE_8_1_0); + public static final Version V_8_0_0 = new Version(8000099, org.apache.lucene.util.Version.LUCENE_8_1_0); public static final Version CURRENT = V_8_0_0; + private static final ImmutableOpenIntMap idToVersion; static { + final ImmutableOpenIntMap.Builder builder = ImmutableOpenIntMap.builder(); + + for (final Field declaredField : Version.class.getFields()) { + if (declaredField.getType().equals(Version.class)) { + final String fieldName = declaredField.getName(); + if (fieldName.equals("CURRENT") || fieldName.equals("V_EMPTY")) { + continue; + } + assert fieldName.matches("V_\\d+_\\d+_\\d+") + : "expected Version field [" + fieldName + "] to match V_\\d+_\\d+_\\d+"; + try { + final Version version = (Version) declaredField.get(null); + if (Assertions.ENABLED) { + final String[] fields = fieldName.split("_"); + final int major = Integer.valueOf(fields[1]) * 1000000; + final int minor = Integer.valueOf(fields[2]) * 10000; + final int revision = Integer.valueOf(fields[3]) * 100; + final int expectedId = major + minor + revision + 99; + assert version.id == expectedId : + "expected version [" + fieldName + "] to have id [" + expectedId + "] but was [" + version.id + "]"; + } + final Version maybePrevious = builder.put(version.id, version); + assert maybePrevious == null : + "expected [" + version.id + "] to be uniquely mapped but saw [" + maybePrevious + "] and [" + version + "]"; + } catch (final IllegalAccessException e) { + assert false : "Version field [" + fieldName + "] should be public"; + } + } + } assert CURRENT.luceneVersion.equals(org.apache.lucene.util.Version.LATEST) : "Version must be upgraded to [" + org.apache.lucene.util.Version.LATEST + "] is still set to [" + CURRENT.luceneVersion + "]"; + + idToVersion = builder.build(); } public static Version readVersion(StreamInput in) throws IOException { @@ -101,49 +101,10 @@ public static Version readVersion(StreamInput in) throws IOException { } public static Version fromId(int id) { + if (idToVersion.containsKey(id)) { + return idToVersion.get(id); + } switch (id) { - case V_8_0_0_ID: - return V_8_0_0; - case V_7_3_0_ID: - return V_7_3_0; - case V_7_2_0_ID: - return V_7_2_0; - case V_7_1_1_ID: - return V_7_1_1; - case V_7_1_0_ID: - return V_7_1_0; - case V_7_0_1_ID: - return V_7_0_1; - case V_7_0_0_ID: - return V_7_0_0; - case V_6_8_1_ID: - return V_6_8_1; - case V_6_8_0_ID: - return V_6_8_0; - case V_6_6_2_ID: - return V_6_6_2; - case V_6_6_1_ID: - return V_6_6_1; - case V_6_6_0_ID: - return V_6_6_0; - case V_6_5_4_ID: - return V_6_5_4; - case V_6_5_3_ID: - return V_6_5_3; - case V_6_5_2_ID: - return V_6_5_2; - case V_6_5_1_ID: - return V_6_5_1; - case V_6_5_0_ID: - return V_6_5_0; - case V_6_4_3_ID: - return V_6_4_3; - case V_6_4_2_ID: - return V_6_4_2; - case V_6_4_1_ID: - return V_6_4_1; - case V_6_4_0_ID: - return V_6_4_0; case V_EMPTY_ID: return V_EMPTY; default: @@ -221,7 +182,7 @@ public static Version fromString(String version) { version = version.substring(0, version.length() - 9); } String[] parts = version.split("[.-]"); - if (parts.length < 3 || parts.length > 4) { + if (parts.length != 3) { throw new IllegalArgumentException( "the version needs to contain major, minor, and revision, and optionally the build: " + version); } @@ -234,31 +195,13 @@ public static Version fromString(String version) { if (rawMajor >=7 && parts.length == 4) { // we don't support qualifier as part of the version anymore throw new IllegalArgumentException("illegal version format - qualifiers are only supported until version 6.x"); } - final int betaOffset = rawMajor < 5 ? 0 : 25; //we reverse the version id calculation based on some assumption as we can't reliably reverse the modulo final int major = rawMajor * 1000000; final int minor = Integer.parseInt(parts[1]) * 10000; final int revision = Integer.parseInt(parts[2]) * 100; - - int build = 99; - if (parts.length == 4) { - String buildStr = parts[3]; - if (buildStr.startsWith("alpha")) { - assert rawMajor >= 5 : "major must be >= 5 but was " + major; - build = Integer.parseInt(buildStr.substring(5)); - assert build < 25 : "expected a alpha build but " + build + " >= 25"; - } else if (buildStr.startsWith("Beta") || buildStr.startsWith("beta")) { - build = betaOffset + Integer.parseInt(buildStr.substring(4)); - assert build < 50 : "expected a beta build but " + build + " >= 50"; - } else if (buildStr.startsWith("RC") || buildStr.startsWith("rc")) { - build = Integer.parseInt(buildStr.substring(2)) + 50; - } else { - throw new IllegalArgumentException("unable to parse version " + version); - } - } - - return fromId(major + minor + revision + build); + // TODO: 99 is leftover from alpha/beta/rc, it should be removed + return fromId(major + minor + revision + 99); } catch (NumberFormatException e) { throw new IllegalArgumentException("unable to parse version " + version, e); @@ -327,13 +270,16 @@ public Version minimumCompatibilityVersion() { if (major == 6) { // force the minimum compatibility for version 6 to 5.6 since we don't reference version 5 anymore return Version.fromId(5060099); - } else if (major >= 7) { - // all major versions from 7 onwards are compatible with last minor series of the previous major + } else if (major == 7) { + // force the minimum compatibility for version 7 to 6.8 since we don't reference version 6 anymore + return Version.fromId(6080099); + } else if (major >= 8) { + // all major versions from 8 onwards are compatible with last minor series of the previous major Version bwcVersion = null; for (int i = DeclaredVersionsHolder.DECLARED_VERSIONS.size() - 1; i >= 0; i--) { final Version candidateVersion = DeclaredVersionsHolder.DECLARED_VERSIONS.get(i); - if (candidateVersion.major == major - 1 && candidateVersion.isRelease() && after(candidateVersion)) { + if (candidateVersion.major == major - 1 && after(candidateVersion)) { if (bwcVersion != null && candidateVersion.minor < bwcVersion.minor) { break; } @@ -389,27 +335,7 @@ public static void main(String[] args) { @Override public String toString() { - StringBuilder sb = new StringBuilder(); - sb.append(major).append('.').append(minor).append('.').append(revision); - if (isAlpha()) { - sb.append("-alpha"); - sb.append(build); - } else if (isBeta()) { - if (major >= 2) { - sb.append("-beta"); - } else { - sb.append(".Beta"); - } - sb.append(major < 5 ? build : build-25); - } else if (build < 99) { - if (major >= 2) { - sb.append("-rc"); - } else { - sb.append(".RC"); - } - sb.append(build - 50); - } - return sb.toString(); + return major + "." + minor + "." + revision; } @Override @@ -435,27 +361,6 @@ public int hashCode() { return id; } - public boolean isBeta() { - return major < 5 ? build < 50 : build >= 25 && build < 50; - } - - /** - * Returns true iff this version is an alpha version - * Note: This has been introduced in elasticsearch version 5. Previous versions will never - * have an alpha version. - */ - public boolean isAlpha() { - return major < 5 ? false : build < 25; - } - - public boolean isRC() { - return build > 50 && build < 99; - } - - public boolean isRelease() { - return build == 99; - } - /** * Extracts a sorted list of declared version constants from a class. * The argument would normally be Version.class but is exposed for @@ -477,7 +382,7 @@ public static List getDeclaredVersions(final Class versionClass) { case "V_EMPTY": continue; } - assert field.getName().matches("V(_\\d+)+(_(alpha|beta|rc)\\d+)?") : field.getName(); + assert field.getName().matches("V(_\\d+){3}?") : field.getName(); try { versions.add(((Version) field.get(null))); } catch (final IllegalAccessException e) { diff --git a/server/src/main/java/org/elasticsearch/action/ActionModule.java b/server/src/main/java/org/elasticsearch/action/ActionModule.java index 2cfe66372115f..9ccd4c9fdfdb0 100644 --- a/server/src/main/java/org/elasticsearch/action/ActionModule.java +++ b/server/src/main/java/org/elasticsearch/action/ActionModule.java @@ -85,9 +85,8 @@ import org.elasticsearch.action.admin.cluster.tasks.PendingClusterTasksAction; import org.elasticsearch.action.admin.cluster.tasks.TransportPendingClusterTasksAction; import org.elasticsearch.action.admin.indices.alias.IndicesAliasesAction; +import org.elasticsearch.action.admin.indices.alias.IndicesAliasesRequest; import org.elasticsearch.action.admin.indices.alias.TransportIndicesAliasesAction; -import org.elasticsearch.action.admin.indices.alias.exists.AliasesExistAction; -import org.elasticsearch.action.admin.indices.alias.exists.TransportAliasesExistAction; import org.elasticsearch.action.admin.indices.alias.get.GetAliasesAction; import org.elasticsearch.action.admin.indices.alias.get.TransportGetAliasesAction; import org.elasticsearch.action.admin.indices.analyze.AnalyzeAction; @@ -100,10 +99,6 @@ import org.elasticsearch.action.admin.indices.create.TransportCreateIndexAction; import org.elasticsearch.action.admin.indices.delete.DeleteIndexAction; import org.elasticsearch.action.admin.indices.delete.TransportDeleteIndexAction; -import org.elasticsearch.action.admin.indices.exists.indices.IndicesExistsAction; -import org.elasticsearch.action.admin.indices.exists.indices.TransportIndicesExistsAction; -import org.elasticsearch.action.admin.indices.exists.types.TransportTypesExistsAction; -import org.elasticsearch.action.admin.indices.exists.types.TypesExistsAction; import org.elasticsearch.action.admin.indices.flush.FlushAction; import org.elasticsearch.action.admin.indices.flush.SyncedFlushAction; import org.elasticsearch.action.admin.indices.flush.TransportFlushAction; @@ -118,6 +113,7 @@ import org.elasticsearch.action.admin.indices.mapping.get.TransportGetFieldMappingsIndexAction; import org.elasticsearch.action.admin.indices.mapping.get.TransportGetMappingsAction; import org.elasticsearch.action.admin.indices.mapping.put.PutMappingAction; +import org.elasticsearch.action.admin.indices.mapping.put.PutMappingRequest; import org.elasticsearch.action.admin.indices.mapping.put.TransportPutMappingAction; import org.elasticsearch.action.admin.indices.open.OpenIndexAction; import org.elasticsearch.action.admin.indices.open.TransportOpenIndexAction; @@ -204,12 +200,16 @@ import org.elasticsearch.cluster.node.DiscoveryNodes; import org.elasticsearch.common.NamedRegistry; import org.elasticsearch.common.inject.AbstractModule; +import org.elasticsearch.common.inject.TypeLiteral; import org.elasticsearch.common.inject.multibindings.MapBinder; import org.elasticsearch.common.settings.ClusterSettings; import org.elasticsearch.common.settings.IndexScopedSettings; import org.elasticsearch.common.settings.Settings; import org.elasticsearch.common.settings.SettingsFilter; +import org.elasticsearch.index.seqno.GlobalCheckpointSyncAction; import org.elasticsearch.index.seqno.RetentionLeaseActions; +import org.elasticsearch.index.seqno.RetentionLeaseBackgroundSyncAction; +import org.elasticsearch.index.seqno.RetentionLeaseSyncAction; import org.elasticsearch.indices.breaker.CircuitBreakerService; import org.elasticsearch.persistent.CompletionPersistentTaskAction; import org.elasticsearch.persistent.RemovePersistentTaskAction; @@ -347,7 +347,6 @@ public class ActionModule extends AbstractModule { private static final Logger logger = LogManager.getLogger(ActionModule.class); - private final boolean transportClient; private final Settings settings; private final IndexNameExpressionResolver indexNameExpressionResolver; private final IndexScopedSettings indexScopedSettings; @@ -359,13 +358,13 @@ public class ActionModule extends AbstractModule { private final AutoCreateIndex autoCreateIndex; private final DestructiveOperations destructiveOperations; private final RestController restController; - private final TransportPutMappingAction.RequestValidators mappingRequestValidators; + private final RequestValidators mappingRequestValidators; + private final RequestValidators indicesAliasesRequestRequestValidators; - public ActionModule(boolean transportClient, Settings settings, IndexNameExpressionResolver indexNameExpressionResolver, + public ActionModule(Settings settings, IndexNameExpressionResolver indexNameExpressionResolver, IndexScopedSettings indexScopedSettings, ClusterSettings clusterSettings, SettingsFilter settingsFilter, ThreadPool threadPool, List actionPlugins, NodeClient nodeClient, - CircuitBreakerService circuitBreakerService, UsageService usageService) { - this.transportClient = transportClient; + CircuitBreakerService circuitBreakerService, UsageService usageService) { this.settings = settings; this.indexNameExpressionResolver = indexNameExpressionResolver; this.indexScopedSettings = indexScopedSettings; @@ -374,7 +373,7 @@ public ActionModule(boolean transportClient, Settings settings, IndexNameExpress this.actionPlugins = actionPlugins; actions = setupActions(actionPlugins); actionFilters = setupActionFilters(actionPlugins); - autoCreateIndex = transportClient ? null : new AutoCreateIndex(settings, clusterSettings, indexNameExpressionResolver); + autoCreateIndex = new AutoCreateIndex(settings, clusterSettings, indexNameExpressionResolver); destructiveOperations = new DestructiveOperations(settings, clusterSettings); Set headers = Stream.concat( actionPlugins.stream().flatMap(p -> p.getRestHeaders().stream()), @@ -391,15 +390,12 @@ public ActionModule(boolean transportClient, Settings settings, IndexNameExpress restWrapper = newRestWrapper; } } - mappingRequestValidators = new TransportPutMappingAction.RequestValidators( - actionPlugins.stream().flatMap(p -> p.mappingRequestValidators().stream()).collect(Collectors.toList()) - ); - - if (transportClient) { - restController = null; - } else { - restController = new RestController(headers, restWrapper, nodeClient, circuitBreakerService, usageService); - } + mappingRequestValidators = new RequestValidators<>( + actionPlugins.stream().flatMap(p -> p.mappingRequestValidators().stream()).collect(Collectors.toList())); + indicesAliasesRequestRequestValidators = new RequestValidators<>( + actionPlugins.stream().flatMap(p -> p.indicesAliasesRequestValidators().stream()).collect(Collectors.toList())); + + restController = new RestController(headers, restWrapper, nodeClient, circuitBreakerService, usageService); } @@ -419,9 +415,8 @@ public void register(ActionHandler handler) { } public void register( - Action action, Class> transportAction, - Class... supportTransportActions) { - register(new ActionHandler<>(action, transportAction, supportTransportActions)); + ActionType action, Class> transportAction) { + register(new ActionHandler<>(action, transportAction)); } } ActionRegistry actions = new ActionRegistry(); @@ -466,11 +461,9 @@ public void reg actions.register(GetIndexAction.INSTANCE, TransportGetIndexAction.class); actions.register(OpenIndexAction.INSTANCE, TransportOpenIndexAction.class); actions.register(CloseIndexAction.INSTANCE, TransportCloseIndexAction.class); - actions.register(IndicesExistsAction.INSTANCE, TransportIndicesExistsAction.class); - actions.register(TypesExistsAction.INSTANCE, TransportTypesExistsAction.class); actions.register(GetMappingsAction.INSTANCE, TransportGetMappingsAction.class); - actions.register(GetFieldMappingsAction.INSTANCE, TransportGetFieldMappingsAction.class, - TransportGetFieldMappingsIndexAction.class); + actions.register(GetFieldMappingsAction.INSTANCE, TransportGetFieldMappingsAction.class); + actions.register(TransportGetFieldMappingsIndexAction.TYPE, TransportGetFieldMappingsIndexAction.class); actions.register(PutMappingAction.INSTANCE, TransportPutMappingAction.class); actions.register(IndicesAliasesAction.INSTANCE, TransportIndicesAliasesAction.class); actions.register(UpdateSettingsAction.INSTANCE, TransportUpdateSettingsAction.class); @@ -488,20 +481,19 @@ public void reg actions.register(UpgradeSettingsAction.INSTANCE, TransportUpgradeSettingsAction.class); actions.register(ClearIndicesCacheAction.INSTANCE, TransportClearIndicesCacheAction.class); actions.register(GetAliasesAction.INSTANCE, TransportGetAliasesAction.class); - actions.register(AliasesExistAction.INSTANCE, TransportAliasesExistAction.class); actions.register(GetSettingsAction.INSTANCE, TransportGetSettingsAction.class); actions.register(IndexAction.INSTANCE, TransportIndexAction.class); actions.register(GetAction.INSTANCE, TransportGetAction.class); actions.register(TermVectorsAction.INSTANCE, TransportTermVectorsAction.class); - actions.register(MultiTermVectorsAction.INSTANCE, TransportMultiTermVectorsAction.class, - TransportShardMultiTermsVectorAction.class); + actions.register(MultiTermVectorsAction.INSTANCE, TransportMultiTermVectorsAction.class); + actions.register(TransportShardMultiTermsVectorAction.TYPE, TransportShardMultiTermsVectorAction.class); actions.register(DeleteAction.INSTANCE, TransportDeleteAction.class); actions.register(UpdateAction.INSTANCE, TransportUpdateAction.class); - actions.register(MultiGetAction.INSTANCE, TransportMultiGetAction.class, - TransportShardMultiGetAction.class); - actions.register(BulkAction.INSTANCE, TransportBulkAction.class, - TransportShardBulkAction.class); + actions.register(MultiGetAction.INSTANCE, TransportMultiGetAction.class); + actions.register(TransportShardMultiGetAction.TYPE, TransportShardMultiGetAction.class); + actions.register(BulkAction.INSTANCE, TransportBulkAction.class); + actions.register(TransportShardBulkAction.TYPE, TransportShardBulkAction.class); actions.register(SearchAction.INSTANCE, TransportSearchAction.class); actions.register(SearchScrollAction.INSTANCE, TransportSearchScrollAction.class); actions.register(MultiSearchAction.INSTANCE, TransportMultiSearchAction.class); @@ -515,8 +507,8 @@ public void reg actions.register(GetStoredScriptAction.INSTANCE, TransportGetStoredScriptAction.class); actions.register(DeleteStoredScriptAction.INSTANCE, TransportDeleteStoredScriptAction.class); - actions.register(FieldCapabilitiesAction.INSTANCE, TransportFieldCapabilitiesAction.class, - TransportFieldCapabilitiesIndexAction.class); + actions.register(FieldCapabilitiesAction.INSTANCE, TransportFieldCapabilitiesAction.class); + actions.register(TransportFieldCapabilitiesIndexAction.TYPE, TransportFieldCapabilitiesIndexAction.class); actions.register(PutPipelineAction.INSTANCE, PutPipelineTransportAction.class); actions.register(GetPipelineAction.INSTANCE, GetPipelineTransportAction.class); @@ -536,6 +528,11 @@ public void reg actions.register(RetentionLeaseActions.Renew.INSTANCE, RetentionLeaseActions.Renew.TransportAction.class); actions.register(RetentionLeaseActions.Remove.INSTANCE, RetentionLeaseActions.Remove.TransportAction.class); + // internal actions + actions.register(GlobalCheckpointSyncAction.TYPE, GlobalCheckpointSyncAction.class); + actions.register(RetentionLeaseBackgroundSyncAction.TYPE, RetentionLeaseBackgroundSyncAction.class); + actions.register(RetentionLeaseSyncAction.TYPE, RetentionLeaseSyncAction.class); + return unmodifiableMap(actions.getRegistry()); } @@ -661,7 +658,7 @@ public void initRestHandlers(Supplier nodesInCluster) { registerHandler.accept(new RestMasterAction(settings, restController)); registerHandler.accept(new RestNodesAction(settings, restController)); registerHandler.accept(new RestTasksAction(settings, restController, nodesInCluster)); - registerHandler.accept(new RestIndicesAction(settings, restController, indexNameExpressionResolver)); + registerHandler.accept(new RestIndicesAction(settings, restController)); registerHandler.accept(new RestSegmentsAction(settings, restController)); // Fully qualified to prevent interference with rest.action.count.RestCountAction registerHandler.accept(new org.elasticsearch.rest.action.cat.RestCountAction(settings, restController)); @@ -690,26 +687,21 @@ public void initRestHandlers(Supplier nodesInCluster) { protected void configure() { bind(ActionFilters.class).toInstance(actionFilters); bind(DestructiveOperations.class).toInstance(destructiveOperations); - bind(TransportPutMappingAction.RequestValidators.class).toInstance(mappingRequestValidators); - - if (false == transportClient) { - // Supporting classes only used when not a transport client - bind(AutoCreateIndex.class).toInstance(autoCreateIndex); - bind(TransportLivenessAction.class).asEagerSingleton(); - - // register Action -> transportAction Map used by NodeClient - @SuppressWarnings("rawtypes") - MapBinder transportActionsBinder - = MapBinder.newMapBinder(binder(), Action.class, TransportAction.class); - for (ActionHandler action : actions.values()) { - // bind the action as eager singleton, so the map binder one will reuse it - bind(action.getTransportAction()).asEagerSingleton(); - transportActionsBinder.addBinding(action.getAction()).to(action.getTransportAction()).asEagerSingleton(); - for (Class supportAction : action.getSupportTransportActions()) { - bind(supportAction).asEagerSingleton(); - } - } + bind(new TypeLiteral>() {}).toInstance(mappingRequestValidators); + bind(new TypeLiteral>() {}).toInstance(indicesAliasesRequestRequestValidators); + bind(AutoCreateIndex.class).toInstance(autoCreateIndex); + bind(TransportLivenessAction.class).asEagerSingleton(); + + // register ActionType -> transportAction Map used by NodeClient + @SuppressWarnings("rawtypes") + MapBinder transportActionsBinder + = MapBinder.newMapBinder(binder(), ActionType.class, TransportAction.class); + for (ActionHandler action : actions.values()) { + // bind the action as eager singleton, so the map binder one will reuse it + bind(action.getTransportAction()).asEagerSingleton(); + transportActionsBinder.addBinding(action.getAction()).to(action.getTransportAction()).asEagerSingleton(); } + } public ActionFilters getActionFilters() { diff --git a/server/src/main/java/org/elasticsearch/action/ActionRequestBuilder.java b/server/src/main/java/org/elasticsearch/action/ActionRequestBuilder.java index a3aa8ac2a5222..166bec9e065b0 100644 --- a/server/src/main/java/org/elasticsearch/action/ActionRequestBuilder.java +++ b/server/src/main/java/org/elasticsearch/action/ActionRequestBuilder.java @@ -26,11 +26,11 @@ public abstract class ActionRequestBuilder { - protected final Action action; + protected final ActionType action; protected final Request request; protected final ElasticsearchClient client; - protected ActionRequestBuilder(ElasticsearchClient client, Action action, Request request) { + protected ActionRequestBuilder(ElasticsearchClient client, ActionType action, Request request) { Objects.requireNonNull(action, "action must not be null"); this.action = action; this.request = request; diff --git a/server/src/main/java/org/elasticsearch/action/Action.java b/server/src/main/java/org/elasticsearch/action/ActionType.java similarity index 75% rename from server/src/main/java/org/elasticsearch/action/Action.java rename to server/src/main/java/org/elasticsearch/action/ActionType.java index f0df6202072a4..02f8f3c6fc22f 100644 --- a/server/src/main/java/org/elasticsearch/action/Action.java +++ b/server/src/main/java/org/elasticsearch/action/ActionType.java @@ -26,15 +26,27 @@ /** * A generic action. Should strive to make it a singleton. */ -public abstract class Action { +public class ActionType { private final String name; + private final Writeable.Reader responseReader; /** * @param name The name of the action, must be unique across actions. + * @deprecated Pass a {@link Writeable.Reader} with {@link } */ - protected Action(String name) { + @Deprecated + protected ActionType(String name) { + this(name, null); + } + + /** + * @param name The name of the action, must be unique across actions. + * @param responseReader A reader for the response type + */ + public ActionType(String name, Writeable.Reader responseReader) { this.name = name; + this.responseReader = responseReader; } /** @@ -44,23 +56,11 @@ public String name() { return this.name; } - /** - * Creates a new response instance. - * @deprecated Implement {@link #getResponseReader()} instead and make this method throw an - * {@link UnsupportedOperationException} - */ - @Deprecated - public abstract Response newResponse(); - /** * Get a reader that can create a new instance of the class from a {@link org.elasticsearch.common.io.stream.StreamInput} */ public Writeable.Reader getResponseReader() { - return in -> { - Response response = newResponse(); - response.readFrom(in); - return response; - }; + return responseReader; } /** @@ -72,7 +72,7 @@ public TransportRequestOptions transportOptions(Settings settings) { @Override public boolean equals(Object o) { - return o instanceof Action && name.equals(((Action) o).name()); + return o instanceof ActionType && name.equals(((ActionType) o).name()); } @Override diff --git a/server/src/main/java/org/elasticsearch/action/RequestValidators.java b/server/src/main/java/org/elasticsearch/action/RequestValidators.java new file mode 100644 index 0000000000000..95bfcca51d218 --- /dev/null +++ b/server/src/main/java/org/elasticsearch/action/RequestValidators.java @@ -0,0 +1,67 @@ +/* + * 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; + +import org.elasticsearch.cluster.ClusterState; +import org.elasticsearch.index.Index; + +import java.util.Collection; +import java.util.Optional; + +public class RequestValidators { + + private final Collection> validators; + + public RequestValidators(Collection> validators) { + this.validators = validators; + } + + public Optional validateRequest(final T request, final ClusterState state, final Index[] indices) { + Exception exception = null; + for (final var validator : validators) { + final Optional maybeException = validator.validateRequest(request, state, indices); + if (maybeException.isEmpty()) continue; + if (exception == null) { + exception = maybeException.get(); + } else { + exception.addSuppressed(maybeException.get()); + } + } + return Optional.ofNullable(exception); + } + + /** + * A validator that validates an request associated with indices before executing it. + */ + public interface RequestValidator { + + /** + * Validates a given request with its associated concrete indices and the current state. + * + * @param request the request to validate + * @param state the current cluster state + * @param indices the concrete indices that associated with the given request + * @return an optional exception indicates a reason that the given request should be aborted, otherwise empty + */ + Optional validateRequest(T request, ClusterState state, Index[] indices); + + } + +} diff --git a/server/src/main/java/org/elasticsearch/action/RoutingMissingException.java b/server/src/main/java/org/elasticsearch/action/RoutingMissingException.java index 1ecd9593e1c36..e3ef791814ac8 100644 --- a/server/src/main/java/org/elasticsearch/action/RoutingMissingException.java +++ b/server/src/main/java/org/elasticsearch/action/RoutingMissingException.java @@ -22,6 +22,7 @@ import org.elasticsearch.ElasticsearchException; import org.elasticsearch.common.io.stream.StreamInput; import org.elasticsearch.common.io.stream.StreamOutput; +import org.elasticsearch.index.mapper.MapperService; import org.elasticsearch.rest.RestStatus; import java.io.IOException; @@ -33,6 +34,10 @@ public class RoutingMissingException extends ElasticsearchException { private final String id; + public RoutingMissingException(String index, String id) { + this(index, MapperService.SINGLE_MAPPING_NAME, id); + } + public RoutingMissingException(String index, String type, String id) { super("routing is required for [" + index + "]/[" + type + "]/[" + id + "]"); Objects.requireNonNull(index, "index must not be null"); diff --git a/server/src/main/java/org/elasticsearch/action/StreamableResponseActionType.java b/server/src/main/java/org/elasticsearch/action/StreamableResponseActionType.java new file mode 100644 index 0000000000000..b8206bb03f89d --- /dev/null +++ b/server/src/main/java/org/elasticsearch/action/StreamableResponseActionType.java @@ -0,0 +1,51 @@ +/* + * 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; + +import org.elasticsearch.common.io.stream.Writeable; + +/** + * An action for with the response type implements {@link org.elasticsearch.common.io.stream.Streamable}. + * @deprecated Use {@link ActionType} directly and provide a {@link Writeable.Reader} + */ +@Deprecated +public abstract class StreamableResponseActionType extends ActionType { + + protected StreamableResponseActionType(String name) { + super(name); + } + + /** + * Creates a new response instance. + * @deprecated Implement {@link #getResponseReader()} instead and make this method throw an + * {@link UnsupportedOperationException} + */ + @Deprecated + public abstract Response newResponse(); + + @Override + public final Writeable.Reader getResponseReader() { + return in -> { + Response response = newResponse(); + response.readFrom(in); + return response; + }; + } +} diff --git a/server/src/main/java/org/elasticsearch/action/TransportActionNodeProxy.java b/server/src/main/java/org/elasticsearch/action/TransportActionNodeProxy.java index a4c3e17e80208..4319a745ba816 100644 --- a/server/src/main/java/org/elasticsearch/action/TransportActionNodeProxy.java +++ b/server/src/main/java/org/elasticsearch/action/TransportActionNodeProxy.java @@ -30,10 +30,10 @@ public class TransportActionNodeProxy { private final TransportService transportService; - private final Action action; + private final ActionType action; private final TransportRequestOptions transportOptions; - public TransportActionNodeProxy(Settings settings, Action action, TransportService transportService) { + public TransportActionNodeProxy(Settings settings, ActionType action, TransportService transportService) { this.action = action; this.transportService = transportService; this.transportOptions = action.transportOptions(settings); diff --git a/server/src/main/java/org/elasticsearch/action/admin/cluster/allocation/ClusterAllocationExplainAction.java b/server/src/main/java/org/elasticsearch/action/admin/cluster/allocation/ClusterAllocationExplainAction.java index 19d5378b305ed..acaaed9eaa985 100644 --- a/server/src/main/java/org/elasticsearch/action/admin/cluster/allocation/ClusterAllocationExplainAction.java +++ b/server/src/main/java/org/elasticsearch/action/admin/cluster/allocation/ClusterAllocationExplainAction.java @@ -19,12 +19,12 @@ package org.elasticsearch.action.admin.cluster.allocation; -import org.elasticsearch.action.Action; +import org.elasticsearch.action.StreamableResponseActionType; /** - * Action for explaining shard allocation for a shard in the cluster + * ActionType for explaining shard allocation for a shard in the cluster */ -public class ClusterAllocationExplainAction extends Action { +public class ClusterAllocationExplainAction extends StreamableResponseActionType { public static final ClusterAllocationExplainAction INSTANCE = new ClusterAllocationExplainAction(); public static final String NAME = "cluster:monitor/allocation/explain"; diff --git a/server/src/main/java/org/elasticsearch/action/admin/cluster/allocation/TransportClusterAllocationExplainAction.java b/server/src/main/java/org/elasticsearch/action/admin/cluster/allocation/TransportClusterAllocationExplainAction.java index e3ee0dd7b1524..2fe4982acc63e 100644 --- a/server/src/main/java/org/elasticsearch/action/admin/cluster/allocation/TransportClusterAllocationExplainAction.java +++ b/server/src/main/java/org/elasticsearch/action/admin/cluster/allocation/TransportClusterAllocationExplainAction.java @@ -41,6 +41,7 @@ import org.elasticsearch.cluster.service.ClusterService; import org.elasticsearch.common.inject.Inject; import org.elasticsearch.gateway.GatewayAllocator; +import org.elasticsearch.tasks.Task; import org.elasticsearch.threadpool.ThreadPool; import org.elasticsearch.transport.TransportService; @@ -88,7 +89,7 @@ protected ClusterAllocationExplainResponse newResponse() { } @Override - protected void masterOperation(final ClusterAllocationExplainRequest request, final ClusterState state, + protected void masterOperation(Task task, final ClusterAllocationExplainRequest request, final ClusterState state, final ActionListener listener) { final RoutingNodes routingNodes = state.getRoutingNodes(); final ClusterInfo clusterInfo = clusterInfoService.getClusterInfo(); diff --git a/server/src/main/java/org/elasticsearch/action/admin/cluster/configuration/AddVotingConfigExclusionsAction.java b/server/src/main/java/org/elasticsearch/action/admin/cluster/configuration/AddVotingConfigExclusionsAction.java index 6fdd324fa47c6..a2f0c721b5da4 100644 --- a/server/src/main/java/org/elasticsearch/action/admin/cluster/configuration/AddVotingConfigExclusionsAction.java +++ b/server/src/main/java/org/elasticsearch/action/admin/cluster/configuration/AddVotingConfigExclusionsAction.java @@ -18,10 +18,10 @@ */ package org.elasticsearch.action.admin.cluster.configuration; -import org.elasticsearch.action.Action; +import org.elasticsearch.action.ActionType; import org.elasticsearch.common.io.stream.Writeable.Reader; -public class AddVotingConfigExclusionsAction extends Action { +public class AddVotingConfigExclusionsAction extends ActionType { public static final AddVotingConfigExclusionsAction INSTANCE = new AddVotingConfigExclusionsAction(); public static final String NAME = "cluster:admin/voting_config/add_exclusions"; @@ -29,11 +29,6 @@ private AddVotingConfigExclusionsAction() { super(NAME); } - @Override - public AddVotingConfigExclusionsResponse newResponse() { - throw new UnsupportedOperationException("usage of Streamable is to be replaced by Writeable"); - } - @Override public Reader getResponseReader() { return AddVotingConfigExclusionsResponse::new; diff --git a/server/src/main/java/org/elasticsearch/action/admin/cluster/configuration/ClearVotingConfigExclusionsAction.java b/server/src/main/java/org/elasticsearch/action/admin/cluster/configuration/ClearVotingConfigExclusionsAction.java index 49b578f48adf8..6091800693f49 100644 --- a/server/src/main/java/org/elasticsearch/action/admin/cluster/configuration/ClearVotingConfigExclusionsAction.java +++ b/server/src/main/java/org/elasticsearch/action/admin/cluster/configuration/ClearVotingConfigExclusionsAction.java @@ -18,10 +18,10 @@ */ package org.elasticsearch.action.admin.cluster.configuration; -import org.elasticsearch.action.Action; +import org.elasticsearch.action.ActionType; import org.elasticsearch.common.io.stream.Writeable.Reader; -public class ClearVotingConfigExclusionsAction extends Action { +public class ClearVotingConfigExclusionsAction extends ActionType { public static final ClearVotingConfigExclusionsAction INSTANCE = new ClearVotingConfigExclusionsAction(); public static final String NAME = "cluster:admin/voting_config/clear_exclusions"; @@ -29,11 +29,6 @@ private ClearVotingConfigExclusionsAction() { super(NAME); } - @Override - public ClearVotingConfigExclusionsResponse newResponse() { - throw new UnsupportedOperationException("usage of Streamable is to be replaced by Writeable"); - } - @Override public Reader getResponseReader() { return ClearVotingConfigExclusionsResponse::new; diff --git a/server/src/main/java/org/elasticsearch/action/admin/cluster/configuration/TransportAddVotingConfigExclusionsAction.java b/server/src/main/java/org/elasticsearch/action/admin/cluster/configuration/TransportAddVotingConfigExclusionsAction.java index 512321c7e4432..a6474510e88f3 100644 --- a/server/src/main/java/org/elasticsearch/action/admin/cluster/configuration/TransportAddVotingConfigExclusionsAction.java +++ b/server/src/main/java/org/elasticsearch/action/admin/cluster/configuration/TransportAddVotingConfigExclusionsAction.java @@ -40,6 +40,7 @@ import org.elasticsearch.common.settings.Setting; import org.elasticsearch.common.settings.Setting.Property; import org.elasticsearch.common.unit.TimeValue; +import org.elasticsearch.tasks.Task; import org.elasticsearch.threadpool.ThreadPool; import org.elasticsearch.threadpool.ThreadPool.Names; import org.elasticsearch.transport.TransportService; @@ -78,7 +79,7 @@ protected AddVotingConfigExclusionsResponse read(StreamInput in) throws IOExcept } @Override - protected void masterOperation(AddVotingConfigExclusionsRequest request, ClusterState state, + protected void masterOperation(Task task, AddVotingConfigExclusionsRequest request, ClusterState state, ActionListener listener) throws Exception { resolveVotingConfigExclusionsAndCheckMaximum(request, state); // throws IAE if no nodes matched or maximum exceeded diff --git a/server/src/main/java/org/elasticsearch/action/admin/cluster/configuration/TransportClearVotingConfigExclusionsAction.java b/server/src/main/java/org/elasticsearch/action/admin/cluster/configuration/TransportClearVotingConfigExclusionsAction.java index 7cd19a824be05..3b441e7ea9db7 100644 --- a/server/src/main/java/org/elasticsearch/action/admin/cluster/configuration/TransportClearVotingConfigExclusionsAction.java +++ b/server/src/main/java/org/elasticsearch/action/admin/cluster/configuration/TransportClearVotingConfigExclusionsAction.java @@ -38,6 +38,7 @@ import org.elasticsearch.common.inject.Inject; import org.elasticsearch.common.io.stream.StreamInput; import org.elasticsearch.common.unit.TimeValue; +import org.elasticsearch.tasks.Task; import org.elasticsearch.threadpool.ThreadPool; import org.elasticsearch.threadpool.ThreadPool.Names; import org.elasticsearch.transport.TransportService; @@ -72,7 +73,7 @@ protected ClearVotingConfigExclusionsResponse read(StreamInput in) throws IOExce } @Override - protected void masterOperation(ClearVotingConfigExclusionsRequest request, ClusterState initialState, + protected void masterOperation(Task task, ClearVotingConfigExclusionsRequest request, ClusterState initialState, ActionListener listener) throws Exception { final long startTimeMillis = threadPool.relativeTimeInMillis(); diff --git a/server/src/main/java/org/elasticsearch/action/admin/cluster/health/ClusterHealthAction.java b/server/src/main/java/org/elasticsearch/action/admin/cluster/health/ClusterHealthAction.java index 0cd148ee231e0..e8f5ecfaf5b66 100644 --- a/server/src/main/java/org/elasticsearch/action/admin/cluster/health/ClusterHealthAction.java +++ b/server/src/main/java/org/elasticsearch/action/admin/cluster/health/ClusterHealthAction.java @@ -19,9 +19,9 @@ package org.elasticsearch.action.admin.cluster.health; -import org.elasticsearch.action.Action; +import org.elasticsearch.action.StreamableResponseActionType; -public class ClusterHealthAction extends Action { +public class ClusterHealthAction extends StreamableResponseActionType { public static final ClusterHealthAction INSTANCE = new ClusterHealthAction(); public static final String NAME = "cluster:monitor/health"; diff --git a/server/src/main/java/org/elasticsearch/action/admin/cluster/health/TransportClusterHealthAction.java b/server/src/main/java/org/elasticsearch/action/admin/cluster/health/TransportClusterHealthAction.java index 0caae77d7dead..08b8730b02419 100644 --- a/server/src/main/java/org/elasticsearch/action/admin/cluster/health/TransportClusterHealthAction.java +++ b/server/src/main/java/org/elasticsearch/action/admin/cluster/health/TransportClusterHealthAction.java @@ -77,13 +77,6 @@ protected ClusterHealthResponse newResponse() { return new ClusterHealthResponse(); } - @Override - protected final void masterOperation(ClusterHealthRequest request, ClusterState state, - ActionListener listener) throws Exception { - logger.warn("attempt to execute a cluster health operation without a task"); - throw new UnsupportedOperationException("task parameter is required for this operation"); - } - @Override protected void masterOperation(Task task, final ClusterHealthRequest request, final ClusterState unusedState, final ActionListener listener) { diff --git a/server/src/main/java/org/elasticsearch/action/admin/cluster/node/hotthreads/NodesHotThreadsAction.java b/server/src/main/java/org/elasticsearch/action/admin/cluster/node/hotthreads/NodesHotThreadsAction.java index 4ea7ee5bc3bbe..4833625d29522 100644 --- a/server/src/main/java/org/elasticsearch/action/admin/cluster/node/hotthreads/NodesHotThreadsAction.java +++ b/server/src/main/java/org/elasticsearch/action/admin/cluster/node/hotthreads/NodesHotThreadsAction.java @@ -19,9 +19,9 @@ package org.elasticsearch.action.admin.cluster.node.hotthreads; -import org.elasticsearch.action.Action; +import org.elasticsearch.action.StreamableResponseActionType; -public class NodesHotThreadsAction extends Action { +public class NodesHotThreadsAction extends StreamableResponseActionType { public static final NodesHotThreadsAction INSTANCE = new NodesHotThreadsAction(); public static final String NAME = "cluster:monitor/nodes/hot_threads"; diff --git a/server/src/main/java/org/elasticsearch/action/admin/cluster/node/hotthreads/TransportNodesHotThreadsAction.java b/server/src/main/java/org/elasticsearch/action/admin/cluster/node/hotthreads/TransportNodesHotThreadsAction.java index 6321813f189fb..4f85177b6e671 100644 --- a/server/src/main/java/org/elasticsearch/action/admin/cluster/node/hotthreads/TransportNodesHotThreadsAction.java +++ b/server/src/main/java/org/elasticsearch/action/admin/cluster/node/hotthreads/TransportNodesHotThreadsAction.java @@ -29,6 +29,7 @@ import org.elasticsearch.common.io.stream.StreamInput; import org.elasticsearch.common.io.stream.StreamOutput; import org.elasticsearch.monitor.jvm.HotThreads; +import org.elasticsearch.tasks.Task; import org.elasticsearch.threadpool.ThreadPool; import org.elasticsearch.transport.TransportService; @@ -54,8 +55,8 @@ protected NodesHotThreadsResponse newResponse(NodesHotThreadsRequest request, } @Override - protected NodeRequest newNodeRequest(String nodeId, NodesHotThreadsRequest request) { - return new NodeRequest(nodeId, request); + protected NodeRequest newNodeRequest(NodesHotThreadsRequest request) { + return new NodeRequest(request); } @Override @@ -64,7 +65,7 @@ protected NodeHotThreads newNodeResponse() { } @Override - protected NodeHotThreads nodeOperation(NodeRequest request) { + protected NodeHotThreads nodeOperation(NodeRequest request, Task task) { HotThreads hotThreads = new HotThreads() .busiestThreads(request.request.threads) .type(request.request.type) @@ -85,8 +86,7 @@ public static class NodeRequest extends BaseNodeRequest { public NodeRequest() { } - NodeRequest(String nodeId, NodesHotThreadsRequest request) { - super(nodeId); + NodeRequest(NodesHotThreadsRequest request) { this.request = request; } diff --git a/server/src/main/java/org/elasticsearch/action/admin/cluster/node/info/NodesInfoAction.java b/server/src/main/java/org/elasticsearch/action/admin/cluster/node/info/NodesInfoAction.java index edc5ed7e83f0f..e94390d8f92d8 100644 --- a/server/src/main/java/org/elasticsearch/action/admin/cluster/node/info/NodesInfoAction.java +++ b/server/src/main/java/org/elasticsearch/action/admin/cluster/node/info/NodesInfoAction.java @@ -19,9 +19,9 @@ package org.elasticsearch.action.admin.cluster.node.info; -import org.elasticsearch.action.Action; +import org.elasticsearch.action.StreamableResponseActionType; -public class NodesInfoAction extends Action { +public class NodesInfoAction extends StreamableResponseActionType { public static final NodesInfoAction INSTANCE = new NodesInfoAction(); public static final String NAME = "cluster:monitor/nodes/info"; diff --git a/server/src/main/java/org/elasticsearch/action/admin/cluster/node/info/NodesInfoResponse.java b/server/src/main/java/org/elasticsearch/action/admin/cluster/node/info/NodesInfoResponse.java index 3bee722a472ec..4d70b97ca2cf6 100644 --- a/server/src/main/java/org/elasticsearch/action/admin/cluster/node/info/NodesInfoResponse.java +++ b/server/src/main/java/org/elasticsearch/action/admin/cluster/node/info/NodesInfoResponse.java @@ -22,7 +22,7 @@ import org.elasticsearch.action.FailedNodeException; import org.elasticsearch.action.support.nodes.BaseNodesResponse; import org.elasticsearch.cluster.ClusterName; -import org.elasticsearch.cluster.node.DiscoveryNode; +import org.elasticsearch.cluster.node.DiscoveryNodeRole; import org.elasticsearch.common.Strings; import org.elasticsearch.common.io.stream.StreamInput; import org.elasticsearch.common.io.stream.StreamOutput; @@ -74,8 +74,8 @@ public XContentBuilder toXContent(XContentBuilder builder, Params params) throws } builder.startArray("roles"); - for (DiscoveryNode.Role role : nodeInfo.getNode().getRoles()) { - builder.value(role.getRoleName()); + for (DiscoveryNodeRole role : nodeInfo.getNode().getRoles()) { + builder.value(role.roleName()); } builder.endArray(); diff --git a/server/src/main/java/org/elasticsearch/action/admin/cluster/node/info/TransportNodesInfoAction.java b/server/src/main/java/org/elasticsearch/action/admin/cluster/node/info/TransportNodesInfoAction.java index a1f9790af9351..903f6adb7b931 100644 --- a/server/src/main/java/org/elasticsearch/action/admin/cluster/node/info/TransportNodesInfoAction.java +++ b/server/src/main/java/org/elasticsearch/action/admin/cluster/node/info/TransportNodesInfoAction.java @@ -28,6 +28,7 @@ import org.elasticsearch.common.io.stream.StreamInput; import org.elasticsearch.common.io.stream.StreamOutput; import org.elasticsearch.node.NodeService; +import org.elasticsearch.tasks.Task; import org.elasticsearch.threadpool.ThreadPool; import org.elasticsearch.transport.TransportService; @@ -56,8 +57,8 @@ protected NodesInfoResponse newResponse(NodesInfoRequest nodesInfoRequest, } @Override - protected NodeInfoRequest newNodeRequest(String nodeId, NodesInfoRequest request) { - return new NodeInfoRequest(nodeId, request); + protected NodeInfoRequest newNodeRequest(NodesInfoRequest request) { + return new NodeInfoRequest(request); } @Override @@ -66,7 +67,7 @@ protected NodeInfo newNodeResponse() { } @Override - protected NodeInfo nodeOperation(NodeInfoRequest nodeRequest) { + protected NodeInfo nodeOperation(NodeInfoRequest nodeRequest, Task task) { NodesInfoRequest request = nodeRequest.request; return nodeService.info(request.settings(), request.os(), request.process(), request.jvm(), request.threadPool(), request.transport(), request.http(), request.plugins(), request.ingest(), request.indices()); @@ -79,8 +80,7 @@ public static class NodeInfoRequest extends BaseNodeRequest { public NodeInfoRequest() { } - public NodeInfoRequest(String nodeId, NodesInfoRequest request) { - super(nodeId); + public NodeInfoRequest(NodesInfoRequest request) { this.request = request; } diff --git a/server/src/main/java/org/elasticsearch/action/admin/cluster/node/reload/NodesReloadSecureSettingsAction.java b/server/src/main/java/org/elasticsearch/action/admin/cluster/node/reload/NodesReloadSecureSettingsAction.java index 19e8fc1929c5d..e22595c187092 100644 --- a/server/src/main/java/org/elasticsearch/action/admin/cluster/node/reload/NodesReloadSecureSettingsAction.java +++ b/server/src/main/java/org/elasticsearch/action/admin/cluster/node/reload/NodesReloadSecureSettingsAction.java @@ -19,10 +19,10 @@ package org.elasticsearch.action.admin.cluster.node.reload; -import org.elasticsearch.action.Action; +import org.elasticsearch.action.StreamableResponseActionType; public class NodesReloadSecureSettingsAction - extends Action { + extends StreamableResponseActionType { public static final NodesReloadSecureSettingsAction INSTANCE = new NodesReloadSecureSettingsAction(); public static final String NAME = "cluster:admin/nodes/reload_secure_settings"; diff --git a/server/src/main/java/org/elasticsearch/action/admin/cluster/node/reload/TransportNodesReloadSecureSettingsAction.java b/server/src/main/java/org/elasticsearch/action/admin/cluster/node/reload/TransportNodesReloadSecureSettingsAction.java index f2fef743a0d37..27860b52557e7 100644 --- a/server/src/main/java/org/elasticsearch/action/admin/cluster/node/reload/TransportNodesReloadSecureSettingsAction.java +++ b/server/src/main/java/org/elasticsearch/action/admin/cluster/node/reload/TransportNodesReloadSecureSettingsAction.java @@ -35,6 +35,7 @@ import org.elasticsearch.env.Environment; import org.elasticsearch.plugins.PluginsService; import org.elasticsearch.plugins.ReloadablePlugin; +import org.elasticsearch.tasks.Task; import org.elasticsearch.threadpool.ThreadPool; import org.elasticsearch.transport.TransportService; @@ -68,8 +69,8 @@ protected NodesReloadSecureSettingsResponse newResponse(NodesReloadSecureSetting } @Override - protected NodeRequest newNodeRequest(String nodeId, NodesReloadSecureSettingsRequest request) { - return new NodeRequest(nodeId, request); + protected NodeRequest newNodeRequest(NodesReloadSecureSettingsRequest request) { + return new NodeRequest(request); } @Override @@ -78,7 +79,7 @@ protected NodesReloadSecureSettingsResponse.NodeResponse newNodeResponse() { } @Override - protected NodesReloadSecureSettingsResponse.NodeResponse nodeOperation(NodeRequest nodeReloadRequest) { + protected NodesReloadSecureSettingsResponse.NodeResponse nodeOperation(NodeRequest nodeReloadRequest, Task task) { try (KeyStoreWrapper keystore = KeyStoreWrapper.load(environment.configFile())) { // reread keystore from config file if (keystore == null) { @@ -116,8 +117,7 @@ public static class NodeRequest extends BaseNodeRequest { public NodeRequest() { } - NodeRequest(String nodeId, NodesReloadSecureSettingsRequest request) { - super(nodeId); + NodeRequest(NodesReloadSecureSettingsRequest request) { this.request = request; } diff --git a/server/src/main/java/org/elasticsearch/action/admin/cluster/node/stats/NodeStats.java b/server/src/main/java/org/elasticsearch/action/admin/cluster/node/stats/NodeStats.java index 7a69bc0e506bd..1d32fbef2c557 100644 --- a/server/src/main/java/org/elasticsearch/action/admin/cluster/node/stats/NodeStats.java +++ b/server/src/main/java/org/elasticsearch/action/admin/cluster/node/stats/NodeStats.java @@ -21,6 +21,7 @@ import org.elasticsearch.action.support.nodes.BaseNodeResponse; import org.elasticsearch.cluster.node.DiscoveryNode; +import org.elasticsearch.cluster.node.DiscoveryNodeRole; import org.elasticsearch.common.Nullable; import org.elasticsearch.common.io.stream.StreamInput; import org.elasticsearch.common.io.stream.StreamOutput; @@ -269,8 +270,8 @@ public XContentBuilder toXContent(XContentBuilder builder, Params params) throws builder.field("ip", getNode().getAddress()); builder.startArray("roles"); - for (DiscoveryNode.Role role : getNode().getRoles()) { - builder.value(role.getRoleName()); + for (DiscoveryNodeRole role : getNode().getRoles()) { + builder.value(role.roleName()); } builder.endArray(); diff --git a/server/src/main/java/org/elasticsearch/action/admin/cluster/node/stats/NodesStatsAction.java b/server/src/main/java/org/elasticsearch/action/admin/cluster/node/stats/NodesStatsAction.java index bc8c81ef1e0f5..dbe7deed74a73 100644 --- a/server/src/main/java/org/elasticsearch/action/admin/cluster/node/stats/NodesStatsAction.java +++ b/server/src/main/java/org/elasticsearch/action/admin/cluster/node/stats/NodesStatsAction.java @@ -19,9 +19,9 @@ package org.elasticsearch.action.admin.cluster.node.stats; -import org.elasticsearch.action.Action; +import org.elasticsearch.action.StreamableResponseActionType; -public class NodesStatsAction extends Action { +public class NodesStatsAction extends StreamableResponseActionType { public static final NodesStatsAction INSTANCE = new NodesStatsAction(); public static final String NAME = "cluster:monitor/nodes/stats"; diff --git a/server/src/main/java/org/elasticsearch/action/admin/cluster/node/stats/TransportNodesStatsAction.java b/server/src/main/java/org/elasticsearch/action/admin/cluster/node/stats/TransportNodesStatsAction.java index 1028da916a2c1..f399304a8a10e 100644 --- a/server/src/main/java/org/elasticsearch/action/admin/cluster/node/stats/TransportNodesStatsAction.java +++ b/server/src/main/java/org/elasticsearch/action/admin/cluster/node/stats/TransportNodesStatsAction.java @@ -28,6 +28,7 @@ import org.elasticsearch.common.io.stream.StreamInput; import org.elasticsearch.common.io.stream.StreamOutput; import org.elasticsearch.node.NodeService; +import org.elasticsearch.tasks.Task; import org.elasticsearch.threadpool.ThreadPool; import org.elasticsearch.transport.TransportService; @@ -55,8 +56,8 @@ protected NodesStatsResponse newResponse(NodesStatsRequest request, List { +public class CancelTasksAction extends ActionType { public static final CancelTasksAction INSTANCE = new CancelTasksAction(); public static final String NAME = "cluster:admin/tasks/cancel"; @@ -34,11 +34,6 @@ private CancelTasksAction() { super(NAME); } - @Override - public CancelTasksResponse newResponse() { - throw new UnsupportedOperationException("usage of Streamable is to be replaced by Writeable"); - } - @Override public Writeable.Reader getResponseReader() { return CancelTasksResponse::new; diff --git a/server/src/main/java/org/elasticsearch/action/admin/cluster/node/tasks/get/GetTaskAction.java b/server/src/main/java/org/elasticsearch/action/admin/cluster/node/tasks/get/GetTaskAction.java index 4e88963de4c98..978e07555b517 100644 --- a/server/src/main/java/org/elasticsearch/action/admin/cluster/node/tasks/get/GetTaskAction.java +++ b/server/src/main/java/org/elasticsearch/action/admin/cluster/node/tasks/get/GetTaskAction.java @@ -19,12 +19,12 @@ package org.elasticsearch.action.admin.cluster.node.tasks.get; -import org.elasticsearch.action.Action; +import org.elasticsearch.action.StreamableResponseActionType; /** - * Action for retrieving a list of currently running tasks + * ActionType for retrieving a list of currently running tasks */ -public class GetTaskAction extends Action { +public class GetTaskAction extends StreamableResponseActionType { public static final String TASKS_ORIGIN = "tasks"; public static final GetTaskAction INSTANCE = new GetTaskAction(); diff --git a/server/src/main/java/org/elasticsearch/action/admin/cluster/node/tasks/get/TransportGetTaskAction.java b/server/src/main/java/org/elasticsearch/action/admin/cluster/node/tasks/get/TransportGetTaskAction.java index d1d72da544560..2b0ac0233be29 100644 --- a/server/src/main/java/org/elasticsearch/action/admin/cluster/node/tasks/get/TransportGetTaskAction.java +++ b/server/src/main/java/org/elasticsearch/action/admin/cluster/node/tasks/get/TransportGetTaskAction.java @@ -56,7 +56,7 @@ import static org.elasticsearch.action.admin.cluster.node.tasks.list.TransportListTasksAction.waitForCompletionTimeout; /** - * Action to get a single task. If the task isn't running then it'll try to request the status from request index. + * ActionType to get a single task. If the task isn't running then it'll try to request the status from request index. * * The general flow is: *

      diff --git a/server/src/main/java/org/elasticsearch/action/admin/cluster/node/tasks/list/ListTasksAction.java b/server/src/main/java/org/elasticsearch/action/admin/cluster/node/tasks/list/ListTasksAction.java index abba798c83cc1..7fb0b94cd8766 100644 --- a/server/src/main/java/org/elasticsearch/action/admin/cluster/node/tasks/list/ListTasksAction.java +++ b/server/src/main/java/org/elasticsearch/action/admin/cluster/node/tasks/list/ListTasksAction.java @@ -19,13 +19,13 @@ package org.elasticsearch.action.admin.cluster.node.tasks.list; -import org.elasticsearch.action.Action; +import org.elasticsearch.action.ActionType; import org.elasticsearch.common.io.stream.Writeable; /** - * Action for retrieving a list of currently running tasks + * ActionType for retrieving a list of currently running tasks */ -public class ListTasksAction extends Action { +public class ListTasksAction extends ActionType { public static final ListTasksAction INSTANCE = new ListTasksAction(); public static final String NAME = "cluster:monitor/tasks/lists"; @@ -34,11 +34,6 @@ private ListTasksAction() { super(NAME); } - @Override - public ListTasksResponse newResponse() { - throw new UnsupportedOperationException("usage of Streamable is to be replaced by Writeable"); - } - @Override public Writeable.Reader getResponseReader() { return ListTasksResponse::new; diff --git a/server/src/main/java/org/elasticsearch/action/admin/cluster/node/tasks/list/ListTasksResponse.java b/server/src/main/java/org/elasticsearch/action/admin/cluster/node/tasks/list/ListTasksResponse.java index 557338ed0f77e..d6fc541ba687a 100644 --- a/server/src/main/java/org/elasticsearch/action/admin/cluster/node/tasks/list/ListTasksResponse.java +++ b/server/src/main/java/org/elasticsearch/action/admin/cluster/node/tasks/list/ListTasksResponse.java @@ -23,6 +23,7 @@ import org.elasticsearch.action.TaskOperationFailure; import org.elasticsearch.action.support.tasks.BaseTasksResponse; import org.elasticsearch.cluster.node.DiscoveryNode; +import org.elasticsearch.cluster.node.DiscoveryNodeRole; import org.elasticsearch.cluster.node.DiscoveryNodes; import org.elasticsearch.common.ParseField; import org.elasticsearch.common.Strings; @@ -175,8 +176,8 @@ public XContentBuilder toXContentGroupedByNode(XContentBuilder builder, Params p builder.field("ip", node.getAddress()); builder.startArray("roles"); - for (DiscoveryNode.Role role : node.getRoles()) { - builder.value(role.getRoleName()); + for (DiscoveryNodeRole role : node.getRoles()) { + builder.value(role.roleName()); } builder.endArray(); diff --git a/server/src/main/java/org/elasticsearch/action/admin/cluster/node/usage/NodesUsageAction.java b/server/src/main/java/org/elasticsearch/action/admin/cluster/node/usage/NodesUsageAction.java index 6bc6dce54945a..aaf7616a19b1b 100644 --- a/server/src/main/java/org/elasticsearch/action/admin/cluster/node/usage/NodesUsageAction.java +++ b/server/src/main/java/org/elasticsearch/action/admin/cluster/node/usage/NodesUsageAction.java @@ -19,9 +19,9 @@ package org.elasticsearch.action.admin.cluster.node.usage; -import org.elasticsearch.action.Action; +import org.elasticsearch.action.StreamableResponseActionType; -public class NodesUsageAction extends Action { +public class NodesUsageAction extends StreamableResponseActionType { public static final NodesUsageAction INSTANCE = new NodesUsageAction(); public static final String NAME = "cluster:monitor/nodes/usage"; diff --git a/server/src/main/java/org/elasticsearch/action/admin/cluster/node/usage/NodesUsageRequestBuilder.java b/server/src/main/java/org/elasticsearch/action/admin/cluster/node/usage/NodesUsageRequestBuilder.java index d8af249be0251..3133a6a18947b 100644 --- a/server/src/main/java/org/elasticsearch/action/admin/cluster/node/usage/NodesUsageRequestBuilder.java +++ b/server/src/main/java/org/elasticsearch/action/admin/cluster/node/usage/NodesUsageRequestBuilder.java @@ -19,14 +19,14 @@ package org.elasticsearch.action.admin.cluster.node.usage; -import org.elasticsearch.action.Action; +import org.elasticsearch.action.ActionType; import org.elasticsearch.action.support.nodes.NodesOperationRequestBuilder; import org.elasticsearch.client.ElasticsearchClient; public class NodesUsageRequestBuilder extends NodesOperationRequestBuilder { - public NodesUsageRequestBuilder(ElasticsearchClient client, Action action) { + public NodesUsageRequestBuilder(ElasticsearchClient client, ActionType action) { super(client, action, new NodesUsageRequest()); } diff --git a/server/src/main/java/org/elasticsearch/action/admin/cluster/node/usage/TransportNodesUsageAction.java b/server/src/main/java/org/elasticsearch/action/admin/cluster/node/usage/TransportNodesUsageAction.java index c34ca9b614432..ed40aaddae24a 100644 --- a/server/src/main/java/org/elasticsearch/action/admin/cluster/node/usage/TransportNodesUsageAction.java +++ b/server/src/main/java/org/elasticsearch/action/admin/cluster/node/usage/TransportNodesUsageAction.java @@ -27,6 +27,7 @@ import org.elasticsearch.common.inject.Inject; import org.elasticsearch.common.io.stream.StreamInput; import org.elasticsearch.common.io.stream.StreamOutput; +import org.elasticsearch.tasks.Task; import org.elasticsearch.threadpool.ThreadPool; import org.elasticsearch.transport.TransportService; import org.elasticsearch.usage.UsageService; @@ -53,8 +54,8 @@ protected NodesUsageResponse newResponse(NodesUsageRequest request, List { +public final class RemoteInfoAction extends StreamableResponseActionType { public static final String NAME = "cluster:monitor/remote/info"; public static final RemoteInfoAction INSTANCE = new RemoteInfoAction(); diff --git a/server/src/main/java/org/elasticsearch/action/admin/cluster/repositories/delete/DeleteRepositoryAction.java b/server/src/main/java/org/elasticsearch/action/admin/cluster/repositories/delete/DeleteRepositoryAction.java index a31984e222793..791e4c6e57d70 100644 --- a/server/src/main/java/org/elasticsearch/action/admin/cluster/repositories/delete/DeleteRepositoryAction.java +++ b/server/src/main/java/org/elasticsearch/action/admin/cluster/repositories/delete/DeleteRepositoryAction.java @@ -19,13 +19,14 @@ package org.elasticsearch.action.admin.cluster.repositories.delete; -import org.elasticsearch.action.Action; +import org.elasticsearch.action.ActionType; import org.elasticsearch.action.support.master.AcknowledgedResponse; +import org.elasticsearch.common.io.stream.Writeable; /** * Unregister repository action */ -public class DeleteRepositoryAction extends Action { +public class DeleteRepositoryAction extends ActionType { public static final DeleteRepositoryAction INSTANCE = new DeleteRepositoryAction(); public static final String NAME = "cluster:admin/repository/delete"; @@ -35,8 +36,8 @@ private DeleteRepositoryAction() { } @Override - public AcknowledgedResponse newResponse() { - return new AcknowledgedResponse(); + public Writeable.Reader getResponseReader() { + return AcknowledgedResponse::new; } } diff --git a/server/src/main/java/org/elasticsearch/action/admin/cluster/repositories/delete/TransportDeleteRepositoryAction.java b/server/src/main/java/org/elasticsearch/action/admin/cluster/repositories/delete/TransportDeleteRepositoryAction.java index 48b9bdaa4511b..5a93c9dd6f2e0 100644 --- a/server/src/main/java/org/elasticsearch/action/admin/cluster/repositories/delete/TransportDeleteRepositoryAction.java +++ b/server/src/main/java/org/elasticsearch/action/admin/cluster/repositories/delete/TransportDeleteRepositoryAction.java @@ -29,10 +29,14 @@ import org.elasticsearch.cluster.metadata.IndexNameExpressionResolver; import org.elasticsearch.cluster.service.ClusterService; import org.elasticsearch.common.inject.Inject; +import org.elasticsearch.common.io.stream.StreamInput; import org.elasticsearch.repositories.RepositoriesService; +import org.elasticsearch.tasks.Task; import org.elasticsearch.threadpool.ThreadPool; import org.elasticsearch.transport.TransportService; +import java.io.IOException; + /** * Transport action for unregister repository operation */ @@ -54,9 +58,14 @@ protected String executor() { return ThreadPool.Names.SAME; } + @Override + protected AcknowledgedResponse read(StreamInput in) throws IOException { + return new AcknowledgedResponse(in); + } + @Override protected AcknowledgedResponse newResponse() { - return new AcknowledgedResponse(); + throw new UnsupportedOperationException("usage of Streamable is to be replaced by Writeable"); } @Override @@ -65,7 +74,7 @@ protected ClusterBlockException checkBlock(DeleteRepositoryRequest request, Clus } @Override - protected void masterOperation(final DeleteRepositoryRequest request, ClusterState state, + protected void masterOperation(Task task, final DeleteRepositoryRequest request, ClusterState state, final ActionListener listener) { repositoriesService.unregisterRepository( request, ActionListener.delegateFailure(listener, diff --git a/server/src/main/java/org/elasticsearch/action/admin/cluster/repositories/get/GetRepositoriesAction.java b/server/src/main/java/org/elasticsearch/action/admin/cluster/repositories/get/GetRepositoriesAction.java index d89e466461d9b..3b4408ea3dd3e 100644 --- a/server/src/main/java/org/elasticsearch/action/admin/cluster/repositories/get/GetRepositoriesAction.java +++ b/server/src/main/java/org/elasticsearch/action/admin/cluster/repositories/get/GetRepositoriesAction.java @@ -19,12 +19,13 @@ package org.elasticsearch.action.admin.cluster.repositories.get; -import org.elasticsearch.action.Action; +import org.elasticsearch.action.ActionType; +import org.elasticsearch.common.io.stream.Writeable; /** * Get repositories action */ -public class GetRepositoriesAction extends Action { +public class GetRepositoriesAction extends ActionType { public static final GetRepositoriesAction INSTANCE = new GetRepositoriesAction(); public static final String NAME = "cluster:admin/repository/get"; @@ -34,8 +35,8 @@ private GetRepositoriesAction() { } @Override - public GetRepositoriesResponse newResponse() { - return new GetRepositoriesResponse(); + public Writeable.Reader getResponseReader() { + return GetRepositoriesResponse::new; } } diff --git a/server/src/main/java/org/elasticsearch/action/admin/cluster/repositories/get/GetRepositoriesResponse.java b/server/src/main/java/org/elasticsearch/action/admin/cluster/repositories/get/GetRepositoriesResponse.java index 24228aa565871..4fb9cbaebe941 100644 --- a/server/src/main/java/org/elasticsearch/action/admin/cluster/repositories/get/GetRepositoriesResponse.java +++ b/server/src/main/java/org/elasticsearch/action/admin/cluster/repositories/get/GetRepositoriesResponse.java @@ -29,7 +29,6 @@ import org.elasticsearch.common.xcontent.XContentParser; import java.io.IOException; -import java.util.Collections; import java.util.List; import static org.elasticsearch.common.xcontent.XContentParserUtils.ensureExpectedToken; @@ -39,16 +38,16 @@ */ public class GetRepositoriesResponse extends ActionResponse implements ToXContentObject { - private RepositoriesMetaData repositories; - - GetRepositoriesResponse() { - repositories = new RepositoriesMetaData(Collections.emptyList()); - } + private final RepositoriesMetaData repositories; GetRepositoriesResponse(RepositoriesMetaData repositories) { this.repositories = repositories; } + public GetRepositoriesResponse(StreamInput in) throws IOException { + repositories = new RepositoriesMetaData(in); + } + /** * List of repositories to return * @@ -61,7 +60,7 @@ public List repositories() { @Override public void readFrom(StreamInput in) throws IOException { - repositories = new RepositoriesMetaData(in); + throw new UnsupportedOperationException("usage of Streamable is to be replaced by Writeable"); } @Override diff --git a/server/src/main/java/org/elasticsearch/action/admin/cluster/repositories/get/TransportGetRepositoriesAction.java b/server/src/main/java/org/elasticsearch/action/admin/cluster/repositories/get/TransportGetRepositoriesAction.java index 4b3ee1cd9251a..3f0ee69d2a5aa 100644 --- a/server/src/main/java/org/elasticsearch/action/admin/cluster/repositories/get/TransportGetRepositoriesAction.java +++ b/server/src/main/java/org/elasticsearch/action/admin/cluster/repositories/get/TransportGetRepositoriesAction.java @@ -31,11 +31,14 @@ import org.elasticsearch.cluster.metadata.RepositoryMetaData; import org.elasticsearch.cluster.service.ClusterService; import org.elasticsearch.common.inject.Inject; +import org.elasticsearch.common.io.stream.StreamInput; import org.elasticsearch.common.regex.Regex; import org.elasticsearch.repositories.RepositoryMissingException; +import org.elasticsearch.tasks.Task; import org.elasticsearch.threadpool.ThreadPool; import org.elasticsearch.transport.TransportService; +import java.io.IOException; import java.util.ArrayList; import java.util.Collections; import java.util.LinkedHashSet; @@ -62,7 +65,12 @@ protected String executor() { @Override protected GetRepositoriesResponse newResponse() { - return new GetRepositoriesResponse(); + throw new UnsupportedOperationException("usage of Streamable is to be replaced by Writeable"); + } + + @Override + protected GetRepositoriesResponse read(StreamInput in) throws IOException { + return new GetRepositoriesResponse(in); } @Override @@ -71,7 +79,7 @@ protected ClusterBlockException checkBlock(GetRepositoriesRequest request, Clust } @Override - protected void masterOperation(final GetRepositoriesRequest request, ClusterState state, + protected void masterOperation(Task task, final GetRepositoriesRequest request, ClusterState state, final ActionListener listener) { MetaData metaData = state.metaData(); RepositoriesMetaData repositories = metaData.custom(RepositoriesMetaData.TYPE); diff --git a/server/src/main/java/org/elasticsearch/action/admin/cluster/repositories/put/PutRepositoryAction.java b/server/src/main/java/org/elasticsearch/action/admin/cluster/repositories/put/PutRepositoryAction.java index 7c98f591e0e1d..0aa5c6a5d51f9 100644 --- a/server/src/main/java/org/elasticsearch/action/admin/cluster/repositories/put/PutRepositoryAction.java +++ b/server/src/main/java/org/elasticsearch/action/admin/cluster/repositories/put/PutRepositoryAction.java @@ -19,13 +19,14 @@ package org.elasticsearch.action.admin.cluster.repositories.put; -import org.elasticsearch.action.Action; +import org.elasticsearch.action.ActionType; import org.elasticsearch.action.support.master.AcknowledgedResponse; +import org.elasticsearch.common.io.stream.Writeable; /** * Register repository action */ -public class PutRepositoryAction extends Action { +public class PutRepositoryAction extends ActionType { public static final PutRepositoryAction INSTANCE = new PutRepositoryAction(); public static final String NAME = "cluster:admin/repository/put"; @@ -35,8 +36,8 @@ private PutRepositoryAction() { } @Override - public AcknowledgedResponse newResponse() { - return new AcknowledgedResponse(); + public Writeable.Reader getResponseReader() { + return AcknowledgedResponse::new; } } diff --git a/server/src/main/java/org/elasticsearch/action/admin/cluster/repositories/put/TransportPutRepositoryAction.java b/server/src/main/java/org/elasticsearch/action/admin/cluster/repositories/put/TransportPutRepositoryAction.java index a10e5c878a7c9..21f4c7e6f384d 100644 --- a/server/src/main/java/org/elasticsearch/action/admin/cluster/repositories/put/TransportPutRepositoryAction.java +++ b/server/src/main/java/org/elasticsearch/action/admin/cluster/repositories/put/TransportPutRepositoryAction.java @@ -29,10 +29,14 @@ import org.elasticsearch.cluster.metadata.IndexNameExpressionResolver; import org.elasticsearch.cluster.service.ClusterService; import org.elasticsearch.common.inject.Inject; +import org.elasticsearch.common.io.stream.StreamInput; import org.elasticsearch.repositories.RepositoriesService; +import org.elasticsearch.tasks.Task; import org.elasticsearch.threadpool.ThreadPool; import org.elasticsearch.transport.TransportService; +import java.io.IOException; + /** * Transport action for register repository operation */ @@ -54,9 +58,14 @@ protected String executor() { return ThreadPool.Names.SAME; } + @Override + protected AcknowledgedResponse read(StreamInput in) throws IOException { + return new AcknowledgedResponse(in); + } + @Override protected AcknowledgedResponse newResponse() { - return new AcknowledgedResponse(); + throw new UnsupportedOperationException("usage of Streamable is to be replaced by Writeable"); } @Override @@ -65,7 +74,7 @@ protected ClusterBlockException checkBlock(PutRepositoryRequest request, Cluster } @Override - protected void masterOperation(final PutRepositoryRequest request, ClusterState state, + protected void masterOperation(Task task, final PutRepositoryRequest request, ClusterState state, final ActionListener listener) { repositoriesService.registerRepository(request, ActionListener.delegateFailure(listener, (delegatedListener, response) -> delegatedListener.onResponse(new AcknowledgedResponse(response.isAcknowledged())))); diff --git a/server/src/main/java/org/elasticsearch/action/admin/cluster/repositories/verify/TransportVerifyRepositoryAction.java b/server/src/main/java/org/elasticsearch/action/admin/cluster/repositories/verify/TransportVerifyRepositoryAction.java index d4ec1d3a8bcb4..3de71e346f3c4 100644 --- a/server/src/main/java/org/elasticsearch/action/admin/cluster/repositories/verify/TransportVerifyRepositoryAction.java +++ b/server/src/main/java/org/elasticsearch/action/admin/cluster/repositories/verify/TransportVerifyRepositoryAction.java @@ -30,6 +30,7 @@ import org.elasticsearch.cluster.service.ClusterService; import org.elasticsearch.common.inject.Inject; import org.elasticsearch.repositories.RepositoriesService; +import org.elasticsearch.tasks.Task; import org.elasticsearch.threadpool.ThreadPool; import org.elasticsearch.transport.TransportService; @@ -66,7 +67,7 @@ protected ClusterBlockException checkBlock(VerifyRepositoryRequest request, Clus } @Override - protected void masterOperation(final VerifyRepositoryRequest request, ClusterState state, + protected void masterOperation(Task task, final VerifyRepositoryRequest request, ClusterState state, final ActionListener listener) { repositoriesService.verifyRepository(request.name(), ActionListener.delegateFailure(listener, (delegatedListener, verifyResponse) -> diff --git a/server/src/main/java/org/elasticsearch/action/admin/cluster/repositories/verify/VerifyRepositoryAction.java b/server/src/main/java/org/elasticsearch/action/admin/cluster/repositories/verify/VerifyRepositoryAction.java index 743b0a57f3aa3..9601ceecf0dcd 100644 --- a/server/src/main/java/org/elasticsearch/action/admin/cluster/repositories/verify/VerifyRepositoryAction.java +++ b/server/src/main/java/org/elasticsearch/action/admin/cluster/repositories/verify/VerifyRepositoryAction.java @@ -19,12 +19,12 @@ package org.elasticsearch.action.admin.cluster.repositories.verify; -import org.elasticsearch.action.Action; +import org.elasticsearch.action.StreamableResponseActionType; /** * Unregister repository action */ -public class VerifyRepositoryAction extends Action { +public class VerifyRepositoryAction extends StreamableResponseActionType { public static final VerifyRepositoryAction INSTANCE = new VerifyRepositoryAction(); public static final String NAME = "cluster:admin/repository/verify"; diff --git a/server/src/main/java/org/elasticsearch/action/admin/cluster/reroute/ClusterRerouteAction.java b/server/src/main/java/org/elasticsearch/action/admin/cluster/reroute/ClusterRerouteAction.java index 06b083e3414b2..82577a58cace7 100644 --- a/server/src/main/java/org/elasticsearch/action/admin/cluster/reroute/ClusterRerouteAction.java +++ b/server/src/main/java/org/elasticsearch/action/admin/cluster/reroute/ClusterRerouteAction.java @@ -19,9 +19,10 @@ package org.elasticsearch.action.admin.cluster.reroute; -import org.elasticsearch.action.Action; +import org.elasticsearch.action.ActionType; +import org.elasticsearch.common.io.stream.Writeable; -public class ClusterRerouteAction extends Action { +public class ClusterRerouteAction extends ActionType { public static final ClusterRerouteAction INSTANCE = new ClusterRerouteAction(); public static final String NAME = "cluster:admin/reroute"; @@ -31,7 +32,7 @@ private ClusterRerouteAction() { } @Override - public ClusterRerouteResponse newResponse() { - return new ClusterRerouteResponse(); + public Writeable.Reader getResponseReader() { + return ClusterRerouteResponse::new; } } diff --git a/server/src/main/java/org/elasticsearch/action/admin/cluster/reroute/ClusterRerouteResponse.java b/server/src/main/java/org/elasticsearch/action/admin/cluster/reroute/ClusterRerouteResponse.java index 652a794e0f1cb..95ec34acc6b69 100644 --- a/server/src/main/java/org/elasticsearch/action/admin/cluster/reroute/ClusterRerouteResponse.java +++ b/server/src/main/java/org/elasticsearch/action/admin/cluster/reroute/ClusterRerouteResponse.java @@ -35,11 +35,13 @@ */ public class ClusterRerouteResponse extends AcknowledgedResponse implements ToXContentObject { - private ClusterState state; - private RoutingExplanations explanations; - - ClusterRerouteResponse() { + private final ClusterState state; + private final RoutingExplanations explanations; + ClusterRerouteResponse(StreamInput in) throws IOException { + super(in); + state = ClusterState.readFrom(in, null); + explanations = RoutingExplanations.readFrom(in); } ClusterRerouteResponse(boolean acknowledged, ClusterState state, RoutingExplanations explanations) { @@ -59,13 +61,6 @@ public RoutingExplanations getExplanations() { return this.explanations; } - @Override - public void readFrom(StreamInput in) throws IOException { - super.readFrom(in); - state = ClusterState.readFrom(in, null); - explanations = RoutingExplanations.readFrom(in); - } - @Override public void writeTo(StreamOutput out) throws IOException { super.writeTo(out); diff --git a/server/src/main/java/org/elasticsearch/action/admin/cluster/reroute/TransportClusterRerouteAction.java b/server/src/main/java/org/elasticsearch/action/admin/cluster/reroute/TransportClusterRerouteAction.java index 4b51145204a61..eea3dafcb1eba 100644 --- a/server/src/main/java/org/elasticsearch/action/admin/cluster/reroute/TransportClusterRerouteAction.java +++ b/server/src/main/java/org/elasticsearch/action/admin/cluster/reroute/TransportClusterRerouteAction.java @@ -46,9 +46,12 @@ import org.elasticsearch.common.collect.ImmutableOpenIntMap; import org.elasticsearch.common.collect.ImmutableOpenMap; import org.elasticsearch.common.inject.Inject; +import org.elasticsearch.common.io.stream.StreamInput; +import org.elasticsearch.tasks.Task; import org.elasticsearch.threadpool.ThreadPool; import org.elasticsearch.transport.TransportService; +import java.io.IOException; import java.util.ArrayList; import java.util.HashMap; import java.util.List; @@ -80,11 +83,16 @@ protected ClusterBlockException checkBlock(ClusterRerouteRequest request, Cluste @Override protected ClusterRerouteResponse newResponse() { - return new ClusterRerouteResponse(); + throw new UnsupportedOperationException("usage of Streamable is to be replaced by Writeable"); } @Override - protected void masterOperation(final ClusterRerouteRequest request, final ClusterState state, + protected ClusterRerouteResponse read(StreamInput in) throws IOException { + return new ClusterRerouteResponse(in); + } + + @Override + protected void masterOperation(Task task, final ClusterRerouteRequest request, final ClusterState state, final ActionListener listener) { Map> stalePrimaryAllocations = new HashMap<>(); for (AllocationCommand command : request.getCommands().commands()) { diff --git a/server/src/main/java/org/elasticsearch/action/admin/cluster/settings/ClusterUpdateSettingsAction.java b/server/src/main/java/org/elasticsearch/action/admin/cluster/settings/ClusterUpdateSettingsAction.java index af6a87f5a57e6..0bdb77d0c670a 100644 --- a/server/src/main/java/org/elasticsearch/action/admin/cluster/settings/ClusterUpdateSettingsAction.java +++ b/server/src/main/java/org/elasticsearch/action/admin/cluster/settings/ClusterUpdateSettingsAction.java @@ -19,9 +19,10 @@ package org.elasticsearch.action.admin.cluster.settings; -import org.elasticsearch.action.Action; +import org.elasticsearch.action.ActionType; +import org.elasticsearch.common.io.stream.Writeable; -public class ClusterUpdateSettingsAction extends Action { +public class ClusterUpdateSettingsAction extends ActionType { public static final ClusterUpdateSettingsAction INSTANCE = new ClusterUpdateSettingsAction(); public static final String NAME = "cluster:admin/settings/update"; @@ -31,7 +32,7 @@ private ClusterUpdateSettingsAction() { } @Override - public ClusterUpdateSettingsResponse newResponse() { - return new ClusterUpdateSettingsResponse(); + public Writeable.Reader getResponseReader() { + return ClusterUpdateSettingsResponse::new; } } diff --git a/server/src/main/java/org/elasticsearch/action/admin/cluster/settings/ClusterUpdateSettingsResponse.java b/server/src/main/java/org/elasticsearch/action/admin/cluster/settings/ClusterUpdateSettingsResponse.java index 4e925b398b6d5..5023e16249a4c 100644 --- a/server/src/main/java/org/elasticsearch/action/admin/cluster/settings/ClusterUpdateSettingsResponse.java +++ b/server/src/main/java/org/elasticsearch/action/admin/cluster/settings/ClusterUpdateSettingsResponse.java @@ -31,6 +31,8 @@ import java.io.IOException; import java.util.Objects; +import static org.elasticsearch.common.xcontent.ConstructingObjectParser.constructorArg; + /** * A response for a cluster update settings action. */ @@ -40,23 +42,22 @@ public class ClusterUpdateSettingsResponse extends AcknowledgedResponse { private static final ParseField TRANSIENT = new ParseField("transient"); private static final ConstructingObjectParser PARSER = new ConstructingObjectParser<>( - "cluster_update_settings_response", true, a -> new ClusterUpdateSettingsResponse((boolean) a[0])); + "cluster_update_settings_response", true, args -> { + return new ClusterUpdateSettingsResponse((boolean) args[0], (Settings) args[1], (Settings) args[2]); + }); static { declareAcknowledgedField(PARSER); - PARSER.declareObject((r, p) -> r.persistentSettings = p, (p, c) -> Settings.fromXContent(p), PERSISTENT); - PARSER.declareObject((r, t) -> r.transientSettings = t, (p, c) -> Settings.fromXContent(p), TRANSIENT); + PARSER.declareObject(constructorArg(), (p, c) -> Settings.fromXContent(p), TRANSIENT); + PARSER.declareObject(constructorArg(), (p, c) -> Settings.fromXContent(p), PERSISTENT); } - Settings transientSettings; - Settings persistentSettings; - - ClusterUpdateSettingsResponse() { - this.persistentSettings = Settings.EMPTY; - this.transientSettings = Settings.EMPTY; - } + final Settings transientSettings; + final Settings persistentSettings; - ClusterUpdateSettingsResponse(boolean acknowledged) { - super(acknowledged); + ClusterUpdateSettingsResponse(StreamInput in) throws IOException { + super(in); + transientSettings = Settings.readSettingsFromStream(in); + persistentSettings = Settings.readSettingsFromStream(in); } ClusterUpdateSettingsResponse(boolean acknowledged, Settings transientSettings, Settings persistentSettings) { @@ -65,13 +66,6 @@ public class ClusterUpdateSettingsResponse extends AcknowledgedResponse { this.transientSettings = transientSettings; } - @Override - public void readFrom(StreamInput in) throws IOException { - super.readFrom(in); - transientSettings = Settings.readSettingsFromStream(in); - persistentSettings = Settings.readSettingsFromStream(in); - } - public Settings getTransientSettings() { return transientSettings; } diff --git a/server/src/main/java/org/elasticsearch/action/admin/cluster/settings/TransportClusterUpdateSettingsAction.java b/server/src/main/java/org/elasticsearch/action/admin/cluster/settings/TransportClusterUpdateSettingsAction.java index cf34094290a87..ec55a21bac4bf 100644 --- a/server/src/main/java/org/elasticsearch/action/admin/cluster/settings/TransportClusterUpdateSettingsAction.java +++ b/server/src/main/java/org/elasticsearch/action/admin/cluster/settings/TransportClusterUpdateSettingsAction.java @@ -36,10 +36,14 @@ import org.elasticsearch.common.Nullable; import org.elasticsearch.common.Priority; import org.elasticsearch.common.inject.Inject; +import org.elasticsearch.common.io.stream.StreamInput; import org.elasticsearch.common.settings.ClusterSettings; +import org.elasticsearch.tasks.Task; import org.elasticsearch.threadpool.ThreadPool; import org.elasticsearch.transport.TransportService; +import java.io.IOException; + public class TransportClusterUpdateSettingsAction extends TransportMasterNodeAction { @@ -81,11 +85,16 @@ protected ClusterBlockException checkBlock(ClusterUpdateSettingsRequest request, @Override protected ClusterUpdateSettingsResponse newResponse() { - return new ClusterUpdateSettingsResponse(); + throw new UnsupportedOperationException("usage of Streamable is to be replaced by Writeable"); + } + + @Override + protected ClusterUpdateSettingsResponse read(StreamInput in) throws IOException { + return new ClusterUpdateSettingsResponse(in); } @Override - protected void masterOperation(final ClusterUpdateSettingsRequest request, final ClusterState state, + protected void masterOperation(Task task, final ClusterUpdateSettingsRequest request, final ClusterState state, final ActionListener listener) { final SettingsUpdater updater = new SettingsUpdater(clusterSettings); clusterService.submitStateUpdateTask("cluster_update_settings", diff --git a/server/src/main/java/org/elasticsearch/action/admin/cluster/shards/ClusterSearchShardsAction.java b/server/src/main/java/org/elasticsearch/action/admin/cluster/shards/ClusterSearchShardsAction.java index 869aecf095431..837fb9e39709f 100644 --- a/server/src/main/java/org/elasticsearch/action/admin/cluster/shards/ClusterSearchShardsAction.java +++ b/server/src/main/java/org/elasticsearch/action/admin/cluster/shards/ClusterSearchShardsAction.java @@ -19,10 +19,10 @@ package org.elasticsearch.action.admin.cluster.shards; -import org.elasticsearch.action.Action; +import org.elasticsearch.action.ActionType; import org.elasticsearch.common.io.stream.Writeable; -public class ClusterSearchShardsAction extends Action { +public class ClusterSearchShardsAction extends ActionType { public static final ClusterSearchShardsAction INSTANCE = new ClusterSearchShardsAction(); public static final String NAME = "indices:admin/shards/search_shards"; @@ -31,11 +31,6 @@ private ClusterSearchShardsAction() { super(NAME); } - @Override - public ClusterSearchShardsResponse newResponse() { - throw new UnsupportedOperationException("usage of Streamable is to be replaced by Writeable"); - } - @Override public Writeable.Reader getResponseReader() { return ClusterSearchShardsResponse::new; diff --git a/server/src/main/java/org/elasticsearch/action/admin/cluster/shards/TransportClusterSearchShardsAction.java b/server/src/main/java/org/elasticsearch/action/admin/cluster/shards/TransportClusterSearchShardsAction.java index 39006cd1e8407..23b9f34a241c3 100644 --- a/server/src/main/java/org/elasticsearch/action/admin/cluster/shards/TransportClusterSearchShardsAction.java +++ b/server/src/main/java/org/elasticsearch/action/admin/cluster/shards/TransportClusterSearchShardsAction.java @@ -36,6 +36,7 @@ import org.elasticsearch.index.shard.ShardId; import org.elasticsearch.indices.IndicesService; import org.elasticsearch.search.internal.AliasFilter; +import org.elasticsearch.tasks.Task; import org.elasticsearch.threadpool.ThreadPool; import org.elasticsearch.transport.TransportService; @@ -82,7 +83,7 @@ protected ClusterSearchShardsResponse read(StreamInput in) throws IOException { } @Override - protected void masterOperation(final ClusterSearchShardsRequest request, final ClusterState state, + protected void masterOperation(Task task, final ClusterSearchShardsRequest request, final ClusterState state, final ActionListener listener) { ClusterState clusterState = clusterService.state(); String[] concreteIndices = indexNameExpressionResolver.concreteIndexNames(clusterState, request); diff --git a/server/src/main/java/org/elasticsearch/action/admin/cluster/snapshots/create/CreateSnapshotAction.java b/server/src/main/java/org/elasticsearch/action/admin/cluster/snapshots/create/CreateSnapshotAction.java index d37132a1d81e7..a7a8e5467c71e 100644 --- a/server/src/main/java/org/elasticsearch/action/admin/cluster/snapshots/create/CreateSnapshotAction.java +++ b/server/src/main/java/org/elasticsearch/action/admin/cluster/snapshots/create/CreateSnapshotAction.java @@ -19,12 +19,12 @@ package org.elasticsearch.action.admin.cluster.snapshots.create; -import org.elasticsearch.action.Action; +import org.elasticsearch.action.StreamableResponseActionType; /** * Create snapshot action */ -public class CreateSnapshotAction extends Action { +public class CreateSnapshotAction extends StreamableResponseActionType { public static final CreateSnapshotAction INSTANCE = new CreateSnapshotAction(); public static final String NAME = "cluster:admin/snapshot/create"; diff --git a/server/src/main/java/org/elasticsearch/action/admin/cluster/snapshots/create/CreateSnapshotRequest.java b/server/src/main/java/org/elasticsearch/action/admin/cluster/snapshots/create/CreateSnapshotRequest.java index 8f83da053b215..e0250da5ed1cb 100644 --- a/server/src/main/java/org/elasticsearch/action/admin/cluster/snapshots/create/CreateSnapshotRequest.java +++ b/server/src/main/java/org/elasticsearch/action/admin/cluster/snapshots/create/CreateSnapshotRequest.java @@ -19,12 +19,14 @@ package org.elasticsearch.action.admin.cluster.snapshots.create; +import org.elasticsearch.ElasticsearchException; import org.elasticsearch.ElasticsearchGenerationException; import org.elasticsearch.action.ActionRequestValidationException; import org.elasticsearch.action.IndicesRequest; import org.elasticsearch.action.support.IndicesOptions; import org.elasticsearch.action.support.master.MasterNodeRequest; import org.elasticsearch.common.Strings; +import org.elasticsearch.common.bytes.BytesReference; import org.elasticsearch.common.io.stream.StreamInput; import org.elasticsearch.common.io.stream.StreamOutput; import org.elasticsearch.common.settings.Settings; @@ -46,6 +48,7 @@ import static org.elasticsearch.common.settings.Settings.readSettingsFromStream; import static org.elasticsearch.common.settings.Settings.writeSettingsToStream; import static org.elasticsearch.common.xcontent.support.XContentMapValues.nodeBooleanValue; +import static org.elasticsearch.snapshots.SnapshotInfo.METADATA_FIELD_INTRODUCED; /** * Create snapshot request @@ -63,6 +66,7 @@ */ public class CreateSnapshotRequest extends MasterNodeRequest implements IndicesRequest.Replaceable, ToXContentObject { + public static int MAXIMUM_METADATA_BYTES = 1024; // chosen arbitrarily private String snapshot; @@ -80,6 +84,8 @@ public class CreateSnapshotRequest extends MasterNodeRequest userMetadata; + public CreateSnapshotRequest() { } @@ -104,6 +110,9 @@ public CreateSnapshotRequest(StreamInput in) throws IOException { includeGlobalState = in.readBoolean(); waitForCompletion = in.readBoolean(); partial = in.readBoolean(); + if (in.getVersion().onOrAfter(METADATA_FIELD_INTRODUCED)) { + userMetadata = in.readMap(); + } } @Override @@ -117,6 +126,9 @@ public void writeTo(StreamOutput out) throws IOException { out.writeBoolean(includeGlobalState); out.writeBoolean(waitForCompletion); out.writeBoolean(partial); + if (out.getVersion().onOrAfter(METADATA_FIELD_INTRODUCED)) { + out.writeMap(userMetadata); + } } @Override @@ -144,9 +156,28 @@ public ActionRequestValidationException validate() { if (settings == null) { validationException = addValidationError("settings is null", validationException); } + final int metadataSize = metadataSize(userMetadata); + if (metadataSize > MAXIMUM_METADATA_BYTES) { + validationException = addValidationError("metadata must be smaller than 1024 bytes, but was [" + metadataSize + "]", + validationException); + } return validationException; } + private static int metadataSize(Map userMetadata) { + if (userMetadata == null) { + return 0; + } + try (XContentBuilder builder = XContentFactory.jsonBuilder()) { + builder.value(userMetadata); + int size = BytesReference.bytes(builder).length(); + return size; + } catch (IOException e) { + // This should not be possible as we are just rendering the xcontent in memory + throw new ElasticsearchException(e); + } + } + /** * Sets the snapshot name * @@ -378,6 +409,15 @@ public boolean includeGlobalState() { return includeGlobalState; } + public Map userMetadata() { + return userMetadata; + } + + public CreateSnapshotRequest userMetadata(Map userMetadata) { + this.userMetadata = userMetadata; + return this; + } + /** * Parses snapshot definition. * @@ -405,6 +445,11 @@ public CreateSnapshotRequest source(Map source) { settings((Map) entry.getValue()); } else if (name.equals("include_global_state")) { includeGlobalState = nodeBooleanValue(entry.getValue(), "include_global_state"); + } else if (name.equals("metadata")) { + if (entry.getValue() != null && (entry.getValue() instanceof Map == false)) { + throw new IllegalArgumentException("malformed metadata, should be an object"); + } + userMetadata((Map) entry.getValue()); } } indicesOptions(IndicesOptions.fromMap(source, indicesOptions)); @@ -433,6 +478,7 @@ public XContentBuilder toXContent(XContentBuilder builder, Params params) throws if (indicesOptions != null) { indicesOptions.toXContent(builder, params); } + builder.field("metadata", userMetadata); builder.endObject(); return builder; } @@ -460,12 +506,14 @@ public boolean equals(Object o) { Arrays.equals(indices, that.indices) && Objects.equals(indicesOptions, that.indicesOptions) && Objects.equals(settings, that.settings) && - Objects.equals(masterNodeTimeout, that.masterNodeTimeout); + Objects.equals(masterNodeTimeout, that.masterNodeTimeout) && + Objects.equals(userMetadata, that.userMetadata); } @Override public int hashCode() { - int result = Objects.hash(snapshot, repository, indicesOptions, partial, settings, includeGlobalState, waitForCompletion); + int result = Objects.hash(snapshot, repository, indicesOptions, partial, settings, includeGlobalState, + waitForCompletion, userMetadata); result = 31 * result + Arrays.hashCode(indices); return result; } @@ -482,6 +530,7 @@ public String toString() { ", includeGlobalState=" + includeGlobalState + ", waitForCompletion=" + waitForCompletion + ", masterNodeTimeout=" + masterNodeTimeout + + ", metadata=" + userMetadata + '}'; } } diff --git a/server/src/main/java/org/elasticsearch/action/admin/cluster/snapshots/create/TransportCreateSnapshotAction.java b/server/src/main/java/org/elasticsearch/action/admin/cluster/snapshots/create/TransportCreateSnapshotAction.java index 73f9a0742a719..a3de0d4f0a975 100644 --- a/server/src/main/java/org/elasticsearch/action/admin/cluster/snapshots/create/TransportCreateSnapshotAction.java +++ b/server/src/main/java/org/elasticsearch/action/admin/cluster/snapshots/create/TransportCreateSnapshotAction.java @@ -29,6 +29,7 @@ import org.elasticsearch.cluster.service.ClusterService; import org.elasticsearch.common.inject.Inject; import org.elasticsearch.snapshots.SnapshotsService; +import org.elasticsearch.tasks.Task; import org.elasticsearch.threadpool.ThreadPool; import org.elasticsearch.transport.TransportService; @@ -70,8 +71,8 @@ protected ClusterBlockException checkBlock(CreateSnapshotRequest request, Cluste } @Override - protected void masterOperation(final CreateSnapshotRequest request, ClusterState state, - final ActionListener listener) { + protected void masterOperation(Task task, final CreateSnapshotRequest request, ClusterState state, + final ActionListener listener) { if (request.waitForCompletion()) { snapshotsService.executeSnapshot(request, ActionListener.map(listener, CreateSnapshotResponse::new)); } else { diff --git a/server/src/main/java/org/elasticsearch/action/admin/cluster/snapshots/delete/DeleteSnapshotAction.java b/server/src/main/java/org/elasticsearch/action/admin/cluster/snapshots/delete/DeleteSnapshotAction.java index 64f4e60fb2d0e..97a3405e9bdf6 100644 --- a/server/src/main/java/org/elasticsearch/action/admin/cluster/snapshots/delete/DeleteSnapshotAction.java +++ b/server/src/main/java/org/elasticsearch/action/admin/cluster/snapshots/delete/DeleteSnapshotAction.java @@ -19,13 +19,14 @@ package org.elasticsearch.action.admin.cluster.snapshots.delete; -import org.elasticsearch.action.Action; +import org.elasticsearch.action.ActionType; import org.elasticsearch.action.support.master.AcknowledgedResponse; +import org.elasticsearch.common.io.stream.Writeable; /** * Delete snapshot action */ -public class DeleteSnapshotAction extends Action { +public class DeleteSnapshotAction extends ActionType { public static final DeleteSnapshotAction INSTANCE = new DeleteSnapshotAction(); public static final String NAME = "cluster:admin/snapshot/delete"; @@ -35,8 +36,8 @@ private DeleteSnapshotAction() { } @Override - public AcknowledgedResponse newResponse() { - return new AcknowledgedResponse(); + public Writeable.Reader getResponseReader() { + return AcknowledgedResponse::new; } } diff --git a/server/src/main/java/org/elasticsearch/action/admin/cluster/snapshots/delete/TransportDeleteSnapshotAction.java b/server/src/main/java/org/elasticsearch/action/admin/cluster/snapshots/delete/TransportDeleteSnapshotAction.java index 140d480242dc0..3f980225f434c 100644 --- a/server/src/main/java/org/elasticsearch/action/admin/cluster/snapshots/delete/TransportDeleteSnapshotAction.java +++ b/server/src/main/java/org/elasticsearch/action/admin/cluster/snapshots/delete/TransportDeleteSnapshotAction.java @@ -29,10 +29,14 @@ import org.elasticsearch.cluster.metadata.IndexNameExpressionResolver; import org.elasticsearch.cluster.service.ClusterService; import org.elasticsearch.common.inject.Inject; +import org.elasticsearch.common.io.stream.StreamInput; import org.elasticsearch.snapshots.SnapshotsService; +import org.elasticsearch.tasks.Task; import org.elasticsearch.threadpool.ThreadPool; import org.elasticsearch.transport.TransportService; +import java.io.IOException; + /** * Transport action for delete snapshot operation */ @@ -53,9 +57,14 @@ protected String executor() { return ThreadPool.Names.GENERIC; } + @Override + protected AcknowledgedResponse read(StreamInput in) throws IOException { + return new AcknowledgedResponse(in); + } + @Override protected AcknowledgedResponse newResponse() { - return new AcknowledgedResponse(); + throw new UnsupportedOperationException("usage of Streamable is to be replaced by Writeable"); } @Override @@ -65,7 +74,7 @@ protected ClusterBlockException checkBlock(DeleteSnapshotRequest request, Cluste } @Override - protected void masterOperation(final DeleteSnapshotRequest request, ClusterState state, + protected void masterOperation(Task task, final DeleteSnapshotRequest request, ClusterState state, final ActionListener listener) { snapshotsService.deleteSnapshot(request.repository(), request.snapshot(), ActionListener.map(listener, v -> new AcknowledgedResponse(true)), false); diff --git a/server/src/main/java/org/elasticsearch/action/admin/cluster/snapshots/get/GetSnapshotsAction.java b/server/src/main/java/org/elasticsearch/action/admin/cluster/snapshots/get/GetSnapshotsAction.java index b5015ff5c23b0..ab0823b98f71c 100644 --- a/server/src/main/java/org/elasticsearch/action/admin/cluster/snapshots/get/GetSnapshotsAction.java +++ b/server/src/main/java/org/elasticsearch/action/admin/cluster/snapshots/get/GetSnapshotsAction.java @@ -19,12 +19,13 @@ package org.elasticsearch.action.admin.cluster.snapshots.get; -import org.elasticsearch.action.Action; +import org.elasticsearch.action.ActionType; +import org.elasticsearch.common.io.stream.Writeable; /** * Get snapshots action */ -public class GetSnapshotsAction extends Action { +public class GetSnapshotsAction extends ActionType { public static final GetSnapshotsAction INSTANCE = new GetSnapshotsAction(); public static final String NAME = "cluster:admin/snapshot/get"; @@ -34,8 +35,8 @@ private GetSnapshotsAction() { } @Override - public GetSnapshotsResponse newResponse() { - return new GetSnapshotsResponse(); + public Writeable.Reader getResponseReader() { + return GetSnapshotsResponse::new; } } diff --git a/server/src/main/java/org/elasticsearch/action/admin/cluster/snapshots/get/GetSnapshotsRequest.java b/server/src/main/java/org/elasticsearch/action/admin/cluster/snapshots/get/GetSnapshotsRequest.java index 41ae57031d320..4bfd656700684 100644 --- a/server/src/main/java/org/elasticsearch/action/admin/cluster/snapshots/get/GetSnapshotsRequest.java +++ b/server/src/main/java/org/elasticsearch/action/admin/cluster/snapshots/get/GetSnapshotsRequest.java @@ -19,6 +19,7 @@ package org.elasticsearch.action.admin.cluster.snapshots.get; +import org.elasticsearch.Version; import org.elasticsearch.action.ActionRequestValidationException; import org.elasticsearch.action.support.master.MasterNodeRequest; import org.elasticsearch.common.Strings; @@ -37,8 +38,9 @@ public class GetSnapshotsRequest extends MasterNodeRequest public static final String ALL_SNAPSHOTS = "_all"; public static final String CURRENT_SNAPSHOT = "_current"; public static final boolean DEFAULT_VERBOSE_MODE = true; + public static final Version MULTIPLE_REPOSITORIES_SUPPORT_ADDED = Version.V_8_0_0; - private String repository; + private String[] repositories; private String[] snapshots = Strings.EMPTY_ARRAY; @@ -50,28 +52,32 @@ public GetSnapshotsRequest() { } /** - * Constructs a new get snapshots request with given repository name and list of snapshots + * Constructs a new get snapshots request with given repository names and list of snapshots * - * @param repository repository name + * @param repositories repository names * @param snapshots list of snapshots */ - public GetSnapshotsRequest(String repository, String[] snapshots) { - this.repository = repository; + public GetSnapshotsRequest(String[] repositories, String[] snapshots) { + this.repositories = repositories; this.snapshots = snapshots; } /** - * Constructs a new get snapshots request with given repository name + * Constructs a new get snapshots request with given repository names * - * @param repository repository name + * @param repositories repository names */ - public GetSnapshotsRequest(String repository) { - this.repository = repository; + public GetSnapshotsRequest(String... repositories) { + this.repositories = repositories; } public GetSnapshotsRequest(StreamInput in) throws IOException { super(in); - repository = in.readString(); + if (in.getVersion().onOrAfter(MULTIPLE_REPOSITORIES_SUPPORT_ADDED)) { + repositories = in.readStringArray(); + } else { + repositories = new String[]{in.readString()}; + } snapshots = in.readStringArray(); ignoreUnavailable = in.readBoolean(); verbose = in.readBoolean(); @@ -80,7 +86,15 @@ public GetSnapshotsRequest(StreamInput in) throws IOException { @Override public void writeTo(StreamOutput out) throws IOException { super.writeTo(out); - out.writeString(repository); + if (out.getVersion().onOrAfter(MULTIPLE_REPOSITORIES_SUPPORT_ADDED)) { + out.writeStringArray(repositories); + } else { + if (repositories.length != 1) { + throw new IllegalArgumentException("Requesting snapshots from multiple repositories is not supported in versions prior " + + "to " + MULTIPLE_REPOSITORIES_SUPPORT_ADDED.toString()); + } + out.writeString(repositories[0]); + } out.writeStringArray(snapshots); out.writeBoolean(ignoreUnavailable); out.writeBoolean(verbose); @@ -89,30 +103,30 @@ public void writeTo(StreamOutput out) throws IOException { @Override public ActionRequestValidationException validate() { ActionRequestValidationException validationException = null; - if (repository == null) { - validationException = addValidationError("repository is missing", validationException); + if (repositories == null || repositories.length == 0) { + validationException = addValidationError("repositories are missing", validationException); } return validationException; } /** - * Sets repository name + * Sets repository names * - * @param repository repository name + * @param repositories repository names * @return this request */ - public GetSnapshotsRequest repository(String repository) { - this.repository = repository; + public GetSnapshotsRequest repositories(String... repositories) { + this.repositories = repositories; return this; } /** - * Returns repository name + * Returns repository names * - * @return repository name + * @return repository names */ - public String repository() { - return this.repository; + public String[] repositories() { + return this.repositories; } /** @@ -176,4 +190,4 @@ public boolean verbose() { public void readFrom(StreamInput in) throws IOException { throw new UnsupportedOperationException("usage of Streamable is to be replaced by Writeable"); } -} +} \ No newline at end of file diff --git a/server/src/main/java/org/elasticsearch/action/admin/cluster/snapshots/get/GetSnapshotsRequestBuilder.java b/server/src/main/java/org/elasticsearch/action/admin/cluster/snapshots/get/GetSnapshotsRequestBuilder.java index 052f8da0c7508..e4219c858551a 100644 --- a/server/src/main/java/org/elasticsearch/action/admin/cluster/snapshots/get/GetSnapshotsRequestBuilder.java +++ b/server/src/main/java/org/elasticsearch/action/admin/cluster/snapshots/get/GetSnapshotsRequestBuilder.java @@ -30,27 +30,20 @@ public class GetSnapshotsRequestBuilder extends MasterNodeOperationRequestBuilde GetSnapshotsResponse, GetSnapshotsRequestBuilder> { /** - * Constructs the new get snapshot request + * Constructs the new get snapshot request with specified repositories */ - public GetSnapshotsRequestBuilder(ElasticsearchClient client, GetSnapshotsAction action) { - super(client, action, new GetSnapshotsRequest()); + public GetSnapshotsRequestBuilder(ElasticsearchClient client, GetSnapshotsAction action, String... repositories) { + super(client, action, new GetSnapshotsRequest(repositories)); } /** - * Constructs the new get snapshot request with specified repository - */ - public GetSnapshotsRequestBuilder(ElasticsearchClient client, GetSnapshotsAction action, String repository) { - super(client, action, new GetSnapshotsRequest(repository)); - } - - /** - * Sets the repository name + * Sets the repository names * - * @param repository repository name + * @param repositories repository names * @return this builder */ - public GetSnapshotsRequestBuilder setRepository(String repository) { - request.repository(repository); + public GetSnapshotsRequestBuilder setRepositories(String... repositories) { + request.repositories(repositories); return this; } diff --git a/server/src/main/java/org/elasticsearch/action/admin/cluster/snapshots/get/GetSnapshotsResponse.java b/server/src/main/java/org/elasticsearch/action/admin/cluster/snapshots/get/GetSnapshotsResponse.java index 6f757cb60ca86..1ecbd222c6d84 100644 --- a/server/src/main/java/org/elasticsearch/action/admin/cluster/snapshots/get/GetSnapshotsResponse.java +++ b/server/src/main/java/org/elasticsearch/action/admin/cluster/snapshots/get/GetSnapshotsResponse.java @@ -19,12 +19,14 @@ package org.elasticsearch.action.admin.cluster.snapshots.get; +import org.elasticsearch.ElasticsearchException; import org.elasticsearch.action.ActionResponse; import org.elasticsearch.common.ParseField; +import org.elasticsearch.common.Strings; import org.elasticsearch.common.io.stream.StreamInput; import org.elasticsearch.common.io.stream.StreamOutput; +import org.elasticsearch.common.util.set.Sets; import org.elasticsearch.common.xcontent.ConstructingObjectParser; -import org.elasticsearch.common.xcontent.ToXContent; import org.elasticsearch.common.xcontent.ToXContentObject; import org.elasticsearch.common.xcontent.XContentBuilder; import org.elasticsearch.common.xcontent.XContentParser; @@ -32,89 +34,241 @@ import java.io.IOException; import java.util.ArrayList; +import java.util.Collection; import java.util.Collections; +import java.util.HashMap; import java.util.List; -import java.util.Objects; +import java.util.Map; +import java.util.Set; /** * Get snapshots response */ public class GetSnapshotsResponse extends ActionResponse implements ToXContentObject { - @SuppressWarnings("unchecked") - private static final ConstructingObjectParser GET_SNAPSHOT_PARSER = - new ConstructingObjectParser<>(GetSnapshotsResponse.class.getName(), true, - (args) -> new GetSnapshotsResponse((List) args[0])); + private static final ConstructingObjectParser PARSER = + new ConstructingObjectParser<>(GetSnapshotsResponse.class.getName(), true, + (args) -> new GetSnapshotsResponse((List) args[0])); static { - GET_SNAPSHOT_PARSER.declareObjectArray(ConstructingObjectParser.constructorArg(), - (p, c) -> SnapshotInfo.SNAPSHOT_INFO_PARSER.apply(p, c).build(), new ParseField("snapshots")); + PARSER.declareObjectArray(ConstructingObjectParser.constructorArg(), + (p, c) -> Response.fromXContent(p), new ParseField("responses")); } - private List snapshots = Collections.emptyList(); + public GetSnapshotsResponse(StreamInput in) throws IOException { + if (in.getVersion().onOrAfter(GetSnapshotsRequest.MULTIPLE_REPOSITORIES_SUPPORT_ADDED)) { + int successfulSize = in.readVInt(); + Map> successfulResponses = new HashMap<>(successfulSize); + for (int i = 0; i < successfulSize; i++) { + String repository = in.readString(); + int size = in.readVInt(); + List snapshotInfos = new ArrayList<>(size); + for (int j = 0; j < size; j++) { + snapshotInfos.add(new SnapshotInfo(in)); + } + successfulResponses.put(repository, snapshotInfos); + } - GetSnapshotsResponse() { + int failedSize = in.readVInt(); + Map failedResponses = new HashMap<>(failedSize); + for (int i = 0; i < failedSize; i++) { + String repository = in.readString(); + ElasticsearchException error = in.readException(); + failedResponses.put(repository, error); + } + this.successfulResponses = Collections.unmodifiableMap(successfulResponses); + this.failedResponses = Collections.unmodifiableMap(failedResponses); + } else { + int size = in.readVInt(); + List snapshots = new ArrayList<>(size); + for (int i = 0; i < size; i++) { + snapshots.add(new SnapshotInfo(in)); + } + this.successfulResponses = Collections.singletonMap("unknown", snapshots); + this.failedResponses = Collections.emptyMap(); + } } - GetSnapshotsResponse(List snapshots) { - this.snapshots = Collections.unmodifiableList(snapshots); + + public static class Response { + private String repository; + private List snapshots; + private ElasticsearchException error; + + private static final ConstructingObjectParser RESPONSE_PARSER = + new ConstructingObjectParser<>(Response.class.getName(), true, + (args) -> new Response((String) args[0], + (List) args[1], (ElasticsearchException) args[2])); + + static { + RESPONSE_PARSER.declareString(ConstructingObjectParser.constructorArg(), new ParseField("repository")); + RESPONSE_PARSER.declareObjectArray(ConstructingObjectParser.optionalConstructorArg(), + (p, c) -> SnapshotInfo.SNAPSHOT_INFO_PARSER.apply(p, c).build(), new ParseField("snapshots")); + RESPONSE_PARSER.declareObject(ConstructingObjectParser.optionalConstructorArg(), + (p, c) -> ElasticsearchException.fromXContent(p), new ParseField("error")); + } + + private Response(String repository, List snapshots, ElasticsearchException error) { + this.repository = repository; + this.snapshots = snapshots; + this.error = error; + } + + public static Response snapshots(String repository, List snapshots) { + return new Response(repository, snapshots, null); + } + + public static Response error(String repository, ElasticsearchException error) { + return new Response(repository, null, error); + } + + private static Response fromXContent(XContentParser parser) throws IOException { + return RESPONSE_PARSER.parse(parser, null); + } + } + + private final Map> successfulResponses; + private final Map failedResponses; + + public GetSnapshotsResponse(Collection responses) { + Map> successfulResponses = new HashMap<>(); + Map failedResponses = new HashMap<>(); + for (Response response : responses) { + if (response.snapshots != null) { + assert response.error == null; + successfulResponses.put(response.repository, response.snapshots); + } else { + assert response.snapshots == null; + failedResponses.put(response.repository, response.error); + } + } + this.successfulResponses = Collections.unmodifiableMap(successfulResponses); + this.failedResponses = Collections.unmodifiableMap(failedResponses); } /** - * Returns the list of snapshots - * - * @return the list of snapshots + * Returns list of snapshots for the specified repository. + * @param repo - repository name. + * @return list of snapshots. + * @throws IllegalArgumentException if there is no such repository in the response. + * @throws ElasticsearchException if an exception occurred when retrieving snapshots from the repository. */ - public List getSnapshots() { - return snapshots; + public List getSnapshots(String repo) { + List snapshots = successfulResponses.get(repo); + if (snapshots != null) { + return snapshots; + } + ElasticsearchException error = failedResponses.get(repo); + if (error == null) { + throw new IllegalArgumentException("No such repository"); + } + throw error; } - @Override - public void readFrom(StreamInput in) throws IOException { - super.readFrom(in); - int size = in.readVInt(); - List builder = new ArrayList<>(size); - for (int i = 0; i < size; i++) { - builder.add(new SnapshotInfo(in)); - } - snapshots = Collections.unmodifiableList(builder); + /** + * Returns list of repositories for both successful and unsuccessful responses. + */ + public Set getRepositories() { + return Sets.union(successfulResponses.keySet(), failedResponses.keySet()); } - @Override - public void writeTo(StreamOutput out) throws IOException { - super.writeTo(out); - out.writeVInt(snapshots.size()); - for (SnapshotInfo snapshotInfo : snapshots) { - snapshotInfo.writeTo(out); - } + /** + * Returns a map of repository name to the list of {@link SnapshotInfo} for each successful response. + */ + public Map> getSuccessfulResponses() { + return successfulResponses; + } + + /** + * Returns a map of repository name to {@link ElasticsearchException} for each unsuccessful response. + */ + public Map getFailedResponses() { + return failedResponses; + } + + /** + * Returns true if there is a least one failed response. + */ + public boolean isFailed() { + return failedResponses.isEmpty() == false; } @Override - public XContentBuilder toXContent(XContentBuilder builder, ToXContent.Params params) throws IOException { + public XContentBuilder toXContent(XContentBuilder builder, Params params) throws IOException { builder.startObject(); - builder.startArray("snapshots"); - for (SnapshotInfo snapshotInfo : snapshots) { - snapshotInfo.toXContent(builder, params); + builder.startArray("responses"); + + for (Map.Entry> snapshots : successfulResponses.entrySet()) { + builder.startObject(); + builder.field("repository", snapshots.getKey()); + builder.startArray("snapshots"); + for (SnapshotInfo snapshot : snapshots.getValue()) { + snapshot.toXContent(builder, params); + } + builder.endArray(); + builder.endObject(); + } + + for (Map.Entry error : failedResponses.entrySet()) { + builder.startObject(); + builder.field("repository", error.getKey()); + ElasticsearchException.generateFailureXContent(builder, params, error.getValue(), true); + builder.endObject(); } + builder.endArray(); builder.endObject(); return builder; } - public static GetSnapshotsResponse fromXContent(XContentParser parser) throws IOException { - return GET_SNAPSHOT_PARSER.parse(parser, null); + @Override + public void writeTo(StreamOutput out) throws IOException { + super.writeTo(out); + if (out.getVersion().onOrAfter(GetSnapshotsRequest.MULTIPLE_REPOSITORIES_SUPPORT_ADDED)) { + out.writeVInt(successfulResponses.size()); + for (Map.Entry> snapshots : successfulResponses.entrySet()) { + out.writeString(snapshots.getKey()); + out.writeVInt(snapshots.getValue().size()); + for (SnapshotInfo snapshotInfo : snapshots.getValue()) { + snapshotInfo.writeTo(out); + } + } + out.writeVInt(failedResponses.size()); + for (Map.Entry error : failedResponses.entrySet()) { + out.writeString(error.getKey()); + out.writeException(error.getValue()); + } + } else { + if (successfulResponses.size() + failedResponses.size() != 1) { + throw new IllegalArgumentException("Requesting snapshots from multiple repositories is not supported in versions prior " + + "to " + GetSnapshotsRequest.MULTIPLE_REPOSITORIES_SUPPORT_ADDED.toString()); + } + + if (successfulResponses.size() == 1) { + List snapshotInfos = successfulResponses.values().iterator().next(); + out.writeVInt(snapshotInfos.size()); + for (SnapshotInfo snapshotInfo : snapshotInfos) { + snapshotInfo.writeTo(out); + } + } + + if (failedResponses.isEmpty() == false) { + throw failedResponses.values().iterator().next(); + } + } } @Override - public boolean equals(Object o) { - if (this == o) return true; - if (o == null || getClass() != o.getClass()) return false; - GetSnapshotsResponse that = (GetSnapshotsResponse) o; - return Objects.equals(snapshots, that.snapshots); + public void readFrom(StreamInput in) throws IOException { + throw new UnsupportedOperationException("usage of Streamable is to be replaced by Writeable"); + } + + public static GetSnapshotsResponse fromXContent(XContentParser parser) throws IOException { + return PARSER.parse(parser, null); } @Override - public int hashCode() { - return Objects.hash(snapshots); + public String toString() { + return Strings.toString(this); } -} +} \ No newline at end of file diff --git a/server/src/main/java/org/elasticsearch/action/admin/cluster/snapshots/get/TransportGetSnapshotsAction.java b/server/src/main/java/org/elasticsearch/action/admin/cluster/snapshots/get/TransportGetSnapshotsAction.java index 23ffbd0dd1e3c..96fe976ea0fbb 100644 --- a/server/src/main/java/org/elasticsearch/action/admin/cluster/snapshots/get/TransportGetSnapshotsAction.java +++ b/server/src/main/java/org/elasticsearch/action/admin/cluster/snapshots/get/TransportGetSnapshotsAction.java @@ -20,15 +20,24 @@ package org.elasticsearch.action.admin.cluster.snapshots.get; import org.apache.lucene.util.CollectionUtil; +import org.elasticsearch.ElasticsearchException; import org.elasticsearch.action.ActionListener; +import org.elasticsearch.action.ActionListenerResponseHandler; +import org.elasticsearch.action.ActionRunnable; +import org.elasticsearch.action.admin.cluster.repositories.get.GetRepositoriesAction; +import org.elasticsearch.action.admin.cluster.repositories.get.GetRepositoriesRequest; +import org.elasticsearch.action.admin.cluster.repositories.get.GetRepositoriesResponse; import org.elasticsearch.action.support.ActionFilters; +import org.elasticsearch.action.support.GroupedActionListener; import org.elasticsearch.action.support.master.TransportMasterNodeAction; import org.elasticsearch.cluster.ClusterState; import org.elasticsearch.cluster.block.ClusterBlockException; import org.elasticsearch.cluster.block.ClusterBlockLevel; import org.elasticsearch.cluster.metadata.IndexNameExpressionResolver; +import org.elasticsearch.cluster.metadata.RepositoryMetaData; import org.elasticsearch.cluster.service.ClusterService; import org.elasticsearch.common.inject.Inject; +import org.elasticsearch.common.io.stream.StreamInput; import org.elasticsearch.common.regex.Regex; import org.elasticsearch.repositories.IndexId; import org.elasticsearch.repositories.RepositoryData; @@ -36,9 +45,12 @@ import org.elasticsearch.snapshots.SnapshotInfo; import org.elasticsearch.snapshots.SnapshotMissingException; import org.elasticsearch.snapshots.SnapshotsService; +import org.elasticsearch.tasks.Task; import org.elasticsearch.threadpool.ThreadPool; +import org.elasticsearch.transport.TransportRequestOptions; import org.elasticsearch.transport.TransportService; +import java.io.IOException; import java.util.ArrayList; import java.util.Collections; import java.util.HashMap; @@ -70,7 +82,12 @@ protected String executor() { @Override protected GetSnapshotsResponse newResponse() { - return new GetSnapshotsResponse(); + throw new UnsupportedOperationException("usage of Streamable is to be replaced by Writeable"); + } + + @Override + protected GetSnapshotsResponse read(StreamInput in) throws IOException { + return new GetSnapshotsResponse(in); } @Override @@ -79,75 +96,116 @@ protected ClusterBlockException checkBlock(GetSnapshotsRequest request, ClusterS } @Override - protected void masterOperation(final GetSnapshotsRequest request, final ClusterState state, + protected void masterOperation(Task task, final GetSnapshotsRequest request, final ClusterState state, final ActionListener listener) { - try { - final String repository = request.repository(); - final Map allSnapshotIds = new HashMap<>(); - final List currentSnapshots = new ArrayList<>(); - for (SnapshotInfo snapshotInfo : snapshotsService.currentSnapshots(repository)) { - SnapshotId snapshotId = snapshotInfo.snapshotId(); - allSnapshotIds.put(snapshotId.getName(), snapshotId); - currentSnapshots.add(snapshotInfo); - } + final String[] repositories = request.repositories(); + transportService.sendChildRequest(transportService.getLocalNode(), GetRepositoriesAction.NAME, + new GetRepositoriesRequest(repositories), task, TransportRequestOptions.EMPTY, + new ActionListenerResponseHandler<>( + ActionListener.wrap( + response -> + // switch to GENERIC thread pool because it might be long running operation + threadPool.executor(ThreadPool.Names.GENERIC).execute( + () -> getMultipleReposSnapshotInfo(response.repositories(), request.snapshots(), + request.ignoreUnavailable(), request.verbose(), listener)), + listener::onFailure), + GetRepositoriesResponse::new)); + } - final RepositoryData repositoryData; - if (isCurrentSnapshotsOnly(request.snapshots()) == false) { - repositoryData = snapshotsService.getRepositoryData(repository); - for (SnapshotId snapshotId : repositoryData.getAllSnapshotIds()) { - allSnapshotIds.put(snapshotId.getName(), snapshotId); + private void getMultipleReposSnapshotInfo(List repos, String[] snapshots, boolean ignoreUnavailable, + boolean verbose, ActionListener listener) { + // short-circuit if there are no repos, because we can not create GroupedActionListener of size 0 + if (repos.isEmpty()) { + listener.onResponse(new GetSnapshotsResponse(Collections.emptyList())); + return; + } + final GroupedActionListener groupedActionListener = + new GroupedActionListener<>( + ActionListener.map(listener, responses -> { + assert repos.size() == responses.size(); + return new GetSnapshotsResponse(responses); + }), repos.size()); + + // run concurrently for all repos on GENERIC thread pool + for (final RepositoryMetaData repo : repos) { + threadPool.executor(ThreadPool.Names.GENERIC).execute(new ActionRunnable<>(groupedActionListener) { + @Override + protected void doRun() { + try { + groupedActionListener.onResponse(GetSnapshotsResponse.Response.snapshots( + repo.name(), getSingleRepoSnapshotInfo(repo.name(), snapshots, ignoreUnavailable, verbose))); + } catch (ElasticsearchException e) { + groupedActionListener.onResponse(GetSnapshotsResponse.Response.error(repo.name(), e)); + } } - } else { - repositoryData = null; + }); + } + } + + private List getSingleRepoSnapshotInfo(String repo, String[] snapshots, boolean ignoreUnavailable, boolean verbose) { + final Map allSnapshotIds = new HashMap<>(); + final List currentSnapshots = new ArrayList<>(); + for (SnapshotInfo snapshotInfo : snapshotsService.currentSnapshots(repo)) { + SnapshotId snapshotId = snapshotInfo.snapshotId(); + allSnapshotIds.put(snapshotId.getName(), snapshotId); + currentSnapshots.add(snapshotInfo); + } + + final RepositoryData repositoryData; + if (isCurrentSnapshotsOnly(snapshots) == false) { + repositoryData = snapshotsService.getRepositoryData(repo); + for (SnapshotId snapshotId : repositoryData.getAllSnapshotIds()) { + allSnapshotIds.put(snapshotId.getName(), snapshotId); } + } else { + repositoryData = null; + } - final Set toResolve = new HashSet<>(); - if (isAllSnapshots(request.snapshots())) { - toResolve.addAll(allSnapshotIds.values()); - } else { - for (String snapshotOrPattern : request.snapshots()) { - if (GetSnapshotsRequest.CURRENT_SNAPSHOT.equalsIgnoreCase(snapshotOrPattern)) { - toResolve.addAll(currentSnapshots.stream().map(SnapshotInfo::snapshotId).collect(Collectors.toList())); - } else if (Regex.isSimpleMatchPattern(snapshotOrPattern) == false) { - if (allSnapshotIds.containsKey(snapshotOrPattern)) { - toResolve.add(allSnapshotIds.get(snapshotOrPattern)); - } else if (request.ignoreUnavailable() == false) { - throw new SnapshotMissingException(repository, snapshotOrPattern); - } - } else { - for (Map.Entry entry : allSnapshotIds.entrySet()) { - if (Regex.simpleMatch(snapshotOrPattern, entry.getKey())) { - toResolve.add(entry.getValue()); - } + final Set toResolve = new HashSet<>(); + if (isAllSnapshots(snapshots)) { + toResolve.addAll(allSnapshotIds.values()); + } else { + for (String snapshotOrPattern : snapshots) { + if (GetSnapshotsRequest.CURRENT_SNAPSHOT.equalsIgnoreCase(snapshotOrPattern)) { + toResolve.addAll(currentSnapshots.stream().map(SnapshotInfo::snapshotId).collect(Collectors.toList())); + } else if (Regex.isSimpleMatchPattern(snapshotOrPattern) == false) { + if (allSnapshotIds.containsKey(snapshotOrPattern)) { + toResolve.add(allSnapshotIds.get(snapshotOrPattern)); + } else if (ignoreUnavailable == false) { + throw new SnapshotMissingException(repo, snapshotOrPattern); + } + } else { + for (Map.Entry entry : allSnapshotIds.entrySet()) { + if (Regex.simpleMatch(snapshotOrPattern, entry.getKey())) { + toResolve.add(entry.getValue()); } } } + } - if (toResolve.isEmpty() && request.ignoreUnavailable() == false && isCurrentSnapshotsOnly(request.snapshots()) == false) { - throw new SnapshotMissingException(repository, request.snapshots()[0]); - } + if (toResolve.isEmpty() && ignoreUnavailable == false && isCurrentSnapshotsOnly(snapshots) == false) { + throw new SnapshotMissingException(repo, snapshots[0]); } + } - final List snapshotInfos; - if (request.verbose()) { - final Set incompatibleSnapshots = repositoryData != null ? + final List snapshotInfos; + if (verbose) { + final Set incompatibleSnapshots = repositoryData != null ? new HashSet<>(repositoryData.getIncompatibleSnapshotIds()) : Collections.emptySet(); - snapshotInfos = snapshotsService.snapshots(repository, new ArrayList<>(toResolve), - incompatibleSnapshots, request.ignoreUnavailable()); + snapshotInfos = snapshotsService.snapshots(repo, new ArrayList<>(toResolve), + incompatibleSnapshots, ignoreUnavailable); + } else { + if (repositoryData != null) { + // want non-current snapshots as well, which are found in the repository data + snapshotInfos = buildSimpleSnapshotInfos(toResolve, repositoryData, currentSnapshots); } else { - if (repositoryData != null) { - // want non-current snapshots as well, which are found in the repository data - snapshotInfos = buildSimpleSnapshotInfos(toResolve, repositoryData, currentSnapshots); - } else { - // only want current snapshots - snapshotInfos = currentSnapshots.stream().map(SnapshotInfo::basic).collect(Collectors.toList()); - CollectionUtil.timSort(snapshotInfos); - } + // only want current snapshots + snapshotInfos = currentSnapshots.stream().map(SnapshotInfo::basic).collect(Collectors.toList()); + CollectionUtil.timSort(snapshotInfos); } - listener.onResponse(new GetSnapshotsResponse(snapshotInfos)); - } catch (Exception e) { - listener.onFailure(e); } + + return snapshotInfos; } private boolean isAllSnapshots(String[] snapshots) { @@ -158,7 +216,7 @@ private boolean isCurrentSnapshotsOnly(String[] snapshots) { return (snapshots.length == 1 && GetSnapshotsRequest.CURRENT_SNAPSHOT.equalsIgnoreCase(snapshots[0])); } - private List buildSimpleSnapshotInfos(final Set toResolve, + private static List buildSimpleSnapshotInfos(final Set toResolve, final RepositoryData repositoryData, final List currentSnapshots) { List snapshotInfos = new ArrayList<>(); @@ -172,7 +230,7 @@ private List buildSimpleSnapshotInfos(final Set toReso for (SnapshotId snapshotId : repositoryData.getSnapshots(indexId)) { if (toResolve.contains(snapshotId)) { snapshotsToIndices.computeIfAbsent(snapshotId, (k) -> new ArrayList<>()) - .add(indexId.getName()); + .add(indexId.getName()); } } } diff --git a/server/src/main/java/org/elasticsearch/action/admin/cluster/snapshots/restore/RestoreSnapshotAction.java b/server/src/main/java/org/elasticsearch/action/admin/cluster/snapshots/restore/RestoreSnapshotAction.java index e633ce43e66cd..7f473d10a28ab 100644 --- a/server/src/main/java/org/elasticsearch/action/admin/cluster/snapshots/restore/RestoreSnapshotAction.java +++ b/server/src/main/java/org/elasticsearch/action/admin/cluster/snapshots/restore/RestoreSnapshotAction.java @@ -19,12 +19,12 @@ package org.elasticsearch.action.admin.cluster.snapshots.restore; -import org.elasticsearch.action.Action; +import org.elasticsearch.action.StreamableResponseActionType; /** * Restore snapshot action */ -public class RestoreSnapshotAction extends Action { +public class RestoreSnapshotAction extends StreamableResponseActionType { public static final RestoreSnapshotAction INSTANCE = new RestoreSnapshotAction(); public static final String NAME = "cluster:admin/snapshot/restore"; diff --git a/server/src/main/java/org/elasticsearch/action/admin/cluster/snapshots/restore/TransportRestoreSnapshotAction.java b/server/src/main/java/org/elasticsearch/action/admin/cluster/snapshots/restore/TransportRestoreSnapshotAction.java index 935902432d90f..5178a5224a058 100644 --- a/server/src/main/java/org/elasticsearch/action/admin/cluster/snapshots/restore/TransportRestoreSnapshotAction.java +++ b/server/src/main/java/org/elasticsearch/action/admin/cluster/snapshots/restore/TransportRestoreSnapshotAction.java @@ -29,6 +29,7 @@ import org.elasticsearch.cluster.service.ClusterService; import org.elasticsearch.common.inject.Inject; import org.elasticsearch.snapshots.RestoreService; +import org.elasticsearch.tasks.Task; import org.elasticsearch.threadpool.ThreadPool; import org.elasticsearch.transport.TransportService; @@ -72,7 +73,7 @@ protected ClusterBlockException checkBlock(RestoreSnapshotRequest request, Clust } @Override - protected void masterOperation(final RestoreSnapshotRequest request, final ClusterState state, + protected void masterOperation(Task task, final RestoreSnapshotRequest request, final ClusterState state, final ActionListener listener) { restoreService.restoreSnapshot(request, ActionListener.delegateFailure(listener, (delegatedListener, restoreCompletionResponse) -> { diff --git a/server/src/main/java/org/elasticsearch/action/admin/cluster/snapshots/status/SnapshotsStatusAction.java b/server/src/main/java/org/elasticsearch/action/admin/cluster/snapshots/status/SnapshotsStatusAction.java index ea28d26b40ffa..ec95e0e14f13f 100644 --- a/server/src/main/java/org/elasticsearch/action/admin/cluster/snapshots/status/SnapshotsStatusAction.java +++ b/server/src/main/java/org/elasticsearch/action/admin/cluster/snapshots/status/SnapshotsStatusAction.java @@ -19,12 +19,12 @@ package org.elasticsearch.action.admin.cluster.snapshots.status; -import org.elasticsearch.action.Action; +import org.elasticsearch.action.StreamableResponseActionType; /** * Snapshots status action */ -public class SnapshotsStatusAction extends Action { +public class SnapshotsStatusAction extends StreamableResponseActionType { public static final SnapshotsStatusAction INSTANCE = new SnapshotsStatusAction(); public static final String NAME = "cluster:admin/snapshot/status"; diff --git a/server/src/main/java/org/elasticsearch/action/admin/cluster/snapshots/status/TransportNodesSnapshotsStatus.java b/server/src/main/java/org/elasticsearch/action/admin/cluster/snapshots/status/TransportNodesSnapshotsStatus.java index 1f55c1e00cef6..74b9e8cb3c592 100644 --- a/server/src/main/java/org/elasticsearch/action/admin/cluster/snapshots/status/TransportNodesSnapshotsStatus.java +++ b/server/src/main/java/org/elasticsearch/action/admin/cluster/snapshots/status/TransportNodesSnapshotsStatus.java @@ -36,6 +36,7 @@ import org.elasticsearch.index.snapshots.IndexShardSnapshotStatus; import org.elasticsearch.snapshots.Snapshot; import org.elasticsearch.snapshots.SnapshotShardsService; +import org.elasticsearch.tasks.Task; import org.elasticsearch.threadpool.ThreadPool; import org.elasticsearch.transport.TransportService; @@ -48,7 +49,7 @@ import static java.util.Collections.unmodifiableMap; /** - * Transport client that collects snapshot shard statuses from data nodes + * Transport action that collects snapshot shard statuses from data nodes */ public class TransportNodesSnapshotsStatus extends TransportNodesAction> snapshotMapBuilder = new HashMap<>(); try { final String nodeId = clusterService.localNode().getId(); @@ -168,8 +169,7 @@ public static class NodeRequest extends BaseNodeRequest { public NodeRequest() { } - NodeRequest(String nodeId, TransportNodesSnapshotsStatus.Request request) { - super(nodeId); + NodeRequest(TransportNodesSnapshotsStatus.Request request) { snapshots = Arrays.asList(request.snapshots); } diff --git a/server/src/main/java/org/elasticsearch/action/admin/cluster/snapshots/status/TransportSnapshotsStatusAction.java b/server/src/main/java/org/elasticsearch/action/admin/cluster/snapshots/status/TransportSnapshotsStatusAction.java index 8430d1868c88d..234bc51b95b25 100644 --- a/server/src/main/java/org/elasticsearch/action/admin/cluster/snapshots/status/TransportSnapshotsStatusAction.java +++ b/server/src/main/java/org/elasticsearch/action/admin/cluster/snapshots/status/TransportSnapshotsStatusAction.java @@ -42,6 +42,7 @@ import org.elasticsearch.snapshots.SnapshotInfo; import org.elasticsearch.snapshots.SnapshotMissingException; import org.elasticsearch.snapshots.SnapshotsService; +import org.elasticsearch.tasks.Task; import org.elasticsearch.threadpool.ThreadPool; import org.elasticsearch.transport.TransportService; @@ -90,7 +91,7 @@ protected SnapshotsStatusResponse newResponse() { } @Override - protected void masterOperation(final SnapshotsStatusRequest request, + protected void masterOperation(Task task, final SnapshotsStatusRequest request, final ClusterState state, final ActionListener listener) throws Exception { List currentSnapshots = diff --git a/server/src/main/java/org/elasticsearch/action/admin/cluster/state/ClusterStateAction.java b/server/src/main/java/org/elasticsearch/action/admin/cluster/state/ClusterStateAction.java index f48df06d53c6f..b8a08444efccd 100644 --- a/server/src/main/java/org/elasticsearch/action/admin/cluster/state/ClusterStateAction.java +++ b/server/src/main/java/org/elasticsearch/action/admin/cluster/state/ClusterStateAction.java @@ -19,9 +19,9 @@ package org.elasticsearch.action.admin.cluster.state; -import org.elasticsearch.action.Action; +import org.elasticsearch.action.StreamableResponseActionType; -public class ClusterStateAction extends Action { +public class ClusterStateAction extends StreamableResponseActionType { public static final ClusterStateAction INSTANCE = new ClusterStateAction(); public static final String NAME = "cluster:monitor/state"; diff --git a/server/src/main/java/org/elasticsearch/action/admin/cluster/state/TransportClusterStateAction.java b/server/src/main/java/org/elasticsearch/action/admin/cluster/state/TransportClusterStateAction.java index cedca2d77e192..9669abf8e931a 100644 --- a/server/src/main/java/org/elasticsearch/action/admin/cluster/state/TransportClusterStateAction.java +++ b/server/src/main/java/org/elasticsearch/action/admin/cluster/state/TransportClusterStateAction.java @@ -38,6 +38,7 @@ import org.elasticsearch.common.inject.Inject; import org.elasticsearch.common.unit.TimeValue; import org.elasticsearch.node.NodeClosedException; +import org.elasticsearch.tasks.Task; import org.elasticsearch.threadpool.ThreadPool; import org.elasticsearch.transport.TransportService; @@ -77,7 +78,7 @@ protected ClusterStateResponse newResponse() { } @Override - protected void masterOperation(final ClusterStateRequest request, final ClusterState state, + protected void masterOperation(Task task, final ClusterStateRequest request, final ClusterState state, final ActionListener listener) throws IOException { final Predicate acceptableClusterStatePredicate diff --git a/server/src/main/java/org/elasticsearch/action/admin/cluster/stats/ClusterStatsAction.java b/server/src/main/java/org/elasticsearch/action/admin/cluster/stats/ClusterStatsAction.java index 049ce62d9df40..0652ab35e266a 100644 --- a/server/src/main/java/org/elasticsearch/action/admin/cluster/stats/ClusterStatsAction.java +++ b/server/src/main/java/org/elasticsearch/action/admin/cluster/stats/ClusterStatsAction.java @@ -19,9 +19,9 @@ package org.elasticsearch.action.admin.cluster.stats; -import org.elasticsearch.action.Action; +import org.elasticsearch.action.StreamableResponseActionType; -public class ClusterStatsAction extends Action { +public class ClusterStatsAction extends StreamableResponseActionType { public static final ClusterStatsAction INSTANCE = new ClusterStatsAction(); public static final String NAME = "cluster:monitor/stats"; diff --git a/server/src/main/java/org/elasticsearch/action/admin/cluster/stats/ClusterStatsNodes.java b/server/src/main/java/org/elasticsearch/action/admin/cluster/stats/ClusterStatsNodes.java index 49d82e574fb6b..bcadbc8e3292a 100644 --- a/server/src/main/java/org/elasticsearch/action/admin/cluster/stats/ClusterStatsNodes.java +++ b/server/src/main/java/org/elasticsearch/action/admin/cluster/stats/ClusterStatsNodes.java @@ -21,11 +21,11 @@ import com.carrotsearch.hppc.ObjectIntHashMap; import com.carrotsearch.hppc.cursors.ObjectIntCursor; - import org.elasticsearch.Version; import org.elasticsearch.action.admin.cluster.node.info.NodeInfo; import org.elasticsearch.action.admin.cluster.node.stats.NodeStats; import org.elasticsearch.cluster.node.DiscoveryNode; +import org.elasticsearch.cluster.node.DiscoveryNodeRole; import org.elasticsearch.common.Strings; import org.elasticsearch.common.collect.Tuple; import org.elasticsearch.common.network.NetworkModule; @@ -49,6 +49,7 @@ import java.util.List; import java.util.Map; import java.util.Set; +import java.util.TreeMap; import java.util.concurrent.atomic.AtomicInteger; public class ClusterStatsNodes implements ToXContentFragment { @@ -186,27 +187,27 @@ public static class Counts implements ToXContentFragment { private final int total; private final Map roles; - private Counts(List nodeInfos) { - this.roles = new HashMap<>(); - for (DiscoveryNode.Role role : DiscoveryNode.Role.values()) { - this.roles.put(role.getRoleName(), 0); + private Counts(final List nodeInfos) { + // TODO: do we need to report zeros? + final Map roles = new HashMap<>(DiscoveryNode.getPossibleRoleNames().size()); + roles.put(COORDINATING_ONLY, 0); + for (final String possibleRoleName : DiscoveryNode.getPossibleRoleNames()) { + roles.put(possibleRoleName, 0); } - this.roles.put(COORDINATING_ONLY, 0); int total = 0; - for (NodeInfo nodeInfo : nodeInfos) { + for (final NodeInfo nodeInfo : nodeInfos) { total++; if (nodeInfo.getNode().getRoles().isEmpty()) { - Integer count = roles.get(COORDINATING_ONLY); - roles.put(COORDINATING_ONLY, ++count); + roles.merge(COORDINATING_ONLY, 1, Integer::sum); } else { - for (DiscoveryNode.Role role : nodeInfo.getNode().getRoles()) { - Integer count = roles.get(role.getRoleName()); - roles.put(role.getRoleName(), ++count); + for (DiscoveryNodeRole role : nodeInfo.getNode().getRoles()) { + roles.merge(role.roleName(), 1, Integer::sum); } } } this.total = total; + this.roles = Map.copyOf(roles); } public int getTotal() { @@ -225,7 +226,7 @@ static final class Fields { public XContentBuilder toXContent(XContentBuilder builder, Params params) throws IOException { builder.field(Fields.TOTAL, total); - for (Map.Entry entry : roles.entrySet()) { + for (Map.Entry entry : new TreeMap<>(roles).entrySet()) { builder.field(entry.getKey(), entry.getValue()); } return builder; diff --git a/server/src/main/java/org/elasticsearch/action/admin/cluster/stats/TransportClusterStatsAction.java b/server/src/main/java/org/elasticsearch/action/admin/cluster/stats/TransportClusterStatsAction.java index 4cf81c24fbf1a..ef3bb6dfab24e 100644 --- a/server/src/main/java/org/elasticsearch/action/admin/cluster/stats/TransportClusterStatsAction.java +++ b/server/src/main/java/org/elasticsearch/action/admin/cluster/stats/TransportClusterStatsAction.java @@ -42,6 +42,7 @@ import org.elasticsearch.index.shard.IndexShard; import org.elasticsearch.indices.IndicesService; import org.elasticsearch.node.NodeService; +import org.elasticsearch.tasks.Task; import org.elasticsearch.threadpool.ThreadPool; import org.elasticsearch.transport.TransportService; @@ -81,8 +82,8 @@ protected ClusterStatsResponse newResponse(ClusterStatsRequest request, } @Override - protected ClusterStatsNodeRequest newNodeRequest(String nodeId, ClusterStatsRequest request) { - return new ClusterStatsNodeRequest(nodeId, request); + protected ClusterStatsNodeRequest newNodeRequest(ClusterStatsRequest request) { + return new ClusterStatsNodeRequest(request); } @Override @@ -91,7 +92,7 @@ protected ClusterStatsNodeResponse newNodeResponse() { } @Override - protected ClusterStatsNodeResponse nodeOperation(ClusterStatsNodeRequest nodeRequest) { + protected ClusterStatsNodeResponse nodeOperation(ClusterStatsNodeRequest nodeRequest, Task task) { NodeInfo nodeInfo = nodeService.info(true, true, false, true, false, true, false, true, false, false); NodeStats nodeStats = nodeService.stats(CommonStatsFlags.NONE, true, true, true, false, true, false, false, false, false, false, false, false); @@ -142,8 +143,7 @@ public static class ClusterStatsNodeRequest extends BaseNodeRequest { public ClusterStatsNodeRequest() { } - ClusterStatsNodeRequest(String nodeId, ClusterStatsRequest request) { - super(nodeId); + ClusterStatsNodeRequest(ClusterStatsRequest request) { this.request = request; } diff --git a/server/src/main/java/org/elasticsearch/action/admin/cluster/storedscripts/DeleteStoredScriptAction.java b/server/src/main/java/org/elasticsearch/action/admin/cluster/storedscripts/DeleteStoredScriptAction.java index 7ba8ee0754fed..35814f2221f76 100644 --- a/server/src/main/java/org/elasticsearch/action/admin/cluster/storedscripts/DeleteStoredScriptAction.java +++ b/server/src/main/java/org/elasticsearch/action/admin/cluster/storedscripts/DeleteStoredScriptAction.java @@ -19,10 +19,11 @@ package org.elasticsearch.action.admin.cluster.storedscripts; -import org.elasticsearch.action.Action; +import org.elasticsearch.action.ActionType; import org.elasticsearch.action.support.master.AcknowledgedResponse; +import org.elasticsearch.common.io.stream.Writeable; -public class DeleteStoredScriptAction extends Action { +public class DeleteStoredScriptAction extends ActionType { public static final DeleteStoredScriptAction INSTANCE = new DeleteStoredScriptAction(); public static final String NAME = "cluster:admin/script/delete"; @@ -32,7 +33,7 @@ private DeleteStoredScriptAction() { } @Override - public AcknowledgedResponse newResponse() { - return new AcknowledgedResponse(); + public Writeable.Reader getResponseReader() { + return AcknowledgedResponse::new; } } diff --git a/server/src/main/java/org/elasticsearch/action/admin/cluster/storedscripts/GetStoredScriptAction.java b/server/src/main/java/org/elasticsearch/action/admin/cluster/storedscripts/GetStoredScriptAction.java index e8015a4487496..ed74c5b928e0e 100644 --- a/server/src/main/java/org/elasticsearch/action/admin/cluster/storedscripts/GetStoredScriptAction.java +++ b/server/src/main/java/org/elasticsearch/action/admin/cluster/storedscripts/GetStoredScriptAction.java @@ -19,9 +19,9 @@ package org.elasticsearch.action.admin.cluster.storedscripts; -import org.elasticsearch.action.Action; +import org.elasticsearch.action.StreamableResponseActionType; -public class GetStoredScriptAction extends Action { +public class GetStoredScriptAction extends StreamableResponseActionType { public static final GetStoredScriptAction INSTANCE = new GetStoredScriptAction(); public static final String NAME = "cluster:admin/script/get"; diff --git a/server/src/main/java/org/elasticsearch/action/admin/cluster/storedscripts/PutStoredScriptAction.java b/server/src/main/java/org/elasticsearch/action/admin/cluster/storedscripts/PutStoredScriptAction.java index 8e5475283c2bc..6cbf57f3bd891 100644 --- a/server/src/main/java/org/elasticsearch/action/admin/cluster/storedscripts/PutStoredScriptAction.java +++ b/server/src/main/java/org/elasticsearch/action/admin/cluster/storedscripts/PutStoredScriptAction.java @@ -19,23 +19,22 @@ package org.elasticsearch.action.admin.cluster.storedscripts; -import org.elasticsearch.action.Action; +import org.elasticsearch.action.ActionType; import org.elasticsearch.action.support.master.AcknowledgedResponse; +import org.elasticsearch.common.io.stream.Writeable; -public class PutStoredScriptAction extends Action { +public class PutStoredScriptAction extends ActionType { public static final PutStoredScriptAction INSTANCE = new PutStoredScriptAction(); public static final String NAME = "cluster:admin/script/put"; - private PutStoredScriptAction() { super(NAME); } - @Override - public AcknowledgedResponse newResponse() { - return new AcknowledgedResponse(); + public Writeable.Reader getResponseReader() { + return AcknowledgedResponse::new; } } diff --git a/server/src/main/java/org/elasticsearch/action/admin/cluster/storedscripts/TransportDeleteStoredScriptAction.java b/server/src/main/java/org/elasticsearch/action/admin/cluster/storedscripts/TransportDeleteStoredScriptAction.java index 085850f118777..86fb16c63d4d6 100644 --- a/server/src/main/java/org/elasticsearch/action/admin/cluster/storedscripts/TransportDeleteStoredScriptAction.java +++ b/server/src/main/java/org/elasticsearch/action/admin/cluster/storedscripts/TransportDeleteStoredScriptAction.java @@ -29,10 +29,14 @@ import org.elasticsearch.cluster.metadata.IndexNameExpressionResolver; import org.elasticsearch.cluster.service.ClusterService; import org.elasticsearch.common.inject.Inject; +import org.elasticsearch.common.io.stream.StreamInput; import org.elasticsearch.script.ScriptService; +import org.elasticsearch.tasks.Task; import org.elasticsearch.threadpool.ThreadPool; import org.elasticsearch.transport.TransportService; +import java.io.IOException; + public class TransportDeleteStoredScriptAction extends TransportMasterNodeAction { private final ScriptService scriptService; @@ -51,13 +55,18 @@ protected String executor() { return ThreadPool.Names.SAME; } + @Override + protected AcknowledgedResponse read(StreamInput in) throws IOException { + return new AcknowledgedResponse(in); + } + @Override protected AcknowledgedResponse newResponse() { - return new AcknowledgedResponse(); + throw new UnsupportedOperationException("usage of Streamable is to be replaced by Writeable"); } @Override - protected void masterOperation(DeleteStoredScriptRequest request, ClusterState state, + protected void masterOperation(Task task, DeleteStoredScriptRequest request, ClusterState state, ActionListener listener) throws Exception { scriptService.deleteStoredScript(clusterService, request, listener); } diff --git a/server/src/main/java/org/elasticsearch/action/admin/cluster/storedscripts/TransportGetStoredScriptAction.java b/server/src/main/java/org/elasticsearch/action/admin/cluster/storedscripts/TransportGetStoredScriptAction.java index 19e86d1b6722d..1e67e025b61b9 100644 --- a/server/src/main/java/org/elasticsearch/action/admin/cluster/storedscripts/TransportGetStoredScriptAction.java +++ b/server/src/main/java/org/elasticsearch/action/admin/cluster/storedscripts/TransportGetStoredScriptAction.java @@ -29,6 +29,7 @@ import org.elasticsearch.cluster.service.ClusterService; import org.elasticsearch.common.inject.Inject; import org.elasticsearch.script.ScriptService; +import org.elasticsearch.tasks.Task; import org.elasticsearch.threadpool.ThreadPool; import org.elasticsearch.transport.TransportService; @@ -57,7 +58,7 @@ protected GetStoredScriptResponse newResponse() { } @Override - protected void masterOperation(GetStoredScriptRequest request, ClusterState state, + protected void masterOperation(Task task, GetStoredScriptRequest request, ClusterState state, ActionListener listener) throws Exception { listener.onResponse(new GetStoredScriptResponse(request.id(), scriptService.getStoredScript(state, request))); } diff --git a/server/src/main/java/org/elasticsearch/action/admin/cluster/storedscripts/TransportPutStoredScriptAction.java b/server/src/main/java/org/elasticsearch/action/admin/cluster/storedscripts/TransportPutStoredScriptAction.java index 99fa3979d87d0..ceabd31c45604 100644 --- a/server/src/main/java/org/elasticsearch/action/admin/cluster/storedscripts/TransportPutStoredScriptAction.java +++ b/server/src/main/java/org/elasticsearch/action/admin/cluster/storedscripts/TransportPutStoredScriptAction.java @@ -29,10 +29,14 @@ import org.elasticsearch.cluster.metadata.IndexNameExpressionResolver; import org.elasticsearch.cluster.service.ClusterService; import org.elasticsearch.common.inject.Inject; +import org.elasticsearch.common.io.stream.StreamInput; import org.elasticsearch.script.ScriptService; +import org.elasticsearch.tasks.Task; import org.elasticsearch.threadpool.ThreadPool; import org.elasticsearch.transport.TransportService; +import java.io.IOException; + public class TransportPutStoredScriptAction extends TransportMasterNodeAction { private final ScriptService scriptService; @@ -51,13 +55,18 @@ protected String executor() { return ThreadPool.Names.SAME; } + @Override + protected AcknowledgedResponse read(StreamInput in) throws IOException { + return new AcknowledgedResponse(in); + } + @Override protected AcknowledgedResponse newResponse() { - return new AcknowledgedResponse(); + throw new UnsupportedOperationException("usage of Streamable is to be replaced by Writeable"); } @Override - protected void masterOperation(PutStoredScriptRequest request, ClusterState state, + protected void masterOperation(Task task, PutStoredScriptRequest request, ClusterState state, ActionListener listener) throws Exception { scriptService.putStoredScript(clusterService, request, listener); } diff --git a/server/src/main/java/org/elasticsearch/action/admin/cluster/tasks/PendingClusterTasksAction.java b/server/src/main/java/org/elasticsearch/action/admin/cluster/tasks/PendingClusterTasksAction.java index 296c65146a03c..240575bb1da45 100644 --- a/server/src/main/java/org/elasticsearch/action/admin/cluster/tasks/PendingClusterTasksAction.java +++ b/server/src/main/java/org/elasticsearch/action/admin/cluster/tasks/PendingClusterTasksAction.java @@ -19,9 +19,9 @@ package org.elasticsearch.action.admin.cluster.tasks; -import org.elasticsearch.action.Action; +import org.elasticsearch.action.StreamableResponseActionType; -public class PendingClusterTasksAction extends Action { +public class PendingClusterTasksAction extends StreamableResponseActionType { public static final PendingClusterTasksAction INSTANCE = new PendingClusterTasksAction(); public static final String NAME = "cluster:monitor/task"; diff --git a/server/src/main/java/org/elasticsearch/action/admin/cluster/tasks/TransportPendingClusterTasksAction.java b/server/src/main/java/org/elasticsearch/action/admin/cluster/tasks/TransportPendingClusterTasksAction.java index cb061a25363cc..d62e7c13247bc 100644 --- a/server/src/main/java/org/elasticsearch/action/admin/cluster/tasks/TransportPendingClusterTasksAction.java +++ b/server/src/main/java/org/elasticsearch/action/admin/cluster/tasks/TransportPendingClusterTasksAction.java @@ -29,6 +29,7 @@ import org.elasticsearch.cluster.service.ClusterService; import org.elasticsearch.cluster.service.PendingClusterTask; import org.elasticsearch.common.inject.Inject; +import org.elasticsearch.tasks.Task; import org.elasticsearch.threadpool.ThreadPool; import org.elasticsearch.transport.TransportService; @@ -65,7 +66,7 @@ protected PendingClusterTasksResponse newResponse() { } @Override - protected void masterOperation(PendingClusterTasksRequest request, ClusterState state, + protected void masterOperation(Task task, PendingClusterTasksRequest request, ClusterState state, ActionListener listener) { logger.trace("fetching pending tasks from cluster service"); final List pendingTasks = clusterService.getMasterService().pendingTasks(); diff --git a/server/src/main/java/org/elasticsearch/action/admin/indices/alias/IndicesAliasesAction.java b/server/src/main/java/org/elasticsearch/action/admin/indices/alias/IndicesAliasesAction.java index 7f7743f81730a..fe18bd9315459 100644 --- a/server/src/main/java/org/elasticsearch/action/admin/indices/alias/IndicesAliasesAction.java +++ b/server/src/main/java/org/elasticsearch/action/admin/indices/alias/IndicesAliasesAction.java @@ -19,10 +19,11 @@ package org.elasticsearch.action.admin.indices.alias; -import org.elasticsearch.action.Action; +import org.elasticsearch.action.ActionType; import org.elasticsearch.action.support.master.AcknowledgedResponse; +import org.elasticsearch.common.io.stream.Writeable; -public class IndicesAliasesAction extends Action { +public class IndicesAliasesAction extends ActionType { public static final IndicesAliasesAction INSTANCE = new IndicesAliasesAction(); public static final String NAME = "indices:admin/aliases"; @@ -32,7 +33,7 @@ private IndicesAliasesAction() { } @Override - public AcknowledgedResponse newResponse() { - return new AcknowledgedResponse(); + public Writeable.Reader getResponseReader() { + return AcknowledgedResponse::new; } } diff --git a/server/src/main/java/org/elasticsearch/action/admin/indices/alias/IndicesAliasesRequest.java b/server/src/main/java/org/elasticsearch/action/admin/indices/alias/IndicesAliasesRequest.java index 20b01989285ad..4fe3094f33cef 100644 --- a/server/src/main/java/org/elasticsearch/action/admin/indices/alias/IndicesAliasesRequest.java +++ b/server/src/main/java/org/elasticsearch/action/admin/indices/alias/IndicesAliasesRequest.java @@ -20,6 +20,7 @@ package org.elasticsearch.action.admin.indices.alias; import org.elasticsearch.ElasticsearchGenerationException; +import org.elasticsearch.Version; import org.elasticsearch.action.ActionRequestValidationException; import org.elasticsearch.action.AliasesRequest; import org.elasticsearch.action.support.IndicesOptions; @@ -62,6 +63,7 @@ public class IndicesAliasesRequest extends AcknowledgedRequest implements ToXContentObject { private List allAliasActions = new ArrayList<>(); + private String origin = ""; // indices options that require every specified index to exist, expand wildcards only to open // indices, don't allow that no indices are resolved from wildcard expressions and resolve the @@ -526,6 +528,15 @@ public int hashCode() { } } + public String origin() { + return origin; + } + + public IndicesAliasesRequest origin(final String origin) { + this.origin = Objects.requireNonNull(origin); + return this; + } + /** * Add the action to this request and validate it. */ @@ -556,12 +567,23 @@ public ActionRequestValidationException validate() { public void readFrom(StreamInput in) throws IOException { super.readFrom(in); allAliasActions = in.readList(AliasActions::new); + if (in.getVersion().onOrAfter(Version.V_7_3_0)) { + origin = in.readOptionalString(); + } else { + origin = null; + } } @Override public void writeTo(StreamOutput out) throws IOException { super.writeTo(out); out.writeList(allAliasActions); + // noinspection StatementWithEmptyBody + if (out.getVersion().onOrAfter(Version.V_7_3_0)) { + out.writeOptionalString(origin); + } else { + // nothing to do here, here for symmetry with IndicesAliasesRequest#readFrom + } } public IndicesOptions indicesOptions() { diff --git a/server/src/main/java/org/elasticsearch/action/admin/indices/alias/TransportIndicesAliasesAction.java b/server/src/main/java/org/elasticsearch/action/admin/indices/alias/TransportIndicesAliasesAction.java index 252a91a0d5b37..e4f460d4aa7c3 100644 --- a/server/src/main/java/org/elasticsearch/action/admin/indices/alias/TransportIndicesAliasesAction.java +++ b/server/src/main/java/org/elasticsearch/action/admin/indices/alias/TransportIndicesAliasesAction.java @@ -21,6 +21,7 @@ import com.carrotsearch.hppc.cursors.ObjectCursor; import org.elasticsearch.action.ActionListener; +import org.elasticsearch.action.RequestValidators; import org.elasticsearch.action.admin.indices.alias.IndicesAliasesRequest.AliasActions; import org.elasticsearch.action.support.ActionFilters; import org.elasticsearch.action.support.master.AcknowledgedResponse; @@ -37,14 +38,20 @@ import org.elasticsearch.cluster.service.ClusterService; import org.elasticsearch.common.collect.ImmutableOpenMap; import org.elasticsearch.common.inject.Inject; +import org.elasticsearch.common.io.stream.StreamInput; +import org.elasticsearch.index.Index; import org.elasticsearch.rest.action.admin.indices.AliasesNotFoundException; +import org.elasticsearch.tasks.Task; import org.elasticsearch.threadpool.ThreadPool; import org.elasticsearch.transport.TransportService; +import java.io.IOException; import java.util.ArrayList; import java.util.Collections; import java.util.HashSet; import java.util.List; +import java.util.Objects; +import java.util.Optional; import java.util.Set; import static java.util.Collections.unmodifiableList; @@ -55,14 +62,21 @@ public class TransportIndicesAliasesAction extends TransportMasterNodeAction { private final MetaDataIndexAliasesService indexAliasesService; + private final RequestValidators requestValidators; @Inject - public TransportIndicesAliasesAction(TransportService transportService, ClusterService clusterService, - ThreadPool threadPool, MetaDataIndexAliasesService indexAliasesService, - ActionFilters actionFilters, IndexNameExpressionResolver indexNameExpressionResolver) { + public TransportIndicesAliasesAction( + final TransportService transportService, + final ClusterService clusterService, + final ThreadPool threadPool, + final MetaDataIndexAliasesService indexAliasesService, + final ActionFilters actionFilters, + final IndexNameExpressionResolver indexNameExpressionResolver, + final RequestValidators requestValidators) { super(IndicesAliasesAction.NAME, transportService, clusterService, threadPool, actionFilters, indexNameExpressionResolver, IndicesAliasesRequest::new); this.indexAliasesService = indexAliasesService; + this.requestValidators = Objects.requireNonNull(requestValidators); } @Override @@ -71,9 +85,14 @@ protected String executor() { return ThreadPool.Names.SAME; } + @Override + protected AcknowledgedResponse read(StreamInput in) throws IOException { + return new AcknowledgedResponse(in); + } + @Override protected AcknowledgedResponse newResponse() { - return new AcknowledgedResponse(); + throw new UnsupportedOperationException("usage of Streamable is to be replaced by Writeable"); } @Override @@ -86,7 +105,7 @@ protected ClusterBlockException checkBlock(IndicesAliasesRequest request, Cluste } @Override - protected void masterOperation(final IndicesAliasesRequest request, final ClusterState state, + protected void masterOperation(Task task, final IndicesAliasesRequest request, final ClusterState state, final ActionListener listener) { //Expand the indices names @@ -96,23 +115,28 @@ protected void masterOperation(final IndicesAliasesRequest request, final Cluste // Resolve all the AliasActions into AliasAction instances and gather all the aliases Set aliases = new HashSet<>(); for (AliasActions action : actions) { - String[] concreteIndices = indexNameExpressionResolver.concreteIndexNames(state, request.indicesOptions(), action.indices()); + final Index[] concreteIndices = indexNameExpressionResolver.concreteIndices(state, request.indicesOptions(), action.indices()); + final Optional maybeException = requestValidators.validateRequest(request, state, concreteIndices); + if (maybeException.isPresent()) { + listener.onFailure(maybeException.get()); + return; + } Collections.addAll(aliases, action.getOriginalAliases()); - for (String index : concreteIndices) { + for (final Index index : concreteIndices) { switch (action.actionType()) { case ADD: - for (String alias : concreteAliases(action, state.metaData(), index)) { - finalActions.add(new AliasAction.Add(index, alias, action.filter(), action.indexRouting(), + for (String alias : concreteAliases(action, state.metaData(), index.getName())) { + finalActions.add(new AliasAction.Add(index.getName(), alias, action.filter(), action.indexRouting(), action.searchRouting(), action.writeIndex())); } break; case REMOVE: - for (String alias : concreteAliases(action, state.metaData(), index)) { - finalActions.add(new AliasAction.Remove(index, alias)); + for (String alias : concreteAliases(action, state.metaData(), index.getName())) { + finalActions.add(new AliasAction.Remove(index.getName(), alias)); } break; case REMOVE_INDEX: - finalActions.add(new AliasAction.RemoveIndex(index)); + finalActions.add(new AliasAction.RemoveIndex(index.getName())); break; default: throw new IllegalArgumentException("Unsupported action [" + action.actionType() + "]"); diff --git a/server/src/main/java/org/elasticsearch/action/admin/indices/alias/exists/TransportAliasesExistAction.java b/server/src/main/java/org/elasticsearch/action/admin/indices/alias/exists/TransportAliasesExistAction.java deleted file mode 100644 index 858c4184bfcba..0000000000000 --- a/server/src/main/java/org/elasticsearch/action/admin/indices/alias/exists/TransportAliasesExistAction.java +++ /dev/null @@ -1,68 +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.action.admin.indices.alias.exists; - -import org.elasticsearch.action.ActionListener; -import org.elasticsearch.action.admin.indices.alias.get.GetAliasesRequest; -import org.elasticsearch.action.support.ActionFilters; -import org.elasticsearch.action.support.master.TransportMasterNodeReadAction; -import org.elasticsearch.cluster.ClusterState; -import org.elasticsearch.cluster.block.ClusterBlockException; -import org.elasticsearch.cluster.block.ClusterBlockLevel; -import org.elasticsearch.cluster.metadata.IndexNameExpressionResolver; -import org.elasticsearch.cluster.service.ClusterService; -import org.elasticsearch.common.inject.Inject; -import org.elasticsearch.threadpool.ThreadPool; -import org.elasticsearch.transport.TransportService; - -public class TransportAliasesExistAction extends TransportMasterNodeReadAction { - - @Inject - public TransportAliasesExistAction(TransportService transportService, ClusterService clusterService, - ThreadPool threadPool, ActionFilters actionFilters, - IndexNameExpressionResolver indexNameExpressionResolver) { - super(AliasesExistAction.NAME, transportService, clusterService, threadPool, actionFilters, GetAliasesRequest::new, - indexNameExpressionResolver); - } - - @Override - protected String executor() { - // very lightweight operation, no need to fork - return ThreadPool.Names.SAME; - } - - @Override - protected ClusterBlockException checkBlock(GetAliasesRequest request, ClusterState state) { - return state.blocks().indicesBlockedException(ClusterBlockLevel.METADATA_READ, - indexNameExpressionResolver.concreteIndexNames(state, request)); - } - - @Override - protected AliasesExistResponse newResponse() { - return new AliasesExistResponse(); - } - - @Override - protected void masterOperation(GetAliasesRequest request, ClusterState state, ActionListener listener) { - String[] concreteIndices = indexNameExpressionResolver.concreteIndexNames(state, request); - boolean result = state.metaData().hasAliases(request.aliases(), concreteIndices); - listener.onResponse(new AliasesExistResponse(result)); - } - -} diff --git a/server/src/main/java/org/elasticsearch/action/admin/indices/alias/get/BaseAliasesRequestBuilder.java b/server/src/main/java/org/elasticsearch/action/admin/indices/alias/get/BaseAliasesRequestBuilder.java index ba8b45582ea5f..57aa9cb468e5e 100644 --- a/server/src/main/java/org/elasticsearch/action/admin/indices/alias/get/BaseAliasesRequestBuilder.java +++ b/server/src/main/java/org/elasticsearch/action/admin/indices/alias/get/BaseAliasesRequestBuilder.java @@ -19,7 +19,7 @@ package org.elasticsearch.action.admin.indices.alias.get; -import org.elasticsearch.action.Action; +import org.elasticsearch.action.ActionType; import org.elasticsearch.action.ActionResponse; import org.elasticsearch.action.support.IndicesOptions; import org.elasticsearch.action.support.master.MasterNodeReadOperationRequestBuilder; @@ -30,7 +30,7 @@ public abstract class BaseAliasesRequestBuilder> extends MasterNodeReadOperationRequestBuilder { - public BaseAliasesRequestBuilder(ElasticsearchClient client, Action action, String... aliases) { + public BaseAliasesRequestBuilder(ElasticsearchClient client, ActionType action, String... aliases) { super(client, action, new GetAliasesRequest(aliases)); } diff --git a/server/src/main/java/org/elasticsearch/action/admin/indices/alias/get/GetAliasesAction.java b/server/src/main/java/org/elasticsearch/action/admin/indices/alias/get/GetAliasesAction.java index db423c2aaaa07..2798f21e820a4 100644 --- a/server/src/main/java/org/elasticsearch/action/admin/indices/alias/get/GetAliasesAction.java +++ b/server/src/main/java/org/elasticsearch/action/admin/indices/alias/get/GetAliasesAction.java @@ -19,9 +19,9 @@ package org.elasticsearch.action.admin.indices.alias.get; -import org.elasticsearch.action.Action; +import org.elasticsearch.action.StreamableResponseActionType; -public class GetAliasesAction extends Action { +public class GetAliasesAction extends StreamableResponseActionType { public static final GetAliasesAction INSTANCE = new GetAliasesAction(); public static final String NAME = "indices:admin/aliases/get"; diff --git a/server/src/main/java/org/elasticsearch/action/admin/indices/alias/get/TransportGetAliasesAction.java b/server/src/main/java/org/elasticsearch/action/admin/indices/alias/get/TransportGetAliasesAction.java index d1a573e6da8d7..30de49031e101 100644 --- a/server/src/main/java/org/elasticsearch/action/admin/indices/alias/get/TransportGetAliasesAction.java +++ b/server/src/main/java/org/elasticsearch/action/admin/indices/alias/get/TransportGetAliasesAction.java @@ -29,6 +29,7 @@ import org.elasticsearch.cluster.service.ClusterService; import org.elasticsearch.common.collect.ImmutableOpenMap; import org.elasticsearch.common.inject.Inject; +import org.elasticsearch.tasks.Task; import org.elasticsearch.threadpool.ThreadPool; import org.elasticsearch.transport.TransportService; @@ -63,7 +64,7 @@ protected GetAliasesResponse newResponse() { } @Override - protected void masterOperation(GetAliasesRequest request, ClusterState state, ActionListener listener) { + protected void masterOperation(Task task, GetAliasesRequest request, ClusterState state, ActionListener listener) { String[] concreteIndices = indexNameExpressionResolver.concreteIndexNames(state, request); ImmutableOpenMap> aliases = state.metaData().findAliases(request, concreteIndices); listener.onResponse(new GetAliasesResponse(postProcess(request, concreteIndices, aliases))); diff --git a/server/src/main/java/org/elasticsearch/action/admin/indices/analyze/AnalyzeAction.java b/server/src/main/java/org/elasticsearch/action/admin/indices/analyze/AnalyzeAction.java index 3677cd6cb4e43..59d882fe8e18f 100644 --- a/server/src/main/java/org/elasticsearch/action/admin/indices/analyze/AnalyzeAction.java +++ b/server/src/main/java/org/elasticsearch/action/admin/indices/analyze/AnalyzeAction.java @@ -19,10 +19,33 @@ package org.elasticsearch.action.admin.indices.analyze; -import org.elasticsearch.action.Action; +import org.elasticsearch.action.ActionType; +import org.elasticsearch.action.ActionRequestValidationException; +import org.elasticsearch.action.ActionResponse; +import org.elasticsearch.action.support.single.shard.SingleShardRequest; +import org.elasticsearch.common.ParseField; +import org.elasticsearch.common.Strings; +import org.elasticsearch.common.io.stream.StreamInput; +import org.elasticsearch.common.io.stream.StreamOutput; import org.elasticsearch.common.io.stream.Writeable; +import org.elasticsearch.common.xcontent.ObjectParser; +import org.elasticsearch.common.xcontent.ToXContentFragment; +import org.elasticsearch.common.xcontent.ToXContentObject; +import org.elasticsearch.common.xcontent.XContentBuilder; +import org.elasticsearch.common.xcontent.XContentParser; +import org.elasticsearch.index.analysis.NameOrDefinition; -public class AnalyzeAction extends Action { +import java.io.IOException; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.List; +import java.util.Map; +import java.util.Objects; +import java.util.TreeMap; + +import static org.elasticsearch.action.ValidateActions.addValidationError; + +public class AnalyzeAction extends ActionType { public static final AnalyzeAction INSTANCE = new AnalyzeAction(); public static final String NAME = "indices:admin/analyze"; @@ -32,12 +55,754 @@ private AnalyzeAction() { } @Override - public Writeable.Reader getResponseReader() { - return AnalyzeResponse::new; + public Writeable.Reader getResponseReader() { + return Response::new; } - @Override - public AnalyzeResponse newResponse() { - throw new UnsupportedOperationException("usage of Streamable is to be replaced by Writeable"); + /** + * A request to analyze a text associated with a specific index. Allow to provide + * the actual analyzer name to perform the analysis with. + */ + public static class Request extends SingleShardRequest { + + private String[] text; + private String analyzer; + private NameOrDefinition tokenizer; + private final List tokenFilters = new ArrayList<>(); + private final List charFilters = new ArrayList<>(); + private String field; + private boolean explain = false; + private String[] attributes = Strings.EMPTY_ARRAY; + private String normalizer; + + public Request() { + } + + Request(StreamInput in) throws IOException { + super(in); + text = in.readStringArray(); + analyzer = in.readOptionalString(); + tokenizer = in.readOptionalWriteable(NameOrDefinition::new); + tokenFilters.addAll(in.readList(NameOrDefinition::new)); + charFilters.addAll(in.readList(NameOrDefinition::new)); + field = in.readOptionalString(); + explain = in.readBoolean(); + attributes = in.readStringArray(); + normalizer = in.readOptionalString(); + } + + /** + * Constructs a new analyzer request for the provided index. + * + * @param index The text to analyze + */ + public Request(String index) { + this.index(index); + } + + public String[] text() { + return this.text; + } + + public Request text(String... text) { + this.text = text; + return this; + } + + public Request text(List text) { + this.text = text.toArray(new String[]{}); + return this; + } + + public Request analyzer(String analyzer) { + this.analyzer = analyzer; + return this; + } + + public String analyzer() { + return this.analyzer; + } + + public Request tokenizer(String tokenizer) { + this.tokenizer = new NameOrDefinition(tokenizer); + return this; + } + + public Request tokenizer(Map tokenizer) { + this.tokenizer = new NameOrDefinition(tokenizer); + return this; + } + + public void tokenizer(NameOrDefinition tokenizer) { + this.tokenizer = tokenizer; + } + + public NameOrDefinition tokenizer() { + return this.tokenizer; + } + + public Request addTokenFilter(String tokenFilter) { + this.tokenFilters.add(new NameOrDefinition(tokenFilter)); + return this; + } + + public Request addTokenFilter(Map tokenFilter) { + this.tokenFilters.add(new NameOrDefinition(tokenFilter)); + return this; + } + + public void setTokenFilters(List tokenFilters) { + this.tokenFilters.addAll(tokenFilters); + } + + public List tokenFilters() { + return this.tokenFilters; + } + + public Request addCharFilter(Map charFilter) { + this.charFilters.add(new NameOrDefinition(charFilter)); + return this; + } + + public Request addCharFilter(String charFilter) { + this.charFilters.add(new NameOrDefinition(charFilter)); + return this; + } + + public void setCharFilters(List charFilters) { + this.charFilters.addAll(charFilters); + } + + public List charFilters() { + return this.charFilters; + } + + public Request field(String field) { + this.field = field; + return this; + } + + public String field() { + return this.field; + } + + public Request explain(boolean explain) { + this.explain = explain; + return this; + } + + public boolean explain() { + return this.explain; + } + + public Request attributes(String... attributes) { + if (attributes == null) { + throw new IllegalArgumentException("attributes must not be null"); + } + this.attributes = attributes; + return this; + } + + public void attributes(List attributes) { + this.attributes = attributes.toArray(new String[]{}); + } + + public String[] attributes() { + return this.attributes; + } + + public String normalizer() { + return this.normalizer; + } + + public Request normalizer(String normalizer) { + this.normalizer = normalizer; + return this; + } + + @Override + public ActionRequestValidationException validate() { + ActionRequestValidationException validationException = null; + if (text == null || text.length == 0) { + validationException = addValidationError("text is missing", validationException); + } + if ((index == null || index.length() == 0) && normalizer != null) { + validationException = addValidationError("index is required if normalizer is specified", validationException); + } + if (normalizer != null && (tokenizer != null || analyzer != null)) { + validationException + = addValidationError("tokenizer/analyze should be null if normalizer is specified", validationException); + } + if (analyzer != null && (tokenizer != null || charFilters.isEmpty() == false || tokenFilters.isEmpty() == false)) { + validationException + = addValidationError("cannot define extra components on a named analyzer", validationException); + } + if (normalizer != null && (tokenizer != null || charFilters.isEmpty() == false || tokenFilters.isEmpty() == false)) { + validationException + = addValidationError("cannot define extra components on a named normalizer", validationException); + } + if (field != null && (tokenizer != null || charFilters.isEmpty() == false || tokenFilters.isEmpty() == false)) { + validationException + = addValidationError("cannot define extra components on a field-specific analyzer", validationException); + } + return validationException; + } + + @Override + public void writeTo(StreamOutput out) throws IOException { + super.writeTo(out); + out.writeStringArray(text); + out.writeOptionalString(analyzer); + out.writeOptionalWriteable(tokenizer); + out.writeList(tokenFilters); + out.writeList(charFilters); + out.writeOptionalString(field); + out.writeBoolean(explain); + out.writeStringArray(attributes); + out.writeOptionalString(normalizer); + } + + public static Request fromXContent(XContentParser parser, String index) throws IOException { + Request request = new Request(index); + PARSER.parse(parser, request, null); + return request; + } + + private static final ObjectParser PARSER = new ObjectParser<>("analyze_request", null); + static { + PARSER.declareStringArray(Request::text, new ParseField("text")); + PARSER.declareString(Request::analyzer, new ParseField("analyzer")); + PARSER.declareField(Request::tokenizer, (p, c) -> NameOrDefinition.fromXContent(p), + new ParseField("tokenizer"), ObjectParser.ValueType.OBJECT_OR_STRING); + PARSER.declareObjectArray(Request::setTokenFilters, (p, c) -> NameOrDefinition.fromXContent(p), + new ParseField("filter")); + PARSER.declareObjectArray(Request::setCharFilters, (p, c) -> NameOrDefinition.fromXContent(p), + new ParseField("char_filter")); + PARSER.declareString(Request::field, new ParseField("field")); + PARSER.declareBoolean(Request::explain, new ParseField("explain")); + PARSER.declareStringArray(Request::attributes, new ParseField("attributes")); + PARSER.declareString(Request::normalizer, new ParseField("normalizer")); + } + + } + + public static class Response extends ActionResponse implements ToXContentObject { + + private final DetailAnalyzeResponse detail; + private final List tokens; + + public Response(List tokens, DetailAnalyzeResponse detail) { + this.tokens = tokens; + this.detail = detail; + } + + public Response(StreamInput in) throws IOException { + super.readFrom(in); + int size = in.readVInt(); + if (size > 0) { + tokens = new ArrayList<>(size); + for (int i = 0; i < size; i++) { + tokens.add(new AnalyzeToken(in)); + } + } + else { + tokens = null; + } + detail = in.readOptionalWriteable(DetailAnalyzeResponse::new); + } + + @Override + public void readFrom(StreamInput in) throws IOException { + throw new UnsupportedOperationException("usage of Streamable is to be replaced by Writeable"); + } + + public List getTokens() { + return this.tokens; + } + + public DetailAnalyzeResponse detail() { + return this.detail; + } + + @Override + public XContentBuilder toXContent(XContentBuilder builder, Params params) throws IOException { + builder.startObject(); + if (tokens != null) { + builder.startArray(Fields.TOKENS); + for (AnalyzeToken token : tokens) { + token.toXContent(builder, params); + } + builder.endArray(); + } + + if (detail != null) { + builder.startObject(Fields.DETAIL); + detail.toXContent(builder, params); + builder.endObject(); + } + builder.endObject(); + return builder; + } + + @Override + public void writeTo(StreamOutput out) throws IOException { + super.writeTo(out); + if (tokens != null) { + out.writeVInt(tokens.size()); + for (AnalyzeToken token : tokens) { + token.writeTo(out); + } + } else { + out.writeVInt(0); + } + out.writeOptionalWriteable(detail); + } + + @Override + public boolean equals(Object o) { + if (this == o) return true; + if (o == null || getClass() != o.getClass()) return false; + Response that = (Response) o; + return Objects.equals(detail, that.detail) && + Objects.equals(tokens, that.tokens); + } + + @Override + public int hashCode() { + return Objects.hash(detail, tokens); + } + + @Override + public String toString() { + return Strings.toString(this, true, true); + } + + static final class Fields { + static final String TOKENS = "tokens"; + + static final String DETAIL = "detail"; + } } + + public static class AnalyzeToken implements Writeable, ToXContentObject { + private final String term; + private final int startOffset; + private final int endOffset; + private final int position; + private final int positionLength; + private final Map attributes; + private final String type; + + static final String TOKEN = "token"; + static final String START_OFFSET = "start_offset"; + static final String END_OFFSET = "end_offset"; + static final String TYPE = "type"; + static final String POSITION = "position"; + static final String POSITION_LENGTH = "positionLength"; + + @Override + public boolean equals(Object o) { + if (this == o) return true; + if (o == null || getClass() != o.getClass()) return false; + AnalyzeToken that = (AnalyzeToken) o; + return startOffset == that.startOffset && + endOffset == that.endOffset && + position == that.position && + positionLength == that.positionLength && + Objects.equals(term, that.term) && + Objects.equals(attributes, that.attributes) && + Objects.equals(type, that.type); + } + + @Override + public int hashCode() { + return Objects.hash(term, startOffset, endOffset, position, positionLength, attributes, type); + } + + public AnalyzeToken(String term, int position, int startOffset, int endOffset, int positionLength, + String type, Map attributes) { + this.term = term; + this.position = position; + this.startOffset = startOffset; + this.endOffset = endOffset; + this.positionLength = positionLength; + this.type = type; + this.attributes = attributes; + } + + AnalyzeToken(StreamInput in) throws IOException { + term = in.readString(); + startOffset = in.readInt(); + endOffset = in.readInt(); + position = in.readVInt(); + Integer len = in.readOptionalVInt(); + if (len != null) { + positionLength = len; + } else { + positionLength = 1; + } + type = in.readOptionalString(); + attributes = in.readMap(); + } + + public String getTerm() { + return this.term; + } + + public int getStartOffset() { + return this.startOffset; + } + + public int getEndOffset() { + return this.endOffset; + } + + public int getPosition() { + return this.position; + } + + public int getPositionLength() { + return this.positionLength; + } + + public String getType() { + return this.type; + } + + public Map getAttributes(){ + return this.attributes; + } + + @Override + public XContentBuilder toXContent(XContentBuilder builder, Params params) throws IOException { + builder.startObject(); + builder.field(TOKEN, term); + builder.field(START_OFFSET, startOffset); + builder.field(END_OFFSET, endOffset); + builder.field(TYPE, type); + builder.field(POSITION, position); + if (positionLength > 1) { + builder.field(POSITION_LENGTH, positionLength); + } + if (attributes != null && !attributes.isEmpty()) { + Map sortedAttributes = new TreeMap<>(attributes); + for (Map.Entry entity : sortedAttributes.entrySet()) { + builder.field(entity.getKey(), entity.getValue()); + } + } + builder.endObject(); + return builder; + } + + @Override + public void writeTo(StreamOutput out) throws IOException { + out.writeString(term); + out.writeInt(startOffset); + out.writeInt(endOffset); + out.writeVInt(position); + out.writeOptionalVInt(positionLength > 1 ? positionLength : null); + out.writeOptionalString(type); + out.writeMapWithConsistentOrder(attributes); + } + } + + public static class DetailAnalyzeResponse implements Writeable, ToXContentFragment { + + private final boolean customAnalyzer; + private final AnalyzeTokenList analyzer; + private final CharFilteredText[] charfilters; + private final AnalyzeTokenList tokenizer; + private final AnalyzeTokenList[] tokenfilters; + + public DetailAnalyzeResponse(AnalyzeTokenList analyzer) { + this(false, analyzer, null, null, null); + } + + public DetailAnalyzeResponse(CharFilteredText[] charfilters, AnalyzeTokenList tokenizer, AnalyzeTokenList[] tokenfilters) { + this(true, null, charfilters, tokenizer, tokenfilters); + } + + DetailAnalyzeResponse(boolean customAnalyzer, + AnalyzeTokenList analyzer, + CharFilteredText[] charfilters, + AnalyzeTokenList tokenizer, + AnalyzeTokenList[] tokenfilters) { + this.customAnalyzer = customAnalyzer; + this.analyzer = analyzer; + this.charfilters = charfilters; + this.tokenizer = tokenizer; + this.tokenfilters = tokenfilters; + } + + DetailAnalyzeResponse(StreamInput in) throws IOException { + this.customAnalyzer = in.readBoolean(); + if (customAnalyzer) { + tokenizer = new AnalyzeTokenList(in); + int size = in.readVInt(); + if (size > 0) { + charfilters = new CharFilteredText[size]; + for (int i = 0; i < size; i++) { + charfilters[i] = new CharFilteredText(in); + } + } else { + charfilters = null; + } + size = in.readVInt(); + if (size > 0) { + tokenfilters = new AnalyzeTokenList[size]; + for (int i = 0; i < size; i++) { + tokenfilters[i] = new AnalyzeTokenList(in); + } + } else { + tokenfilters = null; + } + analyzer = null; + } else { + analyzer = new AnalyzeTokenList(in); + tokenfilters = null; + tokenizer = null; + charfilters = null; + } + } + + public AnalyzeTokenList analyzer() { + return this.analyzer; + } + + public CharFilteredText[] charfilters() { + return this.charfilters; + } + + public AnalyzeTokenList tokenizer() { + return tokenizer; + } + + public AnalyzeTokenList[] tokenfilters() { + return tokenfilters; + } + + @Override + public boolean equals(Object o) { + if (this == o) return true; + if (o == null || getClass() != o.getClass()) return false; + DetailAnalyzeResponse that = (DetailAnalyzeResponse) o; + return customAnalyzer == that.customAnalyzer && + Objects.equals(analyzer, that.analyzer) && + Arrays.equals(charfilters, that.charfilters) && + Objects.equals(tokenizer, that.tokenizer) && + Arrays.equals(tokenfilters, that.tokenfilters); + } + + @Override + public int hashCode() { + int result = Objects.hash(customAnalyzer, analyzer, tokenizer); + result = 31 * result + Arrays.hashCode(charfilters); + result = 31 * result + Arrays.hashCode(tokenfilters); + return result; + } + + @Override + public XContentBuilder toXContent(XContentBuilder builder, Params params) throws IOException { + builder.field("custom_analyzer", customAnalyzer); + + if (analyzer != null) { + builder.startObject("analyzer"); + analyzer.toXContentWithoutObject(builder, params); + builder.endObject(); + } + + if (charfilters != null) { + builder.startArray("charfilters"); + for (CharFilteredText charfilter : charfilters) { + charfilter.toXContent(builder, params); + } + builder.endArray(); + } + + if (tokenizer != null) { + builder.startObject("tokenizer"); + tokenizer.toXContentWithoutObject(builder, params); + builder.endObject(); + } + + if (tokenfilters != null) { + builder.startArray("tokenfilters"); + for (AnalyzeTokenList tokenfilter : tokenfilters) { + tokenfilter.toXContent(builder, params); + } + builder.endArray(); + } + return builder; + } + + @Override + public void writeTo(StreamOutput out) throws IOException { + out.writeBoolean(customAnalyzer); + if (customAnalyzer) { + tokenizer.writeTo(out); + if (charfilters != null) { + out.writeVInt(charfilters.length); + for (CharFilteredText charfilter : charfilters) { + charfilter.writeTo(out); + } + } else { + out.writeVInt(0); + } + if (tokenfilters != null) { + out.writeVInt(tokenfilters.length); + for (AnalyzeTokenList tokenfilter : tokenfilters) { + tokenfilter.writeTo(out); + } + } else { + out.writeVInt(0); + } + } else { + analyzer.writeTo(out); + } + } + } + + public static class AnalyzeTokenList implements Writeable, ToXContentObject { + private final String name; + private final AnalyzeToken[] tokens; + + static final String NAME = "name"; + + @Override + public boolean equals(Object o) { + if (this == o) return true; + if (o == null || getClass() != o.getClass()) return false; + AnalyzeTokenList that = (AnalyzeTokenList) o; + return Objects.equals(name, that.name) && + Arrays.equals(tokens, that.tokens); + } + + @Override + public int hashCode() { + int result = Objects.hash(name); + result = 31 * result + Arrays.hashCode(tokens); + return result; + } + + public AnalyzeTokenList(String name, AnalyzeToken[] tokens) { + this.name = name; + this.tokens = tokens; + } + + AnalyzeTokenList(StreamInput in) throws IOException { + name = in.readString(); + int size = in.readVInt(); + if (size > 0) { + tokens = new AnalyzeToken[size]; + for (int i = 0; i < size; i++) { + tokens[i] = new AnalyzeToken(in); + } + } + else { + tokens = null; + } + } + + public String getName() { + return name; + } + + public AnalyzeToken[] getTokens() { + return tokens; + } + + void toXContentWithoutObject(XContentBuilder builder, Params params) throws IOException { + builder.field(NAME, this.name); + builder.startArray(Response.Fields.TOKENS); + if (tokens != null) { + for (AnalyzeToken token : tokens) { + token.toXContent(builder, params); + } + } + builder.endArray(); + } + + @Override + public XContentBuilder toXContent(XContentBuilder builder, Params params) throws IOException { + builder.startObject(); + toXContentWithoutObject(builder, params); + builder.endObject(); + return builder; + } + + @Override + public void writeTo(StreamOutput out) throws IOException { + out.writeString(name); + if (tokens != null) { + out.writeVInt(tokens.length); + for (AnalyzeToken token : tokens) { + token.writeTo(out); + } + } else { + out.writeVInt(0); + } + } + } + + public static class CharFilteredText implements Writeable, ToXContentObject { + private final String name; + private final String[] texts; + + static final String NAME = "name"; + static final String FILTERED_TEXT = "filtered_text"; + + public CharFilteredText(String name, String[] texts) { + this.name = name; + if (texts != null) { + this.texts = texts; + } else { + this.texts = Strings.EMPTY_ARRAY; + } + } + + CharFilteredText(StreamInput in) throws IOException { + name = in.readString(); + texts = in.readStringArray(); + } + + public String getName() { + return name; + } + + public String[] getTexts() { + return texts; + } + + @Override + public XContentBuilder toXContent(XContentBuilder builder, Params params) throws IOException { + builder.startObject(); + builder.field(NAME, name); + builder.array(FILTERED_TEXT, texts); + builder.endObject(); + return builder; + } + + @Override + public void writeTo(StreamOutput out) throws IOException { + out.writeString(name); + out.writeStringArray(texts); + } + + @Override + public boolean equals(Object o) { + if (this == o) return true; + if (o == null || getClass() != o.getClass()) return false; + CharFilteredText that = (CharFilteredText) o; + return Objects.equals(name, that.name) && + Arrays.equals(texts, that.texts); + } + + @Override + public int hashCode() { + int result = Objects.hash(name); + result = 31 * result + Arrays.hashCode(texts); + return result; + } + } + } diff --git a/server/src/main/java/org/elasticsearch/action/admin/indices/analyze/AnalyzeRequest.java b/server/src/main/java/org/elasticsearch/action/admin/indices/analyze/AnalyzeRequest.java deleted file mode 100644 index a2712c2d4c107..0000000000000 --- a/server/src/main/java/org/elasticsearch/action/admin/indices/analyze/AnalyzeRequest.java +++ /dev/null @@ -1,302 +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.action.admin.indices.analyze; - -import org.elasticsearch.action.ActionRequestValidationException; -import org.elasticsearch.action.support.single.shard.SingleShardRequest; -import org.elasticsearch.common.Strings; -import org.elasticsearch.common.io.stream.StreamInput; -import org.elasticsearch.common.io.stream.StreamOutput; -import org.elasticsearch.common.io.stream.Writeable; -import org.elasticsearch.common.settings.Settings; -import org.elasticsearch.common.xcontent.ToXContentFragment; -import org.elasticsearch.common.xcontent.ToXContentObject; -import org.elasticsearch.common.xcontent.XContentBuilder; -import org.elasticsearch.common.xcontent.XContentFactory; -import org.elasticsearch.common.xcontent.XContentType; - -import java.io.IOException; -import java.util.ArrayList; -import java.util.List; -import java.util.Map; -import java.util.Objects; - -import static org.elasticsearch.action.ValidateActions.addValidationError; - -/** - * A request to analyze a text associated with a specific index. Allow to provide - * the actual analyzer name to perform the analysis with. - */ -public class AnalyzeRequest extends SingleShardRequest implements ToXContentObject { - - private String[] text; - - private String analyzer; - - private NameOrDefinition tokenizer; - - private final List tokenFilters = new ArrayList<>(); - - private final List charFilters = new ArrayList<>(); - - private String field; - - private boolean explain = false; - - private String[] attributes = Strings.EMPTY_ARRAY; - - private String normalizer; - - public static class NameOrDefinition implements Writeable, ToXContentFragment { - // exactly one of these two members is not null - public final String name; - public final Settings definition; - - NameOrDefinition(String name) { - this.name = Objects.requireNonNull(name); - this.definition = null; - } - - NameOrDefinition(Map definition) { - this.name = null; - Objects.requireNonNull(definition); - try { - XContentBuilder builder = XContentFactory.contentBuilder(XContentType.JSON); - builder.map(definition); - this.definition = Settings.builder().loadFromSource(Strings.toString(builder), builder.contentType()).build(); - } catch (IOException e) { - throw new IllegalArgumentException("Failed to parse [" + definition + "]", e); - } - } - - NameOrDefinition(StreamInput in) throws IOException { - name = in.readOptionalString(); - if (in.readBoolean()) { - definition = Settings.readSettingsFromStream(in); - } else { - definition = null; - } - } - - @Override - public void writeTo(StreamOutput out) throws IOException { - out.writeOptionalString(name); - boolean isNotNullDefinition = this.definition != null; - out.writeBoolean(isNotNullDefinition); - if (isNotNullDefinition) { - Settings.writeSettingsToStream(definition, out); - } - } - - @Override - public XContentBuilder toXContent(XContentBuilder builder, Params params) throws IOException { - if (definition == null) { - return builder.value(name); - } - return definition.toXContent(builder, params); - } - - } - - public AnalyzeRequest() { - } - - /** - * Constructs a new analyzer request for the provided index. - * - * @param index The text to analyze - */ - public AnalyzeRequest(String index) { - this.index(index); - } - - public String[] text() { - return this.text; - } - - public AnalyzeRequest text(String... text) { - this.text = text; - return this; - } - - public AnalyzeRequest analyzer(String analyzer) { - this.analyzer = analyzer; - return this; - } - - public String analyzer() { - return this.analyzer; - } - - public AnalyzeRequest tokenizer(String tokenizer) { - this.tokenizer = new NameOrDefinition(tokenizer); - return this; - } - - public AnalyzeRequest tokenizer(Map tokenizer) { - this.tokenizer = new NameOrDefinition(tokenizer); - return this; - } - - public NameOrDefinition tokenizer() { - return this.tokenizer; - } - - public AnalyzeRequest addTokenFilter(String tokenFilter) { - this.tokenFilters.add(new NameOrDefinition(tokenFilter)); - return this; - } - - public AnalyzeRequest addTokenFilter(Map tokenFilter) { - this.tokenFilters.add(new NameOrDefinition(tokenFilter)); - return this; - } - - public List tokenFilters() { - return this.tokenFilters; - } - - public AnalyzeRequest addCharFilter(Map charFilter) { - this.charFilters.add(new NameOrDefinition(charFilter)); - return this; - } - - public AnalyzeRequest addCharFilter(String charFilter) { - this.charFilters.add(new NameOrDefinition(charFilter)); - return this; - } - - public List charFilters() { - return this.charFilters; - } - - public AnalyzeRequest field(String field) { - this.field = field; - return this; - } - - public String field() { - return this.field; - } - - public AnalyzeRequest explain(boolean explain) { - this.explain = explain; - return this; - } - - public boolean explain() { - return this.explain; - } - - public AnalyzeRequest attributes(String... attributes) { - if (attributes == null) { - throw new IllegalArgumentException("attributes must not be null"); - } - this.attributes = attributes; - return this; - } - - public String[] attributes() { - return this.attributes; - } - - public String normalizer() { - return this.normalizer; - } - - public AnalyzeRequest normalizer(String normalizer) { - this.normalizer = normalizer; - return this; - } - - @Override - public ActionRequestValidationException validate() { - ActionRequestValidationException validationException = null; - if (text == null || text.length == 0) { - validationException = addValidationError("text is missing", validationException); - } - if ((index == null || index.length() == 0) && normalizer != null) { - validationException = addValidationError("index is required if normalizer is specified", validationException); - } - if (normalizer != null && (tokenizer != null || analyzer != null)) { - validationException = addValidationError("tokenizer/analyze should be null if normalizer is specified", validationException); - } - return validationException; - } - - @Override - public void readFrom(StreamInput in) throws IOException { - super.readFrom(in); - text = in.readStringArray(); - analyzer = in.readOptionalString(); - tokenizer = in.readOptionalWriteable(NameOrDefinition::new); - tokenFilters.addAll(in.readList(NameOrDefinition::new)); - charFilters.addAll(in.readList(NameOrDefinition::new)); - field = in.readOptionalString(); - explain = in.readBoolean(); - attributes = in.readStringArray(); - normalizer = in.readOptionalString(); - } - - @Override - public void writeTo(StreamOutput out) throws IOException { - super.writeTo(out); - out.writeStringArray(text); - out.writeOptionalString(analyzer); - out.writeOptionalWriteable(tokenizer); - out.writeList(tokenFilters); - out.writeList(charFilters); - out.writeOptionalString(field); - out.writeBoolean(explain); - out.writeStringArray(attributes); - out.writeOptionalString(normalizer); - } - - @Override - public XContentBuilder toXContent(XContentBuilder builder, Params params) throws IOException { - builder.startObject(); - builder.field("text", text); - if (Strings.isNullOrEmpty(analyzer) == false) { - builder.field("analyzer", analyzer); - } - if (tokenizer != null) { - tokenizer.toXContent(builder, params); - } - if (tokenFilters.size() > 0) { - builder.field("filter", tokenFilters); - } - if (charFilters.size() > 0) { - builder.field("char_filter", charFilters); - } - if (Strings.isNullOrEmpty(field) == false) { - builder.field("field", field); - } - if (explain) { - builder.field("explain", true); - } - if (attributes.length > 0) { - builder.field("attributes", attributes); - } - if (Strings.isNullOrEmpty(normalizer) == false) { - builder.field("normalizer", normalizer); - } - return builder.endObject(); - } - -} diff --git a/server/src/main/java/org/elasticsearch/action/admin/indices/analyze/AnalyzeRequestBuilder.java b/server/src/main/java/org/elasticsearch/action/admin/indices/analyze/AnalyzeRequestBuilder.java index 3893cb25d9dbb..2bd1724c5e69f 100644 --- a/server/src/main/java/org/elasticsearch/action/admin/indices/analyze/AnalyzeRequestBuilder.java +++ b/server/src/main/java/org/elasticsearch/action/admin/indices/analyze/AnalyzeRequestBuilder.java @@ -23,14 +23,15 @@ import java.util.Map; -public class AnalyzeRequestBuilder extends SingleShardOperationRequestBuilder { +public class AnalyzeRequestBuilder + extends SingleShardOperationRequestBuilder { public AnalyzeRequestBuilder(ElasticsearchClient client, AnalyzeAction action) { - super(client, action, new AnalyzeRequest()); + super(client, action, new AnalyzeAction.Request()); } public AnalyzeRequestBuilder(ElasticsearchClient client, AnalyzeAction action, String index, String... text) { - super(client, action, new AnalyzeRequest(index).text(text)); + super(client, action, new AnalyzeAction.Request(index).text(text)); } /** diff --git a/server/src/main/java/org/elasticsearch/action/admin/indices/analyze/AnalyzeResponse.java b/server/src/main/java/org/elasticsearch/action/admin/indices/analyze/AnalyzeResponse.java deleted file mode 100644 index 7e6d525cefb93..0000000000000 --- a/server/src/main/java/org/elasticsearch/action/admin/indices/analyze/AnalyzeResponse.java +++ /dev/null @@ -1,320 +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.action.admin.indices.analyze; - -import org.elasticsearch.action.ActionResponse; -import org.elasticsearch.common.ParseField; -import org.elasticsearch.common.Strings; -import org.elasticsearch.common.io.stream.StreamInput; -import org.elasticsearch.common.io.stream.StreamOutput; -import org.elasticsearch.common.io.stream.Writeable; -import org.elasticsearch.common.xcontent.ConstructingObjectParser; -import org.elasticsearch.common.xcontent.ToXContentObject; -import org.elasticsearch.common.xcontent.XContentBuilder; -import org.elasticsearch.common.xcontent.XContentParser; - -import java.io.IOException; -import java.util.ArrayList; -import java.util.HashMap; -import java.util.Iterator; -import java.util.List; -import java.util.Map; -import java.util.Objects; -import java.util.TreeMap; - -import static org.elasticsearch.common.xcontent.ConstructingObjectParser.optionalConstructorArg; -import static org.elasticsearch.common.xcontent.XContentParserUtils.ensureExpectedToken; - -public class AnalyzeResponse extends ActionResponse implements Iterable, ToXContentObject { - - public static class AnalyzeToken implements Writeable, ToXContentObject { - private final String term; - private final int startOffset; - private final int endOffset; - private final int position; - private final int positionLength; - private final Map attributes; - private final String type; - - @Override - public boolean equals(Object o) { - if (this == o) return true; - if (o == null || getClass() != o.getClass()) return false; - AnalyzeToken that = (AnalyzeToken) o; - return startOffset == that.startOffset && - endOffset == that.endOffset && - position == that.position && - positionLength == that.positionLength && - Objects.equals(term, that.term) && - Objects.equals(attributes, that.attributes) && - Objects.equals(type, that.type); - } - - @Override - public int hashCode() { - return Objects.hash(term, startOffset, endOffset, position, positionLength, attributes, type); - } - - AnalyzeToken(String term, int position, int startOffset, int endOffset, int positionLength, - String type, Map attributes) { - this.term = term; - this.position = position; - this.startOffset = startOffset; - this.endOffset = endOffset; - this.positionLength = positionLength; - this.type = type; - this.attributes = attributes; - } - - AnalyzeToken(StreamInput in) throws IOException { - term = in.readString(); - startOffset = in.readInt(); - endOffset = in.readInt(); - position = in.readVInt(); - Integer len = in.readOptionalVInt(); - if (len != null) { - positionLength = len; - } else { - positionLength = 1; - } - type = in.readOptionalString(); - attributes = in.readMap(); - } - - public String getTerm() { - return this.term; - } - - public int getStartOffset() { - return this.startOffset; - } - - public int getEndOffset() { - return this.endOffset; - } - - public int getPosition() { - return this.position; - } - - public int getPositionLength() { - return this.positionLength; - } - - public String getType() { - return this.type; - } - - public Map getAttributes(){ - return this.attributes; - } - - @Override - public XContentBuilder toXContent(XContentBuilder builder, Params params) throws IOException { - builder.startObject(); - builder.field(Fields.TOKEN, term); - builder.field(Fields.START_OFFSET, startOffset); - builder.field(Fields.END_OFFSET, endOffset); - builder.field(Fields.TYPE, type); - builder.field(Fields.POSITION, position); - if (positionLength > 1) { - builder.field(Fields.POSITION_LENGTH, positionLength); - } - if (attributes != null && !attributes.isEmpty()) { - Map sortedAttributes = new TreeMap<>(attributes); - for (Map.Entry entity : sortedAttributes.entrySet()) { - builder.field(entity.getKey(), entity.getValue()); - } - } - builder.endObject(); - return builder; - } - - public static AnalyzeToken fromXContent(XContentParser parser) throws IOException { - ensureExpectedToken(XContentParser.Token.START_OBJECT, parser.currentToken(), parser::getTokenLocation); - String field = null; - String term = ""; - int position = -1; - int startOffset = -1; - int endOffset = -1; - int positionLength = 1; - String type = ""; - Map attributes = new HashMap<>(); - for (XContentParser.Token t = parser.nextToken(); t != XContentParser.Token.END_OBJECT; t = parser.nextToken()) { - if (t == XContentParser.Token.FIELD_NAME) { - field = parser.currentName(); - continue; - } - if (Fields.TOKEN.equals(field)) { - term = parser.text(); - } else if (Fields.POSITION.equals(field)) { - position = parser.intValue(); - } else if (Fields.START_OFFSET.equals(field)) { - startOffset = parser.intValue(); - } else if (Fields.END_OFFSET.equals(field)) { - endOffset = parser.intValue(); - } else if (Fields.POSITION_LENGTH.equals(field)) { - positionLength = parser.intValue(); - } else if (Fields.TYPE.equals(field)) { - type = parser.text(); - } else { - if (t == XContentParser.Token.VALUE_STRING) { - attributes.put(field, parser.text()); - } else if (t == XContentParser.Token.VALUE_NUMBER) { - attributes.put(field, parser.numberValue()); - } else if (t == XContentParser.Token.VALUE_BOOLEAN) { - attributes.put(field, parser.booleanValue()); - } else if (t == XContentParser.Token.START_OBJECT) { - attributes.put(field, parser.map()); - } else if (t == XContentParser.Token.START_ARRAY) { - attributes.put(field, parser.list()); - } - } - } - return new AnalyzeToken(term, position, startOffset, endOffset, positionLength, type, attributes); - } - - @Override - public void writeTo(StreamOutput out) throws IOException { - out.writeString(term); - out.writeInt(startOffset); - out.writeInt(endOffset); - out.writeVInt(position); - out.writeOptionalVInt(positionLength > 1 ? positionLength : null); - out.writeOptionalString(type); - out.writeMapWithConsistentOrder(attributes); - } - } - - private final DetailAnalyzeResponse detail; - private final List tokens; - - public AnalyzeResponse(List tokens, DetailAnalyzeResponse detail) { - this.tokens = tokens; - this.detail = detail; - } - - public AnalyzeResponse(StreamInput in) throws IOException { - super.readFrom(in); - int size = in.readVInt(); - if (size > 0) { - tokens = new ArrayList<>(size); - for (int i = 0; i < size; i++) { - tokens.add(new AnalyzeToken(in)); - } - } - else { - tokens = null; - } - detail = in.readOptionalWriteable(DetailAnalyzeResponse::new); - } - - @Override - public void readFrom(StreamInput in) throws IOException { - throw new UnsupportedOperationException("usage of Streamable is to be replaced by Writeable"); - } - - public List getTokens() { - return this.tokens; - } - - public DetailAnalyzeResponse detail() { - return this.detail; - } - - @Override - public Iterator iterator() { - return tokens.iterator(); - } - - @Override - public XContentBuilder toXContent(XContentBuilder builder, Params params) throws IOException { - builder.startObject(); - if (tokens != null) { - builder.startArray(Fields.TOKENS); - for (AnalyzeToken token : tokens) { - token.toXContent(builder, params); - } - builder.endArray(); - } - - if (detail != null) { - builder.startObject(Fields.DETAIL); - detail.toXContent(builder, params); - builder.endObject(); - } - builder.endObject(); - return builder; - } - - private static final ConstructingObjectParser PARSER = new ConstructingObjectParser<>("analyze_response", - true, args -> new AnalyzeResponse((List) args[0], (DetailAnalyzeResponse) args[1])); - static { - PARSER.declareObjectArray(optionalConstructorArg(), (p, c) -> AnalyzeToken.fromXContent(p), new ParseField(Fields.TOKENS)); - PARSER.declareObject(optionalConstructorArg(), DetailAnalyzeResponse.PARSER, new ParseField(Fields.DETAIL)); - } - - public static AnalyzeResponse fromXContent(XContentParser parser) throws IOException { - return PARSER.parse(parser, null); - } - - @Override - public void writeTo(StreamOutput out) throws IOException { - super.writeTo(out); - if (tokens != null) { - out.writeVInt(tokens.size()); - for (AnalyzeToken token : tokens) { - token.writeTo(out); - } - } else { - out.writeVInt(0); - } - out.writeOptionalWriteable(detail); - } - - @Override - public boolean equals(Object o) { - if (this == o) return true; - if (o == null || getClass() != o.getClass()) return false; - AnalyzeResponse that = (AnalyzeResponse) o; - return Objects.equals(detail, that.detail) && - Objects.equals(tokens, that.tokens); - } - - @Override - public int hashCode() { - return Objects.hash(detail, tokens); - } - - @Override - public String toString() { - return Strings.toString(this, true, true); - } - - static final class Fields { - static final String TOKENS = "tokens"; - static final String TOKEN = "token"; - static final String START_OFFSET = "start_offset"; - static final String END_OFFSET = "end_offset"; - static final String TYPE = "type"; - static final String POSITION = "position"; - static final String POSITION_LENGTH = "positionLength"; - static final String DETAIL = "detail"; - } -} diff --git a/server/src/main/java/org/elasticsearch/action/admin/indices/analyze/DetailAnalyzeResponse.java b/server/src/main/java/org/elasticsearch/action/admin/indices/analyze/DetailAnalyzeResponse.java deleted file mode 100644 index 1e84d9e0a2e1a..0000000000000 --- a/server/src/main/java/org/elasticsearch/action/admin/indices/analyze/DetailAnalyzeResponse.java +++ /dev/null @@ -1,400 +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.action.admin.indices.analyze; - - -import org.elasticsearch.common.ParseField; -import org.elasticsearch.common.Strings; -import org.elasticsearch.common.io.stream.StreamInput; -import org.elasticsearch.common.io.stream.StreamOutput; -import org.elasticsearch.common.io.stream.Writeable; -import org.elasticsearch.common.xcontent.ConstructingObjectParser; -import org.elasticsearch.common.xcontent.ToXContentFragment; -import org.elasticsearch.common.xcontent.ToXContentObject; -import org.elasticsearch.common.xcontent.XContentBuilder; -import org.elasticsearch.common.xcontent.XContentParser; - -import java.io.IOException; -import java.lang.reflect.Array; -import java.util.Arrays; -import java.util.List; -import java.util.Objects; - -import static org.elasticsearch.common.xcontent.ConstructingObjectParser.constructorArg; -import static org.elasticsearch.common.xcontent.ConstructingObjectParser.optionalConstructorArg; - -public class DetailAnalyzeResponse implements Writeable, ToXContentFragment { - - private final boolean customAnalyzer; - private final AnalyzeTokenList analyzer; - private final CharFilteredText[] charfilters; - private final AnalyzeTokenList tokenizer; - private final AnalyzeTokenList[] tokenfilters; - - public DetailAnalyzeResponse(AnalyzeTokenList analyzer) { - this(false, analyzer, null, null, null); - } - - public DetailAnalyzeResponse(CharFilteredText[] charfilters, AnalyzeTokenList tokenizer, AnalyzeTokenList[] tokenfilters) { - this(true, null, charfilters, tokenizer, tokenfilters); - } - - public DetailAnalyzeResponse(boolean customAnalyzer, - AnalyzeTokenList analyzer, - CharFilteredText[] charfilters, - AnalyzeTokenList tokenizer, - AnalyzeTokenList[] tokenfilters) { - this.customAnalyzer = customAnalyzer; - this.analyzer = analyzer; - this.charfilters = charfilters; - this.tokenizer = tokenizer; - this.tokenfilters = tokenfilters; - } - - public DetailAnalyzeResponse(StreamInput in) throws IOException { - this.customAnalyzer = in.readBoolean(); - if (customAnalyzer) { - tokenizer = new AnalyzeTokenList(in); - int size = in.readVInt(); - if (size > 0) { - charfilters = new CharFilteredText[size]; - for (int i = 0; i < size; i++) { - charfilters[i] = new CharFilteredText(in); - } - } - else { - charfilters = null; - } - size = in.readVInt(); - if (size > 0) { - tokenfilters = new AnalyzeTokenList[size]; - for (int i = 0; i < size; i++) { - tokenfilters[i] = new AnalyzeTokenList(in); - } - } - else { - tokenfilters = null; - } - analyzer = null; - } else { - analyzer = new AnalyzeTokenList(in); - tokenfilters = null; - tokenizer = null; - charfilters = null; - } - } - - public AnalyzeTokenList analyzer() { - return this.analyzer; - } - - public CharFilteredText[] charfilters() { - return this.charfilters; - } - - public AnalyzeTokenList tokenizer() { - return tokenizer; - } - - public AnalyzeTokenList[] tokenfilters() { - return tokenfilters; - } - - @Override - public boolean equals(Object o) { - if (this == o) return true; - if (o == null || getClass() != o.getClass()) return false; - DetailAnalyzeResponse that = (DetailAnalyzeResponse) o; - return customAnalyzer == that.customAnalyzer && - Objects.equals(analyzer, that.analyzer) && - Arrays.equals(charfilters, that.charfilters) && - Objects.equals(tokenizer, that.tokenizer) && - Arrays.equals(tokenfilters, that.tokenfilters); - } - - @Override - public int hashCode() { - int result = Objects.hash(customAnalyzer, analyzer, tokenizer); - result = 31 * result + Arrays.hashCode(charfilters); - result = 31 * result + Arrays.hashCode(tokenfilters); - return result; - } - - @Override - public XContentBuilder toXContent(XContentBuilder builder, Params params) throws IOException { - builder.field(Fields.CUSTOM_ANALYZER, customAnalyzer); - - if (analyzer != null) { - builder.startObject(Fields.ANALYZER); - analyzer.toXContentWithoutObject(builder, params); - builder.endObject(); - } - - if (charfilters != null) { - builder.startArray(Fields.CHARFILTERS); - for (CharFilteredText charfilter : charfilters) { - charfilter.toXContent(builder, params); - } - builder.endArray(); - } - - if (tokenizer != null) { - builder.startObject(Fields.TOKENIZER); - tokenizer.toXContentWithoutObject(builder, params); - builder.endObject(); - } - - if (tokenfilters != null) { - builder.startArray(Fields.TOKENFILTERS); - for (AnalyzeTokenList tokenfilter : tokenfilters) { - tokenfilter.toXContent(builder, params); - } - builder.endArray(); - } - return builder; - } - - @SuppressWarnings("unchecked") - private static T[] fromList(Class clazz, List list) { - if (list == null) { - return null; - } - return list.toArray((T[])Array.newInstance(clazz, 0)); - } - - static final ConstructingObjectParser PARSER = new ConstructingObjectParser<>("detail", - true, args -> new DetailAnalyzeResponse((boolean) args[0], (AnalyzeTokenList) args[1], - fromList(CharFilteredText.class, (List)args[2]), - (AnalyzeTokenList) args[3], - fromList(AnalyzeTokenList.class, (List)args[4]))); - - static { - PARSER.declareBoolean(constructorArg(), new ParseField(Fields.CUSTOM_ANALYZER)); - PARSER.declareObject(optionalConstructorArg(), AnalyzeTokenList.PARSER, new ParseField(Fields.ANALYZER)); - PARSER.declareObjectArray(optionalConstructorArg(), CharFilteredText.PARSER, new ParseField(Fields.CHARFILTERS)); - PARSER.declareObject(optionalConstructorArg(), AnalyzeTokenList.PARSER, new ParseField(Fields.TOKENIZER)); - PARSER.declareObjectArray(optionalConstructorArg(), AnalyzeTokenList.PARSER, new ParseField(Fields.TOKENFILTERS)); - } - - public static DetailAnalyzeResponse fromXContent(XContentParser parser) throws IOException { - return PARSER.parse(parser, null); - } - - static final class Fields { - static final String NAME = "name"; - static final String FILTERED_TEXT = "filtered_text"; - static final String CUSTOM_ANALYZER = "custom_analyzer"; - static final String ANALYZER = "analyzer"; - static final String CHARFILTERS = "charfilters"; - static final String TOKENIZER = "tokenizer"; - static final String TOKENFILTERS = "tokenfilters"; - } - - @Override - public void writeTo(StreamOutput out) throws IOException { - out.writeBoolean(customAnalyzer); - if (customAnalyzer) { - tokenizer.writeTo(out); - if (charfilters != null) { - out.writeVInt(charfilters.length); - for (CharFilteredText charfilter : charfilters) { - charfilter.writeTo(out); - } - } else { - out.writeVInt(0); - } - if (tokenfilters != null) { - out.writeVInt(tokenfilters.length); - for (AnalyzeTokenList tokenfilter : tokenfilters) { - tokenfilter.writeTo(out); - } - } else { - out.writeVInt(0); - } - } else { - analyzer.writeTo(out); - } - } - - public static class AnalyzeTokenList implements Writeable, ToXContentObject { - private final String name; - private final AnalyzeResponse.AnalyzeToken[] tokens; - - @Override - public boolean equals(Object o) { - if (this == o) return true; - if (o == null || getClass() != o.getClass()) return false; - AnalyzeTokenList that = (AnalyzeTokenList) o; - return Objects.equals(name, that.name) && - Arrays.equals(tokens, that.tokens); - } - - @Override - public int hashCode() { - int result = Objects.hash(name); - result = 31 * result + Arrays.hashCode(tokens); - return result; - } - - public AnalyzeTokenList(String name, AnalyzeResponse.AnalyzeToken[] tokens) { - this.name = name; - this.tokens = tokens; - } - - public AnalyzeTokenList(StreamInput in) throws IOException { - name = in.readString(); - int size = in.readVInt(); - if (size > 0) { - tokens = new AnalyzeResponse.AnalyzeToken[size]; - for (int i = 0; i < size; i++) { - tokens[i] = new AnalyzeResponse.AnalyzeToken(in); - } - } - else { - tokens = null; - } - } - - public String getName() { - return name; - } - - public AnalyzeResponse.AnalyzeToken[] getTokens() { - return tokens; - } - - XContentBuilder toXContentWithoutObject(XContentBuilder builder, Params params) throws IOException { - builder.field(Fields.NAME, this.name); - builder.startArray(AnalyzeResponse.Fields.TOKENS); - if (tokens != null) { - for (AnalyzeResponse.AnalyzeToken token : tokens) { - token.toXContent(builder, params); - } - } - builder.endArray(); - return builder; - } - - @Override - public XContentBuilder toXContent(XContentBuilder builder, Params params) throws IOException { - builder.startObject(); - toXContentWithoutObject(builder, params); - builder.endObject(); - return builder; - } - - private static final ConstructingObjectParser PARSER = new ConstructingObjectParser<>("token_list", - true, args -> new AnalyzeTokenList((String) args[0], - fromList(AnalyzeResponse.AnalyzeToken.class, (List)args[1]))); - - static { - PARSER.declareString(constructorArg(), new ParseField(Fields.NAME)); - PARSER.declareObjectArray(constructorArg(), (p, c) -> AnalyzeResponse.AnalyzeToken.fromXContent(p), - new ParseField(AnalyzeResponse.Fields.TOKENS)); - } - - public static AnalyzeTokenList fromXContent(XContentParser parser) throws IOException { - return PARSER.parse(parser, null); - } - - @Override - public void writeTo(StreamOutput out) throws IOException { - out.writeString(name); - if (tokens != null) { - out.writeVInt(tokens.length); - for (AnalyzeResponse.AnalyzeToken token : tokens) { - token.writeTo(out); - } - } else { - out.writeVInt(0); - } - } - } - - public static class CharFilteredText implements Writeable, ToXContentObject { - private final String name; - private final String[] texts; - - public CharFilteredText(String name, String[] texts) { - this.name = name; - if (texts != null) { - this.texts = texts; - } else { - this.texts = Strings.EMPTY_ARRAY; - } - } - - public CharFilteredText(StreamInput in) throws IOException { - name = in.readString(); - texts = in.readStringArray(); - } - - public String getName() { - return name; - } - - public String[] getTexts() { - return texts; - } - - @Override - public XContentBuilder toXContent(XContentBuilder builder, Params params) throws IOException { - builder.startObject(); - builder.field(Fields.NAME, name); - builder.array(Fields.FILTERED_TEXT, texts); - builder.endObject(); - return builder; - } - - private static final ConstructingObjectParser PARSER = new ConstructingObjectParser<>("char_filtered_text", - true, args -> new CharFilteredText((String) args[0], ((List) args[1]).toArray(new String[0]))); - - static { - PARSER.declareString(constructorArg(), new ParseField(Fields.NAME)); - PARSER.declareStringArray(constructorArg(), new ParseField(Fields.FILTERED_TEXT)); - } - - public static CharFilteredText fromXContent(XContentParser parser) throws IOException { - return PARSER.parse(parser, null); - } - - @Override - public void writeTo(StreamOutput out) throws IOException { - out.writeString(name); - out.writeStringArray(texts); - } - - @Override - public boolean equals(Object o) { - if (this == o) return true; - if (o == null || getClass() != o.getClass()) return false; - CharFilteredText that = (CharFilteredText) o; - return Objects.equals(name, that.name) && - Arrays.equals(texts, that.texts); - } - - @Override - public int hashCode() { - int result = Objects.hash(name); - result = 31 * result + Arrays.hashCode(texts); - return result; - } - } -} diff --git a/server/src/main/java/org/elasticsearch/action/admin/indices/analyze/TransportAnalyzeAction.java b/server/src/main/java/org/elasticsearch/action/admin/indices/analyze/TransportAnalyzeAction.java index 55bd593742667..773852860c60e 100644 --- a/server/src/main/java/org/elasticsearch/action/admin/indices/analyze/TransportAnalyzeAction.java +++ b/server/src/main/java/org/elasticsearch/action/admin/indices/analyze/TransportAnalyzeAction.java @@ -28,45 +28,37 @@ import org.apache.lucene.analysis.tokenattributes.TypeAttribute; import org.apache.lucene.util.BytesRef; import org.elasticsearch.ElasticsearchException; -import org.elasticsearch.Version; import org.elasticsearch.action.support.ActionFilters; import org.elasticsearch.action.support.single.shard.TransportSingleShardAction; import org.elasticsearch.cluster.ClusterState; import org.elasticsearch.cluster.block.ClusterBlockException; -import org.elasticsearch.cluster.metadata.IndexMetaData; import org.elasticsearch.cluster.metadata.IndexNameExpressionResolver; import org.elasticsearch.cluster.routing.ShardsIterator; import org.elasticsearch.cluster.service.ClusterService; -import org.elasticsearch.common.UUIDs; -import org.elasticsearch.common.collect.Tuple; import org.elasticsearch.common.inject.Inject; import org.elasticsearch.common.io.stream.Writeable; import org.elasticsearch.common.settings.Settings; import org.elasticsearch.core.internal.io.IOUtils; -import org.elasticsearch.env.Environment; import org.elasticsearch.index.IndexService; import org.elasticsearch.index.IndexSettings; import org.elasticsearch.index.analysis.AnalysisRegistry; +import org.elasticsearch.index.analysis.AnalyzerComponents; +import org.elasticsearch.index.analysis.AnalyzerComponentsProvider; import org.elasticsearch.index.analysis.CharFilterFactory; -import org.elasticsearch.index.analysis.CustomAnalyzer; -import org.elasticsearch.index.analysis.IndexAnalyzers; +import org.elasticsearch.index.analysis.NameOrDefinition; import org.elasticsearch.index.analysis.NamedAnalyzer; -import org.elasticsearch.index.analysis.NormalizingCharFilterFactory; -import org.elasticsearch.index.analysis.NormalizingTokenFilterFactory; import org.elasticsearch.index.analysis.TokenFilterFactory; import org.elasticsearch.index.analysis.TokenizerFactory; import org.elasticsearch.index.mapper.KeywordFieldMapper; import org.elasticsearch.index.mapper.MappedFieldType; import org.elasticsearch.index.shard.ShardId; import org.elasticsearch.indices.IndicesService; -import org.elasticsearch.indices.analysis.AnalysisModule; import org.elasticsearch.threadpool.ThreadPool; import org.elasticsearch.transport.TransportService; import java.io.IOException; import java.io.Reader; import java.io.StringReader; -import java.io.UncheckedIOException; import java.util.ArrayList; import java.util.HashSet; import java.util.List; @@ -74,35 +66,32 @@ import java.util.Map; import java.util.Set; import java.util.TreeMap; -import java.util.function.Function; /** * Transport action used to execute analyze requests */ -public class TransportAnalyzeAction extends TransportSingleShardAction { +public class TransportAnalyzeAction extends TransportSingleShardAction { private final Settings settings; private final IndicesService indicesService; - private final Environment environment; @Inject public TransportAnalyzeAction(Settings settings, ThreadPool threadPool, ClusterService clusterService, TransportService transportService, IndicesService indicesService, ActionFilters actionFilters, - IndexNameExpressionResolver indexNameExpressionResolver, Environment environment) { + IndexNameExpressionResolver indexNameExpressionResolver) { super(AnalyzeAction.NAME, threadPool, clusterService, transportService, actionFilters, indexNameExpressionResolver, - AnalyzeRequest::new, ThreadPool.Names.ANALYZE); + AnalyzeAction.Request::new, ThreadPool.Names.ANALYZE); this.settings = settings; this.indicesService = indicesService; - this.environment = environment; } @Override - protected Writeable.Reader getResponseReader() { - return AnalyzeResponse::new; + protected Writeable.Reader getResponseReader() { + return AnalyzeAction.Response::new; } @Override - protected boolean resolveIndex(AnalyzeRequest request) { + protected boolean resolveIndex(AnalyzeAction.Request request) { return request.index() != null; } @@ -124,143 +113,116 @@ protected ShardsIterator shards(ClusterState state, InternalRequest request) { } @Override - protected AnalyzeResponse shardOperation(AnalyzeRequest request, ShardId shardId) { - try { - final IndexService indexService; - if (shardId != null) { - indexService = indicesService.indexServiceSafe(shardId.getIndex()); - } else { - indexService = null; - } - String field = null; - Analyzer analyzer = null; - if (request.field() != null) { - if (indexService == null) { - throw new IllegalArgumentException( - "No index provided, and trying to analyzer based on a specific field which requires the index parameter"); - } - MappedFieldType fieldType = indexService.mapperService().fullName(request.field()); - if (fieldType != null) { - if (fieldType.tokenized() || fieldType instanceof KeywordFieldMapper.KeywordFieldType) { - analyzer = fieldType.indexAnalyzer(); - } else { - throw new IllegalArgumentException("Can't process field [" + request.field() + - "], Analysis requests are only supported on tokenized fields"); - } - field = fieldType.name(); - } - } - if (field == null) { - /** - * TODO: _all is disabled by default and index.query.default_field can define multiple fields or patterns so we should - * probably makes the field name mandatory in analyze query. - **/ - if (indexService != null) { - field = indexService.getIndexSettings().getDefaultFields().get(0); - } + protected AnalyzeAction.Response shardOperation(AnalyzeAction.Request request, ShardId shardId) throws IOException { + final IndexService indexService = getIndexService(shardId); + final int maxTokenCount = indexService == null ? + IndexSettings.MAX_TOKEN_COUNT_SETTING.get(settings) : indexService.getIndexSettings().getMaxTokenCount(); + + return analyze(request, indicesService.getAnalysis(), indexService, maxTokenCount); + } + + public static AnalyzeAction.Response analyze(AnalyzeAction.Request request, AnalysisRegistry analysisRegistry, + IndexService indexService, int maxTokenCount) throws IOException { + + IndexSettings settings = indexService == null ? null : indexService.getIndexSettings(); + + // First, we check to see if the request requires a custom analyzer. If so, then we + // need to build it and then close it after use. + try (Analyzer analyzer = buildCustomAnalyzer(request, analysisRegistry, settings)) { + if (analyzer != null) { + return analyze(request, analyzer, maxTokenCount); } - final AnalysisRegistry analysisRegistry = indicesService.getAnalysis(); - final int maxTokenCount = indexService == null ? - IndexSettings.MAX_TOKEN_COUNT_SETTING.get(settings) : indexService.getIndexSettings().getMaxTokenCount(); - return analyze(request, field, analyzer, indexService != null ? indexService.getIndexAnalyzers() : null, - analysisRegistry, environment, maxTokenCount); - } catch (IOException e) { - throw new ElasticsearchException("analysis failed", e); } + // Otherwise we use a built-in analyzer, which should not be closed + return analyze(request, getAnalyzer(request, analysisRegistry, indexService), maxTokenCount); + } + + private IndexService getIndexService(ShardId shardId) { + if (shardId != null) { + return indicesService.indexServiceSafe(shardId.getIndex()); + } + return null; } - public static AnalyzeResponse analyze(AnalyzeRequest request, String field, Analyzer analyzer, IndexAnalyzers indexAnalyzers, - AnalysisRegistry analysisRegistry, Environment environment, int maxTokenCount) throws IOException { - boolean closeAnalyzer = false; - if (analyzer == null && request.analyzer() != null) { - if (indexAnalyzers == null) { - analyzer = analysisRegistry.getAnalyzer(request.analyzer()); + private static Analyzer getAnalyzer(AnalyzeAction.Request request, AnalysisRegistry analysisRegistry, + IndexService indexService) throws IOException { + if (request.analyzer() != null) { + if (indexService == null) { + Analyzer analyzer = analysisRegistry.getAnalyzer(request.analyzer()); if (analyzer == null) { throw new IllegalArgumentException("failed to find global analyzer [" + request.analyzer() + "]"); } + return analyzer; } else { - analyzer = indexAnalyzers.get(request.analyzer()); + Analyzer analyzer = indexService.getIndexAnalyzers().get(request.analyzer()); if (analyzer == null) { throw new IllegalArgumentException("failed to find analyzer [" + request.analyzer() + "]"); } + return analyzer; } - } else if (request.tokenizer() != null) { - final IndexSettings indexSettings = indexAnalyzers == null ? null : indexAnalyzers.getIndexSettings(); - Tuple tokenizerFactory = parseTokenizerFactory(request, indexAnalyzers, - analysisRegistry, environment); - - List charFilterFactoryList = - parseCharFilterFactories(request, indexSettings, analysisRegistry, environment, false); - - List tokenFilterFactoryList = parseTokenFilterFactories(request, indexSettings, analysisRegistry, - environment, tokenizerFactory, charFilterFactoryList, false); - - analyzer = new CustomAnalyzer(tokenizerFactory.v1(), tokenizerFactory.v2(), - charFilterFactoryList.toArray(new CharFilterFactory[charFilterFactoryList.size()]), - tokenFilterFactoryList.toArray(new TokenFilterFactory[tokenFilterFactoryList.size()])); - closeAnalyzer = true; - } else if (request.normalizer() != null) { + } + if (request.normalizer() != null) { // Get normalizer from indexAnalyzers - analyzer = indexAnalyzers.getNormalizer(request.normalizer()); + if (indexService == null) { + throw new IllegalArgumentException("analysis based on a normalizer requires an index"); + } + Analyzer analyzer = indexService.getIndexAnalyzers().getNormalizer(request.normalizer()); if (analyzer == null) { throw new IllegalArgumentException("failed to find normalizer under [" + request.normalizer() + "]"); } - } else if (((request.tokenFilters() != null && request.tokenFilters().size() > 0) - || (request.charFilters() != null && request.charFilters().size() > 0))) { - final IndexSettings indexSettings = indexAnalyzers == null ? null : indexAnalyzers.getIndexSettings(); - // custom normalizer = if normalizer == null but filter or char_filter is not null and tokenizer/analyzer is null - // get charfilter and filter from request - List charFilterFactoryList = - parseCharFilterFactories(request, indexSettings, analysisRegistry, environment, true); - - final String keywordTokenizerName = "keyword"; - TokenizerFactory keywordTokenizerFactory = getTokenizerFactory(analysisRegistry, environment, keywordTokenizerName); - - List tokenFilterFactoryList = - parseTokenFilterFactories(request, indexSettings, analysisRegistry, environment, - new Tuple<>(keywordTokenizerName, keywordTokenizerFactory), charFilterFactoryList, true); - - analyzer = new CustomAnalyzer("keyword_for_normalizer", - keywordTokenizerFactory, - charFilterFactoryList.toArray(new CharFilterFactory[charFilterFactoryList.size()]), - tokenFilterFactoryList.toArray(new TokenFilterFactory[tokenFilterFactoryList.size()])); - closeAnalyzer = true; - } else if (analyzer == null) { - if (indexAnalyzers == null) { - analyzer = analysisRegistry.getAnalyzer("standard"); - } else { - analyzer = indexAnalyzers.getDefaultIndexAnalyzer(); - } } - if (analyzer == null) { - throw new IllegalArgumentException("failed to find analyzer"); + if (request.field() != null) { + if (indexService == null) { + throw new IllegalArgumentException("analysis based on a specific field requires an index"); + } + MappedFieldType fieldType = indexService.mapperService().fullName(request.field()); + if (fieldType != null) { + if (fieldType.tokenized() || fieldType instanceof KeywordFieldMapper.KeywordFieldType) { + return fieldType.indexAnalyzer(); + } else { + throw new IllegalArgumentException("Can't process field [" + request.field() + + "], Analysis requests are only supported on tokenized fields"); + } + } } - - List tokens = null; - DetailAnalyzeResponse detail = null; - - if (request.explain()) { - detail = detailAnalyze(request, analyzer, field, maxTokenCount); + if (indexService == null) { + return analysisRegistry.getAnalyzer("standard"); } else { - tokens = simpleAnalyze(request, analyzer, field, maxTokenCount); + return indexService.getIndexAnalyzers().getDefaultIndexAnalyzer(); } + } - if (closeAnalyzer) { - analyzer.close(); + private static Analyzer buildCustomAnalyzer(AnalyzeAction.Request request, AnalysisRegistry analysisRegistry, + IndexSettings indexSettings) throws IOException { + if (request.tokenizer() != null) { + return analysisRegistry.buildCustomAnalyzer(indexSettings, false, + request.tokenizer(), request.charFilters(), request.tokenFilters()); + } else if (((request.tokenFilters() != null && request.tokenFilters().size() > 0) + || (request.charFilters() != null && request.charFilters().size() > 0))) { + return analysisRegistry.buildCustomAnalyzer(indexSettings, true, new NameOrDefinition("keyword"), + request.charFilters(), request.tokenFilters()); } + return null; + } - return new AnalyzeResponse(tokens, detail); + private static AnalyzeAction.Response analyze(AnalyzeAction.Request request, Analyzer analyzer, int maxTokenCount) { + if (request.explain()) { + return new AnalyzeAction.Response(null, detailAnalyze(request, analyzer, maxTokenCount)); + } + return new AnalyzeAction.Response(simpleAnalyze(request, analyzer, maxTokenCount), null); } - private static List simpleAnalyze(AnalyzeRequest request, - Analyzer analyzer, String field, int maxTokenCount) { + private static List simpleAnalyze(AnalyzeAction.Request request, + Analyzer analyzer, int maxTokenCount) { TokenCounter tc = new TokenCounter(maxTokenCount); - List tokens = new ArrayList<>(); + List tokens = new ArrayList<>(); int lastPosition = -1; int lastOffset = 0; + // Note that we always pass "" as the field to the various Analyzer methods, because + // the analyzers we use here are all field-specific and so ignore this parameter for (String text : request.text()) { - try (TokenStream stream = analyzer.tokenStream(field, text)) { + try (TokenStream stream = analyzer.tokenStream("", text)) { stream.reset(); CharTermAttribute term = stream.addAttribute(CharTermAttribute.class); PositionIncrementAttribute posIncr = stream.addAttribute(PositionIncrementAttribute.class); @@ -273,7 +235,7 @@ private static List simpleAnalyze(AnalyzeRequest r if (increment > 0) { lastPosition = lastPosition + increment; } - tokens.add(new AnalyzeResponse.AnalyzeToken(term.toString(), lastPosition, lastOffset + offset.startOffset(), + tokens.add(new AnalyzeAction.AnalyzeToken(term.toString(), lastPosition, lastOffset + offset.startOffset(), lastOffset + offset.endOffset(), posLen.getPositionLength(), type.type(), null)); tc.increment(); } @@ -281,8 +243,8 @@ private static List simpleAnalyze(AnalyzeRequest r lastOffset += offset.endOffset(); lastPosition += posIncr.getPositionIncrement(); - lastPosition += analyzer.getPositionIncrementGap(field); - lastOffset += analyzer.getOffsetGap(field); + lastPosition += analyzer.getPositionIncrementGap(""); + lastOffset += analyzer.getOffsetGap(""); } catch (IOException e) { throw new ElasticsearchException("failed to analyze", e); } @@ -290,8 +252,9 @@ private static List simpleAnalyze(AnalyzeRequest r return tokens; } - private static DetailAnalyzeResponse detailAnalyze(AnalyzeRequest request, Analyzer analyzer, String field, int maxTokenCount) { - DetailAnalyzeResponse detailResponse; + private static AnalyzeAction.DetailAnalyzeResponse detailAnalyze(AnalyzeAction.Request request, Analyzer analyzer, + int maxTokenCount) { + AnalyzeAction.DetailAnalyzeResponse detailResponse; final Set includeAttributes = new HashSet<>(); if (request.attributes() != null) { for (String attribute : request.attributes()) { @@ -299,18 +262,23 @@ private static DetailAnalyzeResponse detailAnalyze(AnalyzeRequest request, Analy } } - CustomAnalyzer customAnalyzer = null; - if (analyzer instanceof CustomAnalyzer) { - customAnalyzer = (CustomAnalyzer) analyzer; - } else if (analyzer instanceof NamedAnalyzer && ((NamedAnalyzer) analyzer).analyzer() instanceof CustomAnalyzer) { - customAnalyzer = (CustomAnalyzer) ((NamedAnalyzer) analyzer).analyzer(); + // maybe unwrap analyzer from NamedAnalyzer + Analyzer potentialCustomAnalyzer = analyzer; + if (analyzer instanceof NamedAnalyzer) { + potentialCustomAnalyzer = ((NamedAnalyzer) analyzer).analyzer(); } - if (customAnalyzer != null) { - // customAnalyzer = divide charfilter, tokenizer tokenfilters - CharFilterFactory[] charFilterFactories = customAnalyzer.charFilters(); - TokenizerFactory tokenizerFactory = customAnalyzer.tokenizerFactory(); - TokenFilterFactory[] tokenFilterFactories = customAnalyzer.tokenFilters(); + if (potentialCustomAnalyzer instanceof AnalyzerComponentsProvider) { + AnalyzerComponentsProvider customAnalyzer = (AnalyzerComponentsProvider) potentialCustomAnalyzer; + // note: this is not field-name dependent in our cases so we can leave out the argument + int positionIncrementGap = potentialCustomAnalyzer.getPositionIncrementGap(""); + int offsetGap = potentialCustomAnalyzer.getOffsetGap(""); + AnalyzerComponents components = customAnalyzer.getComponents(); + // divide charfilter, tokenizer tokenfilters + CharFilterFactory[] charFilterFactories = components.getCharFilters(); + TokenizerFactory tokenizerFactory = components.getTokenizerFactory(); + TokenFilterFactory[] tokenFilterFactories = components.getTokenFilters(); + String tokenizerName = components.getTokenizerName(); String[][] charFiltersTexts = new String[charFilterFactories != null ? charFilterFactories.length : 0][request.text().length]; TokenListCreator[] tokenFiltersTokenListCreator = new TokenListCreator[tokenFilterFactories != null ? @@ -336,7 +304,7 @@ private static DetailAnalyzeResponse detailAnalyze(AnalyzeRequest request, Analy // analyzing only tokenizer Tokenizer tokenizer = tokenizerFactory.create(); tokenizer.setReader(reader); - tokenizerTokenListCreator.analyze(tokenizer, customAnalyzer, field, includeAttributes); + tokenizerTokenListCreator.analyze(tokenizer, includeAttributes, positionIncrementGap, offsetGap); // analyzing each tokenfilter if (tokenFilterFactories != null) { @@ -346,31 +314,31 @@ private static DetailAnalyzeResponse detailAnalyze(AnalyzeRequest request, Analy } TokenStream stream = createStackedTokenStream(request.text()[textIndex], charFilterFactories, tokenizerFactory, tokenFilterFactories, tokenFilterIndex + 1); - tokenFiltersTokenListCreator[tokenFilterIndex].analyze(stream, customAnalyzer, field, includeAttributes); + tokenFiltersTokenListCreator[tokenFilterIndex].analyze(stream, includeAttributes, positionIncrementGap, offsetGap); } } } - DetailAnalyzeResponse.CharFilteredText[] charFilteredLists = - new DetailAnalyzeResponse.CharFilteredText[charFiltersTexts.length]; + AnalyzeAction.CharFilteredText[] charFilteredLists = + new AnalyzeAction.CharFilteredText[charFiltersTexts.length]; if (charFilterFactories != null) { for (int charFilterIndex = 0; charFilterIndex < charFiltersTexts.length; charFilterIndex++) { - charFilteredLists[charFilterIndex] = new DetailAnalyzeResponse.CharFilteredText( + charFilteredLists[charFilterIndex] = new AnalyzeAction.CharFilteredText( charFilterFactories[charFilterIndex].name(), charFiltersTexts[charFilterIndex]); } } - DetailAnalyzeResponse.AnalyzeTokenList[] tokenFilterLists = - new DetailAnalyzeResponse.AnalyzeTokenList[tokenFiltersTokenListCreator.length]; + AnalyzeAction.AnalyzeTokenList[] tokenFilterLists = + new AnalyzeAction.AnalyzeTokenList[tokenFiltersTokenListCreator.length]; if (tokenFilterFactories != null) { for (int tokenFilterIndex = 0; tokenFilterIndex < tokenFiltersTokenListCreator.length; tokenFilterIndex++) { - tokenFilterLists[tokenFilterIndex] = new DetailAnalyzeResponse.AnalyzeTokenList( + tokenFilterLists[tokenFilterIndex] = new AnalyzeAction.AnalyzeTokenList( tokenFilterFactories[tokenFilterIndex].name(), tokenFiltersTokenListCreator[tokenFilterIndex].getArrayTokens()); } } - detailResponse = new DetailAnalyzeResponse(charFilteredLists, new DetailAnalyzeResponse.AnalyzeTokenList( - customAnalyzer.getTokenizerName(), tokenizerTokenListCreator.getArrayTokens()), tokenFilterLists); + detailResponse = new AnalyzeAction.DetailAnalyzeResponse(charFilteredLists, + new AnalyzeAction.AnalyzeTokenList(tokenizerName, tokenizerTokenListCreator.getArrayTokens()), tokenFilterLists); } else { String name; if (analyzer instanceof NamedAnalyzer) { @@ -381,10 +349,11 @@ private static DetailAnalyzeResponse detailAnalyze(AnalyzeRequest request, Analy TokenListCreator tokenListCreator = new TokenListCreator(maxTokenCount); for (String text : request.text()) { - tokenListCreator.analyze(analyzer.tokenStream(field, text), analyzer, field, - includeAttributes); + tokenListCreator.analyze(analyzer.tokenStream("", text), includeAttributes, analyzer.getPositionIncrementGap(""), + analyzer.getOffsetGap("")); } - detailResponse = new DetailAnalyzeResponse(new DetailAnalyzeResponse.AnalyzeTokenList(name, tokenListCreator.getArrayTokens())); + detailResponse + = new AnalyzeAction.DetailAnalyzeResponse(new AnalyzeAction.AnalyzeTokenList(name, tokenListCreator.getArrayTokens())); } return detailResponse; } @@ -443,7 +412,7 @@ private void increment(){ private static class TokenListCreator { int lastPosition = -1; int lastOffset = 0; - List tokens; + List tokens; private TokenCounter tc; TokenListCreator(int maxTokenCount) { @@ -451,7 +420,7 @@ private static class TokenListCreator { tc = new TokenCounter(maxTokenCount); } - private void analyze(TokenStream stream, Analyzer analyzer, String field, Set includeAttributes) { + private void analyze(TokenStream stream, Set includeAttributes, int positionIncrementGap, int offsetGap) { try { stream.reset(); CharTermAttribute term = stream.addAttribute(CharTermAttribute.class); @@ -465,7 +434,7 @@ private void analyze(TokenStream stream, Analyzer analyzer, String field, Set 0) { lastPosition = lastPosition + increment; } - tokens.add(new AnalyzeResponse.AnalyzeToken(term.toString(), lastPosition, lastOffset + offset.startOffset(), + tokens.add(new AnalyzeAction.AnalyzeToken(term.toString(), lastPosition, lastOffset + offset.startOffset(), lastOffset + offset.endOffset(), posLen.getPositionLength(), type.type(), extractExtendedAttributes(stream, includeAttributes))); tc.increment(); @@ -474,8 +443,8 @@ private void analyze(TokenStream stream, Analyzer analyzer, String field, Set extractExtendedAttributes(TokenStream stream, return extendedAttributes; } - private static List parseCharFilterFactories(AnalyzeRequest request, IndexSettings indexSettings, - AnalysisRegistry analysisRegistry, Environment environment, - boolean normalizer) throws IOException { - List charFilterFactoryList = new ArrayList<>(); - if (request.charFilters() != null && request.charFilters().size() > 0) { - List charFilters = request.charFilters(); - for (AnalyzeRequest.NameOrDefinition charFilter : charFilters) { - CharFilterFactory charFilterFactory; - // parse anonymous settings - if (charFilter.definition != null) { - Settings settings = getAnonymousSettings(charFilter.definition); - String charFilterTypeName = settings.get("type"); - if (charFilterTypeName == null) { - throw new IllegalArgumentException("Missing [type] setting for anonymous char filter: " + charFilter.definition); - } - AnalysisModule.AnalysisProvider charFilterFactoryFactory = - analysisRegistry.getCharFilterProvider(charFilterTypeName); - if (charFilterFactoryFactory == null) { - throw new IllegalArgumentException("failed to find global char filter under [" + charFilterTypeName + "]"); - } - // Need to set anonymous "name" of char_filter - charFilterFactory = charFilterFactoryFactory.get(getNaIndexSettings(settings), environment, "_anonymous_charfilter", - settings); - } else { - AnalysisModule.AnalysisProvider charFilterFactoryFactory; - if (indexSettings == null) { - charFilterFactoryFactory = analysisRegistry.getCharFilterProvider(charFilter.name); - if (charFilterFactoryFactory == null) { - throw new IllegalArgumentException("failed to find global char filter under [" + charFilter.name + "]"); - } - charFilterFactory = charFilterFactoryFactory.get(environment, charFilter.name); - } else { - charFilterFactoryFactory = analysisRegistry.getCharFilterProvider(charFilter.name, indexSettings); - if (charFilterFactoryFactory == null) { - throw new IllegalArgumentException("failed to find char filter under [" + charFilter.name + "]"); - } - charFilterFactory = charFilterFactoryFactory.get(indexSettings, environment, charFilter.name, - AnalysisRegistry.getSettingsFromIndexSettings(indexSettings, - AnalysisRegistry.INDEX_ANALYSIS_CHAR_FILTER + "." + charFilter.name)); - } - } - if (charFilterFactory == null) { - throw new IllegalArgumentException("failed to find char filter under [" + charFilter.name + "]"); - } - if (normalizer) { - if (charFilterFactory instanceof NormalizingCharFilterFactory == false) { - throw new IllegalArgumentException("Custom normalizer may not use char filter [" - + charFilterFactory.name() + "]"); - } - } - charFilterFactoryList.add(charFilterFactory); - } - } - return charFilterFactoryList; - } - - public static class DeferredTokenFilterRegistry implements Function { - - private final AnalysisRegistry analysisRegistry; - private final IndexSettings indexSettings; - Map prebuiltFilters; - - public DeferredTokenFilterRegistry(AnalysisRegistry analysisRegistry, IndexSettings indexSettings) { - this.analysisRegistry = analysisRegistry; - if (indexSettings == null) { - // Settings are null when _analyze is called with no index name, so - // we create dummy settings which will make prebuilt analysis components - // available - Settings settings = Settings.builder() - .put(IndexMetaData.SETTING_VERSION_CREATED, Version.CURRENT) - .put(IndexMetaData.SETTING_NUMBER_OF_REPLICAS, 0) - .put(IndexMetaData.SETTING_NUMBER_OF_SHARDS, 1) - .put(IndexMetaData.SETTING_INDEX_UUID, UUIDs.randomBase64UUID()) - .build(); - IndexMetaData metaData = IndexMetaData.builder(IndexMetaData.INDEX_UUID_NA_VALUE).settings(settings).build(); - indexSettings = new IndexSettings(metaData, Settings.EMPTY); - } - this.indexSettings = indexSettings; - } - - @Override - public TokenFilterFactory apply(String s) { - if (prebuiltFilters == null) { - try { - prebuiltFilters = analysisRegistry.buildTokenFilterFactories(indexSettings); - } catch (IOException e) { - throw new UncheckedIOException(e); - } - } - return prebuiltFilters.get(s); - } - } - - private static List parseTokenFilterFactories(AnalyzeRequest request, IndexSettings indexSettings, - AnalysisRegistry analysisRegistry, Environment environment, - Tuple tokenizerFactory, - List charFilterFactoryList, - boolean normalizer) throws IOException { - List tokenFilterFactoryList = new ArrayList<>(); - DeferredTokenFilterRegistry deferredRegistry = new DeferredTokenFilterRegistry(analysisRegistry, indexSettings); - if (request.tokenFilters() != null && request.tokenFilters().size() > 0) { - List tokenFilters = request.tokenFilters(); - for (AnalyzeRequest.NameOrDefinition tokenFilter : tokenFilters) { - TokenFilterFactory tokenFilterFactory; - // parse anonymous settings - if (tokenFilter.definition != null) { - Settings settings = getAnonymousSettings(tokenFilter.definition); - String filterTypeName = settings.get("type"); - if (filterTypeName == null) { - throw new IllegalArgumentException("Missing [type] setting for anonymous token filter: " + tokenFilter.definition); - } - AnalysisModule.AnalysisProvider tokenFilterFactoryFactory = - analysisRegistry.getTokenFilterProvider(filterTypeName); - if (tokenFilterFactoryFactory == null) { - throw new IllegalArgumentException("failed to find global token filter under [" + filterTypeName + "]"); - } - // Need to set anonymous "name" of tokenfilter - tokenFilterFactory = tokenFilterFactoryFactory.get(getNaIndexSettings(settings), environment, "_anonymous_tokenfilter", - settings); - tokenFilterFactory = tokenFilterFactory.getChainAwareTokenFilterFactory(tokenizerFactory.v2(), charFilterFactoryList, - tokenFilterFactoryList, deferredRegistry); - - } else { - AnalysisModule.AnalysisProvider tokenFilterFactoryFactory; - if (indexSettings == null) { - tokenFilterFactoryFactory = analysisRegistry.getTokenFilterProvider(tokenFilter.name); - if (tokenFilterFactoryFactory == null) { - throw new IllegalArgumentException("failed to find global token filter under [" + tokenFilter.name + "]"); - } - tokenFilterFactory = tokenFilterFactoryFactory.get(environment, tokenFilter.name); - } else { - tokenFilterFactoryFactory = analysisRegistry.getTokenFilterProvider(tokenFilter.name, indexSettings); - if (tokenFilterFactoryFactory == null) { - throw new IllegalArgumentException("failed to find token filter under [" + tokenFilter.name + "]"); - } - Settings settings = AnalysisRegistry.getSettingsFromIndexSettings(indexSettings, - AnalysisRegistry.INDEX_ANALYSIS_FILTER + "." + tokenFilter.name); - tokenFilterFactory = tokenFilterFactoryFactory.get(indexSettings, environment, tokenFilter.name, settings); - tokenFilterFactory = tokenFilterFactory.getChainAwareTokenFilterFactory(tokenizerFactory.v2(), - charFilterFactoryList, tokenFilterFactoryList, deferredRegistry); - } - } - if (tokenFilterFactory == null) { - throw new IllegalArgumentException("failed to find or create token filter under [" + tokenFilter.name + "]"); - } - if (normalizer) { - if (tokenFilterFactory instanceof NormalizingTokenFilterFactory == false) { - throw new IllegalArgumentException("Custom normalizer may not use filter [" - + tokenFilterFactory.name() + "]"); - } - } - tokenFilterFactoryList.add(tokenFilterFactory); - } - } - return tokenFilterFactoryList; - } - - private static Tuple parseTokenizerFactory(AnalyzeRequest request, IndexAnalyzers indexAnalzyers, - AnalysisRegistry analysisRegistry, Environment environment) throws IOException { - String name; - TokenizerFactory tokenizerFactory; - final AnalyzeRequest.NameOrDefinition tokenizer = request.tokenizer(); - // parse anonymous settings - if (tokenizer.definition != null) { - Settings settings = getAnonymousSettings(tokenizer.definition); - String tokenizerTypeName = settings.get("type"); - if (tokenizerTypeName == null) { - throw new IllegalArgumentException("Missing [type] setting for anonymous tokenizer: " + tokenizer.definition); - } - AnalysisModule.AnalysisProvider tokenizerFactoryFactory = - analysisRegistry.getTokenizerProvider(tokenizerTypeName); - if (tokenizerFactoryFactory == null) { - throw new IllegalArgumentException("failed to find global tokenizer under [" + tokenizerTypeName + "]"); - } - // Need to set anonymous "name" of tokenizer - name = "_anonymous_tokenizer"; - tokenizerFactory = tokenizerFactoryFactory.get(getNaIndexSettings(settings), environment, "_anonymous_tokenizer", settings); - } else { - AnalysisModule.AnalysisProvider tokenizerFactoryFactory; - if (indexAnalzyers == null) { - tokenizerFactory = getTokenizerFactory(analysisRegistry, environment, tokenizer.name); - name = tokenizer.name; - } else { - tokenizerFactoryFactory = analysisRegistry.getTokenizerProvider(tokenizer.name, indexAnalzyers.getIndexSettings()); - if (tokenizerFactoryFactory == null) { - throw new IllegalArgumentException("failed to find tokenizer under [" + tokenizer.name + "]"); - } - name = tokenizer.name; - tokenizerFactory = tokenizerFactoryFactory.get(indexAnalzyers.getIndexSettings(), environment, tokenizer.name, - AnalysisRegistry.getSettingsFromIndexSettings(indexAnalzyers.getIndexSettings(), - AnalysisRegistry.INDEX_ANALYSIS_TOKENIZER + "." + tokenizer.name)); - } - } - return new Tuple<>(name, tokenizerFactory); - } - - private static TokenizerFactory getTokenizerFactory(AnalysisRegistry analysisRegistry, Environment environment, - String name) throws IOException { - AnalysisModule.AnalysisProvider tokenizerFactoryFactory; - TokenizerFactory tokenizerFactory; - tokenizerFactoryFactory = analysisRegistry.getTokenizerProvider(name); - if (tokenizerFactoryFactory == null) { - throw new IllegalArgumentException("failed to find global tokenizer under [" + name + "]"); - } - tokenizerFactory = tokenizerFactoryFactory.get(environment, name); - return tokenizerFactory; - } - - private static IndexSettings getNaIndexSettings(Settings settings) { - IndexMetaData metaData = IndexMetaData.builder(IndexMetaData.INDEX_UUID_NA_VALUE).settings(settings).build(); - return new IndexSettings(metaData, Settings.EMPTY); - } - - private static Settings getAnonymousSettings(Settings providerSetting) { - return Settings.builder().put(providerSetting) - // for _na_ - .put(IndexMetaData.SETTING_VERSION_CREATED, Version.CURRENT) - .put(IndexMetaData.SETTING_NUMBER_OF_REPLICAS, 0) - .put(IndexMetaData.SETTING_NUMBER_OF_SHARDS, 1) - .put(IndexMetaData.SETTING_INDEX_UUID, UUIDs.randomBase64UUID()) - .build(); - } - } diff --git a/server/src/main/java/org/elasticsearch/action/admin/indices/cache/clear/ClearIndicesCacheAction.java b/server/src/main/java/org/elasticsearch/action/admin/indices/cache/clear/ClearIndicesCacheAction.java index e5bdd53bdc7a7..8f020df00edda 100644 --- a/server/src/main/java/org/elasticsearch/action/admin/indices/cache/clear/ClearIndicesCacheAction.java +++ b/server/src/main/java/org/elasticsearch/action/admin/indices/cache/clear/ClearIndicesCacheAction.java @@ -19,9 +19,9 @@ package org.elasticsearch.action.admin.indices.cache.clear; -import org.elasticsearch.action.Action; +import org.elasticsearch.action.StreamableResponseActionType; -public class ClearIndicesCacheAction extends Action { +public class ClearIndicesCacheAction extends StreamableResponseActionType { public static final ClearIndicesCacheAction INSTANCE = new ClearIndicesCacheAction(); public static final String NAME = "indices:admin/cache/clear"; diff --git a/server/src/main/java/org/elasticsearch/action/admin/indices/close/CloseIndexAction.java b/server/src/main/java/org/elasticsearch/action/admin/indices/close/CloseIndexAction.java index 5c3d60dd44013..f2a6cff2d3949 100644 --- a/server/src/main/java/org/elasticsearch/action/admin/indices/close/CloseIndexAction.java +++ b/server/src/main/java/org/elasticsearch/action/admin/indices/close/CloseIndexAction.java @@ -19,9 +19,10 @@ package org.elasticsearch.action.admin.indices.close; -import org.elasticsearch.action.Action; +import org.elasticsearch.action.ActionType; +import org.elasticsearch.common.io.stream.Writeable; -public class CloseIndexAction extends Action { +public class CloseIndexAction extends ActionType { public static final CloseIndexAction INSTANCE = new CloseIndexAction(); public static final String NAME = "indices:admin/close"; @@ -31,7 +32,7 @@ private CloseIndexAction() { } @Override - public CloseIndexResponse newResponse() { - return new CloseIndexResponse(); + public Writeable.Reader getResponseReader() { + return CloseIndexResponse::new; } } diff --git a/server/src/main/java/org/elasticsearch/action/admin/indices/close/CloseIndexResponse.java b/server/src/main/java/org/elasticsearch/action/admin/indices/close/CloseIndexResponse.java index c653c264e95db..925b9dd4d5231 100644 --- a/server/src/main/java/org/elasticsearch/action/admin/indices/close/CloseIndexResponse.java +++ b/server/src/main/java/org/elasticsearch/action/admin/indices/close/CloseIndexResponse.java @@ -40,9 +40,15 @@ public class CloseIndexResponse extends ShardsAcknowledgedResponse { - private List indices; + private final List indices; - CloseIndexResponse() { + CloseIndexResponse(StreamInput in) throws IOException { + super(in, in.getVersion().onOrAfter(Version.V_7_2_0)); + if (in.getVersion().onOrAfter(Version.V_7_3_0)) { + indices = unmodifiableList(in.readList(IndexResult::new)); + } else { + indices = unmodifiableList(emptyList()); + } } public CloseIndexResponse(final boolean acknowledged, final boolean shardsAcknowledged, final List indices) { @@ -54,19 +60,6 @@ public List getIndices() { return indices; } - @Override - public void readFrom(StreamInput in) throws IOException { - super.readFrom(in); - if (in.getVersion().onOrAfter(Version.V_7_2_0)) { - readShardsAcknowledged(in); - } - if (in.getVersion().onOrAfter(Version.V_7_3_0)) { - indices = unmodifiableList(in.readList(IndexResult::new)); - } else { - indices = unmodifiableList(emptyList()); - } - } - @Override public void writeTo(StreamOutput out) throws IOException { super.writeTo(out); @@ -78,6 +71,7 @@ public void writeTo(StreamOutput out) throws IOException { } } + @Override protected void addCustomFields(final XContentBuilder builder, final Params params) throws IOException { super.addCustomFields(builder, params); builder.startObject("indices"); @@ -190,7 +184,7 @@ public String toString() { public static class ShardResult implements Writeable, ToXContentFragment { private final int id; - private final ShardResult.Failure[] failures; + private final Failure[] failures; public ShardResult(final int id, final Failure[] failures) { this.id = id; @@ -199,7 +193,7 @@ public ShardResult(final int id, final Failure[] failures) { ShardResult(final StreamInput in) throws IOException { this.id = in.readVInt(); - this.failures = in.readOptionalArray(Failure::readFailure, ShardResult.Failure[]::new); + this.failures = in.readOptionalArray(Failure::readFailure, Failure[]::new); } @Override @@ -227,9 +221,7 @@ public XContentBuilder toXContent(final XContentBuilder builder, final Params pa builder.startArray("failures"); if (failures != null) { for (Failure failure : failures) { - builder.startObject(); failure.toXContent(builder, params); - builder.endObject(); } } builder.endArray(); @@ -242,7 +234,7 @@ public String toString() { return Strings.toString(this); } - public static class Failure extends DefaultShardOperationFailedException implements Writeable { + public static class Failure extends DefaultShardOperationFailedException { private @Nullable String nodeId; @@ -275,11 +267,11 @@ public void writeTo(final StreamOutput out) throws IOException { } @Override - public XContentBuilder toXContent(final XContentBuilder builder, final Params params) throws IOException { + public XContentBuilder innerToXContent(final XContentBuilder builder, final Params params) throws IOException { if (nodeId != null) { builder.field("node", nodeId); } - return super.toXContent(builder, params); + return super.innerToXContent(builder, params); } @Override diff --git a/server/src/main/java/org/elasticsearch/action/admin/indices/close/TransportCloseIndexAction.java b/server/src/main/java/org/elasticsearch/action/admin/indices/close/TransportCloseIndexAction.java index 3c231d13845b2..d653f000d0bc5 100644 --- a/server/src/main/java/org/elasticsearch/action/admin/indices/close/TransportCloseIndexAction.java +++ b/server/src/main/java/org/elasticsearch/action/admin/indices/close/TransportCloseIndexAction.java @@ -31,6 +31,7 @@ import org.elasticsearch.cluster.metadata.MetaDataIndexStateService; import org.elasticsearch.cluster.service.ClusterService; import org.elasticsearch.common.inject.Inject; +import org.elasticsearch.common.io.stream.StreamInput; import org.elasticsearch.common.settings.ClusterSettings; import org.elasticsearch.common.settings.Setting; import org.elasticsearch.common.settings.Setting.Property; @@ -40,6 +41,7 @@ import org.elasticsearch.threadpool.ThreadPool; import org.elasticsearch.transport.TransportService; +import java.io.IOException; import java.util.Collections; /** @@ -78,7 +80,12 @@ protected String executor() { @Override protected CloseIndexResponse newResponse() { - return new CloseIndexResponse(); + throw new UnsupportedOperationException("usage of Streamable is to be replaced by Writeable"); + } + + @Override + protected CloseIndexResponse read(StreamInput in) throws IOException { + return new CloseIndexResponse(in); } @Override @@ -97,13 +104,6 @@ protected ClusterBlockException checkBlock(CloseIndexRequest request, ClusterSta indexNameExpressionResolver.concreteIndexNames(state, request)); } - @Override - protected void masterOperation(final CloseIndexRequest request, - final ClusterState state, - final ActionListener listener) { - throw new UnsupportedOperationException("The task parameter is required"); - } - @Override protected void masterOperation(final Task task, final CloseIndexRequest request, diff --git a/server/src/main/java/org/elasticsearch/action/admin/indices/close/TransportVerifyShardBeforeCloseAction.java b/server/src/main/java/org/elasticsearch/action/admin/indices/close/TransportVerifyShardBeforeCloseAction.java index 22a0777f7bffb..79cbab4781947 100644 --- a/server/src/main/java/org/elasticsearch/action/admin/indices/close/TransportVerifyShardBeforeCloseAction.java +++ b/server/src/main/java/org/elasticsearch/action/admin/indices/close/TransportVerifyShardBeforeCloseAction.java @@ -20,6 +20,7 @@ import org.apache.logging.log4j.LogManager; import org.apache.logging.log4j.Logger; +import org.elasticsearch.Version; import org.elasticsearch.action.ActionListener; import org.elasticsearch.action.admin.indices.flush.FlushRequest; import org.elasticsearch.action.support.ActionFilters; @@ -94,12 +95,12 @@ protected void shardOperationOnPrimary(final ShardRequest shardRequest, final In } @Override - protected ReplicaResult shardOperationOnReplica(final ShardRequest shardRequest, final IndexShard replica) { + protected ReplicaResult shardOperationOnReplica(final ShardRequest shardRequest, final IndexShard replica) throws IOException { executeShardOperation(shardRequest, replica); return new ReplicaResult(); } - private void executeShardOperation(final ShardRequest request, final IndexShard indexShard) { + private void executeShardOperation(final ShardRequest request, final IndexShard indexShard) throws IOException { final ShardId shardId = indexShard.shardId(); if (indexShard.getActiveOperationsCount() != IndexShard.OPERATIONS_BLOCKED) { throw new IllegalStateException("Index shard " + shardId + " is not blocking all operations during closing"); @@ -109,9 +110,19 @@ private void executeShardOperation(final ShardRequest request, final IndexShard if (clusterBlocks.hasIndexBlock(shardId.getIndexName(), request.clusterBlock()) == false) { throw new IllegalStateException("Index shard " + shardId + " must be blocked by " + request.clusterBlock() + " before closing"); } - indexShard.verifyShardBeforeIndexClosing(); - indexShard.flush(new FlushRequest().force(true).waitIfOngoing(true)); - logger.trace("{} shard is ready for closing", shardId); + if (request.isPhase1()) { + // in order to advance the global checkpoint to the maximum sequence number, the (persisted) local checkpoint needs to be + // advanced first, which, when using async translog syncing, does not automatically hold at the time where we have acquired + // all operation permits. Instead, this requires and explicit sync, which communicates the updated (persisted) local checkpoint + // to the primary (we call this phase1), and phase2 can then use the fact that the global checkpoint has moved to the maximum + // sequence number to pass the verifyShardBeforeIndexClosing check and create a safe commit where the maximum sequence number + // is equal to the global checkpoint. + indexShard.sync(); + } else { + indexShard.verifyShardBeforeIndexClosing(); + indexShard.flush(new FlushRequest().force(true).waitIfOngoing(true)); + logger.trace("{} shard is ready for closing", shardId); + } } @Override @@ -136,14 +147,22 @@ public static class ShardRequest extends ReplicationRequest { private final ClusterBlock clusterBlock; + private final boolean phase1; + ShardRequest(StreamInput in) throws IOException { super(in); clusterBlock = new ClusterBlock(in); + if (in.getVersion().onOrAfter(Version.V_7_3_0)) { + phase1 = in.readBoolean(); + } else { + phase1 = false; + } } - public ShardRequest(final ShardId shardId, final ClusterBlock clusterBlock, final TaskId parentTaskId) { + public ShardRequest(final ShardId shardId, final ClusterBlock clusterBlock, final boolean phase1, final TaskId parentTaskId) { super(shardId); this.clusterBlock = Objects.requireNonNull(clusterBlock); + this.phase1 = phase1; setParentTask(parentTaskId); } @@ -161,10 +180,17 @@ public void readFrom(final StreamInput in) { public void writeTo(final StreamOutput out) throws IOException { super.writeTo(out); clusterBlock.writeTo(out); + if (out.getVersion().onOrAfter(Version.V_7_3_0)) { + out.writeBoolean(phase1); + } } public ClusterBlock clusterBlock() { return clusterBlock; } + + public boolean isPhase1() { + return phase1; + } } } diff --git a/server/src/main/java/org/elasticsearch/action/admin/indices/create/CreateIndexAction.java b/server/src/main/java/org/elasticsearch/action/admin/indices/create/CreateIndexAction.java index 3993ea7256f61..cf7c7d08e0f72 100644 --- a/server/src/main/java/org/elasticsearch/action/admin/indices/create/CreateIndexAction.java +++ b/server/src/main/java/org/elasticsearch/action/admin/indices/create/CreateIndexAction.java @@ -19,9 +19,10 @@ package org.elasticsearch.action.admin.indices.create; -import org.elasticsearch.action.Action; +import org.elasticsearch.action.ActionType; +import org.elasticsearch.common.io.stream.Writeable; -public class CreateIndexAction extends Action { +public class CreateIndexAction extends ActionType { public static final CreateIndexAction INSTANCE = new CreateIndexAction(); public static final String NAME = "indices:admin/create"; @@ -31,7 +32,7 @@ private CreateIndexAction() { } @Override - public CreateIndexResponse newResponse() { - return new CreateIndexResponse(); + public Writeable.Reader getResponseReader() { + return CreateIndexResponse::new; } } diff --git a/server/src/main/java/org/elasticsearch/action/admin/indices/create/CreateIndexResponse.java b/server/src/main/java/org/elasticsearch/action/admin/indices/create/CreateIndexResponse.java index 79192693620dd..54142c2a90465 100644 --- a/server/src/main/java/org/elasticsearch/action/admin/indices/create/CreateIndexResponse.java +++ b/server/src/main/java/org/elasticsearch/action/admin/indices/create/CreateIndexResponse.java @@ -52,22 +52,18 @@ protected static void declareFields(Constructing objectParser.declareField(constructorArg(), (parser, context) -> parser.textOrNull(), INDEX, ObjectParser.ValueType.STRING_OR_NULL); } - private String index; + private final String index; - public CreateIndexResponse() {} + protected CreateIndexResponse(StreamInput in) throws IOException { + super(in, true); + index = in.readString(); + } - protected CreateIndexResponse(boolean acknowledged, boolean shardsAcknowledged, String index) { + public CreateIndexResponse(boolean acknowledged, boolean shardsAcknowledged, String index) { super(acknowledged, shardsAcknowledged); this.index = index; } - @Override - public void readFrom(StreamInput in) throws IOException { - super.readFrom(in); - readShardsAcknowledged(in); - index = in.readString(); - } - @Override public void writeTo(StreamOutput out) throws IOException { super.writeTo(out); diff --git a/server/src/main/java/org/elasticsearch/action/admin/indices/create/TransportCreateIndexAction.java b/server/src/main/java/org/elasticsearch/action/admin/indices/create/TransportCreateIndexAction.java index 03bbd0903a6d5..fdf8cc0c00c25 100644 --- a/server/src/main/java/org/elasticsearch/action/admin/indices/create/TransportCreateIndexAction.java +++ b/server/src/main/java/org/elasticsearch/action/admin/indices/create/TransportCreateIndexAction.java @@ -29,9 +29,13 @@ import org.elasticsearch.cluster.metadata.MetaDataCreateIndexService; import org.elasticsearch.cluster.service.ClusterService; import org.elasticsearch.common.inject.Inject; +import org.elasticsearch.common.io.stream.StreamInput; +import org.elasticsearch.tasks.Task; import org.elasticsearch.threadpool.ThreadPool; import org.elasticsearch.transport.TransportService; +import java.io.IOException; + /** * Create index action. */ @@ -56,7 +60,12 @@ protected String executor() { @Override protected CreateIndexResponse newResponse() { - return new CreateIndexResponse(); + throw new UnsupportedOperationException("usage of Streamable is to be replaced by Writeable"); + } + + @Override + protected CreateIndexResponse read(StreamInput in) throws IOException { + return new CreateIndexResponse(in); } @Override @@ -65,7 +74,7 @@ protected ClusterBlockException checkBlock(CreateIndexRequest request, ClusterSt } @Override - protected void masterOperation(final CreateIndexRequest request, final ClusterState state, + protected void masterOperation(Task task, final CreateIndexRequest request, final ClusterState state, final ActionListener listener) { String cause = request.cause(); if (cause.length() == 0) { diff --git a/server/src/main/java/org/elasticsearch/action/admin/indices/delete/DeleteIndexAction.java b/server/src/main/java/org/elasticsearch/action/admin/indices/delete/DeleteIndexAction.java index 6fde9089cef62..7e3a812223e78 100644 --- a/server/src/main/java/org/elasticsearch/action/admin/indices/delete/DeleteIndexAction.java +++ b/server/src/main/java/org/elasticsearch/action/admin/indices/delete/DeleteIndexAction.java @@ -19,10 +19,11 @@ package org.elasticsearch.action.admin.indices.delete; -import org.elasticsearch.action.Action; +import org.elasticsearch.action.ActionType; import org.elasticsearch.action.support.master.AcknowledgedResponse; +import org.elasticsearch.common.io.stream.Writeable; -public class DeleteIndexAction extends Action { +public class DeleteIndexAction extends ActionType { public static final DeleteIndexAction INSTANCE = new DeleteIndexAction(); public static final String NAME = "indices:admin/delete"; @@ -32,7 +33,7 @@ private DeleteIndexAction() { } @Override - public AcknowledgedResponse newResponse() { - return new AcknowledgedResponse(); + public Writeable.Reader getResponseReader() { + return AcknowledgedResponse::new; } } diff --git a/server/src/main/java/org/elasticsearch/action/admin/indices/delete/TransportDeleteIndexAction.java b/server/src/main/java/org/elasticsearch/action/admin/indices/delete/TransportDeleteIndexAction.java index 50f65312afbf0..384452ab8c945 100644 --- a/server/src/main/java/org/elasticsearch/action/admin/indices/delete/TransportDeleteIndexAction.java +++ b/server/src/main/java/org/elasticsearch/action/admin/indices/delete/TransportDeleteIndexAction.java @@ -32,11 +32,13 @@ import org.elasticsearch.cluster.metadata.MetaDataDeleteIndexService; import org.elasticsearch.cluster.service.ClusterService; import org.elasticsearch.common.inject.Inject; +import org.elasticsearch.common.io.stream.StreamInput; import org.elasticsearch.index.Index; import org.elasticsearch.tasks.Task; import org.elasticsearch.threadpool.ThreadPool; import org.elasticsearch.transport.TransportService; +import java.io.IOException; import java.util.Arrays; import java.util.HashSet; import java.util.Set; @@ -65,9 +67,14 @@ protected String executor() { return ThreadPool.Names.SAME; } + @Override + protected AcknowledgedResponse read(StreamInput in) throws IOException { + return new AcknowledgedResponse(in); + } + @Override protected AcknowledgedResponse newResponse() { - return new AcknowledgedResponse(); + throw new UnsupportedOperationException("usage of Streamable is to be replaced by Writeable"); } @Override @@ -82,7 +89,7 @@ protected ClusterBlockException checkBlock(DeleteIndexRequest request, ClusterSt } @Override - protected void masterOperation(final DeleteIndexRequest request, final ClusterState state, + protected void masterOperation(Task task, final DeleteIndexRequest request, final ClusterState state, final ActionListener listener) { final Set concreteIndices = new HashSet<>(Arrays.asList(indexNameExpressionResolver.concreteIndices(state, request))); if (concreteIndices.isEmpty()) { diff --git a/server/src/main/java/org/elasticsearch/action/admin/indices/exists/indices/IndicesExistsRequest.java b/server/src/main/java/org/elasticsearch/action/admin/indices/exists/indices/IndicesExistsRequest.java deleted file mode 100644 index 0c99175387db0..0000000000000 --- a/server/src/main/java/org/elasticsearch/action/admin/indices/exists/indices/IndicesExistsRequest.java +++ /dev/null @@ -1,102 +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.action.admin.indices.exists.indices; - -import org.elasticsearch.action.ActionRequestValidationException; -import org.elasticsearch.action.IndicesRequest; -import org.elasticsearch.action.support.IndicesOptions; -import org.elasticsearch.action.support.master.MasterNodeReadRequest; -import org.elasticsearch.common.Strings; -import org.elasticsearch.common.io.stream.StreamInput; -import org.elasticsearch.common.io.stream.StreamOutput; - -import java.io.IOException; - -import static org.elasticsearch.action.ValidateActions.addValidationError; - -public class IndicesExistsRequest extends MasterNodeReadRequest implements IndicesRequest.Replaceable { - - private String[] indices = Strings.EMPTY_ARRAY; - private IndicesOptions indicesOptions = IndicesOptions.fromOptions(false, false, true, true); - - // for serialization - public IndicesExistsRequest() { - - } - - public IndicesExistsRequest(String... indices) { - this.indices = indices; - } - - public IndicesExistsRequest(StreamInput in) throws IOException { - super(in); - indices = in.readStringArray(); - indicesOptions = IndicesOptions.readIndicesOptions(in); - } - - @Override - public void writeTo(StreamOutput out) throws IOException { - super.writeTo(out); - out.writeStringArray(indices); - indicesOptions.writeIndicesOptions(out); - } - - @Override - public String[] indices() { - return indices; - } - - @Override - public IndicesExistsRequest indices(String... indices) { - this.indices = indices; - return this; - } - - @Override - public IndicesOptions indicesOptions() { - return indicesOptions; - } - - public IndicesExistsRequest expandWilcardsOpen(boolean expandWildcardsOpen) { - this.indicesOptions = IndicesOptions.fromOptions(indicesOptions.ignoreUnavailable(), indicesOptions.allowNoIndices(), - expandWildcardsOpen, indicesOptions.expandWildcardsClosed()); - return this; - } - - public IndicesExistsRequest expandWilcardsClosed(boolean expandWildcardsClosed) { - this.indicesOptions = IndicesOptions.fromOptions(indicesOptions.ignoreUnavailable(), indicesOptions.allowNoIndices(), - indicesOptions.expandWildcardsOpen(), expandWildcardsClosed); - return this; - } - - @Override - public ActionRequestValidationException validate() { - ActionRequestValidationException validationException = null; - if (indices == null || indices.length == 0) { - validationException = addValidationError("index/indices is missing", validationException); - } - return validationException; - } - - @Override - public void readFrom(StreamInput in) throws IOException { - throw new UnsupportedOperationException("usage of Streamable is to be replaced by Writeable"); - } -} diff --git a/server/src/main/java/org/elasticsearch/action/admin/indices/exists/indices/IndicesExistsRequestBuilder.java b/server/src/main/java/org/elasticsearch/action/admin/indices/exists/indices/IndicesExistsRequestBuilder.java deleted file mode 100644 index 39a2ca7ef0a13..0000000000000 --- a/server/src/main/java/org/elasticsearch/action/admin/indices/exists/indices/IndicesExistsRequestBuilder.java +++ /dev/null @@ -1,52 +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.action.admin.indices.exists.indices; - -import org.elasticsearch.action.support.master.MasterNodeReadOperationRequestBuilder; -import org.elasticsearch.client.ElasticsearchClient; - -public class IndicesExistsRequestBuilder - extends MasterNodeReadOperationRequestBuilder { - - public IndicesExistsRequestBuilder(ElasticsearchClient client, IndicesExistsAction action, String... indices) { - super(client, action, new IndicesExistsRequest(indices)); - } - - public IndicesExistsRequestBuilder setIndices(String... indices) { - request.indices(indices); - return this; - } - - /** - * Controls whether wildcard expressions will be expanded to existing open indices - */ - public IndicesExistsRequestBuilder setExpandWildcardsOpen(boolean expandWildcardsOpen) { - request.expandWilcardsOpen(expandWildcardsOpen); - return this; - } - - /** - * Controls whether wildcard expressions will be expanded to existing closed indices - */ - public IndicesExistsRequestBuilder setExpandWildcardsClosed(boolean expandWildcardsClosed) { - request.expandWilcardsClosed(expandWildcardsClosed); - return this; - } -} diff --git a/server/src/main/java/org/elasticsearch/action/admin/indices/exists/indices/TransportIndicesExistsAction.java b/server/src/main/java/org/elasticsearch/action/admin/indices/exists/indices/TransportIndicesExistsAction.java deleted file mode 100644 index ba4d5eeea69d6..0000000000000 --- a/server/src/main/java/org/elasticsearch/action/admin/indices/exists/indices/TransportIndicesExistsAction.java +++ /dev/null @@ -1,82 +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.action.admin.indices.exists.indices; - -import org.elasticsearch.action.ActionListener; -import org.elasticsearch.action.support.ActionFilters; -import org.elasticsearch.action.support.IndicesOptions; -import org.elasticsearch.action.support.master.TransportMasterNodeReadAction; -import org.elasticsearch.cluster.ClusterState; -import org.elasticsearch.cluster.block.ClusterBlockException; -import org.elasticsearch.cluster.block.ClusterBlockLevel; -import org.elasticsearch.cluster.metadata.IndexNameExpressionResolver; -import org.elasticsearch.cluster.service.ClusterService; -import org.elasticsearch.common.inject.Inject; -import org.elasticsearch.index.IndexNotFoundException; -import org.elasticsearch.threadpool.ThreadPool; -import org.elasticsearch.transport.TransportService; - -/** - * Indices exists action. - */ -public class TransportIndicesExistsAction extends TransportMasterNodeReadAction { - - @Inject - public TransportIndicesExistsAction(TransportService transportService, ClusterService clusterService, - ThreadPool threadPool, ActionFilters actionFilters, - IndexNameExpressionResolver indexNameExpressionResolver) { - super(IndicesExistsAction.NAME, transportService, clusterService, threadPool, actionFilters, IndicesExistsRequest::new, - indexNameExpressionResolver); - } - - @Override - protected String executor() { - // lightweight in memory check - return ThreadPool.Names.SAME; - } - - @Override - protected IndicesExistsResponse newResponse() { - return new IndicesExistsResponse(); - } - - @Override - protected ClusterBlockException checkBlock(IndicesExistsRequest request, ClusterState state) { - //make sure through indices options that the concrete indices call never throws IndexMissingException - IndicesOptions indicesOptions = IndicesOptions.fromOptions(true, true, request.indicesOptions().expandWildcardsOpen(), - request.indicesOptions().expandWildcardsClosed()); - return state.blocks().indicesBlockedException(ClusterBlockLevel.METADATA_READ, - indexNameExpressionResolver.concreteIndexNames(state, indicesOptions, request.indices())); - } - - @Override - protected void masterOperation(final IndicesExistsRequest request, final ClusterState state, - final ActionListener listener) { - boolean exists; - try { - // Similar as the previous behaviour, but now also aliases and wildcards are supported. - indexNameExpressionResolver.concreteIndexNames(state, request); - exists = true; - } catch (IndexNotFoundException e) { - exists = false; - } - listener.onResponse(new IndicesExistsResponse(exists)); - } -} diff --git a/server/src/main/java/org/elasticsearch/action/admin/indices/exists/types/TransportTypesExistsAction.java b/server/src/main/java/org/elasticsearch/action/admin/indices/exists/types/TransportTypesExistsAction.java deleted file mode 100644 index 9b9c0ca7b1640..0000000000000 --- a/server/src/main/java/org/elasticsearch/action/admin/indices/exists/types/TransportTypesExistsAction.java +++ /dev/null @@ -1,95 +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.action.admin.indices.exists.types; - -import org.elasticsearch.action.ActionListener; -import org.elasticsearch.action.support.ActionFilters; -import org.elasticsearch.action.support.master.TransportMasterNodeReadAction; -import org.elasticsearch.cluster.ClusterState; -import org.elasticsearch.cluster.block.ClusterBlockException; -import org.elasticsearch.cluster.block.ClusterBlockLevel; -import org.elasticsearch.cluster.metadata.IndexNameExpressionResolver; -import org.elasticsearch.cluster.metadata.MappingMetaData; -import org.elasticsearch.cluster.service.ClusterService; -import org.elasticsearch.common.inject.Inject; -import org.elasticsearch.threadpool.ThreadPool; -import org.elasticsearch.transport.TransportService; - -/** - * Types exists transport action. - */ -public class TransportTypesExistsAction extends TransportMasterNodeReadAction { - - @Inject - public TransportTypesExistsAction(TransportService transportService, ClusterService clusterService, - ThreadPool threadPool, ActionFilters actionFilters, - IndexNameExpressionResolver indexNameExpressionResolver) { - super(TypesExistsAction.NAME, transportService, clusterService, threadPool, actionFilters, TypesExistsRequest::new, - indexNameExpressionResolver); - } - - @Override - protected String executor() { - // lightweight check - return ThreadPool.Names.SAME; - } - - @Override - protected TypesExistsResponse newResponse() { - return new TypesExistsResponse(); - } - - @Override - protected ClusterBlockException checkBlock(TypesExistsRequest request, ClusterState state) { - return state.blocks().indicesBlockedException(ClusterBlockLevel.METADATA_READ, - indexNameExpressionResolver.concreteIndexNames(state, request)); - } - - @Override - protected void masterOperation(final TypesExistsRequest request, final ClusterState state, - final ActionListener listener) { - String[] concreteIndices = indexNameExpressionResolver.concreteIndexNames(state, request.indicesOptions(), request.indices()); - if (concreteIndices.length == 0) { - listener.onResponse(new TypesExistsResponse(false)); - return; - } - - for (String concreteIndex : concreteIndices) { - if (!state.metaData().hasConcreteIndex(concreteIndex)) { - listener.onResponse(new TypesExistsResponse(false)); - return; - } - - MappingMetaData mapping = state.metaData().getIndices().get(concreteIndex).mapping(); - if (mapping == null) { - listener.onResponse(new TypesExistsResponse(false)); - return; - } - - for (String type : request.types()) { - if (mapping.type().equals(type) == false) { - listener.onResponse(new TypesExistsResponse(false)); - return; - } - } - } - - listener.onResponse(new TypesExistsResponse(true)); - } -} diff --git a/server/src/main/java/org/elasticsearch/action/admin/indices/exists/types/TypesExistsRequest.java b/server/src/main/java/org/elasticsearch/action/admin/indices/exists/types/TypesExistsRequest.java deleted file mode 100644 index 851f175f4a4b7..0000000000000 --- a/server/src/main/java/org/elasticsearch/action/admin/indices/exists/types/TypesExistsRequest.java +++ /dev/null @@ -1,108 +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.action.admin.indices.exists.types; - -import org.elasticsearch.action.ActionRequestValidationException; -import org.elasticsearch.action.IndicesRequest; -import org.elasticsearch.action.support.IndicesOptions; -import org.elasticsearch.action.support.master.MasterNodeReadRequest; -import org.elasticsearch.common.io.stream.StreamInput; -import org.elasticsearch.common.io.stream.StreamOutput; - -import java.io.IOException; - -import static org.elasticsearch.action.ValidateActions.addValidationError; - -public class TypesExistsRequest extends MasterNodeReadRequest implements IndicesRequest.Replaceable { - - private String[] indices; - private String[] types; - - private IndicesOptions indicesOptions = IndicesOptions.strictExpandOpen(); - - public TypesExistsRequest() { - } - - public TypesExistsRequest(String[] indices, String... types) { - this.indices = indices; - this.types = types; - } - - public TypesExistsRequest(StreamInput in) throws IOException { - super(in); - indices = in.readStringArray(); - types = in.readStringArray(); - indicesOptions = IndicesOptions.readIndicesOptions(in); - } - - @Override - public void writeTo(StreamOutput out) throws IOException { - super.writeTo(out); - out.writeStringArray(indices); - out.writeStringArray(types); - indicesOptions.writeIndicesOptions(out); - } - - @Override - public String[] indices() { - return indices; - } - - @Override - public TypesExistsRequest indices(String... indices) { - this.indices = indices; - return this; - } - - public String[] types() { - return types; - } - - public void types(String[] types) { - this.types = types; - } - - @Override - public IndicesOptions indicesOptions() { - return indicesOptions; - } - - public TypesExistsRequest indicesOptions(IndicesOptions indicesOptions) { - this.indicesOptions = indicesOptions; - return this; - } - - @Override - public ActionRequestValidationException validate() { - ActionRequestValidationException validationException = null; - if (indices == null) { // Specifying '*' via rest api results in an empty array - validationException = addValidationError("index/indices is missing", validationException); - } - if (types == null || types.length == 0) { - validationException = addValidationError("type/types is missing", validationException); - } - - return validationException; - } - - @Override - public void readFrom(StreamInput in) throws IOException { - throw new UnsupportedOperationException("usage of Streamable is to be replaced by Writeable"); - } -} diff --git a/server/src/main/java/org/elasticsearch/action/admin/indices/exists/types/TypesExistsRequestBuilder.java b/server/src/main/java/org/elasticsearch/action/admin/indices/exists/types/TypesExistsRequestBuilder.java deleted file mode 100644 index 607a423605bfc..0000000000000 --- a/server/src/main/java/org/elasticsearch/action/admin/indices/exists/types/TypesExistsRequestBuilder.java +++ /dev/null @@ -1,67 +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.action.admin.indices.exists.types; - -import org.elasticsearch.action.support.IndicesOptions; -import org.elasticsearch.action.support.master.MasterNodeReadOperationRequestBuilder; -import org.elasticsearch.client.ElasticsearchClient; -import org.elasticsearch.common.Strings; - -/** - * A builder for {@link TypesExistsRequest}. - */ -@Deprecated -public class TypesExistsRequestBuilder - extends MasterNodeReadOperationRequestBuilder { - - /** - * @param indices What indices to check for types - */ - public TypesExistsRequestBuilder(ElasticsearchClient client, TypesExistsAction action, String... indices) { - super(client, action, new TypesExistsRequest(indices, Strings.EMPTY_ARRAY)); - } - - TypesExistsRequestBuilder(ElasticsearchClient client, TypesExistsAction action) { - super(client, action, new TypesExistsRequest()); - } - - /** - * @param indices What indices to check for types - */ - public TypesExistsRequestBuilder setIndices(String[] indices) { - request.indices(indices); - return this; - } - - /** - * @param types The types to check if they exist - */ - public TypesExistsRequestBuilder setTypes(String... types) { - request.types(types); - return this; - } - - /** - * @param indicesOptions Specifies how to resolve indices that aren't active / ready and indices wildcard expressions - */ - public TypesExistsRequestBuilder setIndicesOptions(IndicesOptions indicesOptions) { - request.indicesOptions(indicesOptions); - return this; - } -} diff --git a/server/src/main/java/org/elasticsearch/action/admin/indices/flush/FlushAction.java b/server/src/main/java/org/elasticsearch/action/admin/indices/flush/FlushAction.java index 60d5b43a6c189..aab1dc775b99e 100644 --- a/server/src/main/java/org/elasticsearch/action/admin/indices/flush/FlushAction.java +++ b/server/src/main/java/org/elasticsearch/action/admin/indices/flush/FlushAction.java @@ -19,9 +19,9 @@ package org.elasticsearch.action.admin.indices.flush; -import org.elasticsearch.action.Action; +import org.elasticsearch.action.StreamableResponseActionType; -public class FlushAction extends Action { +public class FlushAction extends StreamableResponseActionType { public static final FlushAction INSTANCE = new FlushAction(); public static final String NAME = "indices:admin/flush"; diff --git a/server/src/main/java/org/elasticsearch/action/admin/indices/flush/SyncedFlushAction.java b/server/src/main/java/org/elasticsearch/action/admin/indices/flush/SyncedFlushAction.java index 5005cd2ec0878..70153b767eef1 100644 --- a/server/src/main/java/org/elasticsearch/action/admin/indices/flush/SyncedFlushAction.java +++ b/server/src/main/java/org/elasticsearch/action/admin/indices/flush/SyncedFlushAction.java @@ -19,10 +19,10 @@ package org.elasticsearch.action.admin.indices.flush; -import org.elasticsearch.action.Action; +import org.elasticsearch.action.StreamableResponseActionType; -public class SyncedFlushAction extends Action { +public class SyncedFlushAction extends StreamableResponseActionType { public static final SyncedFlushAction INSTANCE = new SyncedFlushAction(); public static final String NAME = "indices:admin/synced_flush"; diff --git a/server/src/main/java/org/elasticsearch/action/admin/indices/forcemerge/ForceMergeAction.java b/server/src/main/java/org/elasticsearch/action/admin/indices/forcemerge/ForceMergeAction.java index 51095435343cb..98d8c61dad788 100644 --- a/server/src/main/java/org/elasticsearch/action/admin/indices/forcemerge/ForceMergeAction.java +++ b/server/src/main/java/org/elasticsearch/action/admin/indices/forcemerge/ForceMergeAction.java @@ -19,9 +19,9 @@ package org.elasticsearch.action.admin.indices.forcemerge; -import org.elasticsearch.action.Action; +import org.elasticsearch.action.StreamableResponseActionType; -public class ForceMergeAction extends Action { +public class ForceMergeAction extends StreamableResponseActionType { public static final ForceMergeAction INSTANCE = new ForceMergeAction(); public static final String NAME = "indices:admin/forcemerge"; diff --git a/server/src/main/java/org/elasticsearch/action/admin/indices/get/GetIndexAction.java b/server/src/main/java/org/elasticsearch/action/admin/indices/get/GetIndexAction.java index 86396f246a414..c8a9eb85068d0 100644 --- a/server/src/main/java/org/elasticsearch/action/admin/indices/get/GetIndexAction.java +++ b/server/src/main/java/org/elasticsearch/action/admin/indices/get/GetIndexAction.java @@ -19,9 +19,9 @@ package org.elasticsearch.action.admin.indices.get; -import org.elasticsearch.action.Action; +import org.elasticsearch.action.StreamableResponseActionType; -public class GetIndexAction extends Action { +public class GetIndexAction extends StreamableResponseActionType { public static final GetIndexAction INSTANCE = new GetIndexAction(); public static final String NAME = "indices:admin/get"; diff --git a/server/src/main/java/org/elasticsearch/action/admin/indices/mapping/get/GetFieldMappingsAction.java b/server/src/main/java/org/elasticsearch/action/admin/indices/mapping/get/GetFieldMappingsAction.java index d372d8cf93f30..a16c6d4c532f4 100644 --- a/server/src/main/java/org/elasticsearch/action/admin/indices/mapping/get/GetFieldMappingsAction.java +++ b/server/src/main/java/org/elasticsearch/action/admin/indices/mapping/get/GetFieldMappingsAction.java @@ -19,10 +19,10 @@ package org.elasticsearch.action.admin.indices.mapping.get; -import org.elasticsearch.action.Action; +import org.elasticsearch.action.ActionType; import org.elasticsearch.common.io.stream.Writeable; -public class GetFieldMappingsAction extends Action { +public class GetFieldMappingsAction extends ActionType { public static final GetFieldMappingsAction INSTANCE = new GetFieldMappingsAction(); public static final String NAME = "indices:admin/mappings/fields/get"; @@ -31,11 +31,6 @@ private GetFieldMappingsAction() { super(NAME); } - @Override - public GetFieldMappingsResponse newResponse() { - throw new UnsupportedOperationException("usage of Streamable is to be replaced by Writeable"); - } - @Override public Writeable.Reader getResponseReader() { return GetFieldMappingsResponse::new; diff --git a/server/src/main/java/org/elasticsearch/action/admin/indices/mapping/get/GetFieldMappingsIndexRequest.java b/server/src/main/java/org/elasticsearch/action/admin/indices/mapping/get/GetFieldMappingsIndexRequest.java index 149cba9a33e17..576d3812c0cf4 100644 --- a/server/src/main/java/org/elasticsearch/action/admin/indices/mapping/get/GetFieldMappingsIndexRequest.java +++ b/server/src/main/java/org/elasticsearch/action/admin/indices/mapping/get/GetFieldMappingsIndexRequest.java @@ -23,7 +23,6 @@ import org.elasticsearch.action.OriginalIndices; import org.elasticsearch.action.support.IndicesOptions; import org.elasticsearch.action.support.single.shard.SingleShardRequest; -import org.elasticsearch.common.Strings; import org.elasticsearch.common.io.stream.StreamInput; import org.elasticsearch.common.io.stream.StreamOutput; @@ -31,14 +30,20 @@ public class GetFieldMappingsIndexRequest extends SingleShardRequest { - private boolean probablySingleFieldRequest; - private boolean includeDefaults; - private String[] fields = Strings.EMPTY_ARRAY; - private String[] types = Strings.EMPTY_ARRAY; + private final boolean probablySingleFieldRequest; + private final boolean includeDefaults; + private final String[] fields; + private final String[] types; private OriginalIndices originalIndices; - public GetFieldMappingsIndexRequest() { + GetFieldMappingsIndexRequest(StreamInput in) throws IOException { + super(in); + types = in.readStringArray(); + fields = in.readStringArray(); + includeDefaults = in.readBoolean(); + probablySingleFieldRequest = in.readBoolean(); + originalIndices = OriginalIndices.readOriginalIndices(in); } GetFieldMappingsIndexRequest(GetFieldMappingsRequest other, String index, boolean probablySingleFieldRequest) { @@ -92,14 +97,4 @@ public void writeTo(StreamOutput out) throws IOException { OriginalIndices.writeOriginalIndices(originalIndices, out); } - @Override - public void readFrom(StreamInput in) throws IOException { - super.readFrom(in); - types = in.readStringArray(); - fields = in.readStringArray(); - includeDefaults = in.readBoolean(); - probablySingleFieldRequest = in.readBoolean(); - originalIndices = OriginalIndices.readOriginalIndices(in); - } - } diff --git a/server/src/main/java/org/elasticsearch/action/admin/indices/mapping/get/GetMappingsAction.java b/server/src/main/java/org/elasticsearch/action/admin/indices/mapping/get/GetMappingsAction.java index 8bae685fff5dd..332fe97bca6e2 100644 --- a/server/src/main/java/org/elasticsearch/action/admin/indices/mapping/get/GetMappingsAction.java +++ b/server/src/main/java/org/elasticsearch/action/admin/indices/mapping/get/GetMappingsAction.java @@ -19,9 +19,9 @@ package org.elasticsearch.action.admin.indices.mapping.get; -import org.elasticsearch.action.Action; +import org.elasticsearch.action.StreamableResponseActionType; -public class GetMappingsAction extends Action { +public class GetMappingsAction extends StreamableResponseActionType { public static final GetMappingsAction INSTANCE = new GetMappingsAction(); public static final String NAME = "indices:admin/mappings/get"; diff --git a/server/src/main/java/org/elasticsearch/action/admin/indices/mapping/get/TransportGetFieldMappingsAction.java b/server/src/main/java/org/elasticsearch/action/admin/indices/mapping/get/TransportGetFieldMappingsAction.java index 2ebbf81f693ca..97c6493dfd950 100644 --- a/server/src/main/java/org/elasticsearch/action/admin/indices/mapping/get/TransportGetFieldMappingsAction.java +++ b/server/src/main/java/org/elasticsearch/action/admin/indices/mapping/get/TransportGetFieldMappingsAction.java @@ -22,6 +22,7 @@ import org.elasticsearch.action.ActionListener; import org.elasticsearch.action.support.ActionFilters; import org.elasticsearch.action.support.HandledTransportAction; +import org.elasticsearch.client.node.NodeClient; import org.elasticsearch.cluster.ClusterState; import org.elasticsearch.cluster.metadata.IndexNameExpressionResolver; import org.elasticsearch.cluster.service.ClusterService; @@ -39,17 +40,17 @@ public class TransportGetFieldMappingsAction extends HandledTransportAction { private final ClusterService clusterService; - private final TransportGetFieldMappingsIndexAction shardAction; private final IndexNameExpressionResolver indexNameExpressionResolver; + private final NodeClient client; @Inject public TransportGetFieldMappingsAction(TransportService transportService, ClusterService clusterService, - TransportGetFieldMappingsIndexAction shardAction, - ActionFilters actionFilters, IndexNameExpressionResolver indexNameExpressionResolver) { + ActionFilters actionFilters, IndexNameExpressionResolver indexNameExpressionResolver, + NodeClient client) { super(GetFieldMappingsAction.NAME, transportService, actionFilters, GetFieldMappingsRequest::new); this.clusterService = clusterService; - this.shardAction = shardAction; this.indexNameExpressionResolver = indexNameExpressionResolver; + this.client = client; } @Override @@ -66,7 +67,8 @@ protected void doExecute(Task task, GetFieldMappingsRequest request, final Actio boolean probablySingleFieldRequest = concreteIndices.length == 1 && request.types().length == 1 && request.fields().length == 1; for (final String index : concreteIndices) { GetFieldMappingsIndexRequest shardRequest = new GetFieldMappingsIndexRequest(request, index, probablySingleFieldRequest); - shardAction.execute(shardRequest, new ActionListener() { + + client.executeLocally(TransportGetFieldMappingsIndexAction.TYPE, shardRequest, new ActionListener<>() { @Override public void onResponse(GetFieldMappingsResponse result) { indexResponses.set(indexCounter.getAndIncrement(), result); diff --git a/server/src/main/java/org/elasticsearch/action/admin/indices/mapping/get/TransportGetFieldMappingsIndexAction.java b/server/src/main/java/org/elasticsearch/action/admin/indices/mapping/get/TransportGetFieldMappingsIndexAction.java index 61a598c361cc9..7e8845773bab8 100644 --- a/server/src/main/java/org/elasticsearch/action/admin/indices/mapping/get/TransportGetFieldMappingsIndexAction.java +++ b/server/src/main/java/org/elasticsearch/action/admin/indices/mapping/get/TransportGetFieldMappingsIndexAction.java @@ -21,6 +21,7 @@ import org.elasticsearch.ElasticsearchException; import org.elasticsearch.Version; +import org.elasticsearch.action.ActionType; import org.elasticsearch.action.admin.indices.mapping.get.GetFieldMappingsResponse.FieldMappingMetaData; import org.elasticsearch.action.support.ActionFilters; import org.elasticsearch.action.support.single.shard.TransportSingleShardAction; @@ -63,6 +64,7 @@ public class TransportGetFieldMappingsIndexAction extends TransportSingleShardAction { private static final String ACTION_NAME = GetFieldMappingsAction.NAME + "[index]"; + public static final ActionType TYPE = new ActionType<>(ACTION_NAME, GetFieldMappingsResponse::new); protected final ClusterService clusterService; private final IndicesService indicesService; diff --git a/server/src/main/java/org/elasticsearch/action/admin/indices/mapping/put/MappingRequestValidator.java b/server/src/main/java/org/elasticsearch/action/admin/indices/mapping/put/MappingRequestValidator.java deleted file mode 100644 index 8d6608c575874..0000000000000 --- a/server/src/main/java/org/elasticsearch/action/admin/indices/mapping/put/MappingRequestValidator.java +++ /dev/null @@ -1,40 +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.action.admin.indices.mapping.put; - -import org.elasticsearch.cluster.ClusterState; -import org.elasticsearch.index.Index; - -/** - * A validator that validates a {@link PutMappingRequest} before executing it. - * @see TransportPutMappingAction.RequestValidators - */ -public interface MappingRequestValidator { - - /** - * Validates a given put mapping request with its associated concrete indices and the current state. - * - * @param request the request to validate - * @param state the current cluster state - * @param indices the concrete indices that associated with the given put mapping request - * @return a non-null exception indicates a reason that the given request should be aborted; otherwise returns null. - */ - Exception validateRequest(PutMappingRequest request, ClusterState state, Index[] indices); -} diff --git a/server/src/main/java/org/elasticsearch/action/admin/indices/mapping/put/PutMappingAction.java b/server/src/main/java/org/elasticsearch/action/admin/indices/mapping/put/PutMappingAction.java index ac845e6644c7b..97d908f01eec1 100644 --- a/server/src/main/java/org/elasticsearch/action/admin/indices/mapping/put/PutMappingAction.java +++ b/server/src/main/java/org/elasticsearch/action/admin/indices/mapping/put/PutMappingAction.java @@ -19,10 +19,11 @@ package org.elasticsearch.action.admin.indices.mapping.put; -import org.elasticsearch.action.Action; +import org.elasticsearch.action.ActionType; import org.elasticsearch.action.support.master.AcknowledgedResponse; +import org.elasticsearch.common.io.stream.Writeable; -public class PutMappingAction extends Action { +public class PutMappingAction extends ActionType { public static final PutMappingAction INSTANCE = new PutMappingAction(); public static final String NAME = "indices:admin/mapping/put"; @@ -32,7 +33,7 @@ private PutMappingAction() { } @Override - public AcknowledgedResponse newResponse() { - return new AcknowledgedResponse(); + public Writeable.Reader getResponseReader() { + return AcknowledgedResponse::new; } } diff --git a/server/src/main/java/org/elasticsearch/action/admin/indices/mapping/put/PutMappingRequest.java b/server/src/main/java/org/elasticsearch/action/admin/indices/mapping/put/PutMappingRequest.java index 2dd67299e7512..cd88947482747 100644 --- a/server/src/main/java/org/elasticsearch/action/admin/indices/mapping/put/PutMappingRequest.java +++ b/server/src/main/java/org/elasticsearch/action/admin/indices/mapping/put/PutMappingRequest.java @@ -20,7 +20,6 @@ package org.elasticsearch.action.admin.indices.mapping.put; import com.carrotsearch.hppc.ObjectHashSet; - import org.elasticsearch.ElasticsearchGenerationException; import org.elasticsearch.action.ActionRequestValidationException; import org.elasticsearch.action.IndicesRequest; diff --git a/server/src/main/java/org/elasticsearch/action/admin/indices/mapping/put/TransportPutMappingAction.java b/server/src/main/java/org/elasticsearch/action/admin/indices/mapping/put/TransportPutMappingAction.java index acd0d10281463..f2a21858f5330 100644 --- a/server/src/main/java/org/elasticsearch/action/admin/indices/mapping/put/TransportPutMappingAction.java +++ b/server/src/main/java/org/elasticsearch/action/admin/indices/mapping/put/TransportPutMappingAction.java @@ -21,6 +21,7 @@ import org.apache.logging.log4j.message.ParameterizedMessage; import org.elasticsearch.action.ActionListener; +import org.elasticsearch.action.RequestValidators; import org.elasticsearch.action.support.ActionFilters; import org.elasticsearch.action.support.master.AcknowledgedResponse; import org.elasticsearch.action.support.master.TransportMasterNodeAction; @@ -32,12 +33,16 @@ import org.elasticsearch.cluster.metadata.MetaDataMappingService; import org.elasticsearch.cluster.service.ClusterService; import org.elasticsearch.common.inject.Inject; +import org.elasticsearch.common.io.stream.StreamInput; import org.elasticsearch.index.Index; import org.elasticsearch.index.IndexNotFoundException; +import org.elasticsearch.tasks.Task; import org.elasticsearch.threadpool.ThreadPool; import org.elasticsearch.transport.TransportService; -import java.util.Collection; +import java.io.IOException; +import java.util.Objects; +import java.util.Optional; /** * Put mapping action. @@ -45,17 +50,21 @@ public class TransportPutMappingAction extends TransportMasterNodeAction { private final MetaDataMappingService metaDataMappingService; - private final RequestValidators requestValidators; + private final RequestValidators requestValidators; @Inject - public TransportPutMappingAction(TransportService transportService, ClusterService clusterService, - ThreadPool threadPool, MetaDataMappingService metaDataMappingService, - ActionFilters actionFilters, IndexNameExpressionResolver indexNameExpressionResolver, - RequestValidators requestValidators) { + public TransportPutMappingAction( + final TransportService transportService, + final ClusterService clusterService, + final ThreadPool threadPool, + final MetaDataMappingService metaDataMappingService, + final ActionFilters actionFilters, + final IndexNameExpressionResolver indexNameExpressionResolver, + final RequestValidators requestValidators) { super(PutMappingAction.NAME, transportService, clusterService, threadPool, actionFilters, indexNameExpressionResolver, PutMappingRequest::new); this.metaDataMappingService = metaDataMappingService; - this.requestValidators = requestValidators; + this.requestValidators = Objects.requireNonNull(requestValidators); } @Override @@ -64,9 +73,14 @@ protected String executor() { return ThreadPool.Names.SAME; } + @Override + protected AcknowledgedResponse read(StreamInput in) throws IOException { + return new AcknowledgedResponse(in); + } + @Override protected AcknowledgedResponse newResponse() { - return new AcknowledgedResponse(); + throw new UnsupportedOperationException("usage of Streamable is to be replaced by Writeable"); } @Override @@ -81,15 +95,15 @@ protected ClusterBlockException checkBlock(PutMappingRequest request, ClusterSta } @Override - protected void masterOperation(final PutMappingRequest request, final ClusterState state, + protected void masterOperation(Task task, final PutMappingRequest request, final ClusterState state, final ActionListener listener) { try { final Index[] concreteIndices = request.getConcreteIndex() == null ? indexNameExpressionResolver.concreteIndices(state, request) : new Index[] {request.getConcreteIndex()}; - final Exception validationException = requestValidators.validateRequest(request, state, concreteIndices); - if (validationException != null) { - listener.onFailure(validationException); + final Optional maybeValidationException = requestValidators.validateRequest(request, state, concreteIndices); + if (maybeValidationException.isPresent()) { + listener.onFailure(maybeValidationException.get()); return; } PutMappingClusterStateUpdateRequest updateRequest = new PutMappingClusterStateUpdateRequest() @@ -118,25 +132,4 @@ public void onFailure(Exception t) { } } - - public static class RequestValidators { - private final Collection validators; - - public RequestValidators(Collection validators) { - this.validators = validators; - } - - private Exception validateRequest(PutMappingRequest request, ClusterState state, Index[] indices) { - Exception firstException = null; - for (MappingRequestValidator validator : validators) { - final Exception e = validator.validateRequest(request, state, indices); - if (firstException == null) { - firstException = e; - } else { - firstException.addSuppressed(e); - } - } - return firstException; - } - } } diff --git a/server/src/main/java/org/elasticsearch/action/admin/indices/open/OpenIndexAction.java b/server/src/main/java/org/elasticsearch/action/admin/indices/open/OpenIndexAction.java index ea6f1eb6afd95..3dd871e97604c 100644 --- a/server/src/main/java/org/elasticsearch/action/admin/indices/open/OpenIndexAction.java +++ b/server/src/main/java/org/elasticsearch/action/admin/indices/open/OpenIndexAction.java @@ -19,9 +19,10 @@ package org.elasticsearch.action.admin.indices.open; -import org.elasticsearch.action.Action; +import org.elasticsearch.action.ActionType; +import org.elasticsearch.common.io.stream.Writeable; -public class OpenIndexAction extends Action { +public class OpenIndexAction extends ActionType { public static final OpenIndexAction INSTANCE = new OpenIndexAction(); public static final String NAME = "indices:admin/open"; @@ -31,7 +32,7 @@ private OpenIndexAction() { } @Override - public OpenIndexResponse newResponse() { - return new OpenIndexResponse(); + public Writeable.Reader getResponseReader() { + return OpenIndexResponse::new; } } diff --git a/server/src/main/java/org/elasticsearch/action/admin/indices/open/OpenIndexResponse.java b/server/src/main/java/org/elasticsearch/action/admin/indices/open/OpenIndexResponse.java index 91f27e8c0c3f7..c7abd0367b319 100644 --- a/server/src/main/java/org/elasticsearch/action/admin/indices/open/OpenIndexResponse.java +++ b/server/src/main/java/org/elasticsearch/action/admin/indices/open/OpenIndexResponse.java @@ -39,19 +39,14 @@ public class OpenIndexResponse extends ShardsAcknowledgedResponse { declareAcknowledgedAndShardsAcknowledgedFields(PARSER); } - public OpenIndexResponse() { + public OpenIndexResponse(StreamInput in) throws IOException { + super(in, true); } public OpenIndexResponse(boolean acknowledged, boolean shardsAcknowledged) { super(acknowledged, shardsAcknowledged); } - @Override - public void readFrom(StreamInput in) throws IOException { - super.readFrom(in); - readShardsAcknowledged(in); - } - @Override public void writeTo(StreamOutput out) throws IOException { super.writeTo(out); diff --git a/server/src/main/java/org/elasticsearch/action/admin/indices/open/TransportOpenIndexAction.java b/server/src/main/java/org/elasticsearch/action/admin/indices/open/TransportOpenIndexAction.java index b9323933f7601..c815a6b1132d7 100644 --- a/server/src/main/java/org/elasticsearch/action/admin/indices/open/TransportOpenIndexAction.java +++ b/server/src/main/java/org/elasticsearch/action/admin/indices/open/TransportOpenIndexAction.java @@ -32,11 +32,14 @@ import org.elasticsearch.cluster.metadata.MetaDataIndexStateService; import org.elasticsearch.cluster.service.ClusterService; import org.elasticsearch.common.inject.Inject; +import org.elasticsearch.common.io.stream.StreamInput; import org.elasticsearch.index.Index; import org.elasticsearch.tasks.Task; import org.elasticsearch.threadpool.ThreadPool; import org.elasticsearch.transport.TransportService; +import java.io.IOException; + /** * Open index action */ @@ -64,7 +67,12 @@ protected String executor() { @Override protected OpenIndexResponse newResponse() { - return new OpenIndexResponse(); + throw new UnsupportedOperationException("usage of Streamable is to be replaced by Writeable"); + } + + @Override + protected OpenIndexResponse read(StreamInput in) throws IOException { + return new OpenIndexResponse(in); } @Override @@ -80,7 +88,7 @@ protected ClusterBlockException checkBlock(OpenIndexRequest request, ClusterStat } @Override - protected void masterOperation(final OpenIndexRequest request, final ClusterState state, + protected void masterOperation(Task task, final OpenIndexRequest request, final ClusterState state, final ActionListener listener) { final Index[] concreteIndices = indexNameExpressionResolver.concreteIndices(state, request); if (concreteIndices == null || concreteIndices.length == 0) { diff --git a/server/src/main/java/org/elasticsearch/action/admin/indices/recovery/RecoveryAction.java b/server/src/main/java/org/elasticsearch/action/admin/indices/recovery/RecoveryAction.java index bfe261b58843a..e41a766aa4b08 100644 --- a/server/src/main/java/org/elasticsearch/action/admin/indices/recovery/RecoveryAction.java +++ b/server/src/main/java/org/elasticsearch/action/admin/indices/recovery/RecoveryAction.java @@ -19,12 +19,12 @@ package org.elasticsearch.action.admin.indices.recovery; -import org.elasticsearch.action.Action; +import org.elasticsearch.action.StreamableResponseActionType; /** * Recovery information action */ -public class RecoveryAction extends Action { +public class RecoveryAction extends StreamableResponseActionType { public static final RecoveryAction INSTANCE = new RecoveryAction(); public static final String NAME = "indices:monitor/recovery"; diff --git a/server/src/main/java/org/elasticsearch/action/admin/indices/refresh/RefreshAction.java b/server/src/main/java/org/elasticsearch/action/admin/indices/refresh/RefreshAction.java index b0dac076b2f49..f26d899371ecb 100644 --- a/server/src/main/java/org/elasticsearch/action/admin/indices/refresh/RefreshAction.java +++ b/server/src/main/java/org/elasticsearch/action/admin/indices/refresh/RefreshAction.java @@ -19,9 +19,9 @@ package org.elasticsearch.action.admin.indices.refresh; -import org.elasticsearch.action.Action; +import org.elasticsearch.action.StreamableResponseActionType; -public class RefreshAction extends Action { +public class RefreshAction extends StreamableResponseActionType { public static final RefreshAction INSTANCE = new RefreshAction(); public static final String NAME = "indices:admin/refresh"; diff --git a/server/src/main/java/org/elasticsearch/action/admin/indices/rollover/RolloverAction.java b/server/src/main/java/org/elasticsearch/action/admin/indices/rollover/RolloverAction.java index 0bcd4eefe0762..d0893d35a6754 100644 --- a/server/src/main/java/org/elasticsearch/action/admin/indices/rollover/RolloverAction.java +++ b/server/src/main/java/org/elasticsearch/action/admin/indices/rollover/RolloverAction.java @@ -19,9 +19,10 @@ package org.elasticsearch.action.admin.indices.rollover; -import org.elasticsearch.action.Action; +import org.elasticsearch.action.ActionType; +import org.elasticsearch.common.io.stream.Writeable; -public class RolloverAction extends Action { +public class RolloverAction extends ActionType { public static final RolloverAction INSTANCE = new RolloverAction(); public static final String NAME = "indices:admin/rollover"; @@ -31,7 +32,7 @@ private RolloverAction() { } @Override - public RolloverResponse newResponse() { - return new RolloverResponse(); + public Writeable.Reader getResponseReader() { + return RolloverResponse::new; } } diff --git a/server/src/main/java/org/elasticsearch/action/admin/indices/rollover/RolloverResponse.java b/server/src/main/java/org/elasticsearch/action/admin/indices/rollover/RolloverResponse.java index cd58253d5e7c2..3c0431c7ea6c9 100644 --- a/server/src/main/java/org/elasticsearch/action/admin/indices/rollover/RolloverResponse.java +++ b/server/src/main/java/org/elasticsearch/action/admin/indices/rollover/RolloverResponse.java @@ -65,13 +65,27 @@ public final class RolloverResponse extends ShardsAcknowledgedResponse implement declareAcknowledgedAndShardsAcknowledgedFields(PARSER); } - private String oldIndex; - private String newIndex; - private Map conditionStatus; - private boolean dryRun; - private boolean rolledOver; - - RolloverResponse() { + private final String oldIndex; + private final String newIndex; + private final Map conditionStatus; + private final boolean dryRun; + private final boolean rolledOver; + // Needs to be duplicated, because shardsAcknowledged gets (de)serailized as last field whereas + // in other subclasses of ShardsAcknowledgedResponse this field (de)serailized as first field. + private final boolean shardsAcknowledged; + + RolloverResponse(StreamInput in) throws IOException { + super(in, false); + oldIndex = in.readString(); + newIndex = in.readString(); + int conditionSize = in.readVInt(); + conditionStatus = new HashMap<>(conditionSize); + for (int i = 0; i < conditionSize; i++) { + conditionStatus.put(in.readString(), in.readBoolean()); + } + dryRun = in.readBoolean(); + rolledOver = in.readBoolean(); + shardsAcknowledged = in.readBoolean(); } public RolloverResponse(String oldIndex, String newIndex, Map conditionResults, @@ -82,6 +96,7 @@ public RolloverResponse(String oldIndex, String newIndex, Map c this.dryRun = dryRun; this.rolledOver = rolledOver; this.conditionStatus = conditionResults; + this.shardsAcknowledged = shardsAcknowledged; } /** @@ -120,18 +135,8 @@ public boolean isRolledOver() { } @Override - public void readFrom(StreamInput in) throws IOException { - super.readFrom(in); - oldIndex = in.readString(); - newIndex = in.readString(); - int conditionSize = in.readVInt(); - conditionStatus = new HashMap<>(conditionSize); - for (int i = 0; i < conditionSize; i++) { - conditionStatus.put(in.readString(), in.readBoolean()); - } - dryRun = in.readBoolean(); - rolledOver = in.readBoolean(); - readShardsAcknowledged(in); + public boolean isShardsAcknowledged() { + return shardsAcknowledged; } @Override @@ -146,7 +151,7 @@ public void writeTo(StreamOutput out) throws IOException { } out.writeBoolean(dryRun); out.writeBoolean(rolledOver); - writeShardsAcknowledged(out); + out.writeBoolean(shardsAcknowledged); } @Override diff --git a/server/src/main/java/org/elasticsearch/action/admin/indices/rollover/TransportRolloverAction.java b/server/src/main/java/org/elasticsearch/action/admin/indices/rollover/TransportRolloverAction.java index c36d6092a7e49..60f099cb1d034 100644 --- a/server/src/main/java/org/elasticsearch/action/admin/indices/rollover/TransportRolloverAction.java +++ b/server/src/main/java/org/elasticsearch/action/admin/indices/rollover/TransportRolloverAction.java @@ -23,6 +23,8 @@ import org.elasticsearch.action.admin.indices.alias.IndicesAliasesClusterStateUpdateRequest; import org.elasticsearch.action.admin.indices.create.CreateIndexClusterStateUpdateRequest; import org.elasticsearch.action.admin.indices.create.CreateIndexRequest; +import org.elasticsearch.action.admin.indices.stats.IndicesStatsAction; +import org.elasticsearch.action.admin.indices.stats.IndicesStatsRequest; import org.elasticsearch.action.admin.indices.stats.IndicesStatsResponse; import org.elasticsearch.action.support.ActionFilters; import org.elasticsearch.action.support.ActiveShardCount; @@ -44,16 +46,21 @@ import org.elasticsearch.cluster.metadata.MetaDataIndexAliasesService; import org.elasticsearch.cluster.metadata.MetaDataIndexTemplateService; import org.elasticsearch.cluster.service.ClusterService; +import org.elasticsearch.common.Nullable; import org.elasticsearch.common.inject.Inject; +import org.elasticsearch.common.io.stream.StreamInput; import org.elasticsearch.common.unit.ByteSizeValue; import org.elasticsearch.index.shard.DocsStats; +import org.elasticsearch.tasks.Task; import org.elasticsearch.threadpool.ThreadPool; import org.elasticsearch.transport.TransportService; +import java.io.IOException; import java.util.Collection; import java.util.List; import java.util.Locale; import java.util.Map; +import java.util.Optional; import java.util.regex.Pattern; import java.util.stream.Collectors; @@ -89,7 +96,12 @@ protected String executor() { @Override protected RolloverResponse newResponse() { - return new RolloverResponse(); + throw new UnsupportedOperationException("usage of Streamable is to be replaced by Writeable"); + } + + @Override + protected RolloverResponse read(StreamInput in) throws IOException { + return new RolloverResponse(in); } @Override @@ -101,7 +113,7 @@ protected ClusterBlockException checkBlock(RolloverRequest request, ClusterState } @Override - protected void masterOperation(final RolloverRequest rolloverRequest, final ClusterState state, + protected void masterOperation(Task task, final RolloverRequest rolloverRequest, final ClusterState state, final ActionListener listener) { final MetaData metaData = state.metaData(); validate(metaData, rolloverRequest); @@ -117,7 +129,9 @@ protected void masterOperation(final RolloverRequest rolloverRequest, final Clus final String rolloverIndexName = indexNameExpressionResolver.resolveDateMathExpression(unresolvedName); MetaDataCreateIndexService.validateIndexName(rolloverIndexName, state); // will fail if the index already exists checkNoDuplicatedAliasInIndexTemplate(metaData, rolloverIndexName, rolloverRequest.getAlias()); - client.admin().indices().prepareStats(rolloverRequest.getAlias()).clear().setDocs(true).execute( + IndicesStatsRequest statsRequest = new IndicesStatsRequest().indices(rolloverRequest.getAlias()).clear().docs(true); + statsRequest.setParentTask(clusterService.localNode().getId(), task.getId()); + client.execute(IndicesStatsAction.INSTANCE, statsRequest, new ActionListener() { @Override public void onResponse(IndicesStatsResponse statsResponse) { @@ -233,7 +247,11 @@ static String generateRolloverIndexName(String sourceIndexName, IndexNameExpress } static Map evaluateConditions(final Collection> conditions, - final DocsStats docsStats, final IndexMetaData metaData) { + @Nullable final DocsStats docsStats, + @Nullable final IndexMetaData metaData) { + if (metaData == null) { + return conditions.stream().collect(Collectors.toMap(Condition::toString, cond -> false)); + } final long numDocs = docsStats == null ? 0 : docsStats.getCount(); final long indexSize = docsStats == null ? 0 : docsStats.getTotalSizeInBytes(); final Condition.Stats stats = new Condition.Stats(numDocs, metaData.getCreationDate(), new ByteSizeValue(indexSize)); @@ -242,9 +260,18 @@ static Map evaluateConditions(final Collection> co .collect(Collectors.toMap(result -> result.condition.toString(), result -> result.matched)); } - static Map evaluateConditions(final Collection> conditions, final IndexMetaData metaData, - final IndicesStatsResponse statsResponse) { - return evaluateConditions(conditions, statsResponse.getIndex(metaData.getIndex().getName()).getPrimaries().getDocs(), metaData); + static Map evaluateConditions(final Collection> conditions, + @Nullable final IndexMetaData metaData, + @Nullable final IndicesStatsResponse statsResponse) { + if (metaData == null) { + return conditions.stream().collect(Collectors.toMap(Condition::toString, cond -> false)); + } else { + final DocsStats docsStats = Optional.ofNullable(statsResponse) + .map(stats -> stats.getIndex(metaData.getIndex().getName())) + .map(indexStats -> indexStats.getPrimaries().getDocs()) + .orElse(null); + return evaluateConditions(conditions, docsStats, metaData); + } } static void validate(MetaData metaData, RolloverRequest request) { diff --git a/server/src/main/java/org/elasticsearch/action/admin/indices/segments/IndicesSegmentsAction.java b/server/src/main/java/org/elasticsearch/action/admin/indices/segments/IndicesSegmentsAction.java index 669c31d6b087d..edc9df6e91207 100644 --- a/server/src/main/java/org/elasticsearch/action/admin/indices/segments/IndicesSegmentsAction.java +++ b/server/src/main/java/org/elasticsearch/action/admin/indices/segments/IndicesSegmentsAction.java @@ -19,9 +19,9 @@ package org.elasticsearch.action.admin.indices.segments; -import org.elasticsearch.action.Action; +import org.elasticsearch.action.StreamableResponseActionType; -public class IndicesSegmentsAction extends Action { +public class IndicesSegmentsAction extends StreamableResponseActionType { public static final IndicesSegmentsAction INSTANCE = new IndicesSegmentsAction(); public static final String NAME = "indices:monitor/segments"; diff --git a/server/src/main/java/org/elasticsearch/action/admin/indices/settings/get/GetSettingsAction.java b/server/src/main/java/org/elasticsearch/action/admin/indices/settings/get/GetSettingsAction.java index e4149aaf8f15f..6e17be7241745 100644 --- a/server/src/main/java/org/elasticsearch/action/admin/indices/settings/get/GetSettingsAction.java +++ b/server/src/main/java/org/elasticsearch/action/admin/indices/settings/get/GetSettingsAction.java @@ -19,9 +19,9 @@ package org.elasticsearch.action.admin.indices.settings.get; -import org.elasticsearch.action.Action; +import org.elasticsearch.action.StreamableResponseActionType; -public class GetSettingsAction extends Action { +public class GetSettingsAction extends StreamableResponseActionType { public static final GetSettingsAction INSTANCE = new GetSettingsAction(); public static final String NAME = "indices:monitor/settings/get"; diff --git a/server/src/main/java/org/elasticsearch/action/admin/indices/settings/get/TransportGetSettingsAction.java b/server/src/main/java/org/elasticsearch/action/admin/indices/settings/get/TransportGetSettingsAction.java index 7f76439793365..fd0004beba292 100644 --- a/server/src/main/java/org/elasticsearch/action/admin/indices/settings/get/TransportGetSettingsAction.java +++ b/server/src/main/java/org/elasticsearch/action/admin/indices/settings/get/TransportGetSettingsAction.java @@ -36,6 +36,7 @@ import org.elasticsearch.common.settings.SettingsFilter; import org.elasticsearch.common.util.CollectionUtils; import org.elasticsearch.index.Index; +import org.elasticsearch.tasks.Task; import org.elasticsearch.threadpool.ThreadPool; import org.elasticsearch.transport.TransportService; @@ -79,7 +80,8 @@ private static boolean isFilteredRequest(GetSettingsRequest request) { } @Override - protected void masterOperation(GetSettingsRequest request, ClusterState state, ActionListener listener) { + protected void masterOperation(Task task, GetSettingsRequest request, ClusterState state, + ActionListener listener) { Index[] concreteIndices = indexNameExpressionResolver.concreteIndices(state, request); ImmutableOpenMap.Builder indexToSettingsBuilder = ImmutableOpenMap.builder(); ImmutableOpenMap.Builder indexToDefaultSettingsBuilder = ImmutableOpenMap.builder(); diff --git a/server/src/main/java/org/elasticsearch/action/admin/indices/settings/put/TransportUpdateSettingsAction.java b/server/src/main/java/org/elasticsearch/action/admin/indices/settings/put/TransportUpdateSettingsAction.java index 9de0343365ea5..c3192631e2cf5 100644 --- a/server/src/main/java/org/elasticsearch/action/admin/indices/settings/put/TransportUpdateSettingsAction.java +++ b/server/src/main/java/org/elasticsearch/action/admin/indices/settings/put/TransportUpdateSettingsAction.java @@ -33,10 +33,14 @@ import org.elasticsearch.cluster.metadata.MetaDataUpdateSettingsService; import org.elasticsearch.cluster.service.ClusterService; import org.elasticsearch.common.inject.Inject; +import org.elasticsearch.common.io.stream.StreamInput; import org.elasticsearch.index.Index; +import org.elasticsearch.tasks.Task; import org.elasticsearch.threadpool.ThreadPool; import org.elasticsearch.transport.TransportService; +import java.io.IOException; + public class TransportUpdateSettingsAction extends TransportMasterNodeAction { private final MetaDataUpdateSettingsService updateSettingsService; @@ -73,13 +77,18 @@ protected ClusterBlockException checkBlock(UpdateSettingsRequest request, Cluste indexNameExpressionResolver.concreteIndexNames(state, request)); } + @Override + protected AcknowledgedResponse read(StreamInput in) throws IOException { + return new AcknowledgedResponse(in); + } + @Override protected AcknowledgedResponse newResponse() { - return new AcknowledgedResponse(); + throw new UnsupportedOperationException("usage of Streamable is to be replaced by Writeable"); } @Override - protected void masterOperation(final UpdateSettingsRequest request, final ClusterState state, + protected void masterOperation(Task task, final UpdateSettingsRequest request, final ClusterState state, final ActionListener listener) { final Index[] concreteIndices = indexNameExpressionResolver.concreteIndices(state, request); UpdateSettingsClusterStateUpdateRequest clusterStateUpdateRequest = new UpdateSettingsClusterStateUpdateRequest() diff --git a/server/src/main/java/org/elasticsearch/action/admin/indices/settings/put/UpdateSettingsAction.java b/server/src/main/java/org/elasticsearch/action/admin/indices/settings/put/UpdateSettingsAction.java index eb6574fb53094..df9b5137bb3d1 100644 --- a/server/src/main/java/org/elasticsearch/action/admin/indices/settings/put/UpdateSettingsAction.java +++ b/server/src/main/java/org/elasticsearch/action/admin/indices/settings/put/UpdateSettingsAction.java @@ -19,10 +19,11 @@ package org.elasticsearch.action.admin.indices.settings.put; -import org.elasticsearch.action.Action; +import org.elasticsearch.action.ActionType; import org.elasticsearch.action.support.master.AcknowledgedResponse; +import org.elasticsearch.common.io.stream.Writeable; -public class UpdateSettingsAction extends Action { +public class UpdateSettingsAction extends ActionType { public static final UpdateSettingsAction INSTANCE = new UpdateSettingsAction(); public static final String NAME = "indices:admin/settings/update"; @@ -32,7 +33,7 @@ private UpdateSettingsAction() { } @Override - public AcknowledgedResponse newResponse() { - return new AcknowledgedResponse(); + public Writeable.Reader getResponseReader() { + return AcknowledgedResponse::new; } } diff --git a/server/src/main/java/org/elasticsearch/action/admin/indices/shards/IndicesShardStoreRequestBuilder.java b/server/src/main/java/org/elasticsearch/action/admin/indices/shards/IndicesShardStoreRequestBuilder.java index f44c24144030b..6146dcf87d691 100644 --- a/server/src/main/java/org/elasticsearch/action/admin/indices/shards/IndicesShardStoreRequestBuilder.java +++ b/server/src/main/java/org/elasticsearch/action/admin/indices/shards/IndicesShardStoreRequestBuilder.java @@ -19,7 +19,7 @@ package org.elasticsearch.action.admin.indices.shards; -import org.elasticsearch.action.Action; +import org.elasticsearch.action.ActionType; import org.elasticsearch.action.support.IndicesOptions; import org.elasticsearch.action.support.master.MasterNodeReadOperationRequestBuilder; import org.elasticsearch.client.ElasticsearchClient; @@ -33,7 +33,7 @@ public class IndicesShardStoreRequestBuilder extends MasterNodeReadOperationRequ IndicesShardStoresResponse, IndicesShardStoreRequestBuilder> { - public IndicesShardStoreRequestBuilder(ElasticsearchClient client, Action action, String... indices) { + public IndicesShardStoreRequestBuilder(ElasticsearchClient client, ActionType action, String... indices) { super(client, action, new IndicesShardStoresRequest(indices)); } diff --git a/server/src/main/java/org/elasticsearch/action/admin/indices/shards/IndicesShardStoresAction.java b/server/src/main/java/org/elasticsearch/action/admin/indices/shards/IndicesShardStoresAction.java index d3ce0077d5e94..b06145bff0931 100644 --- a/server/src/main/java/org/elasticsearch/action/admin/indices/shards/IndicesShardStoresAction.java +++ b/server/src/main/java/org/elasticsearch/action/admin/indices/shards/IndicesShardStoresAction.java @@ -19,16 +19,16 @@ package org.elasticsearch.action.admin.indices.shards; -import org.elasticsearch.action.Action; +import org.elasticsearch.action.StreamableResponseActionType; /** - * Action for {@link TransportIndicesShardStoresAction} + * ActionType for {@link TransportIndicesShardStoresAction} * * Exposes shard store information for requested indices. * Shard store information reports which nodes hold shard copies, how recent they are * and any exceptions on opening the shard index or from previous engine failures */ -public class IndicesShardStoresAction extends Action { +public class IndicesShardStoresAction extends StreamableResponseActionType { public static final IndicesShardStoresAction INSTANCE = new IndicesShardStoresAction(); public static final String NAME = "indices:monitor/shard_stores"; diff --git a/server/src/main/java/org/elasticsearch/action/admin/indices/shards/IndicesShardStoresResponse.java b/server/src/main/java/org/elasticsearch/action/admin/indices/shards/IndicesShardStoresResponse.java index 86198cb74391d..87433ce044e07 100644 --- a/server/src/main/java/org/elasticsearch/action/admin/indices/shards/IndicesShardStoresResponse.java +++ b/server/src/main/java/org/elasticsearch/action/admin/indices/shards/IndicesShardStoresResponse.java @@ -258,12 +258,9 @@ public void writeTo(StreamOutput out) throws IOException { } @Override - public XContentBuilder toXContent(XContentBuilder builder, Params params) throws IOException { - builder.startObject(); + public XContentBuilder innerToXContent(XContentBuilder builder, Params params) throws IOException { builder.field("node", nodeId()); - super.innerToXContent(builder, params); - builder.endObject(); - return builder; + return super.innerToXContent(builder, params); } } diff --git a/server/src/main/java/org/elasticsearch/action/admin/indices/shards/TransportIndicesShardStoresAction.java b/server/src/main/java/org/elasticsearch/action/admin/indices/shards/TransportIndicesShardStoresAction.java index 38b11530c2d93..3967e5e31f930 100644 --- a/server/src/main/java/org/elasticsearch/action/admin/indices/shards/TransportIndicesShardStoresAction.java +++ b/server/src/main/java/org/elasticsearch/action/admin/indices/shards/TransportIndicesShardStoresAction.java @@ -46,6 +46,7 @@ import org.elasticsearch.gateway.TransportNodesListGatewayStartedShards; import org.elasticsearch.gateway.TransportNodesListGatewayStartedShards.NodeGatewayStartedShards; import org.elasticsearch.index.shard.ShardId; +import org.elasticsearch.tasks.Task; import org.elasticsearch.threadpool.ThreadPool; import org.elasticsearch.transport.TransportService; @@ -87,7 +88,7 @@ protected IndicesShardStoresResponse newResponse() { } @Override - protected void masterOperation(IndicesShardStoresRequest request, ClusterState state, + protected void masterOperation(Task task, IndicesShardStoresRequest request, ClusterState state, ActionListener listener) { final RoutingTable routingTables = state.routingTable(); final RoutingNodes routingNodes = state.getRoutingNodes(); diff --git a/server/src/main/java/org/elasticsearch/action/admin/indices/shrink/ResizeAction.java b/server/src/main/java/org/elasticsearch/action/admin/indices/shrink/ResizeAction.java index 42226570c0930..28b650cf0b747 100644 --- a/server/src/main/java/org/elasticsearch/action/admin/indices/shrink/ResizeAction.java +++ b/server/src/main/java/org/elasticsearch/action/admin/indices/shrink/ResizeAction.java @@ -19,9 +19,10 @@ package org.elasticsearch.action.admin.indices.shrink; -import org.elasticsearch.action.Action; +import org.elasticsearch.action.ActionType; +import org.elasticsearch.common.io.stream.Writeable; -public class ResizeAction extends Action { +public class ResizeAction extends ActionType { public static final ResizeAction INSTANCE = new ResizeAction(); public static final String NAME = "indices:admin/resize"; @@ -31,7 +32,7 @@ private ResizeAction() { } @Override - public ResizeResponse newResponse() { - return new ResizeResponse(); + public Writeable.Reader getResponseReader() { + return ResizeResponse::new; } } diff --git a/server/src/main/java/org/elasticsearch/action/admin/indices/shrink/ResizeRequestBuilder.java b/server/src/main/java/org/elasticsearch/action/admin/indices/shrink/ResizeRequestBuilder.java index e4b9a34b00415..eddd3a0da203a 100644 --- a/server/src/main/java/org/elasticsearch/action/admin/indices/shrink/ResizeRequestBuilder.java +++ b/server/src/main/java/org/elasticsearch/action/admin/indices/shrink/ResizeRequestBuilder.java @@ -18,7 +18,7 @@ */ package org.elasticsearch.action.admin.indices.shrink; -import org.elasticsearch.action.Action; +import org.elasticsearch.action.ActionType; import org.elasticsearch.action.admin.indices.create.CreateIndexRequest; import org.elasticsearch.action.support.ActiveShardCount; import org.elasticsearch.action.support.master.AcknowledgedRequestBuilder; @@ -27,7 +27,7 @@ public class ResizeRequestBuilder extends AcknowledgedRequestBuilder { - public ResizeRequestBuilder(ElasticsearchClient client, Action action) { + public ResizeRequestBuilder(ElasticsearchClient client, ActionType action) { super(client, action, new ResizeRequest()); } diff --git a/server/src/main/java/org/elasticsearch/action/admin/indices/shrink/ResizeResponse.java b/server/src/main/java/org/elasticsearch/action/admin/indices/shrink/ResizeResponse.java index ffe59f1e3ae18..c017c8dc9fa90 100644 --- a/server/src/main/java/org/elasticsearch/action/admin/indices/shrink/ResizeResponse.java +++ b/server/src/main/java/org/elasticsearch/action/admin/indices/shrink/ResizeResponse.java @@ -20,9 +20,12 @@ package org.elasticsearch.action.admin.indices.shrink; import org.elasticsearch.action.admin.indices.create.CreateIndexResponse; +import org.elasticsearch.common.io.stream.StreamInput; import org.elasticsearch.common.xcontent.ConstructingObjectParser; import org.elasticsearch.common.xcontent.XContentParser; +import java.io.IOException; + /** * A response for a resize index action, either shrink or split index. */ @@ -35,7 +38,8 @@ public final class ResizeResponse extends CreateIndexResponse { declareFields(PARSER); } - ResizeResponse() { + ResizeResponse(StreamInput in) throws IOException { + super(in); } public ResizeResponse(boolean acknowledged, boolean shardsAcknowledged, String index) { diff --git a/server/src/main/java/org/elasticsearch/action/admin/indices/shrink/ShrinkAction.java b/server/src/main/java/org/elasticsearch/action/admin/indices/shrink/ShrinkAction.java index a2c689ba360ee..57de49307018c 100644 --- a/server/src/main/java/org/elasticsearch/action/admin/indices/shrink/ShrinkAction.java +++ b/server/src/main/java/org/elasticsearch/action/admin/indices/shrink/ShrinkAction.java @@ -19,9 +19,10 @@ package org.elasticsearch.action.admin.indices.shrink; -import org.elasticsearch.action.Action; +import org.elasticsearch.action.ActionType; +import org.elasticsearch.common.io.stream.Writeable; -public class ShrinkAction extends Action { +public class ShrinkAction extends ActionType { public static final ShrinkAction INSTANCE = new ShrinkAction(); public static final String NAME = "indices:admin/shrink"; @@ -31,7 +32,7 @@ private ShrinkAction() { } @Override - public ResizeResponse newResponse() { - return new ResizeResponse(); + public Writeable.Reader getResponseReader() { + return ResizeResponse::new; } } diff --git a/server/src/main/java/org/elasticsearch/action/admin/indices/shrink/TransportResizeAction.java b/server/src/main/java/org/elasticsearch/action/admin/indices/shrink/TransportResizeAction.java index ff4e643a08227..178c15e473bd4 100644 --- a/server/src/main/java/org/elasticsearch/action/admin/indices/shrink/TransportResizeAction.java +++ b/server/src/main/java/org/elasticsearch/action/admin/indices/shrink/TransportResizeAction.java @@ -24,6 +24,9 @@ import org.elasticsearch.action.admin.indices.create.CreateIndexClusterStateUpdateRequest; import org.elasticsearch.action.admin.indices.create.CreateIndexRequest; import org.elasticsearch.action.admin.indices.stats.IndexShardStats; +import org.elasticsearch.action.admin.indices.stats.IndicesStatsAction; +import org.elasticsearch.action.admin.indices.stats.IndicesStatsRequest; +import org.elasticsearch.action.admin.indices.stats.IndicesStatsRequestBuilder; import org.elasticsearch.action.support.ActionFilters; import org.elasticsearch.action.support.master.TransportMasterNodeAction; import org.elasticsearch.client.Client; @@ -35,14 +38,17 @@ import org.elasticsearch.cluster.metadata.MetaDataCreateIndexService; import org.elasticsearch.cluster.service.ClusterService; import org.elasticsearch.common.inject.Inject; +import org.elasticsearch.common.io.stream.StreamInput; import org.elasticsearch.common.settings.Settings; import org.elasticsearch.index.IndexNotFoundException; import org.elasticsearch.index.IndexSettings; import org.elasticsearch.index.shard.DocsStats; import org.elasticsearch.index.shard.ShardId; +import org.elasticsearch.tasks.Task; import org.elasticsearch.threadpool.ThreadPool; import org.elasticsearch.transport.TransportService; +import java.io.IOException; import java.util.Locale; import java.util.Objects; import java.util.Set; @@ -81,7 +87,12 @@ protected String executor() { @Override protected ResizeResponse newResponse() { - return new ResizeResponse(); + throw new UnsupportedOperationException("usage of Streamable is to be replaced by Writeable"); + } + + @Override + protected ResizeResponse read(StreamInput in) throws IOException { + return new ResizeResponse(in); } @Override @@ -90,13 +101,16 @@ protected ClusterBlockException checkBlock(ResizeRequest request, ClusterState s } @Override - protected void masterOperation(final ResizeRequest resizeRequest, final ClusterState state, + protected void masterOperation(Task task, final ResizeRequest resizeRequest, final ClusterState state, final ActionListener listener) { // there is no need to fetch docs stats for split but we keep it simple and do it anyway for simplicity of the code final String sourceIndex = indexNameExpressionResolver.resolveDateMathExpression(resizeRequest.getSourceIndex()); final String targetIndex = indexNameExpressionResolver.resolveDateMathExpression(resizeRequest.getTargetIndexRequest().index()); - client.admin().indices().prepareStats(sourceIndex).clear().setDocs(true).execute( + IndicesStatsRequestBuilder statsRequestBuilder = client.admin().indices().prepareStats(sourceIndex).clear().setDocs(true); + IndicesStatsRequest statsRequest = statsRequestBuilder.request(); + statsRequest.setParentTask(clusterService.localNode().getId(), task.getId()); + client.execute(IndicesStatsAction.INSTANCE, statsRequest, ActionListener.delegateFailure(listener, (delegatedListener, indicesStatsResponse) -> { CreateIndexClusterStateUpdateRequest updateRequest = prepareCreateIndexRequest(resizeRequest, state, i -> { diff --git a/server/src/main/java/org/elasticsearch/action/admin/indices/stats/IndicesStatsAction.java b/server/src/main/java/org/elasticsearch/action/admin/indices/stats/IndicesStatsAction.java index 6765279eb6b87..4d2717d6a006f 100644 --- a/server/src/main/java/org/elasticsearch/action/admin/indices/stats/IndicesStatsAction.java +++ b/server/src/main/java/org/elasticsearch/action/admin/indices/stats/IndicesStatsAction.java @@ -19,9 +19,9 @@ package org.elasticsearch.action.admin.indices.stats; -import org.elasticsearch.action.Action; +import org.elasticsearch.action.StreamableResponseActionType; -public class IndicesStatsAction extends Action { +public class IndicesStatsAction extends StreamableResponseActionType { public static final IndicesStatsAction INSTANCE = new IndicesStatsAction(); public static final String NAME = "indices:monitor/stats"; diff --git a/server/src/main/java/org/elasticsearch/action/admin/indices/template/delete/DeleteIndexTemplateAction.java b/server/src/main/java/org/elasticsearch/action/admin/indices/template/delete/DeleteIndexTemplateAction.java index a8614277b409d..b08d2886b495e 100644 --- a/server/src/main/java/org/elasticsearch/action/admin/indices/template/delete/DeleteIndexTemplateAction.java +++ b/server/src/main/java/org/elasticsearch/action/admin/indices/template/delete/DeleteIndexTemplateAction.java @@ -19,10 +19,11 @@ package org.elasticsearch.action.admin.indices.template.delete; -import org.elasticsearch.action.Action; +import org.elasticsearch.action.ActionType; import org.elasticsearch.action.support.master.AcknowledgedResponse; +import org.elasticsearch.common.io.stream.Writeable; -public class DeleteIndexTemplateAction extends Action { +public class DeleteIndexTemplateAction extends ActionType { public static final DeleteIndexTemplateAction INSTANCE = new DeleteIndexTemplateAction(); public static final String NAME = "indices:admin/template/delete"; @@ -32,7 +33,7 @@ private DeleteIndexTemplateAction() { } @Override - public AcknowledgedResponse newResponse() { - return new AcknowledgedResponse(); + public Writeable.Reader getResponseReader() { + return AcknowledgedResponse::new; } } diff --git a/server/src/main/java/org/elasticsearch/action/admin/indices/template/delete/TransportDeleteIndexTemplateAction.java b/server/src/main/java/org/elasticsearch/action/admin/indices/template/delete/TransportDeleteIndexTemplateAction.java index b2421ed5174e5..c59aac00f109d 100644 --- a/server/src/main/java/org/elasticsearch/action/admin/indices/template/delete/TransportDeleteIndexTemplateAction.java +++ b/server/src/main/java/org/elasticsearch/action/admin/indices/template/delete/TransportDeleteIndexTemplateAction.java @@ -30,9 +30,13 @@ import org.elasticsearch.cluster.metadata.MetaDataIndexTemplateService; import org.elasticsearch.cluster.service.ClusterService; import org.elasticsearch.common.inject.Inject; +import org.elasticsearch.common.io.stream.StreamInput; +import org.elasticsearch.tasks.Task; import org.elasticsearch.threadpool.ThreadPool; import org.elasticsearch.transport.TransportService; +import java.io.IOException; + /** * Delete index action. */ @@ -56,9 +60,14 @@ protected String executor() { return ThreadPool.Names.SAME; } + @Override + protected AcknowledgedResponse read(StreamInput in) throws IOException { + return new AcknowledgedResponse(in); + } + @Override protected AcknowledgedResponse newResponse() { - return new AcknowledgedResponse(); + throw new UnsupportedOperationException("usage of Streamable is to be replaced by Writeable"); } @Override @@ -67,7 +76,7 @@ protected ClusterBlockException checkBlock(DeleteIndexTemplateRequest request, C } @Override - protected void masterOperation(final DeleteIndexTemplateRequest request, final ClusterState state, + protected void masterOperation(Task task, final DeleteIndexTemplateRequest request, final ClusterState state, final ActionListener listener) { indexTemplateService.removeTemplates( new MetaDataIndexTemplateService diff --git a/server/src/main/java/org/elasticsearch/action/admin/indices/template/get/GetIndexTemplatesAction.java b/server/src/main/java/org/elasticsearch/action/admin/indices/template/get/GetIndexTemplatesAction.java index ac00b80079ca1..743c8e4f1a937 100644 --- a/server/src/main/java/org/elasticsearch/action/admin/indices/template/get/GetIndexTemplatesAction.java +++ b/server/src/main/java/org/elasticsearch/action/admin/indices/template/get/GetIndexTemplatesAction.java @@ -18,9 +18,9 @@ */ package org.elasticsearch.action.admin.indices.template.get; -import org.elasticsearch.action.Action; +import org.elasticsearch.action.StreamableResponseActionType; -public class GetIndexTemplatesAction extends Action { +public class GetIndexTemplatesAction extends StreamableResponseActionType { public static final GetIndexTemplatesAction INSTANCE = new GetIndexTemplatesAction(); public static final String NAME = "indices:admin/template/get"; diff --git a/server/src/main/java/org/elasticsearch/action/admin/indices/template/get/TransportGetIndexTemplatesAction.java b/server/src/main/java/org/elasticsearch/action/admin/indices/template/get/TransportGetIndexTemplatesAction.java index cd4cb6494e92d..1d733667fc3d5 100644 --- a/server/src/main/java/org/elasticsearch/action/admin/indices/template/get/TransportGetIndexTemplatesAction.java +++ b/server/src/main/java/org/elasticsearch/action/admin/indices/template/get/TransportGetIndexTemplatesAction.java @@ -30,6 +30,7 @@ import org.elasticsearch.cluster.service.ClusterService; import org.elasticsearch.common.inject.Inject; import org.elasticsearch.common.regex.Regex; +import org.elasticsearch.tasks.Task; import org.elasticsearch.threadpool.ThreadPool; import org.elasticsearch.transport.TransportService; @@ -63,7 +64,7 @@ protected GetIndexTemplatesResponse newResponse() { } @Override - protected void masterOperation(GetIndexTemplatesRequest request, ClusterState state, + protected void masterOperation(Task task, GetIndexTemplatesRequest request, ClusterState state, ActionListener listener) { List results; diff --git a/server/src/main/java/org/elasticsearch/action/admin/indices/template/put/PutIndexTemplateAction.java b/server/src/main/java/org/elasticsearch/action/admin/indices/template/put/PutIndexTemplateAction.java index 1edc49e991453..2e57d252deab4 100644 --- a/server/src/main/java/org/elasticsearch/action/admin/indices/template/put/PutIndexTemplateAction.java +++ b/server/src/main/java/org/elasticsearch/action/admin/indices/template/put/PutIndexTemplateAction.java @@ -19,10 +19,11 @@ package org.elasticsearch.action.admin.indices.template.put; -import org.elasticsearch.action.Action; +import org.elasticsearch.action.ActionType; import org.elasticsearch.action.support.master.AcknowledgedResponse; +import org.elasticsearch.common.io.stream.Writeable; -public class PutIndexTemplateAction extends Action { +public class PutIndexTemplateAction extends ActionType { public static final PutIndexTemplateAction INSTANCE = new PutIndexTemplateAction(); public static final String NAME = "indices:admin/template/put"; @@ -32,7 +33,7 @@ private PutIndexTemplateAction() { } @Override - public AcknowledgedResponse newResponse() { - return new AcknowledgedResponse(); + public Writeable.Reader getResponseReader() { + return AcknowledgedResponse::new; } } diff --git a/server/src/main/java/org/elasticsearch/action/admin/indices/template/put/TransportPutIndexTemplateAction.java b/server/src/main/java/org/elasticsearch/action/admin/indices/template/put/TransportPutIndexTemplateAction.java index 0202d67814888..95e6cf9a7d158 100644 --- a/server/src/main/java/org/elasticsearch/action/admin/indices/template/put/TransportPutIndexTemplateAction.java +++ b/server/src/main/java/org/elasticsearch/action/admin/indices/template/put/TransportPutIndexTemplateAction.java @@ -31,11 +31,15 @@ import org.elasticsearch.cluster.metadata.MetaDataIndexTemplateService; import org.elasticsearch.cluster.service.ClusterService; import org.elasticsearch.common.inject.Inject; +import org.elasticsearch.common.io.stream.StreamInput; import org.elasticsearch.common.settings.IndexScopedSettings; import org.elasticsearch.common.settings.Settings; +import org.elasticsearch.tasks.Task; import org.elasticsearch.threadpool.ThreadPool; import org.elasticsearch.transport.TransportService; +import java.io.IOException; + /** * Put index template action. */ @@ -61,9 +65,14 @@ protected String executor() { return ThreadPool.Names.SAME; } + @Override + protected AcknowledgedResponse read(StreamInput in) throws IOException { + return new AcknowledgedResponse(in); + } + @Override protected AcknowledgedResponse newResponse() { - return new AcknowledgedResponse(); + throw new UnsupportedOperationException("usage of Streamable is to be replaced by Writeable"); } @Override @@ -72,7 +81,7 @@ protected ClusterBlockException checkBlock(PutIndexTemplateRequest request, Clus } @Override - protected void masterOperation(final PutIndexTemplateRequest request, final ClusterState state, + protected void masterOperation(Task task, final PutIndexTemplateRequest request, final ClusterState state, final ActionListener listener) { String cause = request.cause(); if (cause.length() == 0) { diff --git a/server/src/main/java/org/elasticsearch/action/admin/indices/upgrade/get/UpgradeStatusAction.java b/server/src/main/java/org/elasticsearch/action/admin/indices/upgrade/get/UpgradeStatusAction.java index 57506b615d692..f111e342aaff3 100644 --- a/server/src/main/java/org/elasticsearch/action/admin/indices/upgrade/get/UpgradeStatusAction.java +++ b/server/src/main/java/org/elasticsearch/action/admin/indices/upgrade/get/UpgradeStatusAction.java @@ -19,9 +19,9 @@ package org.elasticsearch.action.admin.indices.upgrade.get; -import org.elasticsearch.action.Action; +import org.elasticsearch.action.StreamableResponseActionType; -public class UpgradeStatusAction extends Action { +public class UpgradeStatusAction extends StreamableResponseActionType { public static final UpgradeStatusAction INSTANCE = new UpgradeStatusAction(); public static final String NAME = "indices:monitor/upgrade"; diff --git a/server/src/main/java/org/elasticsearch/action/admin/indices/upgrade/post/TransportUpgradeSettingsAction.java b/server/src/main/java/org/elasticsearch/action/admin/indices/upgrade/post/TransportUpgradeSettingsAction.java index 645c67b3fdc05..a747b6f058bc6 100644 --- a/server/src/main/java/org/elasticsearch/action/admin/indices/upgrade/post/TransportUpgradeSettingsAction.java +++ b/server/src/main/java/org/elasticsearch/action/admin/indices/upgrade/post/TransportUpgradeSettingsAction.java @@ -32,9 +32,13 @@ import org.elasticsearch.cluster.metadata.MetaDataUpdateSettingsService; import org.elasticsearch.cluster.service.ClusterService; import org.elasticsearch.common.inject.Inject; +import org.elasticsearch.common.io.stream.StreamInput; +import org.elasticsearch.tasks.Task; import org.elasticsearch.threadpool.ThreadPool; import org.elasticsearch.transport.TransportService; +import java.io.IOException; + public class TransportUpgradeSettingsAction extends TransportMasterNodeAction { private final MetaDataUpdateSettingsService updateSettingsService; @@ -59,13 +63,18 @@ protected ClusterBlockException checkBlock(UpgradeSettingsRequest request, Clust return state.blocks().globalBlockedException(ClusterBlockLevel.METADATA_WRITE); } + @Override + protected AcknowledgedResponse read(StreamInput in) throws IOException { + return new AcknowledgedResponse(in); + } + @Override protected AcknowledgedResponse newResponse() { - return new AcknowledgedResponse(); + throw new UnsupportedOperationException("usage of Streamable is to be replaced by Writeable"); } @Override - protected void masterOperation(final UpgradeSettingsRequest request, final ClusterState state, + protected void masterOperation(Task task, final UpgradeSettingsRequest request, final ClusterState state, final ActionListener listener) { UpgradeSettingsClusterStateUpdateRequest clusterStateUpdateRequest = new UpgradeSettingsClusterStateUpdateRequest() .ackTimeout(request.timeout()) diff --git a/server/src/main/java/org/elasticsearch/action/admin/indices/upgrade/post/UpgradeAction.java b/server/src/main/java/org/elasticsearch/action/admin/indices/upgrade/post/UpgradeAction.java index 7ec83930e44bf..3a3344d1a2edb 100644 --- a/server/src/main/java/org/elasticsearch/action/admin/indices/upgrade/post/UpgradeAction.java +++ b/server/src/main/java/org/elasticsearch/action/admin/indices/upgrade/post/UpgradeAction.java @@ -19,12 +19,12 @@ package org.elasticsearch.action.admin.indices.upgrade.post; -import org.elasticsearch.action.Action; +import org.elasticsearch.action.StreamableResponseActionType; /** * Upgrade index/indices action. */ -public class UpgradeAction extends Action { +public class UpgradeAction extends StreamableResponseActionType { public static final UpgradeAction INSTANCE = new UpgradeAction(); public static final String NAME = "indices:admin/upgrade"; diff --git a/server/src/main/java/org/elasticsearch/action/admin/indices/upgrade/post/UpgradeSettingsAction.java b/server/src/main/java/org/elasticsearch/action/admin/indices/upgrade/post/UpgradeSettingsAction.java index 9b1ff0f616a54..3876e7be88225 100644 --- a/server/src/main/java/org/elasticsearch/action/admin/indices/upgrade/post/UpgradeSettingsAction.java +++ b/server/src/main/java/org/elasticsearch/action/admin/indices/upgrade/post/UpgradeSettingsAction.java @@ -19,10 +19,11 @@ package org.elasticsearch.action.admin.indices.upgrade.post; -import org.elasticsearch.action.Action; +import org.elasticsearch.action.ActionType; import org.elasticsearch.action.support.master.AcknowledgedResponse; +import org.elasticsearch.common.io.stream.Writeable; -public class UpgradeSettingsAction extends Action { +public class UpgradeSettingsAction extends ActionType { public static final UpgradeSettingsAction INSTANCE = new UpgradeSettingsAction(); public static final String NAME = "internal:indices/admin/upgrade"; @@ -32,7 +33,7 @@ private UpgradeSettingsAction() { } @Override - public AcknowledgedResponse newResponse() { - return new AcknowledgedResponse(); + public Writeable.Reader getResponseReader() { + return AcknowledgedResponse::new; } } diff --git a/server/src/main/java/org/elasticsearch/action/admin/indices/validate/query/TransportValidateQueryAction.java b/server/src/main/java/org/elasticsearch/action/admin/indices/validate/query/TransportValidateQueryAction.java index 6e10d3d42187f..8f85e91d29ee2 100644 --- a/server/src/main/java/org/elasticsearch/action/admin/indices/validate/query/TransportValidateQueryAction.java +++ b/server/src/main/java/org/elasticsearch/action/admin/indices/validate/query/TransportValidateQueryAction.java @@ -193,7 +193,7 @@ protected ShardValidateQueryResponse shardOperation(ShardValidateQueryRequest re boolean valid; String explanation = null; String error = null; - ShardSearchLocalRequest shardSearchLocalRequest = new ShardSearchLocalRequest(request.shardId(), request.types(), + ShardSearchLocalRequest shardSearchLocalRequest = new ShardSearchLocalRequest(request.shardId(), request.nowInMillis(), request.filteringAliases()); SearchContext searchContext = searchService.createSearchContext(shardSearchLocalRequest, SearchService.NO_TIMEOUT); try { diff --git a/server/src/main/java/org/elasticsearch/action/admin/indices/validate/query/ValidateQueryAction.java b/server/src/main/java/org/elasticsearch/action/admin/indices/validate/query/ValidateQueryAction.java index 93151dd8a2ba9..713f94003642a 100644 --- a/server/src/main/java/org/elasticsearch/action/admin/indices/validate/query/ValidateQueryAction.java +++ b/server/src/main/java/org/elasticsearch/action/admin/indices/validate/query/ValidateQueryAction.java @@ -19,9 +19,9 @@ package org.elasticsearch.action.admin.indices.validate.query; -import org.elasticsearch.action.Action; +import org.elasticsearch.action.StreamableResponseActionType; -public class ValidateQueryAction extends Action { +public class ValidateQueryAction extends StreamableResponseActionType { public static final ValidateQueryAction INSTANCE = new ValidateQueryAction(); public static final String NAME = "indices:admin/validate/query"; diff --git a/server/src/main/java/org/elasticsearch/action/bulk/BulkAction.java b/server/src/main/java/org/elasticsearch/action/bulk/BulkAction.java index f835b57a38b4c..bd4ac7ddf4788 100644 --- a/server/src/main/java/org/elasticsearch/action/bulk/BulkAction.java +++ b/server/src/main/java/org/elasticsearch/action/bulk/BulkAction.java @@ -19,11 +19,11 @@ package org.elasticsearch.action.bulk; -import org.elasticsearch.action.Action; +import org.elasticsearch.action.StreamableResponseActionType; import org.elasticsearch.common.settings.Settings; import org.elasticsearch.transport.TransportRequestOptions; -public class BulkAction extends Action { +public class BulkAction extends StreamableResponseActionType { public static final BulkAction INSTANCE = new BulkAction(); public static final String NAME = "indices:data/write/bulk"; diff --git a/server/src/main/java/org/elasticsearch/action/bulk/TransportBulkAction.java b/server/src/main/java/org/elasticsearch/action/bulk/TransportBulkAction.java index eae849407666e..b0efa35819acb 100644 --- a/server/src/main/java/org/elasticsearch/action/bulk/TransportBulkAction.java +++ b/server/src/main/java/org/elasticsearch/action/bulk/TransportBulkAction.java @@ -96,7 +96,6 @@ public class TransportBulkAction extends HandledTransportAction) BulkRequest::new, ThreadPool.Names.WRITE); Objects.requireNonNull(relativeTimeProvider); this.threadPool = threadPool; this.clusterService = clusterService; this.ingestService = ingestService; - this.shardBulkAction = shardBulkAction; this.autoCreateIndex = autoCreateIndex; this.relativeTimeProvider = relativeTimeProvider; this.ingestForwarder = new IngestActionForwarder(transportService); @@ -165,16 +161,24 @@ protected void doExecute(Task task, BulkRequest bulkRequest, ActionListener() { + client.executeLocally(TransportShardBulkAction.TYPE, bulkShardRequest, new ActionListener<>() { @Override public void onResponse(BulkShardResponse bulkShardResponse) { for (BulkItemResponse bulkItemResponse : bulkShardResponse.getResponses()) { diff --git a/server/src/main/java/org/elasticsearch/action/bulk/TransportShardBulkAction.java b/server/src/main/java/org/elasticsearch/action/bulk/TransportShardBulkAction.java index f49521a37463e..6479af1c7afc5 100644 --- a/server/src/main/java/org/elasticsearch/action/bulk/TransportShardBulkAction.java +++ b/server/src/main/java/org/elasticsearch/action/bulk/TransportShardBulkAction.java @@ -24,10 +24,12 @@ import org.apache.logging.log4j.message.ParameterizedMessage; import org.apache.logging.log4j.util.MessageSupplier; import org.elasticsearch.ExceptionsHelper; +import org.elasticsearch.action.ActionType; import org.elasticsearch.action.ActionListener; import org.elasticsearch.action.ActionRunnable; import org.elasticsearch.action.DocWriteRequest; import org.elasticsearch.action.DocWriteResponse; +import org.elasticsearch.action.StreamableResponseActionType; import org.elasticsearch.action.delete.DeleteRequest; import org.elasticsearch.action.delete.DeleteResponse; import org.elasticsearch.action.index.IndexRequest; @@ -77,6 +79,12 @@ public class TransportShardBulkAction extends TransportWriteAction { public static final String ACTION_NAME = BulkAction.NAME + "[s]"; + public static final ActionType TYPE = new StreamableResponseActionType<>(ACTION_NAME) { + @Override + public BulkShardResponse newResponse() { + return new BulkShardResponse(); + } + }; private static final Logger logger = LogManager.getLogger(TransportShardBulkAction.class); @@ -341,11 +349,10 @@ private static boolean isConflictException(final Exception e) { private static BulkItemResponse processUpdateResponse(final UpdateRequest updateRequest, final String concreteIndex, BulkItemResponse operationResponse, final UpdateHelper.Result translate) { final BulkItemResponse response; - DocWriteResponse.Result translatedResult = translate.getResponseResult(); if (operationResponse.isFailed()) { response = new BulkItemResponse(operationResponse.getItemId(), DocWriteRequest.OpType.UPDATE, operationResponse.getFailure()); } else { - + final DocWriteResponse.Result translatedResult = translate.getResponseResult(); final UpdateResponse updateResponse; if (translatedResult == DocWriteResponse.Result.CREATED || translatedResult == DocWriteResponse.Result.UPDATED) { final IndexRequest updateIndexRequest = translate.action(); diff --git a/server/src/main/java/org/elasticsearch/action/delete/DeleteAction.java b/server/src/main/java/org/elasticsearch/action/delete/DeleteAction.java index d78b6f60bffc3..be299715095dc 100644 --- a/server/src/main/java/org/elasticsearch/action/delete/DeleteAction.java +++ b/server/src/main/java/org/elasticsearch/action/delete/DeleteAction.java @@ -19,9 +19,9 @@ package org.elasticsearch.action.delete; -import org.elasticsearch.action.Action; +import org.elasticsearch.action.StreamableResponseActionType; -public class DeleteAction extends Action { +public class DeleteAction extends StreamableResponseActionType { public static final DeleteAction INSTANCE = new DeleteAction(); public static final String NAME = "indices:data/write/delete"; diff --git a/server/src/main/java/org/elasticsearch/action/explain/ExplainAction.java b/server/src/main/java/org/elasticsearch/action/explain/ExplainAction.java index ba5618ce7de21..2eab7215fce46 100644 --- a/server/src/main/java/org/elasticsearch/action/explain/ExplainAction.java +++ b/server/src/main/java/org/elasticsearch/action/explain/ExplainAction.java @@ -19,13 +19,13 @@ package org.elasticsearch.action.explain; -import org.elasticsearch.action.Action; +import org.elasticsearch.action.ActionType; import org.elasticsearch.common.io.stream.Writeable; /** * Entry point for the explain feature. */ -public class ExplainAction extends Action { +public class ExplainAction extends ActionType { public static final ExplainAction INSTANCE = new ExplainAction(); public static final String NAME = "indices:data/read/explain"; @@ -34,11 +34,6 @@ private ExplainAction() { super(NAME); } - @Override - public ExplainResponse newResponse() { - throw new UnsupportedOperationException("usage of Streamable is to be replaced by Writeable"); - } - @Override public Writeable.Reader getResponseReader() { return ExplainResponse::new; diff --git a/server/src/main/java/org/elasticsearch/action/explain/ExplainRequest.java b/server/src/main/java/org/elasticsearch/action/explain/ExplainRequest.java index 260c63d340a6f..bbb9d24469c9e 100644 --- a/server/src/main/java/org/elasticsearch/action/explain/ExplainRequest.java +++ b/server/src/main/java/org/elasticsearch/action/explain/ExplainRequest.java @@ -74,6 +74,19 @@ public ExplainRequest(String index, String id) { this.id = id; } + ExplainRequest(StreamInput in) throws IOException { + super(in); + type = in.readString(); + id = in.readString(); + routing = in.readOptionalString(); + preference = in.readOptionalString(); + query = in.readNamedWriteable(QueryBuilder.class); + filteringAlias = new AliasFilter(in); + storedFields = in.readOptionalStringArray(); + fetchSourceContext = in.readOptionalWriteable(FetchSourceContext::new); + nowInMillis = in.readVLong(); + } + /** * @deprecated Types are in the process of being removed. */ @@ -184,20 +197,6 @@ public ActionRequestValidationException validate() { return validationException; } - @Override - public void readFrom(StreamInput in) throws IOException { - super.readFrom(in); - type = in.readString(); - id = in.readString(); - routing = in.readOptionalString(); - preference = in.readOptionalString(); - query = in.readNamedWriteable(QueryBuilder.class); - filteringAlias = new AliasFilter(in); - storedFields = in.readOptionalStringArray(); - fetchSourceContext = in.readOptionalWriteable(FetchSourceContext::new); - nowInMillis = in.readVLong(); - } - @Override public void writeTo(StreamOutput out) throws IOException { super.writeTo(out); diff --git a/server/src/main/java/org/elasticsearch/action/explain/TransportExplainAction.java b/server/src/main/java/org/elasticsearch/action/explain/TransportExplainAction.java index c29da21fe4afe..e6a2ef5be9d04 100644 --- a/server/src/main/java/org/elasticsearch/action/explain/TransportExplainAction.java +++ b/server/src/main/java/org/elasticsearch/action/explain/TransportExplainAction.java @@ -30,7 +30,6 @@ import org.elasticsearch.cluster.metadata.IndexNameExpressionResolver; import org.elasticsearch.cluster.routing.ShardIterator; import org.elasticsearch.cluster.service.ClusterService; -import org.elasticsearch.common.Strings; import org.elasticsearch.common.inject.Inject; import org.elasticsearch.common.io.stream.Writeable; import org.elasticsearch.common.lease.Releasables; @@ -38,7 +37,6 @@ import org.elasticsearch.index.engine.Engine; import org.elasticsearch.index.get.GetResult; import org.elasticsearch.index.mapper.IdFieldMapper; -import org.elasticsearch.index.mapper.MapperService; import org.elasticsearch.index.mapper.Uid; import org.elasticsearch.index.shard.IndexShard; import org.elasticsearch.index.shard.ShardId; @@ -110,14 +108,8 @@ protected void asyncShardOperation(ExplainRequest request, ShardId shardId, @Override protected ExplainResponse shardOperation(ExplainRequest request, ShardId shardId) throws IOException { - String[] types; - if (MapperService.SINGLE_MAPPING_NAME.equals(request.type())) { // typeless explain call - types = Strings.EMPTY_ARRAY; - } else { - types = new String[] { request.type() }; - } - ShardSearchLocalRequest shardSearchLocalRequest = new ShardSearchLocalRequest(shardId, - types, request.nowInMillis, request.filteringAlias()); + ShardSearchLocalRequest shardSearchLocalRequest = new ShardSearchLocalRequest(shardId, request.nowInMillis, + request.filteringAlias()); SearchContext context = searchService.createSearchContext(shardSearchLocalRequest, SearchService.NO_TIMEOUT); Engine.GetResult result = null; try { diff --git a/server/src/main/java/org/elasticsearch/action/fieldcaps/FieldCapabilitiesAction.java b/server/src/main/java/org/elasticsearch/action/fieldcaps/FieldCapabilitiesAction.java index 39c6ecce308e0..e64a3b67c102b 100644 --- a/server/src/main/java/org/elasticsearch/action/fieldcaps/FieldCapabilitiesAction.java +++ b/server/src/main/java/org/elasticsearch/action/fieldcaps/FieldCapabilitiesAction.java @@ -19,9 +19,9 @@ package org.elasticsearch.action.fieldcaps; -import org.elasticsearch.action.Action; +import org.elasticsearch.action.StreamableResponseActionType; -public class FieldCapabilitiesAction extends Action { +public class FieldCapabilitiesAction extends StreamableResponseActionType { public static final FieldCapabilitiesAction INSTANCE = new FieldCapabilitiesAction(); public static final String NAME = "indices:data/read/field_caps"; diff --git a/server/src/main/java/org/elasticsearch/action/fieldcaps/FieldCapabilitiesIndexRequest.java b/server/src/main/java/org/elasticsearch/action/fieldcaps/FieldCapabilitiesIndexRequest.java index 81266f12e37b3..6be8daacd51e8 100644 --- a/server/src/main/java/org/elasticsearch/action/fieldcaps/FieldCapabilitiesIndexRequest.java +++ b/server/src/main/java/org/elasticsearch/action/fieldcaps/FieldCapabilitiesIndexRequest.java @@ -30,11 +30,15 @@ public class FieldCapabilitiesIndexRequest extends SingleShardRequest { - private String[] fields; - private OriginalIndices originalIndices; + private final String[] fields; + private final OriginalIndices originalIndices; // For serialization - FieldCapabilitiesIndexRequest() {} + FieldCapabilitiesIndexRequest(StreamInput in) throws IOException { + super(in); + fields = in.readStringArray(); + originalIndices = OriginalIndices.readOriginalIndices(in); + } FieldCapabilitiesIndexRequest(String[] fields, String index, OriginalIndices originalIndices) { super(index); @@ -61,13 +65,6 @@ public IndicesOptions indicesOptions() { return originalIndices.indicesOptions(); } - @Override - public void readFrom(StreamInput in) throws IOException { - super.readFrom(in); - fields = in.readStringArray(); - originalIndices = OriginalIndices.readOriginalIndices(in); - } - @Override public void writeTo(StreamOutput out) throws IOException { super.writeTo(out); 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 ffb98dd5224ba..3176d0d31390b 100644 --- a/server/src/main/java/org/elasticsearch/action/fieldcaps/TransportFieldCapabilitiesAction.java +++ b/server/src/main/java/org/elasticsearch/action/fieldcaps/TransportFieldCapabilitiesAction.java @@ -24,6 +24,7 @@ import org.elasticsearch.action.support.ActionFilters; import org.elasticsearch.action.support.HandledTransportAction; import org.elasticsearch.client.Client; +import org.elasticsearch.client.node.NodeClient; import org.elasticsearch.cluster.ClusterState; import org.elasticsearch.cluster.metadata.IndexNameExpressionResolver; import org.elasticsearch.cluster.service.ClusterService; @@ -47,21 +48,20 @@ public class TransportFieldCapabilitiesAction extends HandledTransportAction { private final ThreadPool threadPool; + private final NodeClient client; private final ClusterService clusterService; - private final TransportFieldCapabilitiesIndexAction shardAction; private final RemoteClusterService remoteClusterService; private final IndexNameExpressionResolver indexNameExpressionResolver; @Inject - public TransportFieldCapabilitiesAction(TransportService transportService, - ClusterService clusterService, ThreadPool threadPool, - TransportFieldCapabilitiesIndexAction shardAction, - ActionFilters actionFilters, IndexNameExpressionResolver indexNameExpressionResolver) { + public TransportFieldCapabilitiesAction(TransportService transportService, ClusterService clusterService, ThreadPool threadPool, + NodeClient client, ActionFilters actionFilters, + IndexNameExpressionResolver indexNameExpressionResolver) { super(FieldCapabilitiesAction.NAME, transportService, actionFilters, FieldCapabilitiesRequest::new); this.threadPool = threadPool; + this.client = client; this.clusterService = clusterService; this.remoteClusterService = transportService.getRemoteClusterService(); - this.shardAction = shardAction; this.indexNameExpressionResolver = indexNameExpressionResolver; } @@ -108,7 +108,8 @@ public void onFailure(Exception e) { } }; for (String index : concreteIndices) { - shardAction.execute(new FieldCapabilitiesIndexRequest(request.fields(), index, localIndices), innerListener); + client.executeLocally(TransportFieldCapabilitiesIndexAction.TYPE, + new FieldCapabilitiesIndexRequest(request.fields(), index, localIndices), innerListener); } // this is the cross cluster part of this API - we force the other cluster to not merge the results but instead diff --git a/server/src/main/java/org/elasticsearch/action/fieldcaps/TransportFieldCapabilitiesIndexAction.java b/server/src/main/java/org/elasticsearch/action/fieldcaps/TransportFieldCapabilitiesIndexAction.java index 274633b12a613..f391bf82eb944 100644 --- a/server/src/main/java/org/elasticsearch/action/fieldcaps/TransportFieldCapabilitiesIndexAction.java +++ b/server/src/main/java/org/elasticsearch/action/fieldcaps/TransportFieldCapabilitiesIndexAction.java @@ -19,6 +19,7 @@ package org.elasticsearch.action.fieldcaps; +import org.elasticsearch.action.ActionType; import org.elasticsearch.action.support.ActionFilters; import org.elasticsearch.action.support.single.shard.TransportSingleShardAction; import org.elasticsearch.cluster.ClusterState; @@ -47,6 +48,8 @@ public class TransportFieldCapabilitiesIndexAction extends TransportSingleShardA FieldCapabilitiesIndexResponse> { private static final String ACTION_NAME = FieldCapabilitiesAction.NAME + "[index]"; + public static final ActionType TYPE = + new ActionType<>(ACTION_NAME, FieldCapabilitiesIndexResponse::new); private final IndicesService indicesService; diff --git a/server/src/main/java/org/elasticsearch/action/get/GetAction.java b/server/src/main/java/org/elasticsearch/action/get/GetAction.java index 05d1b6c5a4c02..a36ec9d2fdc79 100644 --- a/server/src/main/java/org/elasticsearch/action/get/GetAction.java +++ b/server/src/main/java/org/elasticsearch/action/get/GetAction.java @@ -19,10 +19,10 @@ package org.elasticsearch.action.get; -import org.elasticsearch.action.Action; +import org.elasticsearch.action.ActionType; import org.elasticsearch.common.io.stream.Writeable; -public class GetAction extends Action { +public class GetAction extends ActionType { public static final GetAction INSTANCE = new GetAction(); public static final String NAME = "indices:data/read/get"; @@ -31,11 +31,6 @@ private GetAction() { super(NAME); } - @Override - public GetResponse newResponse() { - throw new UnsupportedOperationException("usage of Streamable is to be replaced by Writeable"); - } - @Override public Writeable.Reader getResponseReader() { return GetResponse::new; diff --git a/server/src/main/java/org/elasticsearch/action/get/GetRequest.java b/server/src/main/java/org/elasticsearch/action/get/GetRequest.java index 390553fd8b55b..932602b3a2d22 100644 --- a/server/src/main/java/org/elasticsearch/action/get/GetRequest.java +++ b/server/src/main/java/org/elasticsearch/action/get/GetRequest.java @@ -69,6 +69,21 @@ public GetRequest() { type = MapperService.SINGLE_MAPPING_NAME; } + GetRequest(StreamInput in) throws IOException { + super(in); + type = in.readString(); + id = in.readString(); + routing = in.readOptionalString(); + preference = in.readOptionalString(); + refresh = in.readBoolean(); + storedFields = in.readOptionalStringArray(); + realtime = in.readBoolean(); + + this.versionType = VersionType.fromValue(in.readByte()); + this.version = in.readLong(); + fetchSourceContext = in.readOptionalWriteable(FetchSourceContext::new); + } + /** * Constructs a new get request against the specified index. The {@link #id(String)} must also be set. */ @@ -261,22 +276,6 @@ public VersionType versionType() { return this.versionType; } - @Override - public void readFrom(StreamInput in) throws IOException { - super.readFrom(in); - type = in.readString(); - id = in.readString(); - routing = in.readOptionalString(); - preference = in.readOptionalString(); - refresh = in.readBoolean(); - storedFields = in.readOptionalStringArray(); - realtime = in.readBoolean(); - - this.versionType = VersionType.fromValue(in.readByte()); - this.version = in.readLong(); - fetchSourceContext = in.readOptionalWriteable(FetchSourceContext::new); - } - @Override public void writeTo(StreamOutput out) throws IOException { super.writeTo(out); diff --git a/server/src/main/java/org/elasticsearch/action/get/MultiGetAction.java b/server/src/main/java/org/elasticsearch/action/get/MultiGetAction.java index 9b69e33239b82..9d989c5644d00 100644 --- a/server/src/main/java/org/elasticsearch/action/get/MultiGetAction.java +++ b/server/src/main/java/org/elasticsearch/action/get/MultiGetAction.java @@ -19,9 +19,9 @@ package org.elasticsearch.action.get; -import org.elasticsearch.action.Action; +import org.elasticsearch.action.StreamableResponseActionType; -public class MultiGetAction extends Action { +public class MultiGetAction extends StreamableResponseActionType { public static final MultiGetAction INSTANCE = new MultiGetAction(); public static final String NAME = "indices:data/read/mget"; diff --git a/server/src/main/java/org/elasticsearch/action/get/MultiGetShardRequest.java b/server/src/main/java/org/elasticsearch/action/get/MultiGetShardRequest.java index 55f1fc6f5b9dd..1e66766aaabfa 100644 --- a/server/src/main/java/org/elasticsearch/action/get/MultiGetShardRequest.java +++ b/server/src/main/java/org/elasticsearch/action/get/MultiGetShardRequest.java @@ -33,14 +33,26 @@ public class MultiGetShardRequest extends SingleShardRequest items; - public MultiGetShardRequest() { + MultiGetShardRequest(StreamInput in) throws IOException { + super(in); + int size = in.readVInt(); + locations = new IntArrayList(size); + items = new ArrayList<>(size); + for (int i = 0; i < size; i++) { + locations.add(in.readVInt()); + items.add(MultiGetRequest.Item.readItem(in)); + } + + preference = in.readOptionalString(); + refresh = in.readBoolean(); + realtime = in.readBoolean(); } MultiGetShardRequest(MultiGetRequest multiGetRequest, String index, int shardId) { @@ -108,23 +120,6 @@ public String[] indices() { return indices; } - @Override - public void readFrom(StreamInput in) throws IOException { - super.readFrom(in); - int size = in.readVInt(); - locations = new IntArrayList(size); - items = new ArrayList<>(size); - - for (int i = 0; i < size; i++) { - locations.add(in.readVInt()); - items.add(MultiGetRequest.Item.readItem(in)); - } - - preference = in.readOptionalString(); - refresh = in.readBoolean(); - realtime = in.readBoolean(); - } - @Override public void writeTo(StreamOutput out) throws IOException { super.writeTo(out); diff --git a/server/src/main/java/org/elasticsearch/action/get/TransportMultiGetAction.java b/server/src/main/java/org/elasticsearch/action/get/TransportMultiGetAction.java index 375bdc18b605a..d98fd30b00d14 100644 --- a/server/src/main/java/org/elasticsearch/action/get/TransportMultiGetAction.java +++ b/server/src/main/java/org/elasticsearch/action/get/TransportMultiGetAction.java @@ -23,6 +23,7 @@ import org.elasticsearch.action.RoutingMissingException; import org.elasticsearch.action.support.ActionFilters; import org.elasticsearch.action.support.HandledTransportAction; +import org.elasticsearch.client.node.NodeClient; import org.elasticsearch.cluster.ClusterState; import org.elasticsearch.cluster.block.ClusterBlockLevel; import org.elasticsearch.cluster.metadata.IndexNameExpressionResolver; @@ -40,16 +41,16 @@ public class TransportMultiGetAction extends HandledTransportAction { private final ClusterService clusterService; - private final TransportShardMultiGetAction shardAction; + private final NodeClient client; private final IndexNameExpressionResolver indexNameExpressionResolver; @Inject public TransportMultiGetAction(TransportService transportService, ClusterService clusterService, - TransportShardMultiGetAction shardAction, ActionFilters actionFilters, + NodeClient client, ActionFilters actionFilters, IndexNameExpressionResolver resolver) { super(MultiGetAction.NAME, transportService, actionFilters, MultiGetRequest::new); this.clusterService = clusterService; - this.shardAction = shardAction; + this.client = client; this.indexNameExpressionResolver = resolver; } @@ -105,7 +106,7 @@ protected void executeShardAction(ActionListener listener, final AtomicInteger counter = new AtomicInteger(shardRequests.size()); for (final MultiGetShardRequest shardRequest : shardRequests.values()) { - shardAction.execute(shardRequest, new ActionListener() { + client.executeLocally(TransportShardMultiGetAction.TYPE, shardRequest, new ActionListener<>() { @Override public void onResponse(MultiGetShardResponse response) { for (int i = 0; i < response.locations.size(); i++) { diff --git a/server/src/main/java/org/elasticsearch/action/get/TransportShardMultiGetAction.java b/server/src/main/java/org/elasticsearch/action/get/TransportShardMultiGetAction.java index 9b8ea6bd6cac5..e7921ca489ce8 100644 --- a/server/src/main/java/org/elasticsearch/action/get/TransportShardMultiGetAction.java +++ b/server/src/main/java/org/elasticsearch/action/get/TransportShardMultiGetAction.java @@ -20,6 +20,7 @@ package org.elasticsearch.action.get; import org.apache.logging.log4j.message.ParameterizedMessage; +import org.elasticsearch.action.ActionType; import org.elasticsearch.action.support.ActionFilters; import org.elasticsearch.action.support.TransportActions; import org.elasticsearch.action.support.single.shard.TransportSingleShardAction; @@ -40,6 +41,7 @@ public class TransportShardMultiGetAction extends TransportSingleShardAction { private static final String ACTION_NAME = MultiGetAction.NAME + "[shard]"; + public static final ActionType TYPE = new ActionType<>(ACTION_NAME, MultiGetShardResponse::new); private final IndicesService indicesService; diff --git a/server/src/main/java/org/elasticsearch/action/index/IndexAction.java b/server/src/main/java/org/elasticsearch/action/index/IndexAction.java index 4f3e6068a2a23..b6afe88b770dd 100644 --- a/server/src/main/java/org/elasticsearch/action/index/IndexAction.java +++ b/server/src/main/java/org/elasticsearch/action/index/IndexAction.java @@ -19,9 +19,9 @@ package org.elasticsearch.action.index; -import org.elasticsearch.action.Action; +import org.elasticsearch.action.StreamableResponseActionType; -public class IndexAction extends Action { +public class IndexAction extends StreamableResponseActionType { public static final IndexAction INSTANCE = new IndexAction(); public static final String NAME = "indices:data/write/index"; diff --git a/server/src/main/java/org/elasticsearch/action/ingest/DeletePipelineAction.java b/server/src/main/java/org/elasticsearch/action/ingest/DeletePipelineAction.java index 902614a3e83b3..55565960c658c 100644 --- a/server/src/main/java/org/elasticsearch/action/ingest/DeletePipelineAction.java +++ b/server/src/main/java/org/elasticsearch/action/ingest/DeletePipelineAction.java @@ -19,10 +19,11 @@ package org.elasticsearch.action.ingest; -import org.elasticsearch.action.Action; +import org.elasticsearch.action.ActionType; import org.elasticsearch.action.support.master.AcknowledgedResponse; +import org.elasticsearch.common.io.stream.Writeable; -public class DeletePipelineAction extends Action { +public class DeletePipelineAction extends ActionType { public static final DeletePipelineAction INSTANCE = new DeletePipelineAction(); public static final String NAME = "cluster:admin/ingest/pipeline/delete"; @@ -32,7 +33,7 @@ public DeletePipelineAction() { } @Override - public AcknowledgedResponse newResponse() { - return new AcknowledgedResponse(); + public Writeable.Reader getResponseReader() { + return AcknowledgedResponse::new; } } diff --git a/server/src/main/java/org/elasticsearch/action/ingest/DeletePipelineTransportAction.java b/server/src/main/java/org/elasticsearch/action/ingest/DeletePipelineTransportAction.java index 0b5145ff18cb9..8f69d20968014 100644 --- a/server/src/main/java/org/elasticsearch/action/ingest/DeletePipelineTransportAction.java +++ b/server/src/main/java/org/elasticsearch/action/ingest/DeletePipelineTransportAction.java @@ -28,10 +28,14 @@ import org.elasticsearch.cluster.block.ClusterBlockLevel; import org.elasticsearch.cluster.metadata.IndexNameExpressionResolver; import org.elasticsearch.common.inject.Inject; +import org.elasticsearch.common.io.stream.StreamInput; import org.elasticsearch.ingest.IngestService; +import org.elasticsearch.tasks.Task; import org.elasticsearch.threadpool.ThreadPool; import org.elasticsearch.transport.TransportService; +import java.io.IOException; + public class DeletePipelineTransportAction extends TransportMasterNodeAction { private final IngestService ingestService; @@ -49,14 +53,19 @@ protected String executor() { return ThreadPool.Names.SAME; } + @Override + protected AcknowledgedResponse read(StreamInput in) throws IOException { + return new AcknowledgedResponse(in); + } + @Override protected AcknowledgedResponse newResponse() { - return new AcknowledgedResponse(); + throw new UnsupportedOperationException("usage of Streamable is to be replaced by Writeable"); } @Override - protected void masterOperation(DeletePipelineRequest request, ClusterState state, - ActionListener listener) throws Exception { + protected void masterOperation(Task task, DeletePipelineRequest request, ClusterState state, + ActionListener listener) throws Exception { ingestService.delete(request, listener); } diff --git a/server/src/main/java/org/elasticsearch/action/ingest/GetPipelineAction.java b/server/src/main/java/org/elasticsearch/action/ingest/GetPipelineAction.java index b2305227ac67f..48340af4039c3 100644 --- a/server/src/main/java/org/elasticsearch/action/ingest/GetPipelineAction.java +++ b/server/src/main/java/org/elasticsearch/action/ingest/GetPipelineAction.java @@ -19,9 +19,9 @@ package org.elasticsearch.action.ingest; -import org.elasticsearch.action.Action; +import org.elasticsearch.action.StreamableResponseActionType; -public class GetPipelineAction extends Action { +public class GetPipelineAction extends StreamableResponseActionType { public static final GetPipelineAction INSTANCE = new GetPipelineAction(); public static final String NAME = "cluster:admin/ingest/pipeline/get"; diff --git a/server/src/main/java/org/elasticsearch/action/ingest/GetPipelineTransportAction.java b/server/src/main/java/org/elasticsearch/action/ingest/GetPipelineTransportAction.java index e24e0a833b4c9..4501629284e63 100644 --- a/server/src/main/java/org/elasticsearch/action/ingest/GetPipelineTransportAction.java +++ b/server/src/main/java/org/elasticsearch/action/ingest/GetPipelineTransportAction.java @@ -29,6 +29,7 @@ import org.elasticsearch.cluster.service.ClusterService; import org.elasticsearch.common.inject.Inject; import org.elasticsearch.ingest.IngestService; +import org.elasticsearch.tasks.Task; import org.elasticsearch.threadpool.ThreadPool; import org.elasticsearch.transport.TransportService; @@ -53,7 +54,7 @@ protected GetPipelineResponse newResponse() { } @Override - protected void masterOperation(GetPipelineRequest request, ClusterState state, ActionListener listener) + protected void masterOperation(Task task, GetPipelineRequest request, ClusterState state, ActionListener listener) throws Exception { listener.onResponse(new GetPipelineResponse(IngestService.getPipelines(state, request.getIds()))); } diff --git a/server/src/main/java/org/elasticsearch/action/ingest/IngestActionForwarder.java b/server/src/main/java/org/elasticsearch/action/ingest/IngestActionForwarder.java index ae5a736bde66d..db4ecf119421f 100644 --- a/server/src/main/java/org/elasticsearch/action/ingest/IngestActionForwarder.java +++ b/server/src/main/java/org/elasticsearch/action/ingest/IngestActionForwarder.java @@ -19,7 +19,7 @@ package org.elasticsearch.action.ingest; -import org.elasticsearch.action.Action; +import org.elasticsearch.action.ActionType; import org.elasticsearch.action.ActionListener; import org.elasticsearch.action.ActionListenerResponseHandler; import org.elasticsearch.action.ActionRequest; @@ -47,7 +47,7 @@ public IngestActionForwarder(TransportService transportService) { ingestNodes = new DiscoveryNode[0]; } - public void forwardIngestRequest(Action action, ActionRequest request, ActionListener listener) { + public void forwardIngestRequest(ActionType action, ActionRequest request, ActionListener listener) { transportService.sendRequest(randomIngestNode(), action.name(), request, new ActionListenerResponseHandler(listener, action.getResponseReader())); } diff --git a/server/src/main/java/org/elasticsearch/action/ingest/PutPipelineAction.java b/server/src/main/java/org/elasticsearch/action/ingest/PutPipelineAction.java index 4ebcff127cccd..1a2aa7725129a 100644 --- a/server/src/main/java/org/elasticsearch/action/ingest/PutPipelineAction.java +++ b/server/src/main/java/org/elasticsearch/action/ingest/PutPipelineAction.java @@ -19,10 +19,11 @@ package org.elasticsearch.action.ingest; -import org.elasticsearch.action.Action; +import org.elasticsearch.action.ActionType; import org.elasticsearch.action.support.master.AcknowledgedResponse; +import org.elasticsearch.common.io.stream.Writeable; -public class PutPipelineAction extends Action { +public class PutPipelineAction extends ActionType { public static final PutPipelineAction INSTANCE = new PutPipelineAction(); public static final String NAME = "cluster:admin/ingest/pipeline/put"; @@ -32,7 +33,7 @@ public PutPipelineAction() { } @Override - public AcknowledgedResponse newResponse() { - return new AcknowledgedResponse(); + public Writeable.Reader getResponseReader() { + return AcknowledgedResponse::new; } } diff --git a/server/src/main/java/org/elasticsearch/action/ingest/PutPipelineTransportAction.java b/server/src/main/java/org/elasticsearch/action/ingest/PutPipelineTransportAction.java index be1528a354bc3..95a9dbc7254b3 100644 --- a/server/src/main/java/org/elasticsearch/action/ingest/PutPipelineTransportAction.java +++ b/server/src/main/java/org/elasticsearch/action/ingest/PutPipelineTransportAction.java @@ -32,11 +32,14 @@ import org.elasticsearch.cluster.metadata.IndexNameExpressionResolver; import org.elasticsearch.cluster.node.DiscoveryNode; import org.elasticsearch.common.inject.Inject; +import org.elasticsearch.common.io.stream.StreamInput; import org.elasticsearch.ingest.IngestInfo; import org.elasticsearch.ingest.IngestService; +import org.elasticsearch.tasks.Task; import org.elasticsearch.threadpool.ThreadPool; import org.elasticsearch.transport.TransportService; +import java.io.IOException; import java.util.HashMap; import java.util.Map; @@ -62,13 +65,18 @@ protected String executor() { return ThreadPool.Names.SAME; } + @Override + protected AcknowledgedResponse read(StreamInput in) throws IOException { + return new AcknowledgedResponse(in); + } + @Override protected AcknowledgedResponse newResponse() { - return new AcknowledgedResponse(); + throw new UnsupportedOperationException("usage of Streamable is to be replaced by Writeable"); } @Override - protected void masterOperation(PutPipelineRequest request, ClusterState state, ActionListener listener) + protected void masterOperation(Task task, PutPipelineRequest request, ClusterState state, ActionListener listener) throws Exception { NodesInfoRequest nodesInfoRequest = new NodesInfoRequest(); nodesInfoRequest.clear(); diff --git a/server/src/main/java/org/elasticsearch/action/ingest/SimulatePipelineAction.java b/server/src/main/java/org/elasticsearch/action/ingest/SimulatePipelineAction.java index afeb4e01fb0ff..befa729282e7e 100644 --- a/server/src/main/java/org/elasticsearch/action/ingest/SimulatePipelineAction.java +++ b/server/src/main/java/org/elasticsearch/action/ingest/SimulatePipelineAction.java @@ -19,9 +19,9 @@ package org.elasticsearch.action.ingest; -import org.elasticsearch.action.Action; +import org.elasticsearch.action.StreamableResponseActionType; -public class SimulatePipelineAction extends Action { +public class SimulatePipelineAction extends StreamableResponseActionType { public static final SimulatePipelineAction INSTANCE = new SimulatePipelineAction(); public static final String NAME = "cluster:admin/ingest/pipeline/simulate"; diff --git a/server/src/main/java/org/elasticsearch/action/main/MainAction.java b/server/src/main/java/org/elasticsearch/action/main/MainAction.java index 831ddd0983fad..535c63d2a8800 100644 --- a/server/src/main/java/org/elasticsearch/action/main/MainAction.java +++ b/server/src/main/java/org/elasticsearch/action/main/MainAction.java @@ -19,9 +19,9 @@ package org.elasticsearch.action.main; -import org.elasticsearch.action.Action; +import org.elasticsearch.action.StreamableResponseActionType; -public class MainAction extends Action { +public class MainAction extends StreamableResponseActionType { public static final String NAME = "cluster:monitor/main"; public static final MainAction INSTANCE = new MainAction(); diff --git a/server/src/main/java/org/elasticsearch/action/search/AbstractSearchAsyncAction.java b/server/src/main/java/org/elasticsearch/action/search/AbstractSearchAsyncAction.java index fdccfad7b47f5..8475272a5e2cd 100644 --- a/server/src/main/java/org/elasticsearch/action/search/AbstractSearchAsyncAction.java +++ b/server/src/main/java/org/elasticsearch/action/search/AbstractSearchAsyncAction.java @@ -140,24 +140,29 @@ public final void executeNextPhase(SearchPhase currentPhase, SearchPhase nextPha } else { Boolean allowPartialResults = request.allowPartialSearchResults(); assert allowPartialResults != null : "SearchRequest missing setting for allowPartialSearchResults"; - if (allowPartialResults == false && shardFailures.get() != null ){ - if (logger.isDebugEnabled()) { - final ShardOperationFailedException[] shardSearchFailures = ExceptionsHelper.groupBy(buildShardFailures()); - Throwable cause = shardSearchFailures.length == 0 ? null : - ElasticsearchException.guessRootCauses(shardSearchFailures[0].getCause())[0]; - logger.debug(() -> new ParameterizedMessage("{} shards failed for phase: [{}]", - shardSearchFailures.length, getName()), cause); - } - onPhaseFailure(currentPhase, "Partial shards failure", null); - } else { - if (logger.isTraceEnabled()) { - final String resultsFrom = results.getSuccessfulResults() - .map(r -> r.getSearchShardTarget().toString()).collect(Collectors.joining(",")); - logger.trace("[{}] Moving to next phase: [{}], based on results from: {} (cluster state version: {})", - currentPhase.getName(), nextPhase.getName(), resultsFrom, clusterStateVersion); + if (allowPartialResults == false && shardFailures.get() != null) { + // check if there are actual failures in the atomic array since + // successful retries can reset the failures to null + ShardOperationFailedException[] shardSearchFailures = buildShardFailures(); + if (shardSearchFailures.length > 0) { + if (logger.isDebugEnabled()) { + int numShardFailures = shardSearchFailures.length; + shardSearchFailures = ExceptionsHelper.groupBy(shardSearchFailures); + Throwable cause = ElasticsearchException.guessRootCauses(shardSearchFailures[0].getCause())[0]; + logger.debug(() -> new ParameterizedMessage("{} shards failed for phase: [{}]", + numShardFailures, getName()), cause); + } + onPhaseFailure(currentPhase, "Partial shards failure", null); + return; } - executePhase(nextPhase); } + if (logger.isTraceEnabled()) { + final String resultsFrom = results.getSuccessfulResults() + .map(r -> r.getSearchShardTarget().toString()).collect(Collectors.joining(",")); + logger.trace("[{}] Moving to next phase: [{}], based on results from: {} (cluster state version: {})", + currentPhase.getName(), nextPhase.getName(), resultsFrom, clusterStateVersion); + } + executePhase(nextPhase); } } @@ -279,8 +284,7 @@ public final SearchRequest getRequest() { return request; } - @Override - public final SearchResponse buildSearchResponse(InternalSearchResponse internalSearchResponse, String scrollId) { + protected final SearchResponse buildSearchResponse(InternalSearchResponse internalSearchResponse, String scrollId) { ShardSearchFailure[] failures = buildShardFailures(); Boolean allowPartialResults = request.allowPartialSearchResults(); assert allowPartialResults != null : "SearchRequest missing setting for allowPartialSearchResults"; @@ -291,6 +295,11 @@ public final SearchResponse buildSearchResponse(InternalSearchResponse internalS skippedOps.get(), buildTookInMillis(), failures, clusters); } + @Override + public void sendSearchResponse(InternalSearchResponse internalSearchResponse, String scrollId) { + listener.onResponse(buildSearchResponse(internalSearchResponse, scrollId)); + } + @Override public final void onPhaseFailure(SearchPhase phase, String msg, Throwable cause) { raisePhaseFailure(new SearchPhaseExecutionException(phase.getName(), msg, cause, buildShardFailures())); @@ -311,11 +320,6 @@ public final void execute(Runnable command) { executor.execute(command); } - @Override - public final void onResponse(SearchResponse response) { - listener.onResponse(response); - } - @Override public final void onFailure(Exception e) { listener.onFailure(e); diff --git a/server/src/main/java/org/elasticsearch/action/search/ClearScrollAction.java b/server/src/main/java/org/elasticsearch/action/search/ClearScrollAction.java index 660ed1ee17860..c7b959ed0bd44 100644 --- a/server/src/main/java/org/elasticsearch/action/search/ClearScrollAction.java +++ b/server/src/main/java/org/elasticsearch/action/search/ClearScrollAction.java @@ -19,9 +19,9 @@ package org.elasticsearch.action.search; -import org.elasticsearch.action.Action; +import org.elasticsearch.action.StreamableResponseActionType; -public class ClearScrollAction extends Action { +public class ClearScrollAction extends StreamableResponseActionType { public static final ClearScrollAction INSTANCE = new ClearScrollAction(); public static final String NAME = "indices:data/read/scroll/clear"; diff --git a/server/src/main/java/org/elasticsearch/action/search/ExpandSearchPhase.java b/server/src/main/java/org/elasticsearch/action/search/ExpandSearchPhase.java index afc81b21da4d5..301cb600452b2 100644 --- a/server/src/main/java/org/elasticsearch/action/search/ExpandSearchPhase.java +++ b/server/src/main/java/org/elasticsearch/action/search/ExpandSearchPhase.java @@ -30,7 +30,6 @@ import org.elasticsearch.search.collapse.CollapseBuilder; import org.elasticsearch.search.internal.InternalSearchResponse; -import java.io.IOException; import java.util.HashMap; import java.util.Iterator; import java.util.List; @@ -65,7 +64,7 @@ private boolean isCollapseRequest() { } @Override - public void run() throws IOException { + public void run() { if (isCollapseRequest() && searchResponse.hits().getHits().length > 0) { SearchRequest searchRequest = context.getRequest(); CollapseBuilder collapseBuilder = searchRequest.source().collapse(); diff --git a/server/src/main/java/org/elasticsearch/action/search/FetchSearchPhase.java b/server/src/main/java/org/elasticsearch/action/search/FetchSearchPhase.java index 1bbca35cb9a54..2115b4fa99849 100644 --- a/server/src/main/java/org/elasticsearch/action/search/FetchSearchPhase.java +++ b/server/src/main/java/org/elasticsearch/action/search/FetchSearchPhase.java @@ -22,8 +22,8 @@ import org.apache.logging.log4j.Logger; import org.apache.logging.log4j.message.ParameterizedMessage; import org.apache.lucene.search.ScoreDoc; -import org.elasticsearch.action.ActionRunnable; import org.elasticsearch.action.OriginalIndices; +import org.elasticsearch.common.util.concurrent.AbstractRunnable; import org.elasticsearch.common.util.concurrent.AtomicArray; import org.elasticsearch.search.SearchPhaseResult; import org.elasticsearch.search.SearchShardTarget; @@ -76,10 +76,10 @@ final class FetchSearchPhase extends SearchPhase { } @Override - public void run() throws IOException { - context.execute(new ActionRunnable(context) { + public void run() { + context.execute(new AbstractRunnable() { @Override - public void doRun() throws IOException { + protected void doRun() throws Exception { // we do the heavy lifting in this inner run method where we reduce aggs etc. that's why we fork this phase // off immediately instead of forking when we send back the response to the user since there we only need // to merge together the fetched results which is a linear operation. @@ -209,8 +209,8 @@ private void moveToNextPhase(SearchPhaseController searchPhaseController, private static SearchPhase sendResponsePhase(InternalSearchResponse response, String scrollId, SearchPhaseContext context) { return new SearchPhase("response") { @Override - public void run() throws IOException { - context.onResponse(context.buildSearchResponse(response, scrollId)); + public void run() { + context.sendSearchResponse(response, scrollId); } }; } diff --git a/server/src/main/java/org/elasticsearch/action/search/InitialSearchPhase.java b/server/src/main/java/org/elasticsearch/action/search/InitialSearchPhase.java index 65f613521c0a8..6b9efe30fb791 100644 --- a/server/src/main/java/org/elasticsearch/action/search/InitialSearchPhase.java +++ b/server/src/main/java/org/elasticsearch/action/search/InitialSearchPhase.java @@ -139,13 +139,13 @@ public final void run() { for (int index = 0; index < shardsIts.size(); index++) { final SearchShardIterator shardRoutings = shardsIts.get(index); if (shardRoutings.size() == 0) { - if(missingShards.length() >0 ){ + if(missingShards.length() > 0){ missingShards.append(", "); } missingShards.append(shardRoutings.shardId()); } } - if (missingShards.length() >0) { + if (missingShards.length() > 0) { //Status red - shard is missing all copies and would produce partial results for an index search final String msg = "Search rejected due to missing shards ["+ missingShards + "]. Consider using `allow_partial_search_results` setting to bypass this error."; diff --git a/server/src/main/java/org/elasticsearch/action/search/MultiSearchAction.java b/server/src/main/java/org/elasticsearch/action/search/MultiSearchAction.java index 9017a7b94ecb4..dd34dd94e69f7 100644 --- a/server/src/main/java/org/elasticsearch/action/search/MultiSearchAction.java +++ b/server/src/main/java/org/elasticsearch/action/search/MultiSearchAction.java @@ -19,10 +19,10 @@ package org.elasticsearch.action.search; -import org.elasticsearch.action.Action; +import org.elasticsearch.action.ActionType; import org.elasticsearch.common.io.stream.Writeable; -public class MultiSearchAction extends Action { +public class MultiSearchAction extends ActionType { public static final MultiSearchAction INSTANCE = new MultiSearchAction(); public static final String NAME = "indices:data/read/msearch"; @@ -31,11 +31,6 @@ private MultiSearchAction() { super(NAME); } - @Override - public MultiSearchResponse newResponse() { - throw new UnsupportedOperationException("usage of Streamable is to be replaced by Writeable"); - } - @Override public Writeable.Reader getResponseReader() { return MultiSearchResponse::new; diff --git a/server/src/main/java/org/elasticsearch/action/search/MultiSearchRequest.java b/server/src/main/java/org/elasticsearch/action/search/MultiSearchRequest.java index 528be0369166e..25500efd5ed5b 100644 --- a/server/src/main/java/org/elasticsearch/action/search/MultiSearchRequest.java +++ b/server/src/main/java/org/elasticsearch/action/search/MultiSearchRequest.java @@ -170,7 +170,6 @@ public static void readMultiLineFormat(BytesReference data, CheckedBiConsumer consumer, String[] indices, IndicesOptions indicesOptions, - String[] types, String routing, String searchType, Boolean ccsMinimizeRoundtrips, @@ -190,9 +189,6 @@ public static void readMultiLineFormat(BytesReference data, if (indicesOptions != null) { searchRequest.indicesOptions(indicesOptions); } - if (types != null && types.length > 0) { - searchRequest.types(types); - } if (routing != null) { searchRequest.routing(routing); } @@ -219,8 +215,6 @@ public static void readMultiLineFormat(BytesReference data, throw new IllegalArgumentException("explicit index in multi search is not allowed"); } searchRequest.indices(nodeStringArrayValue(value)); - } else if ("type".equals(entry.getKey()) || "types".equals(entry.getKey())) { - searchRequest.types(nodeStringArrayValue(value)); } else if ("search_type".equals(entry.getKey()) || "searchType".equals(entry.getKey())) { searchRequest.searchType(nodeStringValue(value, null)); } else if ("ccs_minimize_roundtrips".equals(entry.getKey()) || "ccsMinimizeRoundtrips".equals(entry.getKey())) { @@ -320,9 +314,6 @@ public static void writeSearchRequestParams(SearchRequest request, XContentBuild xContentBuilder.field("ignore_unavailable", request.indicesOptions().ignoreUnavailable()); xContentBuilder.field("allow_no_indices", request.indicesOptions().allowNoIndices()); } - if (request.types() != null) { - xContentBuilder.field("types", request.types()); - } if (request.searchType() != null) { xContentBuilder.field("search_type", request.searchType().name().toLowerCase(Locale.ROOT)); } diff --git a/server/src/main/java/org/elasticsearch/action/search/SearchAction.java b/server/src/main/java/org/elasticsearch/action/search/SearchAction.java index d665595e8d34d..402219001a28d 100644 --- a/server/src/main/java/org/elasticsearch/action/search/SearchAction.java +++ b/server/src/main/java/org/elasticsearch/action/search/SearchAction.java @@ -19,10 +19,10 @@ package org.elasticsearch.action.search; -import org.elasticsearch.action.Action; +import org.elasticsearch.action.ActionType; import org.elasticsearch.common.io.stream.Writeable; -public class SearchAction extends Action { +public class SearchAction extends ActionType { public static final SearchAction INSTANCE = new SearchAction(); public static final String NAME = "indices:data/read/search"; @@ -31,11 +31,6 @@ private SearchAction() { super(NAME); } - @Override - public SearchResponse newResponse() { - throw new UnsupportedOperationException("usage of Streamable is to be replaced by Writeable"); - } - @Override public Writeable.Reader getResponseReader() { return SearchResponse::new; diff --git a/server/src/main/java/org/elasticsearch/action/search/SearchPhaseContext.java b/server/src/main/java/org/elasticsearch/action/search/SearchPhaseContext.java index 9829ff6a98337..28838defa3ec4 100644 --- a/server/src/main/java/org/elasticsearch/action/search/SearchPhaseContext.java +++ b/server/src/main/java/org/elasticsearch/action/search/SearchPhaseContext.java @@ -19,7 +19,6 @@ package org.elasticsearch.action.search; import org.apache.logging.log4j.Logger; -import org.elasticsearch.action.ActionListener; import org.elasticsearch.action.OriginalIndices; import org.elasticsearch.common.Nullable; import org.elasticsearch.search.SearchShardTarget; @@ -32,7 +31,7 @@ /** * This class provide contextual state and access to resources across multiple search phases. */ -interface SearchPhaseContext extends ActionListener, Executor { +interface SearchPhaseContext extends Executor { // TODO maybe we can make this concrete later - for now we just implement this in the base class for all initial phases /** @@ -56,11 +55,16 @@ interface SearchPhaseContext extends ActionListener, Executor { SearchRequest getRequest(); /** - * Builds the final search response that should be send back to the user. + * Builds and sends the final search response back to the user. * @param internalSearchResponse the internal search response * @param scrollId an optional scroll ID if this search is a scroll search */ - SearchResponse buildSearchResponse(InternalSearchResponse internalSearchResponse, String scrollId); + void sendSearchResponse(InternalSearchResponse internalSearchResponse, String scrollId); + + /** + * Notifies the top-level listener of the provided exception + */ + void onFailure(Exception e); /** * This method will communicate a fatal phase failure back to the user. In contrast to a shard failure @@ -113,5 +117,4 @@ default void sendReleaseSearchContext(long contextId, Transport.Connection conne * a response is returned to the user indicating that all shards have failed. */ void executeNextPhase(SearchPhase currentPhase, SearchPhase nextPhase); - } diff --git a/server/src/main/java/org/elasticsearch/action/search/SearchPhaseController.java b/server/src/main/java/org/elasticsearch/action/search/SearchPhaseController.java index 0125084c37099..ca6cba2f43f91 100644 --- a/server/src/main/java/org/elasticsearch/action/search/SearchPhaseController.java +++ b/server/src/main/java/org/elasticsearch/action/search/SearchPhaseController.java @@ -21,6 +21,7 @@ import com.carrotsearch.hppc.IntArrayList; import com.carrotsearch.hppc.ObjectObjectHashMap; + import org.apache.lucene.index.Term; import org.apache.lucene.search.CollectionStatistics; import org.apache.lucene.search.FieldDoc; diff --git a/server/src/main/java/org/elasticsearch/action/search/SearchRequest.java b/server/src/main/java/org/elasticsearch/action/search/SearchRequest.java index 53dafc153fc4b..602853e10b292 100644 --- a/server/src/main/java/org/elasticsearch/action/search/SearchRequest.java +++ b/server/src/main/java/org/elasticsearch/action/search/SearchRequest.java @@ -19,6 +19,7 @@ package org.elasticsearch.action.search; +import org.elasticsearch.Version; import org.elasticsearch.action.ActionRequest; import org.elasticsearch.action.ActionRequestValidationException; import org.elasticsearch.action.IndicesRequest; @@ -91,8 +92,6 @@ public final class SearchRequest extends ActionRequest implements IndicesRequest private int preFilterShardSize = DEFAULT_PRE_FILTER_SHARD_SIZE; - private String[] types = Strings.EMPTY_ARRAY; - private boolean ccsMinimizeRoundtrips = true; public static final IndicesOptions DEFAULT_INDICES_OPTIONS = IndicesOptions.strictExpandOpenAndForbidClosedIgnoreThrottled(); @@ -172,7 +171,6 @@ private SearchRequest(SearchRequest searchRequest, String[] indices, String loca this.scroll = searchRequest.scroll; this.searchType = searchRequest.searchType; this.source = searchRequest.source; - this.types = searchRequest.types; this.localClusterAlias = localClusterAlias; this.absoluteStartMillis = absoluteStartMillis; this.finalReduce = finalReduce; @@ -192,7 +190,14 @@ public SearchRequest(StreamInput in) throws IOException { preference = in.readOptionalString(); scroll = in.readOptionalWriteable(Scroll::new); source = in.readOptionalWriteable(SearchSourceBuilder::new); - types = in.readStringArray(); + if (in.getVersion().before(Version.V_8_0_0)) { + // types no longer relevant so ignore + String[] types = in.readStringArray(); + if (types.length > 0) { + throw new IllegalStateException( + "types are no longer supported in search requests but found [" + Arrays.toString(types) + "]"); + } + } indicesOptions = IndicesOptions.readIndicesOptions(in); requestCache = in.readOptionalBoolean(); batchedReduceSize = in.readVInt(); @@ -219,7 +224,10 @@ public void writeTo(StreamOutput out) throws IOException { out.writeOptionalString(preference); out.writeOptionalWriteable(scroll); out.writeOptionalWriteable(source); - out.writeStringArray(types); + if (out.getVersion().before(Version.V_8_0_0)) { + // types not supported so send an empty array to previous versions + out.writeStringArray(Strings.EMPTY_ARRAY); + } indicesOptions.writeIndicesOptions(out); out.writeOptionalBoolean(requestCache); out.writeVInt(batchedReduceSize); @@ -342,35 +350,6 @@ public void setCcsMinimizeRoundtrips(boolean ccsMinimizeRoundtrips) { this.ccsMinimizeRoundtrips = ccsMinimizeRoundtrips; } - /** - * The document types to execute the search against. Defaults to be executed against - * all types. - * - * @deprecated Types are in the process of being removed. Instead of using a type, prefer to - * filter on a field on the document. - */ - @Deprecated - public String[] types() { - return types; - } - - /** - * The document types to execute the search against. Defaults to be executed against - * all types. - * - * @deprecated Types are in the process of being removed. Instead of using a type, prefer to - * filter on a field on the document. - */ - @Deprecated - public SearchRequest types(String... types) { - Objects.requireNonNull(types, "types must not be null"); - for (String type : types) { - Objects.requireNonNull(type, "type must not be null"); - } - this.types = types; - return this; - } - /** * A comma separated list of routing values to control the shards the search will be executed on. */ @@ -590,9 +569,6 @@ public String getDescription() { sb.append("indices["); Strings.arrayToDelimitedString(indices, ",", sb); sb.append("], "); - sb.append("types["); - Strings.arrayToDelimitedString(types, ",", sb); - sb.append("], "); sb.append("search_type[").append(searchType).append("], "); if (source != null) { @@ -626,7 +602,6 @@ public boolean equals(Object o) { Objects.equals(source, that.source) && Objects.equals(requestCache, that.requestCache) && Objects.equals(scroll, that.scroll) && - Arrays.equals(types, that.types) && Objects.equals(batchedReduceSize, that.batchedReduceSize) && Objects.equals(maxConcurrentShardRequests, that.maxConcurrentShardRequests) && Objects.equals(preFilterShardSize, that.preFilterShardSize) && @@ -640,7 +615,7 @@ public boolean equals(Object o) { @Override public int hashCode() { return Objects.hash(searchType, Arrays.hashCode(indices), routing, preference, source, requestCache, - scroll, Arrays.hashCode(types), indicesOptions, batchedReduceSize, maxConcurrentShardRequests, preFilterShardSize, + scroll, indicesOptions, batchedReduceSize, maxConcurrentShardRequests, preFilterShardSize, allowPartialSearchResults, localClusterAlias, absoluteStartMillis, ccsMinimizeRoundtrips); } @@ -650,7 +625,6 @@ public String toString() { "searchType=" + searchType + ", indices=" + Arrays.toString(indices) + ", indicesOptions=" + indicesOptions + - ", types=" + Arrays.toString(types) + ", routing='" + routing + '\'' + ", preference='" + preference + '\'' + ", requestCache=" + requestCache + diff --git a/server/src/main/java/org/elasticsearch/action/search/SearchRequestBuilder.java b/server/src/main/java/org/elasticsearch/action/search/SearchRequestBuilder.java index 96c93c974cabb..ceaee96f5c131 100644 --- a/server/src/main/java/org/elasticsearch/action/search/SearchRequestBuilder.java +++ b/server/src/main/java/org/elasticsearch/action/search/SearchRequestBuilder.java @@ -58,17 +58,6 @@ public SearchRequestBuilder setIndices(String... indices) { return this; } - /** - * The document types to execute the search against. Defaults to be executed against - * all types. - * @deprecated Types are going away, prefer filtering on a field. - */ - @Deprecated - public SearchRequestBuilder setTypes(String... types) { - request.types(types); - return this; - } - /** * The search type to execute, defaults to {@link org.elasticsearch.action.search.SearchType#DEFAULT}. */ @@ -378,7 +367,9 @@ public SearchRequestBuilder setTrackScores(boolean trackScores) { } /** - * Indicates if the total hit count for the query should be tracked. Defaults to {@code true} + * Indicates if the total hit count for the query should be tracked. Requests will count total hit count accurately + * up to 10,000 by default, see {@link #setTrackTotalHitsUpTo(int)} to change this value or set to true/false to always/never + * count accurately. */ public SearchRequestBuilder setTrackTotalHits(boolean trackTotalHits) { sourceBuilder().trackTotalHits(trackTotalHits); @@ -386,7 +377,7 @@ public SearchRequestBuilder setTrackTotalHits(boolean trackTotalHits) { } /** - * Indicates if the total hit count for the query should be tracked. Defaults to {@code true} + * Indicates the total hit count that should be tracked accurately or null if the value is unset. Defaults to 10,000. */ public SearchRequestBuilder setTrackTotalHitsUpTo(int trackTotalHitsUpTo) { sourceBuilder().trackTotalHitsUpTo(trackTotalHitsUpTo); diff --git a/server/src/main/java/org/elasticsearch/action/search/SearchScrollAction.java b/server/src/main/java/org/elasticsearch/action/search/SearchScrollAction.java index 0b4adfc1ba55c..32c84f86ea038 100644 --- a/server/src/main/java/org/elasticsearch/action/search/SearchScrollAction.java +++ b/server/src/main/java/org/elasticsearch/action/search/SearchScrollAction.java @@ -19,10 +19,10 @@ package org.elasticsearch.action.search; -import org.elasticsearch.action.Action; +import org.elasticsearch.action.ActionType; import org.elasticsearch.common.io.stream.Writeable; -public class SearchScrollAction extends Action { +public class SearchScrollAction extends ActionType { public static final SearchScrollAction INSTANCE = new SearchScrollAction(); public static final String NAME = "indices:data/read/scroll"; @@ -31,11 +31,6 @@ private SearchScrollAction() { super(NAME); } - @Override - public SearchResponse newResponse() { - throw new UnsupportedOperationException("usage of Streamable is to be replaced by Writeable"); - } - @Override public Writeable.Reader getResponseReader() { return SearchResponse::new; diff --git a/server/src/main/java/org/elasticsearch/action/search/TransportSearchAction.java b/server/src/main/java/org/elasticsearch/action/search/TransportSearchAction.java index a7c0a785c7fce..d4832fb0d7a10 100644 --- a/server/src/main/java/org/elasticsearch/action/search/TransportSearchAction.java +++ b/server/src/main/java/org/elasticsearch/action/search/TransportSearchAction.java @@ -29,6 +29,7 @@ import org.elasticsearch.action.support.IndicesOptions; import org.elasticsearch.client.Client; import org.elasticsearch.cluster.ClusterState; +import org.elasticsearch.cluster.block.ClusterBlockException; import org.elasticsearch.cluster.block.ClusterBlockLevel; import org.elasticsearch.cluster.metadata.IndexNameExpressionResolver; import org.elasticsearch.cluster.node.DiscoveryNode; @@ -472,10 +473,89 @@ private void executeSearch(SearchTask task, SearchTimeProvider timeProvider, Sea Map> routingMap = indexNameExpressionResolver.resolveSearchRouting(clusterState, searchRequest.routing(), searchRequest.indices()); routingMap = routingMap == null ? Collections.emptyMap() : Collections.unmodifiableMap(routingMap); - String[] concreteIndices = new String[indices.length]; - for (int i = 0; i < indices.length; i++) { - concreteIndices[i] = indices[i].getName(); + Map concreteIndexBoosts = resolveIndexBoosts(searchRequest, clusterState); + + if (shouldSplitIndices(searchRequest)) { + //Execute two separate searches when we can, so that indices that are being written to are searched as quickly as possible. + //Otherwise their search context would need to stay open for too long between the query and the fetch phase, due to other + //indices (possibly slower) being searched at the same time. + List writeIndicesList = new ArrayList<>(); + List readOnlyIndicesList = new ArrayList<>(); + splitIndices(indices, clusterState, writeIndicesList, readOnlyIndicesList); + String[] writeIndices = writeIndicesList.toArray(new String[0]); + String[] readOnlyIndices = readOnlyIndicesList.toArray(new String[0]); + + if (readOnlyIndices.length == 0) { + executeSearch(task, timeProvider, searchRequest, localIndices, writeIndices, routingMap, + aliasFilter, concreteIndexBoosts, remoteShardIterators, remoteConnections, clusterState, listener, clusters); + } else if (writeIndices.length == 0 && remoteShardIterators.isEmpty()) { + executeSearch(task, timeProvider, searchRequest, localIndices, readOnlyIndices, routingMap, + aliasFilter, concreteIndexBoosts, remoteShardIterators, remoteConnections, clusterState, listener, clusters); + } else { + //Split the search in two whenever throttled indices are searched together with ordinary indices (local or remote), so + //that we don't keep the search context open for too long between query and fetch for ordinary indices due to slow indices. + CountDown countDown = new CountDown(2); + AtomicReference exceptions = new AtomicReference<>(); + SearchResponseMerger searchResponseMerger = createSearchResponseMerger(searchRequest.source(), timeProvider, + searchService::createReduceContext); + CountDownActionListener countDownActionListener = + new CountDownActionListener<>(countDown, exceptions, listener) { + @Override + void innerOnResponse(SearchResponse searchResponse) { + searchResponseMerger.add(searchResponse); + } + + @Override + SearchResponse createFinalResponse() { + return searchResponseMerger.getMergedResponse(clusters); + } + }; + + //Note that the indices set to the new SearchRequest won't be retrieved from it, as they have been already resolved and + //will be provided separately to executeSearch. + SearchRequest writeIndicesRequest = SearchRequest.subSearchRequest(searchRequest, writeIndices, + RemoteClusterService.LOCAL_CLUSTER_GROUP_KEY, timeProvider.getAbsoluteStartMillis(), false); + executeSearch(task, timeProvider, writeIndicesRequest, localIndices, writeIndices, routingMap, + aliasFilter, concreteIndexBoosts, remoteShardIterators, remoteConnections, clusterState, countDownActionListener, + SearchResponse.Clusters.EMPTY); + + //Note that the indices set to the new SearchRequest won't be retrieved from it, as they have been already resolved and + //will be provided separately to executeSearch. + SearchRequest readOnlyIndicesRequest = SearchRequest.subSearchRequest(searchRequest, readOnlyIndices, + RemoteClusterService.LOCAL_CLUSTER_GROUP_KEY, timeProvider.getAbsoluteStartMillis(), false); + executeSearch(task, timeProvider, readOnlyIndicesRequest, localIndices, readOnlyIndices, routingMap, + aliasFilter, concreteIndexBoosts, Collections.emptyList(), (alias, id) -> null, clusterState, countDownActionListener, + SearchResponse.Clusters.EMPTY); + } + } else { + String[] concreteIndices = Arrays.stream(indices).map(Index::getName).toArray(String[]::new); + executeSearch(task, timeProvider, searchRequest, localIndices, concreteIndices, routingMap, + aliasFilter, concreteIndexBoosts, remoteShardIterators, remoteConnections, clusterState, listener, clusters); + } + } + + static boolean shouldSplitIndices(SearchRequest searchRequest) { + return searchRequest.scroll() == null && searchRequest.searchType() != DFS_QUERY_THEN_FETCH + && (searchRequest.source() == null || searchRequest.source().size() != 0); + } + + static void splitIndices(Index[] indices, ClusterState clusterState, List writeIndices, List readOnlyIndices) { + for (Index index : indices) { + ClusterBlockException writeBlock = clusterState.blocks().indexBlockedException(ClusterBlockLevel.WRITE, index.getName()); + if (writeBlock == null) { + writeIndices.add(index.getName()); + } else { + readOnlyIndices.add(index.getName()); + } } + } + + private void executeSearch(SearchTask task, SearchTimeProvider timeProvider, SearchRequest searchRequest, + OriginalIndices localIndices, String[] concreteIndices, Map> routingMap, + Map aliasFilter, Map concreteIndexBoosts, + List remoteShardIterators, BiFunction remoteConnections, + ClusterState clusterState, ActionListener listener, SearchResponse.Clusters clusters) { + Map nodeSearchCounts = searchTransportService.getPendingSearchRequests(); GroupShardsIterator localShardsIterator = clusterService.operationRouting().searchShards(clusterState, concreteIndices, routingMap, searchRequest.preference(), searchService.getResponseCollectorService(), nodeSearchCounts); @@ -484,8 +564,6 @@ private void executeSearch(SearchTask task, SearchTimeProvider timeProvider, Sea failIfOverShardCountLimit(clusterService, shardIterators.size()); - Map concreteIndexBoosts = resolveIndexBoosts(searchRequest, clusterState); - // optimize search type for cases where there is only one shard group to search on if (shardIterators.size() == 1) { // if we only have one group, then we always want Q_T_F, no need for DFS, and no need to do THEN since we hit one shard @@ -498,11 +576,9 @@ private void executeSearch(SearchTask task, SearchTimeProvider timeProvider, Sea if (searchRequest.isSuggestOnly()) { // disable request cache if we have only suggest searchRequest.requestCache(false); - switch (searchRequest.searchType()) { - case DFS_QUERY_THEN_FETCH: - // convert to Q_T_F if we have only suggest - searchRequest.searchType(QUERY_THEN_FETCH); - break; + if (searchRequest.searchType() == DFS_QUERY_THEN_FETCH) { + // convert to Q_T_F if we have only suggest + searchRequest.searchType(QUERY_THEN_FETCH); } } @@ -611,22 +687,16 @@ private static void failIfOverShardCountLimit(ClusterService clusterService, int } } - abstract static class CCSActionListener implements ActionListener { - private final String clusterAlias; - private final boolean skipUnavailable; + abstract static class CountDownActionListener implements ActionListener { private final CountDown countDown; - private final AtomicInteger skippedClusters; private final AtomicReference exceptions; - private final ActionListener originalListener; + private final ActionListener delegateListener; - CCSActionListener(String clusterAlias, boolean skipUnavailable, CountDown countDown, AtomicInteger skippedClusters, - AtomicReference exceptions, ActionListener originalListener) { - this.clusterAlias = clusterAlias; - this.skipUnavailable = skipUnavailable; + CountDownActionListener(CountDown countDown, AtomicReference exceptions, + ActionListener delegateListener) { this.countDown = countDown; - this.skippedClusters = skippedClusters; this.exceptions = exceptions; - this.originalListener = originalListener; + this.delegateListener = delegateListener; } @Override @@ -637,26 +707,7 @@ public final void onResponse(Response response) { abstract void innerOnResponse(Response response); - @Override - public final void onFailure(Exception e) { - if (skipUnavailable) { - skippedClusters.incrementAndGet(); - } else { - Exception exception = e; - if (RemoteClusterAware.LOCAL_CLUSTER_GROUP_KEY.equals(clusterAlias) == false) { - exception = wrapRemoteClusterFailure(clusterAlias, e); - } - if (exceptions.compareAndSet(null, exception) == false) { - exceptions.accumulateAndGet(exception, (previous, current) -> { - current.addSuppressed(previous); - return current; - }); - } - } - maybeFinish(); - } - - private void maybeFinish() { + final void maybeFinish() { if (countDown.countDown()) { Exception exception = exceptions.get(); if (exception == null) { @@ -664,17 +715,56 @@ private void maybeFinish() { try { response = createFinalResponse(); } catch(Exception e) { - originalListener.onFailure(e); + delegateListener.onFailure(e); return; } - originalListener.onResponse(response); + delegateListener.onResponse(response); } else { - originalListener.onFailure(exceptions.get()); + delegateListener.onFailure(exceptions.get()); } } } abstract FinalResponse createFinalResponse(); + + @Override + public void onFailure(Exception e) { + if (exceptions.compareAndSet(null, e) == false) { + exceptions.accumulateAndGet(e, (previous, current) -> { + current.addSuppressed(previous); + return current; + }); + } + maybeFinish(); + } + } + + abstract static class CCSActionListener extends CountDownActionListener { + private final String clusterAlias; + private final boolean skipUnavailable; + private final AtomicInteger skippedClusters; + + CCSActionListener(String clusterAlias, boolean skipUnavailable, CountDown countDown, AtomicInteger skippedClusters, + AtomicReference exceptions, ActionListener originalListener) { + super(countDown, exceptions, originalListener); + this.clusterAlias = clusterAlias; + this.skipUnavailable = skipUnavailable; + this.skippedClusters = skippedClusters; + } + + @Override + public final void onFailure(Exception e) { + if (skipUnavailable) { + skippedClusters.incrementAndGet(); + maybeFinish(); + } else { + Exception exception = e; + if (RemoteClusterAware.LOCAL_CLUSTER_GROUP_KEY.equals(clusterAlias) == false) { + exception = wrapRemoteClusterFailure(clusterAlias, e); + } + super.onFailure(exception); + } + } } private static RemoteTransportException wrapRemoteClusterFailure(String clusterAlias, Exception e) { diff --git a/server/src/main/java/org/elasticsearch/action/support/DefaultShardOperationFailedException.java b/server/src/main/java/org/elasticsearch/action/support/DefaultShardOperationFailedException.java index 7aa7dfb62a63c..aa3e91c634a80 100644 --- a/server/src/main/java/org/elasticsearch/action/support/DefaultShardOperationFailedException.java +++ b/server/src/main/java/org/elasticsearch/action/support/DefaultShardOperationFailedException.java @@ -25,6 +25,7 @@ import org.elasticsearch.common.ParseField; import org.elasticsearch.common.io.stream.StreamInput; import org.elasticsearch.common.io.stream.StreamOutput; +import org.elasticsearch.common.io.stream.Writeable; import org.elasticsearch.common.xcontent.ConstructingObjectParser; import org.elasticsearch.common.xcontent.XContentBuilder; import org.elasticsearch.common.xcontent.XContentParser; @@ -35,7 +36,7 @@ import static org.elasticsearch.ExceptionsHelper.detailedMessage; import static org.elasticsearch.common.xcontent.ConstructingObjectParser.constructorArg; -public class DefaultShardOperationFailedException extends ShardOperationFailedException { +public class DefaultShardOperationFailedException extends ShardOperationFailedException implements Writeable { private static final String INDEX = "index"; private static final String SHARD_ID = "shard"; @@ -90,13 +91,13 @@ public String toString() { } @Override - public XContentBuilder toXContent(XContentBuilder builder, Params params) throws IOException { + public final XContentBuilder toXContent(XContentBuilder builder, Params params) throws IOException { builder.startObject(); innerToXContent(builder, params); builder.endObject(); return builder; } - + protected XContentBuilder innerToXContent(XContentBuilder builder, Params params) throws IOException { builder.field("shard", shardId()); builder.field("index", index()); diff --git a/server/src/main/java/org/elasticsearch/action/support/ThreadedActionListener.java b/server/src/main/java/org/elasticsearch/action/support/ThreadedActionListener.java index ad72ef10139ba..00a89c982e02b 100644 --- a/server/src/main/java/org/elasticsearch/action/support/ThreadedActionListener.java +++ b/server/src/main/java/org/elasticsearch/action/support/ThreadedActionListener.java @@ -23,53 +23,14 @@ import org.apache.logging.log4j.message.ParameterizedMessage; import org.elasticsearch.action.ActionListener; import org.elasticsearch.action.ActionRunnable; -import org.elasticsearch.client.Client; -import org.elasticsearch.client.transport.TransportClient; -import org.elasticsearch.common.settings.Settings; import org.elasticsearch.common.util.concurrent.AbstractRunnable; import org.elasticsearch.threadpool.ThreadPool; -import java.util.concurrent.Future; - /** * An action listener that wraps another action listener and threading its execution. */ public final class ThreadedActionListener implements ActionListener { - /** - * Wrapper that can be used to automatically wrap a listener in a threaded listener if needed. - */ - public static class Wrapper { - - private final Logger logger; - private final ThreadPool threadPool; - - private final boolean threadedListener; - - public Wrapper(Logger logger, Settings settings, ThreadPool threadPool) { - this.logger = logger; - this.threadPool = threadPool; - // Should the action listener be threaded or not by default. Action listeners are automatically threaded for - // the transport client in order to make sure client side code is not executed on IO threads. - this.threadedListener = TransportClient.CLIENT_TYPE.equals(Client.CLIENT_TYPE_SETTING_S.get(settings)); - } - - public ActionListener wrap(ActionListener listener) { - if (threadedListener == false) { - return listener; - } - // if its a future, the callback is very lightweight (flipping a bit) so no need to wrap it - if (listener instanceof Future) { - return listener; - } - // already threaded... - if (listener instanceof ThreadedActionListener) { - return listener; - } - return new ThreadedActionListener<>(logger, threadPool, ThreadPool.Names.LISTENER, listener, false); - } - } - private final Logger logger; private final ThreadPool threadPool; private final String executor; diff --git a/server/src/main/java/org/elasticsearch/action/support/broadcast/BroadcastOperationRequestBuilder.java b/server/src/main/java/org/elasticsearch/action/support/broadcast/BroadcastOperationRequestBuilder.java index fb6fdd3fa2421..dfcca19883e50 100644 --- a/server/src/main/java/org/elasticsearch/action/support/broadcast/BroadcastOperationRequestBuilder.java +++ b/server/src/main/java/org/elasticsearch/action/support/broadcast/BroadcastOperationRequestBuilder.java @@ -19,7 +19,7 @@ package org.elasticsearch.action.support.broadcast; -import org.elasticsearch.action.Action; +import org.elasticsearch.action.ActionType; import org.elasticsearch.action.ActionRequestBuilder; import org.elasticsearch.action.support.IndicesOptions; import org.elasticsearch.client.ElasticsearchClient; @@ -30,7 +30,7 @@ public abstract class BroadcastOperationRequestBuilder< RequestBuilder extends BroadcastOperationRequestBuilder > extends ActionRequestBuilder { - protected BroadcastOperationRequestBuilder(ElasticsearchClient client, Action action, Request request) { + protected BroadcastOperationRequestBuilder(ElasticsearchClient client, ActionType action, Request request) { super(client, action, request); } diff --git a/server/src/main/java/org/elasticsearch/action/support/master/AcknowledgedRequestBuilder.java b/server/src/main/java/org/elasticsearch/action/support/master/AcknowledgedRequestBuilder.java index fd4432f27d690..d37d2d45917fe 100644 --- a/server/src/main/java/org/elasticsearch/action/support/master/AcknowledgedRequestBuilder.java +++ b/server/src/main/java/org/elasticsearch/action/support/master/AcknowledgedRequestBuilder.java @@ -18,7 +18,7 @@ */ package org.elasticsearch.action.support.master; -import org.elasticsearch.action.Action; +import org.elasticsearch.action.ActionType; import org.elasticsearch.client.ElasticsearchClient; import org.elasticsearch.common.unit.TimeValue; @@ -29,7 +29,7 @@ public abstract class AcknowledgedRequestBuilder> extends MasterNodeOperationRequestBuilder { - protected AcknowledgedRequestBuilder(ElasticsearchClient client, Action action, Request request) { + protected AcknowledgedRequestBuilder(ElasticsearchClient client, ActionType action, Request request) { super(client, action, request); } diff --git a/server/src/main/java/org/elasticsearch/action/support/master/AcknowledgedResponse.java b/server/src/main/java/org/elasticsearch/action/support/master/AcknowledgedResponse.java index 21a0d3f21a804..4e1540fd995e1 100644 --- a/server/src/main/java/org/elasticsearch/action/support/master/AcknowledgedResponse.java +++ b/server/src/main/java/org/elasticsearch/action/support/master/AcknowledgedResponse.java @@ -45,9 +45,11 @@ protected static void declareAcknowledgedField( ObjectParser.ValueType.BOOLEAN); } - protected boolean acknowledged; + protected final boolean acknowledged; - public AcknowledgedResponse() { + public AcknowledgedResponse(StreamInput in) throws IOException { + super(in); + acknowledged = in.readBoolean(); } public AcknowledgedResponse(boolean acknowledged) { @@ -64,8 +66,7 @@ public final boolean isAcknowledged() { @Override public void readFrom(StreamInput in) throws IOException { - super.readFrom(in); - acknowledged = in.readBoolean(); + throw new UnsupportedOperationException("usage of Streamable is to be replaced by Writeable"); } @Override diff --git a/server/src/main/java/org/elasticsearch/action/support/master/MasterNodeOperationRequestBuilder.java b/server/src/main/java/org/elasticsearch/action/support/master/MasterNodeOperationRequestBuilder.java index c99cfa0f13b2f..08c59b72e0919 100644 --- a/server/src/main/java/org/elasticsearch/action/support/master/MasterNodeOperationRequestBuilder.java +++ b/server/src/main/java/org/elasticsearch/action/support/master/MasterNodeOperationRequestBuilder.java @@ -19,7 +19,7 @@ package org.elasticsearch.action.support.master; -import org.elasticsearch.action.Action; +import org.elasticsearch.action.ActionType; import org.elasticsearch.action.ActionRequestBuilder; import org.elasticsearch.action.ActionResponse; import org.elasticsearch.client.ElasticsearchClient; @@ -32,7 +32,7 @@ public abstract class MasterNodeOperationRequestBuilder> extends ActionRequestBuilder { - protected MasterNodeOperationRequestBuilder(ElasticsearchClient client, Action action, Request request) { + protected MasterNodeOperationRequestBuilder(ElasticsearchClient client, ActionType action, Request request) { super(client, action, request); } diff --git a/server/src/main/java/org/elasticsearch/action/support/master/MasterNodeReadOperationRequestBuilder.java b/server/src/main/java/org/elasticsearch/action/support/master/MasterNodeReadOperationRequestBuilder.java index e17987264a28d..f385746365e9a 100644 --- a/server/src/main/java/org/elasticsearch/action/support/master/MasterNodeReadOperationRequestBuilder.java +++ b/server/src/main/java/org/elasticsearch/action/support/master/MasterNodeReadOperationRequestBuilder.java @@ -19,7 +19,7 @@ package org.elasticsearch.action.support.master; -import org.elasticsearch.action.Action; +import org.elasticsearch.action.ActionType; import org.elasticsearch.action.ActionResponse; import org.elasticsearch.client.ElasticsearchClient; @@ -30,7 +30,7 @@ public abstract class MasterNodeReadOperationRequestBuilder> extends MasterNodeOperationRequestBuilder { - protected MasterNodeReadOperationRequestBuilder(ElasticsearchClient client, Action action, Request request) { + protected MasterNodeReadOperationRequestBuilder(ElasticsearchClient client, ActionType action, Request request) { super(client, action, request); } diff --git a/server/src/main/java/org/elasticsearch/action/support/master/ShardsAcknowledgedResponse.java b/server/src/main/java/org/elasticsearch/action/support/master/ShardsAcknowledgedResponse.java index 21ff8fe644997..e088f4414a217 100644 --- a/server/src/main/java/org/elasticsearch/action/support/master/ShardsAcknowledgedResponse.java +++ b/server/src/main/java/org/elasticsearch/action/support/master/ShardsAcknowledgedResponse.java @@ -42,10 +42,15 @@ protected static void declareAcknowledged ObjectParser.ValueType.BOOLEAN); } - private boolean shardsAcknowledged; - - - protected ShardsAcknowledgedResponse() { + private final boolean shardsAcknowledged; + + protected ShardsAcknowledgedResponse(StreamInput in, boolean readShardsAcknowledged) throws IOException { + super(in); + if (readShardsAcknowledged) { + this.shardsAcknowledged = in.readBoolean(); + } else { + this.shardsAcknowledged = false; + } } protected ShardsAcknowledgedResponse(boolean acknowledged, boolean shardsAcknowledged) { @@ -63,10 +68,6 @@ public boolean isShardsAcknowledged() { return shardsAcknowledged; } - protected void readShardsAcknowledged(StreamInput in) throws IOException { - shardsAcknowledged = in.readBoolean(); - } - protected void writeShardsAcknowledged(StreamOutput out) throws IOException { out.writeBoolean(shardsAcknowledged); } @@ -80,14 +81,14 @@ protected void addCustomFields(XContentBuilder builder, Params params) throws IO public boolean equals(Object o) { if (super.equals(o)) { ShardsAcknowledgedResponse that = (ShardsAcknowledgedResponse) o; - return shardsAcknowledged == that.shardsAcknowledged; + return isShardsAcknowledged() == that.isShardsAcknowledged(); } return false; } @Override public int hashCode() { - return Objects.hash(super.hashCode(), shardsAcknowledged); + return Objects.hash(super.hashCode(), isShardsAcknowledged()); } } diff --git a/server/src/main/java/org/elasticsearch/action/support/master/TransportMasterNodeAction.java b/server/src/main/java/org/elasticsearch/action/support/master/TransportMasterNodeAction.java index 595b72f8da803..03c7346df5aab 100644 --- a/server/src/main/java/org/elasticsearch/action/support/master/TransportMasterNodeAction.java +++ b/server/src/main/java/org/elasticsearch/action/support/master/TransportMasterNodeAction.java @@ -118,14 +118,8 @@ protected Response read(StreamInput in) throws IOException { return response; } - protected abstract void masterOperation(Request request, ClusterState state, ActionListener listener) throws Exception; - - /** - * Override this operation if access to the task parameter is needed - */ - protected void masterOperation(Task task, Request request, ClusterState state, ActionListener listener) throws Exception { - masterOperation(request, state, listener); - } + protected abstract void masterOperation(Task task, Request request, ClusterState state, + ActionListener listener) throws Exception; protected boolean localExecute(Request request) { return false; diff --git a/server/src/main/java/org/elasticsearch/action/support/master/info/ClusterInfoRequestBuilder.java b/server/src/main/java/org/elasticsearch/action/support/master/info/ClusterInfoRequestBuilder.java index d6193b8d726cf..41833a563c4e4 100644 --- a/server/src/main/java/org/elasticsearch/action/support/master/info/ClusterInfoRequestBuilder.java +++ b/server/src/main/java/org/elasticsearch/action/support/master/info/ClusterInfoRequestBuilder.java @@ -18,7 +18,7 @@ */ package org.elasticsearch.action.support.master.info; -import org.elasticsearch.action.Action; +import org.elasticsearch.action.ActionType; import org.elasticsearch.action.ActionResponse; import org.elasticsearch.action.support.IndicesOptions; import org.elasticsearch.action.support.master.MasterNodeReadOperationRequestBuilder; @@ -30,7 +30,7 @@ public abstract class ClusterInfoRequestBuilder { - protected ClusterInfoRequestBuilder(ElasticsearchClient client, Action action, Request request) { + protected ClusterInfoRequestBuilder(ElasticsearchClient client, ActionType action, Request request) { super(client, action, request); } diff --git a/server/src/main/java/org/elasticsearch/action/support/master/info/TransportClusterInfoAction.java b/server/src/main/java/org/elasticsearch/action/support/master/info/TransportClusterInfoAction.java index dc46bfc6d1d31..0c3c70c9d02dd 100644 --- a/server/src/main/java/org/elasticsearch/action/support/master/info/TransportClusterInfoAction.java +++ b/server/src/main/java/org/elasticsearch/action/support/master/info/TransportClusterInfoAction.java @@ -26,6 +26,7 @@ import org.elasticsearch.cluster.metadata.IndexNameExpressionResolver; import org.elasticsearch.cluster.service.ClusterService; import org.elasticsearch.common.io.stream.Writeable; +import org.elasticsearch.tasks.Task; import org.elasticsearch.threadpool.ThreadPool; import org.elasticsearch.transport.TransportService; @@ -46,7 +47,8 @@ protected String executor() { } @Override - protected final void masterOperation(final Request request, final ClusterState state, final ActionListener listener) { + protected final void masterOperation(Task task, final Request request, final ClusterState state, + final ActionListener listener) { String[] concreteIndices = indexNameExpressionResolver.concreteIndexNames(state, request); doMasterOperation(request, concreteIndices, state, listener); } diff --git a/server/src/main/java/org/elasticsearch/action/support/nodes/BaseNodeRequest.java b/server/src/main/java/org/elasticsearch/action/support/nodes/BaseNodeRequest.java index 1ce32247eab18..21b09bdaaae0a 100644 --- a/server/src/main/java/org/elasticsearch/action/support/nodes/BaseNodeRequest.java +++ b/server/src/main/java/org/elasticsearch/action/support/nodes/BaseNodeRequest.java @@ -19,33 +19,31 @@ package org.elasticsearch.action.support.nodes; +import org.elasticsearch.Version; import org.elasticsearch.common.io.stream.StreamInput; import org.elasticsearch.common.io.stream.StreamOutput; import org.elasticsearch.transport.TransportRequest; import java.io.IOException; +// TODO: this class can be removed in master once 7.x is bumped to 7.4.0 public abstract class BaseNodeRequest extends TransportRequest { - private String nodeId; - - public BaseNodeRequest() { - - } - - protected BaseNodeRequest(String nodeId) { - this.nodeId = nodeId; - } + public BaseNodeRequest() {} @Override public void readFrom(StreamInput in) throws IOException { super.readFrom(in); - nodeId = in.readString(); + if (in.getVersion().before(Version.V_7_3_0)) { + in.readString(); // previously nodeId + } } @Override public void writeTo(StreamOutput out) throws IOException { super.writeTo(out); - out.writeString(nodeId); + if (out.getVersion().before(Version.V_7_3_0)) { + out.writeString(""); // previously nodeId + } } } diff --git a/server/src/main/java/org/elasticsearch/action/support/nodes/BaseNodesRequest.java b/server/src/main/java/org/elasticsearch/action/support/nodes/BaseNodesRequest.java index dac584a614429..ca2932291fd3a 100644 --- a/server/src/main/java/org/elasticsearch/action/support/nodes/BaseNodesRequest.java +++ b/server/src/main/java/org/elasticsearch/action/support/nodes/BaseNodesRequest.java @@ -38,8 +38,7 @@ public abstract class BaseNodesRequest * * See {@link DiscoveryNodes#resolveNodes} for a full description of the options. * - * TODO: once we stop using the transport client as a gateway to the cluster, we can get rid of this and resolve it to concrete nodes - * in the rest layer + * TODO: we can get rid of this and resolve it to concrete nodes in the rest layer **/ private String[] nodesIds; diff --git a/server/src/main/java/org/elasticsearch/action/support/nodes/NodesOperationRequestBuilder.java b/server/src/main/java/org/elasticsearch/action/support/nodes/NodesOperationRequestBuilder.java index 626481cef80c6..ca31a956224f3 100644 --- a/server/src/main/java/org/elasticsearch/action/support/nodes/NodesOperationRequestBuilder.java +++ b/server/src/main/java/org/elasticsearch/action/support/nodes/NodesOperationRequestBuilder.java @@ -19,7 +19,7 @@ package org.elasticsearch.action.support.nodes; -import org.elasticsearch.action.Action; +import org.elasticsearch.action.ActionType; import org.elasticsearch.action.ActionRequestBuilder; import org.elasticsearch.client.ElasticsearchClient; import org.elasticsearch.common.unit.TimeValue; @@ -28,7 +28,7 @@ public abstract class NodesOperationRequestBuilder> extends ActionRequestBuilder { - protected NodesOperationRequestBuilder(ElasticsearchClient client, Action action, Request request) { + protected NodesOperationRequestBuilder(ElasticsearchClient client, ActionType action, Request request) { super(client, action, request); } diff --git a/server/src/main/java/org/elasticsearch/action/support/nodes/TransportNodesAction.java b/server/src/main/java/org/elasticsearch/action/support/nodes/TransportNodesAction.java index bc4fb5d38c328..43f661084a89d 100644 --- a/server/src/main/java/org/elasticsearch/action/support/nodes/TransportNodesAction.java +++ b/server/src/main/java/org/elasticsearch/action/support/nodes/TransportNodesAction.java @@ -119,15 +119,11 @@ protected NodesResponse newResponse(NodesRequest request, AtomicReferenceArray n */ protected abstract NodesResponse newResponse(NodesRequest request, List responses, List failures); - protected abstract NodeRequest newNodeRequest(String nodeId, NodesRequest request); + protected abstract NodeRequest newNodeRequest(NodesRequest request); protected abstract NodeResponse newNodeResponse(); - protected abstract NodeResponse nodeOperation(NodeRequest request); - - protected NodeResponse nodeOperation(NodeRequest request, Task task) { - return nodeOperation(request); - } + protected abstract NodeResponse nodeOperation(NodeRequest request, Task task); /** * resolve node ids to concrete nodes of the incoming request @@ -174,7 +170,7 @@ void start() { final DiscoveryNode node = nodes[i]; final String nodeId = node.getId(); try { - TransportRequest nodeRequest = newNodeRequest(nodeId, request); + TransportRequest nodeRequest = newNodeRequest(request); if (task != null) { nodeRequest.setParentTask(clusterService.localNode().getId(), task.getId()); } diff --git a/server/src/main/java/org/elasticsearch/action/support/replication/ReplicationOperation.java b/server/src/main/java/org/elasticsearch/action/support/replication/ReplicationOperation.java index 7917d9c05078b..d328f06eb6895 100644 --- a/server/src/main/java/org/elasticsearch/action/support/replication/ReplicationOperation.java +++ b/server/src/main/java/org/elasticsearch/action/support/replication/ReplicationOperation.java @@ -111,6 +111,7 @@ public void execute() throws Exception { private void handlePrimaryResult(final PrimaryResultT primaryResult) { this.primaryResult = primaryResult; primary.updateLocalCheckpointForShard(primary.routingEntry().allocationId().getId(), primary.localCheckpoint()); + primary.updateGlobalCheckpointForShard(primary.routingEntry().allocationId().getId(), primary.globalCheckpoint()); final ReplicaRequest replicaRequest = primaryResult.replicaRequest(); if (replicaRequest != null) { if (logger.isTraceEnabled()) { @@ -123,7 +124,7 @@ private void handlePrimaryResult(final PrimaryResultT primaryResult) { // is valid for this replication group. If we would sample in the reverse, the global checkpoint might be based on a subset // of the sampled replication group, and advanced further than what the given replication group would allow it to. // This would entail that some shards could learn about a global checkpoint that would be higher than its local checkpoint. - final long globalCheckpoint = primary.globalCheckpoint(); + final long globalCheckpoint = primary.computedGlobalCheckpoint(); // we have to capture the max_seq_no_of_updates after this request was completed on the primary to make sure the value of // max_seq_no_of_updates on replica when this request is executed is at least the value on the primary when it was executed // on. @@ -341,16 +342,23 @@ public interface Primary< void updateGlobalCheckpointForShard(String allocationId, long globalCheckpoint); /** - * Returns the local checkpoint on the primary shard. + * Returns the persisted local checkpoint on the primary shard. * * @return the local checkpoint */ long localCheckpoint(); /** - * Returns the global checkpoint on the primary shard. + * Returns the global checkpoint computed on the primary shard. * - * @return the global checkpoint + * @return the computed global checkpoint + */ + long computedGlobalCheckpoint(); + + /** + * Returns the persisted global checkpoint on the primary shard. + * + * @return the persisted global checkpoint */ long globalCheckpoint(); @@ -419,16 +427,16 @@ void performOn(ShardRouting replica, RequestT replicaRequest, public interface ReplicaResponse { /** - * The local checkpoint for the shard. + * The persisted local checkpoint for the shard. * - * @return the local checkpoint + * @return the persisted local checkpoint **/ long localCheckpoint(); /** - * The global checkpoint for the shard. + * The persisted global checkpoint for the shard. * - * @return the global checkpoint + * @return the persisted global checkpoint **/ long globalCheckpoint(); diff --git a/server/src/main/java/org/elasticsearch/action/support/replication/ReplicationRequestBuilder.java b/server/src/main/java/org/elasticsearch/action/support/replication/ReplicationRequestBuilder.java index fed4c9a5f101f..d83319f62f0de 100644 --- a/server/src/main/java/org/elasticsearch/action/support/replication/ReplicationRequestBuilder.java +++ b/server/src/main/java/org/elasticsearch/action/support/replication/ReplicationRequestBuilder.java @@ -19,7 +19,7 @@ package org.elasticsearch.action.support.replication; -import org.elasticsearch.action.Action; +import org.elasticsearch.action.ActionType; import org.elasticsearch.action.ActionRequestBuilder; import org.elasticsearch.action.ActionResponse; import org.elasticsearch.action.support.ActiveShardCount; @@ -30,7 +30,7 @@ public abstract class ReplicationRequestBuilder> extends ActionRequestBuilder { - protected ReplicationRequestBuilder(ElasticsearchClient client, Action action, Request request) { + protected ReplicationRequestBuilder(ElasticsearchClient client, ActionType action, Request request) { super(client, action, request); } diff --git a/server/src/main/java/org/elasticsearch/action/support/replication/TransportReplicationAction.java b/server/src/main/java/org/elasticsearch/action/support/replication/TransportReplicationAction.java index d19009433deb5..e338c6e5c329e 100644 --- a/server/src/main/java/org/elasticsearch/action/support/replication/TransportReplicationAction.java +++ b/server/src/main/java/org/elasticsearch/action/support/replication/TransportReplicationAction.java @@ -526,7 +526,7 @@ public void onResponse(Releasable releasable) { final ReplicaResult replicaResult = shardOperationOnReplica(replicaRequest.getRequest(), replica); releasable.close(); // release shard operation lock before responding to caller final TransportReplicationAction.ReplicaResponse response = - new ReplicaResponse(replica.getLocalCheckpoint(), replica.getGlobalCheckpoint()); + new ReplicaResponse(replica.getLocalCheckpoint(), replica.getLastSyncedGlobalCheckpoint()); replicaResult.respond(new ResponseListener(response)); } catch (final Exception e) { Releasables.closeWhileHandlingException(releasable); // release shard operation lock before responding to caller @@ -892,10 +892,6 @@ public void close() { operationLock.close(); } - public long getLocalCheckpoint() { - return indexShard.getLocalCheckpoint(); - } - public ShardRouting routingEntry() { return indexShard.routingEntry(); } @@ -943,7 +939,12 @@ public long localCheckpoint() { @Override public long globalCheckpoint() { - return indexShard.getGlobalCheckpoint(); + return indexShard.getLastSyncedGlobalCheckpoint(); + } + + @Override + public long computedGlobalCheckpoint() { + return indexShard.getLastKnownGlobalCheckpoint(); } @Override diff --git a/server/src/main/java/org/elasticsearch/action/support/single/instance/InstanceShardOperationRequestBuilder.java b/server/src/main/java/org/elasticsearch/action/support/single/instance/InstanceShardOperationRequestBuilder.java index 924815d75b85a..722826615ae21 100644 --- a/server/src/main/java/org/elasticsearch/action/support/single/instance/InstanceShardOperationRequestBuilder.java +++ b/server/src/main/java/org/elasticsearch/action/support/single/instance/InstanceShardOperationRequestBuilder.java @@ -19,7 +19,7 @@ package org.elasticsearch.action.support.single.instance; -import org.elasticsearch.action.Action; +import org.elasticsearch.action.ActionType; import org.elasticsearch.action.ActionRequestBuilder; import org.elasticsearch.action.ActionResponse; import org.elasticsearch.client.ElasticsearchClient; @@ -29,7 +29,7 @@ public abstract class InstanceShardOperationRequestBuilder> extends ActionRequestBuilder { - protected InstanceShardOperationRequestBuilder(ElasticsearchClient client, Action action, Request request) { + protected InstanceShardOperationRequestBuilder(ElasticsearchClient client, ActionType action, Request request) { super(client, action, request); } diff --git a/server/src/main/java/org/elasticsearch/action/support/single/shard/SingleShardOperationRequestBuilder.java b/server/src/main/java/org/elasticsearch/action/support/single/shard/SingleShardOperationRequestBuilder.java index 4f1da43fd11bb..197b7e57bfeea 100644 --- a/server/src/main/java/org/elasticsearch/action/support/single/shard/SingleShardOperationRequestBuilder.java +++ b/server/src/main/java/org/elasticsearch/action/support/single/shard/SingleShardOperationRequestBuilder.java @@ -19,7 +19,7 @@ package org.elasticsearch.action.support.single.shard; -import org.elasticsearch.action.Action; +import org.elasticsearch.action.ActionType; import org.elasticsearch.action.ActionRequestBuilder; import org.elasticsearch.action.ActionResponse; import org.elasticsearch.client.ElasticsearchClient; @@ -28,7 +28,7 @@ public abstract class SingleShardOperationRequestBuilder> extends ActionRequestBuilder { - protected SingleShardOperationRequestBuilder(ElasticsearchClient client, Action action, Request request) { + protected SingleShardOperationRequestBuilder(ElasticsearchClient client, ActionType action, Request request) { super(client, action, request); } diff --git a/server/src/main/java/org/elasticsearch/action/support/single/shard/SingleShardRequest.java b/server/src/main/java/org/elasticsearch/action/support/single/shard/SingleShardRequest.java index 15b10832b4eff..747b6c54ee8a2 100644 --- a/server/src/main/java/org/elasticsearch/action/support/single/shard/SingleShardRequest.java +++ b/server/src/main/java/org/elasticsearch/action/support/single/shard/SingleShardRequest.java @@ -48,6 +48,15 @@ public abstract class SingleShardRequest request, + IndexNameExpressionResolver indexNameExpressionResolver, Writeable.Reader request, String executor) { super(actionName, actionFilters, transportService.getTaskManager()); this.threadPool = threadPool; @@ -85,9 +84,9 @@ protected TransportSingleShardAction(String actionName, ThreadPool threadPool, C this.executor = executor; if (!isSubAction()) { - transportService.registerRequestHandler(actionName, request, ThreadPool.Names.SAME, new TransportHandler()); + transportService.registerRequestHandler(actionName, ThreadPool.Names.SAME, request, new TransportHandler()); } - transportService.registerRequestHandler(transportShardAction, request, ThreadPool.Names.SAME, new ShardTransportHandler()); + transportService.registerRequestHandler(transportShardAction, ThreadPool.Names.SAME, request, new ShardTransportHandler()); } /** diff --git a/server/src/main/java/org/elasticsearch/action/support/tasks/TasksRequestBuilder.java b/server/src/main/java/org/elasticsearch/action/support/tasks/TasksRequestBuilder.java index 52e3c2cf44e33..9fba027e751a0 100644 --- a/server/src/main/java/org/elasticsearch/action/support/tasks/TasksRequestBuilder.java +++ b/server/src/main/java/org/elasticsearch/action/support/tasks/TasksRequestBuilder.java @@ -18,7 +18,7 @@ */ package org.elasticsearch.action.support.tasks; -import org.elasticsearch.action.Action; +import org.elasticsearch.action.ActionType; import org.elasticsearch.action.ActionRequestBuilder; import org.elasticsearch.client.ElasticsearchClient; import org.elasticsearch.common.unit.TimeValue; @@ -33,7 +33,7 @@ public class TasksRequestBuilder< RequestBuilder extends TasksRequestBuilder > extends ActionRequestBuilder { - protected TasksRequestBuilder(ElasticsearchClient client, Action action, Request request) { + protected TasksRequestBuilder(ElasticsearchClient client, ActionType action, Request request) { super(client, action, request); } diff --git a/server/src/main/java/org/elasticsearch/action/termvectors/MultiTermVectorsAction.java b/server/src/main/java/org/elasticsearch/action/termvectors/MultiTermVectorsAction.java index a894b3480f10f..e02969856e9b0 100644 --- a/server/src/main/java/org/elasticsearch/action/termvectors/MultiTermVectorsAction.java +++ b/server/src/main/java/org/elasticsearch/action/termvectors/MultiTermVectorsAction.java @@ -19,9 +19,9 @@ package org.elasticsearch.action.termvectors; -import org.elasticsearch.action.Action; +import org.elasticsearch.action.StreamableResponseActionType; -public class MultiTermVectorsAction extends Action { +public class MultiTermVectorsAction extends StreamableResponseActionType { public static final MultiTermVectorsAction INSTANCE = new MultiTermVectorsAction(); public static final String NAME = "indices:data/read/mtv"; diff --git a/server/src/main/java/org/elasticsearch/action/termvectors/MultiTermVectorsItemResponse.java b/server/src/main/java/org/elasticsearch/action/termvectors/MultiTermVectorsItemResponse.java index 14ac59cb132bd..9cacf166952fb 100644 --- a/server/src/main/java/org/elasticsearch/action/termvectors/MultiTermVectorsItemResponse.java +++ b/server/src/main/java/org/elasticsearch/action/termvectors/MultiTermVectorsItemResponse.java @@ -53,16 +53,6 @@ public String getIndex() { return response.getIndex(); } - /** - * The type of the document. - */ - public String getType() { - if (failure != null) { - return failure.getType(); - } - return response.getType(); - } - /** * The id of the document. */ diff --git a/server/src/main/java/org/elasticsearch/action/termvectors/MultiTermVectorsRequest.java b/server/src/main/java/org/elasticsearch/action/termvectors/MultiTermVectorsRequest.java index 5f869e2ac7235..96a207ca1a595 100644 --- a/server/src/main/java/org/elasticsearch/action/termvectors/MultiTermVectorsRequest.java +++ b/server/src/main/java/org/elasticsearch/action/termvectors/MultiTermVectorsRequest.java @@ -29,7 +29,6 @@ import org.elasticsearch.common.io.stream.StreamInput; import org.elasticsearch.common.io.stream.StreamOutput; import org.elasticsearch.common.xcontent.XContentParser; -import org.elasticsearch.index.mapper.MapperService; import java.io.IOException; import java.util.ArrayList; @@ -52,8 +51,8 @@ public MultiTermVectorsRequest add(TermVectorsRequest termVectorsRequest) { return this; } - public MultiTermVectorsRequest add(String index, @Nullable String type, String id) { - requests.add(new TermVectorsRequest(index, type, id)); + public MultiTermVectorsRequest add(String index, String id) { + requests.add(new TermVectorsRequest(index, id)); return this; } @@ -102,9 +101,6 @@ public void add(TermVectorsRequest template, @Nullable XContentParser parser) th throw new IllegalArgumentException("docs array element should include an object"); } TermVectorsRequest termVectorsRequest = new TermVectorsRequest(template); - if (termVectorsRequest.type() == null) { - termVectorsRequest.type(MapperService.SINGLE_MAPPING_NAME); - } TermVectorsRequest.parseRequest(termVectorsRequest, parser); add(termVectorsRequest); } @@ -143,7 +139,7 @@ public void readFrom(StreamInput in) throws IOException { int size = in.readVInt(); requests = new ArrayList<>(size); for (int i = 0; i < size; i++) { - requests.add(TermVectorsRequest.readTermVectorsRequest(in)); + requests.add(new TermVectorsRequest(in)); } } diff --git a/server/src/main/java/org/elasticsearch/action/termvectors/MultiTermVectorsRequestBuilder.java b/server/src/main/java/org/elasticsearch/action/termvectors/MultiTermVectorsRequestBuilder.java index b982c4f720930..7d0d1c781ec75 100644 --- a/server/src/main/java/org/elasticsearch/action/termvectors/MultiTermVectorsRequestBuilder.java +++ b/server/src/main/java/org/elasticsearch/action/termvectors/MultiTermVectorsRequestBuilder.java @@ -21,7 +21,6 @@ import org.elasticsearch.action.ActionRequestBuilder; import org.elasticsearch.client.ElasticsearchClient; -import org.elasticsearch.common.Nullable; public class MultiTermVectorsRequestBuilder extends ActionRequestBuilder { @@ -29,16 +28,16 @@ public MultiTermVectorsRequestBuilder(ElasticsearchClient client, MultiTermVecto super(client, action, new MultiTermVectorsRequest()); } - public MultiTermVectorsRequestBuilder add(String index, @Nullable String type, Iterable ids) { + public MultiTermVectorsRequestBuilder add(String index, Iterable ids) { for (String id : ids) { - request.add(index, type, id); + request.add(index, id); } return this; } - public MultiTermVectorsRequestBuilder add(String index, @Nullable String type, String... ids) { + public MultiTermVectorsRequestBuilder add(String index, String... ids) { for (String id : ids) { - request.add(index, type, id); + request.add(index, id); } return this; } diff --git a/server/src/main/java/org/elasticsearch/action/termvectors/MultiTermVectorsResponse.java b/server/src/main/java/org/elasticsearch/action/termvectors/MultiTermVectorsResponse.java index 97a2007410edd..2264958422ad1 100644 --- a/server/src/main/java/org/elasticsearch/action/termvectors/MultiTermVectorsResponse.java +++ b/server/src/main/java/org/elasticsearch/action/termvectors/MultiTermVectorsResponse.java @@ -20,6 +20,7 @@ package org.elasticsearch.action.termvectors; import org.elasticsearch.ElasticsearchException; +import org.elasticsearch.Version; import org.elasticsearch.action.ActionResponse; import org.elasticsearch.common.io.stream.StreamInput; import org.elasticsearch.common.io.stream.StreamOutput; @@ -38,7 +39,6 @@ public class MultiTermVectorsResponse extends ActionResponse implements Iterable */ public static class Failure implements Streamable { private String index; - private String type; private String id; private Exception cause; @@ -46,9 +46,8 @@ public static class Failure implements Streamable { } - public Failure(String index, String type, String id, Exception cause) { + public Failure(String index, String id, Exception cause) { this.index = index; - this.type = type; this.id = id; this.cause = cause; } @@ -60,16 +59,6 @@ public String getIndex() { return this.index; } - /** - * The type of the action. - * - * @deprecated Types are in the process of being removed. - */ - @Deprecated - public String getType() { - return type; - } - /** * The id of the action. */ @@ -93,7 +82,13 @@ public static Failure readFailure(StreamInput in) throws IOException { @Override public void readFrom(StreamInput in) throws IOException { index = in.readString(); - type = in.readOptionalString(); + if (in.getVersion().before(Version.V_8_0_0)) { + // types no longer relevant so ignore + String type = in.readOptionalString(); + if (type != null) { + throw new IllegalStateException("types are no longer supported but found [" + type + "]"); + } + } id = in.readString(); cause = in.readException(); } @@ -101,7 +96,10 @@ public void readFrom(StreamInput in) throws IOException { @Override public void writeTo(StreamOutput out) throws IOException { out.writeString(index); - out.writeOptionalString(type); + if (out.getVersion().before(Version.V_8_0_0)) { + // types not supported so send an empty array to previous versions + out.writeOptionalString(null); + } out.writeString(id); out.writeException(cause); } @@ -134,7 +132,6 @@ public XContentBuilder toXContent(XContentBuilder builder, Params params) throws builder.startObject(); Failure failure = response.getFailure(); builder.field(Fields._INDEX, failure.getIndex()); - builder.field(Fields._TYPE, failure.getType()); builder.field(Fields._ID, failure.getId()); ElasticsearchException.generateFailureXContent(builder, params, failure.getCause(), true); builder.endObject(); diff --git a/server/src/main/java/org/elasticsearch/action/termvectors/MultiTermVectorsShardRequest.java b/server/src/main/java/org/elasticsearch/action/termvectors/MultiTermVectorsShardRequest.java index 81f1b023578a7..2255b3243f123 100644 --- a/server/src/main/java/org/elasticsearch/action/termvectors/MultiTermVectorsShardRequest.java +++ b/server/src/main/java/org/elasticsearch/action/termvectors/MultiTermVectorsShardRequest.java @@ -37,8 +37,17 @@ public class MultiTermVectorsShardRequest extends SingleShardRequest requests; - public MultiTermVectorsShardRequest() { + MultiTermVectorsShardRequest(StreamInput in) throws IOException { + super(in); + int size = in.readVInt(); + locations = new IntArrayList(size); + requests = new ArrayList<>(size); + for (int i = 0; i < size; i++) { + locations.add(in.readVInt()); + requests.add(new TermVectorsRequest(in)); + } + preference = in.readOptionalString(); } MultiTermVectorsShardRequest(String index, int shardId) { @@ -86,20 +95,6 @@ public String[] indices() { return indices; } - @Override - public void readFrom(StreamInput in) throws IOException { - super.readFrom(in); - int size = in.readVInt(); - locations = new IntArrayList(size); - requests = new ArrayList<>(size); - for (int i = 0; i < size; i++) { - locations.add(in.readVInt()); - requests.add(TermVectorsRequest.readTermVectorsRequest(in)); - } - - preference = in.readOptionalString(); - } - @Override public void writeTo(StreamOutput out) throws IOException { super.writeTo(out); diff --git a/server/src/main/java/org/elasticsearch/action/termvectors/TermVectorsAction.java b/server/src/main/java/org/elasticsearch/action/termvectors/TermVectorsAction.java index 9b223eed3a3c8..a6696a28608d6 100644 --- a/server/src/main/java/org/elasticsearch/action/termvectors/TermVectorsAction.java +++ b/server/src/main/java/org/elasticsearch/action/termvectors/TermVectorsAction.java @@ -19,10 +19,10 @@ package org.elasticsearch.action.termvectors; -import org.elasticsearch.action.Action; +import org.elasticsearch.action.ActionType; import org.elasticsearch.common.io.stream.Writeable; -public class TermVectorsAction extends Action { +public class TermVectorsAction extends ActionType { public static final TermVectorsAction INSTANCE = new TermVectorsAction(); public static final String NAME = "indices:data/read/tv"; @@ -31,11 +31,6 @@ private TermVectorsAction() { super(NAME); } - @Override - public TermVectorsResponse newResponse() { - throw new UnsupportedOperationException("usage of Streamable is to be replaced by Writeable"); - } - @Override public Writeable.Reader getResponseReader() { return TermVectorsResponse::new; diff --git a/server/src/main/java/org/elasticsearch/action/termvectors/TermVectorsRequest.java b/server/src/main/java/org/elasticsearch/action/termvectors/TermVectorsRequest.java index 0206ef3def02d..09f7a15dff795 100644 --- a/server/src/main/java/org/elasticsearch/action/termvectors/TermVectorsRequest.java +++ b/server/src/main/java/org/elasticsearch/action/termvectors/TermVectorsRequest.java @@ -19,8 +19,8 @@ package org.elasticsearch.action.termvectors; -import org.apache.logging.log4j.LogManager; import org.elasticsearch.ElasticsearchParseException; +import org.elasticsearch.Version; import org.elasticsearch.action.ActionRequestValidationException; import org.elasticsearch.action.RealtimeRequest; import org.elasticsearch.action.ValidateActions; @@ -31,7 +31,6 @@ import org.elasticsearch.common.bytes.BytesReference; import org.elasticsearch.common.io.stream.StreamInput; import org.elasticsearch.common.io.stream.StreamOutput; -import org.elasticsearch.common.logging.DeprecationLogger; import org.elasticsearch.common.lucene.uid.Versions; import org.elasticsearch.common.util.set.Sets; import org.elasticsearch.common.xcontent.XContentBuilder; @@ -39,7 +38,6 @@ import org.elasticsearch.common.xcontent.XContentParser; import org.elasticsearch.common.xcontent.XContentType; import org.elasticsearch.index.VersionType; -import org.elasticsearch.rest.action.document.RestTermVectorsAction; import java.io.IOException; import java.util.ArrayList; @@ -54,18 +52,13 @@ import static org.elasticsearch.common.xcontent.XContentFactory.jsonBuilder; /** - * Request returning the term vector (doc frequency, positions, offsets) for a - * document. + * Request returning the term vector (doc frequency, positions, offsets) for a document. *

      - * Note, the {@link #index()}, {@link #type(String)} and {@link #id(String)} are - * required. + * Note, the {@link #index()} and {@link #id(String)} are required. */ public class TermVectorsRequest extends SingleShardRequest implements RealtimeRequest { - private static final DeprecationLogger deprecationLogger = new DeprecationLogger( - LogManager.getLogger(TermVectorsRequest.class)); private static final ParseField INDEX = new ParseField("_index"); - private static final ParseField TYPE = new ParseField("_type"); private static final ParseField ID = new ParseField("_id"); private static final ParseField ROUTING = new ParseField("routing"); private static final ParseField VERSION = new ParseField("version"); @@ -79,8 +72,6 @@ public class TermVectorsRequest extends SingleShardRequest i private static final ParseField DOC = new ParseField("doc"); - private String type; - private String id; private BytesReference doc; @@ -142,26 +133,65 @@ public void writeTo(StreamOutput out) throws IOException { public TermVectorsRequest() { } + TermVectorsRequest(StreamInput in) throws IOException { + super(in); + if (in.getVersion().before(Version.V_8_0_0)) { + // types no longer relevant so ignore + in.readString(); + } + id = in.readString(); + + if (in.readBoolean()) { + doc = in.readBytesReference(); + xContentType = in.readEnum(XContentType.class); + } + routing = in.readOptionalString(); + preference = in.readOptionalString(); + long flags = in.readVLong(); + + flagsEnum.clear(); + for (Flag flag : Flag.values()) { + if ((flags & (1 << flag.ordinal())) != 0) { + flagsEnum.add(flag); + } + } + int numSelectedFields = in.readVInt(); + if (numSelectedFields > 0) { + selectedFields = new HashSet<>(); + for (int i = 0; i < numSelectedFields; i++) { + selectedFields.add(in.readString()); + } + } + if (in.readBoolean()) { + perFieldAnalyzer = readPerFieldAnalyzer(in.readMap()); + } + if (in.readBoolean()) { + filterSettings = new FilterSettings(); + filterSettings.readFrom(in); + } + realtime = in.readBoolean(); + versionType = VersionType.fromValue(in.readByte()); + version = in.readLong(); + } + /** * Constructs a new term vector request for a document that will be fetch - * from the provided index. Use {@link #type(String)} and - * {@link #id(String)} to specify the document to load. + * from the provided index. Use and {@link #id(String)} to specify the + * document to load. */ - public TermVectorsRequest(String index, String type, String id) { + public TermVectorsRequest(String index, String id) { super(index); this.id = id; - this.type = type; } /** * Constructs a new term vector request for a document that will be fetch - * from the provided index. Use {@link #type(String)} and - * {@link #id(String)} to specify the document to load. + * from the provided index. Use {@link #id(String)} to specify the + * document to load. */ public TermVectorsRequest(TermVectorsRequest other) { super(other.index()); this.id = other.id(); - this.type = other.type(); if (other.doc != null) { this.doc = new BytesArray(other.doc().toBytesRef(), true); this.xContentType = other.xContentType; @@ -184,7 +214,6 @@ public TermVectorsRequest(TermVectorsRequest other) { public TermVectorsRequest(MultiGetRequest.Item item) { super(item.index()); this.id = item.id(); - this.type = item.type(); this.selectedFields(item.storedFields()); this.routing(item.routing()); } @@ -193,21 +222,6 @@ public EnumSet getFlags() { return flagsEnum; } - /** - * Sets the type of document to get the term vector for. - */ - public TermVectorsRequest type(String type) { - this.type = type; - return this; - } - - /** - * Returns the type of document to get the term vector for. - */ - public String type() { - return type; - } - /** * Returns the id of document the term vector is requested for. */ @@ -461,65 +475,19 @@ private void setFlag(Flag flag, boolean set) { @Override public ActionRequestValidationException validate() { ActionRequestValidationException validationException = super.validateNonNullIndex(); - if (type == null) { - validationException = ValidateActions.addValidationError("type is missing", validationException); - } if (id == null && doc == null) { validationException = ValidateActions.addValidationError("id or doc is missing", validationException); } return validationException; } - public static TermVectorsRequest readTermVectorsRequest(StreamInput in) throws IOException { - TermVectorsRequest termVectorsRequest = new TermVectorsRequest(); - termVectorsRequest.readFrom(in); - return termVectorsRequest; - } - - - @Override - public void readFrom(StreamInput in) throws IOException { - super.readFrom(in); - type = in.readString(); - id = in.readString(); - - if (in.readBoolean()) { - doc = in.readBytesReference(); - xContentType = in.readEnum(XContentType.class); - } - routing = in.readOptionalString(); - preference = in.readOptionalString(); - long flags = in.readVLong(); - - flagsEnum.clear(); - for (Flag flag : Flag.values()) { - if ((flags & (1 << flag.ordinal())) != 0) { - flagsEnum.add(flag); - } - } - int numSelectedFields = in.readVInt(); - if (numSelectedFields > 0) { - selectedFields = new HashSet<>(); - for (int i = 0; i < numSelectedFields; i++) { - selectedFields.add(in.readString()); - } - } - if (in.readBoolean()) { - perFieldAnalyzer = readPerFieldAnalyzer(in.readMap()); - } - if (in.readBoolean()) { - filterSettings = new FilterSettings(); - filterSettings.readFrom(in); - } - realtime = in.readBoolean(); - versionType = VersionType.fromValue(in.readByte()); - version = in.readLong(); - } - @Override public void writeTo(StreamOutput out) throws IOException { super.writeTo(out); - out.writeString(type); + if (out.getVersion().before(Version.V_8_0_0)) { + // types not supported so send an empty array to previous versions + out.writeString("_doc"); + } out.writeString(id); out.writeBoolean(doc != null); @@ -599,10 +567,6 @@ public static void parseRequest(TermVectorsRequest termVectorsRequest, XContentP } else if (INDEX.match(currentFieldName, parser.getDeprecationHandler())) { // the following is important for multi request parsing. termVectorsRequest.index = parser.text(); - } else if (TYPE.match(currentFieldName, parser.getDeprecationHandler())) { - termVectorsRequest.type = parser.text(); - deprecationLogger.deprecatedAndMaybeLog("termvectors_with_types", - RestTermVectorsAction.TYPES_DEPRECATION_MESSAGE); } else if (ID.match(currentFieldName, parser.getDeprecationHandler())) { if (termVectorsRequest.doc != null) { throw new ElasticsearchParseException("failed to parse term vectors request. " + diff --git a/server/src/main/java/org/elasticsearch/action/termvectors/TermVectorsRequestBuilder.java b/server/src/main/java/org/elasticsearch/action/termvectors/TermVectorsRequestBuilder.java index 80bf769b9714e..7bdf46bf08cba 100644 --- a/server/src/main/java/org/elasticsearch/action/termvectors/TermVectorsRequestBuilder.java +++ b/server/src/main/java/org/elasticsearch/action/termvectors/TermVectorsRequestBuilder.java @@ -44,8 +44,8 @@ public TermVectorsRequestBuilder(ElasticsearchClient client, TermVectorsAction a * from the provided index. Use {@code index}, {@code type} and * {@code id} to specify the document to load. */ - public TermVectorsRequestBuilder(ElasticsearchClient client, TermVectorsAction action, String index, String type, String id) { - super(client, action, new TermVectorsRequest(index, type, id)); + public TermVectorsRequestBuilder(ElasticsearchClient client, TermVectorsAction action, String index, String id) { + super(client, action, new TermVectorsRequest(index, id)); } /** @@ -56,14 +56,6 @@ public TermVectorsRequestBuilder setIndex(String index) { return this; } - /** - * Sets the type of the document. - */ - public TermVectorsRequestBuilder setType(String type) { - request.type(type); - return this; - } - /** * Sets the id of the document. */ diff --git a/server/src/main/java/org/elasticsearch/action/termvectors/TermVectorsResponse.java b/server/src/main/java/org/elasticsearch/action/termvectors/TermVectorsResponse.java index 3d0fb75f8d3eb..2cd1e3cccd7e5 100644 --- a/server/src/main/java/org/elasticsearch/action/termvectors/TermVectorsResponse.java +++ b/server/src/main/java/org/elasticsearch/action/termvectors/TermVectorsResponse.java @@ -27,6 +27,7 @@ import org.apache.lucene.util.ArrayUtil; import org.apache.lucene.util.BytesRef; import org.apache.lucene.util.CharsRefBuilder; +import org.elasticsearch.Version; import org.elasticsearch.action.ActionResponse; import org.elasticsearch.action.termvectors.TermVectorsRequest.Flag; import org.elasticsearch.common.Nullable; @@ -38,6 +39,7 @@ import org.elasticsearch.common.unit.TimeValue; import org.elasticsearch.common.xcontent.ToXContentObject; import org.elasticsearch.common.xcontent.XContentBuilder; +import org.elasticsearch.index.mapper.MapperService; import org.elasticsearch.search.dfs.AggregatedDfs; import java.io.IOException; @@ -79,7 +81,6 @@ private static class FieldStrings { private BytesReference termVectors; private BytesReference headerRef; private String index; - private String type; private String id; private long docVersion; private boolean exists = false; @@ -94,9 +95,8 @@ private static class FieldStrings { int[] currentEndOffset = new int[0]; BytesReference[] currentPayloads = new BytesReference[0]; - public TermVectorsResponse(String index, String type, String id) { + public TermVectorsResponse(String index, String id) { this.index = index; - this.type = type; this.id = id; } @@ -105,7 +105,10 @@ public TermVectorsResponse(String index, String type, String id) { TermVectorsResponse(StreamInput in) throws IOException { index = in.readString(); - type = in.readString(); + if (in.getVersion().before(Version.V_8_0_0)) { + // types no longer relevant so ignore + in.readString(); + } id = in.readString(); docVersion = in.readVLong(); exists = in.readBoolean(); @@ -120,7 +123,10 @@ public TermVectorsResponse(String index, String type, String id) { @Override public void writeTo(StreamOutput out) throws IOException { out.writeString(index); - out.writeString(type); + if (out.getVersion().before(Version.V_8_0_0)) { + // types not supported so send an empty array to previous versions + out.writeString(MapperService.SINGLE_MAPPING_NAME); + } out.writeString(id); out.writeVLong(docVersion); final boolean docExists = isExists(); @@ -176,11 +182,9 @@ public int size() { @Override public XContentBuilder toXContent(XContentBuilder builder, Params params) throws IOException { assert index != null; - assert type != null; assert id != null; builder.startObject(); builder.field(FieldStrings._INDEX, index); - builder.field(FieldStrings._TYPE, type); if (!isArtificial()) { builder.field(FieldStrings._ID, id); } @@ -391,10 +395,6 @@ public String getIndex() { return index; } - public String getType() { - return type; - } - public String getId() { return id; } diff --git a/server/src/main/java/org/elasticsearch/action/termvectors/TransportMultiTermVectorsAction.java b/server/src/main/java/org/elasticsearch/action/termvectors/TransportMultiTermVectorsAction.java index 20514733aff31..1edd589a08a55 100644 --- a/server/src/main/java/org/elasticsearch/action/termvectors/TransportMultiTermVectorsAction.java +++ b/server/src/main/java/org/elasticsearch/action/termvectors/TransportMultiTermVectorsAction.java @@ -23,6 +23,7 @@ import org.elasticsearch.action.RoutingMissingException; import org.elasticsearch.action.support.ActionFilters; import org.elasticsearch.action.support.HandledTransportAction; +import org.elasticsearch.client.node.NodeClient; import org.elasticsearch.cluster.ClusterState; import org.elasticsearch.cluster.block.ClusterBlockLevel; import org.elasticsearch.cluster.metadata.IndexNameExpressionResolver; @@ -41,16 +42,16 @@ public class TransportMultiTermVectorsAction extends HandledTransportAction { private final ClusterService clusterService; - private final TransportShardMultiTermsVectorAction shardAction; + private final NodeClient client; private final IndexNameExpressionResolver indexNameExpressionResolver; @Inject public TransportMultiTermVectorsAction(TransportService transportService, ClusterService clusterService, - TransportShardMultiTermsVectorAction shardAction, ActionFilters actionFilters, + NodeClient client, ActionFilters actionFilters, IndexNameExpressionResolver indexNameExpressionResolver) { super(MultiTermVectorsAction.NAME, transportService, actionFilters, MultiTermVectorsRequest::new); this.clusterService = clusterService; - this.shardAction = shardAction; + this.client = client; this.indexNameExpressionResolver = indexNameExpressionResolver; } @@ -69,7 +70,7 @@ protected void doExecute(Task task, final MultiTermVectorsRequest request, final termVectorsRequest.index())); if (!clusterState.metaData().hasConcreteIndex(termVectorsRequest.index())) { responses.set(i, new MultiTermVectorsItemResponse(null, - new MultiTermVectorsResponse.Failure(termVectorsRequest.index(), termVectorsRequest.type(), termVectorsRequest.id(), + new MultiTermVectorsResponse.Failure(termVectorsRequest.index(), termVectorsRequest.id(), new IndexNotFoundException(termVectorsRequest.index())))); continue; } @@ -77,8 +78,8 @@ protected void doExecute(Task task, final MultiTermVectorsRequest request, final if (termVectorsRequest.routing() == null && clusterState.getMetaData().routingRequired(concreteSingleIndex)) { responses.set(i, new MultiTermVectorsItemResponse(null, - new MultiTermVectorsResponse.Failure(concreteSingleIndex, termVectorsRequest.type(), termVectorsRequest.id(), - new RoutingMissingException(concreteSingleIndex, termVectorsRequest.type(), termVectorsRequest.id())))); + new MultiTermVectorsResponse.Failure(concreteSingleIndex, termVectorsRequest.id(), + new RoutingMissingException(concreteSingleIndex, termVectorsRequest.id())))); continue; } ShardId shardId = clusterService.operationRouting().shardId(clusterState, concreteSingleIndex, @@ -106,7 +107,7 @@ protected void executeShardAction(ActionListener liste final AtomicInteger counter = new AtomicInteger(shardRequests.size()); for (final MultiTermVectorsShardRequest shardRequest : shardRequests.values()) { - shardAction.execute(shardRequest, new ActionListener() { + client.executeLocally(TransportShardMultiTermsVectorAction.TYPE, shardRequest, new ActionListener<>() { @Override public void onResponse(MultiTermVectorsShardResponse response) { for (int i = 0; i < response.locations.size(); i++) { @@ -124,8 +125,7 @@ public void onFailure(Exception e) { for (int i = 0; i < shardRequest.locations.size(); i++) { TermVectorsRequest termVectorsRequest = shardRequest.requests.get(i); responses.set(shardRequest.locations.get(i), new MultiTermVectorsItemResponse(null, - new MultiTermVectorsResponse.Failure(shardRequest.index(), termVectorsRequest.type(), - termVectorsRequest.id(), e))); + new MultiTermVectorsResponse.Failure(shardRequest.index(), termVectorsRequest.id(), e))); } if (counter.decrementAndGet() == 0) { finishHim(); diff --git a/server/src/main/java/org/elasticsearch/action/termvectors/TransportShardMultiTermsVectorAction.java b/server/src/main/java/org/elasticsearch/action/termvectors/TransportShardMultiTermsVectorAction.java index 0292757121eff..e49ea292c61c8 100644 --- a/server/src/main/java/org/elasticsearch/action/termvectors/TransportShardMultiTermsVectorAction.java +++ b/server/src/main/java/org/elasticsearch/action/termvectors/TransportShardMultiTermsVectorAction.java @@ -20,6 +20,8 @@ package org.elasticsearch.action.termvectors; import org.apache.logging.log4j.message.ParameterizedMessage; +import org.elasticsearch.action.ActionType; +import org.elasticsearch.action.StreamableResponseActionType; import org.elasticsearch.action.support.ActionFilters; import org.elasticsearch.action.support.TransportActions; import org.elasticsearch.action.support.single.shard.TransportSingleShardAction; @@ -43,6 +45,12 @@ public class TransportShardMultiTermsVectorAction extends private final IndicesService indicesService; private static final String ACTION_NAME = MultiTermVectorsAction.NAME + "[shard]"; + public static final ActionType TYPE = new StreamableResponseActionType<>(ACTION_NAME) { + @Override + public MultiTermVectorsShardResponse newResponse() { + return new MultiTermVectorsShardResponse(); + } + }; @Inject public TransportShardMultiTermsVectorAction(ClusterService clusterService, TransportService transportService, @@ -88,10 +96,10 @@ protected MultiTermVectorsShardResponse shardOperation(MultiTermVectorsShardRequ if (TransportActions.isShardNotAvailableException(e)) { throw e; } else { - logger.debug(() -> new ParameterizedMessage("{} failed to execute multi term vectors for [{}]/[{}]", - shardId, termVectorsRequest.type(), termVectorsRequest.id()), e); + logger.debug(() -> new ParameterizedMessage("{} failed to execute multi term vectors for [{}]", shardId, + termVectorsRequest.id()), e); response.add(request.locations.get(i), - new MultiTermVectorsResponse.Failure(request.index(), termVectorsRequest.type(), termVectorsRequest.id(), e)); + new MultiTermVectorsResponse.Failure(request.index(), termVectorsRequest.id(), e)); } } } diff --git a/server/src/main/java/org/elasticsearch/action/termvectors/TransportTermVectorsAction.java b/server/src/main/java/org/elasticsearch/action/termvectors/TransportTermVectorsAction.java index 0e212ab7cce2f..48ff18cb49b18 100644 --- a/server/src/main/java/org/elasticsearch/action/termvectors/TransportTermVectorsAction.java +++ b/server/src/main/java/org/elasticsearch/action/termvectors/TransportTermVectorsAction.java @@ -81,7 +81,7 @@ protected void resolveRequest(ClusterState state, InternalRequest request) { request.request().routing(state.metaData().resolveIndexRouting(request.request().routing(), request.request().index())); // Fail fast on the node that received the request. if (request.request().routing() == null && state.getMetaData().routingRequired(request.concreteIndex())) { - throw new RoutingMissingException(request.concreteIndex(), request.request().type(), request.request().id()); + throw new RoutingMissingException(request.concreteIndex(), request.request().id()); } } diff --git a/server/src/main/java/org/elasticsearch/action/update/UpdateAction.java b/server/src/main/java/org/elasticsearch/action/update/UpdateAction.java index 1c8c80b61cef1..f24f5436f5e4e 100644 --- a/server/src/main/java/org/elasticsearch/action/update/UpdateAction.java +++ b/server/src/main/java/org/elasticsearch/action/update/UpdateAction.java @@ -19,9 +19,9 @@ package org.elasticsearch.action.update; -import org.elasticsearch.action.Action; +import org.elasticsearch.action.StreamableResponseActionType; -public class UpdateAction extends Action { +public class UpdateAction extends StreamableResponseActionType { public static final UpdateAction INSTANCE = new UpdateAction(); public static final String NAME = "indices:data/write/update"; diff --git a/server/src/main/java/org/elasticsearch/bootstrap/Security.java b/server/src/main/java/org/elasticsearch/bootstrap/Security.java index 2a537186f6af8..b198dee979442 100644 --- a/server/src/main/java/org/elasticsearch/bootstrap/Security.java +++ b/server/src/main/java/org/elasticsearch/bootstrap/Security.java @@ -29,7 +29,6 @@ import org.elasticsearch.plugins.PluginsService; import org.elasticsearch.secure_sm.SecureSM; import org.elasticsearch.transport.TcpTransport; -import org.elasticsearch.transport.TransportSettings; import java.io.IOException; import java.net.SocketPermission; @@ -61,7 +60,7 @@ /** * Initializes SecurityManager with necessary permissions. *
      - *

      Initialization

      + *

      Initialization

      * The JVM is not initially started with security manager enabled, * instead we turn it on early in the startup process. This is a tradeoff * between security and ease of use: @@ -72,7 +71,7 @@ * otherwise be permitted. *
    *
    - *

    Permissions

    + *

    Permissions

    * Permissions use a policy file packaged as a resource, this file is * also used in tests. File permissions are generated dynamically and * combined with this policy file. @@ -92,7 +91,7 @@ * cleanups to the scripting apis). But still it can provide some defense for users * that enable dynamic scripting without being fully aware of the consequences. *
    - *

    Debugging Security

    + *

    Debugging Security

    * A good place to start when there is a problem is to turn on security debugging: *
      * ES_JAVA_OPTS="-Djava.security.debug=access,failure" bin/elasticsearch
    @@ -362,17 +361,6 @@ private static void addSocketPermissionForTransportProfiles(final Permissions po
             }
         }
     
    -    /**
    -     * Add dynamic {@link SocketPermission} based on transport settings.
    -     *
    -     * @param policy          the {@link Permissions} instance to apply the dynamic {@link SocketPermission}s to
    -     * @param settings        the {@link Settings} instance to read the transport settings from
    -     */
    -    private static void addSocketPermissionForTransport(final Permissions policy, final Settings settings) {
    -        final String transportRange = TransportSettings.PORT.get(settings);
    -        addSocketPermissionForPortRange(policy, transportRange);
    -    }
    -
         /**
          * Add dynamic {@link SocketPermission} for the specified port range.
          *
    diff --git a/server/src/main/java/org/elasticsearch/client/Client.java b/server/src/main/java/org/elasticsearch/client/Client.java
    index 07871709f5726..3ea28ad074be0 100644
    --- a/server/src/main/java/org/elasticsearch/client/Client.java
    +++ b/server/src/main/java/org/elasticsearch/client/Client.java
    @@ -77,11 +77,9 @@
      * simply returns an {@link org.elasticsearch.action.ActionFuture}, while the second accepts an
      * {@link org.elasticsearch.action.ActionListener}.
      * 

    - * A client can either be retrieved from a {@link org.elasticsearch.node.Node} started, or connected remotely - * to one or more nodes using {@link org.elasticsearch.client.transport.TransportClient}. + * A client can be retrieved from a started {@link org.elasticsearch.node.Node}. * * @see org.elasticsearch.node.Node#client() - * @see org.elasticsearch.client.transport.TransportClient */ public interface Client extends ElasticsearchClient, Releasable { @@ -365,10 +363,9 @@ public interface Client extends ElasticsearchClient, Releasable { * Builder for the term vector request. * * @param index The index to load the document from - * @param type The type of the document * @param id The id of the document */ - TermVectorsRequestBuilder prepareTermVectors(String index, String type, String id); + TermVectorsRequestBuilder prepareTermVectors(String index, String id); /** * Multi get term vectors. diff --git a/server/src/main/java/org/elasticsearch/client/ClusterAdminClient.java b/server/src/main/java/org/elasticsearch/client/ClusterAdminClient.java index 42aaed10d6172..cd874b62a40b4 100644 --- a/server/src/main/java/org/elasticsearch/client/ClusterAdminClient.java +++ b/server/src/main/java/org/elasticsearch/client/ClusterAdminClient.java @@ -484,19 +484,19 @@ public interface ClusterAdminClient extends ElasticsearchClient { CreateSnapshotRequestBuilder prepareCreateSnapshot(String repository, String name); /** - * Get snapshot. + * Get snapshots. */ ActionFuture getSnapshots(GetSnapshotsRequest request); /** - * Get snapshot. + * Get snapshots. */ void getSnapshots(GetSnapshotsRequest request, ActionListener listener); /** - * Get snapshot. + * Get snapshots. */ - GetSnapshotsRequestBuilder prepareGetSnapshots(String repository); + GetSnapshotsRequestBuilder prepareGetSnapshots(String... repository); /** * Delete snapshot. diff --git a/server/src/main/java/org/elasticsearch/client/ElasticsearchClient.java b/server/src/main/java/org/elasticsearch/client/ElasticsearchClient.java index 284c2b8d51420..f59da77d88408 100644 --- a/server/src/main/java/org/elasticsearch/client/ElasticsearchClient.java +++ b/server/src/main/java/org/elasticsearch/client/ElasticsearchClient.java @@ -20,7 +20,7 @@ package org.elasticsearch.client; -import org.elasticsearch.action.Action; +import org.elasticsearch.action.ActionType; import org.elasticsearch.action.ActionFuture; import org.elasticsearch.action.ActionListener; import org.elasticsearch.action.ActionRequest; @@ -30,7 +30,7 @@ public interface ElasticsearchClient { /** - * Executes a generic action, denoted by an {@link org.elasticsearch.action.Action}. + * Executes a generic action, denoted by an {@link ActionType}. * * @param action The action type to execute. * @param request The action request. @@ -39,10 +39,10 @@ public interface ElasticsearchClient { * @return A future allowing to get back the response. */ ActionFuture execute( - Action action, Request request); + ActionType action, Request request); /** - * Executes a generic action, denoted by an {@link Action}. + * Executes a generic action, denoted by an {@link ActionType}. * * @param action The action type to execute. * @param request The action request. @@ -51,7 +51,7 @@ ActionFuture The response type. */ void execute( - Action action, Request request, ActionListener listener); + ActionType action, Request request, ActionListener listener); /** * Returns the threadpool used to execute requests on this client diff --git a/server/src/main/java/org/elasticsearch/client/FilterClient.java b/server/src/main/java/org/elasticsearch/client/FilterClient.java index b4230710414be..245a4d5986ebb 100644 --- a/server/src/main/java/org/elasticsearch/client/FilterClient.java +++ b/server/src/main/java/org/elasticsearch/client/FilterClient.java @@ -18,7 +18,7 @@ */ package org.elasticsearch.client; -import org.elasticsearch.action.Action; +import org.elasticsearch.action.ActionType; import org.elasticsearch.action.ActionListener; import org.elasticsearch.action.ActionRequest; import org.elasticsearch.action.ActionResponse; @@ -61,7 +61,7 @@ public void close() { @Override protected - void doExecute(Action action, Request request, ActionListener listener) { + void doExecute(ActionType action, Request request, ActionListener listener) { in().execute(action, request, listener); } diff --git a/server/src/main/java/org/elasticsearch/client/IndicesAdminClient.java b/server/src/main/java/org/elasticsearch/client/IndicesAdminClient.java index d5a73981f29f1..40c4c1046577b 100644 --- a/server/src/main/java/org/elasticsearch/client/IndicesAdminClient.java +++ b/server/src/main/java/org/elasticsearch/client/IndicesAdminClient.java @@ -23,14 +23,11 @@ import org.elasticsearch.action.ActionListener; import org.elasticsearch.action.admin.indices.alias.IndicesAliasesRequest; import org.elasticsearch.action.admin.indices.alias.IndicesAliasesRequestBuilder; -import org.elasticsearch.action.admin.indices.alias.exists.AliasesExistRequestBuilder; -import org.elasticsearch.action.admin.indices.alias.exists.AliasesExistResponse; import org.elasticsearch.action.admin.indices.alias.get.GetAliasesRequest; import org.elasticsearch.action.admin.indices.alias.get.GetAliasesRequestBuilder; import org.elasticsearch.action.admin.indices.alias.get.GetAliasesResponse; -import org.elasticsearch.action.admin.indices.analyze.AnalyzeRequest; +import org.elasticsearch.action.admin.indices.analyze.AnalyzeAction; import org.elasticsearch.action.admin.indices.analyze.AnalyzeRequestBuilder; -import org.elasticsearch.action.admin.indices.analyze.AnalyzeResponse; import org.elasticsearch.action.admin.indices.cache.clear.ClearIndicesCacheRequest; import org.elasticsearch.action.admin.indices.cache.clear.ClearIndicesCacheRequestBuilder; import org.elasticsearch.action.admin.indices.cache.clear.ClearIndicesCacheResponse; @@ -42,12 +39,6 @@ import org.elasticsearch.action.admin.indices.create.CreateIndexResponse; import org.elasticsearch.action.admin.indices.delete.DeleteIndexRequest; import org.elasticsearch.action.admin.indices.delete.DeleteIndexRequestBuilder; -import org.elasticsearch.action.admin.indices.exists.indices.IndicesExistsRequest; -import org.elasticsearch.action.admin.indices.exists.indices.IndicesExistsRequestBuilder; -import org.elasticsearch.action.admin.indices.exists.indices.IndicesExistsResponse; -import org.elasticsearch.action.admin.indices.exists.types.TypesExistsRequest; -import org.elasticsearch.action.admin.indices.exists.types.TypesExistsRequestBuilder; -import org.elasticsearch.action.admin.indices.exists.types.TypesExistsResponse; import org.elasticsearch.action.admin.indices.flush.FlushRequest; import org.elasticsearch.action.admin.indices.flush.FlushRequestBuilder; import org.elasticsearch.action.admin.indices.flush.FlushResponse; @@ -123,58 +114,6 @@ */ public interface IndicesAdminClient extends ElasticsearchClient { - /** - * Indices Exists. - * - * @param request The indices exists request - * @return The result future - * @see Requests#indicesExistsRequest(String...) - */ - ActionFuture exists(IndicesExistsRequest request); - - /** - * The status of one or more indices. - * - * @param request The indices status request - * @param listener A listener to be notified with a result - * @see Requests#indicesExistsRequest(String...) - */ - void exists(IndicesExistsRequest request, ActionListener listener); - - /** - * Indices exists. - */ - IndicesExistsRequestBuilder prepareExists(String... indices); - - - /** - * Types exists. - * - * @deprecated Types are deprecated and are in the process of being removed. - * @param request The types exists request - * @return The result future - */ - @Deprecated - ActionFuture typesExists(TypesExistsRequest request); - - /** - * Types exists. - * - * @deprecated Types are deprecated and are in the process of being removed. - * @param request The types exists - * @param listener A listener to be notified with a result - */ - @Deprecated - void typesExists(TypesExistsRequest request, ActionListener listener); - - /** - * Types exists. - * - * @deprecated Types are deprecated and are in the process of being removed. - */ - @Deprecated - TypesExistsRequestBuilder prepareTypesExists(String... index); - /** * Indices stats. */ @@ -585,26 +524,6 @@ public interface IndicesAdminClient extends ElasticsearchClient { */ GetAliasesRequestBuilder prepareGetAliases(String... aliases); - /** - * Allows to check to existence of aliases from indices. - */ - AliasesExistRequestBuilder prepareAliasesExist(String... aliases); - - /** - * Check to existence of index aliases. - * - * @param request The result future - */ - ActionFuture aliasesExist(GetAliasesRequest request); - - /** - * Check the existence of specified index aliases. - * - * @param request The index aliases request - * @param listener A listener to be notified with a result - */ - void aliasesExist(GetAliasesRequest request, ActionListener listener); - /** * Get index metadata for particular indices. * @@ -672,12 +591,12 @@ public interface IndicesAdminClient extends ElasticsearchClient { /** * Analyze text under the provided index. */ - ActionFuture analyze(AnalyzeRequest request); + ActionFuture analyze(AnalyzeAction.Request request); /** * Analyze text under the provided index. */ - void analyze(AnalyzeRequest request, ActionListener listener); + void analyze(AnalyzeAction.Request request, ActionListener listener); /** * Analyze text under the provided index. @@ -819,4 +738,5 @@ public interface IndicesAdminClient extends ElasticsearchClient { * Swaps the index pointed to by an alias given all provided conditions are satisfied */ void rolloverIndex(RolloverRequest request, ActionListener listener); + } diff --git a/server/src/main/java/org/elasticsearch/client/OriginSettingClient.java b/server/src/main/java/org/elasticsearch/client/OriginSettingClient.java index ad5a602fbae72..b2a8978f1584a 100644 --- a/server/src/main/java/org/elasticsearch/client/OriginSettingClient.java +++ b/server/src/main/java/org/elasticsearch/client/OriginSettingClient.java @@ -19,7 +19,7 @@ package org.elasticsearch.client; -import org.elasticsearch.action.Action; +import org.elasticsearch.action.ActionType; import org.elasticsearch.action.ActionListener; import org.elasticsearch.action.ActionRequest; import org.elasticsearch.action.ActionResponse; @@ -45,7 +45,7 @@ public OriginSettingClient(Client in, String origin) { @Override protected - void doExecute(Action action, Request request, ActionListener listener) { + void doExecute(ActionType action, Request request, ActionListener listener) { final Supplier supplier = in().threadPool().getThreadContext().newRestorableContext(false); try (ThreadContext.StoredContext ignore = in().threadPool().getThreadContext().stashWithOrigin(origin)) { super.doExecute(action, request, new ContextPreservingActionListener<>(supplier, listener)); diff --git a/server/src/main/java/org/elasticsearch/client/ParentTaskAssigningClient.java b/server/src/main/java/org/elasticsearch/client/ParentTaskAssigningClient.java index 27de6619053b3..f33afdf51ec09 100644 --- a/server/src/main/java/org/elasticsearch/client/ParentTaskAssigningClient.java +++ b/server/src/main/java/org/elasticsearch/client/ParentTaskAssigningClient.java @@ -19,7 +19,7 @@ package org.elasticsearch.client; -import org.elasticsearch.action.Action; +import org.elasticsearch.action.ActionType; import org.elasticsearch.action.ActionListener; import org.elasticsearch.action.ActionRequest; import org.elasticsearch.action.ActionResponse; @@ -58,7 +58,7 @@ public Client unwrap() { @Override protected - void doExecute(Action action, Request request, ActionListener listener) { + void doExecute(ActionType action, Request request, ActionListener listener) { request.setParentTask(parentTask); super.doExecute(action, request, listener); } diff --git a/server/src/main/java/org/elasticsearch/client/Requests.java b/server/src/main/java/org/elasticsearch/client/Requests.java index 19ad2fb397edc..a3eb23eebfe20 100644 --- a/server/src/main/java/org/elasticsearch/client/Requests.java +++ b/server/src/main/java/org/elasticsearch/client/Requests.java @@ -45,7 +45,6 @@ import org.elasticsearch.action.admin.indices.close.CloseIndexRequest; import org.elasticsearch.action.admin.indices.create.CreateIndexRequest; import org.elasticsearch.action.admin.indices.delete.DeleteIndexRequest; -import org.elasticsearch.action.admin.indices.exists.indices.IndicesExistsRequest; import org.elasticsearch.action.admin.indices.flush.FlushRequest; import org.elasticsearch.action.admin.indices.flush.SyncedFlushRequest; import org.elasticsearch.action.admin.indices.forcemerge.ForceMergeRequest; @@ -162,16 +161,6 @@ public static IndicesSegmentsRequest indicesSegmentsRequest(String... indices) { public static IndicesShardStoresRequest indicesShardStoresRequest(String... indices) { return new IndicesShardStoresRequest(indices); } - /** - * Creates an indices exists request. - * - * @param indices The indices to check if they exists or not. - * @return The indices exists request - * @see org.elasticsearch.client.IndicesAdminClient#exists(org.elasticsearch.action.admin.indices.exists.indices.IndicesExistsRequest) - */ - public static IndicesExistsRequest indicesExistsRequest(String... indices) { - return new IndicesExistsRequest(indices); - } /** * Creates a create index request. @@ -494,13 +483,13 @@ public static CreateSnapshotRequest createSnapshotRequest(String repository, Str } /** - * Gets snapshots from repository + * Gets snapshots from repositories * - * @param repository repository name + * @param repositories repository names * @return get snapshot request */ - public static GetSnapshotsRequest getSnapshotsRequest(String repository) { - return new GetSnapshotsRequest(repository); + public static GetSnapshotsRequest getSnapshotsRequest(String... repositories) { + return new GetSnapshotsRequest(repositories); } /** @@ -534,5 +523,4 @@ public static DeleteSnapshotRequest deleteSnapshotRequest(String repository, Str public static SnapshotsStatusRequest snapshotsStatusRequest(String repository) { return new SnapshotsStatusRequest(repository); } - } diff --git a/server/src/main/java/org/elasticsearch/client/node/NodeClient.java b/server/src/main/java/org/elasticsearch/client/node/NodeClient.java index 0ad863c936741..96b2c593a5ded 100644 --- a/server/src/main/java/org/elasticsearch/client/node/NodeClient.java +++ b/server/src/main/java/org/elasticsearch/client/node/NodeClient.java @@ -19,7 +19,7 @@ package org.elasticsearch.client.node; -import org.elasticsearch.action.Action; +import org.elasticsearch.action.ActionType; import org.elasticsearch.action.ActionListener; import org.elasticsearch.action.ActionRequest; import org.elasticsearch.action.ActionResponse; @@ -41,10 +41,10 @@ */ public class NodeClient extends AbstractClient { - private Map actions; + private Map actions; /** * The id of the local {@link DiscoveryNode}. Useful for generating task ids from tasks returned by - * {@link #executeLocally(Action, ActionRequest, TaskListener)}. + * {@link #executeLocally(ActionType, ActionRequest, TaskListener)}. */ private Supplier localNodeId; private RemoteClusterService remoteClusterService; @@ -53,7 +53,7 @@ public NodeClient(Settings settings, ThreadPool threadPool) { super(settings, threadPool); } - public void initialize(Map actions, Supplier localNodeId, + public void initialize(Map actions, Supplier localNodeId, RemoteClusterService remoteClusterService) { this.actions = actions; this.localNodeId = localNodeId; @@ -67,47 +67,47 @@ public void close() { @Override public - void doExecute(Action action, Request request, ActionListener listener) { + void doExecute(ActionType action, Request request, ActionListener listener) { // Discard the task because the Client interface doesn't use it. executeLocally(action, request, listener); } /** - * Execute an {@link Action} locally, returning that {@link Task} used to track it, and linking an {@link ActionListener}. Prefer this - * method if you don't need access to the task when listening for the response. This is the method used to implement the {@link Client} - * interface. + * Execute an {@link ActionType} locally, returning that {@link Task} used to track it, and linking an {@link ActionListener}. + * Prefer this method if you don't need access to the task when listening for the response. This is the method used to + * implement the {@link Client} interface. */ public < Request extends ActionRequest, Response extends ActionResponse - > Task executeLocally(Action action, Request request, ActionListener listener) { + > Task executeLocally(ActionType action, Request request, ActionListener listener) { return transportAction(action).execute(request, listener); } /** - * Execute an {@link Action} locally, returning that {@link Task} used to track it, and linking an {@link TaskListener}. Prefer this - * method if you need access to the task when listening for the response. + * Execute an {@link ActionType} locally, returning that {@link Task} used to track it, and linking an {@link TaskListener}. + * Prefer this method if you need access to the task when listening for the response. */ public < Request extends ActionRequest, Response extends ActionResponse - > Task executeLocally(Action action, Request request, TaskListener listener) { + > Task executeLocally(ActionType action, Request request, TaskListener listener) { return transportAction(action).execute(request, listener); } /** * The id of the local {@link DiscoveryNode}. Useful for generating task ids from tasks returned by - * {@link #executeLocally(Action, ActionRequest, TaskListener)}. + * {@link #executeLocally(ActionType, ActionRequest, TaskListener)}. */ public String getLocalNodeId() { return localNodeId.get(); } /** - * Get the {@link TransportAction} for an {@link Action}, throwing exceptions if the action isn't available. + * Get the {@link TransportAction} for an {@link ActionType}, throwing exceptions if the action isn't available. */ @SuppressWarnings("unchecked") private < Request extends ActionRequest, Response extends ActionResponse - > TransportAction transportAction(Action action) { + > TransportAction transportAction(ActionType action) { if (actions == null) { throw new IllegalStateException("NodeClient has not been initialized"); } diff --git a/server/src/main/java/org/elasticsearch/client/support/AbstractClient.java b/server/src/main/java/org/elasticsearch/client/support/AbstractClient.java index e79f0567babe6..c3119256fc7cf 100644 --- a/server/src/main/java/org/elasticsearch/client/support/AbstractClient.java +++ b/server/src/main/java/org/elasticsearch/client/support/AbstractClient.java @@ -21,7 +21,7 @@ import org.apache.logging.log4j.LogManager; import org.apache.logging.log4j.Logger; -import org.elasticsearch.action.Action; +import org.elasticsearch.action.ActionType; import org.elasticsearch.action.ActionFuture; import org.elasticsearch.action.ActionListener; import org.elasticsearch.action.ActionRequest; @@ -134,17 +134,12 @@ import org.elasticsearch.action.admin.indices.alias.IndicesAliasesAction; import org.elasticsearch.action.admin.indices.alias.IndicesAliasesRequest; import org.elasticsearch.action.admin.indices.alias.IndicesAliasesRequestBuilder; -import org.elasticsearch.action.admin.indices.alias.exists.AliasesExistAction; -import org.elasticsearch.action.admin.indices.alias.exists.AliasesExistRequestBuilder; -import org.elasticsearch.action.admin.indices.alias.exists.AliasesExistResponse; import org.elasticsearch.action.admin.indices.alias.get.GetAliasesAction; import org.elasticsearch.action.admin.indices.alias.get.GetAliasesRequest; import org.elasticsearch.action.admin.indices.alias.get.GetAliasesRequestBuilder; import org.elasticsearch.action.admin.indices.alias.get.GetAliasesResponse; import org.elasticsearch.action.admin.indices.analyze.AnalyzeAction; -import org.elasticsearch.action.admin.indices.analyze.AnalyzeRequest; import org.elasticsearch.action.admin.indices.analyze.AnalyzeRequestBuilder; -import org.elasticsearch.action.admin.indices.analyze.AnalyzeResponse; import org.elasticsearch.action.admin.indices.cache.clear.ClearIndicesCacheAction; import org.elasticsearch.action.admin.indices.cache.clear.ClearIndicesCacheRequest; import org.elasticsearch.action.admin.indices.cache.clear.ClearIndicesCacheRequestBuilder; @@ -160,14 +155,6 @@ import org.elasticsearch.action.admin.indices.delete.DeleteIndexAction; import org.elasticsearch.action.admin.indices.delete.DeleteIndexRequest; import org.elasticsearch.action.admin.indices.delete.DeleteIndexRequestBuilder; -import org.elasticsearch.action.admin.indices.exists.indices.IndicesExistsAction; -import org.elasticsearch.action.admin.indices.exists.indices.IndicesExistsRequest; -import org.elasticsearch.action.admin.indices.exists.indices.IndicesExistsRequestBuilder; -import org.elasticsearch.action.admin.indices.exists.indices.IndicesExistsResponse; -import org.elasticsearch.action.admin.indices.exists.types.TypesExistsAction; -import org.elasticsearch.action.admin.indices.exists.types.TypesExistsRequest; -import org.elasticsearch.action.admin.indices.exists.types.TypesExistsRequestBuilder; -import org.elasticsearch.action.admin.indices.exists.types.TypesExistsResponse; import org.elasticsearch.action.admin.indices.flush.FlushAction; import org.elasticsearch.action.admin.indices.flush.FlushRequest; import org.elasticsearch.action.admin.indices.flush.FlushRequestBuilder; @@ -314,7 +301,6 @@ import org.elasticsearch.action.search.SearchScrollRequest; import org.elasticsearch.action.search.SearchScrollRequestBuilder; import org.elasticsearch.action.support.PlainActionFuture; -import org.elasticsearch.action.support.ThreadedActionListener; import org.elasticsearch.action.support.master.AcknowledgedResponse; import org.elasticsearch.action.termvectors.MultiTermVectorsAction; import org.elasticsearch.action.termvectors.MultiTermVectorsRequest; @@ -351,14 +337,12 @@ public abstract class AbstractClient implements Client { protected final Settings settings; private final ThreadPool threadPool; private final Admin admin; - private final ThreadedActionListener.Wrapper threadedWrapper; public AbstractClient(Settings settings, ThreadPool threadPool) { this.settings = settings; this.threadPool = threadPool; this.admin = new Admin(this); this.logger =LogManager.getLogger(this.getClass()); - this.threadedWrapper = new ThreadedActionListener.Wrapper(logger, settings, threadPool); } @Override @@ -378,7 +362,7 @@ public final AdminClient admin() { @Override public final ActionFuture execute( - Action action, Request request) { + ActionType action, Request request) { PlainActionFuture actionFuture = PlainActionFuture.newFuture(); execute(action, request, actionFuture); return actionFuture; @@ -389,13 +373,12 @@ public final Ac */ @Override public final void execute( - Action action, Request request, ActionListener listener) { - listener = threadedWrapper.wrap(listener); + ActionType action, Request request, ActionListener listener) { doExecute(action, request, listener); } protected abstract - void doExecute(Action action, Request request, ActionListener listener); + void doExecute(ActionType action, Request request, ActionListener listener); @Override public ActionFuture index(final IndexRequest request) { @@ -578,8 +561,8 @@ public TermVectorsRequestBuilder prepareTermVectors() { } @Override - public TermVectorsRequestBuilder prepareTermVectors(String index, String type, String id) { - return new TermVectorsRequestBuilder(this, TermVectorsAction.INSTANCE, index, type, id); + public TermVectorsRequestBuilder prepareTermVectors(String index, String id) { + return new TermVectorsRequestBuilder(this, TermVectorsAction.INSTANCE, index, id); } @Override @@ -673,13 +656,13 @@ static class ClusterAdmin implements ClusterAdminClient { @Override public ActionFuture execute( - Action action, Request request) { + ActionType action, Request request) { return client.execute(action, request); } @Override public void execute( - Action action, Request request, ActionListener listener) { + ActionType action, Request request, ActionListener listener) { client.execute(action, request, listener); } @@ -955,8 +938,8 @@ public void getSnapshots(GetSnapshotsRequest request, ActionListener ActionFuture execute( - Action action, Request request) { + ActionType action, Request request) { return client.execute(action, request); } @Override public void execute( - Action action, Request request, ActionListener listener) { + ActionType action, Request request, ActionListener listener) { client.execute(action, request, listener); } @@ -1219,37 +1202,6 @@ public ThreadPool threadPool() { return client.threadPool(); } - @Override - public ActionFuture exists(final IndicesExistsRequest request) { - return execute(IndicesExistsAction.INSTANCE, request); - } - - @Override - public void exists(final IndicesExistsRequest request, final ActionListener listener) { - execute(IndicesExistsAction.INSTANCE, request, listener); - } - - @Override - public IndicesExistsRequestBuilder prepareExists(String... indices) { - return new IndicesExistsRequestBuilder(this, IndicesExistsAction.INSTANCE, indices); - } - - @Deprecated - @Override - public ActionFuture typesExists(TypesExistsRequest request) { - return execute(TypesExistsAction.INSTANCE, request); - } - - @Override - public void typesExists(TypesExistsRequest request, ActionListener listener) { - execute(TypesExistsAction.INSTANCE, request, listener); - } - - @Override - public TypesExistsRequestBuilder prepareTypesExists(String... index) { - return new TypesExistsRequestBuilder(this, TypesExistsAction.INSTANCE, index); - } - @Override public ActionFuture aliases(final IndicesAliasesRequest request) { return execute(IndicesAliasesAction.INSTANCE, request); @@ -1285,21 +1237,6 @@ public ActionFuture clearCache(final ClearIndicesCach return execute(ClearIndicesCacheAction.INSTANCE, request); } - @Override - public void aliasesExist(GetAliasesRequest request, ActionListener listener) { - execute(AliasesExistAction.INSTANCE, request, listener); - } - - @Override - public ActionFuture aliasesExist(GetAliasesRequest request) { - return execute(AliasesExistAction.INSTANCE, request); - } - - @Override - public AliasesExistRequestBuilder prepareAliasesExist(String... aliases) { - return new AliasesExistRequestBuilder(this, AliasesExistAction.INSTANCE, aliases); - } - @Override public ActionFuture getIndex(GetIndexRequest request) { return execute(GetIndexAction.INSTANCE, request); @@ -1596,12 +1533,12 @@ public UpdateSettingsRequestBuilder prepareUpdateSettings(String... indices) { } @Override - public ActionFuture analyze(final AnalyzeRequest request) { + public ActionFuture analyze(final AnalyzeAction.Request request) { return execute(AnalyzeAction.INSTANCE, request); } @Override - public void analyze(final AnalyzeRequest request, final ActionListener listener) { + public void analyze(final AnalyzeAction.Request request, final ActionListener listener) { execute(AnalyzeAction.INSTANCE, request, listener); } @@ -1732,7 +1669,7 @@ public Client filterWithHeader(Map headers) { return new FilterClient(this) { @Override protected - void doExecute(Action action, Request request, ActionListener listener) { + void doExecute(ActionType action, Request request, ActionListener listener) { ThreadContext threadContext = threadPool().getThreadContext(); try (ThreadContext.StoredContext ctx = threadContext.stashAndMergeHeaders(headers)) { super.doExecute(action, request, listener); diff --git a/server/src/main/java/org/elasticsearch/client/transport/TransportClient.java b/server/src/main/java/org/elasticsearch/client/transport/TransportClient.java deleted file mode 100644 index 4c2f4932de2f2..0000000000000 --- a/server/src/main/java/org/elasticsearch/client/transport/TransportClient.java +++ /dev/null @@ -1,404 +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.client.transport; - -import org.elasticsearch.action.Action; -import org.elasticsearch.action.ActionListener; -import org.elasticsearch.action.ActionModule; -import org.elasticsearch.action.ActionRequest; -import org.elasticsearch.action.ActionResponse; -import org.elasticsearch.client.support.AbstractClient; -import org.elasticsearch.cluster.ClusterModule; -import org.elasticsearch.cluster.node.DiscoveryNode; -import org.elasticsearch.common.UUIDs; -import org.elasticsearch.common.breaker.CircuitBreaker; -import org.elasticsearch.common.component.LifecycleComponent; -import org.elasticsearch.common.inject.Injector; -import org.elasticsearch.common.inject.Module; -import org.elasticsearch.common.inject.ModulesBuilder; -import org.elasticsearch.common.io.stream.NamedWriteableRegistry; -import org.elasticsearch.common.network.NetworkModule; -import org.elasticsearch.common.network.NetworkService; -import org.elasticsearch.common.settings.Setting; -import org.elasticsearch.common.settings.Settings; -import org.elasticsearch.common.settings.SettingsModule; -import org.elasticsearch.common.transport.TransportAddress; -import org.elasticsearch.common.unit.TimeValue; -import org.elasticsearch.common.util.BigArrays; -import org.elasticsearch.common.util.PageCacheRecycler; -import org.elasticsearch.common.xcontent.NamedXContentRegistry; -import org.elasticsearch.core.internal.io.IOUtils; -import org.elasticsearch.indices.IndicesModule; -import org.elasticsearch.indices.breaker.CircuitBreakerService; -import org.elasticsearch.node.InternalSettingsPreparer; -import org.elasticsearch.node.Node; -import org.elasticsearch.plugins.ActionPlugin; -import org.elasticsearch.plugins.NetworkPlugin; -import org.elasticsearch.plugins.Plugin; -import org.elasticsearch.plugins.PluginsService; -import org.elasticsearch.plugins.SearchPlugin; -import org.elasticsearch.search.SearchModule; -import org.elasticsearch.threadpool.ExecutorBuilder; -import org.elasticsearch.threadpool.ThreadPool; -import org.elasticsearch.transport.Transport; -import org.elasticsearch.transport.TransportService; -import org.elasticsearch.transport.TransportSettings; - -import java.io.Closeable; -import java.util.ArrayList; -import java.util.Arrays; -import java.util.Collection; -import java.util.Collections; -import java.util.List; -import java.util.concurrent.TimeUnit; -import java.util.function.Function; -import java.util.stream.Collectors; -import java.util.stream.Stream; - -import static java.util.stream.Collectors.toList; -import static org.elasticsearch.common.unit.TimeValue.timeValueSeconds; - -/** - * The transport client allows to create a client that is not part of the cluster, but simply connects to one - * or more nodes directly by adding their respective addresses using - * {@link #addTransportAddress(org.elasticsearch.common.transport.TransportAddress)}. - *

    - * The transport client important modules used is the {@link org.elasticsearch.common.network.NetworkModule} which is - * started in client mode (only connects, no bind). - * - * @deprecated {@link TransportClient} is deprecated in favour of the High Level REST client and will - * be removed in Elasticsearch 8.0. - */ -@Deprecated -public abstract class TransportClient extends AbstractClient { - - public static final Setting CLIENT_TRANSPORT_NODES_SAMPLER_INTERVAL = - Setting.positiveTimeSetting("client.transport.nodes_sampler_interval", timeValueSeconds(5), Setting.Property.NodeScope); - public static final Setting CLIENT_TRANSPORT_PING_TIMEOUT = - Setting.positiveTimeSetting("client.transport.ping_timeout", timeValueSeconds(5), Setting.Property.NodeScope); - public static final Setting CLIENT_TRANSPORT_IGNORE_CLUSTER_NAME = - Setting.boolSetting("client.transport.ignore_cluster_name", false, Setting.Property.NodeScope); - public static final Setting CLIENT_TRANSPORT_SNIFF = - Setting.boolSetting("client.transport.sniff", false, Setting.Property.NodeScope); - - public static final String TRANSPORT_CLIENT_FEATURE = "transport_client"; - - private static PluginsService newPluginService(final Settings settings, Collection> plugins) { - final Settings.Builder settingsBuilder = Settings.builder() - .put(TransportSettings.PING_SCHEDULE.getKey(), "5s") // enable by default the transport schedule ping interval - .put(InternalSettingsPreparer.prepareSettings(settings)) - .put(NetworkService.NETWORK_SERVER.getKey(), false) - .put(CLIENT_TYPE_SETTING_S.getKey(), CLIENT_TYPE); - return new PluginsService(settingsBuilder.build(), null, null, null, plugins); - } - - protected static Collection> addPlugins(Collection> collection, - Class... plugins) { - return addPlugins(collection, Arrays.asList(plugins)); - } - - protected static Collection> addPlugins(Collection> collection, - Collection> plugins) { - ArrayList> list = new ArrayList<>(collection); - for (Class p : plugins) { - if (list.contains(p)) { - throw new IllegalArgumentException("plugin already exists: " + p); - } - list.add(p); - } - return list; - } - - private static ClientTemplate buildTemplate(Settings providedSettings, Settings defaultSettings, - Collection> plugins, HostFailureListener failureListner) { - if (Node.NODE_NAME_SETTING.exists(providedSettings) == false) { - 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()) - .put(TransportSettings.FEATURE_PREFIX + "." + TRANSPORT_CLIENT_FEATURE, true) - .build(); - final List resourcesToClose = new ArrayList<>(); - final ThreadPool threadPool = new ThreadPool(settings); - resourcesToClose.add(() -> ThreadPool.terminate(threadPool, 10, TimeUnit.SECONDS)); - final NetworkService networkService = new NetworkService(Collections.emptyList()); - try { - final List> additionalSettings = new ArrayList<>(pluginsService.getPluginSettings()); - final List additionalSettingsFilter = new ArrayList<>(pluginsService.getPluginSettingsFilter()); - for (final ExecutorBuilder builder : threadPool.builders()) { - additionalSettings.addAll(builder.getRegisteredSettings()); - } - SettingsModule settingsModule = - new SettingsModule(settings, additionalSettings, additionalSettingsFilter, Collections.emptySet()); - - SearchModule searchModule = new SearchModule(settings, true, pluginsService.filterPlugins(SearchPlugin.class)); - IndicesModule indicesModule = new IndicesModule(Collections.emptyList()); - List entries = new ArrayList<>(); - entries.addAll(NetworkModule.getNamedWriteables()); - entries.addAll(searchModule.getNamedWriteables()); - entries.addAll(indicesModule.getNamedWriteables()); - entries.addAll(ClusterModule.getNamedWriteables()); - entries.addAll(pluginsService.filterPlugins(Plugin.class).stream() - .flatMap(p -> p.getNamedWriteables().stream()) - .collect(Collectors.toList())); - NamedWriteableRegistry namedWriteableRegistry = new NamedWriteableRegistry(entries); - NamedXContentRegistry xContentRegistry = new NamedXContentRegistry(Stream.of( - searchModule.getNamedXContents().stream(), - pluginsService.filterPlugins(Plugin.class).stream() - .flatMap(p -> p.getNamedXContent().stream()) - ).flatMap(Function.identity()).collect(toList())); - - ModulesBuilder modules = new ModulesBuilder(); - // plugin modules must be added here, before others or we can get crazy injection errors... - for (Module pluginModule : pluginsService.createGuiceModules()) { - modules.add(pluginModule); - } - modules.add(b -> b.bind(ThreadPool.class).toInstance(threadPool)); - ActionModule actionModule = new ActionModule(true, settings, null, settingsModule.getIndexScopedSettings(), - settingsModule.getClusterSettings(), settingsModule.getSettingsFilter(), threadPool, - pluginsService.filterPlugins(ActionPlugin.class), null, null, null); - modules.add(actionModule); - - CircuitBreakerService circuitBreakerService = Node.createCircuitBreakerService(settingsModule.getSettings(), - settingsModule.getClusterSettings()); - resourcesToClose.add(circuitBreakerService); - PageCacheRecycler pageCacheRecycler = new PageCacheRecycler(settings); - BigArrays bigArrays = new BigArrays(pageCacheRecycler, circuitBreakerService, CircuitBreaker.REQUEST); - modules.add(settingsModule); - NetworkModule networkModule = new NetworkModule(settings, true, pluginsService.filterPlugins(NetworkPlugin.class), threadPool, - bigArrays, pageCacheRecycler, circuitBreakerService, namedWriteableRegistry, xContentRegistry, networkService, null); - final Transport transport = networkModule.getTransportSupplier().get(); - final TransportService transportService = new TransportService(settings, transport, threadPool, - networkModule.getTransportInterceptor(), - boundTransportAddress -> DiscoveryNode.createLocal(settings, new TransportAddress(TransportAddress.META_ADDRESS, 0), - UUIDs.randomBase64UUID()), null, Collections.emptySet()); - modules.add((b -> { - b.bind(BigArrays.class).toInstance(bigArrays); - b.bind(PageCacheRecycler.class).toInstance(pageCacheRecycler); - b.bind(PluginsService.class).toInstance(pluginsService); - b.bind(CircuitBreakerService.class).toInstance(circuitBreakerService); - b.bind(NamedWriteableRegistry.class).toInstance(namedWriteableRegistry); - b.bind(Transport.class).toInstance(transport); - b.bind(TransportService.class).toInstance(transportService); - b.bind(NetworkService.class).toInstance(networkService); - })); - - Injector injector = modules.createInjector(); - final TransportClientNodesService nodesService = - new TransportClientNodesService(settings, transportService, threadPool, failureListner == null - ? (t, e) -> {} : failureListner); - - // construct the list of client actions - final List actionPlugins = pluginsService.filterPlugins(ActionPlugin.class); - final List clientActions = - actionPlugins.stream().flatMap(p -> p.getClientActions().stream()).collect(Collectors.toList()); - // add all the base actions - final List> baseActions = - actionModule.getActions().values().stream().map(ActionPlugin.ActionHandler::getAction).collect(Collectors.toList()); - clientActions.addAll(baseActions); - final TransportProxyClient proxy = new TransportProxyClient(settings, transportService, nodesService, clientActions); - - List pluginLifecycleComponents = new ArrayList<>(pluginsService.getGuiceServiceClasses().stream() - .map(injector::getInstance).collect(Collectors.toList())); - resourcesToClose.addAll(pluginLifecycleComponents); - - transportService.start(); - transportService.acceptIncomingRequests(); - - ClientTemplate transportClient = new ClientTemplate(injector, pluginLifecycleComponents, nodesService, proxy, - namedWriteableRegistry); - resourcesToClose.clear(); - return transportClient; - } finally { - IOUtils.closeWhileHandlingException(resourcesToClose); - } - } - - private static final class ClientTemplate { - final Injector injector; - private final List pluginLifecycleComponents; - private final TransportClientNodesService nodesService; - private final TransportProxyClient proxy; - private final NamedWriteableRegistry namedWriteableRegistry; - - private ClientTemplate(Injector injector, List pluginLifecycleComponents, - TransportClientNodesService nodesService, TransportProxyClient proxy, NamedWriteableRegistry namedWriteableRegistry) { - this.injector = injector; - this.pluginLifecycleComponents = pluginLifecycleComponents; - this.nodesService = nodesService; - this.proxy = proxy; - this.namedWriteableRegistry = namedWriteableRegistry; - } - - Settings getSettings() { - return injector.getInstance(Settings.class); - } - - ThreadPool getThreadPool() { - return injector.getInstance(ThreadPool.class); - } - } - - public static final String CLIENT_TYPE = "transport"; - - final Injector injector; - protected final NamedWriteableRegistry namedWriteableRegistry; - - private final List pluginLifecycleComponents; - private final TransportClientNodesService nodesService; - private final TransportProxyClient proxy; - - /** - * Creates a new TransportClient with the given settings and plugins - */ - public TransportClient(Settings settings, Collection> plugins) { - this(buildTemplate(settings, Settings.EMPTY, plugins, null)); - } - - /** - * Creates a new TransportClient with the given settings, defaults and plugins. - * @param settings the client settings - * @param defaultSettings default settings that are merged after the plugins have added it's additional settings. - * @param plugins the client plugins - */ - protected TransportClient(Settings settings, Settings defaultSettings, Collection> plugins, - HostFailureListener hostFailureListener) { - this(buildTemplate(settings, defaultSettings, plugins, hostFailureListener)); - } - - private TransportClient(ClientTemplate template) { - super(template.getSettings(), template.getThreadPool()); - this.injector = template.injector; - this.pluginLifecycleComponents = Collections.unmodifiableList(template.pluginLifecycleComponents); - this.nodesService = template.nodesService; - this.proxy = template.proxy; - this.namedWriteableRegistry = template.namedWriteableRegistry; - } - - /** - * Returns the current registered transport addresses to use (added using - * {@link #addTransportAddress(org.elasticsearch.common.transport.TransportAddress)}. - */ - public List transportAddresses() { - return nodesService.transportAddresses(); - } - - /** - * Returns the current connected transport nodes that this client will use. - *

    - * The nodes include all the nodes that are currently alive based on the transport - * addresses provided. - */ - public List connectedNodes() { - return nodesService.connectedNodes(); - } - - /** - * The list of filtered nodes that were not connected to, for example, due to - * mismatch in cluster name. - */ - public List filteredNodes() { - return nodesService.filteredNodes(); - } - - /** - * Returns the listed nodes in the transport client (ones added to it). - */ - public List listedNodes() { - return nodesService.listedNodes(); - } - - /** - * Adds a transport address that will be used to connect to. - *

    - * The Node this transport address represents will be used if its possible to connect to it. - * If it is unavailable, it will be automatically connected to once it is up. - *

    - * In order to get the list of all the current connected nodes, please see {@link #connectedNodes()}. - */ - public TransportClient addTransportAddress(TransportAddress transportAddress) { - nodesService.addTransportAddresses(transportAddress); - return this; - } - - /** - * Adds a list of transport addresses that will be used to connect to. - *

    - * The Node this transport address represents will be used if its possible to connect to it. - * If it is unavailable, it will be automatically connected to once it is up. - *

    - * In order to get the list of all the current connected nodes, please see {@link #connectedNodes()}. - */ - public TransportClient addTransportAddresses(TransportAddress... transportAddress) { - nodesService.addTransportAddresses(transportAddress); - return this; - } - - /** - * Removes a transport address from the list of transport addresses that are used to connect to. - */ - public TransportClient removeTransportAddress(TransportAddress transportAddress) { - nodesService.removeTransportAddress(transportAddress); - return this; - } - - /** - * Closes the client. - */ - @Override - public void close() { - List closeables = new ArrayList<>(); - closeables.add(nodesService); - closeables.add(injector.getInstance(TransportService.class)); - - for (LifecycleComponent plugin : pluginLifecycleComponents) { - closeables.add(plugin); - } - closeables.add(() -> ThreadPool.terminate(injector.getInstance(ThreadPool.class), 10, TimeUnit.SECONDS)); - IOUtils.closeWhileHandlingException(closeables); - } - - @Override - protected - void doExecute(Action action, Request request, ActionListener listener) { - proxy.execute(action, request, listener); - } - - /** - * Listener that allows to be notified whenever a node failure / disconnect happens - */ - @FunctionalInterface - public interface HostFailureListener { - /** - * Called once a node disconnect is detected. - * @param node the node that has been disconnected - * @param ex the exception causing the disconnection - */ - void onNodeDisconnected(DiscoveryNode node, Exception ex); - } - - // pkg private for testing - TransportClientNodesService getNodesService() { - return nodesService; - } -} diff --git a/server/src/main/java/org/elasticsearch/client/transport/TransportClientNodesService.java b/server/src/main/java/org/elasticsearch/client/transport/TransportClientNodesService.java deleted file mode 100644 index 9d28ea0e5bc16..0000000000000 --- a/server/src/main/java/org/elasticsearch/client/transport/TransportClientNodesService.java +++ /dev/null @@ -1,582 +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.client.transport; - -import com.carrotsearch.hppc.cursors.ObjectCursor; -import org.apache.logging.log4j.LogManager; -import org.apache.logging.log4j.Logger; -import org.apache.logging.log4j.message.ParameterizedMessage; -import org.elasticsearch.ExceptionsHelper; -import org.elasticsearch.Version; -import org.elasticsearch.action.ActionListener; -import org.elasticsearch.action.admin.cluster.node.liveness.LivenessRequest; -import org.elasticsearch.action.admin.cluster.node.liveness.LivenessResponse; -import org.elasticsearch.action.admin.cluster.node.liveness.TransportLivenessAction; -import org.elasticsearch.action.admin.cluster.state.ClusterStateAction; -import org.elasticsearch.action.admin.cluster.state.ClusterStateResponse; -import org.elasticsearch.client.Requests; -import org.elasticsearch.cluster.ClusterName; -import org.elasticsearch.cluster.node.DiscoveryNode; -import org.elasticsearch.common.Randomness; -import org.elasticsearch.common.io.stream.StreamInput; -import org.elasticsearch.common.settings.Settings; -import org.elasticsearch.common.transport.TransportAddress; -import org.elasticsearch.common.unit.TimeValue; -import org.elasticsearch.common.util.concurrent.AbstractRunnable; -import org.elasticsearch.common.util.concurrent.ConcurrentCollections; -import org.elasticsearch.core.internal.io.IOUtils; -import org.elasticsearch.threadpool.Scheduler; -import org.elasticsearch.threadpool.ThreadPool; -import org.elasticsearch.transport.ConnectTransportException; -import org.elasticsearch.transport.ConnectionProfile; -import org.elasticsearch.transport.FutureTransportResponseHandler; -import org.elasticsearch.transport.NodeDisconnectedException; -import org.elasticsearch.transport.NodeNotConnectedException; -import org.elasticsearch.transport.PlainTransportFuture; -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.io.Closeable; -import java.io.IOException; -import java.util.ArrayList; -import java.util.Collections; -import java.util.HashSet; -import java.util.Iterator; -import java.util.List; -import java.util.Locale; -import java.util.Map; -import java.util.Set; -import java.util.concurrent.ConcurrentMap; -import java.util.concurrent.CountDownLatch; -import java.util.concurrent.atomic.AtomicInteger; - -final class TransportClientNodesService implements Closeable { - - private static final Logger logger = LogManager.getLogger(TransportClientNodesService.class); - - private final TimeValue nodesSamplerInterval; - - private final long pingTimeout; - - private final ClusterName clusterName; - - private final TransportService transportService; - - private final ThreadPool threadPool; - - private final Version minCompatibilityVersion; - - // nodes that are added to be discovered - private volatile List listedNodes = Collections.emptyList(); - - private final Object mutex = new Object(); - - private volatile List nodes = Collections.emptyList(); - // Filtered nodes are nodes whose cluster name does not match the configured cluster name - private volatile List filteredNodes = Collections.emptyList(); - - private final AtomicInteger tempNodeIdGenerator = new AtomicInteger(); - - private final NodeSampler nodesSampler; - - private volatile Scheduler.Cancellable nodesSamplerCancellable; - - private final AtomicInteger randomNodeGenerator = new AtomicInteger(Randomness.get().nextInt()); - - private final boolean ignoreClusterName; - - private volatile boolean closed; - - private final TransportClient.HostFailureListener hostFailureListener; - - // TODO: migrate this to use low level connections and single type channels - /** {@link ConnectionProfile} to use when to connecting to the listed nodes and doing a liveness check */ - private static final ConnectionProfile LISTED_NODES_PROFILE; - - static { - ConnectionProfile.Builder builder = new ConnectionProfile.Builder(); - builder.addConnections(1, - TransportRequestOptions.Type.BULK, - TransportRequestOptions.Type.PING, - TransportRequestOptions.Type.RECOVERY, - TransportRequestOptions.Type.REG, - TransportRequestOptions.Type.STATE); - LISTED_NODES_PROFILE = builder.build(); - } - - TransportClientNodesService(Settings settings, TransportService transportService, - ThreadPool threadPool, TransportClient.HostFailureListener hostFailureListener) { - this.clusterName = ClusterName.CLUSTER_NAME_SETTING.get(settings); - this.transportService = transportService; - this.threadPool = threadPool; - this.minCompatibilityVersion = Version.CURRENT.minimumCompatibilityVersion(); - - this.nodesSamplerInterval = TransportClient.CLIENT_TRANSPORT_NODES_SAMPLER_INTERVAL.get(settings); - this.pingTimeout = TransportClient.CLIENT_TRANSPORT_PING_TIMEOUT.get(settings).millis(); - this.ignoreClusterName = TransportClient.CLIENT_TRANSPORT_IGNORE_CLUSTER_NAME.get(settings); - - if (logger.isDebugEnabled()) { - logger.debug("node_sampler_interval[{}]", nodesSamplerInterval); - } - - if (TransportClient.CLIENT_TRANSPORT_SNIFF.get(settings)) { - this.nodesSampler = new SniffNodesSampler(); - } else { - this.nodesSampler = new SimpleNodeSampler(); - } - this.hostFailureListener = hostFailureListener; - this.nodesSamplerCancellable = threadPool.schedule(new ScheduledNodeSampler(), nodesSamplerInterval, ThreadPool.Names.GENERIC); - } - - public List transportAddresses() { - List lstBuilder = new ArrayList<>(); - for (DiscoveryNode listedNode : listedNodes) { - lstBuilder.add(listedNode.getAddress()); - } - return Collections.unmodifiableList(lstBuilder); - } - - public List connectedNodes() { - return this.nodes; - } - - public List filteredNodes() { - return this.filteredNodes; - } - - public List listedNodes() { - return this.listedNodes; - } - - public TransportClientNodesService addTransportAddresses(TransportAddress... transportAddresses) { - synchronized (mutex) { - if (closed) { - throw new IllegalStateException("transport client is closed, can't add an address"); - } - List filtered = new ArrayList<>(transportAddresses.length); - for (TransportAddress transportAddress : transportAddresses) { - boolean found = false; - for (DiscoveryNode otherNode : listedNodes) { - if (otherNode.getAddress().equals(transportAddress)) { - found = true; - logger.debug("address [{}] already exists with [{}], ignoring...", transportAddress, otherNode); - break; - } - } - if (!found) { - filtered.add(transportAddress); - } - } - if (filtered.isEmpty()) { - return this; - } - List builder = new ArrayList<>(listedNodes); - for (TransportAddress transportAddress : filtered) { - DiscoveryNode node = new DiscoveryNode("#transport#-" + tempNodeIdGenerator.incrementAndGet(), - transportAddress, Collections.emptyMap(), Collections.emptySet(), minCompatibilityVersion); - logger.debug("adding address [{}]", node); - builder.add(node); - } - listedNodes = Collections.unmodifiableList(builder); - nodesSampler.sample(); - } - return this; - } - - public TransportClientNodesService removeTransportAddress(TransportAddress transportAddress) { - synchronized (mutex) { - if (closed) { - throw new IllegalStateException("transport client is closed, can't remove an address"); - } - List listNodesBuilder = new ArrayList<>(); - for (DiscoveryNode otherNode : listedNodes) { - if (!otherNode.getAddress().equals(transportAddress)) { - listNodesBuilder.add(otherNode); - } else { - logger.debug("removing address [{}] from listed nodes", otherNode); - } - } - listedNodes = Collections.unmodifiableList(listNodesBuilder); - List nodesBuilder = new ArrayList<>(); - for (DiscoveryNode otherNode : nodes) { - if (!otherNode.getAddress().equals(transportAddress)) { - nodesBuilder.add(otherNode); - } else { - logger.debug("disconnecting from node with address [{}]", otherNode); - transportService.disconnectFromNode(otherNode); - } - } - nodes = Collections.unmodifiableList(nodesBuilder); - nodesSampler.sample(); - } - return this; - } - - public void execute(NodeListenerCallback callback, ActionListener listener) { - // we first read nodes before checking the closed state; this - // is because otherwise we could be subject to a race where we - // read the state as not being closed, and then the client is - // closed and the nodes list is cleared, and then a - // NoNodeAvailableException is thrown - // it is important that the order of first setting the state of - // closed and then clearing the list of nodes is maintained in - // the close method - final List nodes = this.nodes; - if (closed) { - throw new IllegalStateException("transport client is closed"); - } - ensureNodesAreAvailable(nodes); - int index = getNodeNumber(); - RetryListener retryListener = new RetryListener<>(callback, listener, nodes, index, hostFailureListener); - DiscoveryNode node = retryListener.getNode(0); - try { - callback.doWithNode(node, retryListener); - } catch (Exception e) { - try { - //this exception can't come from the TransportService as it doesn't throw exception at all - listener.onFailure(e); - } finally { - retryListener.maybeNodeFailed(node, e); - } - } - } - - public static class RetryListener implements ActionListener { - private final NodeListenerCallback callback; - private final ActionListener listener; - private final List nodes; - private final int index; - private final TransportClient.HostFailureListener hostFailureListener; - - private volatile int i; - - RetryListener(NodeListenerCallback callback, ActionListener listener, - List nodes, int index, TransportClient.HostFailureListener hostFailureListener) { - this.callback = callback; - this.listener = listener; - this.nodes = nodes; - this.index = index; - this.hostFailureListener = hostFailureListener; - } - - @Override - public void onResponse(Response response) { - listener.onResponse(response); - } - - @Override - public void onFailure(Exception e) { - Throwable throwable = ExceptionsHelper.unwrapCause(e); - if (throwable instanceof ConnectTransportException) { - maybeNodeFailed(getNode(this.i), (ConnectTransportException) throwable); - int i = ++this.i; - if (i >= nodes.size()) { - listener.onFailure(new NoNodeAvailableException("None of the configured nodes were available: " + nodes, e)); - } else { - try { - callback.doWithNode(getNode(i), this); - } catch(final Exception inner) { - inner.addSuppressed(e); - // this exception can't come from the TransportService as it doesn't throw exceptions at all - listener.onFailure(inner); - } - } - } else { - listener.onFailure(e); - } - } - - final DiscoveryNode getNode(int i) { - return nodes.get((index + i) % nodes.size()); - } - - final void maybeNodeFailed(DiscoveryNode node, Exception ex) { - if (ex instanceof NodeDisconnectedException || ex instanceof NodeNotConnectedException) { - hostFailureListener.onNodeDisconnected(node, ex); - } - } - } - - @Override - public void close() { - synchronized (mutex) { - if (closed) { - return; - } - closed = true; - if (nodesSamplerCancellable != null) { - nodesSamplerCancellable.cancel(); - } - for (DiscoveryNode node : nodes) { - transportService.disconnectFromNode(node); - } - for (DiscoveryNode listedNode : listedNodes) { - transportService.disconnectFromNode(listedNode); - } - nodes = Collections.emptyList(); - } - } - - private int getNodeNumber() { - int index = randomNodeGenerator.incrementAndGet(); - if (index < 0) { - index = 0; - randomNodeGenerator.set(0); - } - return index; - } - - private void ensureNodesAreAvailable(List nodes) { - if (nodes.isEmpty()) { - String message = String.format(Locale.ROOT, "None of the configured nodes are available: %s", this.listedNodes); - throw new NoNodeAvailableException(message); - } - } - - abstract class NodeSampler { - public void sample() { - synchronized (mutex) { - if (closed) { - return; - } - doSample(); - } - } - - protected abstract void doSample(); - - /** - * Establishes the node connections. If validateInHandshake is set to true, the connection will fail if - * node returned in the handshake response is different than the discovery node. - */ - List establishNodeConnections(Set nodes) { - for (Iterator it = nodes.iterator(); it.hasNext(); ) { - DiscoveryNode node = it.next(); - if (!transportService.nodeConnected(node)) { - try { - logger.trace("connecting to node [{}]", node); - transportService.connectToNode(node); - } catch (Exception e) { - it.remove(); - logger.debug(() -> new ParameterizedMessage("failed to connect to discovered node [{}]", node), e); - } - } - } - - return List.copyOf(nodes); - } - } - - class ScheduledNodeSampler implements Runnable { - @Override - public void run() { - try { - nodesSampler.sample(); - if (!closed) { - nodesSamplerCancellable = threadPool.schedule(this, nodesSamplerInterval, ThreadPool.Names.GENERIC); - } - } catch (Exception e) { - logger.warn("failed to sample", e); - } - } - } - - class SimpleNodeSampler extends NodeSampler { - - @Override - protected void doSample() { - HashSet newNodes = new HashSet<>(); - ArrayList newFilteredNodes = new ArrayList<>(); - for (DiscoveryNode listedNode : listedNodes) { - try (Transport.Connection connection = transportService.openConnection(listedNode, LISTED_NODES_PROFILE)){ - final PlainTransportFuture handler = new PlainTransportFuture<>( - new FutureTransportResponseHandler() { - @Override - public LivenessResponse read(StreamInput in) throws IOException { - LivenessResponse response = new LivenessResponse(); - response.readFrom(in); - return response; - } - }); - transportService.sendRequest(connection, TransportLivenessAction.NAME, new LivenessRequest(), - TransportRequestOptions.builder().withType(TransportRequestOptions.Type.STATE).withTimeout(pingTimeout).build(), - handler); - final LivenessResponse livenessResponse = handler.txGet(); - if (!ignoreClusterName && !clusterName.equals(livenessResponse.getClusterName())) { - logger.warn("node {} not part of the cluster {}, ignoring...", listedNode, clusterName); - newFilteredNodes.add(listedNode); - } else { - // use discovered information but do keep the original transport address, - // so people can control which address is exactly used. - DiscoveryNode nodeWithInfo = livenessResponse.getDiscoveryNode(); - newNodes.add(new DiscoveryNode(nodeWithInfo.getName(), nodeWithInfo.getId(), nodeWithInfo.getEphemeralId(), - nodeWithInfo.getHostName(), nodeWithInfo.getHostAddress(), listedNode.getAddress(), - nodeWithInfo.getAttributes(), nodeWithInfo.getRoles(), nodeWithInfo.getVersion())); - } - } catch (ConnectTransportException e) { - logger.debug(() -> new ParameterizedMessage("failed to connect to node [{}], ignoring...", listedNode), e); - hostFailureListener.onNodeDisconnected(listedNode, e); - } catch (Exception e) { - logger.info(() -> new ParameterizedMessage("failed to get node info for {}, disconnecting...", listedNode), e); - } - } - - nodes = establishNodeConnections(newNodes); - filteredNodes = Collections.unmodifiableList(newFilteredNodes); - } - } - - class SniffNodesSampler extends NodeSampler { - - @Override - protected void doSample() { - // the nodes we are going to ping include the core listed nodes that were added - // and the last round of discovered nodes - Set nodesToPing = new HashSet<>(); - for (DiscoveryNode node : listedNodes) { - nodesToPing.add(node); - } - for (DiscoveryNode node : nodes) { - nodesToPing.add(node); - } - - final CountDownLatch latch = new CountDownLatch(nodesToPing.size()); - final ConcurrentMap clusterStateResponses = ConcurrentCollections.newConcurrentMap(); - try { - for (final DiscoveryNode nodeToPing : nodesToPing) { - threadPool.executor(ThreadPool.Names.MANAGEMENT).execute(new AbstractRunnable() { - - /** - * we try to reuse existing connections but if needed we will open a temporary connection - * that will be closed at the end of the execution. - */ - Transport.Connection connectionToClose = null; - - void onDone() { - try { - IOUtils.closeWhileHandlingException(connectionToClose); - } finally { - latch.countDown(); - } - } - - @Override - public void onFailure(Exception e) { - onDone(); - if (e instanceof ConnectTransportException) { - logger.debug(() -> new ParameterizedMessage("failed to connect to node [{}], ignoring...", nodeToPing), e); - hostFailureListener.onNodeDisconnected(nodeToPing, e); - } else { - logger.info(() -> new ParameterizedMessage( - "failed to get local cluster state info for {}, disconnecting...", nodeToPing), e); - } - } - - @Override - protected void doRun() throws Exception { - Transport.Connection pingConnection = null; - if (nodes.contains(nodeToPing)) { - try { - pingConnection = transportService.getConnection(nodeToPing); - } catch (NodeNotConnectedException e) { - // will use a temp connection - } - } - if (pingConnection == null) { - logger.trace("connecting to cluster node [{}]", nodeToPing); - connectionToClose = transportService.openConnection(nodeToPing, LISTED_NODES_PROFILE); - pingConnection = connectionToClose; - } - transportService.sendRequest(pingConnection, ClusterStateAction.NAME, - Requests.clusterStateRequest().clear().nodes(true).local(true), - TransportRequestOptions.builder().withType(TransportRequestOptions.Type.STATE) - .withTimeout(pingTimeout).build(), - new TransportResponseHandler() { - - @Override - public ClusterStateResponse read(StreamInput in) throws IOException { - final ClusterStateResponse clusterStateResponse = new ClusterStateResponse(); - clusterStateResponse.readFrom(in); - return clusterStateResponse; - } - - @Override - public String executor() { - return ThreadPool.Names.SAME; - } - - @Override - public void handleResponse(ClusterStateResponse response) { - clusterStateResponses.put(nodeToPing, response); - onDone(); - } - - @Override - public void handleException(TransportException e) { - logger.info(() -> new ParameterizedMessage( - "failed to get local cluster state for {}, disconnecting...", nodeToPing), e); - try { - hostFailureListener.onNodeDisconnected(nodeToPing, e); - } finally { - onDone(); - } - } - }); - } - }); - } - latch.await(); - } catch (InterruptedException e) { - Thread.currentThread().interrupt(); - return; - } - - HashSet newNodes = new HashSet<>(); - HashSet newFilteredNodes = new HashSet<>(); - for (Map.Entry entry : clusterStateResponses.entrySet()) { - if (!ignoreClusterName && !clusterName.equals(entry.getValue().getClusterName())) { - logger.warn("node {} not part of the cluster {}, ignoring...", - entry.getValue().getState().nodes().getLocalNode(), clusterName); - newFilteredNodes.add(entry.getKey()); - continue; - } - for (ObjectCursor cursor : entry.getValue().getState().nodes().getDataNodes().values()) { - newNodes.add(cursor.value); - } - } - - nodes = establishNodeConnections(newNodes); - filteredNodes = List.copyOf(newFilteredNodes); - } - } - - public interface NodeListenerCallback { - - void doWithNode(DiscoveryNode node, ActionListener listener); - } - - // pkg private for testing - void doSample() { - nodesSampler.doSample(); - } -} diff --git a/server/src/main/java/org/elasticsearch/client/transport/TransportProxyClient.java b/server/src/main/java/org/elasticsearch/client/transport/TransportProxyClient.java deleted file mode 100644 index d79e2a9119e6a..0000000000000 --- a/server/src/main/java/org/elasticsearch/client/transport/TransportProxyClient.java +++ /dev/null @@ -1,59 +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.client.transport; - -import org.elasticsearch.action.Action; -import org.elasticsearch.action.ActionListener; -import org.elasticsearch.action.ActionRequest; -import org.elasticsearch.action.ActionRequestBuilder; -import org.elasticsearch.action.ActionResponse; -import org.elasticsearch.action.TransportActionNodeProxy; -import org.elasticsearch.common.settings.Settings; -import org.elasticsearch.transport.TransportService; - -import java.util.HashMap; -import java.util.List; -import java.util.Map; - -import static java.util.Collections.unmodifiableMap; - -final class TransportProxyClient { - - private final TransportClientNodesService nodesService; - private final Map proxies; - - TransportProxyClient(Settings settings, TransportService transportService, - TransportClientNodesService nodesService, List actions) { - this.nodesService = nodesService; - Map proxies = new HashMap<>(); - for (Action action : actions) { - proxies.put(action, new TransportActionNodeProxy(settings, action, transportService)); - } - this.proxies = unmodifiableMap(proxies); - } - - public > void execute(final Action action, - final Request request, ActionListener listener) { - final TransportActionNodeProxy proxy = proxies.get(action); - assert proxy != null : "no proxy found for action: " + action; - nodesService.execute((n, l) -> proxy.execute(n, request, l), listener); - } -} diff --git a/server/src/main/java/org/elasticsearch/cluster/ClusterModule.java b/server/src/main/java/org/elasticsearch/cluster/ClusterModule.java index d3739fd6cdcba..d0448e2be22c9 100644 --- a/server/src/main/java/org/elasticsearch/cluster/ClusterModule.java +++ b/server/src/main/java/org/elasticsearch/cluster/ClusterModule.java @@ -33,7 +33,6 @@ import org.elasticsearch.cluster.metadata.MetaDataUpdateSettingsService; import org.elasticsearch.cluster.metadata.RepositoriesMetaData; import org.elasticsearch.cluster.routing.DelayedAllocationService; -import org.elasticsearch.cluster.routing.RoutingService; import org.elasticsearch.cluster.routing.allocation.AllocationService; import org.elasticsearch.cluster.routing.allocation.allocator.BalancedShardsAllocator; import org.elasticsearch.cluster.routing.allocation.allocator.ShardsAllocator; @@ -65,7 +64,6 @@ import org.elasticsearch.common.settings.Setting; import org.elasticsearch.common.settings.Setting.Property; import org.elasticsearch.common.settings.Settings; -import org.elasticsearch.common.util.set.Sets; import org.elasticsearch.common.xcontent.NamedXContentRegistry; import org.elasticsearch.gateway.GatewayAllocator; import org.elasticsearch.ingest.IngestMetadata; @@ -78,13 +76,11 @@ import java.util.ArrayList; import java.util.Collection; -import java.util.Collections; import java.util.HashMap; import java.util.LinkedHashMap; import java.util.List; import java.util.Map; import java.util.Objects; -import java.util.Set; import java.util.function.Function; import java.util.function.Supplier; @@ -134,35 +130,6 @@ public static List getNamedWriteables() { return entries; } - static final Set PRE_6_3_METADATA_CUSTOMS_WHITE_LIST = Collections.unmodifiableSet(Sets.newHashSet( - IndexGraveyard.TYPE, IngestMetadata.TYPE, RepositoriesMetaData.TYPE, ScriptMetaData.TYPE)); - - static final Set PRE_6_3_CLUSTER_CUSTOMS_WHITE_LIST = Collections.unmodifiableSet(Sets.newHashSet( - RestoreInProgress.TYPE, SnapshotDeletionsInProgress.TYPE, SnapshotsInProgress.TYPE)); - - /** - * For interoperability with transport clients older than 6.3, we need to strip customs - * from the cluster state that the client might not be able to deserialize - * - * @param clusterState the cluster state to filter the customs from - * @return the adapted cluster state - */ - public static ClusterState filterCustomsForPre63Clients(ClusterState clusterState) { - final ClusterState.Builder builder = ClusterState.builder(clusterState); - clusterState.customs().keysIt().forEachRemaining(name -> { - if (PRE_6_3_CLUSTER_CUSTOMS_WHITE_LIST.contains(name) == false) { - builder.removeCustom(name); - } - }); - final MetaData.Builder metaBuilder = MetaData.builder(clusterState.metaData()); - clusterState.metaData().customs().keysIt().forEachRemaining(name -> { - if (PRE_6_3_METADATA_CUSTOMS_WHITE_LIST.contains(name) == false) { - metaBuilder.removeCustom(name); - } - }); - return builder.metaData(metaBuilder).build(); - } - public static List getNamedXWriteables() { List entries = new ArrayList<>(); // Metadata @@ -274,7 +241,6 @@ protected void configure() { bind(MetaDataUpdateSettingsService.class).asEagerSingleton(); bind(MetaDataIndexTemplateService.class).asEagerSingleton(); bind(IndexNameExpressionResolver.class).toInstance(indexNameExpressionResolver); - bind(RoutingService.class).asEagerSingleton(); bind(DelayedAllocationService.class).asEagerSingleton(); bind(ShardStateAction.class).asEagerSingleton(); bind(NodeMappingRefreshAction.class).asEagerSingleton(); diff --git a/server/src/main/java/org/elasticsearch/cluster/ClusterState.java b/server/src/main/java/org/elasticsearch/cluster/ClusterState.java index 6a5e2a324965f..51a1dc3835e9c 100644 --- a/server/src/main/java/org/elasticsearch/cluster/ClusterState.java +++ b/server/src/main/java/org/elasticsearch/cluster/ClusterState.java @@ -23,7 +23,6 @@ import com.carrotsearch.hppc.cursors.ObjectCursor; import com.carrotsearch.hppc.cursors.ObjectObjectCursor; import org.elasticsearch.Version; -import org.elasticsearch.client.transport.TransportClient; import org.elasticsearch.cluster.block.ClusterBlock; import org.elasticsearch.cluster.block.ClusterBlocks; import org.elasticsearch.cluster.coordination.CoordinationMetaData; @@ -114,12 +113,9 @@ default Optional getRequiredFeature() { * Tests whether or not the custom should be serialized. The criteria are: *

      *
    • the output stream must be at least the minimum supported version of the custom
    • - *
    • the output stream must have the feature required by the custom (if any) or not be a transport client
    • *
    *

    - * That is, we only serialize customs to clients than can understand the custom based on the version of the client and the features - * that the client has. For transport clients we can be lenient in requiring a feature in which case we do not send the custom but - * for connected nodes we always require that the node has the required feature. + * That is, we only serialize customs to clients than can understand the custom based on the version of the client. * * @param out the output stream * @param custom the custom to serialize @@ -127,15 +123,7 @@ default Optional getRequiredFeature() { * @return true if the custom should be serialized and false otherwise */ static boolean shouldSerialize(final StreamOutput out, final T custom) { - if (out.getVersion().before(custom.getMinimalSupportedVersion())) { - return false; - } - if (custom.getRequiredFeature().isPresent()) { - final String requiredFeature = custom.getRequiredFeature().get(); - // if it is a transport client we are lenient yet for a connected node it must have the required feature - return out.hasFeature(requiredFeature) || out.hasFeature(TransportClient.TRANSPORT_CLIENT_FEATURE) == false; - } - return true; + return out.getVersion().onOrAfter(custom.getMinimalSupportedVersion()); } } @@ -297,7 +285,8 @@ public RoutingNodes getRoutingNodes() { public String toString() { StringBuilder sb = new StringBuilder(); final String TAB = " "; - sb.append("cluster uuid: ").append(metaData.clusterUUID()).append("\n"); + sb.append("cluster uuid: ").append(metaData.clusterUUID()) + .append(" [committed: ").append(metaData.clusterUUIDCommitted()).append("]").append("\n"); sb.append("version: ").append(version).append("\n"); sb.append("state uuid: ").append(stateUUID).append("\n"); sb.append("from_diff: ").append(wasReadFromDiff).append("\n"); diff --git a/server/src/main/java/org/elasticsearch/cluster/InternalClusterInfoService.java b/server/src/main/java/org/elasticsearch/cluster/InternalClusterInfoService.java index 8d78f9c838e38..4b8936198911e 100644 --- a/server/src/main/java/org/elasticsearch/cluster/InternalClusterInfoService.java +++ b/server/src/main/java/org/elasticsearch/cluster/InternalClusterInfoService.java @@ -131,13 +131,13 @@ public void onMaster() { logger.trace("I have been elected master, scheduling a ClusterInfoUpdateJob"); } - // Submit a job that will start after DEFAULT_STARTING_INTERVAL, and reschedule itself after running + // Submit a job that will reschedule itself after running threadPool.scheduleUnlessShuttingDown(updateFrequency, executorName(), new SubmitReschedulingClusterInfoUpdatedJob()); try { if (clusterService.state().getNodes().getDataNodes().size() > 1) { // Submit an info update job to be run immediately - threadPool.executor(executorName()).execute(() -> maybeRefresh()); + threadPool.executor(executorName()).execute(this::maybeRefresh); } } catch (EsRejectedExecutionException ex) { logger.debug("Couldn't schedule cluster info update task - node might be shutting down", ex); @@ -173,7 +173,7 @@ public void clusterChanged(ClusterChangedEvent event) { if (logger.isDebugEnabled()) { logger.debug("data node was added, retrieving new cluster info"); } - threadPool.executor(executorName()).execute(() -> maybeRefresh()); + threadPool.executor(executorName()).execute(this::maybeRefresh); } if (this.isMaster && event.nodesRemoved()) { @@ -316,7 +316,7 @@ public void onResponse(IndicesStatsResponse indicesStatsResponse) { ShardStats[] stats = indicesStatsResponse.getShards(); ImmutableOpenMap.Builder newShardSizes = ImmutableOpenMap.builder(); ImmutableOpenMap.Builder newShardRoutingToDataPath = ImmutableOpenMap.builder(); - buildShardLevelInfo(logger, stats, newShardSizes, newShardRoutingToDataPath, clusterService.state()); + buildShardLevelInfo(logger, stats, newShardSizes, newShardRoutingToDataPath); shardSizes = newShardSizes.build(); shardRoutingToDataPath = newShardRoutingToDataPath.build(); } @@ -365,7 +365,7 @@ public void onFailure(Exception e) { } static void buildShardLevelInfo(Logger logger, ShardStats[] stats, ImmutableOpenMap.Builder newShardSizes, - ImmutableOpenMap.Builder newShardRoutingToDataPath, ClusterState state) { + ImmutableOpenMap.Builder newShardRoutingToDataPath) { for (ShardStats s : stats) { newShardRoutingToDataPath.put(s.getShardRouting(), s.getDataPath()); long size = s.getStats().getStore().sizeInBytes(); diff --git a/server/src/main/java/org/elasticsearch/cluster/SnapshotsInProgress.java b/server/src/main/java/org/elasticsearch/cluster/SnapshotsInProgress.java index ae9506706e36a..039c7ec447767 100644 --- a/server/src/main/java/org/elasticsearch/cluster/SnapshotsInProgress.java +++ b/server/src/main/java/org/elasticsearch/cluster/SnapshotsInProgress.java @@ -44,6 +44,8 @@ import java.util.Map; import java.util.Objects; +import static org.elasticsearch.snapshots.SnapshotInfo.METADATA_FIELD_INTRODUCED; + /** * Meta data about snapshots that are currently executing */ @@ -84,11 +86,12 @@ public static class Entry { private final ImmutableOpenMap> waitingIndices; private final long startTime; private final long repositoryStateId; + @Nullable private final Map userMetadata; @Nullable private final String failure; public Entry(Snapshot snapshot, boolean includeGlobalState, boolean partial, State state, List indices, long startTime, long repositoryStateId, ImmutableOpenMap shards, - String failure) { + String failure, Map userMetadata) { this.state = state; this.snapshot = snapshot; this.includeGlobalState = includeGlobalState; @@ -104,21 +107,23 @@ public Entry(Snapshot snapshot, boolean includeGlobalState, boolean partial, Sta } this.repositoryStateId = repositoryStateId; this.failure = failure; + this.userMetadata = userMetadata; } public Entry(Snapshot snapshot, boolean includeGlobalState, boolean partial, State state, List indices, - long startTime, long repositoryStateId, ImmutableOpenMap shards) { - this(snapshot, includeGlobalState, partial, state, indices, startTime, repositoryStateId, shards, null); + long startTime, long repositoryStateId, ImmutableOpenMap shards, + Map userMetadata) { + this(snapshot, includeGlobalState, partial, state, indices, startTime, repositoryStateId, shards, null, userMetadata); } public Entry(Entry entry, State state, ImmutableOpenMap shards) { this(entry.snapshot, entry.includeGlobalState, entry.partial, state, entry.indices, entry.startTime, - entry.repositoryStateId, shards, entry.failure); + entry.repositoryStateId, shards, entry.failure, entry.userMetadata); } public Entry(Entry entry, State state, ImmutableOpenMap shards, String failure) { this(entry.snapshot, entry.includeGlobalState, entry.partial, state, entry.indices, entry.startTime, - entry.repositoryStateId, shards, failure); + entry.repositoryStateId, shards, failure, entry.userMetadata); } public Entry(Entry entry, ImmutableOpenMap shards) { @@ -149,6 +154,10 @@ public boolean includeGlobalState() { return includeGlobalState; } + public Map userMetadata() { + return userMetadata; + } + public boolean partial() { return partial; } @@ -419,6 +428,10 @@ public SnapshotsInProgress(StreamInput in) throws IOException { } long repositoryStateId = in.readLong(); final String failure = in.readOptionalString(); + Map userMetadata = null; + if (in.getVersion().onOrAfter(METADATA_FIELD_INTRODUCED)) { + userMetadata = in.readMap(); + } entries[i] = new Entry(snapshot, includeGlobalState, partial, @@ -427,7 +440,9 @@ public SnapshotsInProgress(StreamInput in) throws IOException { startTime, repositoryStateId, builder.build(), - failure); + failure, + userMetadata + ); } this.entries = Arrays.asList(entries); } @@ -452,6 +467,9 @@ public void writeTo(StreamOutput out) throws IOException { } out.writeLong(entry.repositoryStateId); out.writeOptionalString(entry.failure); + if (out.getVersion().onOrAfter(METADATA_FIELD_INTRODUCED)) { + out.writeMap(entry.userMetadata); + } } } diff --git a/server/src/main/java/org/elasticsearch/cluster/action/shard/ShardStateAction.java b/server/src/main/java/org/elasticsearch/cluster/action/shard/ShardStateAction.java index 1f9a45c4713f6..d28cf40026d6c 100644 --- a/server/src/main/java/org/elasticsearch/cluster/action/shard/ShardStateAction.java +++ b/server/src/main/java/org/elasticsearch/cluster/action/shard/ShardStateAction.java @@ -36,7 +36,7 @@ import org.elasticsearch.cluster.coordination.FailedToCommitClusterStateException; import org.elasticsearch.cluster.metadata.IndexMetaData; import org.elasticsearch.cluster.node.DiscoveryNode; -import org.elasticsearch.cluster.routing.RoutingService; +import org.elasticsearch.cluster.routing.RerouteService; import org.elasticsearch.cluster.routing.ShardRouting; import org.elasticsearch.cluster.routing.allocation.AllocationService; import org.elasticsearch.cluster.routing.allocation.FailedShard; @@ -89,7 +89,7 @@ public class ShardStateAction { @Inject public ShardStateAction(ClusterService clusterService, TransportService transportService, - AllocationService allocationService, RoutingService routingService, ThreadPool threadPool) { + AllocationService allocationService, RerouteService rerouteService, ThreadPool threadPool) { this.transportService = transportService; this.clusterService = clusterService; this.threadPool = threadPool; @@ -98,7 +98,7 @@ public ShardStateAction(ClusterService clusterService, TransportService transpor new ShardStartedTransportHandler(clusterService, new ShardStartedClusterStateTaskExecutor(allocationService, logger), logger)); transportService.registerRequestHandler(SHARD_FAILED_ACTION_NAME, ThreadPool.Names.SAME, FailedShardEntry::new, new ShardFailedTransportHandler(clusterService, - new ShardFailedClusterStateTaskExecutor(allocationService, routingService, logger), logger)); + new ShardFailedClusterStateTaskExecutor(allocationService, rerouteService, logger), logger)); } private void sendShardAction(final String actionName, final ClusterState currentState, @@ -280,12 +280,12 @@ public void clusterStateProcessed(String source, ClusterState oldState, ClusterS public static class ShardFailedClusterStateTaskExecutor implements ClusterStateTaskExecutor { private final AllocationService allocationService; - private final RoutingService routingService; + private final RerouteService rerouteService; private final Logger logger; - public ShardFailedClusterStateTaskExecutor(AllocationService allocationService, RoutingService routingService, Logger logger) { + public ShardFailedClusterStateTaskExecutor(AllocationService allocationService, RerouteService rerouteService, Logger logger) { this.allocationService = allocationService; - this.routingService = routingService; + this.rerouteService = rerouteService; this.logger = logger; } @@ -379,7 +379,9 @@ public void clusterStatePublished(ClusterChangedEvent clusterChangedEvent) { if (logger.isTraceEnabled()) { logger.trace("{}, scheduling a reroute", reason); } - routingService.reroute(reason); + rerouteService.reroute(reason, ActionListener.wrap( + r -> logger.trace("{}, reroute completed", reason), + e -> logger.debug(new ParameterizedMessage("{}, reroute failed", reason), e))); } } } @@ -580,7 +582,11 @@ public ClusterTasksResult execute(ClusterState currentState, @Override public void onFailure(String source, Exception e) { - logger.error(() -> new ParameterizedMessage("unexpected failure during [{}]", source), e); + if (e instanceof FailedToCommitClusterStateException || e instanceof NotMasterException) { + logger.debug(() -> new ParameterizedMessage("failure during [{}]", source), e); + } else { + logger.error(() -> new ParameterizedMessage("unexpected failure during [{}]", source), e); + } } } diff --git a/server/src/main/java/org/elasticsearch/cluster/coordination/ClusterFormationFailureHelper.java b/server/src/main/java/org/elasticsearch/cluster/coordination/ClusterFormationFailureHelper.java index a707a9ae980b7..c249458dd23cf 100644 --- a/server/src/main/java/org/elasticsearch/cluster/coordination/ClusterFormationFailureHelper.java +++ b/server/src/main/java/org/elasticsearch/cluster/coordination/ClusterFormationFailureHelper.java @@ -20,7 +20,6 @@ import org.apache.logging.log4j.LogManager; import org.apache.logging.log4j.Logger; -import org.elasticsearch.Version; import org.elasticsearch.cluster.ClusterState; import org.elasticsearch.cluster.coordination.CoordinationMetaData.VotingConfiguration; import org.elasticsearch.cluster.coordination.CoordinationState.VoteCollection; @@ -122,14 +121,16 @@ static class ClusterFormationState { private final List resolvedAddresses; private final List foundPeers; private final long currentTerm; + private final ElectionStrategy electionStrategy; ClusterFormationState(Settings settings, ClusterState clusterState, List resolvedAddresses, - List foundPeers, long currentTerm) { + List foundPeers, long currentTerm, ElectionStrategy electionStrategy) { this.settings = settings; this.clusterState = clusterState; this.resolvedAddresses = resolvedAddresses; this.foundPeers = foundPeers; this.currentTerm = currentTerm; + this.electionStrategy = electionStrategy; } String getDescription() { @@ -150,8 +151,6 @@ String getDescription() { if (clusterState.getLastAcceptedConfiguration().isEmpty()) { - // TODO handle the case that there is a 6.x node around here, when rolling upgrades are supported - final String bootstrappingDescription; if (INITIAL_MASTER_NODES_SETTING.get(Settings.EMPTY).equals(INITIAL_MASTER_NODES_SETTING.get(settings))) { @@ -164,8 +163,8 @@ String getDescription() { } return String.format(Locale.ROOT, - "master not discovered yet, this node has not previously joined a bootstrapped (v%d+) cluster, and %s: %s", - Version.V_6_6_0.major + 1, bootstrappingDescription, discoveryStateIgnoringQuorum); + "master not discovered yet, this node has not previously joined a bootstrapped cluster, and %s: %s", + bootstrappingDescription, discoveryStateIgnoringQuorum); } assert clusterState.getLastCommittedConfiguration().isEmpty() == false; @@ -188,7 +187,9 @@ String getDescription() { final VoteCollection voteCollection = new VoteCollection(); foundPeers.forEach(voteCollection::addVote); final String isQuorumOrNot - = CoordinationState.isElectionQuorum(voteCollection, clusterState) ? "is a quorum" : "is not a quorum"; + = electionStrategy.isElectionQuorum(clusterState.nodes().getLocalNode(), currentTerm, clusterState.term(), + clusterState.version(), clusterState.getLastCommittedConfiguration(), clusterState.getLastAcceptedConfiguration(), + voteCollection) ? "is a quorum" : "is not a quorum"; return String.format(Locale.ROOT, "master not discovered or elected yet, an election requires %s, have discovered %s which %s; %s", diff --git a/server/src/main/java/org/elasticsearch/cluster/coordination/CoordinationState.java b/server/src/main/java/org/elasticsearch/cluster/coordination/CoordinationState.java index ac75c83c19a26..7aad43aaab288 100644 --- a/server/src/main/java/org/elasticsearch/cluster/coordination/CoordinationState.java +++ b/server/src/main/java/org/elasticsearch/cluster/coordination/CoordinationState.java @@ -24,13 +24,14 @@ import org.elasticsearch.cluster.coordination.CoordinationMetaData.VotingConfiguration; import org.elasticsearch.cluster.metadata.MetaData; import org.elasticsearch.cluster.node.DiscoveryNode; -import org.elasticsearch.common.settings.Settings; import java.util.Collection; import java.util.Collections; import java.util.HashMap; +import java.util.HashSet; import java.util.Map; import java.util.Optional; +import java.util.Set; /** * The core class of the cluster state coordination algorithm, directly implementing the @@ -42,6 +43,8 @@ public class CoordinationState { private final DiscoveryNode localNode; + private final ElectionStrategy electionStrategy; + // persisted state private final PersistedState persistedState; @@ -53,11 +56,12 @@ public class CoordinationState { private VotingConfiguration lastPublishedConfiguration; private VoteCollection publishVotes; - public CoordinationState(Settings settings, DiscoveryNode localNode, PersistedState persistedState) { + public CoordinationState(DiscoveryNode localNode, PersistedState persistedState, ElectionStrategy electionStrategy) { this.localNode = localNode; // persisted state this.persistedState = persistedState; + this.electionStrategy = electionStrategy; // transient state this.joinVotes = new VoteCollection(); @@ -100,13 +104,9 @@ public boolean electionWon() { return electionWon; } - public boolean isElectionQuorum(VoteCollection votes) { - return isElectionQuorum(votes, getLastAcceptedState()); - } - - static boolean isElectionQuorum(VoteCollection votes, ClusterState lastAcceptedState) { - return votes.isQuorum(lastAcceptedState.getLastCommittedConfiguration()) - && votes.isQuorum(lastAcceptedState.getLastAcceptedConfiguration()); + public boolean isElectionQuorum(VoteCollection joinVotes) { + return electionStrategy.isElectionQuorum(localNode, getCurrentTerm(), getLastAcceptedTerm(), getLastAcceptedVersion(), + getLastCommittedConfiguration(), getLastAcceptedConfiguration(), joinVotes); } public boolean isPublishQuorum(VoteCollection votes) { @@ -117,6 +117,11 @@ public boolean containsJoinVoteFor(DiscoveryNode node) { return joinVotes.containsVoteFor(node); } + // used for tests + boolean containsJoin(Join join) { + return joinVotes.getJoins().contains(join); + } + public boolean joinVotesHaveQuorumFor(VotingConfiguration votingConfiguration) { return joinVotes.isQuorum(votingConfiguration); } @@ -243,10 +248,11 @@ public boolean handleJoin(Join join) { throw new CoordinationStateRejectedException("rejecting join since this node has not received its initial configuration yet"); } - boolean added = joinVotes.addVote(join.getSourceNode()); + boolean added = joinVotes.addJoinVote(join); boolean prevElectionWon = electionWon; electionWon = isElectionQuorum(joinVotes); - assert !prevElectionWon || electionWon; // we cannot go from won to not won + assert !prevElectionWon || electionWon : // we cannot go from won to not won + "locaNode= " + localNode + ", join=" + join + ", joinVotes=" + joinVotes; logger.debug("handleJoin: added join {} from [{}] for election, electionWon={} lastAcceptedTerm={} lastAcceptedVersion={}", join, join.getSourceNode(), electionWon, lastAcceptedTerm, getLastAcceptedVersion()); @@ -489,18 +495,28 @@ default void markLastAcceptedStateAsCommitted() { } /** - * A collection of votes, used to calculate quorums. + * A collection of votes, used to calculate quorums. Optionally records the Joins as well. */ public static class VoteCollection { private final Map nodes; + private final Set joins; public boolean addVote(DiscoveryNode sourceNode) { return nodes.put(sourceNode.getId(), sourceNode) == null; } + public boolean addJoinVote(Join join) { + final boolean added = addVote(join.getSourceNode()); + if (added) { + joins.add(join); + } + return added; + } + public VoteCollection() { nodes = new HashMap<>(); + joins = new HashSet<>(); } public boolean isQuorum(VotingConfiguration configuration) { @@ -519,24 +535,31 @@ public Collection nodes() { return Collections.unmodifiableCollection(nodes.values()); } + public Set getJoins() { + return Collections.unmodifiableSet(joins); + } + @Override public String toString() { - return "VoteCollection{" + String.join(",", nodes.keySet()) + "}"; + return "VoteCollection{votes=" + nodes.keySet() + ", joins=" + joins + "}"; } @Override public boolean equals(Object o) { if (this == o) return true; - if (o == null || getClass() != o.getClass()) return false; + if (!(o instanceof VoteCollection)) return false; VoteCollection that = (VoteCollection) o; - return nodes.equals(that.nodes); + if (!nodes.equals(that.nodes)) return false; + return joins.equals(that.joins); } @Override public int hashCode() { - return nodes.hashCode(); + int result = nodes.hashCode(); + result = 31 * result + joins.hashCode(); + return result; } } } diff --git a/server/src/main/java/org/elasticsearch/cluster/coordination/Coordinator.java b/server/src/main/java/org/elasticsearch/cluster/coordination/Coordinator.java index 1e7b38e50d1e9..8de27337f1cba 100644 --- a/server/src/main/java/org/elasticsearch/cluster/coordination/Coordinator.java +++ b/server/src/main/java/org/elasticsearch/cluster/coordination/Coordinator.java @@ -33,11 +33,13 @@ import org.elasticsearch.cluster.coordination.ClusterFormationFailureHelper.ClusterFormationState; import org.elasticsearch.cluster.coordination.CoordinationMetaData.VotingConfigExclusion; import org.elasticsearch.cluster.coordination.CoordinationMetaData.VotingConfiguration; +import org.elasticsearch.cluster.coordination.CoordinationState.VoteCollection; import org.elasticsearch.cluster.coordination.FollowersChecker.FollowerCheckRequest; import org.elasticsearch.cluster.coordination.JoinHelper.InitialJoinAccumulator; import org.elasticsearch.cluster.metadata.MetaData; import org.elasticsearch.cluster.node.DiscoveryNode; import org.elasticsearch.cluster.node.DiscoveryNodes; +import org.elasticsearch.cluster.routing.RerouteService; import org.elasticsearch.cluster.routing.allocation.AllocationService; import org.elasticsearch.cluster.service.ClusterApplier; import org.elasticsearch.cluster.service.ClusterApplier.ClusterApplyListener; @@ -81,6 +83,7 @@ import java.util.function.BiConsumer; import java.util.function.Supplier; import java.util.stream.Collectors; +import java.util.stream.Stream; import java.util.stream.StreamSupport; import static org.elasticsearch.cluster.coordination.NoMasterBlockService.NO_MASTER_BLOCK_ID; @@ -98,6 +101,7 @@ public class Coordinator extends AbstractLifecycleComponent implements Discovery private final Settings settings; private final boolean singleNodeDiscovery; + private final ElectionStrategy electionStrategy; private final TransportService transportService; private final MasterService masterService; private final AllocationService allocationService; @@ -138,18 +142,25 @@ public class Coordinator extends AbstractLifecycleComponent implements Discovery private JoinHelper.JoinAccumulator joinAccumulator; private Optional currentPublication = Optional.empty(); + /** + * @param nodeName The name of the node, used to name the {@link java.util.concurrent.ExecutorService} of the {@link SeedHostsResolver}. + * @param onJoinValidators A collection of join validators to restrict which nodes may join the cluster. + */ public Coordinator(String nodeName, Settings settings, ClusterSettings clusterSettings, TransportService transportService, NamedWriteableRegistry namedWriteableRegistry, AllocationService allocationService, MasterService masterService, Supplier persistedStateSupplier, SeedHostsProvider seedHostsProvider, - ClusterApplier clusterApplier, Collection> onJoinValidators, Random random) { + ClusterApplier clusterApplier, Collection> onJoinValidators, Random random, + RerouteService rerouteService, ElectionStrategy electionStrategy) { this.settings = settings; this.transportService = transportService; this.masterService = masterService; this.allocationService = allocationService; this.onJoinValidators = JoinTaskExecutor.addBuiltInJoinValidators(onJoinValidators); this.singleNodeDiscovery = DiscoveryModule.SINGLE_NODE_DISCOVERY_TYPE.equals(DiscoveryModule.DISCOVERY_TYPE_SETTING.get(settings)); + this.electionStrategy = electionStrategy; this.joinHelper = new JoinHelper(settings, allocationService, masterService, transportService, - this::getCurrentTerm, this::getStateForMasterService, this::handleJoinRequest, this::joinLeaderInTerm, this.onJoinValidators); + this::getCurrentTerm, this::getStateForMasterService, this::handleJoinRequest, this::joinLeaderInTerm, this.onJoinValidators, + rerouteService); this.persistedStateSupplier = persistedStateSupplier; this.noMasterBlockService = new NoMasterBlockService(settings, clusterSettings); this.lastKnownLeader = Optional.empty(); @@ -158,13 +169,13 @@ public Coordinator(String nodeName, Settings settings, ClusterSettings clusterSe this.publishTimeout = PUBLISH_TIMEOUT_SETTING.get(settings); this.random = random; this.electionSchedulerFactory = new ElectionSchedulerFactory(settings, random, transportService.getThreadPool()); - this.preVoteCollector = new PreVoteCollector(transportService, this::startElection, this::updateMaxTermSeen); + this.preVoteCollector = new PreVoteCollector(transportService, this::startElection, this::updateMaxTermSeen, electionStrategy); configuredHostsResolver = new SeedHostsResolver(nodeName, settings, transportService, seedHostsProvider); this.peerFinder = new CoordinatorPeerFinder(settings, transportService, new HandshakingTransportAddressConnector(settings, transportService), configuredHostsResolver); this.publicationHandler = new PublicationTransportHandler(transportService, namedWriteableRegistry, this::handlePublishRequest, this::handleApplyCommit); - this.leaderChecker = new LeaderChecker(settings, transportService, getOnLeaderFailure()); + this.leaderChecker = new LeaderChecker(settings, transportService, this::onLeaderFailure); this.followersChecker = new FollowersChecker(settings, transportService, this::onFollowerCheckRequest, this::removeNode); this.nodeRemovalExecutor = new NodeRemovalClusterStateTaskExecutor(allocationService, logger); this.clusterApplier = clusterApplier; @@ -180,23 +191,18 @@ public Coordinator(String nodeName, Settings settings, ClusterSettings clusterSe private ClusterFormationState getClusterFormationState() { return new ClusterFormationState(settings, getStateForMasterService(), peerFinder.getLastResolvedAddresses(), - StreamSupport.stream(peerFinder.getFoundPeers().spliterator(), false).collect(Collectors.toList()), getCurrentTerm()); + Stream.concat(Stream.of(getLocalNode()), StreamSupport.stream(peerFinder.getFoundPeers().spliterator(), false)) + .collect(Collectors.toList()), getCurrentTerm(), electionStrategy); } - private Runnable getOnLeaderFailure() { - return new Runnable() { - @Override - public void run() { - synchronized (mutex) { - becomeCandidate("onLeaderFailure"); - } - } - - @Override - public String toString() { - return "notification of leader failure"; + private void onLeaderFailure(Exception e) { + synchronized (mutex) { + if (mode != Mode.CANDIDATE) { + assert lastKnownLeader.isPresent(); + logger.info(new ParameterizedMessage("master node [{}] failed, restarting discovery", lastKnownLeader.get()), e); } - }; + becomeCandidate("onLeaderFailure"); + } } private void removeNode(DiscoveryNode discoveryNode, String reason) { @@ -361,9 +367,8 @@ private void startElection() { // The preVoteCollector is only active while we are candidate, but it does not call this method with synchronisation, so we have // to check our mode again here. if (mode == Mode.CANDIDATE) { - if (electionQuorumContainsLocalNode(getLastAcceptedState()) == false) { - logger.trace("skip election as local node is not part of election quorum: {}", - getLastAcceptedState().coordinationMetaData()); + if (localNodeMayWinElection(getLastAcceptedState()) == false) { + logger.trace("skip election as local node may not win it: {}", getLastAcceptedState().coordinationMetaData()); return; } @@ -388,16 +393,17 @@ private void abdicateTo(DiscoveryNode newMaster) { becomeCandidate("after abdicating to " + newMaster); } - private static boolean electionQuorumContainsLocalNode(ClusterState lastAcceptedState) { + private static boolean localNodeMayWinElection(ClusterState lastAcceptedState) { final DiscoveryNode localNode = lastAcceptedState.nodes().getLocalNode(); assert localNode != null; - return electionQuorumContains(lastAcceptedState, localNode); + return nodeMayWinElection(lastAcceptedState, localNode); } - private static boolean electionQuorumContains(ClusterState lastAcceptedState, DiscoveryNode node) { + private static boolean nodeMayWinElection(ClusterState lastAcceptedState, DiscoveryNode node) { final String nodeId = node.getId(); return lastAcceptedState.getLastCommittedConfiguration().getNodeIds().contains(nodeId) - || lastAcceptedState.getLastAcceptedConfiguration().getNodeIds().contains(nodeId); + || lastAcceptedState.getLastAcceptedConfiguration().getNodeIds().contains(nodeId) + || lastAcceptedState.getVotingConfigExclusions().stream().noneMatch(vce -> vce.getNodeId().equals(nodeId)); } private Optional ensureTermAtLeast(DiscoveryNode sourceNode, long targetTerm) { @@ -644,7 +650,7 @@ boolean publicationInProgress() { protected void doStart() { synchronized (mutex) { CoordinationState.PersistedState persistedState = persistedStateSupplier.get(); - coordinationState.set(new CoordinationState(settings, getLocalNode(), persistedState)); + coordinationState.set(new CoordinationState(getLocalNode(), persistedState, electionStrategy)); peerFinder.setCurrentTerm(getCurrentTerm()); configuredHostsResolver.start(); final ClusterState lastAcceptedState = coordinationState.get().getLastAcceptedState(); @@ -699,7 +705,6 @@ public void invariant() { assert followersChecker.getFastResponseState().term == getCurrentTerm() : followersChecker.getFastResponseState(); assert followersChecker.getFastResponseState().mode == getMode() : followersChecker.getFastResponseState(); assert (applierState.nodes().getMasterNodeId() == null) == applierState.blocks().hasGlobalBlockWithId(NO_MASTER_BLOCK_ID); - assert applierState.nodes().getMasterNodeId() == null || applierState.metaData().clusterUUIDCommitted(); assert preVoteCollector.getPreVoteResponse().equals(getPreVoteResponse()) : preVoteCollector + " vs " + getPreVoteResponse(); @@ -835,8 +840,8 @@ public boolean setInitialConfiguration(final VotingConfiguration votingConfigura metaDataBuilder.coordinationMetaData(coordinationMetaData); coordinationState.get().setInitialState(ClusterState.builder(currentState).metaData(metaDataBuilder).build()); - assert electionQuorumContainsLocalNode(getLastAcceptedState()) : - "initial state does not have local node in its election quorum: " + getLastAcceptedState().coordinationMetaData(); + assert localNodeMayWinElection(getLastAcceptedState()) : + "initial state does not allow local node to win election: " + getLastAcceptedState().coordinationMetaData(); preVoteCollector.update(getPreVoteResponse(), null); // pick up the change to last-accepted version startElectionScheduler(); return true; @@ -1097,11 +1102,10 @@ protected void onFoundPeersUpdated() { synchronized (mutex) { final Iterable foundPeers = getFoundPeers(); if (mode == Mode.CANDIDATE) { - final CoordinationState.VoteCollection expectedVotes = new CoordinationState.VoteCollection(); + final VoteCollection expectedVotes = new VoteCollection(); foundPeers.forEach(expectedVotes::addVote); expectedVotes.addVote(Coordinator.this.getLocalNode()); - final ClusterState lastAcceptedState = coordinationState.get().getLastAcceptedState(); - final boolean foundQuorum = CoordinationState.isElectionQuorum(expectedVotes, lastAcceptedState); + final boolean foundQuorum = coordinationState.get().isElectionQuorum(expectedVotes); if (foundQuorum) { if (electionScheduler == null) { @@ -1132,8 +1136,8 @@ public void run() { if (mode == Mode.CANDIDATE) { final ClusterState lastAcceptedState = coordinationState.get().getLastAcceptedState(); - if (electionQuorumContainsLocalNode(lastAcceptedState) == false) { - logger.trace("skip prevoting as local node is not part of election quorum: {}", + if (localNodeMayWinElection(lastAcceptedState) == false) { + logger.trace("skip prevoting as local node may not win election: {}", lastAcceptedState.coordinationMetaData()); return; } @@ -1165,9 +1169,13 @@ public Iterable getFoundPeers() { */ boolean cancelCommittedPublication() { synchronized (mutex) { - if (currentPublication.isPresent() && currentPublication.get().isCommitted()) { - currentPublication.get().cancel("cancelCommittedPublication"); - return true; + if (currentPublication.isPresent()) { + final CoordinatorPublication publication = currentPublication.get(); + if (publication.isCommitted()) { + publication.cancel("cancelCommittedPublication"); + logger.debug("Cancelled publication of [{}].", publication); + return true; + } } return false; } @@ -1290,16 +1298,32 @@ public void onSuccess(String source) { updateMaxTermSeen(getCurrentTerm()); if (mode == Mode.LEADER) { + // if necessary, abdicate to another node or improve the voting configuration + boolean attemptReconfiguration = true; final ClusterState state = getLastAcceptedState(); // committed state - if (electionQuorumContainsLocalNode(state) == false) { + if (localNodeMayWinElection(state) == false) { final List masterCandidates = completedNodes().stream() .filter(DiscoveryNode::isMasterNode) - .filter(node -> electionQuorumContains(state, node)) + .filter(node -> nodeMayWinElection(state, node)) + .filter(node -> { + // check if master candidate would be able to get an election quorum if we were to + // abdicate to it. Assume that every node that completed the publication can provide + // a vote in that next election and has the latest state. + final long futureElectionTerm = state.term() + 1; + final VoteCollection futureVoteCollection = new VoteCollection(); + completedNodes().forEach(completedNode -> futureVoteCollection.addJoinVote( + new Join(completedNode, node, futureElectionTerm, state.term(), state.version()))); + return electionStrategy.isElectionQuorum(node, futureElectionTerm, + state.term(), state.version(), state.getLastCommittedConfiguration(), + state.getLastAcceptedConfiguration(), futureVoteCollection); + }) .collect(Collectors.toList()); if (masterCandidates.isEmpty() == false) { abdicateTo(masterCandidates.get(random.nextInt(masterCandidates.size()))); + attemptReconfiguration = false; } - } else { + } + if (attemptReconfiguration) { scheduleReconfigurationIfNeeded(); } } @@ -1333,7 +1357,7 @@ private void handleAssociatedJoin(Join join) { } @Override - protected boolean isPublishQuorum(CoordinationState.VoteCollection votes) { + protected boolean isPublishQuorum(VoteCollection votes) { assert Thread.holdsLock(mutex) : "Coordinator mutex not held"; return coordinationState.get().isPublishQuorum(votes); } diff --git a/server/src/main/java/org/elasticsearch/cluster/coordination/ElectionStrategy.java b/server/src/main/java/org/elasticsearch/cluster/coordination/ElectionStrategy.java new file mode 100644 index 0000000000000..b9e00cd52fc4e --- /dev/null +++ b/server/src/main/java/org/elasticsearch/cluster/coordination/ElectionStrategy.java @@ -0,0 +1,74 @@ +/* + * 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.cluster.coordination; + +import org.elasticsearch.cluster.coordination.CoordinationMetaData.VotingConfiguration; +import org.elasticsearch.cluster.coordination.CoordinationState.VoteCollection; +import org.elasticsearch.cluster.node.DiscoveryNode; + +/** + * Allows plugging in a custom election strategy, restricting the notion of an election quorum. + * Custom additional quorum restrictions can be defined by implementing the {@link #satisfiesAdditionalQuorumConstraints} method. + */ +public abstract class ElectionStrategy { + + public static final ElectionStrategy DEFAULT_INSTANCE = new ElectionStrategy() { + @Override + protected boolean satisfiesAdditionalQuorumConstraints(DiscoveryNode localNode, long localCurrentTerm, long localAcceptedTerm, + long localAcceptedVersion, VotingConfiguration lastCommittedConfiguration, + VotingConfiguration lastAcceptedConfiguration, VoteCollection joinVotes) { + return true; + } + }; + + protected ElectionStrategy() { + + } + + /** + * Whether there is an election quorum from the point of view of the given local node under the provided voting configurations + */ + public final boolean isElectionQuorum(DiscoveryNode localNode, long localCurrentTerm, long localAcceptedTerm, long localAcceptedVersion, + VotingConfiguration lastCommittedConfiguration, VotingConfiguration lastAcceptedConfiguration, + VoteCollection joinVotes) { + return joinVotes.isQuorum(lastCommittedConfiguration) && + joinVotes.isQuorum(lastAcceptedConfiguration) && + satisfiesAdditionalQuorumConstraints(localNode, localCurrentTerm, localAcceptedTerm, localAcceptedVersion, + lastCommittedConfiguration, lastAcceptedConfiguration, joinVotes); + } + + /** + * The extension point to be overridden by plugins. Defines additional constraints on the election quorum. + * @param localNode the local node for the election quorum + * @param localCurrentTerm the current term of the local node + * @param localAcceptedTerm the last accepted term of the local node + * @param localAcceptedVersion the last accepted version of the local node + * @param lastCommittedConfiguration the last committed configuration for the election quorum + * @param lastAcceptedConfiguration the last accepted configuration for the election quorum + * @param joinVotes the votes that were provided so far + * @return true iff the additional quorum constraints are satisfied + */ + protected abstract boolean satisfiesAdditionalQuorumConstraints(DiscoveryNode localNode, + long localCurrentTerm, + long localAcceptedTerm, + long localAcceptedVersion, + VotingConfiguration lastCommittedConfiguration, + VotingConfiguration lastAcceptedConfiguration, + VoteCollection joinVotes); +} diff --git a/server/src/main/java/org/elasticsearch/cluster/coordination/JoinHelper.java b/server/src/main/java/org/elasticsearch/cluster/coordination/JoinHelper.java index 3a52324661fc0..5d4f847bc7c26 100644 --- a/server/src/main/java/org/elasticsearch/cluster/coordination/JoinHelper.java +++ b/server/src/main/java/org/elasticsearch/cluster/coordination/JoinHelper.java @@ -30,6 +30,7 @@ import org.elasticsearch.cluster.coordination.Coordinator.Mode; import org.elasticsearch.cluster.metadata.MetaData; import org.elasticsearch.cluster.node.DiscoveryNode; +import org.elasticsearch.cluster.routing.RerouteService; import org.elasticsearch.cluster.routing.allocation.AllocationService; import org.elasticsearch.cluster.service.MasterService; import org.elasticsearch.common.Priority; @@ -91,11 +92,11 @@ public class JoinHelper { JoinHelper(Settings settings, AllocationService allocationService, MasterService masterService, TransportService transportService, LongSupplier currentTermSupplier, Supplier currentStateSupplier, BiConsumer joinHandler, Function joinLeaderInTerm, - Collection> joinValidators) { + Collection> joinValidators, RerouteService rerouteService) { this.masterService = masterService; this.transportService = transportService; this.joinTimeout = JOIN_TIMEOUT_SETTING.get(settings); - this.joinTaskExecutor = new JoinTaskExecutor(allocationService, logger) { + this.joinTaskExecutor = new JoinTaskExecutor(allocationService, logger, rerouteService) { @Override public ClusterTasksResult execute(ClusterState currentState, List joiningTasks) diff --git a/server/src/main/java/org/elasticsearch/cluster/coordination/JoinTaskExecutor.java b/server/src/main/java/org/elasticsearch/cluster/coordination/JoinTaskExecutor.java index 7434a246eed8a..c5ed468993ff7 100644 --- a/server/src/main/java/org/elasticsearch/cluster/coordination/JoinTaskExecutor.java +++ b/server/src/main/java/org/elasticsearch/cluster/coordination/JoinTaskExecutor.java @@ -20,6 +20,7 @@ import org.apache.logging.log4j.Logger; import org.elasticsearch.Version; +import org.elasticsearch.action.ActionListener; import org.elasticsearch.cluster.ClusterState; import org.elasticsearch.cluster.ClusterStateTaskExecutor; import org.elasticsearch.cluster.NotMasterException; @@ -28,6 +29,7 @@ import org.elasticsearch.cluster.metadata.MetaData; import org.elasticsearch.cluster.node.DiscoveryNode; import org.elasticsearch.cluster.node.DiscoveryNodes; +import org.elasticsearch.cluster.routing.RerouteService; import org.elasticsearch.cluster.routing.allocation.AllocationService; import org.elasticsearch.persistent.PersistentTasksCustomMetaData; @@ -44,6 +46,7 @@ public class JoinTaskExecutor implements ClusterStateTaskExecutor execute(ClusterState currentState, List jo results.success(joinTask); } if (nodesChanged) { - newState.nodes(nodesBuilder); - return results.build(allocationService.reroute(newState.build(), "node_join")); + rerouteService.reroute("post-join reroute", ActionListener.wrap( + r -> logger.trace("post-join reroute completed"), + e -> logger.debug("post-join reroute failed", e))); + + return results.build(allocationService.adaptAutoExpandReplicas(newState.nodes(nodesBuilder).build())); } else { // we must return a new cluster state instance to force publishing. This is important // for the joining node to finalize its join and set us as a master diff --git a/server/src/main/java/org/elasticsearch/cluster/coordination/LeaderChecker.java b/server/src/main/java/org/elasticsearch/cluster/coordination/LeaderChecker.java index 5bc5ea866ee83..703c08bf260c7 100644 --- a/server/src/main/java/org/elasticsearch/cluster/coordination/LeaderChecker.java +++ b/server/src/main/java/org/elasticsearch/cluster/coordination/LeaderChecker.java @@ -22,6 +22,7 @@ import org.apache.logging.log4j.LogManager; import org.apache.logging.log4j.Logger; import org.apache.logging.log4j.message.ParameterizedMessage; +import org.elasticsearch.ElasticsearchException; import org.elasticsearch.cluster.node.DiscoveryNode; import org.elasticsearch.cluster.node.DiscoveryNodes; import org.elasticsearch.common.Nullable; @@ -33,6 +34,7 @@ import org.elasticsearch.common.unit.TimeValue; import org.elasticsearch.threadpool.ThreadPool.Names; import org.elasticsearch.transport.ConnectTransportException; +import org.elasticsearch.transport.NodeDisconnectedException; import org.elasticsearch.transport.TransportConnectionListener; import org.elasticsearch.transport.TransportException; import org.elasticsearch.transport.TransportRequest; @@ -48,6 +50,7 @@ import java.util.concurrent.atomic.AtomicBoolean; import java.util.concurrent.atomic.AtomicLong; import java.util.concurrent.atomic.AtomicReference; +import java.util.function.Consumer; /** * The LeaderChecker is responsible for allowing followers to check that the currently elected leader is still connected and healthy. We are @@ -75,20 +78,17 @@ public class LeaderChecker { public static final Setting LEADER_CHECK_RETRY_COUNT_SETTING = Setting.intSetting("cluster.fault_detection.leader_check.retry_count", 3, 1, Setting.Property.NodeScope); - private final Settings settings; - private final TimeValue leaderCheckInterval; private final TimeValue leaderCheckTimeout; private final int leaderCheckRetryCount; private final TransportService transportService; - private final Runnable onLeaderFailure; + private final Consumer onLeaderFailure; private AtomicReference currentChecker = new AtomicReference<>(); private volatile DiscoveryNodes discoveryNodes; - public LeaderChecker(final Settings settings, final TransportService transportService, final Runnable onLeaderFailure) { - this.settings = settings; + public LeaderChecker(final Settings settings, final TransportService transportService, final Consumer onLeaderFailure) { leaderCheckInterval = LEADER_CHECK_INTERVAL_SETTING.get(settings); leaderCheckTimeout = LEADER_CHECK_TIMEOUT_SETTING.get(settings); leaderCheckRetryCount = LEADER_CHECK_RETRY_COUNT_SETTING.get(settings); @@ -234,16 +234,19 @@ public void handleException(TransportException exp) { } if (exp instanceof ConnectTransportException || exp.getCause() instanceof ConnectTransportException) { - logger.debug(new ParameterizedMessage("leader [{}] disconnected, failing immediately", leader), exp); - leaderFailed(); + logger.debug(new ParameterizedMessage( + "leader [{}] disconnected during check", leader), exp); + leaderFailed(new ConnectTransportException(leader, "disconnected during check", exp)); return; } long failureCount = failureCountSinceLastSuccess.incrementAndGet(); if (failureCount >= leaderCheckRetryCount) { - logger.debug(new ParameterizedMessage("{} consecutive failures (limit [{}] is {}) so leader [{}] has failed", - failureCount, LEADER_CHECK_RETRY_COUNT_SETTING.getKey(), leaderCheckRetryCount, leader), exp); - leaderFailed(); + logger.debug(new ParameterizedMessage( + "leader [{}] has failed {} consecutive checks (limit [{}] is {}); last failure was:", + leader, failureCount, LEADER_CHECK_RETRY_COUNT_SETTING.getKey(), leaderCheckRetryCount), exp); + leaderFailed(new ElasticsearchException( + "node [" + leader + "] failed [" + failureCount + "] consecutive checks", exp)); return; } @@ -259,9 +262,19 @@ public String executor() { }); } - void leaderFailed() { + void leaderFailed(Exception e) { if (isClosed.compareAndSet(false, true)) { - transportService.getThreadPool().generic().execute(onLeaderFailure); + transportService.getThreadPool().generic().execute(new Runnable() { + @Override + public void run() { + onLeaderFailure.accept(e); + } + + @Override + public String toString() { + return "notification of leader failure: " + e.getMessage(); + } + }); } else { logger.trace("already closed, not failing leader"); } @@ -269,7 +282,8 @@ void leaderFailed() { void handleDisconnectedNode(DiscoveryNode discoveryNode) { if (discoveryNode.equals(leader)) { - leaderFailed(); + logger.debug("leader [{}] disconnected", leader); + leaderFailed(new NodeDisconnectedException(discoveryNode, "disconnected")); } } diff --git a/server/src/main/java/org/elasticsearch/cluster/coordination/PreVoteCollector.java b/server/src/main/java/org/elasticsearch/cluster/coordination/PreVoteCollector.java index 5001e3be18183..f683057b1fcd1 100644 --- a/server/src/main/java/org/elasticsearch/cluster/coordination/PreVoteCollector.java +++ b/server/src/main/java/org/elasticsearch/cluster/coordination/PreVoteCollector.java @@ -35,12 +35,11 @@ import org.elasticsearch.transport.TransportService; import java.io.IOException; -import java.util.Set; +import java.util.Map; import java.util.concurrent.atomic.AtomicBoolean; import java.util.function.LongConsumer; -import static org.elasticsearch.cluster.coordination.CoordinationState.isElectionQuorum; -import static org.elasticsearch.common.util.concurrent.ConcurrentCollections.newConcurrentSet; +import static org.elasticsearch.common.util.concurrent.ConcurrentCollections.newConcurrentMap; public class PreVoteCollector { @@ -51,14 +50,17 @@ public class PreVoteCollector { private final TransportService transportService; private final Runnable startElection; private final LongConsumer updateMaxTermSeen; + private final ElectionStrategy electionStrategy; // Tuple for simple atomic updates. null until the first call to `update()`. private volatile Tuple state; // DiscoveryNode component is null if there is currently no known leader. - PreVoteCollector(final TransportService transportService, final Runnable startElection, final LongConsumer updateMaxTermSeen) { + PreVoteCollector(final TransportService transportService, final Runnable startElection, final LongConsumer updateMaxTermSeen, + final ElectionStrategy electionStrategy) { this.transportService = transportService; this.startElection = startElection; this.updateMaxTermSeen = updateMaxTermSeen; + this.electionStrategy = electionStrategy; // TODO does this need to be on the generic threadpool or can it use SAME? transportService.registerRequestHandler(REQUEST_PRE_VOTE_ACTION_NAME, Names.GENERIC, false, false, @@ -128,7 +130,7 @@ public String toString() { } private class PreVotingRound implements Releasable { - private final Set preVotesReceived = newConcurrentSet(); + private final Map preVotesReceived = newConcurrentMap(); private final AtomicBoolean electionStarted = new AtomicBoolean(); private final PreVoteRequest preVoteRequest; private final ClusterState clusterState; @@ -185,11 +187,20 @@ private void handlePreVoteResponse(final PreVoteResponse response, final Discove return; } - preVotesReceived.add(sender); + preVotesReceived.put(sender, response); + + // create a fake VoteCollection based on the pre-votes and check if there is an election quorum final VoteCollection voteCollection = new VoteCollection(); - preVotesReceived.forEach(voteCollection::addVote); + final DiscoveryNode localNode = clusterState.nodes().getLocalNode(); + final PreVoteResponse localPreVoteResponse = getPreVoteResponse(); + + preVotesReceived.forEach((node, preVoteResponse) -> voteCollection.addJoinVote( + new Join(node, localNode, preVoteResponse.getCurrentTerm(), + preVoteResponse.getLastAcceptedTerm(), preVoteResponse.getLastAcceptedVersion()))); - if (isElectionQuorum(voteCollection, clusterState) == false) { + if (electionStrategy.isElectionQuorum(clusterState.nodes().getLocalNode(), localPreVoteResponse.getCurrentTerm(), + localPreVoteResponse.getLastAcceptedTerm(), localPreVoteResponse.getLastAcceptedVersion(), + clusterState.getLastCommittedConfiguration(), clusterState.getLastAcceptedConfiguration(), voteCollection) == false) { logger.debug("{} added {} from {}, no quorum yet", this, response, sender); return; } diff --git a/server/src/main/java/org/elasticsearch/cluster/coordination/Reconfigurator.java b/server/src/main/java/org/elasticsearch/cluster/coordination/Reconfigurator.java index 7a3a54d73b2fe..fd9bab3af11ec 100644 --- a/server/src/main/java/org/elasticsearch/cluster/coordination/Reconfigurator.java +++ b/server/src/main/java/org/elasticsearch/cluster/coordination/Reconfigurator.java @@ -150,6 +150,11 @@ static class VotingConfigNode implements Comparable { @Override public int compareTo(VotingConfigNode other) { + // prefer current master + final int currentMasterComp = Boolean.compare(other.currentMaster, currentMaster); + if (currentMasterComp != 0) { + return currentMasterComp; + } // prefer nodes that are live final int liveComp = Boolean.compare(other.live, live); if (liveComp != 0) { @@ -160,11 +165,6 @@ public int compareTo(VotingConfigNode other) { if (inCurrentConfigComp != 0) { return inCurrentConfigComp; } - // prefer current master - final int currentMasterComp = Boolean.compare(other.currentMaster, currentMaster); - if (currentMasterComp != 0) { - return currentMasterComp; - } // tiebreak by node id to have stable ordering return id.compareTo(other.id); } diff --git a/server/src/main/java/org/elasticsearch/cluster/metadata/AliasMetaData.java b/server/src/main/java/org/elasticsearch/cluster/metadata/AliasMetaData.java index 0d41d5f86b3ac..478e76b525610 100644 --- a/server/src/main/java/org/elasticsearch/cluster/metadata/AliasMetaData.java +++ b/server/src/main/java/org/elasticsearch/cluster/metadata/AliasMetaData.java @@ -140,15 +140,13 @@ public boolean equals(Object o) { if (this == o) return true; if (o == null || getClass() != o.getClass()) return false; - AliasMetaData that = (AliasMetaData) o; + final AliasMetaData that = (AliasMetaData) o; if (alias != null ? !alias.equals(that.alias) : that.alias != null) return false; if (filter != null ? !filter.equals(that.filter) : that.filter != null) return false; if (indexRouting != null ? !indexRouting.equals(that.indexRouting) : that.indexRouting != null) return false; - if (searchRouting != null ? !searchRouting.equals(that.searchRouting) : that.searchRouting != null) - return false; - if (writeIndex != null ? writeIndex != that.writeIndex : that.writeIndex != null) - return false; + if (searchRouting != null ? !searchRouting.equals(that.searchRouting) : that.searchRouting != null) return false; + if (writeIndex != null ? writeIndex != that.writeIndex : that.writeIndex != null) return false; return true; } diff --git a/server/src/main/java/org/elasticsearch/cluster/metadata/DiffableStringMap.java b/server/src/main/java/org/elasticsearch/cluster/metadata/DiffableStringMap.java index 46433eed8a657..b6e31e92698e6 100644 --- a/server/src/main/java/org/elasticsearch/cluster/metadata/DiffableStringMap.java +++ b/server/src/main/java/org/elasticsearch/cluster/metadata/DiffableStringMap.java @@ -39,6 +39,8 @@ */ public class DiffableStringMap extends AbstractMap implements Diffable { + public static final DiffableStringMap EMPTY = new DiffableStringMap(Collections.emptyMap()); + private final Map innerMap; DiffableStringMap(final Map map) { @@ -75,6 +77,8 @@ public static Diff readDiffFrom(StreamInput in) throws IOExce */ public static class DiffableStringMapDiff implements Diff { + public static final DiffableStringMapDiff EMPTY = new DiffableStringMapDiff(DiffableStringMap.EMPTY, DiffableStringMap.EMPTY); + private final List deletes; private final Map upserts; // diffs also become upserts diff --git a/server/src/main/java/org/elasticsearch/cluster/metadata/IndexMetaData.java b/server/src/main/java/org/elasticsearch/cluster/metadata/IndexMetaData.java index 412e16e14a3a5..3598753f80d9c 100644 --- a/server/src/main/java/org/elasticsearch/cluster/metadata/IndexMetaData.java +++ b/server/src/main/java/org/elasticsearch/cluster/metadata/IndexMetaData.java @@ -1408,10 +1408,10 @@ public static IndexMetaData fromXContent(XContentParser parser) throws IOExcepti throw new IllegalArgumentException("Unexpected token " + token); } } - if (Assertions.ENABLED && Version.indexCreated(builder.settings).onOrAfter(Version.V_6_5_0)) { + if (Assertions.ENABLED) { assert mappingVersion : "mapping version should be present for indices created on or after 6.5.0"; } - if (Assertions.ENABLED && Version.indexCreated(builder.settings).onOrAfter(Version.V_6_5_0)) { + if (Assertions.ENABLED) { assert settingsVersion : "settings version should be present for indices created on or after 6.5.0"; } if (Assertions.ENABLED && Version.indexCreated(builder.settings).onOrAfter(Version.V_7_2_0)) { diff --git a/server/src/main/java/org/elasticsearch/cluster/metadata/IndexTemplateMetaData.java b/server/src/main/java/org/elasticsearch/cluster/metadata/IndexTemplateMetaData.java index 1c84e8ce9fc6a..93832e08cd048 100644 --- a/server/src/main/java/org/elasticsearch/cluster/metadata/IndexTemplateMetaData.java +++ b/server/src/main/java/org/elasticsearch/cluster/metadata/IndexTemplateMetaData.java @@ -20,7 +20,6 @@ import com.carrotsearch.hppc.cursors.ObjectCursor; import com.carrotsearch.hppc.cursors.ObjectObjectCursor; -import org.apache.logging.log4j.LogManager; import org.elasticsearch.ElasticsearchParseException; import org.elasticsearch.cluster.AbstractDiffable; import org.elasticsearch.cluster.Diff; @@ -32,7 +31,6 @@ import org.elasticsearch.common.compress.CompressedXContent; import org.elasticsearch.common.io.stream.StreamInput; import org.elasticsearch.common.io.stream.StreamOutput; -import org.elasticsearch.common.logging.DeprecationLogger; import org.elasticsearch.common.settings.Settings; import org.elasticsearch.common.util.set.Sets; import org.elasticsearch.common.xcontent.ToXContent; @@ -52,8 +50,6 @@ public class IndexTemplateMetaData extends AbstractDiffable { - private static final DeprecationLogger deprecationLogger = new DeprecationLogger(LogManager.getLogger(IndexTemplateMetaData.class)); - private final String name; private final int order; @@ -97,7 +93,7 @@ public IndexTemplateMetaData(String name, int order, Integer version, this.name = name; this.order = order; this.version = version; - this.patterns= patterns; + this.patterns = patterns; this.settings = settings; this.mappings = mappings; this.aliases = aliases; @@ -227,7 +223,7 @@ public void writeTo(StreamOutput out) throws IOException { public static class Builder { private static final Set VALID_FIELDS = Sets.newHashSet( - "template", "order", "mappings", "settings", "index_patterns", "aliases", "version"); + "order", "mappings", "settings", "index_patterns", "aliases", "version"); private String name; @@ -487,11 +483,7 @@ public static IndexTemplateMetaData fromXContent(XContentParser parser, String t builder.patterns(index_patterns); } } else if (token.isValue()) { - // Prior to 5.1.0, elasticsearch only supported a single index pattern called `template` (#21009) - if("template".equals(currentFieldName)) { - deprecationLogger.deprecated("Deprecated field [template] used, replaced by [index_patterns]"); - builder.patterns(Collections.singletonList(parser.text())); - } else if ("order".equals(currentFieldName)) { + if ("order".equals(currentFieldName)) { builder.order(parser.intValue()); } else if ("version".equals(currentFieldName)) { builder.version(parser.intValue()); diff --git a/server/src/main/java/org/elasticsearch/cluster/metadata/MetaData.java b/server/src/main/java/org/elasticsearch/cluster/metadata/MetaData.java index a9f751453f6fa..4c2eeec72a970 100644 --- a/server/src/main/java/org/elasticsearch/cluster/metadata/MetaData.java +++ b/server/src/main/java/org/elasticsearch/cluster/metadata/MetaData.java @@ -25,6 +25,7 @@ import org.apache.logging.log4j.LogManager; import org.apache.logging.log4j.Logger; import org.apache.lucene.util.CollectionUtil; +import org.elasticsearch.Version; import org.elasticsearch.action.AliasesRequest; import org.elasticsearch.cluster.ClusterState; import org.elasticsearch.cluster.ClusterState.FeatureAware; @@ -168,6 +169,7 @@ public interface Custom extends NamedDiffable, ToXContentFragment, Clust private final Settings transientSettings; private final Settings persistentSettings; private final Settings settings; + private final DiffableStringMap hashesOfConsistentSettings; private final ImmutableOpenMap indices; private final ImmutableOpenMap templates; private final ImmutableOpenMap customs; @@ -182,7 +184,7 @@ public interface Custom extends NamedDiffable, ToXContentFragment, Clust private final SortedMap aliasAndIndexLookup; MetaData(String clusterUUID, boolean clusterUUIDCommitted, long version, CoordinationMetaData coordinationMetaData, - Settings transientSettings, Settings persistentSettings, + Settings transientSettings, Settings persistentSettings, DiffableStringMap hashesOfConsistentSettings, ImmutableOpenMap indices, ImmutableOpenMap templates, ImmutableOpenMap customs, String[] allIndices, String[] allOpenIndices, String[] allClosedIndices, SortedMap aliasAndIndexLookup) { @@ -193,6 +195,7 @@ public interface Custom extends NamedDiffable, ToXContentFragment, Clust this.transientSettings = transientSettings; this.persistentSettings = persistentSettings; this.settings = Settings.builder().put(persistentSettings).put(transientSettings).build(); + this.hashesOfConsistentSettings = hashesOfConsistentSettings; this.indices = indices; this.customs = customs; this.templates = templates; @@ -244,6 +247,10 @@ public Settings persistentSettings() { return this.persistentSettings; } + public Map hashesOfConsistentSettings() { + return this.hashesOfConsistentSettings; + } + public CoordinationMetaData coordinationMetaData() { return this.coordinationMetaData; } @@ -360,38 +367,6 @@ private ImmutableOpenMap> findAliases(final String[] return mapBuilder.build(); } - /** - * Checks if at least one of the specified aliases exists in the specified concrete indices. Wildcards are supported in the - * alias names for partial matches. - * - * @param aliases The names of the index aliases to find - * @param concreteIndices The concrete indexes the index aliases must point to order to be returned. - * @return whether at least one of the specified aliases exists in one of the specified concrete indices. - */ - public boolean hasAliases(final String[] aliases, String[] concreteIndices) { - assert aliases != null; - assert concreteIndices != null; - if (concreteIndices.length == 0) { - return false; - } - - Iterable intersection = HppcMaps.intersection(ObjectHashSet.from(concreteIndices), indices.keys()); - for (String index : intersection) { - IndexMetaData indexMetaData = indices.get(index); - List filteredValues = new ArrayList<>(); - for (ObjectCursor cursor : indexMetaData.getAliases().values()) { - AliasMetaData value = cursor.value; - if (Regex.simpleMatch(aliases, value.alias())) { - filteredValues.add(value); - } - } - if (!filteredValues.isEmpty()) { - return true; - } - } - return false; - } - /** * Finds all mappings for types and concrete indices. Types are expanded to include all types that match the glob * patterns in the types array. Empty types array, null or {"_all"} will be expanded to all types available for @@ -765,6 +740,9 @@ public static boolean isGlobalStateEquals(MetaData metaData1, MetaData metaData2 if (!metaData1.persistentSettings.equals(metaData2.persistentSettings)) { return false; } + if (!metaData1.hashesOfConsistentSettings.equals(metaData2.hashesOfConsistentSettings)) { + return false; + } if (!metaData1.templates.equals(metaData2.templates())) { return false; } @@ -819,6 +797,7 @@ private static class MetaDataDiff implements Diff { private CoordinationMetaData coordinationMetaData; private Settings transientSettings; private Settings persistentSettings; + private Diff hashesOfConsistentSettings; private Diff> indices; private Diff> templates; private Diff> customs; @@ -830,6 +809,7 @@ private static class MetaDataDiff implements Diff { coordinationMetaData = after.coordinationMetaData; transientSettings = after.transientSettings; persistentSettings = after.persistentSettings; + hashesOfConsistentSettings = after.hashesOfConsistentSettings.diff(before.hashesOfConsistentSettings); indices = DiffableUtils.diff(before.indices, after.indices, DiffableUtils.getStringKeySerializer()); templates = DiffableUtils.diff(before.templates, after.templates, DiffableUtils.getStringKeySerializer()); customs = DiffableUtils.diff(before.customs, after.customs, DiffableUtils.getStringKeySerializer(), CUSTOM_VALUE_SERIALIZER); @@ -842,6 +822,11 @@ private static class MetaDataDiff implements Diff { coordinationMetaData = new CoordinationMetaData(in); transientSettings = Settings.readSettingsFromStream(in); persistentSettings = Settings.readSettingsFromStream(in); + if (in.getVersion().onOrAfter(Version.V_7_3_0)) { + hashesOfConsistentSettings = DiffableStringMap.readDiffFrom(in); + } else { + hashesOfConsistentSettings = DiffableStringMap.DiffableStringMapDiff.EMPTY; + } indices = DiffableUtils.readImmutableOpenMapDiff(in, DiffableUtils.getStringKeySerializer(), IndexMetaData::readFrom, IndexMetaData::readDiffFrom); templates = DiffableUtils.readImmutableOpenMapDiff(in, DiffableUtils.getStringKeySerializer(), IndexTemplateMetaData::readFrom, @@ -857,6 +842,9 @@ public void writeTo(StreamOutput out) throws IOException { coordinationMetaData.writeTo(out); Settings.writeSettingsToStream(transientSettings, out); Settings.writeSettingsToStream(persistentSettings, out); + if (out.getVersion().onOrAfter(Version.V_7_3_0)) { + hashesOfConsistentSettings.writeTo(out); + } indices.writeTo(out); templates.writeTo(out); customs.writeTo(out); @@ -871,6 +859,7 @@ public MetaData apply(MetaData part) { builder.coordinationMetaData(coordinationMetaData); builder.transientSettings(transientSettings); builder.persistentSettings(persistentSettings); + builder.hashesOfConsistentSettings(hashesOfConsistentSettings.apply(part.hashesOfConsistentSettings)); builder.indices(indices.apply(part.indices)); builder.templates(templates.apply(part.templates)); builder.customs(customs.apply(part.customs)); @@ -886,6 +875,9 @@ public static MetaData readFrom(StreamInput in) throws IOException { builder.coordinationMetaData(new CoordinationMetaData(in)); builder.transientSettings(readSettingsFromStream(in)); builder.persistentSettings(readSettingsFromStream(in)); + if (in.getVersion().onOrAfter(Version.V_7_3_0)) { + builder.hashesOfConsistentSettings(new DiffableStringMap(in)); + } int size = in.readVInt(); for (int i = 0; i < size; i++) { builder.put(IndexMetaData.readFrom(in), false); @@ -910,6 +902,9 @@ public void writeTo(StreamOutput out) throws IOException { coordinationMetaData.writeTo(out); writeSettingsToStream(transientSettings, out); writeSettingsToStream(persistentSettings, out); + if (out.getVersion().onOrAfter(Version.V_7_3_0)) { + hashesOfConsistentSettings.writeTo(out); + } out.writeVInt(indices.size()); for (IndexMetaData indexMetaData : this) { indexMetaData.writeTo(out); @@ -950,6 +945,7 @@ public static class Builder { private CoordinationMetaData coordinationMetaData = CoordinationMetaData.EMPTY_META_DATA; private Settings transientSettings = Settings.Builder.EMPTY_SETTINGS; private Settings persistentSettings = Settings.Builder.EMPTY_SETTINGS; + private DiffableStringMap hashesOfConsistentSettings = new DiffableStringMap(Collections.emptyMap()); private final ImmutableOpenMap.Builder indices; private final ImmutableOpenMap.Builder templates; @@ -969,6 +965,7 @@ public Builder(MetaData metaData) { this.coordinationMetaData = metaData.coordinationMetaData; this.transientSettings = metaData.transientSettings; this.persistentSettings = metaData.persistentSettings; + this.hashesOfConsistentSettings = metaData.hashesOfConsistentSettings; this.version = metaData.version; this.indices = ImmutableOpenMap.builder(metaData.indices); this.templates = ImmutableOpenMap.builder(metaData.templates); @@ -1132,6 +1129,20 @@ public Builder persistentSettings(Settings settings) { return this; } + public DiffableStringMap hashesOfConsistentSettings() { + return this.hashesOfConsistentSettings; + } + + public Builder hashesOfConsistentSettings(DiffableStringMap hashesOfConsistentSettings) { + this.hashesOfConsistentSettings = hashesOfConsistentSettings; + return this; + } + + public Builder hashesOfConsistentSettings(Map hashesOfConsistentSettings) { + this.hashesOfConsistentSettings = new DiffableStringMap(hashesOfConsistentSettings); + return this; + } + public Builder version(long version) { this.version = version; return this; @@ -1205,8 +1216,8 @@ public MetaData build() { String[] allClosedIndicesArray = allClosedIndices.toArray(new String[allClosedIndices.size()]); return new MetaData(clusterUUID, clusterUUIDCommitted, version, coordinationMetaData, transientSettings, persistentSettings, - indices.build(), templates.build(), customs.build(), allIndicesArray, allOpenIndicesArray, allClosedIndicesArray, - aliasAndIndexLookup); + hashesOfConsistentSettings, indices.build(), templates.build(), customs.build(), allIndicesArray, allOpenIndicesArray, + allClosedIndicesArray, aliasAndIndexLookup); } private SortedMap buildAliasAndIndexLookup() { @@ -1330,6 +1341,8 @@ public static MetaData fromXContent(XContentParser parser) throws IOException { while ((token = parser.nextToken()) != XContentParser.Token.END_OBJECT) { builder.put(IndexMetaData.Builder.fromXContent(parser), false); } + } else if ("hashes_of_consistent_settings".equals(currentFieldName)) { + builder.hashesOfConsistentSettings(parser.mapStrings()); } else if ("templates".equals(currentFieldName)) { while ((token = parser.nextToken()) != XContentParser.Token.END_OBJECT) { builder.put(IndexTemplateMetaData.Builder.fromXContent(parser, parser.currentName())); diff --git a/server/src/main/java/org/elasticsearch/cluster/metadata/MetaDataIndexStateService.java b/server/src/main/java/org/elasticsearch/cluster/metadata/MetaDataIndexStateService.java index ef4583e98e544..d37a134c0a77c 100644 --- a/server/src/main/java/org/elasticsearch/cluster/metadata/MetaDataIndexStateService.java +++ b/server/src/main/java/org/elasticsearch/cluster/metadata/MetaDataIndexStateService.java @@ -389,11 +389,26 @@ private void sendVerifyShardBeforeCloseRequest(final IndexShardRoutingTable shar } final TaskId parentTaskId = new TaskId(clusterService.localNode().getId(), request.taskId()); final TransportVerifyShardBeforeCloseAction.ShardRequest shardRequest = - new TransportVerifyShardBeforeCloseAction.ShardRequest(shardId, closingBlock, parentTaskId); + new TransportVerifyShardBeforeCloseAction.ShardRequest(shardId, closingBlock, true, parentTaskId); if (request.ackTimeout() != null) { shardRequest.timeout(request.ackTimeout()); } - transportVerifyShardBeforeCloseAction.execute(shardRequest, listener); + transportVerifyShardBeforeCloseAction.execute(shardRequest, new ActionListener<>() { + @Override + public void onResponse(ReplicationResponse replicationResponse) { + final TransportVerifyShardBeforeCloseAction.ShardRequest shardRequest = + new TransportVerifyShardBeforeCloseAction.ShardRequest(shardId, closingBlock, false, parentTaskId); + if (request.ackTimeout() != null) { + shardRequest.timeout(request.ackTimeout()); + } + transportVerifyShardBeforeCloseAction.execute(shardRequest, listener); + } + + @Override + public void onFailure(Exception e) { + listener.onFailure(e); + } + }); } } diff --git a/server/src/main/java/org/elasticsearch/cluster/metadata/MetaDataIndexUpgradeService.java b/server/src/main/java/org/elasticsearch/cluster/metadata/MetaDataIndexUpgradeService.java index 483835f633e7e..72fdea8e6092d 100644 --- a/server/src/main/java/org/elasticsearch/cluster/metadata/MetaDataIndexUpgradeService.java +++ b/server/src/main/java/org/elasticsearch/cluster/metadata/MetaDataIndexUpgradeService.java @@ -195,7 +195,7 @@ public Set> entrySet() { } }; try (IndexAnalyzers fakeIndexAnalzyers = - new IndexAnalyzers(indexSettings, analyzerMap, analyzerMap, analyzerMap)) { + new IndexAnalyzers(analyzerMap, analyzerMap, analyzerMap)) { MapperService mapperService = new MapperService(indexSettings, fakeIndexAnalzyers, xContentRegistry, similarityService, mapperRegistry, () -> null); mapperService.merge(indexMetaData, MapperService.MergeReason.MAPPING_RECOVERY); diff --git a/server/src/main/java/org/elasticsearch/cluster/node/DiscoveryNode.java b/server/src/main/java/org/elasticsearch/cluster/node/DiscoveryNode.java index afab4af320416..6e9ad85aec56c 100644 --- a/server/src/main/java/org/elasticsearch/cluster/node/DiscoveryNode.java +++ b/server/src/main/java/org/elasticsearch/cluster/node/DiscoveryNode.java @@ -32,11 +32,14 @@ import java.io.IOException; import java.util.Collections; -import java.util.EnumSet; import java.util.HashMap; +import java.util.HashSet; +import java.util.List; import java.util.Map; import java.util.Set; +import java.util.function.Function; import java.util.function.Predicate; +import java.util.stream.Collectors; /** @@ -78,8 +81,7 @@ public static boolean isIngestNode(Settings settings) { private final TransportAddress address; private final Map attributes; private final Version version; - private final Set roles; - + private final Set roles; /** * Creates a new {@link DiscoveryNode} @@ -95,7 +97,7 @@ public static boolean isIngestNode(Settings settings) { * @param version the version of the node */ public DiscoveryNode(final String id, TransportAddress address, Version version) { - this(id, address, Collections.emptyMap(), EnumSet.allOf(Role.class), version); + this(id, address, Collections.emptyMap(), DiscoveryNodeRole.BUILT_IN_ROLES, version); } /** @@ -113,7 +115,7 @@ public DiscoveryNode(final String id, TransportAddress address, Version version) * @param roles node roles * @param version the version of the node */ - public DiscoveryNode(String id, TransportAddress address, Map attributes, Set roles, + public DiscoveryNode(String id, TransportAddress address, Map attributes, Set roles, Version version) { this("", id, address, attributes, roles, version); } @@ -135,7 +137,7 @@ public DiscoveryNode(String id, TransportAddress address, Map at * @param version the version of the node */ public DiscoveryNode(String nodeName, String nodeId, TransportAddress address, - Map attributes, Set roles, Version version) { + Map attributes, Set roles, Version version) { this(nodeName, nodeId, UUIDs.randomBase64UUID(), address.address().getHostString(), address.getAddress(), address, attributes, roles, version); } @@ -159,7 +161,7 @@ public DiscoveryNode(String nodeName, String nodeId, TransportAddress address, * @param version the version of the node */ public DiscoveryNode(String nodeName, String nodeId, String ephemeralId, String hostName, String hostAddress, - TransportAddress address, Map attributes, Set roles, Version version) { + TransportAddress address, Map attributes, Set roles, Version version) { if (nodeName != null) { this.nodeName = nodeName.intern(); } else { @@ -178,37 +180,27 @@ public DiscoveryNode(String nodeName, String nodeId, String ephemeralId, String this.attributes = Collections.unmodifiableMap(attributes); //verify that no node roles are being provided as attributes Predicate> predicate = (attrs) -> { - for (Role role : Role.values()) { - assert attrs.containsKey(role.getRoleName()) == false; + boolean success = true; + for (final DiscoveryNodeRole role : DiscoveryNode.roleNameToPossibleRoles.values()) { + success &= attrs.containsKey(role.roleName()) == false; + assert success : role.roleName(); } - return true; + return success; }; - assert predicate.test(attributes); - Set rolesSet = EnumSet.noneOf(Role.class); - rolesSet.addAll(roles); - this.roles = Collections.unmodifiableSet(rolesSet); + assert predicate.test(attributes) : attributes; + this.roles = Set.copyOf(roles); } /** Creates a DiscoveryNode representing the local node. */ public static DiscoveryNode createLocal(Settings settings, TransportAddress publishAddress, String nodeId) { Map attributes = Node.NODE_ATTRIBUTES.getAsMap(settings); - Set roles = getRolesFromSettings(settings); + Set roles = getRolesFromSettings(settings); return new DiscoveryNode(Node.NODE_NAME_SETTING.get(settings), nodeId, publishAddress, attributes, roles, Version.CURRENT); } /** extract node roles from the given settings */ - public static Set getRolesFromSettings(Settings settings) { - Set roles = EnumSet.noneOf(Role.class); - if (Node.NODE_INGEST_SETTING.get(settings)) { - roles.add(Role.INGEST); - } - if (Node.NODE_MASTER_SETTING.get(settings)) { - roles.add(Role.MASTER); - } - if (Node.NODE_DATA_SETTING.get(settings)) { - roles.add(Role.DATA); - } - return roles; + public static Set getRolesFromSettings(final Settings settings) { + return roleNameToPossibleRoles.values().stream().filter(s -> s.roleSetting().get(settings)).collect(Collectors.toUnmodifiableSet()); } /** @@ -229,10 +221,41 @@ public DiscoveryNode(StreamInput in) throws IOException { this.attributes.put(in.readString(), in.readString()); } int rolesSize = in.readVInt(); - this.roles = EnumSet.noneOf(Role.class); - for (int i = 0; i < rolesSize; i++) { - this.roles.add(in.readEnum(Role.class)); + final Set roles = new HashSet<>(rolesSize); + if (in.getVersion().onOrAfter(Version.V_7_3_0)) { + for (int i = 0; i < rolesSize; i++) { + final String roleName = in.readString(); + final String roleNameAbbreviation = in.readString(); + final DiscoveryNodeRole role = roleNameToPossibleRoles.get(roleName); + if (role == null) { + roles.add(new DiscoveryNodeRole.UnknownRole(roleName, roleNameAbbreviation)); + } else { + assert roleName.equals(role.roleName()) : "role name [" + roleName + "] does not match role [" + role.roleName() + "]"; + assert roleNameAbbreviation.equals(role.roleNameAbbreviation()) + : "role name abbreviation [" + roleName + "] does not match role [" + role.roleNameAbbreviation() + "]"; + roles.add(role); + } + } + } else { + // an old node will only send us legacy roles since pluggable roles is a new concept + for (int i = 0; i < rolesSize; i++) { + final LegacyRole legacyRole = in.readEnum(LegacyRole.class); + switch (legacyRole) { + case MASTER: + roles.add(DiscoveryNodeRole.MASTER_ROLE); + break; + case DATA: + roles.add(DiscoveryNodeRole.DATA_ROLE); + break; + case INGEST: + roles.add(DiscoveryNodeRole.INGEST_ROLE); + break; + default: + throw new AssertionError(legacyRole.roleName()); + } + } } + this.roles = Set.copyOf(roles); this.version = Version.readVersion(in); } @@ -249,9 +272,26 @@ public void writeTo(StreamOutput out) throws IOException { out.writeString(entry.getKey()); out.writeString(entry.getValue()); } - out.writeVInt(roles.size()); - for (Role role : roles) { - out.writeEnum(role); + if (out.getVersion().onOrAfter(Version.V_7_3_0)) { + out.writeVInt(roles.size()); + for (final DiscoveryNodeRole role : roles) { + out.writeString(role.roleName()); + out.writeString(role.roleNameAbbreviation()); + } + } else { + // an old node will only understand legacy roles since pluggable roles is a new concept + final List rolesToWrite = + roles.stream().filter(DiscoveryNodeRole.BUILT_IN_ROLES::contains).collect(Collectors.toUnmodifiableList()); + out.writeVInt(rolesToWrite.size()); + for (final DiscoveryNodeRole role : rolesToWrite) { + if (role == DiscoveryNodeRole.MASTER_ROLE) { + out.writeEnum(LegacyRole.MASTER); + } else if (role == DiscoveryNodeRole.DATA_ROLE) { + out.writeEnum(LegacyRole.DATA); + } else if (role == DiscoveryNodeRole.INGEST_ROLE) { + out.writeEnum(LegacyRole.INGEST); + } + } } Version.writeVersion(version, out); } @@ -299,28 +339,28 @@ public Map getAttributes() { * Should this node hold data (shards) or not. */ public boolean isDataNode() { - return roles.contains(Role.DATA); + return roles.contains(DiscoveryNodeRole.DATA_ROLE); } /** * Can this node become master or not. */ public boolean isMasterNode() { - return roles.contains(Role.MASTER); + return roles.contains(DiscoveryNodeRole.MASTER_ROLE); } /** * Returns a boolean that tells whether this an ingest node or not */ public boolean isIngestNode() { - return roles.contains(Role.INGEST); + return roles.contains(DiscoveryNodeRole.INGEST_ROLE); } /** * Returns a set of all the roles that the node fulfills. * If the node doesn't have any specific role, the set is returned empty, which means that the node is a coordinating only node. */ - public Set getRoles() { + public Set getRoles() { return roles; } @@ -368,6 +408,11 @@ public String toString() { sb.append('{').append(ephemeralId).append('}'); sb.append('{').append(hostName).append('}'); sb.append('{').append(address).append('}'); + if (roles.isEmpty() == false) { + sb.append('{'); + roles.stream().map(DiscoveryNodeRole::roleNameAbbreviation).sorted().forEach(sb::append); + sb.append('}'); + } if (!attributes.isEmpty()) { sb.append(attributes); } @@ -391,29 +436,43 @@ public XContentBuilder toXContent(XContentBuilder builder, Params params) throws return builder; } + private static Map roleNameToPossibleRoles; + + public static void setPossibleRoles(final Set possibleRoles) { + final Map roleNameToPossibleRoles = + possibleRoles.stream().collect(Collectors.toUnmodifiableMap(DiscoveryNodeRole::roleName, Function.identity())); + // collect the abbreviation names into a map to ensure that there are not any duplicate abbreviations + final Map roleNameAbbreviationToPossibleRoles = roleNameToPossibleRoles.values() + .stream() + .collect(Collectors.toUnmodifiableMap(DiscoveryNodeRole::roleNameAbbreviation, Function.identity())); + assert roleNameToPossibleRoles.size() == roleNameAbbreviationToPossibleRoles.size() : + "roles by name [" + roleNameToPossibleRoles + "], roles by name abbreviation [" + roleNameAbbreviationToPossibleRoles + "]"; + DiscoveryNode.roleNameToPossibleRoles = roleNameToPossibleRoles; + } + + public static Set getPossibleRoleNames() { + return roleNameToPossibleRoles.keySet(); + } + /** * Enum that holds all the possible roles that that a node can fulfill in a cluster. * Each role has its name and a corresponding abbreviation used by cat apis. */ - public enum Role { - MASTER("master", "m"), - DATA("data", "d"), - INGEST("ingest", "i"); + private enum LegacyRole { + MASTER("master"), + DATA("data"), + INGEST("ingest"); private final String roleName; - private final String abbreviation; - Role(String roleName, String abbreviation) { + LegacyRole(final String roleName) { this.roleName = roleName; - this.abbreviation = abbreviation; } - public String getRoleName() { + public String roleName() { return roleName; } - public String getAbbreviation() { - return abbreviation; - } } + } diff --git a/server/src/main/java/org/elasticsearch/cluster/node/DiscoveryNodeRole.java b/server/src/main/java/org/elasticsearch/cluster/node/DiscoveryNodeRole.java new file mode 100644 index 0000000000000..4bdf27793f66d --- /dev/null +++ b/server/src/main/java/org/elasticsearch/cluster/node/DiscoveryNodeRole.java @@ -0,0 +1,167 @@ +/* + * 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.cluster.node; + +import org.elasticsearch.common.settings.Setting; +import org.elasticsearch.node.Node; + +import java.util.Objects; +import java.util.Set; + +/** + * Represents a node role. + */ +public abstract class DiscoveryNodeRole { + + private final String roleName; + + /** + * The name of the role. + * + * @return the role name + */ + public final String roleName() { + return roleName; + } + + private final String roleNameAbbreviation; + + /** + * The abbreviation of the name of the role. This is used in the cat nodes API to display an abbreviated version of the name of the + * role. + * + * @return the role name abbreviation + */ + public final String roleNameAbbreviation() { + return roleNameAbbreviation; + } + + private final boolean isKnownRole; + + /** + * Whether this role is known by this node, or is an {@link DiscoveryNodeRole.UnknownRole}. + */ + public final boolean isKnownRole() { + return isKnownRole; + } + + protected DiscoveryNodeRole(final String roleName, final String roleNameAbbreviation) { + this(true, roleName, roleNameAbbreviation); + } + + private DiscoveryNodeRole(final boolean isKnownRole, final String roleName, final String roleNameAbbreviation) { + this.isKnownRole = isKnownRole; + this.roleName = Objects.requireNonNull(roleName); + this.roleNameAbbreviation = Objects.requireNonNull(roleNameAbbreviation); + } + + protected abstract Setting roleSetting(); + + @Override + public final boolean equals(Object o) { + if (this == o) return true; + if (o == null || getClass() != o.getClass()) return false; + DiscoveryNodeRole that = (DiscoveryNodeRole) o; + return roleName.equals(that.roleName) && + roleNameAbbreviation.equals(that.roleNameAbbreviation) && + isKnownRole == that.isKnownRole; + } + + @Override + public final int hashCode() { + return Objects.hash(isKnownRole, roleName(), roleNameAbbreviation()); + } + + @Override + public final String toString() { + return "DiscoveryNodeRole{" + + "roleName='" + roleName + '\'' + + ", roleNameAbbreviation='" + roleNameAbbreviation + '\'' + + (isKnownRole ? "" : ", isKnownRole=false") + + '}'; + } + + /** + * Represents the role for a data node. + */ + public static final DiscoveryNodeRole DATA_ROLE = new DiscoveryNodeRole("data", "d") { + + @Override + protected Setting roleSetting() { + return Node.NODE_DATA_SETTING; + } + + }; + + /** + * Represents the role for an ingest node. + */ + public static final DiscoveryNodeRole INGEST_ROLE = new DiscoveryNodeRole("ingest", "i") { + + @Override + protected Setting roleSetting() { + return Node.NODE_INGEST_SETTING; + } + + }; + + /** + * Represents the role for a master-eligible node. + */ + public static final DiscoveryNodeRole MASTER_ROLE = new DiscoveryNodeRole("master", "m") { + + @Override + protected Setting roleSetting() { + return Node.NODE_MASTER_SETTING; + } + + }; + + /** + * The built-in node roles. + */ + public static Set BUILT_IN_ROLES = Set.of(DATA_ROLE, INGEST_ROLE, MASTER_ROLE); + + /** + * Represents an unknown role. This can occur if a newer version adds a role that an older version does not know about, or a newer + * version removes a role that an older version knows about. + */ + static class UnknownRole extends DiscoveryNodeRole { + + /** + * Construct an unknown role with the specified role name and role name abbreviation. + * + * @param roleName the role name + * @param roleNameAbbreviation the role name abbreviation + */ + UnknownRole(final String roleName, final String roleNameAbbreviation) { + super(false, roleName, roleNameAbbreviation); + } + + @Override + protected Setting roleSetting() { + // since this setting is not registered, it will always return false when testing if the local node has the role + assert false; + return Setting.boolSetting("node. " + roleName(), false, Setting.Property.NodeScope); + } + + } + +} diff --git a/server/src/main/java/org/elasticsearch/cluster/node/DiscoveryNodes.java b/server/src/main/java/org/elasticsearch/cluster/node/DiscoveryNodes.java index 02f4d5d93bf4e..8fb325d594707 100644 --- a/server/src/main/java/org/elasticsearch/cluster/node/DiscoveryNodes.java +++ b/server/src/main/java/org/elasticsearch/cluster/node/DiscoveryNodes.java @@ -32,6 +32,7 @@ import org.elasticsearch.common.io.stream.StreamOutput; import org.elasticsearch.common.regex.Regex; import org.elasticsearch.common.transport.TransportAddress; +import org.elasticsearch.common.util.set.Sets; import java.io.IOException; import java.util.ArrayList; @@ -353,19 +354,19 @@ public String[] resolveNodes(String... nodes) { if (index != -1) { String matchAttrName = nodeId.substring(0, index); String matchAttrValue = nodeId.substring(index + 1); - if (DiscoveryNode.Role.DATA.getRoleName().equals(matchAttrName)) { + if (DiscoveryNodeRole.DATA_ROLE.roleName().equals(matchAttrName)) { if (Booleans.parseBoolean(matchAttrValue, true)) { resolvedNodesIds.addAll(dataNodes.keys()); } else { resolvedNodesIds.removeAll(dataNodes.keys()); } - } else if (DiscoveryNode.Role.MASTER.getRoleName().equals(matchAttrName)) { + } else if (DiscoveryNodeRole.MASTER_ROLE.roleName().equals(matchAttrName)) { if (Booleans.parseBoolean(matchAttrValue, true)) { resolvedNodesIds.addAll(masterNodes.keys()); } else { resolvedNodesIds.removeAll(masterNodes.keys()); } - } else if (DiscoveryNode.Role.INGEST.getRoleName().equals(matchAttrName)) { + } else if (DiscoveryNodeRole.INGEST_ROLE.roleName().equals(matchAttrName)) { if (Booleans.parseBoolean(matchAttrValue, true)) { resolvedNodesIds.addAll(ingestNodes.keys()); } else { @@ -378,6 +379,17 @@ public String[] resolveNodes(String... nodes) { resolvedNodesIds.removeAll(getCoordinatingOnlyNodes().keys()); } } else { + for (DiscoveryNode node : this) { + for (DiscoveryNodeRole role : Sets.difference(node.getRoles(), DiscoveryNodeRole.BUILT_IN_ROLES)) { + if (role.roleName().equals(matchAttrName)) { + if (Booleans.parseBoolean(matchAttrValue, true)) { + resolvedNodesIds.add(node.getId()); + } else { + resolvedNodesIds.remove(node.getId()); + } + } + } + } for (DiscoveryNode node : this) { for (Map.Entry entry : node.getAttributes().entrySet()) { String attrName = entry.getKey(); diff --git a/server/src/main/java/org/elasticsearch/cluster/routing/BatchedRerouteService.java b/server/src/main/java/org/elasticsearch/cluster/routing/BatchedRerouteService.java new file mode 100644 index 0000000000000..4ed4caadabe49 --- /dev/null +++ b/server/src/main/java/org/elasticsearch/cluster/routing/BatchedRerouteService.java @@ -0,0 +1,135 @@ +/* + * 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.cluster.routing; + +import org.apache.logging.log4j.LogManager; +import org.apache.logging.log4j.Logger; +import org.apache.logging.log4j.message.ParameterizedMessage; +import org.elasticsearch.ElasticsearchException; +import org.elasticsearch.action.ActionListener; +import org.elasticsearch.action.support.PlainListenableActionFuture; +import org.elasticsearch.cluster.ClusterState; +import org.elasticsearch.cluster.ClusterStateUpdateTask; +import org.elasticsearch.cluster.NotMasterException; +import org.elasticsearch.cluster.service.ClusterService; +import org.elasticsearch.common.Nullable; +import org.elasticsearch.common.Priority; + +import java.util.function.BiFunction; + +/** + * A {@link BatchedRerouteService} is a {@link RerouteService} that batches together reroute requests to avoid unnecessary extra reroutes. + * This component only does meaningful work on the elected master node. Reroute requests will fail with a {@link NotMasterException} on + * other nodes. + */ +public class BatchedRerouteService implements RerouteService { + private static final Logger logger = LogManager.getLogger(BatchedRerouteService.class); + + private static final String CLUSTER_UPDATE_TASK_SOURCE = "cluster_reroute"; + + private final ClusterService clusterService; + private final BiFunction reroute; + + private final Object mutex = new Object(); + @Nullable // null if no reroute is currently pending + private PlainListenableActionFuture pendingRerouteListeners; + + /** + * @param reroute Function that computes the updated cluster state after it has been rerouted. + */ + public BatchedRerouteService(ClusterService clusterService, BiFunction reroute) { + this.clusterService = clusterService; + this.reroute = reroute; + } + + /** + * Initiates a reroute. + */ + @Override + public final void reroute(String reason, ActionListener listener) { + final PlainListenableActionFuture currentListeners; + synchronized (mutex) { + if (pendingRerouteListeners != null) { + logger.trace("already has pending reroute, adding [{}] to batch", reason); + pendingRerouteListeners.addListener(listener); + return; + } + currentListeners = PlainListenableActionFuture.newListenableFuture(); + currentListeners.addListener(listener); + pendingRerouteListeners = currentListeners; + } + logger.trace("rerouting [{}]", reason); + try { + clusterService.submitStateUpdateTask(CLUSTER_UPDATE_TASK_SOURCE + "(" + reason + ")", + new ClusterStateUpdateTask(Priority.HIGH) { + @Override + public ClusterState execute(ClusterState currentState) { + synchronized (mutex) { + assert pendingRerouteListeners == currentListeners; + pendingRerouteListeners = null; + } + return reroute.apply(currentState, reason); + } + + @Override + public void onNoLongerMaster(String source) { + synchronized (mutex) { + if (pendingRerouteListeners == currentListeners) { + pendingRerouteListeners = null; + } + } + currentListeners.onFailure(new NotMasterException("delayed reroute [" + reason + "] cancelled")); + // no big deal, the new master will reroute again + } + + @Override + public void onFailure(String source, Exception e) { + synchronized (mutex) { + if (pendingRerouteListeners == currentListeners) { + pendingRerouteListeners = null; + } + } + final ClusterState state = clusterService.state(); + if (logger.isTraceEnabled()) { + logger.error(() -> new ParameterizedMessage("unexpected failure during [{}], current state:\n{}", + source, state), e); + } else { + logger.error(() -> new ParameterizedMessage("unexpected failure during [{}], current state version [{}]", + source, state.version()), e); + } + currentListeners.onFailure(new ElasticsearchException("delayed reroute [" + reason + "] failed", e)); + } + + @Override + public void clusterStateProcessed(String source, ClusterState oldState, ClusterState newState) { + currentListeners.onResponse(null); + } + }); + } catch (Exception e) { + synchronized (mutex) { + assert pendingRerouteListeners == currentListeners; + pendingRerouteListeners = null; + } + ClusterState state = clusterService.state(); + logger.warn(() -> new ParameterizedMessage("failed to reroute routing table, current state:\n{}", state), e); + currentListeners.onFailure(new ElasticsearchException("delayed reroute [" + reason + "] could not be submitted", e)); + } + } +} diff --git a/server/src/main/java/org/elasticsearch/cluster/routing/IndexShardRoutingTable.java b/server/src/main/java/org/elasticsearch/cluster/routing/IndexShardRoutingTable.java index ca3661f3e6f9c..5a98e9456f43d 100644 --- a/server/src/main/java/org/elasticsearch/cluster/routing/IndexShardRoutingTable.java +++ b/server/src/main/java/org/elasticsearch/cluster/routing/IndexShardRoutingTable.java @@ -105,6 +105,10 @@ public class IndexShardRoutingTable implements Iterable { // create the target initializing shard routing on the node the shard is relocating to allInitializingShards.add(shard.getTargetRelocatingShard()); allAllocationIds.add(shard.getTargetRelocatingShard().allocationId().getId()); + + assert shard.assignedToNode() : "relocating from unassigned " + shard; + assert shard.getTargetRelocatingShard().assignedToNode() : "relocating to unassigned " + shard.getTargetRelocatingShard(); + assignedShards.add(shard.getTargetRelocatingShard()); } if (shard.assignedToNode()) { assignedShards.add(shard); @@ -211,7 +215,7 @@ public List getActiveShards() { } /** - * Returns a {@link List} of assigned shards + * Returns a {@link List} of assigned shards, including relocation targets * * @return a {@link List} of shards */ @@ -518,11 +522,6 @@ public ShardRouting getByAllocationId(String allocationId) { if (shardRouting.allocationId().getId().equals(allocationId)) { return shardRouting; } - if (shardRouting.relocating()) { - if (shardRouting.getTargetRelocatingShard().allocationId().getId().equals(allocationId)) { - return shardRouting.getTargetRelocatingShard(); - } - } } return null; } diff --git a/server/src/main/java/org/elasticsearch/cluster/routing/LazilyInitializedRerouteService.java b/server/src/main/java/org/elasticsearch/cluster/routing/LazilyInitializedRerouteService.java new file mode 100644 index 0000000000000..672d6dbc3a08c --- /dev/null +++ b/server/src/main/java/org/elasticsearch/cluster/routing/LazilyInitializedRerouteService.java @@ -0,0 +1,42 @@ +/* + * 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.cluster.routing; + +import org.apache.lucene.util.SetOnce; +import org.elasticsearch.action.ActionListener; + +/** + * A {@link RerouteService} that can be initialized lazily. The real reroute service, {@link BatchedRerouteService}, depends on components + * constructed quite late in the construction of the node, but other components constructed earlier eventually need access to the reroute + * service too. + */ +public class LazilyInitializedRerouteService implements RerouteService { + + private final SetOnce delegate = new SetOnce<>(); + + @Override + public void reroute(String reason, ActionListener listener) { + assert delegate.get() != null; + delegate.get().reroute(reason, listener); + } + + public void setRerouteService(RerouteService rerouteService) { + delegate.set(rerouteService); + } +} diff --git a/server/src/main/java/org/elasticsearch/cluster/routing/RerouteService.java b/server/src/main/java/org/elasticsearch/cluster/routing/RerouteService.java new file mode 100644 index 0000000000000..11a49322e10fa --- /dev/null +++ b/server/src/main/java/org/elasticsearch/cluster/routing/RerouteService.java @@ -0,0 +1,29 @@ +/* + * 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.cluster.routing; + +import org.elasticsearch.action.ActionListener; + +/** + * Asynchronously performs a cluster reroute, updating any shard states and rebalancing the cluster if appropriate. + */ +@FunctionalInterface +public interface RerouteService { + void reroute(String reason, ActionListener listener); +} diff --git a/server/src/main/java/org/elasticsearch/cluster/routing/RoutingService.java b/server/src/main/java/org/elasticsearch/cluster/routing/RoutingService.java deleted file mode 100644 index 89e19e02b30ed..0000000000000 --- a/server/src/main/java/org/elasticsearch/cluster/routing/RoutingService.java +++ /dev/null @@ -1,122 +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.cluster.routing; - -import org.apache.logging.log4j.LogManager; -import org.apache.logging.log4j.Logger; -import org.apache.logging.log4j.message.ParameterizedMessage; -import org.elasticsearch.cluster.ClusterChangedEvent; -import org.elasticsearch.cluster.ClusterState; -import org.elasticsearch.cluster.ClusterStateUpdateTask; -import org.elasticsearch.cluster.routing.allocation.AllocationService; -import org.elasticsearch.cluster.service.ClusterService; -import org.elasticsearch.common.Priority; -import org.elasticsearch.common.component.AbstractLifecycleComponent; -import org.elasticsearch.common.inject.Inject; - -import java.util.concurrent.atomic.AtomicBoolean; - -/** - * A {@link RoutingService} listens to clusters state. When this service - * receives a {@link ClusterChangedEvent} the cluster state will be verified and - * the routing tables might be updated. - *

    - * Note: The {@link RoutingService} is responsible for cluster wide operations - * that include modifications to the cluster state. Such an operation can only - * be performed on the clusters master node. Unless the local node this service - * is running on is the clusters master node this service will not perform any - * actions. - *

    - */ -public class RoutingService extends AbstractLifecycleComponent { - private static final Logger logger = LogManager.getLogger(RoutingService.class); - - private static final String CLUSTER_UPDATE_TASK_SOURCE = "cluster_reroute"; - - private final ClusterService clusterService; - private final AllocationService allocationService; - - private AtomicBoolean rerouting = new AtomicBoolean(); - - @Inject - public RoutingService(ClusterService clusterService, AllocationService allocationService) { - this.clusterService = clusterService; - this.allocationService = allocationService; - } - - @Override - protected void doStart() { - } - - @Override - protected void doStop() { - } - - @Override - protected void doClose() { - } - - /** - * Initiates a reroute. - */ - public final void reroute(String reason) { - try { - if (lifecycle.stopped()) { - return; - } - if (rerouting.compareAndSet(false, true) == false) { - logger.trace("already has pending reroute, ignoring {}", reason); - return; - } - logger.trace("rerouting {}", reason); - clusterService.submitStateUpdateTask(CLUSTER_UPDATE_TASK_SOURCE + "(" + reason + ")", - new ClusterStateUpdateTask(Priority.HIGH) { - @Override - public ClusterState execute(ClusterState currentState) { - rerouting.set(false); - return allocationService.reroute(currentState, reason); - } - - @Override - public void onNoLongerMaster(String source) { - rerouting.set(false); - // no biggie - } - - @Override - public void onFailure(String source, Exception e) { - rerouting.set(false); - ClusterState state = clusterService.state(); - if (logger.isTraceEnabled()) { - logger.error(() -> new ParameterizedMessage("unexpected failure during [{}], current state:\n{}", - source, state), e); - } else { - logger.error(() -> new ParameterizedMessage("unexpected failure during [{}], current state version [{}]", - source, state.version()), e); - } - } - }); - } catch (Exception e) { - rerouting.set(false); - ClusterState state = clusterService.state(); - logger.warn(() -> new ParameterizedMessage("failed to reroute routing table, current state:\n{}", state), e); - } - } -} diff --git a/server/src/main/java/org/elasticsearch/cluster/routing/allocation/AllocationService.java b/server/src/main/java/org/elasticsearch/cluster/routing/allocation/AllocationService.java index c688a120a8b6a..9b22e63cf9ae5 100644 --- a/server/src/main/java/org/elasticsearch/cluster/routing/allocation/AllocationService.java +++ b/server/src/main/java/org/elasticsearch/cluster/routing/allocation/AllocationService.java @@ -240,7 +240,7 @@ public ClusterState disassociateDeadNodes(ClusterState clusterState, boolean rer * Checks if the are replicas with the auto-expand feature that need to be adapted. * Returns an updated cluster state if changes were necessary, or the identical cluster if no changes were required. */ - private ClusterState adaptAutoExpandReplicas(ClusterState clusterState) { + public ClusterState adaptAutoExpandReplicas(ClusterState clusterState) { final Map> autoExpandReplicaChanges = AutoExpandReplicas.getAutoExpandReplicaChanges(clusterState.metaData(), clusterState.nodes()); if (autoExpandReplicaChanges.isEmpty()) { @@ -337,16 +337,16 @@ public CommandsResult reroute(final ClusterState clusterState, AllocationCommand allocation.debugDecision(true); // we ignore disable allocation, because commands are explicit allocation.ignoreDisable(true); - RoutingExplanations explanations = commands.execute(allocation, explain); - // we revert the ignore disable flag, since when rerouting, we want the original setting to take place - allocation.ignoreDisable(false); - // the assumption is that commands will move / act on shards (or fail through exceptions) - // so, there will always be shard "movements", so no need to check on reroute if (retryFailed) { resetFailedAllocationCounter(allocation); } + RoutingExplanations explanations = commands.execute(allocation, explain); + // we revert the ignore disable flag, since when rerouting, we want the original setting to take place + allocation.ignoreDisable(false); + // the assumption is that commands will move / act on shards (or fail through exceptions) + // so, there will always be shard "movements", so no need to check on reroute reroute(allocation); return new CommandsResult(explanations, buildResultAndLogHealthChange(clusterState, allocation, "reroute commands")); } diff --git a/server/src/main/java/org/elasticsearch/cluster/routing/allocation/IndexMetaDataUpdater.java b/server/src/main/java/org/elasticsearch/cluster/routing/allocation/IndexMetaDataUpdater.java index 54625a15e8d80..18d3b45fd9c7b 100644 --- a/server/src/main/java/org/elasticsearch/cluster/routing/allocation/IndexMetaDataUpdater.java +++ b/server/src/main/java/org/elasticsearch/cluster/routing/allocation/IndexMetaDataUpdater.java @@ -194,9 +194,13 @@ private IndexMetaData.Builder updateInSyncAllocations(RoutingTable newRoutingTab // of replicas was decreased while shards were unassigned. int maxActiveShards = oldIndexMetaData.getNumberOfReplicas() + 1; // +1 for the primary IndexShardRoutingTable newShardRoutingTable = newRoutingTable.shardRoutingTable(shardId); + assert newShardRoutingTable.assignedShards().stream() + .filter(ShardRouting::isRelocationTarget).map(s -> s.allocationId().getId()).noneMatch(inSyncAllocationIds::contains) + : newShardRoutingTable.assignedShards() + " vs " + inSyncAllocationIds; if (inSyncAllocationIds.size() > oldInSyncAllocationIds.size() && inSyncAllocationIds.size() > maxActiveShards) { // trim entries that have no corresponding shard routing in the cluster state (i.e. trim unavailable copies) - List assignedShards = newShardRoutingTable.assignedShards(); + List assignedShards = newShardRoutingTable.assignedShards() + .stream().filter(s -> s.isRelocationTarget() == false).collect(Collectors.toList()); assert assignedShards.size() <= maxActiveShards : "cannot have more assigned shards " + assignedShards + " than maximum possible active shards " + maxActiveShards; Set assignedAllocations = assignedShards.stream().map(s -> s.allocationId().getId()).collect(Collectors.toSet()); diff --git a/server/src/main/java/org/elasticsearch/cluster/routing/allocation/command/AllocateEmptyPrimaryAllocationCommand.java b/server/src/main/java/org/elasticsearch/cluster/routing/allocation/command/AllocateEmptyPrimaryAllocationCommand.java index 4d037570dd266..2e3219e67c7ae 100644 --- a/server/src/main/java/org/elasticsearch/cluster/routing/allocation/command/AllocateEmptyPrimaryAllocationCommand.java +++ b/server/src/main/java/org/elasticsearch/cluster/routing/allocation/command/AllocateEmptyPrimaryAllocationCommand.java @@ -110,13 +110,20 @@ public RerouteExplanation execute(RoutingAllocation allocation, boolean explain) return explainOrThrowMissingRoutingNode(allocation, explain, discoNode); } - final ShardRouting shardRouting; try { - shardRouting = allocation.routingTable().shardRoutingTable(index, shardId).primaryShard(); + allocation.routingTable().shardRoutingTable(index, shardId).primaryShard(); } catch (IndexNotFoundException | ShardNotFoundException e) { return explainOrThrowRejectedCommand(explain, allocation, e); } - if (shardRouting.unassigned() == false) { + + ShardRouting shardRouting = null; + for (ShardRouting shard : allocation.routingNodes().unassigned()) { + if (shard.getIndexName().equals(index) && shard.getId() == shardId && shard.primary()) { + shardRouting = shard; + break; + } + } + if (shardRouting == null) { return explainOrThrowRejectedCommand(explain, allocation, "primary [" + index + "][" + shardId + "] is already assigned"); } diff --git a/server/src/main/java/org/elasticsearch/cluster/routing/allocation/command/AllocateReplicaAllocationCommand.java b/server/src/main/java/org/elasticsearch/cluster/routing/allocation/command/AllocateReplicaAllocationCommand.java index 709681f2b2008..5e1bcd81bb5fa 100644 --- a/server/src/main/java/org/elasticsearch/cluster/routing/allocation/command/AllocateReplicaAllocationCommand.java +++ b/server/src/main/java/org/elasticsearch/cluster/routing/allocation/command/AllocateReplicaAllocationCommand.java @@ -23,7 +23,6 @@ import org.elasticsearch.cluster.routing.RoutingNode; import org.elasticsearch.cluster.routing.RoutingNodes; import org.elasticsearch.cluster.routing.ShardRouting; -import org.elasticsearch.cluster.routing.ShardRoutingState; import org.elasticsearch.cluster.routing.allocation.RerouteExplanation; import org.elasticsearch.cluster.routing.allocation.RoutingAllocation; import org.elasticsearch.cluster.routing.allocation.decider.Decision; @@ -35,6 +34,7 @@ import org.elasticsearch.index.shard.ShardNotFoundException; import java.io.IOException; +import java.util.ArrayList; import java.util.List; /** @@ -101,20 +101,34 @@ public RerouteExplanation execute(RoutingAllocation allocation, boolean explain) return explainOrThrowMissingRoutingNode(allocation, explain, discoNode); } - final ShardRouting primaryShardRouting; try { - primaryShardRouting = allocation.routingTable().shardRoutingTable(index, shardId).primaryShard(); + allocation.routingTable().shardRoutingTable(index, shardId).primaryShard(); } catch (IndexNotFoundException | ShardNotFoundException e) { return explainOrThrowRejectedCommand(explain, allocation, e); } - if (primaryShardRouting.unassigned()) { + + ShardRouting primaryShardRouting = null; + for (RoutingNode node : allocation.routingNodes()) { + for (ShardRouting shard : node) { + if (shard.getIndexName().equals(index) && shard.getId() == shardId && shard.primary()) { + primaryShardRouting = shard; + break; + } + } + } + if (primaryShardRouting == null) { return explainOrThrowRejectedCommand(explain, allocation, "trying to allocate a replica shard [" + index + "][" + shardId + "], while corresponding primary shard is still unassigned"); } - List replicaShardRoutings = - allocation.routingTable().shardRoutingTable(index, shardId).replicaShardsWithState(ShardRoutingState.UNASSIGNED); + List replicaShardRoutings = new ArrayList<>(); + for (ShardRouting shard : allocation.routingNodes().unassigned()) { + if (shard.getIndexName().equals(index) && shard.getId() == shardId && shard.primary() == false) { + replicaShardRoutings.add(shard); + } + } + ShardRouting shardRouting; if (replicaShardRoutings.isEmpty()) { return explainOrThrowRejectedCommand(explain, allocation, diff --git a/server/src/main/java/org/elasticsearch/cluster/routing/allocation/command/AllocateStalePrimaryAllocationCommand.java b/server/src/main/java/org/elasticsearch/cluster/routing/allocation/command/AllocateStalePrimaryAllocationCommand.java index f4c9aba17d71e..7e645c2cfcb6f 100644 --- a/server/src/main/java/org/elasticsearch/cluster/routing/allocation/command/AllocateStalePrimaryAllocationCommand.java +++ b/server/src/main/java/org/elasticsearch/cluster/routing/allocation/command/AllocateStalePrimaryAllocationCommand.java @@ -108,13 +108,20 @@ public RerouteExplanation execute(RoutingAllocation allocation, boolean explain) return explainOrThrowMissingRoutingNode(allocation, explain, discoNode); } - final ShardRouting shardRouting; try { - shardRouting = allocation.routingTable().shardRoutingTable(index, shardId).primaryShard(); + allocation.routingTable().shardRoutingTable(index, shardId).primaryShard(); } catch (IndexNotFoundException | ShardNotFoundException e) { return explainOrThrowRejectedCommand(explain, allocation, e); } - if (shardRouting.unassigned() == false) { + + ShardRouting shardRouting = null; + for (ShardRouting shard : allocation.routingNodes().unassigned()) { + if (shard.getIndexName().equals(index) && shard.getId() == shardId && shard.primary()) { + shardRouting = shard; + break; + } + } + if (shardRouting == null) { return explainOrThrowRejectedCommand(explain, allocation, "primary [" + index + "][" + shardId + "] is already assigned"); } diff --git a/server/src/main/java/org/elasticsearch/cluster/routing/allocation/decider/DiskThresholdDecider.java b/server/src/main/java/org/elasticsearch/cluster/routing/allocation/decider/DiskThresholdDecider.java index 1f048fca76c09..0838999c4f367 100644 --- a/server/src/main/java/org/elasticsearch/cluster/routing/allocation/decider/DiskThresholdDecider.java +++ b/server/src/main/java/org/elasticsearch/cluster/routing/allocation/decider/DiskThresholdDecider.java @@ -131,12 +131,12 @@ public Decision canAllocate(ShardRouting shardRouting, RoutingNode node, Routing // flag that determines whether the low threshold checks below can be skipped. We use this for a primary shard that is freshly // allocated and empty. - boolean skipLowTresholdChecks = shardRouting.primary() && + boolean skipLowThresholdChecks = shardRouting.primary() && shardRouting.active() == false && shardRouting.recoverySource().getType() == RecoverySource.Type.EMPTY_STORE; // checks for exact byte comparisons if (freeBytes < diskThresholdSettings.getFreeBytesThresholdLow().getBytes()) { - if (skipLowTresholdChecks == false) { + if (skipLowThresholdChecks == false) { if (logger.isDebugEnabled()) { logger.debug("less than the required {} free bytes threshold ({} free) on node {}, preventing allocation", diskThresholdSettings.getFreeBytesThresholdLow(), freeBytesValue, node.nodeId()); @@ -178,7 +178,7 @@ public Decision canAllocate(ShardRouting shardRouting, RoutingNode node, Routing // checks for percentage comparisons if (freeDiskPercentage < diskThresholdSettings.getFreeDiskThresholdLow()) { // If the shard is a replica or is a non-empty primary, check the low threshold - if (skipLowTresholdChecks == false) { + if (skipLowThresholdChecks == false) { if (logger.isDebugEnabled()) { logger.debug("more than the allowed {} used disk threshold ({} used) on node [{}], preventing allocation", Strings.format1Decimals(usedDiskThresholdLow, "%"), diff --git a/server/src/main/java/org/elasticsearch/cluster/service/ClusterService.java b/server/src/main/java/org/elasticsearch/cluster/service/ClusterService.java index f83f2606b14b6..fded43a4bdd19 100644 --- a/server/src/main/java/org/elasticsearch/cluster/service/ClusterService.java +++ b/server/src/main/java/org/elasticsearch/cluster/service/ClusterService.java @@ -73,7 +73,7 @@ public ClusterService(Settings settings, ClusterSettings clusterSettings, Thread } public ClusterService(Settings settings, ClusterSettings clusterSettings, MasterService masterService, - ClusterApplierService clusterApplierService) { + ClusterApplierService clusterApplierService) { this.settings = settings; this.nodeName = Node.NODE_NAME_SETTING.get(settings); this.masterService = masterService; diff --git a/buildSrc/src/main/groovy/org/elasticsearch/gradle/EmptyDirTask.groovy b/server/src/main/java/org/elasticsearch/common/TriConsumer.java similarity index 59% rename from buildSrc/src/main/groovy/org/elasticsearch/gradle/EmptyDirTask.groovy rename to server/src/main/java/org/elasticsearch/common/TriConsumer.java index 12971b7d701ae..94d3a000f326b 100644 --- a/buildSrc/src/main/groovy/org/elasticsearch/gradle/EmptyDirTask.groovy +++ b/server/src/main/java/org/elasticsearch/common/TriConsumer.java @@ -15,35 +15,26 @@ * KIND, either express or implied. See the License for the * specific language governing permissions and limitations * under the License. + * */ -package org.elasticsearch.gradle - -import org.gradle.api.DefaultTask -import org.gradle.api.tasks.Input -import org.gradle.api.tasks.TaskAction -import org.gradle.internal.nativeintegration.filesystem.Chmod -import javax.inject.Inject +package org.elasticsearch.common; /** - * Creates an empty directory. + * Represents an operation that accepts three arguments and returns no result. + * + * @param the type of the first argument + * @param the type of the second argument + * @param the type of the third argument */ -class EmptyDirTask extends DefaultTask { - @Input - Object dir - - @Input - int dirMode = 0755 - - @TaskAction - void create() { - dir = dir as File - dir.mkdirs() - getChmod().chmod(dir, dirMode) - } - - @Inject - Chmod getChmod() { - throw new UnsupportedOperationException() - } +@FunctionalInterface +public interface TriConsumer { + /** + * Applies this function to the given arguments. + * + * @param s the first function argument + * @param t the second function argument + * @param u the third function argument + */ + void apply(S s, T t, U u); } diff --git a/server/src/main/java/org/elasticsearch/common/blobstore/BlobContainer.java b/server/src/main/java/org/elasticsearch/common/blobstore/BlobContainer.java index 9c1bacb51bac7..a44d1fb05308a 100644 --- a/server/src/main/java/org/elasticsearch/common/blobstore/BlobContainer.java +++ b/server/src/main/java/org/elasticsearch/common/blobstore/BlobContainer.java @@ -109,6 +109,12 @@ public interface BlobContainer { */ void deleteBlob(String blobName) throws IOException; + /** + * Deletes this container and all its contents from the repository. + * @throws IOException on failure + */ + void delete() throws IOException; + /** * Deletes the blobs with given names. Unlike {@link #deleteBlob(String)} this method will not throw an exception * when one or multiple of the given blobs don't exist and simply ignore this case. @@ -158,6 +164,16 @@ default void deleteBlobIgnoringIfNotExists(String blobName) throws IOException { */ Map listBlobs() throws IOException; + /** + * Lists all child containers under this container. A child container is defined as a container whose {@link #path()} method returns + * a path that has this containers {@link #path()} return as its prefix and has one more path element than the current + * container's path. + * + * @return Map of name of the child container to child container + * @throws IOException on failure to list child containers + */ + Map children() throws IOException; + /** * Lists all blobs in the container that match the specified prefix. * diff --git a/server/src/main/java/org/elasticsearch/common/blobstore/BlobPath.java b/server/src/main/java/org/elasticsearch/common/blobstore/BlobPath.java index ea02aebb0aaad..d3acd02a06d1f 100644 --- a/server/src/main/java/org/elasticsearch/common/blobstore/BlobPath.java +++ b/server/src/main/java/org/elasticsearch/common/blobstore/BlobPath.java @@ -19,6 +19,8 @@ package org.elasticsearch.common.blobstore; +import org.elasticsearch.common.Nullable; + import java.util.ArrayList; import java.util.Collections; import java.util.Iterator; @@ -68,6 +70,20 @@ public String buildAsString() { return p + SEPARATOR; } + /** + * Returns this path's parent path. + * + * @return Parent path or {@code null} if there is none + */ + @Nullable + public BlobPath parent() { + if (paths.isEmpty()) { + return null; + } else { + return new BlobPath(List.copyOf(paths.subList(0, paths.size() - 1))); + } + } + @Override public String toString() { StringBuilder sb = new StringBuilder(); diff --git a/server/src/main/java/org/elasticsearch/common/blobstore/fs/FsBlobContainer.java b/server/src/main/java/org/elasticsearch/common/blobstore/fs/FsBlobContainer.java index bab984bd85c74..b51115b246673 100644 --- a/server/src/main/java/org/elasticsearch/common/blobstore/fs/FsBlobContainer.java +++ b/server/src/main/java/org/elasticsearch/common/blobstore/fs/FsBlobContainer.java @@ -20,6 +20,7 @@ package org.elasticsearch.common.blobstore.fs; import org.elasticsearch.common.UUIDs; +import org.elasticsearch.common.blobstore.BlobContainer; import org.elasticsearch.common.blobstore.BlobMetaData; import org.elasticsearch.common.blobstore.BlobPath; import org.elasticsearch.common.blobstore.support.AbstractBlobContainer; @@ -73,9 +74,22 @@ public Map listBlobs() throws IOException { return listBlobsByPrefix(null); } + @Override + public Map children() throws IOException { + Map builder = new HashMap<>(); + try (DirectoryStream stream = Files.newDirectoryStream(path)) { + for (Path file : stream) { + if (Files.isDirectory(file)) { + final String name = file.getFileName().toString(); + builder.put(name, new FsBlobContainer(blobStore, path().add(name), file)); + } + } + } + return unmodifiableMap(builder); + } + @Override public Map listBlobsByPrefix(String blobNamePrefix) throws IOException { - // If we get duplicate files we should just take the last entry Map builder = new HashMap<>(); blobNamePrefix = blobNamePrefix == null ? "" : blobNamePrefix; @@ -108,6 +122,11 @@ public FileVisitResult postVisitDirectory(Path dir, IOException exc) throws IOEx } } + @Override + public void delete() throws IOException { + IOUtils.rm(path); + } + @Override public boolean blobExists(String blobName) { return Files.exists(path.resolve(blobName)); diff --git a/server/src/main/java/org/elasticsearch/common/compress/CompressorFactory.java b/server/src/main/java/org/elasticsearch/common/compress/CompressorFactory.java index 3b1202fe66f42..2ff2f4e95dfdd 100644 --- a/server/src/main/java/org/elasticsearch/common/compress/CompressorFactory.java +++ b/server/src/main/java/org/elasticsearch/common/compress/CompressorFactory.java @@ -21,11 +21,9 @@ import org.elasticsearch.common.Nullable; import org.elasticsearch.common.bytes.BytesReference; -import org.elasticsearch.common.io.stream.BytesStreamOutput; -import org.elasticsearch.common.io.stream.StreamInput; +import org.elasticsearch.common.io.Streams; import org.elasticsearch.common.xcontent.XContentHelper; import org.elasticsearch.common.xcontent.XContentType; -import org.elasticsearch.core.internal.io.Streams; import java.io.IOException; import java.util.Objects; @@ -93,10 +91,6 @@ public static BytesReference uncompress(BytesReference bytes) throws IOException } private static BytesReference uncompress(BytesReference bytes, Compressor compressor) throws IOException { - StreamInput compressed = compressor.streamInput(bytes.streamInput()); - BytesStreamOutput bStream = new BytesStreamOutput(); - Streams.copy(compressed, bStream); - compressed.close(); - return bStream.bytes(); + return Streams.readFully(compressor.streamInput(bytes.streamInput())); } } diff --git a/server/src/main/java/org/elasticsearch/common/geo/GeoJson.java b/server/src/main/java/org/elasticsearch/common/geo/GeoJson.java index 3489eca8b58e4..45db221a65dbf 100644 --- a/server/src/main/java/org/elasticsearch/common/geo/GeoJson.java +++ b/server/src/main/java/org/elasticsearch/common/geo/GeoJson.java @@ -44,6 +44,7 @@ import org.elasticsearch.geo.geometry.Polygon; import org.elasticsearch.geo.geometry.Rectangle; import org.elasticsearch.geo.geometry.ShapeType; +import org.elasticsearch.geo.utils.GeometryValidator; import java.io.IOException; import java.util.ArrayList; @@ -64,14 +65,22 @@ public final class GeoJson { private static final ParseField FIELD_ORIENTATION = new ParseField("orientation"); private static final ParseField FIELD_RADIUS = new ParseField("radius"); - private GeoJson() { + private final boolean rightOrientation; + private final boolean coerce; + private final GeometryValidator validator; + public GeoJson(boolean rightOrientation, boolean coerce, GeometryValidator validator) { + this.rightOrientation = rightOrientation; + this.coerce = coerce; + this.validator = validator; } - public static Geometry fromXContent(XContentParser parser, boolean rightOrientation, boolean coerce, boolean ignoreZValue) + public Geometry fromXContent(XContentParser parser) throws IOException { try (XContentSubParser subParser = new XContentSubParser(parser)) { - return PARSER.apply(subParser, new ParserContext(rightOrientation, coerce, ignoreZValue)); + Geometry geometry = PARSER.apply(subParser, this); + validator.validate(geometry); + return geometry; } } @@ -197,31 +206,19 @@ private XContentBuilder coordinatesToXContent(Polygon polygon) throws IOExceptio return builder.endObject(); } - private static class ParserContext { - public final boolean defaultOrientation; - public final boolean coerce; - public final boolean ignoreZValue; - - ParserContext(boolean defaultOrientation, boolean coerce, boolean ignoreZValue) { - this.defaultOrientation = defaultOrientation; - this.coerce = coerce; - this.ignoreZValue = ignoreZValue; - } - } - - private static ConstructingObjectParser PARSER = + private static ConstructingObjectParser PARSER = new ConstructingObjectParser<>("geojson", true, (a, c) -> { String type = (String) a[0]; CoordinateNode coordinates = (CoordinateNode) a[1]; @SuppressWarnings("unchecked") List geometries = (List) a[2]; Boolean orientation = orientationFromString((String) a[3]); DistanceUnit.Distance radius = (DistanceUnit.Distance) a[4]; - return createGeometry(type, geometries, coordinates, orientation, c.defaultOrientation, c.coerce, radius); + return createGeometry(type, geometries, coordinates, orientation, c.rightOrientation, c.coerce, radius); }); static { PARSER.declareString(constructorArg(), FIELD_TYPE); - PARSER.declareField(optionalConstructorArg(), (p, c) -> parseCoordinates(p, c.ignoreZValue), FIELD_COORDINATES, + PARSER.declareField(optionalConstructorArg(), (p, c) -> parseCoordinates(p), FIELD_COORDINATES, ObjectParser.ValueType.VALUE_ARRAY); PARSER.declareObjectArray(optionalConstructorArg(), PARSER, FIELD_GEOMETRIES); PARSER.declareString(optionalConstructorArg(), FIELD_ORIENTATION); @@ -304,20 +301,20 @@ private static void verifyNulls(String type, List geometries, Boolean * Recursive method which parses the arrays of coordinates used to define * Shapes */ - private static CoordinateNode parseCoordinates(XContentParser parser, boolean ignoreZValue) throws IOException { + private static CoordinateNode parseCoordinates(XContentParser parser) throws IOException { XContentParser.Token token = parser.nextToken(); // Base cases if (token != XContentParser.Token.START_ARRAY && token != XContentParser.Token.END_ARRAY && token != XContentParser.Token.VALUE_NULL) { - return new CoordinateNode(parseCoordinate(parser, ignoreZValue)); + return new CoordinateNode(parseCoordinate(parser)); } else if (token == XContentParser.Token.VALUE_NULL) { throw new IllegalArgumentException("coordinates cannot contain NULL values)"); } List nodes = new ArrayList<>(); while (token != XContentParser.Token.END_ARRAY) { - CoordinateNode node = parseCoordinates(parser, ignoreZValue); + CoordinateNode node = parseCoordinates(parser); if (nodes.isEmpty() == false && nodes.get(0).numDimensions() != node.numDimensions()) { throw new ElasticsearchParseException("Exception parsing coordinates: number of dimensions do not match"); } @@ -331,7 +328,7 @@ private static CoordinateNode parseCoordinates(XContentParser parser, boolean ig /** * Parser a singe set of 2 or 3 coordinates */ - private static Point parseCoordinate(XContentParser parser, boolean ignoreZValue) throws IOException { + private static Point parseCoordinate(XContentParser parser) throws IOException { // Add support for coerce here if (parser.currentToken() != XContentParser.Token.VALUE_NUMBER) { throw new ElasticsearchParseException("geo coordinates must be numbers"); @@ -345,7 +342,7 @@ private static Point parseCoordinate(XContentParser parser, boolean ignoreZValue // alt (for storing purposes only - future use includes 3d shapes) double alt = Double.NaN; if (token == XContentParser.Token.VALUE_NUMBER) { - alt = GeoPoint.assertZValue(ignoreZValue, parser.doubleValue()); + alt = parser.doubleValue(); parser.nextToken(); } // do not support > 3 dimensions diff --git a/server/src/main/java/org/apache/lucene/search/XIndexSearcher.java b/server/src/main/java/org/elasticsearch/common/geo/GeometryFormat.java similarity index 52% rename from server/src/main/java/org/apache/lucene/search/XIndexSearcher.java rename to server/src/main/java/org/elasticsearch/common/geo/GeometryFormat.java index 100c5f4944afe..5e14e20050bb3 100644 --- a/server/src/main/java/org/apache/lucene/search/XIndexSearcher.java +++ b/server/src/main/java/org/elasticsearch/common/geo/GeometryFormat.java @@ -17,30 +17,29 @@ * under the License. */ -package org.apache.lucene.search; +package org.elasticsearch.common.geo; -import org.apache.lucene.index.LeafReaderContext; +import org.elasticsearch.common.xcontent.ToXContent; +import org.elasticsearch.common.xcontent.XContentBuilder; +import org.elasticsearch.common.xcontent.XContentParser; +import org.elasticsearch.geo.geometry.Geometry; import java.io.IOException; -import java.util.List; +import java.text.ParseException; /** - * A wrapper for {@link IndexSearcher} that makes {@link IndexSearcher#search(List, Weight, Collector)} - * visible by sub-classes. + * Geometry serializer/deserializer */ -public class XIndexSearcher extends IndexSearcher { - private final IndexSearcher in; +public interface GeometryFormat { - public XIndexSearcher(IndexSearcher in) { - super(in.getIndexReader()); - this.in = in; - setSimilarity(in.getSimilarity()); - setQueryCache(in.getQueryCache()); - setQueryCachingPolicy(in.getQueryCachingPolicy()); - } + /** + * Parser JSON representation of a geometry + */ + Geometry fromXContent(XContentParser parser) throws IOException, ParseException; + + /** + * Serializes the geometry into its JSON representation + */ + XContentBuilder toXContent(Geometry geometry, XContentBuilder builder, ToXContent.Params params) throws IOException; - @Override - public void search(List leaves, Weight weight, Collector collector) throws IOException { - in.search(leaves, weight, collector); - } } diff --git a/server/src/main/java/org/elasticsearch/common/geo/GeometryParser.java b/server/src/main/java/org/elasticsearch/common/geo/GeometryParser.java index 8e1db18ccdd97..b96e41df5e472 100644 --- a/server/src/main/java/org/elasticsearch/common/geo/GeometryParser.java +++ b/server/src/main/java/org/elasticsearch/common/geo/GeometryParser.java @@ -20,8 +20,12 @@ package org.elasticsearch.common.geo; import org.elasticsearch.ElasticsearchParseException; +import org.elasticsearch.common.xcontent.ToXContent; +import org.elasticsearch.common.xcontent.XContentBuilder; import org.elasticsearch.common.xcontent.XContentParser; import org.elasticsearch.geo.geometry.Geometry; +import org.elasticsearch.geo.utils.GeographyValidator; +import org.elasticsearch.geo.utils.GeometryValidator; import org.elasticsearch.geo.utils.WellKnownText; import java.io.IOException; @@ -32,22 +36,77 @@ */ public final class GeometryParser { - private GeometryParser() { + private final GeoJson geoJsonParser; + private final WellKnownText wellKnownTextParser; + private final GeometryValidator validator; + public GeometryParser(boolean rightOrientation, boolean coerce, boolean ignoreZValue) { + validator = new GeographyValidator(ignoreZValue); + geoJsonParser = new GeoJson(rightOrientation, coerce, validator); + wellKnownTextParser = new WellKnownText(coerce, validator); } /** * Parses supplied XContent into Geometry */ - public static Geometry parse(XContentParser parser, boolean orientation, boolean coerce, boolean ignoreZValue) throws IOException, - ParseException { + public Geometry parse(XContentParser parser) throws IOException, ParseException { + return geometryFormat(parser).fromXContent(parser); + } + + /** + * Returns a geometry format object that can parse and then serialize the object back to the same format. + */ + public GeometryFormat geometryFormat(XContentParser parser) { if (parser.currentToken() == XContentParser.Token.VALUE_NULL) { - return null; + return new GeometryFormat() { + @Override + public Geometry fromXContent(XContentParser parser) throws IOException { + return null; + } + + @Override + public XContentBuilder toXContent(Geometry geometry, XContentBuilder builder, ToXContent.Params params) throws IOException { + if (geometry != null) { + // We don't know the format of the original geometry - so going with default + return GeoJson.toXContent(geometry, builder, params); + } else { + return builder.nullValue(); + } + } + }; } else if (parser.currentToken() == XContentParser.Token.START_OBJECT) { - return GeoJson.fromXContent(parser, orientation, coerce, ignoreZValue); + return new GeometryFormat() { + @Override + public Geometry fromXContent(XContentParser parser) throws IOException { + return geoJsonParser.fromXContent(parser); + } + + @Override + public XContentBuilder toXContent(Geometry geometry, XContentBuilder builder, ToXContent.Params params) throws IOException { + if (geometry != null) { + return GeoJson.toXContent(geometry, builder, params); + } else { + return builder.nullValue(); + } + } + }; } else if (parser.currentToken() == XContentParser.Token.VALUE_STRING) { - // TODO: Add support for ignoreZValue and coerce to WKT - return WellKnownText.fromWKT(parser.text()); + return new GeometryFormat() { + @Override + public Geometry fromXContent(XContentParser parser) throws IOException, ParseException { + return wellKnownTextParser.fromWKT(parser.text()); + } + + @Override + public XContentBuilder toXContent(Geometry geometry, XContentBuilder builder, ToXContent.Params params) throws IOException { + if (geometry != null) { + return builder.value(wellKnownTextParser.toWKT(geometry)); + } else { + return builder.nullValue(); + } + } + }; + } throw new ElasticsearchParseException("shape must be an object consisting of type and coordinates"); } diff --git a/server/src/main/java/org/elasticsearch/common/hash/MessageDigests.java b/server/src/main/java/org/elasticsearch/common/hash/MessageDigests.java index 8bcef7b8ff4cb..df8f3e2fa7f43 100644 --- a/server/src/main/java/org/elasticsearch/common/hash/MessageDigests.java +++ b/server/src/main/java/org/elasticsearch/common/hash/MessageDigests.java @@ -95,15 +95,24 @@ private static MessageDigest get(ThreadLocal messageDigest) { * @return a hex representation of the input as a String. */ public static String toHexString(byte[] bytes) { - Objects.requireNonNull(bytes); - StringBuilder sb = new StringBuilder(2 * bytes.length); + return new String(toHexCharArray(bytes)); + } + /** + * Encodes the byte array into a newly created hex char array, without allocating any other temporary variables. + * + * @param bytes the input to be encoded as hex. + * @return the hex encoding of the input as a char array. + */ + public static char[] toHexCharArray(byte[] bytes) { + Objects.requireNonNull(bytes); + final char[] result = new char[2 * bytes.length]; for (int i = 0; i < bytes.length; i++) { byte b = bytes[i]; - sb.append(HEX_DIGITS[b >> 4 & 0xf]).append(HEX_DIGITS[b & 0xf]); + result[2 * i] = HEX_DIGITS[b >> 4 & 0xf]; + result[2 * i + 1] = HEX_DIGITS[b & 0xf]; } - - return sb.toString(); + return result; } } diff --git a/server/src/main/java/org/elasticsearch/common/hash/MurmurHash3.java b/server/src/main/java/org/elasticsearch/common/hash/MurmurHash3.java index a52f0e8acc4ae..a9232e06657ad 100644 --- a/server/src/main/java/org/elasticsearch/common/hash/MurmurHash3.java +++ b/server/src/main/java/org/elasticsearch/common/hash/MurmurHash3.java @@ -21,6 +21,8 @@ import org.elasticsearch.common.util.ByteUtils; +import java.util.Objects; + /** * MurmurHash3 hashing functions. @@ -36,6 +38,24 @@ public static class Hash128 { public long h1; /** higher 64 bits part **/ public long h2; + + @Override + public boolean equals(Object other) { + if (this == other) { + return true; + } + if (other == null || getClass() != other.getClass()) { + return false; + } + Hash128 that = (Hash128) other; + return Objects.equals(this.h1, that.h1) + && Objects.equals(this.h2, that.h2); + } + + @Override + public int hashCode() { + return Objects.hash(h1, h2); + } } private static long C1 = 0x87c37b91114253d5L; @@ -160,4 +180,22 @@ public static Hash128 hash128(byte[] key, int offset, int length, long seed, Has return hash; } + /** + * A 64-bit variant which accepts a long to hash, and returns the 64bit long hash. + * This is useful if the input is already in long (or smaller) format and you don't + * need the full 128b width and flexibility of + * {@link MurmurHash3#hash128(byte[], int, int, long, Hash128)} + * + * Given the limited nature of this variant, it should be faster than the 128b version + * when you only need 128b (many fewer instructions) + */ + public static long murmur64(long h) { + h ^= h >>> 33; + h *= 0xff51afd7ed558ccdL; + h ^= h >>> 33; + h *= 0xc4ceb9fe1a85ec53L; + h ^= h >>> 33; + return h; + } + } diff --git a/server/src/main/java/org/elasticsearch/common/inject/Binder.java b/server/src/main/java/org/elasticsearch/common/inject/Binder.java index 03d164bcbaa52..55e219047c2c1 100644 --- a/server/src/main/java/org/elasticsearch/common/inject/Binder.java +++ b/server/src/main/java/org/elasticsearch/common/inject/Binder.java @@ -31,7 +31,7 @@ * used to create an {@link Injector}. Guice provides this object to your * application's {@link Module} implementors so they may each contribute * their own bindings and other registrations. - *

    The Guice Binding EDSL

    + *

    The Guice Binding EDSL

    *

    * Guice uses an embedded domain-specific language, or EDSL, to help you * create bindings simply and readably. This approach is great for overall diff --git a/server/src/main/java/org/elasticsearch/common/inject/assistedinject/FactoryProvider.java b/server/src/main/java/org/elasticsearch/common/inject/assistedinject/FactoryProvider.java index 9e2d0e379867c..2ef0b89fb1570 100644 --- a/server/src/main/java/org/elasticsearch/common/inject/assistedinject/FactoryProvider.java +++ b/server/src/main/java/org/elasticsearch/common/inject/assistedinject/FactoryProvider.java @@ -39,7 +39,7 @@ /** * Provides a factory that combines the caller's arguments with injector-supplied values to * construct objects. - *

    Defining a factory

    + *

    Defining a factory

    * Create an interface whose methods return the constructed type, or any of its supertypes. The * method's parameters are the arguments required to build the constructed type. *
    public interface PaymentFactory {
    diff --git a/server/src/main/java/org/elasticsearch/common/io/Streams.java b/server/src/main/java/org/elasticsearch/common/io/Streams.java
    index 46a6956914f90..4a8f2f5de5b74 100644
    --- a/server/src/main/java/org/elasticsearch/common/io/Streams.java
    +++ b/server/src/main/java/org/elasticsearch/common/io/Streams.java
    @@ -21,6 +21,7 @@
     
     import org.elasticsearch.common.bytes.BytesReference;
     import org.elasticsearch.common.io.stream.BytesStream;
    +import org.elasticsearch.common.io.stream.BytesStreamOutput;
     import org.elasticsearch.common.io.stream.StreamOutput;
     
     import java.io.BufferedReader;
    @@ -226,6 +227,17 @@ public static BytesStream flushOnCloseStream(BytesStream os) {
             return new FlushOnCloseOutputStream(os);
         }
     
    +    /**
    +     * Reads all bytes from the given {@link InputStream} and closes it afterwards.
    +     */
    +    public static BytesReference readFully(InputStream in) throws IOException {
    +        try (InputStream inputStream = in) {
    +            BytesStreamOutput out = new BytesStreamOutput();
    +            copy(inputStream, out);
    +            return out.bytes();
    +        }
    +    }
    +
         /**
          * A wrapper around a {@link BytesStream} that makes the close operation a flush. This is
          * needed as sometimes a stream will be closed but the bytes that the stream holds still need
    diff --git a/server/src/main/java/org/elasticsearch/common/io/stream/StreamInput.java b/server/src/main/java/org/elasticsearch/common/io/stream/StreamInput.java
    index 7a763c5a04995..2e213cccf1306 100644
    --- a/server/src/main/java/org/elasticsearch/common/io/stream/StreamInput.java
    +++ b/server/src/main/java/org/elasticsearch/common/io/stream/StreamInput.java
    @@ -30,11 +30,13 @@
     import org.apache.lucene.util.CharsRef;
     import org.elasticsearch.ElasticsearchException;
     import org.elasticsearch.Version;
    +import org.elasticsearch.common.CharArrays;
     import org.elasticsearch.common.Nullable;
     import org.elasticsearch.common.Strings;
     import org.elasticsearch.common.bytes.BytesArray;
     import org.elasticsearch.common.bytes.BytesReference;
     import org.elasticsearch.common.geo.GeoPoint;
    +import org.elasticsearch.common.settings.SecureString;
     import org.elasticsearch.common.text.Text;
     import org.elasticsearch.common.unit.TimeValue;
     import org.elasticsearch.common.util.concurrent.EsRejectedExecutionException;
    @@ -59,6 +61,7 @@
     import java.time.ZoneId;
     import java.time.ZonedDateTime;
     import java.util.ArrayList;
    +import java.util.Arrays;
     import java.util.Collection;
     import java.util.Collections;
     import java.util.Date;
    @@ -358,6 +361,21 @@ public String readOptionalString() throws IOException {
             return null;
         }
     
    +    @Nullable
    +    public SecureString readOptionalSecureString() throws IOException {
    +        SecureString value = null;
    +        BytesReference bytesRef = readOptionalBytesReference();
    +        if (bytesRef != null) {
    +            byte[] bytes = BytesReference.toBytes(bytesRef);
    +            try {
    +                value = new SecureString(CharArrays.utf8BytesToChars(bytes));
    +            } finally {
    +                Arrays.fill(bytes, (byte) 0);
    +            }
    +        }
    +        return value;
    +    }
    +
         @Nullable
         public Float readOptionalFloat() throws IOException {
             if (readBoolean()) {
    @@ -415,6 +433,16 @@ public String readString() throws IOException {
             return spare.toString();
         }
     
    +    public SecureString readSecureString() throws IOException {
    +        BytesReference bytesRef = readBytesReference();
    +        byte[] bytes = BytesReference.toBytes(bytesRef);
    +        try {
    +            return new SecureString(CharArrays.utf8BytesToChars(bytes));
    +        } finally {
    +            Arrays.fill(bytes, (byte) 0);
    +        }
    +    }
    +
         public final float readFloat() throws IOException {
             return Float.intBitsToFloat(readInt());
         }
    diff --git a/server/src/main/java/org/elasticsearch/common/io/stream/StreamOutput.java b/server/src/main/java/org/elasticsearch/common/io/stream/StreamOutput.java
    index 431e00fcf063a..8d944e6f8a17c 100644
    --- a/server/src/main/java/org/elasticsearch/common/io/stream/StreamOutput.java
    +++ b/server/src/main/java/org/elasticsearch/common/io/stream/StreamOutput.java
    @@ -32,10 +32,13 @@
     import org.elasticsearch.Version;
     import org.elasticsearch.cluster.ClusterState;
     import org.elasticsearch.cluster.metadata.MetaData;
    +import org.elasticsearch.common.CharArrays;
     import org.elasticsearch.common.Nullable;
    +import org.elasticsearch.common.bytes.BytesArray;
     import org.elasticsearch.common.bytes.BytesReference;
     import org.elasticsearch.common.geo.GeoPoint;
     import org.elasticsearch.common.io.stream.Writeable.Writer;
    +import org.elasticsearch.common.settings.SecureString;
     import org.elasticsearch.common.text.Text;
     import org.elasticsearch.common.unit.TimeValue;
     import org.elasticsearch.common.util.concurrent.EsRejectedExecutionException;
    @@ -58,6 +61,7 @@
     import java.time.Instant;
     import java.time.ZoneId;
     import java.time.ZonedDateTime;
    +import java.util.Arrays;
     import java.util.Collection;
     import java.util.Collections;
     import java.util.Date;
    @@ -324,6 +328,19 @@ public void writeOptionalString(@Nullable String str) throws IOException {
             }
         }
     
    +    public void writeOptionalSecureString(@Nullable SecureString secureStr) throws IOException {
    +        if (secureStr == null) {
    +            writeOptionalBytesReference(null);
    +        } else {
    +            final byte[] secureStrBytes = CharArrays.toUtf8Bytes(secureStr.getChars());
    +            try {
    +                writeOptionalBytesReference(new BytesArray(secureStrBytes));
    +            } finally {
    +                Arrays.fill(secureStrBytes, (byte) 0);
    +            }
    +        }
    +    }
    +
         /**
          * Writes an optional {@link Integer}.
          */
    @@ -414,6 +431,15 @@ public void writeString(String str) throws IOException {
             writeBytes(buffer, offset);
         }
     
    +    public void writeSecureString(SecureString secureStr) throws IOException {
    +        final byte[] secureStrBytes = CharArrays.toUtf8Bytes(secureStr.getChars());
    +        try {
    +            writeBytesReference(new BytesArray(secureStrBytes));
    +        } finally {
    +            Arrays.fill(secureStrBytes, (byte) 0);
    +        }
    +    }
    +
         public void writeFloat(float v) throws IOException {
             writeInt(Float.floatToIntBits(v));
         }
    diff --git a/server/src/main/java/org/elasticsearch/common/lucene/Lucene.java b/server/src/main/java/org/elasticsearch/common/lucene/Lucene.java
    index 88d7c57f5e424..4b0ec9ea08773 100644
    --- a/server/src/main/java/org/elasticsearch/common/lucene/Lucene.java
    +++ b/server/src/main/java/org/elasticsearch/common/lucene/Lucene.java
    @@ -27,7 +27,6 @@
     import org.apache.lucene.codecs.DocValuesFormat;
     import org.apache.lucene.codecs.PostingsFormat;
     import org.apache.lucene.document.LatLonDocValuesField;
    -import org.apache.lucene.document.LongPoint;
     import org.apache.lucene.document.NumericDocValuesField;
     import org.apache.lucene.index.BinaryDocValues;
     import org.apache.lucene.index.CorruptIndexException;
    @@ -95,7 +94,6 @@
     import org.elasticsearch.index.analysis.AnalyzerScope;
     import org.elasticsearch.index.analysis.NamedAnalyzer;
     import org.elasticsearch.index.fielddata.IndexFieldData;
    -import org.elasticsearch.index.mapper.SeqNoFieldMapper;
     
     import java.io.IOException;
     import java.text.ParseException;
    @@ -105,7 +103,6 @@
     import java.util.Collections;
     import java.util.List;
     import java.util.Map;
    -import java.util.function.LongConsumer;
     
     public class Lucene {
         public static final String LATEST_DOC_VALUES_FORMAT = "Lucene70";
    @@ -1050,39 +1047,4 @@ public CacheHelper getReaderCacheHelper() {
                 }
             };
         }
    -
    -    /**
    -     * Scans sequence numbers (i.e., {@link SeqNoFieldMapper#NAME}) between {@code fromSeqNo}(inclusive) and {@code toSeqNo}(inclusive)
    -     * in the provided directory reader. This method invokes the callback {@code onNewSeqNo} whenever a sequence number value is found.
    -     *
    -     * @param directoryReader the directory reader to scan
    -     * @param fromSeqNo       the lower bound of a range of seq_no to scan (inclusive)
    -     * @param toSeqNo         the upper bound of a range of seq_no to scan (inclusive)
    -     * @param onNewSeqNo      the callback to be called whenever a new valid sequence number is found
    -     */
    -    public static void scanSeqNosInReader(DirectoryReader directoryReader, long fromSeqNo, long toSeqNo,
    -                                          LongConsumer onNewSeqNo) throws IOException {
    -        final DirectoryReader reader = Lucene.wrapAllDocsLive(directoryReader);
    -        final IndexSearcher searcher = new IndexSearcher(reader);
    -        searcher.setQueryCache(null);
    -        final Query query = LongPoint.newRangeQuery(SeqNoFieldMapper.NAME, fromSeqNo, toSeqNo);
    -        final Weight weight = searcher.createWeight(query, ScoreMode.COMPLETE_NO_SCORES, 1.0f);
    -        for (LeafReaderContext leaf : reader.leaves()) {
    -            final Scorer scorer = weight.scorer(leaf);
    -            if (scorer == null) {
    -                continue;
    -            }
    -            final DocIdSetIterator docIdSetIterator = scorer.iterator();
    -            final NumericDocValues seqNoDocValues = leaf.reader().getNumericDocValues(SeqNoFieldMapper.NAME);
    -            int docId;
    -            while ((docId = docIdSetIterator.nextDoc()) != DocIdSetIterator.NO_MORE_DOCS) {
    -                if (seqNoDocValues == null || seqNoDocValues.advanceExact(docId) == false) {
    -                    throw new IllegalStateException("seq_no doc_values not found for doc_id=" + docId);
    -                }
    -                final long seqNo = seqNoDocValues.longValue();
    -                assert fromSeqNo <= seqNo && seqNo <= toSeqNo : "from_seq_no=" + fromSeqNo + " seq_no=" + seqNo + " to_seq_no=" + toSeqNo;
    -                onNewSeqNo.accept(seqNo);
    -            }
    -        }
    -    }
     }
    diff --git a/server/src/main/java/org/elasticsearch/common/lucene/MinimumScoreCollector.java b/server/src/main/java/org/elasticsearch/common/lucene/MinimumScoreCollector.java
    index f99d68952e557..4e558f163ebca 100644
    --- a/server/src/main/java/org/elasticsearch/common/lucene/MinimumScoreCollector.java
    +++ b/server/src/main/java/org/elasticsearch/common/lucene/MinimumScoreCollector.java
    @@ -65,6 +65,6 @@ public void doSetNextReader(LeafReaderContext context) throws IOException {
     
         @Override
         public ScoreMode scoreMode() {
    -        return ScoreMode.COMPLETE;
    +        return collector.scoreMode() == ScoreMode.TOP_SCORES ? ScoreMode.TOP_SCORES :  ScoreMode.COMPLETE;
         }
     }
    diff --git a/server/src/main/java/org/elasticsearch/common/lucene/search/Queries.java b/server/src/main/java/org/elasticsearch/common/lucene/search/Queries.java
    index 96a0cafc35b11..2e004dfad2e55 100644
    --- a/server/src/main/java/org/elasticsearch/common/lucene/search/Queries.java
    +++ b/server/src/main/java/org/elasticsearch/common/lucene/search/Queries.java
    @@ -20,7 +20,6 @@
     package org.elasticsearch.common.lucene.search;
     
     import org.apache.lucene.index.Term;
    -import org.apache.lucene.queries.ExtendedCommonTermsQuery;
     import org.apache.lucene.search.BooleanClause;
     import org.apache.lucene.search.BooleanClause.Occur;
     import org.apache.lucene.search.BooleanQuery;
    @@ -148,8 +147,6 @@ public static Query applyMinimumShouldMatch(BooleanQuery query, @Nullable String
         public static Query maybeApplyMinimumShouldMatch(Query query, @Nullable String minimumShouldMatch) {
             if (query instanceof BooleanQuery) {
                 return applyMinimumShouldMatch((BooleanQuery) query, minimumShouldMatch);
    -        } else if (query instanceof ExtendedCommonTermsQuery) {
    -            ((ExtendedCommonTermsQuery)query).setLowFreqMinimumNumberShouldMatch(minimumShouldMatch);
             }
             return query;
         }
    diff --git a/server/src/main/java/org/elasticsearch/common/lucene/search/XMoreLikeThis.java b/server/src/main/java/org/elasticsearch/common/lucene/search/XMoreLikeThis.java
    index 1010c917eca82..b958b293183d2 100644
    --- a/server/src/main/java/org/elasticsearch/common/lucene/search/XMoreLikeThis.java
    +++ b/server/src/main/java/org/elasticsearch/common/lucene/search/XMoreLikeThis.java
    @@ -97,7 +97,7 @@
      * above.  The frequency and length thresholds could be parameters, etc.
      * Doug
      * 
    - *

    Initial Usage

    + *

    Initial Usage

    *

    * This class has lots of options to try to make it efficient and flexible. * The simplest possible usage is as follows. The bold diff --git a/server/src/main/java/org/elasticsearch/common/lucene/uid/PerThreadIDVersionAndSeqNoLookup.java b/server/src/main/java/org/elasticsearch/common/lucene/uid/PerThreadIDVersionAndSeqNoLookup.java index 889721a49e07a..94ba9abf58fde 100644 --- a/server/src/main/java/org/elasticsearch/common/lucene/uid/PerThreadIDVersionAndSeqNoLookup.java +++ b/server/src/main/java/org/elasticsearch/common/lucene/uid/PerThreadIDVersionAndSeqNoLookup.java @@ -74,7 +74,9 @@ final class PerThreadIDVersionAndSeqNoLookup { // If a segment contains only no-ops, it does not have _uid but has both _soft_deletes and _tombstone fields. final NumericDocValues softDeletesDV = reader.getNumericDocValues(Lucene.SOFT_DELETES_FIELD); final NumericDocValues tombstoneDV = reader.getNumericDocValues(SeqNoFieldMapper.TOMBSTONE_NAME); - if (softDeletesDV == null || tombstoneDV == null) { + // this is a special case when we pruned away all IDs in a segment since all docs are deleted. + final boolean allDocsDeleted = (softDeletesDV != null && reader.numDocs() == 0); + if ((softDeletesDV == null || tombstoneDV == null) && allDocsDeleted == false) { throw new IllegalArgumentException("reader does not have _uid terms but not a no-op segment; " + "_soft_deletes [" + softDeletesDV + "], _tombstone [" + tombstoneDV + "]"); } @@ -100,38 +102,20 @@ public DocIdAndVersion lookupVersion(BytesRef id, boolean loadSeqNo, LeafReaderC throws IOException { assert context.reader().getCoreCacheHelper().getKey().equals(readerKey) : "context's reader is not the same as the reader class was initialized on."; - int docID = getDocID(id, context.reader().getLiveDocs()); + int docID = getDocID(id, context); if (docID != DocIdSetIterator.NO_MORE_DOCS) { - final NumericDocValues versions = context.reader().getNumericDocValues(VersionFieldMapper.NAME); - if (versions == null) { - throw new IllegalArgumentException("reader misses the [" + VersionFieldMapper.NAME + "] field"); - } - if (versions.advanceExact(docID) == false) { - throw new IllegalArgumentException("Document [" + docID + "] misses the [" + VersionFieldMapper.NAME + "] field"); - } final long seqNo; final long term; if (loadSeqNo) { - NumericDocValues seqNos = context.reader().getNumericDocValues(SeqNoFieldMapper.NAME); - // remove the null check in 7.0 once we can't read indices with no seq# - if (seqNos != null && seqNos.advanceExact(docID)) { - seqNo = seqNos.longValue(); - } else { - seqNo = UNASSIGNED_SEQ_NO; - } - NumericDocValues terms = context.reader().getNumericDocValues(SeqNoFieldMapper.PRIMARY_TERM_NAME); - if (terms != null && terms.advanceExact(docID)) { - term = terms.longValue(); - } else { - term = UNASSIGNED_PRIMARY_TERM; - } - + seqNo = readNumericDocValues(context.reader(), SeqNoFieldMapper.NAME, docID); + term = readNumericDocValues(context.reader(), SeqNoFieldMapper.PRIMARY_TERM_NAME, docID); } else { seqNo = UNASSIGNED_SEQ_NO; term = UNASSIGNED_PRIMARY_TERM; } - return new DocIdAndVersion(docID, versions.longValue(), seqNo, term, context.reader(), context.docBase); + final long version = readNumericDocValues(context.reader(), VersionFieldMapper.NAME, docID); + return new DocIdAndVersion(docID, version, seqNo, term, context.reader(), context.docBase); } else { return null; } @@ -141,9 +125,10 @@ public DocIdAndVersion lookupVersion(BytesRef id, boolean loadSeqNo, LeafReaderC * returns the internal lucene doc id for the given id bytes. * {@link DocIdSetIterator#NO_MORE_DOCS} is returned if not found * */ - private int getDocID(BytesRef id, Bits liveDocs) throws IOException { + private int getDocID(BytesRef id, LeafReaderContext context) throws IOException { // termsEnum can possibly be null here if this leaf contains only no-ops. if (termsEnum != null && termsEnum.seekExact(id)) { + final Bits liveDocs = context.reader().getLiveDocs(); int docID = DocIdSetIterator.NO_MORE_DOCS; // there may be more than one matching docID, in the case of nested docs, so we want the last one: docsEnum = termsEnum.postings(docsEnum, 0); @@ -159,41 +144,23 @@ private int getDocID(BytesRef id, Bits liveDocs) throws IOException { } } + private static long readNumericDocValues(LeafReader reader, String field, int docId) throws IOException { + final NumericDocValues dv = reader.getNumericDocValues(field); + if (dv == null || dv.advanceExact(docId) == false) { + assert false : "document [" + docId + "] does not have docValues for [" + field + "]"; + throw new IllegalStateException("document [" + docId + "] does not have docValues for [" + field + "]"); + } + return dv.longValue(); + } + /** Return null if id is not found. */ DocIdAndSeqNo lookupSeqNo(BytesRef id, LeafReaderContext context) throws IOException { assert context.reader().getCoreCacheHelper().getKey().equals(readerKey) : "context's reader is not the same as the reader class was initialized on."; - // termsEnum can possibly be null here if this leaf contains only no-ops. - if (termsEnum != null && termsEnum.seekExact(id)) { - docsEnum = termsEnum.postings(docsEnum, 0); - final Bits liveDocs = context.reader().getLiveDocs(); - DocIdAndSeqNo result = null; - int docID = docsEnum.nextDoc(); - if (docID != DocIdSetIterator.NO_MORE_DOCS) { - final NumericDocValues seqNoDV = context.reader().getNumericDocValues(SeqNoFieldMapper.NAME); - for (; docID != DocIdSetIterator.NO_MORE_DOCS; docID = docsEnum.nextDoc()) { - final long seqNo; - // remove the null check in 7.0 once we can't read indices with no seq# - if (seqNoDV != null && seqNoDV.advanceExact(docID)) { - seqNo = seqNoDV.longValue(); - } else { - seqNo = UNASSIGNED_SEQ_NO; - } - final boolean isLive = (liveDocs == null || liveDocs.get(docID)); - if (isLive) { - // The live document must always be the latest copy, thus we can early terminate here. - // If a nested docs is live, we return the first doc which doesn't have term (only the last doc has term). - // This should not be an issue since we no longer use primary term as tier breaker when comparing operations. - assert result == null || result.seqNo <= seqNo : - "the live doc does not have the highest seq_no; live_seq_no=" + seqNo + " < deleted_seq_no=" + result.seqNo; - return new DocIdAndSeqNo(docID, seqNo, context, isLive); - } - if (result == null || result.seqNo < seqNo) { - result = new DocIdAndSeqNo(docID, seqNo, context, isLive); - } - } - } - return result; + final int docID = getDocID(id, context); + if (docID != DocIdSetIterator.NO_MORE_DOCS) { + final long seqNo = readNumericDocValues(context.reader(), SeqNoFieldMapper.NAME, docID); + return new DocIdAndSeqNo(docID, seqNo, context); } else { return null; } diff --git a/server/src/main/java/org/elasticsearch/common/lucene/uid/VersionsAndSeqNoResolver.java b/server/src/main/java/org/elasticsearch/common/lucene/uid/VersionsAndSeqNoResolver.java index 611887342adca..bfd859c115fe2 100644 --- a/server/src/main/java/org/elasticsearch/common/lucene/uid/VersionsAndSeqNoResolver.java +++ b/server/src/main/java/org/elasticsearch/common/lucene/uid/VersionsAndSeqNoResolver.java @@ -114,13 +114,11 @@ public static class DocIdAndSeqNo { public final int docId; public final long seqNo; public final LeafReaderContext context; - public final boolean isLive; - DocIdAndSeqNo(int docId, long seqNo, LeafReaderContext context, boolean isLive) { + DocIdAndSeqNo(int docId, long seqNo, LeafReaderContext context) { this.docId = docId; this.seqNo = seqNo; this.context = context; - this.isLive = isLive; } } @@ -149,32 +147,21 @@ public static DocIdAndVersion loadDocIdAndVersion(IndexReader reader, Term term, /** * Loads the internal docId and sequence number of the latest copy for a given uid from the provided reader. - * The flag {@link DocIdAndSeqNo#isLive} indicates whether the returned document is live or (soft)deleted. - * This returns {@code null} if no such document matching the given term uid. + * The result is either null or the live and latest version of the given uid. */ public static DocIdAndSeqNo loadDocIdAndSeqNo(IndexReader reader, Term term) throws IOException { final PerThreadIDVersionAndSeqNoLookup[] lookups = getLookupState(reader, term.field()); final List leaves = reader.leaves(); - DocIdAndSeqNo latest = null; // iterate backwards to optimize for the frequently updated documents // which are likely to be in the last segments for (int i = leaves.size() - 1; i >= 0; i--) { final LeafReaderContext leaf = leaves.get(i); final PerThreadIDVersionAndSeqNoLookup lookup = lookups[leaf.ord]; final DocIdAndSeqNo result = lookup.lookupSeqNo(term.bytes(), leaf); - if (result == null) { - continue; - } - if (result.isLive) { - // The live document must always be the latest copy, thus we can early terminate here. - assert latest == null || latest.seqNo <= result.seqNo : - "the live doc does not have the highest seq_no; live_seq_no=" + result.seqNo + " < deleted_seq_no=" + latest.seqNo; + if (result != null) { return result; } - if (latest == null || latest.seqNo < result.seqNo) { - latest = result; - } } - return latest; + return null; } } diff --git a/server/src/main/java/org/elasticsearch/common/network/NetworkModule.java b/server/src/main/java/org/elasticsearch/common/network/NetworkModule.java index bbfae10991e84..8b363ebcc0b2f 100644 --- a/server/src/main/java/org/elasticsearch/common/network/NetworkModule.java +++ b/server/src/main/java/org/elasticsearch/common/network/NetworkModule.java @@ -75,7 +75,6 @@ public final class NetworkModule { public static final Setting TRANSPORT_TYPE_SETTING = Setting.simpleString(TRANSPORT_TYPE_KEY, Property.NodeScope); private final Settings settings; - private final boolean transportClient; private static final List namedWriteables = new ArrayList<>(); private static final List namedXContents = new ArrayList<>(); @@ -106,9 +105,8 @@ public final class NetworkModule { /** * Creates a network module that custom networking classes can be plugged into. * @param settings The settings for the node - * @param transportClient True if only transport classes should be allowed to be registered, false otherwise. */ - public NetworkModule(Settings settings, boolean transportClient, List plugins, ThreadPool threadPool, + public NetworkModule(Settings settings, List plugins, ThreadPool threadPool, BigArrays bigArrays, PageCacheRecycler pageCacheRecycler, CircuitBreakerService circuitBreakerService, @@ -116,14 +114,11 @@ public NetworkModule(Settings settings, boolean transportClient, List> httpTransportFactory = plugin.getHttpTransports(settings, threadPool, bigArrays, pageCacheRecycler, circuitBreakerService, xContentRegistry, networkService, dispatcher); - if (transportClient == false) { - for (Map.Entry> entry : httpTransportFactory.entrySet()) { - registerHttpTransport(entry.getKey(), entry.getValue()); - } + for (Map.Entry> entry : httpTransportFactory.entrySet()) { + registerHttpTransport(entry.getKey(), entry.getValue()); } Map> transportFactory = plugin.getTransports(settings, threadPool, pageCacheRecycler, circuitBreakerService, namedWriteableRegistry, networkService); @@ -138,10 +133,6 @@ public NetworkModule(Settings settings, boolean transportClient, List factory) { if (transportFactories.putIfAbsent(key, factory) != null) { @@ -150,21 +141,17 @@ private void registerTransport(String key, Supplier factory) { } /** Adds an http transport implementation that can be selected by setting {@link #HTTP_TYPE_KEY}. */ - // TODO: we need another name than "http transport"....so confusing with transportClient... private void registerHttpTransport(String key, Supplier factory) { - if (transportClient) { - throw new IllegalArgumentException("Cannot register http transport " + key + " for transport client"); - } if (transportHttpFactories.putIfAbsent(key, factory) != null) { throw new IllegalArgumentException("transport for name: " + key + " is already registered"); } } + // TODO: consider moving this to the ClusterModule + // this lives here instead of the more aptly named ClusterModule because it used to be used by the Transport client /** * Register an allocation command. *

    - * This lives here instead of the more aptly named ClusterModule because the Transport client needs these to be registered. - *

    * @param reader the reader to read it from a stream * @param parser the parser to read it from XContent * @param commandName the names under which the command should be parsed. The {@link ParseField#getPreferredName()} is special because diff --git a/server/src/main/java/org/elasticsearch/common/network/NetworkUtils.java b/server/src/main/java/org/elasticsearch/common/network/NetworkUtils.java index 4e8f871ed3044..6a49f0d7658d4 100644 --- a/server/src/main/java/org/elasticsearch/common/network/NetworkUtils.java +++ b/server/src/main/java/org/elasticsearch/common/network/NetworkUtils.java @@ -200,10 +200,14 @@ static InetAddress[] getGlobalAddresses() throws IOException { static InetAddress[] getAllAddresses() throws IOException { return filterAllAddresses(address -> true, "no up-and-running addresses found"); } - + + static Optional maybeGetInterfaceByName(List networkInterfaces, String name) { + return networkInterfaces.stream().filter(netIf -> name.equals(netIf.getName())).findFirst(); + } + /** Returns addresses for the given interface (it must be marked up) */ static InetAddress[] getAddressesForInterface(String name) throws SocketException { - Optional networkInterface = getInterfaces().stream().filter((netIf) -> name.equals(netIf.getName())).findFirst(); + Optional networkInterface = maybeGetInterfaceByName(getInterfaces(), name); if (networkInterface.isPresent() == false) { throw new IllegalArgumentException("No interface named '" + name + "' found, got " + getInterfaces()); diff --git a/server/src/main/java/org/elasticsearch/common/regex/Regex.java b/server/src/main/java/org/elasticsearch/common/regex/Regex.java index 1f4e465141222..8734ab1febe8b 100644 --- a/server/src/main/java/org/elasticsearch/common/regex/Regex.java +++ b/server/src/main/java/org/elasticsearch/common/regex/Regex.java @@ -88,7 +88,7 @@ public static boolean simpleMatch(String pattern, String str) { if (pattern == null || str == null) { return false; } - int firstIndex = pattern.indexOf('*'); + final int firstIndex = pattern.indexOf('*'); if (firstIndex == -1) { return pattern.equals(str); } @@ -96,14 +96,15 @@ public static boolean simpleMatch(String pattern, String str) { if (pattern.length() == 1) { return true; } - int nextIndex = pattern.indexOf('*', firstIndex + 1); + final int nextIndex = pattern.indexOf('*', firstIndex + 1); if (nextIndex == -1) { - return str.endsWith(pattern.substring(1)); + // str.endsWith(pattern.substring(1)), but avoiding the construction of pattern.substring(1): + return str.regionMatches(str.length() - pattern.length() + 1, pattern, 1, pattern.length() - 1); } else if (nextIndex == 1) { // Double wildcard "**" - skipping the first "*" return simpleMatch(pattern.substring(1), str); } - String part = pattern.substring(1, nextIndex); + final String part = pattern.substring(1, nextIndex); int partIndex = str.indexOf(part); while (partIndex != -1) { if (simpleMatch(pattern.substring(nextIndex), str.substring(partIndex + part.length()))) { @@ -113,9 +114,9 @@ public static boolean simpleMatch(String pattern, String str) { } return false; } - return (str.length() >= firstIndex && - pattern.substring(0, firstIndex).equals(str.substring(0, firstIndex)) && - simpleMatch(pattern.substring(firstIndex), str.substring(firstIndex))); + return str.regionMatches(0, pattern, 0, firstIndex) + && (firstIndex == pattern.length() - 1 // only wildcard in pattern is at the end, so no need to look at the rest of the string + || simpleMatch(pattern.substring(firstIndex), str.substring(firstIndex))); } /** diff --git a/server/src/main/java/org/elasticsearch/common/settings/ClusterSettings.java b/server/src/main/java/org/elasticsearch/common/settings/ClusterSettings.java index 026dfa4633991..867b628a5f97c 100644 --- a/server/src/main/java/org/elasticsearch/common/settings/ClusterSettings.java +++ b/server/src/main/java/org/elasticsearch/common/settings/ClusterSettings.java @@ -26,7 +26,6 @@ import org.elasticsearch.action.support.DestructiveOperations; import org.elasticsearch.bootstrap.BootstrapSettings; import org.elasticsearch.client.Client; -import org.elasticsearch.client.transport.TransportClient; import org.elasticsearch.cluster.ClusterModule; import org.elasticsearch.cluster.ClusterName; import org.elasticsearch.cluster.InternalClusterInfoService; @@ -113,12 +112,12 @@ * Encapsulates all valid cluster level settings. */ public final class ClusterSettings extends AbstractScopedSettings { + public ClusterSettings(final Settings nodeSettings, final Set> settingsSet) { this(nodeSettings, settingsSet, Collections.emptySet()); } - public ClusterSettings( - final Settings nodeSettings, final Set> settingsSet, final Set> settingUpgraders) { + public ClusterSettings(final Settings nodeSettings, final Set> settingsSet, final Set> settingUpgraders) { super(nodeSettings, settingsSet, settingUpgraders, Property.NodeScope); addSettingsUpdater(new LoggingSettingUpdater(nodeSettings)); } @@ -176,11 +175,6 @@ public void apply(Settings value, Settings current, Settings previous) { public static Set> BUILT_IN_CLUSTER_SETTINGS = Set.of( AwarenessAllocationDecider.CLUSTER_ROUTING_ALLOCATION_AWARENESS_ATTRIBUTE_SETTING, - // TODO these transport client settings are kind of odd here and should only be valid if we are a transport client - TransportClient.CLIENT_TRANSPORT_NODES_SAMPLER_INTERVAL, - TransportClient.CLIENT_TRANSPORT_PING_TIMEOUT, - TransportClient.CLIENT_TRANSPORT_IGNORE_CLUSTER_NAME, - TransportClient.CLIENT_TRANSPORT_SNIFF, AwarenessAllocationDecider.CLUSTER_ROUTING_ALLOCATION_AWARENESS_FORCE_GROUP_SETTING, BalancedShardsAllocator.INDEX_BALANCE_FACTOR_SETTING, BalancedShardsAllocator.SHARD_BALANCE_FACTOR_SETTING, @@ -358,6 +352,7 @@ public void apply(Settings value, Settings current, Settings previous) { Node.INITIAL_STATE_TIMEOUT_SETTING, DiscoveryModule.DISCOVERY_TYPE_SETTING, DiscoveryModule.DISCOVERY_SEED_PROVIDERS_SETTING, + DiscoveryModule.ELECTION_STRATEGY_SETTING, SettingsBasedSeedHostsProvider.DISCOVERY_SEED_HOSTS_SETTING, SeedHostsResolver.DISCOVERY_SEED_RESOLVER_MAX_CONCURRENT_RESOLVERS_SETTING, SeedHostsResolver.DISCOVERY_SEED_RESOLVER_TIMEOUT_SETTING, diff --git a/server/src/main/java/org/elasticsearch/common/settings/ConsistentSettingsService.java b/server/src/main/java/org/elasticsearch/common/settings/ConsistentSettingsService.java new file mode 100644 index 0000000000000..411a470238638 --- /dev/null +++ b/server/src/main/java/org/elasticsearch/common/settings/ConsistentSettingsService.java @@ -0,0 +1,256 @@ +/* + * 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.settings; + +import org.apache.logging.log4j.LogManager; +import org.apache.logging.log4j.Logger; +import org.elasticsearch.cluster.ClusterState; +import org.elasticsearch.cluster.ClusterStateUpdateTask; +import org.elasticsearch.cluster.LocalNodeMasterListener; +import org.elasticsearch.cluster.metadata.MetaData; +import org.elasticsearch.cluster.service.ClusterService; +import org.elasticsearch.common.Priority; +import org.elasticsearch.common.UUIDs; +import org.elasticsearch.common.hash.MessageDigests; +import org.elasticsearch.threadpool.ThreadPool; + +import java.nio.charset.StandardCharsets; +import java.security.NoSuchAlgorithmException; +import java.security.spec.InvalidKeySpecException; +import java.util.Arrays; +import java.util.Base64; +import java.util.Collection; +import java.util.HashMap; +import java.util.HashSet; +import java.util.Map; +import java.util.Set; +import java.util.concurrent.atomic.AtomicBoolean; +import java.util.function.Consumer; + +import javax.crypto.SecretKey; +import javax.crypto.SecretKeyFactory; +import javax.crypto.spec.PBEKeySpec; + +/** + * Used to publish secure setting hashes in the cluster state and to validate those hashes against the local values of those same settings. + * This is colloquially referred to as the secure setting consistency check. It will publish and verify hashes only for the collection + * of settings passed in the constructor. The settings have to have the {@link Setting.Property#Consistent} property. + */ +public final class ConsistentSettingsService { + private static final Logger logger = LogManager.getLogger(ConsistentSettingsService.class); + + private final Settings settings; + private final ClusterService clusterService; + private final Collection> secureSettingsCollection; + private final SecretKeyFactory pbkdf2KeyFactory; + + public ConsistentSettingsService(Settings settings, ClusterService clusterService, + Collection> secureSettingsCollection) { + this.settings = settings; + this.clusterService = clusterService; + this.secureSettingsCollection = secureSettingsCollection; + // this is used to compute the PBKDF2 hash (the published one) + try { + this.pbkdf2KeyFactory = SecretKeyFactory.getInstance("PBKDF2WithHmacSHA512"); + } catch (NoSuchAlgorithmException e) { + throw new RuntimeException("The \"PBKDF2WithHmacSHA512\" algorithm is required for consistent secure settings' hashes", e); + } + } + + /** + * Returns a {@link LocalNodeMasterListener} that will publish hashes of all the settings passed in the constructor. These hashes are + * published by the master node only. Note that this is not designed for {@link SecureSettings} implementations that are mutable. + */ + public LocalNodeMasterListener newHashPublisher() { + // eagerly compute hashes to be published + final Map computedHashesOfConsistentSettings = computeHashesOfConsistentSecureSettings(); + return new HashesPublisher(computedHashesOfConsistentSettings, clusterService); + } + + /** + * Verifies that the hashes of consistent secure settings in the latest {@code ClusterState} verify for the values of those same + * settings on the local node. The settings to be checked are passed in the constructor. Also, validates that a missing local + * value is also missing in the published set, and vice-versa. + */ + public boolean areAllConsistent() { + final ClusterState state = clusterService.state(); + final Map publishedHashesOfConsistentSettings = state.metaData().hashesOfConsistentSettings(); + final Set publishedSettingKeysToVerify = new HashSet<>(); + publishedSettingKeysToVerify.addAll(publishedHashesOfConsistentSettings.keySet()); + final AtomicBoolean allConsistent = new AtomicBoolean(true); + forEachConcreteSecureSettingDo(concreteSecureSetting -> { + final String publishedSaltAndHash = publishedHashesOfConsistentSettings.get(concreteSecureSetting.getKey()); + final byte[] localHash = concreteSecureSetting.getSecretDigest(settings); + if (publishedSaltAndHash == null && localHash == null) { + // consistency of missing + logger.debug("no published hash for the consistent secure setting [{}] but it also does NOT exist on the local node", + concreteSecureSetting.getKey()); + } else if (publishedSaltAndHash == null && localHash != null) { + // setting missing on master but present locally + logger.warn("no published hash for the consistent secure setting [{}] but it exists on the local node", + concreteSecureSetting.getKey()); + if (state.nodes().isLocalNodeElectedMaster()) { + throw new IllegalStateException("Master node cannot validate consistent setting. No published hash for [" + + concreteSecureSetting.getKey() + "] but setting exists."); + } + allConsistent.set(false); + } else if (publishedSaltAndHash != null && localHash == null) { + // setting missing locally but present on master + logger.warn("the consistent secure setting [{}] does not exist on the local node but there is a published hash for it", + concreteSecureSetting.getKey()); + allConsistent.set(false); + } else { + assert publishedSaltAndHash != null; + assert localHash != null; + final String[] parts = publishedSaltAndHash.split(":"); + if (parts == null || parts.length != 2) { + throw new IllegalArgumentException("published hash [" + publishedSaltAndHash + " ] for secure setting [" + + concreteSecureSetting.getKey() + "] is invalid"); + } + final String publishedSalt = parts[0]; + final String publishedHash = parts[1]; + final byte[] computedSaltedHashBytes = computeSaltedPBKDF2Hash(localHash, publishedSalt.getBytes(StandardCharsets.UTF_8)); + final String computedSaltedHash = new String(Base64.getEncoder().encode(computedSaltedHashBytes), StandardCharsets.UTF_8); + if (false == publishedHash.equals(computedSaltedHash)) { + logger.warn("the published hash [{}] of the consistent secure setting [{}] differs from the locally computed one [{}]", + publishedHash, concreteSecureSetting.getKey(), computedSaltedHash); + if (state.nodes().isLocalNodeElectedMaster()) { + throw new IllegalStateException("Master node cannot validate consistent setting. The published hash [" + + publishedHash + "] of the consistent secure setting [" + concreteSecureSetting.getKey() + + "] differs from the locally computed one [" + computedSaltedHash + "]."); + } + allConsistent.set(false); + } + } + publishedSettingKeysToVerify.remove(concreteSecureSetting.getKey()); + }); + // another case of settings missing locally, when group settings have not expanded to all the keys published + for (String publishedSettingKey : publishedSettingKeysToVerify) { + for (Setting setting : secureSettingsCollection) { + if (setting.match(publishedSettingKey)) { + // setting missing locally but present on master + logger.warn("the consistent secure setting [{}] does not exist on the local node but there is a published hash for it", + publishedSettingKey); + allConsistent.set(false); + } + } + } + return allConsistent.get(); + } + + /** + * Iterate over the passed in secure settings, expanding {@link Setting.AffixSetting} to concrete settings, in the scope of the local + * settings. + */ + private void forEachConcreteSecureSettingDo(Consumer> secureSettingConsumer) { + for (Setting setting : secureSettingsCollection) { + assert setting.isConsistent() : "[" + setting.getKey() + "] is not a consistent setting"; + if (setting instanceof Setting.AffixSetting) { + ((Setting.AffixSetting)setting).getAllConcreteSettings(settings).forEach(concreteSetting -> { + assert concreteSetting instanceof SecureSetting : "[" + concreteSetting.getKey() + "] is not a secure setting"; + secureSettingConsumer.accept((SecureSetting)concreteSetting); + }); + } else if (setting instanceof SecureSetting) { + secureSettingConsumer.accept((SecureSetting) setting); + } else { + assert false : "Unrecognized consistent secure setting [" + setting.getKey() + "]"; + } + } + } + + private Map computeHashesOfConsistentSecureSettings() { + final Map hashesBySettingKey = new HashMap<>(); + forEachConcreteSecureSettingDo(concreteSecureSetting -> { + final byte[] localHash = concreteSecureSetting.getSecretDigest(settings); + if (localHash != null) { + final String salt = UUIDs.randomBase64UUID(); + final byte[] publicHash = computeSaltedPBKDF2Hash(localHash, salt.getBytes(StandardCharsets.UTF_8)); + final String encodedPublicHash = new String(Base64.getEncoder().encode(publicHash), StandardCharsets.UTF_8); + hashesBySettingKey.put(concreteSecureSetting.getKey(), salt + ":" + encodedPublicHash); + } + }); + return hashesBySettingKey; + } + + private byte[] computeSaltedPBKDF2Hash(byte[] bytes, byte[] salt) { + final int iterations = 5000; + final int keyLength = 512; + char[] value = null; + try { + value = MessageDigests.toHexCharArray(bytes); + final PBEKeySpec spec = new PBEKeySpec(value, salt, iterations, keyLength); + final SecretKey key = pbkdf2KeyFactory.generateSecret(spec); + return key.getEncoded(); + } catch (InvalidKeySpecException e) { + throw new RuntimeException("Unexpected exception when computing PBKDF2 hash", e); + } finally { + if (value != null) { + Arrays.fill(value, '0'); + } + } + } + + static final class HashesPublisher implements LocalNodeMasterListener { + + // eagerly compute hashes to be published + final Map computedHashesOfConsistentSettings; + final ClusterService clusterService; + + HashesPublisher(Map computedHashesOfConsistentSettings, ClusterService clusterService) { + this.computedHashesOfConsistentSettings = Map.copyOf(computedHashesOfConsistentSettings); + this.clusterService = clusterService; + } + + @Override + public void onMaster() { + clusterService.submitStateUpdateTask("publish-secure-settings-hashes", new ClusterStateUpdateTask(Priority.URGENT) { + @Override + public ClusterState execute(ClusterState currentState) { + final Map publishedHashesOfConsistentSettings = currentState.metaData() + .hashesOfConsistentSettings(); + if (computedHashesOfConsistentSettings.equals(publishedHashesOfConsistentSettings)) { + logger.debug("Nothing to publish. What is already published matches this node's view."); + return currentState; + } else { + return ClusterState.builder(currentState).metaData(MetaData.builder(currentState.metaData()) + .hashesOfConsistentSettings(computedHashesOfConsistentSettings)).build(); + } + } + + @Override + public void onFailure(String source, Exception e) { + logger.error("unable to publish secure settings hashes", e); + } + + }); + } + + @Override + public void offMaster() { + logger.trace("I am no longer master, nothing to do"); + } + + @Override + public String executorName() { + return ThreadPool.Names.SAME; + } + } + +} diff --git a/server/src/main/java/org/elasticsearch/common/settings/KeyStoreWrapper.java b/server/src/main/java/org/elasticsearch/common/settings/KeyStoreWrapper.java index 64cdd7165f2a3..7ad69c1eebe0c 100644 --- a/server/src/main/java/org/elasticsearch/common/settings/KeyStoreWrapper.java +++ b/server/src/main/java/org/elasticsearch/common/settings/KeyStoreWrapper.java @@ -30,6 +30,7 @@ import org.elasticsearch.cli.ExitCodes; import org.elasticsearch.cli.UserException; import org.elasticsearch.common.Randomness; +import org.elasticsearch.common.hash.MessageDigests; import javax.crypto.Cipher; import javax.crypto.CipherInputStream; @@ -85,6 +86,17 @@ private enum EntryType { FILE } + /** An entry in the keystore. The bytes are opaque and interpreted based on the entry type. */ + private static class Entry { + final byte[] bytes; + final byte[] sha256Digest; + + Entry(byte[] bytes) { + this.bytes = bytes; + this.sha256Digest = MessageDigests.sha256().digest(bytes); + } + } + /** * A regex for the valid characters that a setting name in the keystore may use. */ @@ -148,7 +160,7 @@ private enum EntryType { private final byte[] dataBytes; /** The decrypted secret data. See {@link #decrypt(char[])}. */ - private final SetOnce> entries = new SetOnce<>(); + private final SetOnce> entries = new SetOnce<>(); private volatile boolean closed; private KeyStoreWrapper(int formatVersion, boolean hasPassword, byte[] dataBytes) { @@ -350,7 +362,7 @@ public void decrypt(char[] password) throws GeneralSecurityException, IOExceptio int entrySize = input.readInt(); byte[] entryBytes = new byte[entrySize]; input.readFully(entryBytes); - entries.get().put(setting, entryBytes); + entries.get().put(setting, new Entry(entryBytes)); } if (input.read() != -1) { throw new SecurityException("Keystore has been corrupted or tampered with"); @@ -369,11 +381,11 @@ private byte[] encrypt(char[] password, byte[] salt, byte[] iv) throws GeneralSe try (CipherOutputStream cipherStream = new CipherOutputStream(bytes, cipher); DataOutputStream output = new DataOutputStream(cipherStream)) { output.writeInt(entries.get().size()); - for (Map.Entry mapEntry : entries.get().entrySet()) { + for (Map.Entry mapEntry : entries.get().entrySet()) { output.writeUTF(mapEntry.getKey()); - byte[] entry = mapEntry.getValue(); - output.writeInt(entry.length); - output.write(entry); + byte[] entryBytes = mapEntry.getValue().bytes; + output.writeInt(entryBytes.length); + output.write(entryBytes); } } return bytes.toByteArray(); @@ -448,7 +460,7 @@ private void decryptLegacyEntries() throws GeneralSecurityException, IOException } Arrays.fill(chars, '\0'); - entries.get().put(setting, bytes); + entries.get().put(setting, new Entry(bytes)); } } @@ -521,8 +533,8 @@ public Set getSettingNames() { @Override public synchronized SecureString getString(String setting) { ensureOpen(); - byte[] entry = entries.get().get(setting); - ByteBuffer byteBuffer = ByteBuffer.wrap(entry); + Entry entry = entries.get().get(setting); + ByteBuffer byteBuffer = ByteBuffer.wrap(entry.bytes); CharBuffer charBuffer = StandardCharsets.UTF_8.decode(byteBuffer); return new SecureString(Arrays.copyOfRange(charBuffer.array(), charBuffer.position(), charBuffer.limit())); } @@ -530,8 +542,19 @@ public synchronized SecureString getString(String setting) { @Override public synchronized InputStream getFile(String setting) { ensureOpen(); - byte[] entry = entries.get().get(setting); - return new ByteArrayInputStream(entry); + Entry entry = entries.get().get(setting); + return new ByteArrayInputStream(entry.bytes); + } + + /** + * Returns the SHA256 digest for the setting's value, even after {@code #close()} has been called. The setting must exist. The digest is + * used to check for value changes without actually storing the value. + */ + @Override + public byte[] getSHA256Digest(String setting) { + assert entries.get() != null : "Keystore is not loaded"; + Entry entry = entries.get().get(setting); + return entry.sha256Digest; } /** @@ -553,9 +576,9 @@ synchronized void setString(String setting, char[] value) { ByteBuffer byteBuffer = StandardCharsets.UTF_8.encode(CharBuffer.wrap(value)); byte[] bytes = Arrays.copyOfRange(byteBuffer.array(), byteBuffer.position(), byteBuffer.limit()); - byte[] oldEntry = entries.get().put(setting, bytes); + Entry oldEntry = entries.get().put(setting, new Entry(bytes)); if (oldEntry != null) { - Arrays.fill(oldEntry, (byte)0); + Arrays.fill(oldEntry.bytes, (byte)0); } } @@ -564,18 +587,18 @@ synchronized void setFile(String setting, byte[] bytes) { ensureOpen(); validateSettingName(setting); - byte[] oldEntry = entries.get().put(setting, Arrays.copyOf(bytes, bytes.length)); + Entry oldEntry = entries.get().put(setting, new Entry(Arrays.copyOf(bytes, bytes.length))); if (oldEntry != null) { - Arrays.fill(oldEntry, (byte)0); + Arrays.fill(oldEntry.bytes, (byte)0); } } /** Remove the given setting from the keystore. */ void remove(String setting) { ensureOpen(); - byte[] oldEntry = entries.get().remove(setting); + Entry oldEntry = entries.get().remove(setting); if (oldEntry != null) { - Arrays.fill(oldEntry, (byte)0); + Arrays.fill(oldEntry.bytes, (byte)0); } } @@ -590,8 +613,8 @@ private void ensureOpen() { public synchronized void close() { this.closed = true; if (null != entries.get() && entries.get().isEmpty() == false) { - for (byte[] entry : entries.get().values()) { - Arrays.fill(entry, (byte) 0); + for (Entry entry : entries.get().values()) { + Arrays.fill(entry.bytes, (byte) 0); } } } diff --git a/server/src/main/java/org/elasticsearch/common/settings/SecureSetting.java b/server/src/main/java/org/elasticsearch/common/settings/SecureSetting.java index 33f4718aa45e4..e022e4e3760a5 100644 --- a/server/src/main/java/org/elasticsearch/common/settings/SecureSetting.java +++ b/server/src/main/java/org/elasticsearch/common/settings/SecureSetting.java @@ -37,7 +37,7 @@ public abstract class SecureSetting extends Setting { /** Determines whether legacy settings with sensitive values should be allowed. */ private static final boolean ALLOW_INSECURE_SETTINGS = Booleans.parseBoolean(System.getProperty("es.allow_insecure_settings", "false")); - private static final Set ALLOWED_PROPERTIES = EnumSet.of(Property.Deprecated); + private static final Set ALLOWED_PROPERTIES = EnumSet.of(Property.Deprecated, Property.Consistent); private static final Property[] FIXED_PROPERTIES = { Property.NodeScope @@ -97,6 +97,23 @@ public T get(Settings settings) { } } + /** + * Returns the digest of this secure setting's value or {@code null} if the setting is missing (inside the keystore). This method can be + * called even after the {@code SecureSettings} have been closed, unlike {@code #get(Settings)}. The digest is used to check for changes + * of the value (by re-reading the {@code SecureSettings}), without actually transmitting the value to compare with. + */ + public byte[] getSecretDigest(Settings settings) { + final SecureSettings secureSettings = settings.getSecureSettings(); + if (secureSettings == null || false == secureSettings.getSettingNames().contains(getKey())) { + return null; + } + try { + return secureSettings.getSHA256Digest(getKey()); + } catch (GeneralSecurityException e) { + throw new RuntimeException("failed to read secure setting " + getKey(), e); + } + } + /** Returns the secret setting from the keyStoreReader store. */ abstract T getSecret(SecureSettings secureSettings) throws GeneralSecurityException; diff --git a/server/src/main/java/org/elasticsearch/common/settings/SecureSettings.java b/server/src/main/java/org/elasticsearch/common/settings/SecureSettings.java index 98f980c1ec6c8..7f92b382dd7b1 100644 --- a/server/src/main/java/org/elasticsearch/common/settings/SecureSettings.java +++ b/server/src/main/java/org/elasticsearch/common/settings/SecureSettings.java @@ -42,6 +42,8 @@ public interface SecureSettings extends Closeable { /** Return a file setting. The {@link InputStream} should be closed once it is used. */ InputStream getFile(String setting) throws GeneralSecurityException; + byte[] getSHA256Digest(String setting) throws GeneralSecurityException; + @Override void close() throws IOException; } diff --git a/server/src/main/java/org/elasticsearch/common/settings/Setting.java b/server/src/main/java/org/elasticsearch/common/settings/Setting.java index 1e5079124c345..d4164b474de0b 100644 --- a/server/src/main/java/org/elasticsearch/common/settings/Setting.java +++ b/server/src/main/java/org/elasticsearch/common/settings/Setting.java @@ -112,6 +112,11 @@ public enum Property { */ NodeScope, + /** + * Secure setting values equal on all nodes + */ + Consistent, + /** * Index scope */ @@ -167,6 +172,7 @@ private Setting(Key key, @Nullable Setting fallbackSetting, Function properties, } } + private void checkPropertyRequiresNodeScope(final EnumSet properties, final Property property) { + if (properties.contains(property) && properties.contains(Property.NodeScope) == false) { + throw new IllegalArgumentException("non-node-scoped setting [" + key + "] can not have property [" + property + "]"); + } + } + /** * Creates a new Setting instance * @param key the settings key for this setting. @@ -321,6 +333,14 @@ public boolean hasNodeScope() { return properties.contains(Property.NodeScope); } + /** + * Returns true if this setting's value can be checked for equality across all nodes. Only {@link SecureSetting} instances + * may have this qualifier. + */ + public boolean isConsistent() { + return properties.contains(Property.Consistent); + } + /** * Returns true if this setting has an index scope, otherwise false */ diff --git a/server/src/main/java/org/elasticsearch/common/settings/Settings.java b/server/src/main/java/org/elasticsearch/common/settings/Settings.java index 5789abf76d8b0..e92d53f801183 100644 --- a/server/src/main/java/org/elasticsearch/common/settings/Settings.java +++ b/server/src/main/java/org/elasticsearch/common/settings/Settings.java @@ -1324,15 +1324,20 @@ public Set getSettingNames() { } @Override - public SecureString getString(String setting) throws GeneralSecurityException{ + public SecureString getString(String setting) throws GeneralSecurityException { return delegate.getString(addPrefix.apply(setting)); } @Override - public InputStream getFile(String setting) throws GeneralSecurityException{ + public InputStream getFile(String setting) throws GeneralSecurityException { return delegate.getFile(addPrefix.apply(setting)); } + @Override + public byte[] getSHA256Digest(String setting) throws GeneralSecurityException { + return delegate.getSHA256Digest(addPrefix.apply(setting)); + } + @Override public void close() throws IOException { delegate.close(); diff --git a/server/src/main/java/org/elasticsearch/common/settings/SettingsModule.java b/server/src/main/java/org/elasticsearch/common/settings/SettingsModule.java index 6a78e81d7f3f4..58c9cbc520456 100644 --- a/server/src/main/java/org/elasticsearch/common/settings/SettingsModule.java +++ b/server/src/main/java/org/elasticsearch/common/settings/SettingsModule.java @@ -49,6 +49,7 @@ public class SettingsModule implements Module { private final Set settingsFilterPattern = new HashSet<>(); private final Map> nodeSettings = new HashMap<>(); private final Map> indexSettings = new HashMap<>(); + private final Set> consistentSettings = new HashSet<>(); private final IndexScopedSettings indexScopedSettings; private final ClusterSettings clusterSettings; private final SettingsFilter settingsFilter; @@ -157,7 +158,6 @@ public void configure(Binder binder) { binder.bind(IndexScopedSettings.class).toInstance(indexScopedSettings); } - /** * Registers a new setting. This method should be used by plugins in order to expose any custom settings the plugin defines. * Unless a setting is registered the setting is unusable. If a setting is never the less specified the node will reject @@ -175,6 +175,19 @@ private void registerSetting(Setting setting) { if (existingSetting != null) { throw new IllegalArgumentException("Cannot register setting [" + setting.getKey() + "] twice"); } + if (setting.isConsistent()) { + if (setting instanceof Setting.AffixSetting) { + if (((Setting.AffixSetting)setting).getConcreteSettingForNamespace("_na_") instanceof SecureSetting) { + consistentSettings.add(setting); + } else { + throw new IllegalArgumentException("Invalid consistent secure setting [" + setting.getKey() + "]"); + } + } else if (setting instanceof SecureSetting) { + consistentSettings.add(setting); + } else { + throw new IllegalArgumentException("Invalid consistent secure setting [" + setting.getKey() + "]"); + } + } nodeSettings.put(setting.getKey(), setting); } if (setting.hasIndexScope()) { @@ -182,6 +195,9 @@ private void registerSetting(Setting setting) { if (existingSetting != null) { throw new IllegalArgumentException("Cannot register setting [" + setting.getKey() + "] twice"); } + if (setting.isConsistent()) { + throw new IllegalStateException("Consistent setting [" + setting.getKey() + "] cannot be index scoped"); + } indexSettings.put(setting.getKey(), setting); } } else { @@ -215,6 +231,10 @@ public ClusterSettings getClusterSettings() { return clusterSettings; } + public Set> getConsistentSettings() { + return consistentSettings; + } + public SettingsFilter getSettingsFilter() { return settingsFilter; } diff --git a/server/src/main/java/org/elasticsearch/common/time/DateMathParser.java b/server/src/main/java/org/elasticsearch/common/time/DateMathParser.java index 3ba392822ca0c..fc2d231bb2f6c 100644 --- a/server/src/main/java/org/elasticsearch/common/time/DateMathParser.java +++ b/server/src/main/java/org/elasticsearch/common/time/DateMathParser.java @@ -31,7 +31,7 @@ public interface DateMathParser { /** - * Parse a date math expression without timzeone info and rounding down. + * Parse a date math expression without timezone info and rounding down. */ default Instant parse(String text, LongSupplier now) { return parse(text, now, false, (ZoneId) null); @@ -43,8 +43,8 @@ default Instant parse(String text, LongSupplier now) { // exists for backcompat, do not use! @Deprecated - default Instant parse(String text, LongSupplier now, boolean roundUp, DateTimeZone tz) { - return parse(text, now, roundUp, tz == null ? null : ZoneId.of(tz.getID())); + default Instant parse(String text, LongSupplier now, boolean roundUpProperty, DateTimeZone tz) { + return parse(text, now, roundUpProperty, tz == null ? null : ZoneId.of(tz.getID())); } /** @@ -65,11 +65,11 @@ default Instant parse(String text, LongSupplier now, boolean roundUp, DateTimeZo * s second * * - * @param text the input - * @param now a supplier to retrieve the current date in milliseconds, if needed for additions - * @param roundUp should the result be rounded up - * @param tz an optional timezone that should be applied before returning the milliseconds since the epoch - * @return the parsed date as an Instant since the epoch + * @param text the input + * @param now a supplier to retrieve the current date in milliseconds, if needed for additions + * @param roundUpProperty should the result be rounded up with the granularity of the rounding (e.g. now/M) + * @param tz an optional timezone that should be applied before returning the milliseconds since the epoch + * @return the parsed date as an Instant since the epoch */ - Instant parse(String text, LongSupplier now, boolean roundUp, ZoneId tz); + Instant parse(String text, LongSupplier now, boolean roundUpProperty, ZoneId tz); } diff --git a/server/src/main/java/org/elasticsearch/common/time/JavaDateFormatter.java b/server/src/main/java/org/elasticsearch/common/time/JavaDateFormatter.java index d0f4200b3bafe..3513600582278 100644 --- a/server/src/main/java/org/elasticsearch/common/time/JavaDateFormatter.java +++ b/server/src/main/java/org/elasticsearch/common/time/JavaDateFormatter.java @@ -38,6 +38,7 @@ import java.util.Map; import java.util.Objects; import java.util.function.Consumer; +import java.util.stream.Collectors; class JavaDateFormatter implements DateFormatter { @@ -58,6 +59,12 @@ class JavaDateFormatter implements DateFormatter { private final List parsers; private final DateTimeFormatter roundupParser; + private JavaDateFormatter(String format, DateTimeFormatter printer, DateTimeFormatter roundupParser, List parsers) { + this.format = format; + this.printer = printer; + this.roundupParser = roundupParser; + this.parsers = parsers; + } JavaDateFormatter(String format, DateTimeFormatter printer, DateTimeFormatter... parsers) { this(format, printer, builder -> ROUND_UP_BASE_FIELDS.forEach(builder::parseDefaulting), parsers); } @@ -155,9 +162,8 @@ public DateFormatter withZone(ZoneId zoneId) { if (zoneId.equals(zone())) { return this; } - - return new JavaDateFormatter(format, printer.withZone(zoneId), - parsers.stream().map(p -> p.withZone(zoneId)).toArray(size -> new DateTimeFormatter[size])); + return new JavaDateFormatter(format, printer.withZone(zoneId), getRoundupParser().withZone(zoneId), + parsers.stream().map(p -> p.withZone(zoneId)).collect(Collectors.toList())); } @Override @@ -166,9 +172,8 @@ public DateFormatter withLocale(Locale locale) { if (locale.equals(locale())) { return this; } - - return new JavaDateFormatter(format, printer.withLocale(locale), - parsers.stream().map(p -> p.withLocale(locale)).toArray(size -> new DateTimeFormatter[size])); + return new JavaDateFormatter(format, printer.withLocale(locale), getRoundupParser().withLocale(locale), + parsers.stream().map(p -> p.withLocale(locale)).collect(Collectors.toList())); } @Override diff --git a/server/src/main/java/org/elasticsearch/common/time/JavaDateMathParser.java b/server/src/main/java/org/elasticsearch/common/time/JavaDateMathParser.java index dc7c195e2fd6c..78d4f10d87cbf 100644 --- a/server/src/main/java/org/elasticsearch/common/time/JavaDateMathParser.java +++ b/server/src/main/java/org/elasticsearch/common/time/JavaDateMathParser.java @@ -59,7 +59,7 @@ public class JavaDateMathParser implements DateMathParser { } @Override - public Instant parse(String text, LongSupplier now, boolean roundUp, ZoneId timeZone) { + public Instant parse(String text, LongSupplier now, boolean roundUpProperty, ZoneId timeZone) { Instant time; String mathString; if (text.startsWith("now")) { @@ -73,16 +73,16 @@ public Instant parse(String text, LongSupplier now, boolean roundUp, ZoneId time } else { int index = text.indexOf("||"); if (index == -1) { - return parseDateTime(text, timeZone, roundUp); + return parseDateTime(text, timeZone, roundUpProperty); } time = parseDateTime(text.substring(0, index), timeZone, false); mathString = text.substring(index + 2); } - return parseMath(mathString, time, roundUp, timeZone); + return parseMath(mathString, time, roundUpProperty, timeZone); } - private Instant parseMath(final String mathString, final Instant time, final boolean roundUp, + private Instant parseMath(final String mathString, final Instant time, final boolean roundUpProperty, ZoneId timeZone) throws ElasticsearchParseException { if (timeZone == null) { timeZone = ZoneOffset.UTC; @@ -133,78 +133,79 @@ private Instant parseMath(final String mathString, final Instant time, final boo case 'y': if (round) { dateTime = dateTime.withDayOfYear(1).with(LocalTime.MIN); + if (roundUpProperty) { + dateTime = dateTime.plusYears(1); + } } else { dateTime = dateTime.plusYears(sign * num); } - if (roundUp) { - dateTime = dateTime.plusYears(1); - } break; case 'M': if (round) { dateTime = dateTime.withDayOfMonth(1).with(LocalTime.MIN); + if (roundUpProperty) { + dateTime = dateTime.plusMonths(1); + } } else { dateTime = dateTime.plusMonths(sign * num); } - if (roundUp) { - dateTime = dateTime.plusMonths(1); - } break; case 'w': if (round) { dateTime = dateTime.with(TemporalAdjusters.previousOrSame(DayOfWeek.MONDAY)).with(LocalTime.MIN); + if (roundUpProperty) { + dateTime = dateTime.plusWeeks(1); + } } else { dateTime = dateTime.plusWeeks(sign * num); } - if (roundUp) { - dateTime = dateTime.plusWeeks(1); - } break; case 'd': if (round) { dateTime = dateTime.with(LocalTime.MIN); + if (roundUpProperty) { + dateTime = dateTime.plusDays(1); + } } else { dateTime = dateTime.plusDays(sign * num); } - if (roundUp) { - dateTime = dateTime.plusDays(1); - } break; case 'h': case 'H': if (round) { dateTime = dateTime.withMinute(0).withSecond(0).withNano(0); + if (roundUpProperty) { + dateTime = dateTime.plusHours(1); + } } else { dateTime = dateTime.plusHours(sign * num); } - if (roundUp) { - dateTime = dateTime.plusHours(1); - } break; case 'm': if (round) { dateTime = dateTime.withSecond(0).withNano(0); + if (roundUpProperty) { + dateTime = dateTime.plusMinutes(1); + } } else { dateTime = dateTime.plusMinutes(sign * num); } - if (roundUp) { - dateTime = dateTime.plusMinutes(1); - } break; case 's': if (round) { dateTime = dateTime.withNano(0); + if (roundUpProperty) { + dateTime = dateTime.plusSeconds(1); + } } else { dateTime = dateTime.plusSeconds(sign * num); } - if (roundUp) { - dateTime = dateTime.plusSeconds(1); - } break; default: throw new ElasticsearchParseException("unit [{}] not supported for date math [{}]", unit, mathString); } - if (roundUp) { + if (round && roundUpProperty) { + // subtract 1 millisecond to get the largest inclusive value dateTime = dateTime.minus(1, ChronoField.MILLI_OF_SECOND.getBaseUnit()); } } diff --git a/server/src/main/java/org/elasticsearch/common/util/CuckooFilter.java b/server/src/main/java/org/elasticsearch/common/util/CuckooFilter.java new file mode 100644 index 0000000000000..54099735fba47 --- /dev/null +++ b/server/src/main/java/org/elasticsearch/common/util/CuckooFilter.java @@ -0,0 +1,521 @@ +/* + * 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; + +import org.apache.lucene.store.DataInput; +import org.apache.lucene.store.DataOutput; +import org.apache.lucene.util.packed.PackedInts; +import org.elasticsearch.common.io.stream.StreamInput; +import org.elasticsearch.common.io.stream.StreamOutput; +import org.elasticsearch.common.io.stream.Writeable; + +import java.io.IOException; +import java.util.Iterator; +import java.util.Objects; +import java.util.Random; + +/** + * An approximate set membership datastructure + * + * CuckooFilters are similar to Bloom Filters in usage; values are inserted, and the Cuckoo + * can be asked if it has seen a particular value before. Because the structure is approximate, + * it can return false positives (says it has seen an item when it has not). False negatives + * are not possible though; if the structure says it _has not_ seen an item, that can be + * trusted. + * + * The filter can "saturate" at which point the map has hit it's configured load factor (or near enough + * that a large number of evictions are not able to find a free slot) and will refuse to accept + * any new insertions. + * + * NOTE: this version does not support deletions, and as such does not save duplicate + * fingerprints (e.g. when inserting, if the fingerprint is already present in the + * candidate buckets, it is not inserted). By not saving duplicates, the CuckooFilter + * loses the ability to delete values. But not by allowing deletions, we can save space + * (do not need to waste slots on duplicate fingerprints), and we do not need to worry + * about inserts "overflowing" a bucket because the same item has been repeated repeatedly + * + * NOTE: this CuckooFilter exposes a number of Expert APIs which assume the caller has + * intimate knowledge about how the algorithm works. It is recommended to use + * {@link SetBackedScalingCuckooFilter} instead. + * + * Based on the paper: + * + * Fan, Bin, et al. "Cuckoo filter: Practically better than bloom." + * Proceedings of the 10th ACM International on Conference on emerging Networking Experiments and Technologies. ACM, 2014. + * + * https://www.cs.cmu.edu/~dga/papers/cuckoo-conext2014.pdf + */ +public class CuckooFilter implements Writeable { + + private static final double LN_2 = Math.log(2); + private static final int MAX_EVICTIONS = 500; + static final int EMPTY = 0; + + private final PackedInts.Mutable data; + private final int numBuckets; + private final int bitsPerEntry; + private final int fingerprintMask; + private final int entriesPerBucket; + private final Random rng; + private int count; + private int evictedFingerprint = EMPTY; + + /** + * @param capacity The number of expected inserts. The filter can hold more than this value, it is just an estimate + * @param fpp The desired false positive rate. Smaller values will reduce the + * false positives at expense of larger size + * @param rng A random number generator, used with the cuckoo hashing process + */ + CuckooFilter(long capacity, double fpp, Random rng) { + this.rng = rng; + this.entriesPerBucket = entriesPerBucket(fpp); + double loadFactor = getLoadFactor(entriesPerBucket); + this.bitsPerEntry = bitsPerEntry(fpp, entriesPerBucket); + this.numBuckets = getNumBuckets(capacity, loadFactor, entriesPerBucket); + + if ((long) numBuckets * (long) entriesPerBucket > Integer.MAX_VALUE) { + throw new IllegalArgumentException("Attempted to create [" + numBuckets * entriesPerBucket + + "] entries which is > Integer.MAX_VALUE"); + } + this.data = PackedInts.getMutable(numBuckets * entriesPerBucket, bitsPerEntry, PackedInts.COMPACT); + + // puts the bits at the right side of the mask, e.g. `0000000000001111` for bitsPerEntry = 4 + this.fingerprintMask = (0x80000000 >> (bitsPerEntry - 1)) >>> (Integer.SIZE - bitsPerEntry); + } + + /** + * This ctor is likely slow and should only be used for testing + */ + CuckooFilter(CuckooFilter other) { + this.numBuckets = other.numBuckets; + this.bitsPerEntry = other.bitsPerEntry; + this.entriesPerBucket = other.entriesPerBucket; + this.count = other.count; + this.evictedFingerprint = other.evictedFingerprint; + this.rng = other.rng; + this.fingerprintMask = other.fingerprintMask; + + // This shouldn't happen, but as a sanity check + if ((long) numBuckets * (long) entriesPerBucket > Integer.MAX_VALUE) { + throw new IllegalArgumentException("Attempted to create [" + numBuckets * entriesPerBucket + + "] entries which is > Integer.MAX_VALUE"); + } + // TODO this is probably super slow, but just used for testing atm + this.data = PackedInts.getMutable(numBuckets * entriesPerBucket, bitsPerEntry, PackedInts.COMPACT); + for (int i = 0; i < other.data.size(); i++) { + data.set(i, other.data.get(i)); + } + } + + CuckooFilter(StreamInput in, Random rng) throws IOException { + this.numBuckets = in.readVInt(); + this.bitsPerEntry = in.readVInt(); + this.entriesPerBucket = in.readVInt(); + this.count = in.readVInt(); + this.evictedFingerprint = in.readVInt(); + this.rng = rng; + + this.fingerprintMask = (0x80000000 >> (bitsPerEntry - 1)) >>> (Integer.SIZE - bitsPerEntry); + + data = (PackedInts.Mutable) PackedInts.getReader(new DataInput() { + @Override + public byte readByte() throws IOException { + return in.readByte(); + } + + @Override + public void readBytes(byte[] b, int offset, int len) throws IOException { + in.readBytes(b, offset, len); + } + }); + } + + @Override + public void writeTo(StreamOutput out) throws IOException { + out.writeVInt(numBuckets); + out.writeVInt(bitsPerEntry); + out.writeVInt(entriesPerBucket); + out.writeVInt(count); + out.writeVInt(evictedFingerprint); + + data.save(new DataOutput() { + @Override + public void writeByte(byte b) throws IOException { + out.writeByte(b); + } + + @Override + public void writeBytes(byte[] b, int offset, int length) throws IOException { + out.writeBytes(b, offset, length); + } + }); + } + + /** + * Get the number of unique items that are being tracked + */ + public int getCount() { + return count; + } + + /** + * Returns the number of buckets that has been chosen based + * on the initial configuration + * + * Expert-level API + */ + int getNumBuckets() { + return numBuckets; + } + + /** + * Returns the number of bits used per entry + * + * Expert-level API + */ + int getBitsPerEntry() { + return bitsPerEntry; + } + + /** + * Returns the cached fingerprint mask. This is simply a mask for the + * first bitsPerEntry bits, used by {@link CuckooFilter#fingerprint(int, int, int)} + * to generate the fingerprint of a hash + * + * Expert-level API + */ + int getFingerprintMask() { + return fingerprintMask; + } + + /** + * Returns an iterator that returns the long[] representation of each bucket. The value + * inside each long will be a fingerprint (or 0L, representing empty). + * + * Expert-level API + */ + Iterator getBuckets() { + return new Iterator<>() { + int current = 0; + + @Override + public boolean hasNext() { + return current < numBuckets; + } + + @Override + public long[] next() { + long[] values = new long[entriesPerBucket]; + int offset = getOffset(current, 0); + data.get(offset, values, 0, entriesPerBucket); + current += 1; + return values; + } + }; + } + + /** + * Returns true if the set might contain the provided value, false otherwise. False values are + * 100% accurate, while true values may be a false-positive. + */ + boolean mightContain(long hash) { + int bucket = hashToIndex((int) hash, numBuckets); + int fingerprint = fingerprint((int) (hash >>> 32), bitsPerEntry, fingerprintMask); + int alternateIndex = alternateIndex(bucket, fingerprint, numBuckets); + + return mightContainFingerprint(bucket, fingerprint, alternateIndex); + } + + /** + * Returns true if the bucket or it's alternate bucket contains the fingerprint. + * + * Expert-level API, use {@link CuckooFilter#mightContain(long)} to check if + * a value is in the filter. + */ + boolean mightContainFingerprint(int bucket, int fingerprint, int alternateBucket) { + + // check all entries for both buckets and the evicted slot + return hasFingerprint(bucket, fingerprint) || hasFingerprint(alternateBucket, fingerprint) || evictedFingerprint == fingerprint; + } + + /** + * Return's true if any of the entries in the bucket contain the fingerprint + */ + private boolean hasFingerprint(int bucket, long fingerprint) { + long[] values = new long[entriesPerBucket]; + int offset = getOffset(bucket, 0); + data.get(offset, values, 0, entriesPerBucket); + + for (int i = 0; i < entriesPerBucket; i++) { + if (values[i] == fingerprint) { + return true; + } + } + return false; + } + + /** + * Add's the hash to the bucket or alternate bucket. Returns true if the insertion was + * successful, false if the filter is saturated. + */ + boolean add(long hash) { + // Each bucket needs 32 bits, so we truncate for the first bucket and shift/truncate for second + int bucket = hashToIndex((int) hash, numBuckets); + int fingerprint = fingerprint((int) (hash >>> 32), bitsPerEntry, fingerprintMask); + return mergeFingerprint(bucket, fingerprint); + } + + /** + * Attempts to merge the fingerprint into the specified bucket or it's alternate bucket. + * Returns true if the insertion was successful, false if the filter is saturated. + * + * Expert-level API, use {@link CuckooFilter#add(long)} to insert + * values into the filter + */ + boolean mergeFingerprint(int bucket, int fingerprint) { + // If we already have an evicted fingerprint we are full, no need to try + if (evictedFingerprint != EMPTY) { + return false; + } + + int alternateBucket = alternateIndex(bucket, fingerprint, numBuckets); + if (tryInsert(bucket, fingerprint) || tryInsert(alternateBucket, fingerprint)) { + count += 1; + return true; + } + + for (int i = 0; i < MAX_EVICTIONS; i++) { + // overwrite our alternate bucket, and a random entry + int offset = getOffset(alternateBucket, rng.nextInt(entriesPerBucket - 1)); + int oldFingerprint = (int) data.get(offset); + data.set(offset, fingerprint); + + // replace details and start again + fingerprint = oldFingerprint; + bucket = alternateBucket; + alternateBucket = alternateIndex(bucket, fingerprint, numBuckets); + + // Only try to insert into alternate bucket + if (tryInsert(alternateBucket, fingerprint)) { + count += 1; + return true; + } + } + + // If we get this far, we failed to insert the value after MAX_EVICTION rounds, + // so cache the last evicted value (so we don't lose it) and signal we failed + evictedFingerprint = fingerprint; + return false; + } + + /** + * Low-level insert method. Attempts to write the fingerprint into an empty entry + * at this bucket's position. Returns true if that was sucessful, false if all entries + * were occupied. + * + * If the fingerprint already exists in one of the entries, it will not duplicate the + * fingerprint like the original paper. This means the filter _cannot_ support deletes, + * but is not sensitive to "overflowing" buckets with repeated inserts + */ + private boolean tryInsert(int bucket, int fingerprint) { + long[] values = new long[entriesPerBucket]; + int offset = getOffset(bucket, 0); + data.get(offset, values, 0, entriesPerBucket); + + // TODO implement semi-sorting + for (int i = 0; i < values.length; i++) { + if (values[i] == EMPTY) { + data.set(offset + i, fingerprint); + return true; + } else if (values[i] == fingerprint) { + // Already have the fingerprint, no need to save + return true; + } + } + return false; + } + + /** + * Converts a hash into a bucket index (primary or alternate). + * + * If the hash is negative, this flips the bits. The hash is then modulo numBuckets + * to get the final index. + * + * Expert-level API + */ + static int hashToIndex(int hash, int numBuckets) { + return hash & (numBuckets - 1); + } + + /** + * Calculates the alternate bucket for a given bucket:fingerprint tuple + * + * The alternate bucket is the fingerprint multiplied by a mixing constant, + * then xor'd against the bucket. This new value is modulo'd against + * the buckets via {@link CuckooFilter#hashToIndex(int, int)} to get the final + * index. + * + * Note that the xor makes this operation reversible as long as we have the + * fingerprint and current bucket (regardless of if that bucket was the primary + * or alternate). + * + * Expert-level API + */ + static int alternateIndex(int bucket, int fingerprint, int numBuckets) { + /* + Reference impl uses murmur2 mixing constant: + https://github.com/efficient/cuckoofilter/blob/master/src/cuckoofilter.h#L78 + // NOTE(binfan): originally we use: + // index ^ HashUtil::BobHash((const void*) (&tag), 4)) & table_->INDEXMASK; + // now doing a quick-n-dirty way: + // 0x5bd1e995 is the hash constant from MurmurHash2 + return IndexHash((uint32_t)(index ^ (tag * 0x5bd1e995))); + */ + int index = bucket ^ (fingerprint * 0x5bd1e995); + return hashToIndex(index, numBuckets); + } + + /** + * Given the bucket and entry position, returns the absolute offset + * inside the PackedInts datastructure + */ + private int getOffset(int bucket, int position) { + return (bucket * entriesPerBucket) + position; + } + + /** + * Calculates the fingerprint for a given hash. + * + * The fingerprint is simply the first `bitsPerEntry` number of bits that are non-zero. + * If the entire hash is zero, `(int) 1` is used + * + * Expert-level API + */ + static int fingerprint(int hash, int bitsPerEntry, int fingerprintMask) { + if (hash == 0) { + // we use 0 as "empty" so if the hash actually hashes to zero... return 1 + // Some other impls will re-hash with a salt but this seems simpler + return 1; + } + + for (int i = 0; i + bitsPerEntry <= Long.SIZE; i += bitsPerEntry) { + int v = (hash >> i) & fingerprintMask; + if (v != 0) { + return v; + } + } + return 1; + } + + /** + * Calculate the optimal number of bits per entry + */ + private int bitsPerEntry(double fpp, int numEntriesPerBucket) { + return (int) Math.round(log2((2 * numEntriesPerBucket) / fpp)); + } + + /** + * Calculate the optimal number of entries per bucket. Will return 2, 4 or 8 + * depending on the false positive rate + */ + private int entriesPerBucket(double fpp) { + /* + Empirical constants from paper: + "the space-optimal bucket size depends on the target false positive rate ε: + when ε > 0.002, having two entries per bucket yields slightly better results + than using four entries per bucket; when ε decreases to 0.00001 < ε <= 0.002, + four entries per bucket minimzes space" + */ + + if (fpp > 0.002) { + return 2; + } else if (fpp > 0.00001 && fpp <= 0.002) { + return 4; + } + return 8; + } + + /** + * Calculates the optimal load factor for the filter, given the number of entries + * per bucket. Will return 0.84, 0.955 or 0.98 depending on b + */ + private double getLoadFactor(int b) { + if ((b == 2 || b == 4 || b == 8) == false) { + throw new IllegalArgumentException("b must be one of [2,4,8]"); + } + /* + Empirical constants from the paper: + "With k = 2 hash functions, the load factor α is 50% when bucket size b = 1 (i.e + the hash table is directly mapped), but increases to 84%, 95%, 98% respectively + using bucket size b = 2, 4, 8" + */ + if (b == 2) { + return 0.84D; + } else if (b == 4) { + return 0.955D; + } else { + return 0.98D; + } + } + + /** + * Calculates the optimal number of buckets for this filter. The xor used in the bucketing + * algorithm requires this to be a power of two, so the optimal number of buckets will + * be rounded to the next largest power of two where applicable. + * + * TODO: there are schemes to avoid powers of two, might want to investigate those + */ + private int getNumBuckets(long capacity, double loadFactor, int b) { + long buckets = Math.round((((double) capacity / loadFactor)) / (double) b); + + // Rounds up to nearest power of 2 + return 1 << -Integer.numberOfLeadingZeros((int)buckets - 1); + } + + private double log2(double x) { + return Math.log(x) / LN_2; + } + + public long getSizeInBytes() { + // (numBuckets, bitsPerEntry, fingerprintMask, entriesPerBucket, count, evictedFingerprint) * 4b == 24b + return data.ramBytesUsed() + 24; + } + + @Override + public int hashCode() { + return Objects.hash(numBuckets, bitsPerEntry, entriesPerBucket, count, evictedFingerprint); + } + + @Override + public boolean equals(Object other) { + if (this == other) { + return true; + } + if (other == null || getClass() != other.getClass()) { + return false; + } + + final CuckooFilter that = (CuckooFilter) other; + return Objects.equals(this.numBuckets, that.numBuckets) + && Objects.equals(this.bitsPerEntry, that.bitsPerEntry) + && Objects.equals(this.entriesPerBucket, that.entriesPerBucket) + && Objects.equals(this.count, that.count) + && Objects.equals(this.evictedFingerprint, that.evictedFingerprint); + } +} diff --git a/server/src/main/java/org/elasticsearch/common/util/SetBackedScalingCuckooFilter.java b/server/src/main/java/org/elasticsearch/common/util/SetBackedScalingCuckooFilter.java new file mode 100644 index 0000000000000..0c754aa6e895c --- /dev/null +++ b/server/src/main/java/org/elasticsearch/common/util/SetBackedScalingCuckooFilter.java @@ -0,0 +1,408 @@ +/* + * 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; + +import org.apache.lucene.util.BytesRef; +import org.elasticsearch.common.hash.MurmurHash3; +import org.elasticsearch.common.io.stream.StreamInput; +import org.elasticsearch.common.io.stream.StreamOutput; +import org.elasticsearch.common.io.stream.Writeable; + +import java.io.IOException; +import java.util.ArrayList; +import java.util.HashSet; +import java.util.Iterator; +import java.util.List; +import java.util.Objects; +import java.util.Random; +import java.util.Set; +import java.util.function.Consumer; + +/** + * An approximate set membership datastructure that scales as more unique values are inserted. + * Can definitively say if a member does not exist (no false negatives), but may say an item exists + * when it does not (has false positives). Similar in usage to a Bloom Filter. + * + * Internally, the datastructure maintains a Set of hashes up to a specified threshold. This provides + * 100% accurate membership queries. + * + * When the threshold is breached, a list of CuckooFilters are created and used to track membership. + * These filters are approximate similar to Bloom Filters. + * + * This datastructure scales as more values are inserted by growing the list of CuckooFilters. + * Final size is dependent on the cardinality of data inserted, and the precision specified. + */ +public class SetBackedScalingCuckooFilter implements Writeable { + + /** + * This is the estimated insertion capacity for each individual internal CuckooFilter. + */ + private static final int FILTER_CAPACITY = 1000000; + + /** + * This set is used to track the insertions before we convert over to an approximate + * filter. This gives us 100% accuracy for small cardinalities. This will be null + * if isSetMode = false; + * + * package-private for testing + */ + Set hashes; + + /** + * This list holds our approximate filters, after we have migrated out of a set. + * This will be null if isSetMode = true; + */ + List filters; + + private final int threshold; + private final Random rng; + private final int capacity; + private final double fpp; + private Consumer breaker = aLong -> { + //noop + }; + + // cached here for performance reasons + private int numBuckets = 0; + private int bitsPerEntry = 0; + private int fingerprintMask = 0; + private MurmurHash3.Hash128 scratchHash = new MurmurHash3.Hash128(); + + // True if we are tracking inserts with a set, false otherwise + private boolean isSetMode = true; + + /** + * @param threshold The number of distinct values that should be tracked + * before converting to an approximate representation + * @param rng A random number generator needed for the cuckoo hashing process + * @param fpp the false-positive rate that should be used for the cuckoo filters. + */ + public SetBackedScalingCuckooFilter(int threshold, Random rng, double fpp) { + if (threshold <= 0) { + throw new IllegalArgumentException("[threshold] must be a positive integer"); + } + + // We have to ensure that, in the worst case, two full sets can be converted into + // one cuckoo filter without overflowing. This keeps merging logic simpler + if (threshold * 2 > FILTER_CAPACITY) { + throw new IllegalArgumentException("[threshold] must be smaller than [" + (FILTER_CAPACITY / 2) + "]"); + } + if (fpp < 0) { + throw new IllegalArgumentException("[fpp] must be a positive double"); + } + this.hashes = new HashSet<>(threshold); + this.threshold = threshold; + this.rng = rng; + this.capacity = FILTER_CAPACITY; + this.fpp = fpp; + } + + public SetBackedScalingCuckooFilter(SetBackedScalingCuckooFilter other) { + this.threshold = other.threshold; + this.isSetMode = other.isSetMode; + this.rng = other.rng; + this.breaker = other.breaker; + this.capacity = other.capacity; + this.fpp = other.fpp; + if (isSetMode) { + this.hashes = new HashSet<>(other.hashes); + } else { + this.filters = new ArrayList<>(other.filters); + this.numBuckets = filters.get(0).getNumBuckets(); + this.fingerprintMask = filters.get(0).getFingerprintMask(); + this.bitsPerEntry = filters.get(0).getBitsPerEntry(); + } + } + + public SetBackedScalingCuckooFilter(StreamInput in, Random rng) throws IOException { + this.threshold = in.readVInt(); + this.isSetMode = in.readBoolean(); + this.rng = rng; + this.capacity = in.readVInt(); + this.fpp = in.readDouble(); + + if (isSetMode) { + this.hashes = in.readSet(StreamInput::readZLong); + } else { + this.filters = in.readList(in12 -> new CuckooFilter(in12, rng)); + this.numBuckets = filters.get(0).getNumBuckets(); + this.fingerprintMask = filters.get(0).getFingerprintMask(); + this.bitsPerEntry = filters.get(0).getBitsPerEntry(); + } + } + + @Override + public void writeTo(StreamOutput out) throws IOException { + out.writeVInt(threshold); + out.writeBoolean(isSetMode); + out.writeVInt(capacity); + out.writeDouble(fpp); + if (isSetMode) { + out.writeCollection(hashes, StreamOutput::writeZLong); + } else { + out.writeList(filters); + } + } + + /** + * Registers a circuit breaker with the datastructure. + * + * CuckooFilter's can "saturate" and refuse to accept any new values. When this happens, + * the datastructure scales by adding a new filter. This new filter's bytes will be tracked + * in the registered breaker when configured. + */ + public void registerBreaker(Consumer breaker) { + this.breaker = Objects.requireNonNull(breaker, "Circuit Breaker Consumer cannot be null"); + breaker.accept(getSizeInBytes()); + } + + /** + * Returns true if the set might contain the provided value, false otherwise. False values are + * 100% accurate, while true values may be a false-positive. + */ + public boolean mightContain(BytesRef value) { + MurmurHash3.Hash128 hash = MurmurHash3.hash128(value.bytes, value.offset, value.length, 0, scratchHash); + return mightContainHash(hash.h1); + } + + /** + * Returns true if the set might contain the provided value, false otherwise. False values are + * 100% accurate, while true values may be a false-positive. + */ + public boolean mightContain(long value) { + long hash = MurmurHash3.murmur64(value); + return mightContainHash(hash); + } + + /** + * Returns true if the set might contain the provided value, false otherwise. False values are + * 100% accurate, while true values may be a false-positive. + */ + private boolean mightContainHash(long hash) { + if (isSetMode) { + return hashes.contains(hash); + } + + // We calculate these once up front for all the filters and use the expert API + int bucket = CuckooFilter.hashToIndex((int) hash, numBuckets); + int fingerprint = CuckooFilter.fingerprint((int) (hash >> 32), bitsPerEntry, fingerprintMask); + int alternateIndex = CuckooFilter.alternateIndex(bucket, fingerprint, numBuckets); + + for (CuckooFilter filter : filters) { + if (filter.mightContainFingerprint(bucket, fingerprint, alternateIndex)) { + return true; + } + } + return false; + } + + /** + * Returns true if any of the filters contain this fingerprint at the specified bucket. + * This is an expert-level API since it is dealing with buckets and fingerprints, not raw values + * being hashed. + */ + private boolean mightContainFingerprint(int bucket, int fingerprint) { + int alternateIndex = CuckooFilter.alternateIndex(bucket, fingerprint, numBuckets); + for (CuckooFilter filter : filters) { + if (filter.mightContainFingerprint(bucket, fingerprint, alternateIndex)) { + return true; + } + } + return false; + } + + /** + * Add's the provided value to the set for tracking + */ + public void add(BytesRef value) { + MurmurHash3.Hash128 hash = MurmurHash3.hash128(value.bytes, value.offset, value.length, 0, scratchHash); + addHash(hash.h1); + } + + /** + * Add's the provided value to the set for tracking + */ + public void add(long value) { + addHash(MurmurHash3.murmur64(value)); + } + + private void addHash(long hash) { + if (isSetMode) { + hashes.add(hash); + maybeConvert(); + return; + } + + boolean success = filters.get(filters.size() - 1).add(hash); + if (success == false) { + // filter is full, create a new one and insert there + CuckooFilter t = new CuckooFilter(capacity, fpp, rng); + t.add(hash); + filters.add(t); + breaker.accept(t.getSizeInBytes()); // make sure we account for the new filter + } + } + + private void maybeConvert() { + if (isSetMode && hashes.size() > threshold) { + convert(); + } + } + + /** + * If we still holding values in a set, convert this filter into an approximate, cuckoo-backed filter. + * This will create a list of CuckooFilters, and null out the set of hashes + */ + void convert() { + if (isSetMode == false) { + throw new IllegalStateException("Cannot convert SetBackedScalingCuckooFilter to approximate " + + "when it has already been converted."); + } + long oldSize = getSizeInBytes(); + + filters = new ArrayList<>(); + CuckooFilter t = new CuckooFilter(capacity, fpp, rng); + // Cache the chosen numBuckets for later use + numBuckets = t.getNumBuckets(); + fingerprintMask = t.getFingerprintMask(); + bitsPerEntry = t.getBitsPerEntry(); + + hashes.forEach(t::add); + filters.add(t); + + hashes = null; + isSetMode = false; + + breaker.accept(-oldSize); // this zeros out the overhead of the set + breaker.accept(getSizeInBytes()); // this adds back in the new overhead of the cuckoo filters + + } + + /** + * Get the approximate size of this datastructure. Approximate because only the Set occupants + * are tracked, not the overhead of the Set itself. + */ + public long getSizeInBytes() { + long bytes = 13; // fpp (double), threshold (int), isSetMode (boolean) + if (hashes != null) { + bytes = (hashes.size() * 16); + } + if (filters != null) { + bytes += filters.stream().mapToLong(CuckooFilter::getSizeInBytes).sum(); + } + return bytes; + } + + + /** + * Merge `other` cuckoo filter into this cuckoo. After merging, this filter's state will + * be the union of the two. During the merging process, the internal Set may be upgraded + * to a cuckoo if it goes over threshold + */ + public void merge(SetBackedScalingCuckooFilter other) { + // Some basic sanity checks to make sure we can merge + if (this.threshold != other.threshold) { + throw new IllegalStateException("Cannot merge other CuckooFilter because thresholds do not match: [" + + this.threshold + "] vs [" + other.threshold + "]"); + } + if (this.capacity != other.capacity) { + throw new IllegalStateException("Cannot merge other CuckooFilter because capacities do not match: [" + + this.capacity + "] vs [" + other.capacity + "]"); + } + if (this.fpp != other.fpp) { + throw new IllegalStateException("Cannot merge other CuckooFilter because precisions do not match: [" + + this.fpp + "] vs [" + other.fpp + "]"); + } + + if (isSetMode && other.isSetMode) { + // Both in sets, merge collections then see if we need to convert to cuckoo + hashes.addAll(other.hashes); + maybeConvert(); + } else if (isSetMode && other.isSetMode == false) { + // Other is in cuckoo mode, so we convert our set to a cuckoo, then + // call the merge function again. Since both are now in set-mode + // this will fall through to the last conditional and do a cuckoo-cuckoo merge + convert(); + merge(other); + } else if (isSetMode == false && other.isSetMode) { + // Rather than converting the other to a cuckoo first, we can just + // replay the values directly into our filter. + other.hashes.forEach(this::add); + } else { + // Both are in cuckoo mode, merge raw fingerprints + + CuckooFilter currentFilter = filters.get(filters.size() - 1); + + for (CuckooFilter otherFilter : other.filters) { + + // The iterator returns an array of longs corresponding to the + // fingerprints for buckets at the current position + Iterator iter = otherFilter.getBuckets(); + int bucket = 0; + while (iter.hasNext()) { + long[] fingerprints = iter.next(); + + // We check to see if the fingerprint is present in any of the existing filters + // (in the same bucket/alternate bucket), or if the fingerprint is empty. In these cases + // we can skip the fingerprint + for (long fingerprint : fingerprints) { + if (fingerprint == CuckooFilter.EMPTY || mightContainFingerprint(bucket, (int) fingerprint)) { + continue; + } + // Try to insert into the last filter in our list + if (currentFilter.mergeFingerprint(bucket, (int) fingerprint) == false) { + // if we failed, the filter is now saturated and we need to create a new one + CuckooFilter t = new CuckooFilter(capacity, fpp, rng); + filters.add(t); + breaker.accept(t.getSizeInBytes()); // make sure we account for the new filter + + currentFilter = filters.get(filters.size() - 1); + } + } + bucket += 1; + } + } + } + } + + + @Override + public int hashCode() { + return Objects.hash(hashes, filters, threshold, isSetMode, capacity, fpp); + } + + @Override + public boolean equals(Object other) { + if (this == other) { + return true; + } + if (other == null || getClass() != other.getClass()) { + return false; + } + + final SetBackedScalingCuckooFilter that = (SetBackedScalingCuckooFilter) other; + return Objects.equals(this.hashes, that.hashes) + && Objects.equals(this.filters, that.filters) + && Objects.equals(this.threshold, that.threshold) + && Objects.equals(this.isSetMode, that.isSetMode) + && Objects.equals(this.capacity, that.capacity) + && Objects.equals(this.fpp, that.fpp); + } +} diff --git a/server/src/main/java/org/elasticsearch/common/util/concurrent/AsyncIOProcessor.java b/server/src/main/java/org/elasticsearch/common/util/concurrent/AsyncIOProcessor.java index ad68471041b8d..ad5fd09edc0f5 100644 --- a/server/src/main/java/org/elasticsearch/common/util/concurrent/AsyncIOProcessor.java +++ b/server/src/main/java/org/elasticsearch/common/util/concurrent/AsyncIOProcessor.java @@ -28,6 +28,7 @@ import java.util.concurrent.ArrayBlockingQueue; import java.util.concurrent.Semaphore; import java.util.function.Consumer; +import java.util.function.Supplier; /** * This async IO processor allows to batch IO operations and have a single writer processing the write operations. @@ -39,11 +40,13 @@ public abstract class AsyncIOProcessor { private final Logger logger; private final ArrayBlockingQueue>> queue; + private final ThreadContext threadContext; private final Semaphore promiseSemaphore = new Semaphore(1); - protected AsyncIOProcessor(Logger logger, int queueSize) { + protected AsyncIOProcessor(Logger logger, int queueSize, ThreadContext threadContext) { this.logger = logger; this.queue = new ArrayBlockingQueue<>(queueSize); + this.threadContext = threadContext; } /** @@ -58,11 +61,10 @@ public final void put(Item item, Consumer listener) { // we first try make a promise that we are responsible for the processing final boolean promised = promiseSemaphore.tryAcquire(); - final Tuple> itemTuple = new Tuple<>(item, listener); if (promised == false) { // in this case we are not responsible and can just block until there is space try { - queue.put(new Tuple<>(item, listener)); + queue.put(new Tuple<>(item, preserveContext(listener))); } catch (InterruptedException e) { Thread.currentThread().interrupt(); listener.accept(e); @@ -73,34 +75,33 @@ public final void put(Item item, Consumer listener) { // while we are draining that mean we might exit below too early in the while loop if the drainAndSync call is fast. if (promised || promiseSemaphore.tryAcquire()) { final List>> candidates = new ArrayList<>(); - try { - if (promised) { - // we are responsible for processing we don't need to add the tuple to the queue we can just add it to the candidates - candidates.add(itemTuple); - } - // since we made the promise to process we gotta do it here at least once - drainAndProcess(candidates); - } finally { - promiseSemaphore.release(); // now to ensure we are passing it on we release the promise so another thread can take over + if (promised) { + // we are responsible for processing we don't need to add the tuple to the queue we can just add it to the candidates + // no need to preserve context for listener since it runs in current thread. + candidates.add(new Tuple<>(item, listener)); } + // since we made the promise to process we gotta do it here at least once + drainAndProcessAndRelease(candidates); while (queue.isEmpty() == false && promiseSemaphore.tryAcquire()) { // yet if the queue is not empty AND nobody else has yet made the promise to take over we continue processing - try { - drainAndProcess(candidates); - } finally { - promiseSemaphore.release(); - } + drainAndProcessAndRelease(candidates); } } } - private void drainAndProcess(List>> candidates) { - queue.drainTo(candidates); - processList(candidates); + private void drainAndProcessAndRelease(List>> candidates) { + Exception exception; + try { + queue.drainTo(candidates); + exception = processList(candidates); + } finally { + promiseSemaphore.release(); + } + notifyList(candidates, exception); candidates.clear(); } - private void processList(List>> candidates) { + private Exception processList(List>> candidates) { Exception exception = null; if (candidates.isEmpty() == false) { try { @@ -111,6 +112,10 @@ private void processList(List>> candidates) { exception = ex; } } + return exception; + } + + private void notifyList(List>> candidates, Exception exception) { for (Tuple> tuple : candidates) { Consumer consumer = tuple.v2(); try { @@ -121,6 +126,15 @@ private void processList(List>> candidates) { } } + private Consumer preserveContext(Consumer consumer) { + Supplier restorableContext = threadContext.newRestorableContext(false); + return e -> { + try (ThreadContext.StoredContext ignore = restorableContext.get()) { + consumer.accept(e); + } + }; + } + /** * Writes or processes the items out or to disk. */ diff --git a/server/src/main/java/org/elasticsearch/common/util/concurrent/EsExecutors.java b/server/src/main/java/org/elasticsearch/common/util/concurrent/EsExecutors.java index 29cd7f6682a64..7b304cd092a26 100644 --- a/server/src/main/java/org/elasticsearch/common/util/concurrent/EsExecutors.java +++ b/server/src/main/java/org/elasticsearch/common/util/concurrent/EsExecutors.java @@ -124,6 +124,7 @@ public static EsThreadPoolExecutor newAutoQueueFixed(String name, int size, int */ public static Throwable rethrowErrors(Runnable runnable) { if (runnable instanceof RunnableFuture) { + assert ((RunnableFuture) runnable).isDone(); try { ((RunnableFuture) runnable).get(); } catch (final Exception e) { diff --git a/server/src/main/java/org/elasticsearch/discovery/DiscoveryModule.java b/server/src/main/java/org/elasticsearch/discovery/DiscoveryModule.java index a14def8fa86b5..892dfe9eaa16e 100644 --- a/server/src/main/java/org/elasticsearch/discovery/DiscoveryModule.java +++ b/server/src/main/java/org/elasticsearch/discovery/DiscoveryModule.java @@ -23,7 +23,9 @@ import org.apache.logging.log4j.Logger; import org.elasticsearch.cluster.ClusterState; import org.elasticsearch.cluster.coordination.Coordinator; +import org.elasticsearch.cluster.coordination.ElectionStrategy; import org.elasticsearch.cluster.node.DiscoveryNode; +import org.elasticsearch.cluster.routing.RerouteService; import org.elasticsearch.cluster.routing.allocation.AllocationService; import org.elasticsearch.cluster.service.ClusterApplier; import org.elasticsearch.cluster.service.ClusterApplierService; @@ -74,16 +76,24 @@ public class DiscoveryModule { Setting.listSetting("discovery.seed_providers", Collections.emptyList(), Function.identity(), Property.NodeScope); + public static final String DEFAULT_ELECTION_STRATEGY = "default"; + + public static final Setting ELECTION_STRATEGY_SETTING = + new Setting<>("cluster.election.strategy", DEFAULT_ELECTION_STRATEGY, Function.identity(), Property.NodeScope); + private final Discovery discovery; public DiscoveryModule(Settings settings, ThreadPool threadPool, TransportService transportService, NamedWriteableRegistry namedWriteableRegistry, NetworkService networkService, MasterService masterService, ClusterApplier clusterApplier, ClusterSettings clusterSettings, List plugins, - AllocationService allocationService, Path configFile, GatewayMetaState gatewayMetaState) { + AllocationService allocationService, Path configFile, GatewayMetaState gatewayMetaState, + RerouteService rerouteService) { final Collection> joinValidators = new ArrayList<>(); final Map> hostProviders = new HashMap<>(); hostProviders.put("settings", () -> new SettingsBasedSeedHostsProvider(settings, transportService)); hostProviders.put("file", () -> new FileBasedSeedHostsProvider(configFile)); + final Map electionStrategies = new HashMap<>(); + electionStrategies.put(DEFAULT_ELECTION_STRATEGY, ElectionStrategy.DEFAULT_INSTANCE); for (DiscoveryPlugin plugin : plugins) { plugin.getSeedHostProviders(transportService, networkService).forEach((key, value) -> { if (hostProviders.put(key, value) != null) { @@ -94,6 +104,11 @@ public DiscoveryModule(Settings settings, ThreadPool threadPool, TransportServic if (joinValidator != null) { joinValidators.add(joinValidator); } + plugin.getElectionStrategies().forEach((key, value) -> { + if (electionStrategies.put(key, value) != null) { + throw new IllegalArgumentException("Cannot register election strategy [" + key + "] twice"); + } + }); } List seedProviderNames = DISCOVERY_SEED_PROVIDERS_SETTING.get(settings); @@ -124,12 +139,17 @@ public DiscoveryModule(Settings settings, ThreadPool threadPool, TransportServic return Collections.unmodifiableList(addresses); }; + final ElectionStrategy electionStrategy = electionStrategies.get(ELECTION_STRATEGY_SETTING.get(settings)); + if (electionStrategy == null) { + throw new IllegalArgumentException("Unknown election strategy " + ELECTION_STRATEGY_SETTING.get(settings)); + } + if (ZEN2_DISCOVERY_TYPE.equals(discoveryType) || SINGLE_NODE_DISCOVERY_TYPE.equals(discoveryType)) { discovery = new Coordinator(NODE_NAME_SETTING.get(settings), settings, clusterSettings, transportService, namedWriteableRegistry, allocationService, masterService, () -> gatewayMetaState.getPersistedState(settings, (ClusterApplierService) clusterApplier), seedHostsProvider, - clusterApplier, joinValidators, new Random(Randomness.get().nextLong())); + clusterApplier, joinValidators, new Random(Randomness.get().nextLong()), rerouteService, electionStrategy); } else { throw new IllegalArgumentException("Unknown discovery type [" + discoveryType + "]"); } diff --git a/server/src/main/java/org/elasticsearch/discovery/HandshakingTransportAddressConnector.java b/server/src/main/java/org/elasticsearch/discovery/HandshakingTransportAddressConnector.java index 7f2512f97f87b..4e90ae02e12ac 100644 --- a/server/src/main/java/org/elasticsearch/discovery/HandshakingTransportAddressConnector.java +++ b/server/src/main/java/org/elasticsearch/discovery/HandshakingTransportAddressConnector.java @@ -21,6 +21,7 @@ import org.apache.logging.log4j.LogManager; import org.apache.logging.log4j.Logger; +import org.apache.logging.log4j.message.ParameterizedMessage; import org.elasticsearch.Version; import org.elasticsearch.action.ActionListener; import org.elasticsearch.cluster.node.DiscoveryNode; @@ -89,6 +90,13 @@ protected void doRun() throws Exception { remoteNode = transportService.handshake(connection, probeHandshakeTimeout.millis()); // success means (amongst other things) that the cluster names match logger.trace("[{}] handshake successful: {}", this, remoteNode); + } catch (Exception e) { + // we opened a connection and successfully performed a low-level handshake, so we were definitely talking to an + // Elasticsearch node, but the high-level handshake failed indicating some kind of mismatched configurations + // (e.g. cluster name) that the user should address + logger.warn(new ParameterizedMessage("handshake failed for [{}]", this), e); + listener.onFailure(e); + return; } finally { IOUtils.closeWhileHandlingException(connection); } diff --git a/server/src/main/java/org/elasticsearch/discovery/SeedHostsResolver.java b/server/src/main/java/org/elasticsearch/discovery/SeedHostsResolver.java index aa4c39ad8276d..3efbff52d822c 100644 --- a/server/src/main/java/org/elasticsearch/discovery/SeedHostsResolver.java +++ b/server/src/main/java/org/elasticsearch/discovery/SeedHostsResolver.java @@ -27,6 +27,7 @@ import org.elasticsearch.common.settings.Settings; import org.elasticsearch.common.transport.TransportAddress; import org.elasticsearch.common.unit.TimeValue; +import org.elasticsearch.common.util.CancellableThreads; import org.elasticsearch.common.util.concurrent.AbstractRunnable; import org.elasticsearch.common.util.concurrent.EsExecutors; import org.elasticsearch.discovery.PeerFinder.ConfiguredHostsResolver; @@ -67,6 +68,7 @@ public class SeedHostsResolver extends AbstractLifecycleComponent implements Con private final TimeValue resolveTimeout; private final String nodeName; private final int concurrentConnects; + private final CancellableThreads cancellableThreads = new CancellableThreads(); public SeedHostsResolver(String nodeName, Settings settings, TransportService transportService, SeedHostsProvider seedProvider) { @@ -98,11 +100,11 @@ public List resolveHosts(final List hosts) { .stream() .map(hn -> (Callable) () -> transportService.addressesFromString(hn)) .collect(Collectors.toList()); - final List> futures; + final SetOnce>> futures = new SetOnce<>(); try { - futures = executorService.get().invokeAll(callables, resolveTimeout.nanos(), TimeUnit.NANOSECONDS); - } catch (InterruptedException e) { - Thread.currentThread().interrupt(); + cancellableThreads.execute(() -> + futures.set(executorService.get().invokeAll(callables, resolveTimeout.nanos(), TimeUnit.NANOSECONDS))); + } catch (CancellableThreads.ExecutionCancelledException e) { return Collections.emptyList(); } final List transportAddresses = new ArrayList<>(); @@ -112,10 +114,10 @@ public List resolveHosts(final List hosts) { // ExecutorService#invokeAll guarantees that the futures are returned in the iteration order of the tasks so we can associate the // hostname with the corresponding task by iterating together final Iterator it = hosts.iterator(); - for (final Future future : futures) { + for (final Future future : futures.get()) { + assert future.isDone(); final String hostname = it.next(); if (!future.isCancelled()) { - assert future.isDone(); try { final TransportAddress[] addresses = future.get(); logger.trace("resolved host [{}] to {}", hostname, addresses); @@ -151,6 +153,7 @@ protected void doStart() { @Override protected void doStop() { + cancellableThreads.cancel("stopping SeedHostsResolver"); ThreadPool.terminate(executorService.get(), 10, TimeUnit.SECONDS); } diff --git a/server/src/main/java/org/elasticsearch/env/NodeEnvironment.java b/server/src/main/java/org/elasticsearch/env/NodeEnvironment.java index 4d19dd66732fc..160662a63e5b3 100644 --- a/server/src/main/java/org/elasticsearch/env/NodeEnvironment.java +++ b/server/src/main/java/org/elasticsearch/env/NodeEnvironment.java @@ -35,6 +35,7 @@ import org.elasticsearch.cluster.metadata.IndexMetaData; import org.elasticsearch.cluster.node.DiscoveryNode; import org.elasticsearch.common.CheckedFunction; +import org.elasticsearch.common.CheckedRunnable; import org.elasticsearch.common.Randomness; import org.elasticsearch.common.SuppressForbidden; import org.elasticsearch.common.UUIDs; @@ -45,6 +46,7 @@ import org.elasticsearch.common.settings.Settings; import org.elasticsearch.common.unit.ByteSizeValue; import org.elasticsearch.common.unit.TimeValue; +import org.elasticsearch.common.util.set.Sets; import org.elasticsearch.common.xcontent.NamedXContentRegistry; import org.elasticsearch.core.internal.io.IOUtils; import org.elasticsearch.gateway.MetaDataStateFormat; @@ -70,6 +72,7 @@ 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.Iterator; @@ -81,6 +84,7 @@ import java.util.concurrent.Semaphore; import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicBoolean; +import java.util.function.Function; import java.util.function.Predicate; import java.util.stream.Collectors; import java.util.stream.Stream; @@ -90,9 +94,9 @@ */ public final class NodeEnvironment implements Closeable { public static class NodePath { - /* ${data.paths}/nodes/0 */ + /* ${data.paths} */ public final Path path; - /* ${data.paths}/nodes/0/indices */ + /* ${data.paths}/indices */ public final Path indicesPath; /** Cached FileStore from path */ public final FileStore fileStore; @@ -115,7 +119,7 @@ public NodePath(Path path) throws IOException { /** * Resolves the given shards directory against this NodePath - * ${data.paths}/nodes/{node.id}/indices/{index.uuid}/{shard.id} + * ${data.paths}/indices/{index.uuid}/{shard.id} */ public Path resolve(ShardId shardId) { return resolve(shardId.getIndex()).resolve(Integer.toString(shardId.id())); @@ -123,7 +127,7 @@ public Path resolve(ShardId shardId) { /** * Resolves index directory against this NodePath - * ${data.paths}/nodes/{node.id}/indices/{index.uuid} + * ${data.paths}/indices/{index.uuid} */ public Path resolve(Index index) { return resolve(index.getUUID()); @@ -170,7 +174,6 @@ public String toString() { public static final Setting ENABLE_LUCENE_SEGMENT_INFOS_TRACE_SETTING = Setting.boolSetting("node.enable_lucene_segment_infos_trace", false, Property.NodeScope); - public static final String NODES_FOLDER = "nodes"; public static final String INDICES_FOLDER = "indices"; public static final String NODE_LOCK_FILENAME = "node.lock"; @@ -179,20 +182,28 @@ public static class NodeLock implements Releasable { private final Lock[] locks; private final NodePath[] nodePaths; + + public NodeLock(final Logger logger, + final Environment environment, + final CheckedFunction pathFunction) throws IOException { + this(logger, environment, pathFunction, Function.identity()); + } + /** * Tries to acquire a node lock for a node id, throws {@code IOException} if it is unable to acquire it * @param pathFunction function to check node path before attempt of acquiring a node lock */ public NodeLock(final Logger logger, final Environment environment, - final CheckedFunction pathFunction) throws IOException { + final CheckedFunction pathFunction, + final Function subPathMapping) throws IOException { nodePaths = new NodePath[environment.dataFiles().length]; locks = new Lock[nodePaths.length]; try { final Path[] dataPaths = environment.dataFiles(); for (int dirIndex = 0; dirIndex < dataPaths.length; dirIndex++) { Path dataDir = dataPaths[dirIndex]; - Path dir = resolveNodePath(dataDir); + Path dir = subPathMapping.apply(dataDir); if (pathFunction.apply(dir) == false) { continue; } @@ -247,7 +258,7 @@ public NodeEnvironment(Settings settings, Environment environment) throws IOExce sharedDataPath = environment.sharedDataFile(); for (Path path : environment.dataFiles()) { - Files.createDirectories(resolveNodePath(path)); + Files.createDirectories(path); } final NodeLock nodeLock; @@ -264,7 +275,6 @@ public NodeEnvironment(Settings settings, Environment environment) throws IOExce this.locks = nodeLock.locks; this.nodePaths = nodeLock.nodePaths; - this.nodeMetaData = loadOrCreateNodeMetaData(settings, logger, nodePaths); logger.debug("using node location {}", Arrays.toString(nodePaths)); @@ -278,6 +288,10 @@ public NodeEnvironment(Settings settings, Environment environment) throws IOExce ensureAtomicMoveSupported(nodePaths); } + if (upgradeLegacyNodeFolders(logger, settings, environment, nodeLock)) { + assertCanWrite(); + } + if (DiscoveryNode.isDataNode(settings) == false) { if (DiscoveryNode.isMasterNode(settings) == false) { ensureNoIndexMetaData(nodePaths); @@ -286,6 +300,8 @@ public NodeEnvironment(Settings settings, Environment environment) throws IOExce ensureNoShardData(nodePaths); } + this.nodeMetaData = loadOrCreateNodeMetaData(settings, logger, nodePaths); + success = true; } finally { if (success == false) { @@ -295,13 +311,128 @@ public NodeEnvironment(Settings settings, Environment environment) throws IOExce } /** - * Resolve a specific nodes/{node.id} path for the specified path and node lock id. - * - * @param path the path - * @return the resolved path + * Upgrades all data paths that have been written to by an older ES version to the 8.0+ compatible folder layout, + * removing the "nodes/${lockId}" folder prefix */ - public static Path resolveNodePath(final Path path) { - return path.resolve(NODES_FOLDER).resolve("0"); + private static boolean upgradeLegacyNodeFolders(Logger logger, Settings settings, Environment environment, + NodeLock nodeLock) throws IOException { + boolean upgradeNeeded = false; + + // check if we can do an auto-upgrade + for (Path path : environment.dataFiles()) { + final Path nodesFolderPath = path.resolve("nodes"); + if (Files.isDirectory(nodesFolderPath)) { + final List nodeLockIds = new ArrayList<>(); + + try (DirectoryStream stream = Files.newDirectoryStream(nodesFolderPath)) { + for (Path nodeLockIdPath : stream) { + String fileName = nodeLockIdPath.getFileName().toString(); + if (Files.isDirectory(nodeLockIdPath) && fileName.chars().allMatch(Character::isDigit)) { + int nodeLockId = Integer.parseInt(fileName); + nodeLockIds.add(nodeLockId); + } else if (FileSystemUtils.isDesktopServicesStore(nodeLockIdPath) == false) { + throw new IllegalStateException("unexpected file/folder encountered during data folder upgrade: " + + nodeLockIdPath); + } + } + } + + if (nodeLockIds.isEmpty() == false) { + upgradeNeeded = true; + + if (nodeLockIds.equals(Arrays.asList(0)) == false) { + throw new IllegalStateException("data path " + nodesFolderPath + " cannot be upgraded automatically because it " + + "contains data from nodes with ordinals " + nodeLockIds + ", due to previous use of the now obsolete " + + "[node.max_local_storage_nodes] setting. Please check the breaking changes docs for the current version of " + + "Elasticsearch to find an upgrade path"); + } + } + } + } + + if (upgradeNeeded == false) { + logger.trace("data folder upgrade not required"); + return false; + } + + logger.info("upgrading legacy data folders: {}", Arrays.toString(environment.dataFiles())); + + // acquire locks on legacy path for duration of upgrade (to ensure there is no older ES version running on this path) + final NodeLock legacyNodeLock; + try { + legacyNodeLock = new NodeLock(logger, environment, dir -> true, path -> path.resolve("nodes").resolve("0")); + } catch (IOException e) { + final String message = String.format( + Locale.ROOT, + "failed to obtain legacy node locks, tried %s;" + + " maybe these locations are not writable or multiple nodes were started on the same data path?", + Arrays.toString(environment.dataFiles())); + throw new IllegalStateException(message, e); + } + + // move contents from legacy path to new path + assert nodeLock.getNodePaths().length == legacyNodeLock.getNodePaths().length; + try { + final List> upgradeActions = new ArrayList<>(); + for (int i = 0; i < legacyNodeLock.getNodePaths().length; i++) { + final NodePath legacyNodePath = legacyNodeLock.getNodePaths()[i]; + final NodePath nodePath = nodeLock.getNodePaths()[i]; + + // determine folders to move and check that there are no extra files/folders + final Set folderNames = new HashSet<>(); + + try (DirectoryStream stream = Files.newDirectoryStream(legacyNodePath.path)) { + for (Path subFolderPath : stream) { + final String fileName = subFolderPath.getFileName().toString(); + if (FileSystemUtils.isDesktopServicesStore(subFolderPath)) { + // ignore + } else if (FileSystemUtils.isAccessibleDirectory(subFolderPath, logger)) { + if (fileName.equals(INDICES_FOLDER) == false && // indices folder + fileName.equals(MetaDataStateFormat.STATE_DIR_NAME) == false) { // global metadata & node state folder + throw new IllegalStateException("unexpected folder encountered during data folder upgrade: " + + subFolderPath); + } + final Path targetSubFolderPath = nodePath.path.resolve(fileName); + if (Files.exists(targetSubFolderPath)) { + throw new IllegalStateException("target folder already exists during data folder upgrade: " + + targetSubFolderPath); + } + folderNames.add(fileName); + } else if (fileName.equals(NODE_LOCK_FILENAME) == false && + fileName.equals(TEMP_FILE_NAME) == false) { + throw new IllegalStateException("unexpected file/folder encountered during data folder upgrade: " + + subFolderPath); + } + } + } + + assert Sets.difference(folderNames, Sets.newHashSet(INDICES_FOLDER, MetaDataStateFormat.STATE_DIR_NAME)).isEmpty() : + "expected indices and/or state dir folder but was " + folderNames; + + upgradeActions.add(() -> { + for (String folderName : folderNames) { + final Path sourceSubFolderPath = legacyNodePath.path.resolve(folderName); + final Path targetSubFolderPath = nodePath.path.resolve(folderName); + Files.move(sourceSubFolderPath, targetSubFolderPath, StandardCopyOption.ATOMIC_MOVE); + logger.info("data folder upgrade: moved from [{}] to [{}]", sourceSubFolderPath, targetSubFolderPath); + } + IOUtils.fsync(nodePath.path, true); + }); + } + // now do the actual upgrade. start by upgrading the node metadata file before moving anything, since a downgrade in an + // intermediate state would be pretty disastrous + loadOrCreateNodeMetaData(settings, logger, legacyNodeLock.getNodePaths()); + for (CheckedRunnable upgradeAction : upgradeActions) { + upgradeAction.run(); + } + } finally { + legacyNodeLock.close(); + } + + // upgrade successfully completed, remove legacy nodes folders + IOUtils.rm(Stream.of(environment.dataFiles()).map(path -> path.resolve("nodes")).toArray(Path[]::new)); + + return true; } private void maybeLogPathDetails() throws IOException { @@ -364,10 +495,25 @@ private void maybeLogHeapDetails() { private static NodeMetaData loadOrCreateNodeMetaData(Settings settings, Logger logger, NodePath... nodePaths) throws IOException { final Path[] paths = Arrays.stream(nodePaths).map(np -> np.path).toArray(Path[]::new); + + final Set nodeIds = new HashSet<>(); + for (final Path path : paths) { + final NodeMetaData metaData = NodeMetaData.FORMAT.loadLatestState(logger, NamedXContentRegistry.EMPTY, path); + if (metaData != null) { + nodeIds.add(metaData.nodeId()); + } + } + if (nodeIds.size() > 1) { + throw new IllegalStateException( + "data paths " + Arrays.toString(paths) + " belong to multiple nodes with IDs " + nodeIds); + } + NodeMetaData metaData = NodeMetaData.FORMAT.loadLatestState(logger, NamedXContentRegistry.EMPTY, paths); if (metaData == null) { + assert nodeIds.isEmpty() : nodeIds; metaData = new NodeMetaData(generateNodeId(settings), Version.CURRENT); } else { + assert nodeIds.equals(Collections.singleton(metaData.nodeId())) : nodeIds + " doesn't match " + metaData; metaData = metaData.upgradeToCurrentVersion(); } @@ -801,14 +947,14 @@ public Path[] availableShardPaths(ShardId shardId) { } /** - * Returns all folder names in ${data.paths}/nodes/{node.id}/indices folder + * Returns all folder names in ${data.paths}/indices folder */ public Set availableIndexFolders() throws IOException { return availableIndexFolders(p -> false); } /** - * Returns folder names in ${data.paths}/nodes/{node.id}/indices folder that don't match the given predicate. + * Returns folder names in ${data.paths}/indices folder that don't match the given predicate. * @param excludeIndexPathIdsPredicate folder names to exclude */ public Set availableIndexFolders(Predicate excludeIndexPathIdsPredicate) throws IOException { @@ -825,7 +971,7 @@ public Set availableIndexFolders(Predicate excludeIndexPathIdsPr } /** - * Return all directory names in the nodes/{node.id}/indices directory for the given node path. + * Return all directory names in the indices directory for the given node path. * * @param nodePath the path * @return all directories that could be indices for the given node path. @@ -836,7 +982,7 @@ public Set availableIndexFoldersForPath(final NodePath nodePath) throws } /** - * Return directory names in the nodes/{node.id}/indices directory for the given node path that don't match the given predicate. + * Return directory names in the indices directory for the given node path that don't match the given predicate. * * @param nodePath the path * @param excludeIndexPathIdsPredicate folder names to exclude @@ -865,7 +1011,7 @@ public Set availableIndexFoldersForPath(final NodePath nodePath, Predica } /** - * Resolves all existing paths to indexFolderName in ${data.paths}/nodes/{node.id}/indices + * Resolves all existing paths to indexFolderName in ${data.paths}/indices */ public Path[] resolveIndexFolder(String indexFolderName) { if (nodePaths == null || locks == null) { diff --git a/server/src/main/java/org/elasticsearch/gateway/GatewayAllocator.java b/server/src/main/java/org/elasticsearch/gateway/GatewayAllocator.java index 82627cfdc0b82..7451f25460e0c 100644 --- a/server/src/main/java/org/elasticsearch/gateway/GatewayAllocator.java +++ b/server/src/main/java/org/elasticsearch/gateway/GatewayAllocator.java @@ -21,10 +21,12 @@ import org.apache.logging.log4j.LogManager; import org.apache.logging.log4j.Logger; +import org.apache.logging.log4j.message.ParameterizedMessage; +import org.elasticsearch.action.ActionListener; import org.elasticsearch.action.support.nodes.BaseNodeResponse; import org.elasticsearch.action.support.nodes.BaseNodesResponse; +import org.elasticsearch.cluster.routing.RerouteService; import org.elasticsearch.cluster.routing.RoutingNodes; -import org.elasticsearch.cluster.routing.RoutingService; import org.elasticsearch.cluster.routing.ShardRouting; import org.elasticsearch.cluster.routing.allocation.AllocateUnassignedDecision; import org.elasticsearch.cluster.routing.allocation.FailedShard; @@ -42,7 +44,7 @@ public class GatewayAllocator { private static final Logger logger = LogManager.getLogger(GatewayAllocator.class); - private final RoutingService routingService; + private final RerouteService rerouteService; private final PrimaryShardAllocator primaryShardAllocator; private final ReplicaShardAllocator replicaShardAllocator; @@ -53,10 +55,10 @@ public class GatewayAllocator { asyncFetchStore = ConcurrentCollections.newConcurrentMap(); @Inject - public GatewayAllocator(RoutingService routingService, + public GatewayAllocator(RerouteService rerouteService, TransportNodesListGatewayStartedShards startedAction, TransportNodesListShardStoreMetaData storeAction) { - this.routingService = routingService; + this.rerouteService = rerouteService; this.primaryShardAllocator = new InternalPrimaryShardAllocator(startedAction); this.replicaShardAllocator = new InternalReplicaShardAllocator(storeAction); } @@ -70,7 +72,7 @@ public void cleanCaches() { // for tests protected GatewayAllocator() { - this.routingService = null; + this.rerouteService = null; this.primaryShardAllocator = null; this.replicaShardAllocator = null; } @@ -137,7 +139,9 @@ class InternalAsyncFetch extends AsyncShardFetch @Override protected void reroute(ShardId shardId, String reason) { logger.trace("{} scheduling reroute for {}", shardId, reason); - routingService.reroute("async_shard_fetch"); + rerouteService.reroute("async_shard_fetch", ActionListener.wrap( + r -> logger.trace("{} scheduled reroute completed for {}", shardId, reason), + e -> logger.debug(new ParameterizedMessage("{} scheduled reroute failed for {}", shardId, reason), e))); } } diff --git a/server/src/main/java/org/elasticsearch/gateway/GatewayService.java b/server/src/main/java/org/elasticsearch/gateway/GatewayService.java index b7b7d0759980e..3e9c25847f6a7 100644 --- a/server/src/main/java/org/elasticsearch/gateway/GatewayService.java +++ b/server/src/main/java/org/elasticsearch/gateway/GatewayService.java @@ -85,7 +85,7 @@ public class GatewayService extends AbstractLifecycleComponent implements Cluste private final Runnable recoveryRunnable; - private final AtomicBoolean recovered = new AtomicBoolean(); + private final AtomicBoolean recoveryInProgress = new AtomicBoolean(); private final AtomicBoolean scheduledRecovery = new AtomicBoolean(); @Inject @@ -211,7 +211,7 @@ public void onFailure(Exception e) { @Override protected void doRun() { - if (recovered.compareAndSet(false, true)) { + if (recoveryInProgress.compareAndSet(false, true)) { logger.info("recover_after_time [{}] elapsed. performing state recovery...", recoverAfterTime); recoveryRunnable.run(); } @@ -219,7 +219,7 @@ protected void doRun() { }, recoverAfterTime, ThreadPool.Names.GENERIC); } } else { - if (recovered.compareAndSet(false, true)) { + if (recoveryInProgress.compareAndSet(false, true)) { threadPool.generic().execute(new AbstractRunnable() { @Override public void onFailure(final Exception e) { @@ -237,7 +237,7 @@ protected void doRun() { } private void resetRecoveredFlags() { - recovered.set(false); + recoveryInProgress.set(false); scheduledRecovery.set(false); } @@ -256,6 +256,9 @@ public ClusterState execute(final ClusterState currentState) { @Override public void clusterStateProcessed(final String source, final ClusterState oldState, final ClusterState newState) { logger.info("recovered [{}] indices into cluster_state", newState.metaData().indices().size()); + // reset flag even though state recovery completed, to ensure that if we subsequently become leader again based on a + // not-recovered state, that we again do another state recovery. + resetRecoveredFlags(); } @Override diff --git a/server/src/main/java/org/elasticsearch/gateway/TransportNodesListGatewayMetaState.java b/server/src/main/java/org/elasticsearch/gateway/TransportNodesListGatewayMetaState.java index ab0fad88ecdfa..d06ad91dbbcf2 100644 --- a/server/src/main/java/org/elasticsearch/gateway/TransportNodesListGatewayMetaState.java +++ b/server/src/main/java/org/elasticsearch/gateway/TransportNodesListGatewayMetaState.java @@ -37,6 +37,7 @@ import org.elasticsearch.common.io.stream.StreamInput; import org.elasticsearch.common.io.stream.StreamOutput; import org.elasticsearch.common.unit.TimeValue; +import org.elasticsearch.tasks.Task; import org.elasticsearch.threadpool.ThreadPool; import org.elasticsearch.transport.TransportService; @@ -67,8 +68,8 @@ public ActionFuture list(String[] nodesIds, @Nullable Tim } @Override - protected NodeRequest newNodeRequest(String nodeId, Request request) { - return new NodeRequest(nodeId); + protected NodeRequest newNodeRequest(Request request) { + return new NodeRequest(); } @Override @@ -82,7 +83,7 @@ protected NodesGatewayMetaState newResponse(Request request, List nodes) } public static class NodeRequest extends BaseNodeRequest { - - public NodeRequest() { - } - - NodeRequest(String nodeId) { - super(nodeId); - } - } public static class NodeGatewayMetaState extends BaseNodeResponse { diff --git a/server/src/main/java/org/elasticsearch/gateway/TransportNodesListGatewayStartedShards.java b/server/src/main/java/org/elasticsearch/gateway/TransportNodesListGatewayStartedShards.java index 1893be3acd518..be33bb031c32c 100644 --- a/server/src/main/java/org/elasticsearch/gateway/TransportNodesListGatewayStartedShards.java +++ b/server/src/main/java/org/elasticsearch/gateway/TransportNodesListGatewayStartedShards.java @@ -45,6 +45,7 @@ import org.elasticsearch.index.shard.ShardStateMetaData; import org.elasticsearch.index.store.Store; import org.elasticsearch.indices.IndicesService; +import org.elasticsearch.tasks.Task; import org.elasticsearch.threadpool.ThreadPool; import org.elasticsearch.transport.TransportService; @@ -92,8 +93,8 @@ public void list(ShardId shardId, DiscoveryNode[] nodes, } @Override - protected NodeRequest newNodeRequest(String nodeId, Request request) { - return new NodeRequest(nodeId, request); + protected NodeRequest newNodeRequest(Request request) { + return new NodeRequest(request); } @Override @@ -108,7 +109,7 @@ protected NodesGatewayStartedShards newResponse(Request request, } @Override - protected NodeGatewayStartedShards nodeOperation(NodeRequest request) { + protected NodeGatewayStartedShards nodeOperation(NodeRequest request, Task task) { try { final ShardId shardId = request.getShardId(); logger.trace("{} loading local shard state info", shardId); @@ -222,8 +223,7 @@ public static class NodeRequest extends BaseNodeRequest { public NodeRequest() { } - public NodeRequest(String nodeId, Request request) { - super(nodeId); + public NodeRequest(Request request) { this.shardId = request.shardId(); } diff --git a/server/src/main/java/org/elasticsearch/http/AbstractHttpServerTransport.java b/server/src/main/java/org/elasticsearch/http/AbstractHttpServerTransport.java index ab6168cad522e..6b901958218a6 100644 --- a/server/src/main/java/org/elasticsearch/http/AbstractHttpServerTransport.java +++ b/server/src/main/java/org/elasticsearch/http/AbstractHttpServerTransport.java @@ -73,6 +73,7 @@ public abstract class AbstractHttpServerTransport extends AbstractLifecycleCompo protected final BigArrays bigArrays; protected final ThreadPool threadPool; protected final Dispatcher dispatcher; + protected final CorsHandler.Config corsConfig; private final NamedXContentRegistry xContentRegistry; protected final PortsRange port; @@ -94,6 +95,7 @@ protected AbstractHttpServerTransport(Settings settings, NetworkService networkS this.xContentRegistry = xContentRegistry; this.dispatcher = dispatcher; this.handlingSettings = HttpHandlingSettings.fromSettings(settings); + this.corsConfig = CorsHandler.fromSettings(settings); // we can't make the network.bind_host a fallback since we already fall back to http.host hence the extra conditional here List httpBindHost = SETTING_HTTP_BIND_HOST.get(settings); diff --git a/server/src/main/java/org/elasticsearch/http/CorsHandler.java b/server/src/main/java/org/elasticsearch/http/CorsHandler.java new file mode 100644 index 0000000000000..6bd0b09265dfc --- /dev/null +++ b/server/src/main/java/org/elasticsearch/http/CorsHandler.java @@ -0,0 +1,258 @@ +/* + * 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. + */ +/* + * Copyright 2013 The Netty Project + * + * The Netty Project 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.http; + +import org.elasticsearch.common.Strings; +import org.elasticsearch.common.settings.Settings; +import org.elasticsearch.common.settings.SettingsException; +import org.elasticsearch.rest.RestRequest; +import org.elasticsearch.rest.RestUtils; + +import java.util.Arrays; +import java.util.Collections; +import java.util.HashSet; +import java.util.LinkedHashSet; +import java.util.Locale; +import java.util.Optional; +import java.util.Set; +import java.util.regex.Pattern; +import java.util.regex.PatternSyntaxException; + +import static org.elasticsearch.http.HttpTransportSettings.SETTING_CORS_ALLOW_CREDENTIALS; +import static org.elasticsearch.http.HttpTransportSettings.SETTING_CORS_ALLOW_HEADERS; +import static org.elasticsearch.http.HttpTransportSettings.SETTING_CORS_ALLOW_METHODS; +import static org.elasticsearch.http.HttpTransportSettings.SETTING_CORS_ALLOW_ORIGIN; +import static org.elasticsearch.http.HttpTransportSettings.SETTING_CORS_ENABLED; +import static org.elasticsearch.http.HttpTransportSettings.SETTING_CORS_MAX_AGE; + +/** + * This file is forked from the https://netty.io project. In particular it combines the following three + * files: io.netty.handler.codec.http.cors.CorsHandler, io.netty.handler.codec.http.cors.CorsConfig, and + * io.netty.handler.codec.http.cors.CorsConfigBuilder. + * + * It modifies the original netty code to operation on Elasticsearch http request/response abstractions. + * Additionally, it removes CORS features that are not used by Elasticsearch. + */ +public class CorsHandler { + + public static final String ANY_ORIGIN = "*"; + + private CorsHandler() { + } + + public static class Config { + + private final boolean enabled; + private final Optional> origins; + private final Optional pattern; + private final boolean anyOrigin; + private final boolean credentialsAllowed; + private final Set allowedRequestMethods; + private final Set allowedRequestHeaders; + private final long maxAge; + + public Config(Builder builder) { + this.enabled = builder.enabled; + origins = builder.origins.map(HashSet::new); + pattern = builder.pattern; + anyOrigin = builder.anyOrigin; + this.credentialsAllowed = builder.allowCredentials; + this.allowedRequestMethods = Collections.unmodifiableSet(builder.requestMethods); + this.allowedRequestHeaders = Collections.unmodifiableSet(builder.requestHeaders); + this.maxAge = builder.maxAge; + } + + public boolean isCorsSupportEnabled() { + return enabled; + } + + public boolean isAnyOriginSupported() { + return anyOrigin; + } + + public boolean isOriginAllowed(String origin) { + if (origins.isPresent()) { + return origins.get().contains(origin); + } else if (pattern.isPresent()) { + return pattern.get().matcher(origin).matches(); + } + return false; + } + + public boolean isCredentialsAllowed() { + return credentialsAllowed; + } + + public Set allowedRequestMethods() { + return allowedRequestMethods; + } + + public Set allowedRequestHeaders() { + return allowedRequestHeaders; + } + + public long maxAge() { + return maxAge; + } + + public Optional> origins() { + return origins; + } + + @Override + public String toString() { + return "Config{" + + "enabled=" + enabled + + ", origins=" + origins + + ", pattern=" + pattern + + ", anyOrigin=" + anyOrigin + + ", credentialsAllowed=" + credentialsAllowed + + ", allowedRequestMethods=" + allowedRequestMethods + + ", allowedRequestHeaders=" + allowedRequestHeaders + + ", maxAge=" + maxAge + + '}'; + } + + private static class Builder { + + private boolean enabled = true; + private Optional> origins; + private Optional pattern; + private final boolean anyOrigin; + private boolean allowCredentials = false; + long maxAge; + private final Set requestMethods = new HashSet<>(); + private final Set requestHeaders = new HashSet<>(); + + private Builder() { + anyOrigin = true; + origins = Optional.empty(); + pattern = Optional.empty(); + } + + private Builder(final String... origins) { + this.origins = Optional.of(new LinkedHashSet<>(Arrays.asList(origins))); + pattern = Optional.empty(); + anyOrigin = false; + } + + private Builder(final Pattern pattern) { + this.pattern = Optional.of(pattern); + origins = Optional.empty(); + anyOrigin = false; + } + + static Builder forOrigins(final String... origins) { + return new Builder(origins); + } + + static Builder forAnyOrigin() { + return new Builder(); + } + + static Builder forPattern(Pattern pattern) { + return new Builder(pattern); + } + + Builder allowCredentials() { + this.allowCredentials = true; + return this; + } + + + public Builder allowedRequestMethods(RestRequest.Method[] methods) { + requestMethods.addAll(Arrays.asList(methods)); + return this; + } + + public Builder maxAge(int maxAge) { + this.maxAge = maxAge; + return this; + } + + public Builder allowedRequestHeaders(String[] headers) { + requestHeaders.addAll(Arrays.asList(headers)); + return this; + } + + public Config build() { + return new Config(this); + } + } + } + + public static Config disabled() { + Config.Builder builder = new Config.Builder(); + builder.enabled = false; + return new Config(builder); + } + + public static Config fromSettings(Settings settings) { + if (SETTING_CORS_ENABLED.get(settings) == false) { + return disabled(); + } + String origin = SETTING_CORS_ALLOW_ORIGIN.get(settings); + final CorsHandler.Config.Builder builder; + if (Strings.isNullOrEmpty(origin)) { + builder = CorsHandler.Config.Builder.forOrigins(); + } else if (origin.equals(CorsHandler.ANY_ORIGIN)) { + builder = CorsHandler.Config.Builder.forAnyOrigin(); + } else { + try { + Pattern p = RestUtils.checkCorsSettingForRegex(origin); + if (p == null) { + builder = CorsHandler.Config.Builder.forOrigins(RestUtils.corsSettingAsArray(origin)); + } else { + builder = CorsHandler.Config.Builder.forPattern(p); + } + } catch (PatternSyntaxException e) { + throw new SettingsException("Bad regex in [" + SETTING_CORS_ALLOW_ORIGIN.getKey() + "]: [" + origin + "]", e); + } + } + if (SETTING_CORS_ALLOW_CREDENTIALS.get(settings)) { + builder.allowCredentials(); + } + String[] strMethods = Strings.tokenizeToStringArray(SETTING_CORS_ALLOW_METHODS.get(settings), ","); + RestRequest.Method[] methods = Arrays.stream(strMethods) + .map(s -> s.toUpperCase(Locale.ENGLISH)) + .map(RestRequest.Method::valueOf) + .toArray(RestRequest.Method[]::new); + Config config = builder.allowedRequestMethods(methods) + .maxAge(SETTING_CORS_MAX_AGE.get(settings)) + .allowedRequestHeaders(Strings.tokenizeToStringArray(SETTING_CORS_ALLOW_HEADERS.get(settings), ",")) + .build(); + return config; + } +} diff --git a/server/src/main/java/org/elasticsearch/http/DefaultRestChannel.java b/server/src/main/java/org/elasticsearch/http/DefaultRestChannel.java index 9d21896182c67..5191a8d85cef6 100644 --- a/server/src/main/java/org/elasticsearch/http/DefaultRestChannel.java +++ b/server/src/main/java/org/elasticsearch/http/DefaultRestChannel.java @@ -33,6 +33,7 @@ import org.elasticsearch.rest.RestChannel; import org.elasticsearch.rest.RestRequest; import org.elasticsearch.rest.RestResponse; +import org.elasticsearch.rest.RestStatus; import java.util.ArrayList; import java.util.List; @@ -76,29 +77,42 @@ protected BytesStreamOutput newBytesOutput() { @Override public void sendResponse(RestResponse restResponse) { - HttpResponse httpResponse; - if (RestRequest.Method.HEAD == request.method()) { - httpResponse = httpRequest.createResponse(restResponse.status(), BytesArray.EMPTY); - } else { - httpResponse = httpRequest.createResponse(restResponse.status(), restResponse.content()); + final ArrayList toClose = new ArrayList<>(3); + if (isCloseConnection()) { + toClose.add(() -> CloseableChannel.closeChannel(httpChannel)); } - // TODO: Ideally we should move the setting of Cors headers into :server - // NioCorsHandler.setCorsResponseHeaders(nettyRequest, resp, corsConfig); + boolean success = false; + try { + final BytesReference content = restResponse.content(); + if (content instanceof Releasable) { + toClose.add((Releasable) content); + } - String opaque = request.header(X_OPAQUE_ID); - if (opaque != null) { - setHeaderField(httpResponse, X_OPAQUE_ID, opaque); - } + BytesReference finalContent = content; + try { + if (request.method() == RestRequest.Method.HEAD) { + finalContent = BytesArray.EMPTY; + } + } catch (IllegalArgumentException ignored) { + assert restResponse.status() == RestStatus.METHOD_NOT_ALLOWED : + "request HTTP method is unsupported but HTTP status is not METHOD_NOT_ALLOWED(405)"; + } - // Add all custom headers - addCustomHeaders(httpResponse, restResponse.getHeaders()); - addCustomHeaders(httpResponse, threadContext.getResponseHeaders()); + final HttpResponse httpResponse = httpRequest.createResponse(restResponse.status(), finalContent); - ArrayList toClose = new ArrayList<>(3); + // TODO: Ideally we should move the setting of Cors headers into :server + // NioCorsHandler.setCorsResponseHeaders(nettyRequest, resp, corsConfig); + + String opaque = request.header(X_OPAQUE_ID); + if (opaque != null) { + setHeaderField(httpResponse, X_OPAQUE_ID, opaque); + } + + // Add all custom headers + addCustomHeaders(httpResponse, restResponse.getHeaders()); + addCustomHeaders(httpResponse, threadContext.getResponseHeaders()); - boolean success = false; - try { // If our response doesn't specify a content-type header, set one setHeaderField(httpResponse, CONTENT_TYPE, restResponse.contentType(), false); // If our response has no content-length, calculate and set one @@ -106,19 +120,11 @@ public void sendResponse(RestResponse restResponse) { addCookies(httpResponse); - BytesReference content = restResponse.content(); - if (content instanceof Releasable) { - toClose.add((Releasable) content); - } BytesStreamOutput bytesStreamOutput = bytesOutputOrNull(); if (bytesStreamOutput instanceof ReleasableBytesStreamOutput) { toClose.add((Releasable) bytesStreamOutput); } - if (isCloseConnection()) { - toClose.add(() -> CloseableChannel.closeChannel(httpChannel)); - } - ActionListener listener = ActionListener.wrap(() -> Releasables.close(toClose)); httpChannel.sendResponse(httpResponse, listener); success = true; @@ -127,7 +133,6 @@ public void sendResponse(RestResponse restResponse) { Releasables.close(toClose); } } - } private void setHeaderField(HttpResponse response, String headerField, String value) { diff --git a/server/src/main/java/org/elasticsearch/http/HttpRequest.java b/server/src/main/java/org/elasticsearch/http/HttpRequest.java index 496fec23312b0..02a3a58d1702d 100644 --- a/server/src/main/java/org/elasticsearch/http/HttpRequest.java +++ b/server/src/main/java/org/elasticsearch/http/HttpRequest.java @@ -37,6 +37,12 @@ enum HttpVersion { HTTP_1_1 } + /** + * Returns the HTTP method used in the HTTP request. + * + * @return the {@link RestRequest.Method} used in the REST request + * @throws IllegalArgumentException if the HTTP method is invalid + */ RestRequest.Method method(); /** diff --git a/server/src/main/java/org/elasticsearch/index/IndexModule.java b/server/src/main/java/org/elasticsearch/index/IndexModule.java index ca0f34803cc0c..6ef335144eb52 100644 --- a/server/src/main/java/org/elasticsearch/index/IndexModule.java +++ b/server/src/main/java/org/elasticsearch/index/IndexModule.java @@ -19,6 +19,10 @@ package org.elasticsearch.index; +import org.apache.lucene.index.DirectoryReader; +import org.apache.lucene.index.FilterDirectoryReader; +import org.apache.lucene.index.IndexReader; +import org.apache.lucene.index.LeafReader; import org.apache.lucene.search.similarities.BM25Similarity; import org.apache.lucene.search.similarities.Similarity; import org.apache.lucene.store.MMapDirectory; @@ -26,6 +30,7 @@ import org.apache.lucene.util.SetOnce; import org.elasticsearch.Version; import org.elasticsearch.client.Client; +import org.elasticsearch.common.CheckedFunction; import org.elasticsearch.common.TriFunction; import org.elasticsearch.common.io.stream.NamedWriteableRegistry; import org.elasticsearch.common.settings.Setting; @@ -38,10 +43,10 @@ import org.elasticsearch.index.cache.query.DisabledQueryCache; import org.elasticsearch.index.cache.query.IndexQueryCache; import org.elasticsearch.index.cache.query.QueryCache; +import org.elasticsearch.index.engine.Engine; import org.elasticsearch.index.engine.EngineFactory; import org.elasticsearch.index.mapper.MapperService; import org.elasticsearch.index.shard.IndexEventListener; -import org.elasticsearch.index.shard.IndexSearcherWrapper; import org.elasticsearch.index.shard.IndexingOperationListener; import org.elasticsearch.index.shard.SearchOperationListener; import org.elasticsearch.index.similarity.SimilarityService; @@ -112,7 +117,8 @@ public final class IndexModule { private final IndexSettings indexSettings; private final AnalysisRegistry analysisRegistry; private final EngineFactory engineFactory; - private SetOnce indexSearcherWrapper = new SetOnce<>(); + private SetOnce>> indexReaderWrapper = new SetOnce<>(); private final Set indexEventListeners = new HashSet<>(); private final Map> similarities = new HashMap<>(); private final Map directoryFactories; @@ -277,13 +283,26 @@ public void addSimilarity(String name, TriFunction + * The {@link CheckedFunction} is invoked each time a {@link Engine.Searcher} is requested to do an operation, + * for example search, and must return a new directory reader wrapping the provided directory reader or if no + * wrapping was performed the provided directory reader. + * The wrapped reader can filter out document just like delete documents etc. but must not change any term or + * document content. + * NOTE: The index reader wrapper ({@link CheckedFunction}) has a per-request lifecycle, + * must delegate {@link IndexReader#getReaderCacheHelper()}, {@link LeafReader#getCoreCacheHelper()} + * and must be an instance of {@link FilterDirectoryReader} that eventually exposes the original reader + * via {@link FilterDirectoryReader#getDelegate()}. + * The returned reader is closed once it goes out of scope. + *

    */ - public void setSearcherWrapper(IndexSearcherWrapperFactory indexSearcherWrapperFactory) { + public void setReaderWrapper(Function> indexReaderWrapperFactory) { ensureNotFrozen(); - this.indexSearcherWrapper.set(indexSearcherWrapperFactory); + this.indexReaderWrapper.set(indexReaderWrapperFactory); } IndexEventListener freeze() { // pkg private for testing @@ -348,16 +367,6 @@ public boolean match(String setting) { } - /** - * Factory for creating new {@link IndexSearcherWrapper} instances - */ - public interface IndexSearcherWrapperFactory { - /** - * Returns a new IndexSearcherWrapper. This method is called once per index per node - */ - IndexSearcherWrapper newWrapper(IndexService indexService); - } - public static Type defaultStoreType(final boolean allowMmap) { if (allowMmap && Constants.JRE_IS_64BIT && MMapDirectory.UNMAP_SUPPORTED) { return Type.HYBRIDFS; @@ -384,8 +393,8 @@ public IndexService newIndexService( NamedWriteableRegistry namedWriteableRegistry) throws IOException { final IndexEventListener eventListener = freeze(); - IndexSearcherWrapperFactory searcherWrapperFactory = indexSearcherWrapper.get() == null - ? (shard) -> null : indexSearcherWrapper.get(); + Function> readerWrapperFactory = + indexReaderWrapper.get() == null ? (shard) -> null : indexReaderWrapper.get(); eventListener.beforeIndexCreated(indexSettings.getIndex(), indexSettings.getSettings()); final IndexStorePlugin.DirectoryFactory directoryFactory = getDirectoryFactory(indexSettings, directoryFactories); final QueryCache queryCache; @@ -402,7 +411,7 @@ public IndexService newIndexService( return new IndexService(indexSettings, indexCreationContext, environment, xContentRegistry, new SimilarityService(indexSettings, scriptService, similarities), shardStoreDeleter, analysisRegistry, engineFactory, circuitBreakerService, bigArrays, threadPool, scriptService, - client, queryCache, directoryFactory, eventListener, searcherWrapperFactory, mapperRegistry, + client, queryCache, directoryFactory, eventListener, readerWrapperFactory, mapperRegistry, indicesFieldDataCache, searchOperationListeners, indexOperationListeners, namedWriteableRegistry); } diff --git a/server/src/main/java/org/elasticsearch/index/IndexService.java b/server/src/main/java/org/elasticsearch/index/IndexService.java index 2d86a2b436d9d..c6c5a7b90106a 100644 --- a/server/src/main/java/org/elasticsearch/index/IndexService.java +++ b/server/src/main/java/org/elasticsearch/index/IndexService.java @@ -20,16 +20,19 @@ package org.elasticsearch.index; import org.apache.logging.log4j.message.ParameterizedMessage; +import org.apache.lucene.index.DirectoryReader; import org.apache.lucene.index.IndexReader; +import org.apache.lucene.index.IndexReaderContext; +import org.apache.lucene.search.IndexSearcher; import org.apache.lucene.search.Sort; import org.apache.lucene.store.AlreadyClosedException; import org.apache.lucene.store.Directory; import org.apache.lucene.util.Accountable; import org.elasticsearch.Assertions; -import org.elasticsearch.Version; import org.elasticsearch.client.Client; import org.elasticsearch.cluster.metadata.IndexMetaData; import org.elasticsearch.cluster.routing.ShardRouting; +import org.elasticsearch.common.CheckedFunction; import org.elasticsearch.common.Nullable; import org.elasticsearch.common.io.stream.NamedWriteableRegistry; import org.elasticsearch.common.settings.Setting; @@ -58,7 +61,6 @@ import org.elasticsearch.index.query.QueryShardContext; import org.elasticsearch.index.seqno.RetentionLeaseSyncer; import org.elasticsearch.index.shard.IndexEventListener; -import org.elasticsearch.index.shard.IndexSearcherWrapper; import org.elasticsearch.index.shard.IndexShard; import org.elasticsearch.index.shard.IndexShardClosedException; import org.elasticsearch.index.shard.IndexingOperationListener; @@ -90,6 +92,7 @@ import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicBoolean; import java.util.function.Consumer; +import java.util.function.Function; import java.util.function.LongSupplier; import java.util.function.Supplier; @@ -104,7 +107,7 @@ public class IndexService extends AbstractIndexComponent implements IndicesClust private final NodeEnvironment nodeEnv; private final ShardStoreDeleter shardStoreDeleter; private final IndexStorePlugin.DirectoryFactory directoryFactory; - private final IndexSearcherWrapper searcherWrapper; + private final CheckedFunction readerWrapper; private final IndexCache indexCache; private final MapperService mapperService; private final NamedXContentRegistry xContentRegistry; @@ -151,7 +154,7 @@ public IndexService( QueryCache queryCache, IndexStorePlugin.DirectoryFactory directoryFactory, IndexEventListener eventListener, - IndexModule.IndexSearcherWrapperFactory wrapperFactory, + Function> wrapperFactory, MapperRegistry mapperRegistry, IndicesFieldDataCache indicesFieldDataCache, List searchOperationListeners, @@ -203,7 +206,7 @@ public IndexService( this.directoryFactory = directoryFactory; this.engineFactory = Objects.requireNonNull(engineFactory); // initialize this last -- otherwise if the wrapper requires any other member to be non-null we fail with an NPE - this.searcherWrapper = wrapperFactory.newWrapper(this); + this.readerWrapper = wrapperFactory.apply(this); this.searchOperationListeners = Collections.unmodifiableList(searchOperationListeners); this.indexingOperationListeners = Collections.unmodifiableList(indexingOperationListeners); // kick off async ops for the first shard in this index @@ -416,7 +419,7 @@ public synchronized IndexShard createShard( similarityService, engineFactory, eventListener, - searcherWrapper, + readerWrapper, threadPool, bigArrays, engineWarmer, @@ -518,19 +521,24 @@ public IndexSettings getIndexSettings() { return indexSettings; } + private IndexSearcher newCachedSearcher(int shardId, IndexReaderContext context) { + IndexSearcher searcher = new IndexSearcher(context); + searcher.setQueryCache(cache().query()); + searcher.setQueryCachingPolicy(getShard(shardId).getQueryCachingPolicy()); + return searcher; + } + /** - * Creates a new QueryShardContext. The context has not types set yet, if types are required set them via - * {@link QueryShardContext#setTypes(String...)}. + * Creates a new QueryShardContext. * - * Passing a {@code null} {@link IndexReader} will return a valid context, however it won't be able to make - * {@link IndexReader}-specific optimizations, such as rewriting containing range queries. + * Passing a {@code null} {@link IndexReader} will return a valid context, however it won't be able to make {@link IndexReader}-specific + * optimizations, such as rewriting containing range queries. */ public QueryShardContext newQueryShardContext(int shardId, IndexReader indexReader, LongSupplier nowInMillis, String clusterAlias) { return new QueryShardContext( - shardId, indexSettings, indexCache.bitsetFilterCache(), indexFieldData::getForField, mapperService(), - similarityService(), scriptService, xContentRegistry, - namedWriteableRegistry, client, indexReader, - nowInMillis, clusterAlias); + shardId, indexSettings, indexCache.bitsetFilterCache(), context -> newCachedSearcher(shardId, context), + indexFieldData::getForField, mapperService(), similarityService(), scriptService, xContentRegistry, namedWriteableRegistry, + client, indexReader, nowInMillis, clusterAlias); } /** @@ -661,9 +669,7 @@ public IndexMetaData getMetaData() { public synchronized void updateMetaData(final IndexMetaData currentIndexMetaData, final IndexMetaData newIndexMetaData) { final boolean updateIndexMetaData = indexSettings.updateIndexMetaData(newIndexMetaData); - if (Assertions.ENABLED - && currentIndexMetaData != null - && currentIndexMetaData.getCreationVersion().onOrAfter(Version.V_6_5_0)) { + if (Assertions.ENABLED && currentIndexMetaData != null) { final long currentSettingsVersion = currentIndexMetaData.getSettingsVersion(); final long newSettingsVersion = newIndexMetaData.getSettingsVersion(); if (currentSettingsVersion == newSettingsVersion) { @@ -748,8 +754,8 @@ public final EngineFactory getEngineFactory() { return engineFactory; } - final IndexSearcherWrapper getSearcherWrapper() { - return searcherWrapper; + final CheckedFunction getReaderWrapper() { + return readerWrapper; } // pkg private for testing final IndexStorePlugin.DirectoryFactory getDirectoryFactory() { @@ -934,6 +940,11 @@ final class AsyncTrimTranslogTask extends BaseAsyncTask { .getSettings().getAsTime(INDEX_TRANSLOG_RETENTION_CHECK_INTERVAL_SETTING, TimeValue.timeValueMinutes(10))); } + @Override + protected boolean mustReschedule() { + return indexService.closed.get() == false; + } + @Override protected void runInternal() { indexService.maybeTrimTranslog(); @@ -1025,8 +1036,8 @@ AsyncTranslogFSync getFsyncTask() { // for tests return fsyncTask; } - AsyncGlobalCheckpointTask getGlobalCheckpointTask() { - return globalCheckpointTask; + AsyncTrimTranslogTask getTrimTranslogTask() { // for tests + return trimTranslogTask; } /** diff --git a/server/src/main/java/org/elasticsearch/index/IndexSettings.java b/server/src/main/java/org/elasticsearch/index/IndexSettings.java index d4cc38f0b959c..ebcf362b1c9e4 100644 --- a/server/src/main/java/org/elasticsearch/index/IndexSettings.java +++ b/server/src/main/java/org/elasticsearch/index/IndexSettings.java @@ -458,7 +458,7 @@ public IndexSettings(final IndexMetaData indexMetaData, final Settings nodeSetti generationThresholdSize = scopedSettings.get(INDEX_TRANSLOG_GENERATION_THRESHOLD_SIZE_SETTING); mergeSchedulerConfig = new MergeSchedulerConfig(this); gcDeletesInMillis = scopedSettings.get(INDEX_GC_DELETES_SETTING).getMillis(); - softDeleteEnabled = version.onOrAfter(Version.V_6_5_0) && scopedSettings.get(INDEX_SOFT_DELETES_SETTING); + softDeleteEnabled = scopedSettings.get(INDEX_SOFT_DELETES_SETTING); softDeleteRetentionOperations = scopedSettings.get(INDEX_SOFT_DELETES_RETENTION_OPERATIONS_SETTING); retentionLeaseMillis = scopedSettings.get(INDEX_SOFT_DELETES_RETENTION_LEASE_PERIOD_SETTING).millis(); warmerEnabled = scopedSettings.get(INDEX_WARMER_ENABLED_SETTING); diff --git a/server/src/main/java/org/elasticsearch/index/SearchSlowLog.java b/server/src/main/java/org/elasticsearch/index/SearchSlowLog.java index abd67a47049d3..a2193a1bdf73d 100644 --- a/server/src/main/java/org/elasticsearch/index/SearchSlowLog.java +++ b/server/src/main/java/org/elasticsearch/index/SearchSlowLog.java @@ -19,8 +19,8 @@ package org.elasticsearch.index; -import org.apache.logging.log4j.Logger; import org.apache.logging.log4j.LogManager; +import org.apache.logging.log4j.Logger; import org.elasticsearch.common.Strings; import org.elasticsearch.common.logging.Loggers; import org.elasticsearch.common.settings.Setting; @@ -170,13 +170,6 @@ public String toString() { sb.append("-1"); } sb.append("], "); - if (context.getQueryShardContext().getTypes() == null) { - sb.append("types[], "); - } else { - sb.append("types["); - Strings.arrayToDelimitedString(context.getQueryShardContext().getTypes(), ",", sb); - sb.append("], "); - } if (context.groupStats() == null) { sb.append("stats[], "); } else { diff --git a/server/src/main/java/org/elasticsearch/index/analysis/AnalysisRegistry.java b/server/src/main/java/org/elasticsearch/index/analysis/AnalysisRegistry.java index 684d36c311f8b..755266604add5 100644 --- a/server/src/main/java/org/elasticsearch/index/analysis/AnalysisRegistry.java +++ b/server/src/main/java/org/elasticsearch/index/analysis/AnalysisRegistry.java @@ -21,6 +21,7 @@ import org.apache.lucene.analysis.Analyzer; import org.apache.lucene.analysis.core.WhitespaceTokenizer; import org.elasticsearch.ElasticsearchException; +import org.elasticsearch.Version; import org.elasticsearch.cluster.metadata.IndexMetaData; import org.elasticsearch.common.settings.Settings; import org.elasticsearch.core.internal.io.IOUtils; @@ -33,11 +34,15 @@ import java.io.Closeable; import java.io.IOException; +import java.io.UncheckedIOException; +import java.util.ArrayList; import java.util.Collections; import java.util.HashMap; +import java.util.List; import java.util.Locale; import java.util.Map; import java.util.concurrent.ConcurrentHashMap; +import java.util.function.BiFunction; import java.util.function.Function; import java.util.stream.Collectors; @@ -86,13 +91,7 @@ public AnalysisRegistry(Environment environment, new PrebuiltAnalysis(preConfiguredCharFilters, preConfiguredTokenFilters, preConfiguredTokenizers, preConfiguredAnalyzers); } - /** - * Returns a {@link Settings} by groupName from {@link IndexSettings} or a default {@link Settings} - * @param indexSettings an index settings - * @param groupName tokenizer/token filter/char filter name - * @return {@link Settings} - */ - public static Settings getSettingsFromIndexSettings(IndexSettings indexSettings, String groupName) { + private static Settings getSettingsFromIndexSettings(IndexSettings indexSettings, String groupName) { Settings settings = indexSettings.getSettings().getAsSettings(groupName); if (settings.isEmpty()) { settings = Settings.builder().put(IndexMetaData.SETTING_VERSION_CREATED, indexSettings.getIndexVersionCreated()).build(); @@ -100,24 +99,75 @@ public static Settings getSettingsFromIndexSettings(IndexSettings indexSettings, return settings; } + private static final IndexSettings NO_INDEX_SETTINGS = new IndexSettings( + IndexMetaData.builder(IndexMetaData.INDEX_UUID_NA_VALUE) + .settings(Settings.builder().put(IndexMetaData.SETTING_VERSION_CREATED, Version.CURRENT)) + .numberOfReplicas(0) + .numberOfShards(1) + .build(), + Settings.EMPTY + ); + + private T getComponentFactory(IndexSettings settings, NameOrDefinition nod, + String componentType, + Function> globalComponentProvider, + Function> prebuiltComponentProvider, + BiFunction> indexComponentProvider) throws IOException { + if (nod.definition != null) { + // custom component, so we build it from scratch + String type = nod.definition.get("type"); + if (type == null) { + throw new IllegalArgumentException("Missing [type] setting for anonymous " + componentType + ": " + nod.definition); + } + AnalysisProvider factory = globalComponentProvider.apply(type); + if (factory == null) { + throw new IllegalArgumentException("failed to find global " + componentType + " under [" + type + "]"); + } + if (settings == null) { + settings = NO_INDEX_SETTINGS; + } + return factory.get(settings, environment, "__anonymous__" + type, nod.definition); + } + if (settings == null) { + // no index provided, so we use prebuilt analysis components + AnalysisProvider factory = prebuiltComponentProvider.apply(nod.name); + if (factory == null) { + // if there's no prebuilt component, try loading a global one to build with no settings + factory = globalComponentProvider.apply(nod.name); + if (factory == null) { + throw new IllegalArgumentException("failed to find global " + componentType + " under [" + nod.name + "]"); + } + } + return factory.get(environment, nod.name); + } else { + // get the component from index settings + AnalysisProvider factory = indexComponentProvider.apply(nod.name, settings); + if (factory == null) { + throw new IllegalArgumentException("failed to find " + componentType + " under [" + nod.name + "]"); + } + Settings s = getSettingsFromIndexSettings(settings, "index.analysis." + componentType + "." + nod.name); + return factory.get(settings, environment, nod.name, s); + } + } + /** * Returns a registered {@link TokenizerFactory} provider by name or null if the tokenizer was not registered */ - public AnalysisModule.AnalysisProvider getTokenizerProvider(String tokenizer) { + private AnalysisModule.AnalysisProvider getTokenizerProvider(String tokenizer) { return tokenizers.getOrDefault(tokenizer, this.prebuiltAnalysis.getTokenizerFactory(tokenizer)); } /** * Returns a registered {@link TokenFilterFactory} provider by name or null if the token filter was not registered */ - public AnalysisModule.AnalysisProvider getTokenFilterProvider(String tokenFilter) { + private AnalysisModule.AnalysisProvider getTokenFilterProvider(String tokenFilter) { return tokenFilters.getOrDefault(tokenFilter, this.prebuiltAnalysis.getTokenFilterFactory(tokenFilter)); } /** * Returns a registered {@link CharFilterFactory} provider by name or null if the char filter was not registered */ - public AnalysisModule.AnalysisProvider getCharFilterProvider(String charFilter) { + private AnalysisModule.AnalysisProvider getCharFilterProvider(String charFilter) { return charFilters.getOrDefault(charFilter, this.prebuiltAnalysis.getCharFilterFactory(charFilter)); } @@ -135,11 +185,7 @@ public Analyzer getAnalyzer(String analyzer) throws IOException { throw new ElasticsearchException("failed to load analyzer for name " + key, ex); }} ); - } else if ("standard_html_strip".equals(analyzer)) { - throw new IllegalArgumentException("[standard_html_strip] analyzer is not supported for new indices, " + - "use a custom analyzer using [standard] tokenizer and [html_strip] char_filter, plus [lowercase] filter"); } - return analyzerProvider.get(environment, analyzer).get(); } @@ -164,6 +210,67 @@ public IndexAnalyzers build(IndexSettings indexSettings) throws IOException { return build(indexSettings, analyzerFactories, normalizerFactories, tokenizerFactories, charFilterFactories, tokenFilterFactories); } + /** + * Creates a custom analyzer from a collection of {@link NameOrDefinition} specifications for each component + * + * Callers are responsible for closing the returned Analyzer + */ + public NamedAnalyzer buildCustomAnalyzer(IndexSettings indexSettings, boolean normalizer, NameOrDefinition tokenizer, + List charFilters, List tokenFilters) throws IOException { + TokenizerFactory tokenizerFactory + = getComponentFactory(indexSettings, tokenizer, "tokenizer", + this::getTokenizerProvider, prebuiltAnalysis::getTokenizerFactory, this::getTokenizerProvider); + + List charFilterFactories = new ArrayList<>(); + for (NameOrDefinition nod : charFilters) { + charFilterFactories.add(getComponentFactory(indexSettings, nod, "char_filter", + this::getCharFilterProvider, prebuiltAnalysis::getCharFilterFactory, this::getCharFilterProvider)); + } + + List tokenFilterFactories = new ArrayList<>(); + for (NameOrDefinition nod : tokenFilters) { + TokenFilterFactory tff = getComponentFactory(indexSettings, nod, "filter", + this::getTokenFilterProvider, prebuiltAnalysis::getTokenFilterFactory, this::getTokenFilterProvider); + if (normalizer && tff instanceof NormalizingTokenFilterFactory == false) { + throw new IllegalArgumentException("Custom normalizer may not use filter [" + tff.name() + "]"); + } + tff = tff.getChainAwareTokenFilterFactory(tokenizerFactory, charFilterFactories, tokenFilterFactories, name -> { + try { + return getComponentFactory(indexSettings, new NameOrDefinition(name), "filter", + this::getTokenFilterProvider, prebuiltAnalysis::getTokenFilterFactory, this::getTokenFilterProvider); + } catch (IOException e) { + throw new UncheckedIOException(e); + } + }); + tokenFilterFactories.add(tff); + } + + String tokenizerName = tokenizer.name == null ? "_anonymous_tokenizer" : tokenizer.name; + if (normalizer) { + tokenizerName = "keyword_for_normalizer"; + } + Analyzer analyzer = new CustomAnalyzer(tokenizerName, tokenizerFactory, + charFilterFactories.toArray(new CharFilterFactory[]{}), + tokenFilterFactories.toArray(new TokenFilterFactory[]{})); + return produceAnalyzer("__custom__", new AnalyzerProvider<>() { + @Override + public String name() { + return "__custom__"; + } + + @Override + public AnalyzerScope scope() { + return AnalyzerScope.GLOBAL; + } + + @Override + public Analyzer get() { + return analyzer; + } + }, null, null, null); + + } + public Map buildTokenFilterFactories(IndexSettings indexSettings) throws IOException { final Map tokenFiltersSettings = indexSettings.getSettings().getGroups(INDEX_ANALYSIS_FILTER); return buildMapping(Component.FILTER, indexSettings, tokenFiltersSettings, this.tokenFilters, @@ -181,12 +288,12 @@ public Map buildCharFilterFactories(IndexSettings ind prebuiltAnalysis.preConfiguredCharFilterFactories); } - public Map> buildAnalyzerFactories(IndexSettings indexSettings) throws IOException { + private Map> buildAnalyzerFactories(IndexSettings indexSettings) throws IOException { final Map analyzersSettings = indexSettings.getSettings().getGroups("index.analysis.analyzer"); return buildMapping(Component.ANALYZER, indexSettings, analyzersSettings, analyzers, prebuiltAnalysis.analyzerProviderFactories); } - public Map> buildNormalizerFactories(IndexSettings indexSettings) throws IOException { + private Map> buildNormalizerFactories(IndexSettings indexSettings) throws IOException { final Map normalizersSettings = indexSettings.getSettings().getGroups("index.analysis.normalizer"); // TODO: Have pre-built normalizers return buildMapping(Component.NORMALIZER, indexSettings, normalizersSettings, normalizers, Collections.emptyMap()); @@ -200,7 +307,7 @@ public Map> buildNormalizerFactories(IndexSettings i * @param indexSettings an index settings * @return {@link TokenizerFactory} provider or null */ - public AnalysisProvider getTokenizerProvider(String tokenizer, IndexSettings indexSettings) { + private AnalysisProvider getTokenizerProvider(String tokenizer, IndexSettings indexSettings) { return getProvider(Component.TOKENIZER, tokenizer, indexSettings, "index.analysis.tokenizer", tokenizers, this::getTokenizerProvider); } @@ -213,7 +320,7 @@ public AnalysisProvider getTokenizerProvider(String tokenizer, * @param indexSettings an index settings * @return {@link TokenFilterFactory} provider or null */ - public AnalysisProvider getTokenFilterProvider(String tokenFilter, IndexSettings indexSettings) { + private AnalysisProvider getTokenFilterProvider(String tokenFilter, IndexSettings indexSettings) { return getProvider(Component.FILTER, tokenFilter, indexSettings, "index.analysis.filter", tokenFilters, this::getTokenFilterProvider); } @@ -226,7 +333,7 @@ public AnalysisProvider getTokenFilterProvider(String tokenF * @param indexSettings an index settings * @return {@link CharFilterFactory} provider or null */ - public AnalysisProvider getCharFilterProvider(String charFilter, IndexSettings indexSettings) { + private AnalysisProvider getCharFilterProvider(String charFilter, IndexSettings indexSettings) { return getProvider(Component.CHAR_FILTER, charFilter, indexSettings, "index.analysis.char_filter", charFilters, this::getCharFilterProvider); } @@ -385,19 +492,19 @@ private PrebuiltAnalysis( this.preConfiguredTokenizers = preConfiguredTokenizers; } - public AnalysisProvider getCharFilterFactory(String name) { + AnalysisProvider getCharFilterFactory(String name) { return preConfiguredCharFilterFactories.get(name); } - public AnalysisProvider getTokenFilterFactory(String name) { + AnalysisProvider getTokenFilterFactory(String name) { return preConfiguredTokenFilters.get(name); } - public AnalysisProvider getTokenizerFactory(String name) { + AnalysisProvider getTokenizerFactory(String name) { return preConfiguredTokenizers.get(name); } - public AnalysisProvider> getAnalyzerProvider(String name) { + AnalysisProvider> getAnalyzerProvider(String name) { return analyzerProviderFactories.get(name); } @@ -414,7 +521,6 @@ public IndexAnalyzers build(IndexSettings indexSettings, Map tokenizerFactoryFactories, Map charFilterFactoryFactories, Map tokenFilterFactoryFactories) { - Map analyzers = new HashMap<>(); Map normalizers = new HashMap<>(); Map whitespaceNormalizers = new HashMap<>(); @@ -453,12 +559,14 @@ public IndexAnalyzers build(IndexSettings indexSettings, throw new IllegalArgumentException("analyzer name must not start with '_'. got \"" + analyzer.getKey() + "\""); } } - return new IndexAnalyzers(indexSettings, analyzers, normalizers, whitespaceNormalizers); + return new IndexAnalyzers(analyzers, normalizers, whitespaceNormalizers); } - private static NamedAnalyzer produceAnalyzer(String name, AnalyzerProvider analyzerFactory, - Map tokenFilters, Map charFilters, - Map tokenizers) { + private static NamedAnalyzer produceAnalyzer(String name, + AnalyzerProvider analyzerFactory, + Map tokenFilters, + Map charFilters, + Map tokenizers) { /* * Lucene defaults positionIncrementGap to 0 in all analyzers but * Elasticsearch defaults them to 0 only before version 2.0 diff --git a/server/src/main/java/org/elasticsearch/index/analysis/AnalyzerComponents.java b/server/src/main/java/org/elasticsearch/index/analysis/AnalyzerComponents.java new file mode 100644 index 0000000000000..f150ac54558e0 --- /dev/null +++ b/server/src/main/java/org/elasticsearch/index/analysis/AnalyzerComponents.java @@ -0,0 +1,111 @@ +/* + * 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.analysis; + +import org.elasticsearch.common.settings.Settings; + +import java.util.ArrayList; +import java.util.List; +import java.util.Map; + +/** + * A class that groups analysis components necessary to produce a custom analyzer. + * See {@link ReloadableCustomAnalyzer} for an example usage. + */ +public final class AnalyzerComponents { + private final String tokenizerName; + private final TokenizerFactory tokenizerFactory; + private final CharFilterFactory[] charFilters; + private final TokenFilterFactory[] tokenFilters; + private final AnalysisMode analysisMode; + + AnalyzerComponents(String tokenizerName, TokenizerFactory tokenizerFactory, CharFilterFactory[] charFilters, + TokenFilterFactory[] tokenFilters) { + this.tokenizerName = tokenizerName; + this.tokenizerFactory = tokenizerFactory; + this.charFilters = charFilters; + this.tokenFilters = tokenFilters; + AnalysisMode mode = AnalysisMode.ALL; + for (TokenFilterFactory f : tokenFilters) { + mode = mode.merge(f.getAnalysisMode()); + } + this.analysisMode = mode; + } + + static AnalyzerComponents createComponents(String name, Settings analyzerSettings, final Map tokenizers, + final Map charFilters, final Map tokenFilters) { + String tokenizerName = analyzerSettings.get("tokenizer"); + if (tokenizerName == null) { + throw new IllegalArgumentException("Custom Analyzer [" + name + "] must be configured with a tokenizer"); + } + + TokenizerFactory tokenizer = tokenizers.get(tokenizerName); + if (tokenizer == null) { + throw new IllegalArgumentException( + "Custom Analyzer [" + name + "] failed to find tokenizer under name " + "[" + tokenizerName + "]"); + } + + List charFilterNames = analyzerSettings.getAsList("char_filter"); + List charFiltersList = new ArrayList<>(charFilterNames.size()); + for (String charFilterName : charFilterNames) { + CharFilterFactory charFilter = charFilters.get(charFilterName); + if (charFilter == null) { + throw new IllegalArgumentException( + "Custom Analyzer [" + name + "] failed to find char_filter under name " + "[" + charFilterName + "]"); + } + charFiltersList.add(charFilter); + } + + List tokenFilterNames = analyzerSettings.getAsList("filter"); + List tokenFilterList = new ArrayList<>(tokenFilterNames.size()); + for (String tokenFilterName : tokenFilterNames) { + TokenFilterFactory tokenFilter = tokenFilters.get(tokenFilterName); + if (tokenFilter == null) { + throw new IllegalArgumentException( + "Custom Analyzer [" + name + "] failed to find filter under name " + "[" + tokenFilterName + "]"); + } + tokenFilter = tokenFilter.getChainAwareTokenFilterFactory(tokenizer, charFiltersList, tokenFilterList, tokenFilters::get); + tokenFilterList.add(tokenFilter); + } + + return new AnalyzerComponents(tokenizerName, tokenizer, charFiltersList.toArray(new CharFilterFactory[charFiltersList.size()]), + tokenFilterList.toArray(new TokenFilterFactory[tokenFilterList.size()])); + } + + public String getTokenizerName() { + return tokenizerName; + } + + public TokenizerFactory getTokenizerFactory() { + return tokenizerFactory; + } + + public TokenFilterFactory[] getTokenFilters() { + return tokenFilters; + } + + public CharFilterFactory[] getCharFilters() { + return charFilters; + } + + public AnalysisMode analysisMode() { + return this.analysisMode; + } +} \ No newline at end of file diff --git a/modules/mapper-extras/src/test/java/org/elasticsearch/index/mapper/DenseVectorFieldTypeTests.java b/server/src/main/java/org/elasticsearch/index/analysis/AnalyzerComponentsProvider.java similarity index 76% rename from modules/mapper-extras/src/test/java/org/elasticsearch/index/mapper/DenseVectorFieldTypeTests.java rename to server/src/main/java/org/elasticsearch/index/analysis/AnalyzerComponentsProvider.java index 7bbf862e0fc2b..84a3a14038f92 100644 --- a/modules/mapper-extras/src/test/java/org/elasticsearch/index/mapper/DenseVectorFieldTypeTests.java +++ b/server/src/main/java/org/elasticsearch/index/analysis/AnalyzerComponentsProvider.java @@ -17,12 +17,13 @@ * under the License. */ -package org.elasticsearch.index.mapper; +package org.elasticsearch.index.analysis; -public class DenseVectorFieldTypeTests extends FieldTypeTestCase { +/** + * Analyzers that provide access to their token filters should implement this + */ +public interface AnalyzerComponentsProvider { + + AnalyzerComponents getComponents(); - @Override - protected MappedFieldType createDefaultFieldType() { - return new DenseVectorFieldMapper.DenseVectorFieldType(); - } } diff --git a/server/src/main/java/org/elasticsearch/index/analysis/CustomAnalyzer.java b/server/src/main/java/org/elasticsearch/index/analysis/CustomAnalyzer.java index a41ee33564400..685dd2a7de036 100644 --- a/server/src/main/java/org/elasticsearch/index/analysis/CustomAnalyzer.java +++ b/server/src/main/java/org/elasticsearch/index/analysis/CustomAnalyzer.java @@ -25,15 +25,9 @@ import java.io.Reader; -public final class CustomAnalyzer extends Analyzer { - - private final String tokenizerName; - private final TokenizerFactory tokenizerFactory; - - private final CharFilterFactory[] charFilters; - - private final TokenFilterFactory[] tokenFilters; +public final class CustomAnalyzer extends Analyzer implements AnalyzerComponentsProvider { + private final AnalyzerComponents components; private final int positionIncrementGap; private final int offsetGap; private final AnalysisMode analysisMode; @@ -45,10 +39,7 @@ public CustomAnalyzer(String tokenizerName, TokenizerFactory tokenizerFactory, C public CustomAnalyzer(String tokenizerName, TokenizerFactory tokenizerFactory, CharFilterFactory[] charFilters, TokenFilterFactory[] tokenFilters, int positionIncrementGap, int offsetGap) { - this.tokenizerName = tokenizerName; - this.tokenizerFactory = tokenizerFactory; - this.charFilters = charFilters; - this.tokenFilters = tokenFilters; + this.components = new AnalyzerComponents(tokenizerName, tokenizerFactory, charFilters, tokenFilters); this.positionIncrementGap = positionIncrementGap; this.offsetGap = offsetGap; // merge and transfer token filter analysis modes with analyzer @@ -63,19 +54,19 @@ public CustomAnalyzer(String tokenizerName, TokenizerFactory tokenizerFactory, C * The name of the tokenizer as configured by the user. */ public String getTokenizerName() { - return tokenizerName; + return this.components.getTokenizerName(); } public TokenizerFactory tokenizerFactory() { - return tokenizerFactory; + return this.components.getTokenizerFactory(); } public TokenFilterFactory[] tokenFilters() { - return tokenFilters; + return this.components.getTokenFilters(); } public CharFilterFactory[] charFilters() { - return charFilters; + return this.components.getCharFilters(); } @Override @@ -95,11 +86,16 @@ public AnalysisMode getAnalysisMode() { return this.analysisMode; } + @Override + public AnalyzerComponents getComponents() { + return this.components; + } + @Override protected TokenStreamComponents createComponents(String fieldName) { - Tokenizer tokenizer = tokenizerFactory.create(); + Tokenizer tokenizer = this.tokenizerFactory().create(); TokenStream tokenStream = tokenizer; - for (TokenFilterFactory tokenFilter : tokenFilters) { + for (TokenFilterFactory tokenFilter : tokenFilters()) { tokenStream = tokenFilter.create(tokenStream); } return new TokenStreamComponents(tokenizer, tokenStream); @@ -107,6 +103,7 @@ protected TokenStreamComponents createComponents(String fieldName) { @Override protected Reader initReader(String fieldName, Reader reader) { + CharFilterFactory[] charFilters = charFilters(); if (charFilters != null && charFilters.length > 0) { for (CharFilterFactory charFilter : charFilters) { reader = charFilter.create(reader); @@ -117,18 +114,18 @@ protected Reader initReader(String fieldName, Reader reader) { @Override protected Reader initReaderForNormalization(String fieldName, Reader reader) { - for (CharFilterFactory charFilter : charFilters) { - reader = charFilter.normalize(reader); - } - return reader; + for (CharFilterFactory charFilter : charFilters()) { + reader = charFilter.normalize(reader); + } + return reader; } @Override protected TokenStream normalize(String fieldName, TokenStream in) { - TokenStream result = in; - for (TokenFilterFactory filter : tokenFilters) { - result = filter.normalize(result); - } - return result; + TokenStream result = in; + for (TokenFilterFactory filter : tokenFilters()) { + result = filter.normalize(result); + } + return result; } } diff --git a/server/src/main/java/org/elasticsearch/index/analysis/CustomAnalyzerProvider.java b/server/src/main/java/org/elasticsearch/index/analysis/CustomAnalyzerProvider.java index 8080a6af876a4..d8a50838e9df4 100644 --- a/server/src/main/java/org/elasticsearch/index/analysis/CustomAnalyzerProvider.java +++ b/server/src/main/java/org/elasticsearch/index/analysis/CustomAnalyzerProvider.java @@ -19,23 +19,24 @@ package org.elasticsearch.index.analysis; +import org.apache.lucene.analysis.Analyzer; import org.elasticsearch.common.settings.Settings; import org.elasticsearch.index.IndexSettings; import org.elasticsearch.index.mapper.TextFieldMapper; -import java.util.ArrayList; -import java.util.List; import java.util.Map; +import static org.elasticsearch.index.analysis.AnalyzerComponents.createComponents; + /** * A custom analyzer that is built out of a single {@link org.apache.lucene.analysis.Tokenizer} and a list * of {@link org.apache.lucene.analysis.TokenFilter}s. */ -public class CustomAnalyzerProvider extends AbstractIndexAnalyzerProvider { +public class CustomAnalyzerProvider extends AbstractIndexAnalyzerProvider { private final Settings analyzerSettings; - private CustomAnalyzer customAnalyzer; + private Analyzer customAnalyzer; public CustomAnalyzerProvider(IndexSettings indexSettings, String name, Settings settings) { @@ -43,58 +44,33 @@ public CustomAnalyzerProvider(IndexSettings indexSettings, this.analyzerSettings = settings; } - void build(final Map tokenizers, final Map charFilters, - final Map tokenFilters) { - String tokenizerName = analyzerSettings.get("tokenizer"); - if (tokenizerName == null) { - throw new IllegalArgumentException("Custom Analyzer [" + name() + "] must be configured with a tokenizer"); - } - - TokenizerFactory tokenizer = tokenizers.get(tokenizerName); - if (tokenizer == null) { - throw new IllegalArgumentException("Custom Analyzer [" + name() + "] failed to find tokenizer under name " + - "[" + tokenizerName + "]"); - } - - List charFilterNames = analyzerSettings.getAsList("char_filter"); - List charFiltersList = new ArrayList<>(charFilterNames.size()); - for (String charFilterName : charFilterNames) { - CharFilterFactory charFilter = charFilters.get(charFilterName); - if (charFilter == null) { - throw new IllegalArgumentException("Custom Analyzer [" + name() + "] failed to find char_filter under name " + - "[" + charFilterName + "]"); - } - charFiltersList.add(charFilter); - } + void build(final Map tokenizers, + final Map charFilters, + final Map tokenFilters) { + customAnalyzer = create(name(), analyzerSettings, tokenizers, charFilters, tokenFilters); + } + /** + * Factory method that either returns a plain {@link ReloadableCustomAnalyzer} if the components used for creation are supporting index + * and search time use, or a {@link ReloadableCustomAnalyzer} if the components are intended for search time use only. + */ + private static Analyzer create(String name, Settings analyzerSettings, Map tokenizers, + Map charFilters, + Map tokenFilters) { int positionIncrementGap = TextFieldMapper.Defaults.POSITION_INCREMENT_GAP; - positionIncrementGap = analyzerSettings.getAsInt("position_increment_gap", positionIncrementGap); - int offsetGap = analyzerSettings.getAsInt("offset_gap", -1); - - List tokenFilterNames = analyzerSettings.getAsList("filter"); - List tokenFilterList = new ArrayList<>(tokenFilterNames.size()); - for (String tokenFilterName : tokenFilterNames) { - TokenFilterFactory tokenFilter = tokenFilters.get(tokenFilterName); - if (tokenFilter == null) { - throw new IllegalArgumentException("Custom Analyzer [" + name() + "] failed to find filter under name " + - "[" + tokenFilterName + "]"); - } - tokenFilter = tokenFilter.getChainAwareTokenFilterFactory(tokenizer, charFiltersList, tokenFilterList, tokenFilters::get); - tokenFilterList.add(tokenFilter); + AnalyzerComponents components = createComponents(name, analyzerSettings, tokenizers, charFilters, tokenFilters); + if (components.analysisMode().equals(AnalysisMode.SEARCH_TIME)) { + return new ReloadableCustomAnalyzer(components, positionIncrementGap, offsetGap); + } else { + return new CustomAnalyzer(components.getTokenizerName(), components.getTokenizerFactory(), components.getCharFilters(), + components.getTokenFilters(), positionIncrementGap, offsetGap); } - - this.customAnalyzer = new CustomAnalyzer(tokenizerName, tokenizer, - charFiltersList.toArray(new CharFilterFactory[charFiltersList.size()]), - tokenFilterList.toArray(new TokenFilterFactory[tokenFilterList.size()]), - positionIncrementGap, - offsetGap - ); } @Override - public CustomAnalyzer get() { + public Analyzer get() { return this.customAnalyzer; } } diff --git a/server/src/main/java/org/elasticsearch/index/analysis/IndexAnalyzers.java b/server/src/main/java/org/elasticsearch/index/analysis/IndexAnalyzers.java index 4f1cbeb4022ac..a44ade8d15391 100644 --- a/server/src/main/java/org/elasticsearch/index/analysis/IndexAnalyzers.java +++ b/server/src/main/java/org/elasticsearch/index/analysis/IndexAnalyzers.java @@ -19,13 +19,12 @@ package org.elasticsearch.index.analysis; import org.elasticsearch.core.internal.io.IOUtils; -import org.elasticsearch.index.AbstractIndexComponent; -import org.elasticsearch.index.IndexSettings; import java.io.Closeable; import java.io.IOException; import java.util.Map; import java.util.Objects; +import java.util.stream.Collectors; import java.util.stream.Stream; import static java.util.Collections.unmodifiableMap; @@ -40,14 +39,13 @@ * * @see AnalysisRegistry */ -public final class IndexAnalyzers extends AbstractIndexComponent implements Closeable { +public final class IndexAnalyzers implements Closeable { private final Map analyzers; private final Map normalizers; private final Map whitespaceNormalizers; - public IndexAnalyzers(IndexSettings indexSettings, Map analyzers, Map normalizers, + public IndexAnalyzers(Map analyzers, Map normalizers, Map whitespaceNormalizers) { - super(indexSettings); Objects.requireNonNull(analyzers.get(DEFAULT_ANALYZER_NAME), "the default analyzer must be set"); if (analyzers.get(DEFAULT_ANALYZER_NAME).name().equals(DEFAULT_ANALYZER_NAME) == false) { throw new IllegalStateException( @@ -65,6 +63,13 @@ public NamedAnalyzer get(String name) { return analyzers.get(name); } + /** + * Returns an (unmodifiable) map of containing the index analyzers + */ + public Map getAnalyzers() { + return analyzers; + } + /** * Returns a normalizer mapped to the given name or null if not present */ @@ -102,8 +107,9 @@ public NamedAnalyzer getDefaultSearchQuoteAnalyzer() { @Override public void close() throws IOException { - IOUtils.close(() -> Stream.concat(analyzers.values().stream(), normalizers.values().stream()) - .filter(a -> a.scope() == AnalyzerScope.INDEX) - .iterator()); + IOUtils.close(Stream.of(analyzers.values().stream(), normalizers.values().stream(), whitespaceNormalizers.values().stream()) + .flatMap(s -> s) + .filter(a -> a.scope() == AnalyzerScope.INDEX) + .collect(Collectors.toList())); } } diff --git a/server/src/main/java/org/elasticsearch/index/analysis/NameOrDefinition.java b/server/src/main/java/org/elasticsearch/index/analysis/NameOrDefinition.java new file mode 100644 index 0000000000000..1831904de6b6f --- /dev/null +++ b/server/src/main/java/org/elasticsearch/index/analysis/NameOrDefinition.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.index.analysis; + +import org.elasticsearch.common.Strings; +import org.elasticsearch.common.io.stream.StreamInput; +import org.elasticsearch.common.io.stream.StreamOutput; +import org.elasticsearch.common.io.stream.Writeable; +import org.elasticsearch.common.settings.Settings; +import org.elasticsearch.common.xcontent.ToXContentFragment; +import org.elasticsearch.common.xcontent.XContentBuilder; +import org.elasticsearch.common.xcontent.XContentFactory; +import org.elasticsearch.common.xcontent.XContentParseException; +import org.elasticsearch.common.xcontent.XContentParser; +import org.elasticsearch.common.xcontent.XContentType; + +import java.io.IOException; +import java.util.Map; +import java.util.Objects; + +public class NameOrDefinition implements Writeable, ToXContentFragment { + // exactly one of these two members is not null + public final String name; + public final Settings definition; + + public NameOrDefinition(String name) { + this.name = Objects.requireNonNull(name); + this.definition = null; + } + + public NameOrDefinition(Map definition) { + this.name = null; + Objects.requireNonNull(definition); + try { + XContentBuilder builder = XContentFactory.contentBuilder(XContentType.JSON); + builder.map(definition); + this.definition = Settings.builder().loadFromSource(Strings.toString(builder), builder.contentType()).build(); + } catch (IOException e) { + throw new IllegalArgumentException("Failed to parse [" + definition + "]", e); + } + } + + public NameOrDefinition(StreamInput in) throws IOException { + name = in.readOptionalString(); + if (in.readBoolean()) { + definition = Settings.readSettingsFromStream(in); + } else { + definition = null; + } + } + + @Override + public void writeTo(StreamOutput out) throws IOException { + out.writeOptionalString(name); + boolean isNotNullDefinition = this.definition != null; + out.writeBoolean(isNotNullDefinition); + if (isNotNullDefinition) { + Settings.writeSettingsToStream(definition, out); + } + } + + public static NameOrDefinition fromXContent(XContentParser parser) throws IOException { + if (parser.currentToken() == XContentParser.Token.VALUE_STRING) { + return new NameOrDefinition(parser.text()); + } + if (parser.currentToken() == XContentParser.Token.START_OBJECT) { + return new NameOrDefinition(parser.map()); + } + throw new XContentParseException(parser.getTokenLocation(), + "Expected [VALUE_STRING] or [START_OBJECT], got " + parser.currentToken()); + } + + @Override + public XContentBuilder toXContent(XContentBuilder builder, Params params) throws IOException { + if (definition == null) { + builder.value(name); + } else { + builder.startObject(); + definition.toXContent(builder, params); + builder.endObject(); + } + return builder; + } + + @Override + public boolean equals(Object o) { + if (this == o) return true; + if (o == null || getClass() != o.getClass()) return false; + NameOrDefinition that = (NameOrDefinition) o; + return Objects.equals(name, that.name) && + Objects.equals(definition, that.definition); + } + + @Override + public int hashCode() { + return Objects.hash(name, definition); + } +} diff --git a/server/src/main/java/org/elasticsearch/index/analysis/NamedAnalyzer.java b/server/src/main/java/org/elasticsearch/index/analysis/NamedAnalyzer.java index 4831d88f3aa1f..0c53cc323d678 100644 --- a/server/src/main/java/org/elasticsearch/index/analysis/NamedAnalyzer.java +++ b/server/src/main/java/org/elasticsearch/index/analysis/NamedAnalyzer.java @@ -112,8 +112,8 @@ public void checkAllowedInMode(AnalysisMode mode) { return; // everything allowed if this analyzer is in ALL mode } if (this.getAnalysisMode() != mode) { - if (analyzer instanceof CustomAnalyzer) { - TokenFilterFactory[] tokenFilters = ((CustomAnalyzer) analyzer).tokenFilters(); + if (analyzer instanceof AnalyzerComponentsProvider) { + TokenFilterFactory[] tokenFilters = ((AnalyzerComponentsProvider) analyzer).getComponents().getTokenFilters(); List offendingFilters = new ArrayList<>(); for (TokenFilterFactory tokenFilter : tokenFilters) { if (tokenFilter.getAnalysisMode() != mode) { diff --git a/server/src/main/java/org/elasticsearch/index/analysis/ReloadableCustomAnalyzer.java b/server/src/main/java/org/elasticsearch/index/analysis/ReloadableCustomAnalyzer.java new file mode 100644 index 0000000000000..7d3b8532caeb0 --- /dev/null +++ b/server/src/main/java/org/elasticsearch/index/analysis/ReloadableCustomAnalyzer.java @@ -0,0 +1,162 @@ +/* + * 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.analysis; + +import org.apache.lucene.analysis.Analyzer; +import org.apache.lucene.analysis.TokenStream; +import org.apache.lucene.analysis.Tokenizer; +import org.apache.lucene.util.CloseableThreadLocal; +import org.elasticsearch.common.settings.Settings; + +import java.io.Reader; +import java.util.Map; + +public final class ReloadableCustomAnalyzer extends Analyzer implements AnalyzerComponentsProvider { + + private volatile AnalyzerComponents components; + + private CloseableThreadLocal storedComponents = new CloseableThreadLocal<>(); + + private final int positionIncrementGap; + + private final int offsetGap; + + /** + * An alternative {@link ReuseStrategy} that allows swapping the stored analyzer components when they change. + * This is used to change e.g. token filters in search time analyzers. + */ + private static final ReuseStrategy UPDATE_STRATEGY = new ReuseStrategy() { + @Override + public TokenStreamComponents getReusableComponents(Analyzer analyzer, String fieldName) { + ReloadableCustomAnalyzer custom = (ReloadableCustomAnalyzer) analyzer; + AnalyzerComponents components = custom.getComponents(); + AnalyzerComponents storedComponents = custom.getStoredComponents(); + if (storedComponents == null || components != storedComponents) { + custom.setStoredComponents(components); + return null; + } + TokenStreamComponents tokenStream = (TokenStreamComponents) getStoredValue(analyzer); + assert tokenStream != null; + return tokenStream; + } + + @Override + public void setReusableComponents(Analyzer analyzer, String fieldName, TokenStreamComponents tokenStream) { + setStoredValue(analyzer, tokenStream); + } + }; + + ReloadableCustomAnalyzer(AnalyzerComponents components, int positionIncrementGap, int offsetGap) { + super(UPDATE_STRATEGY); + if (components.analysisMode().equals(AnalysisMode.SEARCH_TIME) == false) { + throw new IllegalArgumentException( + "ReloadableCustomAnalyzer must only be initialized with analysis components in AnalysisMode.SEARCH_TIME mode"); + } + this.components = components; + this.positionIncrementGap = positionIncrementGap; + this.offsetGap = offsetGap; + } + + @Override + public AnalyzerComponents getComponents() { + return this.components; + } + + @Override + public int getPositionIncrementGap(String fieldName) { + return this.positionIncrementGap; + } + + @Override + public int getOffsetGap(String field) { + if (this.offsetGap < 0) { + return super.getOffsetGap(field); + } + return this.offsetGap; + } + + public AnalysisMode getAnalysisMode() { + return this.components.analysisMode(); + } + + @Override + protected Reader initReaderForNormalization(String fieldName, Reader reader) { + final AnalyzerComponents components = getComponents(); + for (CharFilterFactory charFilter : components.getCharFilters()) { + reader = charFilter.normalize(reader); + } + return reader; + } + + @Override + protected TokenStream normalize(String fieldName, TokenStream in) { + final AnalyzerComponents components = getComponents(); + TokenStream result = in; + for (TokenFilterFactory filter : components.getTokenFilters()) { + result = filter.normalize(result); + } + return result; + } + + public synchronized void reload(String name, + Settings settings, + final Map tokenizers, + final Map charFilters, + final Map tokenFilters) { + AnalyzerComponents components = AnalyzerComponents.createComponents(name, settings, tokenizers, charFilters, tokenFilters); + this.components = components; + } + + @Override + public void close() { + super.close(); + storedComponents.close(); + } + + private void setStoredComponents(AnalyzerComponents components) { + storedComponents.set(components); + } + + private AnalyzerComponents getStoredComponents() { + return storedComponents.get(); + } + + @Override + protected TokenStreamComponents createComponents(String fieldName) { + final AnalyzerComponents components = getStoredComponents(); + Tokenizer tokenizer = components.getTokenizerFactory().create(); + TokenStream tokenStream = tokenizer; + for (TokenFilterFactory tokenFilter : components.getTokenFilters()) { + tokenStream = tokenFilter.create(tokenStream); + } + return new TokenStreamComponents(tokenizer, tokenStream); + } + + @Override + protected Reader initReader(String fieldName, Reader reader) { + final AnalyzerComponents components = getStoredComponents(); + if (components.getCharFilters() != null && components.getCharFilters().length > 0) { + for (CharFilterFactory charFilter : components.getCharFilters()) { + reader = charFilter.create(reader); + } + } + return reader; + } +} diff --git a/server/src/main/java/org/elasticsearch/index/engine/CombinedDeletionPolicy.java b/server/src/main/java/org/elasticsearch/index/engine/CombinedDeletionPolicy.java index addb16d58d031..313598e1d8ec7 100644 --- a/server/src/main/java/org/elasticsearch/index/engine/CombinedDeletionPolicy.java +++ b/server/src/main/java/org/elasticsearch/index/engine/CombinedDeletionPolicy.java @@ -144,7 +144,8 @@ synchronized boolean releaseCommit(final IndexCommit snapshotCommit) { /** * Find a safe commit point from a list of existing commits based on the supplied global checkpoint. * The max sequence number of a safe commit point should be at most the global checkpoint. - * If an index was created before v6.2, and we haven't retained a safe commit yet, this method will return the oldest commit. + * If an index was created before 6.2 or recovered from remote, we might not have a safe commit. + * In this case, this method will return the oldest index commit. * * @param commits a list of existing commit points * @param globalCheckpoint the persisted global checkpoint from the translog, see {@link Translog#readGlobalCheckpoint(Path, String)} @@ -172,22 +173,13 @@ private static int indexOfKeptCommits(List commits, long if (expectedTranslogUUID.equals(commitUserData.get(Translog.TRANSLOG_UUID_KEY)) == false) { return i + 1; } - // 5.x commits do not contain MAX_SEQ_NO, we should not keep it and the older commits. - if (commitUserData.containsKey(SequenceNumbers.MAX_SEQ_NO) == false) { - return Math.min(commits.size() - 1, i + 1); - } final long maxSeqNoFromCommit = Long.parseLong(commitUserData.get(SequenceNumbers.MAX_SEQ_NO)); if (maxSeqNoFromCommit <= globalCheckpoint) { return i; } } - /* - * We may reach to this point in these cases: - * 1. In the previous 6.x, we keep only the last commit - which is likely not a safe commit if writes are in progress. - * Thus, after upgrading, we may not find a safe commit until we can reserve one. - * 2. In peer-recovery, if the file-based happens, a replica will be received the latest commit from a primary. - * However, that commit may not be a safe commit if writes are in progress in the primary. - */ + // If an index was created before 6.2 or recovered from remote, we might not have a safe commit. + // In this case, we return the oldest index commit instead. return 0; } @@ -204,11 +196,9 @@ synchronized boolean hasSnapshottedCommits() { boolean hasUnreferencedCommits() throws IOException { final IndexCommit lastCommit = this.lastCommit; if (safeCommit != lastCommit) { // Race condition can happen but harmless - if (lastCommit.getUserData().containsKey(SequenceNumbers.MAX_SEQ_NO)) { - final long maxSeqNoFromLastCommit = Long.parseLong(lastCommit.getUserData().get(SequenceNumbers.MAX_SEQ_NO)); - // We can clean up the current safe commit if the last commit is safe - return globalCheckpointSupplier.getAsLong() >= maxSeqNoFromLastCommit; - } + final long maxSeqNoFromLastCommit = Long.parseLong(lastCommit.getUserData().get(SequenceNumbers.MAX_SEQ_NO)); + // We can clean up the current safe commit if the last commit is safe + return globalCheckpointSupplier.getAsLong() >= maxSeqNoFromLastCommit; } return false; } diff --git a/server/src/main/java/org/elasticsearch/index/engine/CombinedDocValues.java b/server/src/main/java/org/elasticsearch/index/engine/CombinedDocValues.java new file mode 100644 index 0000000000000..009dc9799d52a --- /dev/null +++ b/server/src/main/java/org/elasticsearch/index/engine/CombinedDocValues.java @@ -0,0 +1,90 @@ +/* + * 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.engine; + +import org.apache.lucene.index.LeafReader; +import org.apache.lucene.index.NumericDocValues; +import org.elasticsearch.index.mapper.SeqNoFieldMapper; +import org.elasticsearch.index.mapper.SourceFieldMapper; +import org.elasticsearch.index.mapper.VersionFieldMapper; + +import java.io.IOException; +import java.util.Objects; + +final class CombinedDocValues { + private final NumericDocValues versionDV; + private final NumericDocValues seqNoDV; + private final NumericDocValues primaryTermDV; + private final NumericDocValues tombstoneDV; + private final NumericDocValues recoverySource; + + CombinedDocValues(LeafReader leafReader) throws IOException { + this.versionDV = Objects.requireNonNull(leafReader.getNumericDocValues(VersionFieldMapper.NAME), "VersionDV is missing"); + this.seqNoDV = Objects.requireNonNull(leafReader.getNumericDocValues(SeqNoFieldMapper.NAME), "SeqNoDV is missing"); + this.primaryTermDV = Objects.requireNonNull( + leafReader.getNumericDocValues(SeqNoFieldMapper.PRIMARY_TERM_NAME), "PrimaryTermDV is missing"); + this.tombstoneDV = leafReader.getNumericDocValues(SeqNoFieldMapper.TOMBSTONE_NAME); + this.recoverySource = leafReader.getNumericDocValues(SourceFieldMapper.RECOVERY_SOURCE_NAME); + } + + long docVersion(int segmentDocId) throws IOException { + assert versionDV.docID() < segmentDocId; + if (versionDV.advanceExact(segmentDocId) == false) { + throw new IllegalStateException("DocValues for field [" + VersionFieldMapper.NAME + "] is not found"); + } + return versionDV.longValue(); + } + + long docSeqNo(int segmentDocId) throws IOException { + assert seqNoDV.docID() < segmentDocId; + if (seqNoDV.advanceExact(segmentDocId) == false) { + throw new IllegalStateException("DocValues for field [" + SeqNoFieldMapper.NAME + "] is not found"); + } + return seqNoDV.longValue(); + } + + long docPrimaryTerm(int segmentDocId) throws IOException { + if (primaryTermDV == null) { + return -1L; + } + assert primaryTermDV.docID() < segmentDocId; + // Use -1 for docs which don't have primary term. The caller considers those docs as nested docs. + if (primaryTermDV.advanceExact(segmentDocId) == false) { + return -1; + } + return primaryTermDV.longValue(); + } + + boolean isTombstone(int segmentDocId) throws IOException { + if (tombstoneDV == null) { + return false; + } + assert tombstoneDV.docID() < segmentDocId; + return tombstoneDV.advanceExact(segmentDocId) && tombstoneDV.longValue() > 0; + } + + boolean hasRecoverySource(int segmentDocId) throws IOException { + if (recoverySource == null) { + return false; + } + assert recoverySource.docID() < segmentDocId; + return recoverySource.advanceExact(segmentDocId); + } +} diff --git a/server/src/main/java/org/elasticsearch/index/engine/Engine.java b/server/src/main/java/org/elasticsearch/index/engine/Engine.java index 2d210b716d4b7..e21b816aefd80 100644 --- a/server/src/main/java/org/elasticsearch/index/engine/Engine.java +++ b/server/src/main/java/org/elasticsearch/index/engine/Engine.java @@ -793,9 +793,9 @@ public final CommitStats commitStats() { } /** - * @return the local checkpoint for this Engine + * @return the persisted local checkpoint for this Engine */ - public abstract long getLocalCheckpoint(); + public abstract long getPersistedLocalCheckpoint(); /** * @return a {@link SeqNoStats} object, using local state and the supplied global checkpoint @@ -933,7 +933,7 @@ protected void writerSegmentStats(SegmentsStats stats) { /** How much heap is used that would be freed by a refresh. Note that this may throw {@link AlreadyClosedException}. */ public abstract long getIndexBufferRAMBytesUsed(); - protected Segment[] getSegmentInfo(SegmentInfos lastCommittedSegmentInfos, boolean verbose) { + final Segment[] getSegmentInfo(SegmentInfos lastCommittedSegmentInfos, boolean verbose) { ensureOpen(); Map segments = new HashMap<>(); // first, go over and compute the search ones... @@ -960,8 +960,8 @@ protected Segment[] getSegmentInfo(SegmentInfos lastCommittedSegmentInfos, boole segment = new Segment(info.info.name); segment.search = false; segment.committed = true; - segment.docCount = info.info.maxDoc(); - segment.delDocCount = info.getDelCount(); + segment.delDocCount = info.getDelCount() + info.getSoftDelCount(); + segment.docCount = info.info.maxDoc() - segment.delDocCount; segment.version = info.info.getVersion(); segment.compound = info.info.getUseCompoundFile(); try { @@ -1142,7 +1142,7 @@ public abstract void forceMerge(boolean flush, int maxNumSegments, boolean onlyE */ @SuppressWarnings("finally") private void maybeDie(final String maybeMessage, final Throwable maybeFatal) { - ExceptionsHelper.maybeError(maybeFatal, logger).ifPresent(error -> { + ExceptionsHelper.maybeError(maybeFatal).ifPresent(error -> { try { logger.error(maybeMessage, error); } finally { diff --git a/server/src/main/java/org/elasticsearch/index/engine/InternalEngine.java b/server/src/main/java/org/elasticsearch/index/engine/InternalEngine.java index 9fb63d0de019d..d96220983c971 100644 --- a/server/src/main/java/org/elasticsearch/index/engine/InternalEngine.java +++ b/server/src/main/java/org/elasticsearch/index/engine/InternalEngine.java @@ -20,7 +20,10 @@ package org.elasticsearch.index.engine; import org.apache.logging.log4j.Logger; +import org.apache.lucene.codecs.blocktree.BlockTreeTermsReader; +import org.apache.lucene.codecs.blocktree.BlockTreeTermsReader.FSTLoadMode; import org.apache.lucene.document.Field; +import org.apache.lucene.document.LongPoint; import org.apache.lucene.document.NumericDocValuesField; import org.apache.lucene.index.DirectoryReader; import org.apache.lucene.index.IndexCommit; @@ -29,20 +32,28 @@ import org.apache.lucene.index.IndexWriterConfig; import org.apache.lucene.index.IndexableField; import org.apache.lucene.index.LeafReader; +import org.apache.lucene.index.LeafReaderContext; import org.apache.lucene.index.LiveIndexWriterConfig; import org.apache.lucene.index.MergePolicy; import org.apache.lucene.index.SegmentCommitInfo; import org.apache.lucene.index.SegmentInfos; import org.apache.lucene.index.SoftDeletesRetentionMergePolicy; import org.apache.lucene.index.Term; +import org.apache.lucene.search.DocIdSetIterator; import org.apache.lucene.search.IndexSearcher; +import org.apache.lucene.search.Query; import org.apache.lucene.search.ReferenceManager; +import org.apache.lucene.search.ScoreMode; +import org.apache.lucene.search.Scorer; import org.apache.lucene.search.SearcherFactory; import org.apache.lucene.search.SearcherManager; import org.apache.lucene.search.TermQuery; +import org.apache.lucene.search.Weight; import org.apache.lucene.store.AlreadyClosedException; import org.apache.lucene.store.Directory; +import org.apache.lucene.store.FilterDirectory; import org.apache.lucene.store.LockObtainFailedException; +import org.apache.lucene.store.MMapDirectory; import org.apache.lucene.util.BytesRef; import org.apache.lucene.util.InfoStream; import org.elasticsearch.Assertions; @@ -64,12 +75,14 @@ import org.elasticsearch.core.internal.io.IOUtils; import org.elasticsearch.index.IndexSettings; import org.elasticsearch.index.VersionType; +import org.elasticsearch.index.fieldvisitor.IdOnlyFieldVisitor; import org.elasticsearch.index.mapper.IdFieldMapper; import org.elasticsearch.index.mapper.MapperService; import org.elasticsearch.index.mapper.ParseContext; import org.elasticsearch.index.mapper.ParsedDocument; import org.elasticsearch.index.mapper.SeqNoFieldMapper; import org.elasticsearch.index.mapper.SourceFieldMapper; +import org.elasticsearch.index.mapper.Uid; import org.elasticsearch.index.merge.MergeStats; import org.elasticsearch.index.merge.OnGoingMerge; import org.elasticsearch.index.seqno.LocalCheckpointTracker; @@ -77,6 +90,7 @@ import org.elasticsearch.index.seqno.SequenceNumbers; import org.elasticsearch.index.shard.ElasticsearchMergePolicy; import org.elasticsearch.index.shard.ShardId; +import org.elasticsearch.index.store.FsDirectoryFactory; import org.elasticsearch.index.store.Store; import org.elasticsearch.index.translog.Translog; import org.elasticsearch.index.translog.TranslogConfig; @@ -89,7 +103,6 @@ import java.io.IOException; import java.nio.file.Path; import java.util.Arrays; -import java.util.Collection; import java.util.HashMap; import java.util.List; import java.util.Map; @@ -103,8 +116,9 @@ import java.util.concurrent.locks.Lock; import java.util.concurrent.locks.ReentrantLock; import java.util.function.BiFunction; +import java.util.function.LongConsumer; import java.util.function.LongSupplier; -import java.util.function.Supplier; +import java.util.stream.Collectors; import java.util.stream.Stream; public class InternalEngine extends Engine { @@ -191,13 +205,21 @@ public InternalEngine(EngineConfig engineConfig) { throttle = new IndexThrottle(); try { trimUnsafeCommits(engineConfig); - translog = openTranslog(engineConfig, translogDeletionPolicy, engineConfig.getGlobalCheckpointSupplier()); + translog = openTranslog(engineConfig, translogDeletionPolicy, engineConfig.getGlobalCheckpointSupplier(), + seqNo -> { + final LocalCheckpointTracker tracker = getLocalCheckpointTracker(); + assert tracker != null || getTranslog().isOpen() == false; + if (tracker != null) { + tracker.markSeqNoAsPersisted(seqNo); + } + }); assert translog.getGeneration() != null; this.translog = translog; this.softDeleteEnabled = engineConfig.getIndexSettings().isSoftDeleteEnabled(); this.softDeletesPolicy = newSoftDeletesPolicy(); this.combinedDeletionPolicy = new CombinedDeletionPolicy(logger, translogDeletionPolicy, softDeletesPolicy, translog::getLastSyncedGlobalCheckpoint); + this.localCheckpointTracker = createLocalCheckpointTracker(localCheckpointTrackerSupplier); writer = createWriter(); bootstrapAppendOnlyInfoFromWriter(writer); historyUUID = loadHistoryUUID(writer); @@ -227,11 +249,17 @@ public InternalEngine(EngineConfig engineConfig) { for (ReferenceManager.RefreshListener listener: engineConfig.getInternalRefreshListener()) { this.internalSearcherManager.addListener(listener); } - this.localCheckpointTracker = createLocalCheckpointTracker(engineConfig, lastCommittedSegmentInfos, logger, - () -> acquireSearcher("create_local_checkpoint_tracker", SearcherScope.INTERNAL), localCheckpointTrackerSupplier); - this.lastRefreshedCheckpointListener = new LastRefreshedCheckpointListener(localCheckpointTracker.getCheckpoint()); + this.lastRefreshedCheckpointListener = new LastRefreshedCheckpointListener(localCheckpointTracker.getProcessedCheckpoint()); this.internalSearcherManager.addListener(lastRefreshedCheckpointListener); maxSeqNoOfUpdatesOrDeletes = new AtomicLong(SequenceNumbers.max(localCheckpointTracker.getMaxSeqNo(), translog.getMaxSeqNo())); + if (softDeleteEnabled && localCheckpointTracker.getPersistedCheckpoint() < localCheckpointTracker.getMaxSeqNo()) { + try (Searcher searcher = acquireSearcher("restore_version_map_and_checkpoint_tracker", SearcherScope.INTERNAL)) { + restoreVersionMapAndCheckpointTracker(Lucene.wrapAllDocsLive(searcher.getDirectoryReader())); + } catch (IOException e) { + throw new EngineCreationFailureException(config().getShardId(), + "failed to restore version map and local checkpoint tracker", e); + } + } success = true; } finally { if (success == false) { @@ -245,30 +273,16 @@ public InternalEngine(EngineConfig engineConfig) { logger.trace("created new InternalEngine"); } - private static LocalCheckpointTracker createLocalCheckpointTracker(EngineConfig engineConfig, SegmentInfos lastCommittedSegmentInfos, - Logger logger, Supplier searcherSupplier, BiFunction localCheckpointTrackerSupplier) { - try { - final SequenceNumbers.CommitInfo seqNoStats = - SequenceNumbers.loadSeqNoInfoFromLuceneCommit(lastCommittedSegmentInfos.userData.entrySet()); - final long maxSeqNo = seqNoStats.maxSeqNo; - final long localCheckpoint = seqNoStats.localCheckpoint; - logger.trace("recovered maximum sequence number [{}] and local checkpoint [{}]", maxSeqNo, localCheckpoint); - final LocalCheckpointTracker tracker = localCheckpointTrackerSupplier.apply(maxSeqNo, localCheckpoint); - // Operations that are optimized using max_seq_no_of_updates optimization must not be processed twice; otherwise, they will - // create duplicates in Lucene. To avoid this we check the LocalCheckpointTracker to see if an operation was already processed. - // Thus, we need to restore the LocalCheckpointTracker bit by bit to ensure the consistency between LocalCheckpointTracker and - // Lucene index. This is not the only solution since we can bootstrap max_seq_no_of_updates with max_seq_no of the commit to - // disable the MSU optimization during recovery. Here we prefer to maintain the consistency of LocalCheckpointTracker. - if (localCheckpoint < maxSeqNo && engineConfig.getIndexSettings().isSoftDeleteEnabled()) { - try (Searcher searcher = searcherSupplier.get()) { - Lucene.scanSeqNosInReader(searcher.getDirectoryReader(), localCheckpoint + 1, maxSeqNo, - tracker::markSeqNoAsCompleted); - } - } - return tracker; - } catch (IOException ex) { - throw new EngineCreationFailureException(engineConfig.getShardId(), "failed to create local checkpoint tracker", ex); - } + private LocalCheckpointTracker createLocalCheckpointTracker( + BiFunction localCheckpointTrackerSupplier) throws IOException { + final long maxSeqNo; + final long localCheckpoint; + final SequenceNumbers.CommitInfo seqNoStats = + SequenceNumbers.loadSeqNoInfoFromLuceneCommit(store.readLastCommittedSegmentsInfo().userData.entrySet()); + maxSeqNo = seqNoStats.maxSeqNo; + localCheckpoint = seqNoStats.localCheckpoint; + logger.trace("recovered maximum sequence number [{}] and local checkpoint [{}]", maxSeqNo, localCheckpoint); + return localCheckpointTrackerSupplier.apply(maxSeqNo, localCheckpoint); } private SoftDeletesPolicy newSoftDeletesPolicy() throws IOException { @@ -360,7 +374,7 @@ protected int getRefCount(IndexSearcher reference) { public int restoreLocalHistoryFromTranslog(TranslogRecoveryRunner translogRecoveryRunner) throws IOException { try (ReleasableLock ignored = readLock.acquire()) { ensureOpen(); - final long localCheckpoint = localCheckpointTracker.getCheckpoint(); + final long localCheckpoint = localCheckpointTracker.getProcessedCheckpoint(); try (Translog.Snapshot snapshot = getTranslog().newSnapshotFromMinSeqNo(localCheckpoint + 1)) { return translogRecoveryRunner.run(this, snapshot); } @@ -371,19 +385,23 @@ public int restoreLocalHistoryFromTranslog(TranslogRecoveryRunner translogRecove public int fillSeqNoGaps(long primaryTerm) throws IOException { try (ReleasableLock ignored = writeLock.acquire()) { ensureOpen(); - final long localCheckpoint = localCheckpointTracker.getCheckpoint(); + final long localCheckpoint = localCheckpointTracker.getProcessedCheckpoint(); final long maxSeqNo = localCheckpointTracker.getMaxSeqNo(); int numNoOpsAdded = 0; for ( long seqNo = localCheckpoint + 1; seqNo <= maxSeqNo; - seqNo = localCheckpointTracker.getCheckpoint() + 1 /* the local checkpoint might have advanced so we leap-frog */) { + seqNo = localCheckpointTracker.getProcessedCheckpoint() + 1 /* leap-frog the local checkpoint */) { innerNoOp(new NoOp(seqNo, primaryTerm, Operation.Origin.PRIMARY, System.nanoTime(), "filling gaps")); numNoOpsAdded++; - assert seqNo <= localCheckpointTracker.getCheckpoint() - : "local checkpoint did not advance; was [" + seqNo + "], now [" + localCheckpointTracker.getCheckpoint() + "]"; + assert seqNo <= localCheckpointTracker.getProcessedCheckpoint() : + "local checkpoint did not advance; was [" + seqNo + "], now [" + localCheckpointTracker.getProcessedCheckpoint() + "]"; } + syncTranslog(); // to persist noops associated with the advancement of the local checkpoint + assert localCheckpointTracker.getPersistedCheckpoint() == maxSeqNo + : "persisted local checkpoint did not advance to max seq no; is [" + localCheckpointTracker.getPersistedCheckpoint() + + "], max seq no [" + maxSeqNo + "]"; return numNoOpsAdded; } } @@ -461,13 +479,13 @@ private void recoverFromTranslogInternal(TranslogRecoveryRunner translogRecovery } private Translog openTranslog(EngineConfig engineConfig, TranslogDeletionPolicy translogDeletionPolicy, - LongSupplier globalCheckpointSupplier) throws IOException { + LongSupplier globalCheckpointSupplier, LongConsumer persistedSequenceNumberConsumer) throws IOException { final TranslogConfig translogConfig = engineConfig.getTranslogConfig(); final String translogUUID = loadTranslogUUIDFromLastCommit(); // We expect that this shard already exists, so it must already have an existing translog else something is badly wrong! return new Translog(translogConfig, translogUUID, translogDeletionPolicy, globalCheckpointSupplier, - engineConfig.getPrimaryTermSupplier()); + engineConfig.getPrimaryTermSupplier(), persistedSequenceNumberConsumer); } // Package private for testing purposes only @@ -671,21 +689,26 @@ enum OpVsLuceneDocStatus { LUCENE_DOC_NOT_FOUND } + private static OpVsLuceneDocStatus compareOpToVersionMapOnSeqNo(String id, long seqNo, long primaryTerm, VersionValue versionValue) { + Objects.requireNonNull(versionValue); + if (seqNo > versionValue.seqNo) { + return OpVsLuceneDocStatus.OP_NEWER; + } else if (seqNo == versionValue.seqNo) { + assert versionValue.term == primaryTerm : "primary term not matched; id=" + id + " seq_no=" + seqNo + + " op_term=" + primaryTerm + " existing_term=" + versionValue.term; + return OpVsLuceneDocStatus.OP_STALE_OR_EQUAL; + } else { + return OpVsLuceneDocStatus.OP_STALE_OR_EQUAL; + } + } + private OpVsLuceneDocStatus compareOpToLuceneDocBasedOnSeqNo(final Operation op) throws IOException { assert op.seqNo() != SequenceNumbers.UNASSIGNED_SEQ_NO : "resolving ops based on seq# but no seqNo is found"; final OpVsLuceneDocStatus status; VersionValue versionValue = getVersionFromMap(op.uid().bytes()); assert incrementVersionLookup(); if (versionValue != null) { - if (op.seqNo() > versionValue.seqNo) { - status = OpVsLuceneDocStatus.OP_NEWER; - } else if (op.seqNo() == versionValue.seqNo) { - assert versionValue.term == op.primaryTerm() : "primary term not matched; id=" + op.id() + " seq_no=" + op.seqNo() - + " op_term=" + op.primaryTerm() + " existing_term=" + versionValue.term; - status = OpVsLuceneDocStatus.OP_STALE_OR_EQUAL; - } else { - status = OpVsLuceneDocStatus.OP_STALE_OR_EQUAL; - } + status = compareOpToVersionMapOnSeqNo(op.id(), op.seqNo(), op.primaryTerm(), versionValue); } else { // load from index assert incrementIndexVersionLookup(); @@ -694,13 +717,9 @@ private OpVsLuceneDocStatus compareOpToLuceneDocBasedOnSeqNo(final Operation op) if (docAndSeqNo == null) { status = OpVsLuceneDocStatus.LUCENE_DOC_NOT_FOUND; } else if (op.seqNo() > docAndSeqNo.seqNo) { - if (docAndSeqNo.isLive) { - status = OpVsLuceneDocStatus.OP_NEWER; - } else { - status = OpVsLuceneDocStatus.LUCENE_DOC_NOT_FOUND; - } + status = OpVsLuceneDocStatus.OP_NEWER; } else if (op.seqNo() == docAndSeqNo.seqNo) { - assert localCheckpointTracker.contains(op.seqNo()) || softDeleteEnabled == false : + assert localCheckpointTracker.hasProcessed(op.seqNo()) || softDeleteEnabled == false : "local checkpoint tracker is not updated seq_no=" + op.seqNo() + " id=" + op.id(); status = OpVsLuceneDocStatus.OP_STALE_OR_EQUAL; } else { @@ -903,7 +922,12 @@ public IndexResult index(Index index) throws IOException { versionMap.maybePutIndexUnderLock(index.uid().bytes(), new IndexVersionValue(translogLocation, plan.versionForIndexing, index.seqNo(), index.primaryTerm())); } - localCheckpointTracker.markSeqNoAsCompleted(indexResult.getSeqNo()); + localCheckpointTracker.markSeqNoAsProcessed(indexResult.getSeqNo()); + if (indexResult.getTranslogLocation() == null) { + // the op is coming from the translog (and is hence persisted already) or it does not have a sequence number + assert index.origin().isFromTranslog() || indexResult.getSeqNo() == SequenceNumbers.UNASSIGNED_SEQ_NO; + localCheckpointTracker.markSeqNoAsPersisted(indexResult.getSeqNo()); + } indexResult.setTook(System.nanoTime() - index.startTime()); indexResult.freeze(); return indexResult; @@ -943,7 +967,7 @@ protected final IndexingStrategy planIndexingAsNonPrimary(Index index) throws IO // unlike the primary, replicas don't really care to about creation status of documents // this allows to ignore the case where a document was found in the live version maps in // a delete state and return false for the created flag in favor of code simplicity - if (index.seqNo() <= localCheckpointTracker.getCheckpoint()){ + if (index.seqNo() <= localCheckpointTracker.getProcessedCheckpoint()){ // the operation seq# is lower then the current local checkpoint and thus was already put into lucene // this can happen during recovery where older operations are sent from the translog that are already // part of the lucene commit (either from a peer recovery or a local translog) @@ -1256,7 +1280,12 @@ public DeleteResult delete(Delete delete) throws IOException { final Translog.Location location = translog.add(new Translog.Delete(delete, deleteResult)); deleteResult.setTranslogLocation(location); } - localCheckpointTracker.markSeqNoAsCompleted(deleteResult.getSeqNo()); + localCheckpointTracker.markSeqNoAsProcessed(deleteResult.getSeqNo()); + if (deleteResult.getTranslogLocation() == null) { + // the op is coming from the translog (and is hence persisted already) or does not have a sequence number (version conflict) + assert delete.origin().isFromTranslog() || deleteResult.getSeqNo() == SequenceNumbers.UNASSIGNED_SEQ_NO; + localCheckpointTracker.markSeqNoAsPersisted(deleteResult.getSeqNo()); + } deleteResult.setTook(System.nanoTime() - delete.startTime()); deleteResult.freeze(); } catch (RuntimeException | IOException e) { @@ -1289,7 +1318,7 @@ protected final DeletionStrategy planDeletionAsNonPrimary(Delete delete) throws // this allows to ignore the case where a document was found in the live version maps in // a delete state and return true for the found flag in favor of code simplicity final DeletionStrategy plan; - if (delete.seqNo() <= localCheckpointTracker.getCheckpoint()) { + if (delete.seqNo() <= localCheckpointTracker.getProcessedCheckpoint()) { // the operation seq# is lower then the current local checkpoint and thus was already put into lucene // this can happen during recovery where older operations are sent from the translog that are already // part of the lucene commit (either from a peer recovery or a local translog) @@ -1379,7 +1408,7 @@ private DeleteResult deleteInLucene(Delete delete, DeletionStrategy plan) throws return new DeleteResult( plan.versionOfDeletion, getPrimaryTerm(), delete.seqNo(), plan.currentlyDeleted == false); } catch (Exception ex) { - if (indexWriter.getTragicException() == null) { + if (ex instanceof AlreadyClosedException == false && indexWriter.getTragicException() == null) { throw new AssertionError("delete operation should never fail at document level", ex); } throw ex; @@ -1464,9 +1493,8 @@ private NoOpResult innerNoOp(final NoOp noOp) throws IOException { final NoOpResult noOpResult; final Optional preFlightError = preFlightCheckForNoOp(noOp); if (preFlightError.isPresent()) { - noOpResult = new NoOpResult(getPrimaryTerm(), noOp.seqNo(), preFlightError.get()); + noOpResult = new NoOpResult(getPrimaryTerm(), SequenceNumbers.UNASSIGNED_SEQ_NO, preFlightError.get()); } else { - Exception failure = null; markSeqNoAsSeen(noOp.seqNo()); if (softDeleteEnabled) { try { @@ -1483,23 +1511,24 @@ private NoOpResult innerNoOp(final NoOp noOp) throws IOException { doc.add(softDeletesField); indexWriter.addDocument(doc); } catch (Exception ex) { - if (maybeFailEngine("noop", ex)) { - throw ex; + if (ex instanceof AlreadyClosedException == false && indexWriter.getTragicException() == null) { + throw new AssertionError("noop operation should never fail at document level", ex); } - failure = ex; + throw ex; } } - if (failure == null) { - noOpResult = new NoOpResult(getPrimaryTerm(), noOp.seqNo()); - } else { - noOpResult = new NoOpResult(getPrimaryTerm(), noOp.seqNo(), failure); - } + noOpResult = new NoOpResult(getPrimaryTerm(), noOp.seqNo()); if (noOp.origin().isFromTranslog() == false && noOpResult.getResultType() == Result.Type.SUCCESS) { final Translog.Location location = translog.add(new Translog.NoOp(noOp.seqNo(), noOp.primaryTerm(), noOp.reason())); noOpResult.setTranslogLocation(location); } } - localCheckpointTracker.markSeqNoAsCompleted(seqNo); + localCheckpointTracker.markSeqNoAsProcessed(noOpResult.getSeqNo()); + if (noOpResult.getTranslogLocation() == null) { + // the op is coming from the translog (and is hence persisted already) or it does not have a sequence number + assert noOp.origin().isFromTranslog() || noOpResult.getSeqNo() == SequenceNumbers.UNASSIGNED_SEQ_NO; + localCheckpointTracker.markSeqNoAsPersisted(noOpResult.getSeqNo()); + } noOpResult.setTook(System.nanoTime() - noOp.startTime()); noOpResult.freeze(); return noOpResult; @@ -1529,7 +1558,7 @@ final boolean refresh(String source, SearcherScope scope, boolean block) throws // since it flushes the index as well (though, in terms of concurrency, we are allowed to do it) // both refresh types will result in an internal refresh but only the external will also // pass the new reader reference to the external reader manager. - final long localCheckpointBeforeRefresh = getLocalCheckpoint(); + final long localCheckpointBeforeRefresh = localCheckpointTracker.getProcessedCheckpoint(); boolean refreshed; try (ReleasableLock lock = readLock.acquire()) { ensureOpen(); @@ -1671,9 +1700,9 @@ public boolean shouldPeriodicallyFlush() { * This method is to maintain translog only, thus IndexWriter#hasUncommittedChanges condition is not considered. */ final long translogGenerationOfNewCommit = - translog.getMinGenerationForSeqNo(localCheckpointTracker.getCheckpoint() + 1).translogFileGeneration; + translog.getMinGenerationForSeqNo(localCheckpointTracker.getProcessedCheckpoint() + 1).translogFileGeneration; return translogGenerationOfLastCommit < translogGenerationOfNewCommit - || localCheckpointTracker.getCheckpoint() == localCheckpointTracker.getMaxSeqNo(); + || localCheckpointTracker.getProcessedCheckpoint() == localCheckpointTracker.getMaxSeqNo(); } @Override @@ -1860,7 +1889,7 @@ private void pruneDeletedTombstones() { */ final long timeMSec = engineConfig.getThreadPool().relativeTimeInMillis(); final long maxTimestampToPrune = timeMSec - engineConfig.getIndexSettings().getGcDeletesInMillis(); - versionMap.pruneTombstones(maxTimestampToPrune, localCheckpointTracker.getCheckpoint()); + versionMap.pruneTombstones(maxTimestampToPrune, localCheckpointTracker.getProcessedCheckpoint()); lastDeleteVersionPruneTimeMSec = timeMSec; } @@ -1870,8 +1899,9 @@ void clearDeletedTombstones() { } // for testing - final Collection getDeletedTombstones() { - return versionMap.getAllTombstones().values(); + final Map getVersionMap() { + return Stream.concat(versionMap.getAllCurrent().entrySet().stream(), versionMap.getAllTombstones().entrySet().stream()) + .collect(Collectors.toMap(Map.Entry::getKey, Map.Entry::getValue)); } @Override @@ -2140,10 +2170,21 @@ IndexWriter createWriter(Directory directory, IndexWriterConfig iwc) throws IOEx } } + static Map getReaderAttributes(Directory directory) { + Directory unwrap = FilterDirectory.unwrap(directory); + boolean defaultOffHeap = FsDirectoryFactory.isHybridFs(unwrap) || unwrap instanceof MMapDirectory; + return Map.of( + BlockTreeTermsReader.FST_MODE_KEY, // if we are using MMAP for term dics we force all off heap unless it's the ID field + defaultOffHeap ? FSTLoadMode.OFF_HEAP.name() : FSTLoadMode.ON_HEAP.name() + , BlockTreeTermsReader.FST_MODE_KEY + "." + IdFieldMapper.NAME, // always force ID field on-heap for fast updates + FSTLoadMode.ON_HEAP.name()); + } + private IndexWriterConfig getIndexWriterConfig() { final IndexWriterConfig iwc = new IndexWriterConfig(engineConfig.getAnalyzer()); iwc.setCommitOnClose(false); // we by default don't commit on close iwc.setOpenMode(IndexWriterConfig.OpenMode.APPEND); + iwc.setReaderAttributes(getReaderAttributes(store.directory())); iwc.setIndexDeletionPolicy(combinedDeletionPolicy); // with tests.verbose, lucene sets this up: plumb to align with filesystem stream boolean verbose = false; @@ -2160,7 +2201,8 @@ private IndexWriterConfig getIndexWriterConfig() { iwc.setSoftDeletesField(Lucene.SOFT_DELETES_FIELD); if (softDeleteEnabled) { mergePolicy = new RecoverySourcePruneMergePolicy(SourceFieldMapper.RECOVERY_SOURCE_NAME, softDeletesPolicy::getRetentionQuery, - new SoftDeletesRetentionMergePolicy(Lucene.SOFT_DELETES_FIELD, softDeletesPolicy::getRetentionQuery, mergePolicy)); + new SoftDeletesRetentionMergePolicy(Lucene.SOFT_DELETES_FIELD, softDeletesPolicy::getRetentionQuery, + new PrunePostingsMergePolicy(mergePolicy, IdFieldMapper.NAME))); } iwc.setMergePolicy(new ElasticsearchMergePolicy(mergePolicy)); iwc.setSimilarity(engineConfig.getSimilarity()); @@ -2336,7 +2378,7 @@ protected void doRun() throws Exception { protected void commitIndexWriter(final IndexWriter writer, final Translog translog, @Nullable final String syncId) throws IOException { ensureCanFlush(); try { - final long localCheckpoint = localCheckpointTracker.getCheckpoint(); + final long localCheckpoint = localCheckpointTracker.getProcessedCheckpoint(); final Translog.TranslogGeneration translogGeneration = translog.getMinGenerationForSeqNo(localCheckpoint + 1); final String translogFileGeneration = Long.toString(translogGeneration.translogFileGeneration); final String translogUUID = translogGeneration.translogUUID; @@ -2421,7 +2463,6 @@ public MergeStats getMergeStats() { return mergeScheduler.stats(); } - // Used only for testing! Package private to prevent anyone else from using it LocalCheckpointTracker getLocalCheckpointTracker() { return localCheckpointTracker; } @@ -2431,9 +2472,13 @@ public long getLastSyncedGlobalCheckpoint() { return getTranslog().getLastSyncedGlobalCheckpoint(); } + public long getProcessedLocalCheckpoint() { + return localCheckpointTracker.getProcessedCheckpoint(); + } + @Override - public long getLocalCheckpoint() { - return localCheckpointTracker.getCheckpoint(); + public long getPersistedLocalCheckpoint() { + return localCheckpointTracker.getPersistedCheckpoint(); } /** @@ -2456,7 +2501,7 @@ protected final boolean hasBeenProcessedBefore(Operation op) { assert versionMap.assertKeyedLockHeldByCurrentThread(op.uid().bytes()); } } - return localCheckpointTracker.contains(op.seqNo()); + return localCheckpointTracker.hasProcessed(op.seqNo()); } @Override @@ -2490,10 +2535,6 @@ private boolean incrementIndexVersionLookup() { return true; } - int getVersionMapSize() { - return versionMap.getAllCurrent().size(); - } - boolean isSafeAccessRequired() { return versionMap.isSafeAccessRequired(); } @@ -2550,7 +2591,7 @@ public Translog.Snapshot newChangesSnapshot(String source, MapperService mapperS @Override public boolean hasCompleteOperationHistory(String source, MapperService mapperService, long startingSeqNo) throws IOException { - final long currentLocalCheckpoint = getLocalCheckpointTracker().getCheckpoint(); + final long currentLocalCheckpoint = localCheckpointTracker.getProcessedCheckpoint(); // avoid scanning translog if not necessary if (startingSeqNo > currentLocalCheckpoint) { return true; @@ -2560,11 +2601,11 @@ public boolean hasCompleteOperationHistory(String source, MapperService mapperSe Translog.Operation operation; while ((operation = snapshot.next()) != null) { if (operation.seqNo() != SequenceNumbers.UNASSIGNED_SEQ_NO) { - tracker.markSeqNoAsCompleted(operation.seqNo()); + tracker.markSeqNoAsProcessed(operation.seqNo()); } } } - return tracker.getCheckpoint() >= currentLocalCheckpoint; + return tracker.getProcessedCheckpoint() >= currentLocalCheckpoint; } /** @@ -2680,7 +2721,7 @@ private final class LastRefreshedCheckpointListener implements ReferenceManager. @Override public void beforeRefresh() { // all changes until this point should be visible after refresh - pendingCheckpoint = localCheckpointTracker.getCheckpoint(); + pendingCheckpoint = localCheckpointTracker.getProcessedCheckpoint(); } @Override @@ -2741,7 +2782,7 @@ private boolean assertMaxSeqNoOfUpdatesIsAdvanced(Term id, long seqNo, boolean a // Operations can be processed on a replica in a different order than on the primary. If the order on the primary is index-1, // delete-2, index-3, and the order on a replica is index-1, index-3, delete-2, then the msu of index-3 on the replica is 2 // even though it is an update (overwrites index-1). We should relax this assertion if there is a pending gap in the seq_no. - if (relaxIfGapInSeqNo && getLocalCheckpoint() < maxSeqNoOfUpdates) { + if (relaxIfGapInSeqNo && localCheckpointTracker.getProcessedCheckpoint() < maxSeqNoOfUpdates) { return true; } assert seqNo <= maxSeqNoOfUpdates : "id=" + id + " seq_no=" + seqNo + " msu=" + maxSeqNoOfUpdates; @@ -2756,4 +2797,58 @@ private static void trimUnsafeCommits(EngineConfig engineConfig) throws IOExcept final long minRetainedTranslogGen = Translog.readMinTranslogGeneration(translogPath, translogUUID); store.trimUnsafeCommits(globalCheckpoint, minRetainedTranslogGen, engineConfig.getIndexSettings().getIndexVersionCreated()); } + + /** + * Restores the live version map and local checkpoint of this engine using documents (including soft-deleted) + * after the local checkpoint in the safe commit. This step ensures the live version map and checkpoint tracker + * are in sync with the Lucene commit. + */ + private void restoreVersionMapAndCheckpointTracker(DirectoryReader directoryReader) throws IOException { + final IndexSearcher searcher = new IndexSearcher(directoryReader); + searcher.setQueryCache(null); + final Query query = LongPoint.newRangeQuery(SeqNoFieldMapper.NAME, getPersistedLocalCheckpoint() + 1, Long.MAX_VALUE); + final Weight weight = searcher.createWeight(query, ScoreMode.COMPLETE_NO_SCORES, 1.0f); + for (LeafReaderContext leaf : directoryReader.leaves()) { + final Scorer scorer = weight.scorer(leaf); + if (scorer == null) { + continue; + } + final CombinedDocValues dv = new CombinedDocValues(leaf.reader()); + final IdOnlyFieldVisitor idFieldVisitor = new IdOnlyFieldVisitor(); + final DocIdSetIterator iterator = scorer.iterator(); + int docId; + while ((docId = iterator.nextDoc()) != DocIdSetIterator.NO_MORE_DOCS) { + final long primaryTerm = dv.docPrimaryTerm(docId); + if (primaryTerm == -1L) { + continue; // skip children docs which do not have primary term + } + final long seqNo = dv.docSeqNo(docId); + localCheckpointTracker.markSeqNoAsProcessed(seqNo); + localCheckpointTracker.markSeqNoAsPersisted(seqNo); + idFieldVisitor.reset(); + leaf.reader().document(docId, idFieldVisitor); + if (idFieldVisitor.getId() == null) { + assert dv.isTombstone(docId); + continue; + } + final BytesRef uid = new Term(IdFieldMapper.NAME, Uid.encodeId(idFieldVisitor.getId())).bytes(); + try (Releasable ignored = versionMap.acquireLock(uid)) { + final VersionValue curr = versionMap.getUnderLock(uid); + if (curr == null || + compareOpToVersionMapOnSeqNo(idFieldVisitor.getId(), seqNo, primaryTerm, curr) == OpVsLuceneDocStatus.OP_NEWER) { + if (dv.isTombstone(docId)) { + // use 0L for the start time so we can prune this delete tombstone quickly + // when the local checkpoint advances (i.e., after a recovery completed). + final long startTime = 0L; + versionMap.putDeleteUnderLock(uid, new DeleteVersionValue(dv.docVersion(docId), seqNo, primaryTerm, startTime)); + } else { + versionMap.putIndexUnderLock(uid, new IndexVersionValue(null, dv.docVersion(docId), seqNo, primaryTerm)); + } + } + } + } + } + // remove live entries in the version map + refresh("restore_version_map_and_checkpoint_tracker", SearcherScope.INTERNAL, true); + } } diff --git a/server/src/main/java/org/elasticsearch/index/engine/LuceneChangesSnapshot.java b/server/src/main/java/org/elasticsearch/index/engine/LuceneChangesSnapshot.java index a3e86ab1606df..bf5ecc3350a5b 100644 --- a/server/src/main/java/org/elasticsearch/index/engine/LuceneChangesSnapshot.java +++ b/server/src/main/java/org/elasticsearch/index/engine/LuceneChangesSnapshot.java @@ -40,14 +40,12 @@ import org.elasticsearch.index.mapper.SeqNoFieldMapper; import org.elasticsearch.index.mapper.SourceFieldMapper; import org.elasticsearch.index.mapper.Uid; -import org.elasticsearch.index.mapper.VersionFieldMapper; import org.elasticsearch.index.translog.Translog; import java.io.Closeable; import java.io.IOException; import java.util.Comparator; import java.util.List; -import java.util.Objects; import java.util.concurrent.atomic.AtomicBoolean; /** @@ -213,11 +211,8 @@ private void fillParallelArray(ScoreDoc[] scoreDocs, ParallelArray parallelArray private TopDocs searchOperations(ScoreDoc after) throws IOException { final Query rangeQuery = LongPoint.newRangeQuery(SeqNoFieldMapper.NAME, Math.max(fromSeqNo, lastSeenSeqNo), toSeqNo); - final Sort sortedBySeqNoThenByTerm = new Sort( - new SortField(SeqNoFieldMapper.NAME, SortField.Type.LONG), - new SortField(SeqNoFieldMapper.PRIMARY_TERM_NAME, SortField.Type.LONG, true) - ); - return indexSearcher.searchAfter(after, rangeQuery, searchBatchSize, sortedBySeqNoThenByTerm); + final Sort sortedBySeqNo = new Sort(new SortField(SeqNoFieldMapper.NAME, SortField.Type.LONG)); + return indexSearcher.searchAfter(after, rangeQuery, searchBatchSize, sortedBySeqNo); } private Translog.Operation readDocAsOp(int docIndex) throws IOException { @@ -305,64 +300,4 @@ private static final class ParallelArray { } } - private static final class CombinedDocValues { - private final NumericDocValues versionDV; - private final NumericDocValues seqNoDV; - private final NumericDocValues primaryTermDV; - private final NumericDocValues tombstoneDV; - private final NumericDocValues recoverySource; - - CombinedDocValues(LeafReader leafReader) throws IOException { - this.versionDV = Objects.requireNonNull(leafReader.getNumericDocValues(VersionFieldMapper.NAME), "VersionDV is missing"); - this.seqNoDV = Objects.requireNonNull(leafReader.getNumericDocValues(SeqNoFieldMapper.NAME), "SeqNoDV is missing"); - this.primaryTermDV = Objects.requireNonNull( - leafReader.getNumericDocValues(SeqNoFieldMapper.PRIMARY_TERM_NAME), "PrimaryTermDV is missing"); - this.tombstoneDV = leafReader.getNumericDocValues(SeqNoFieldMapper.TOMBSTONE_NAME); - this.recoverySource = leafReader.getNumericDocValues(SourceFieldMapper.RECOVERY_SOURCE_NAME); - } - - long docVersion(int segmentDocId) throws IOException { - assert versionDV.docID() < segmentDocId; - if (versionDV.advanceExact(segmentDocId) == false) { - throw new IllegalStateException("DocValues for field [" + VersionFieldMapper.NAME + "] is not found"); - } - return versionDV.longValue(); - } - - long docSeqNo(int segmentDocId) throws IOException { - assert seqNoDV.docID() < segmentDocId; - if (seqNoDV.advanceExact(segmentDocId) == false) { - throw new IllegalStateException("DocValues for field [" + SeqNoFieldMapper.NAME + "] is not found"); - } - return seqNoDV.longValue(); - } - - long docPrimaryTerm(int segmentDocId) throws IOException { - if (primaryTermDV == null) { - return -1L; - } - assert primaryTermDV.docID() < segmentDocId; - // Use -1 for docs which don't have primary term. The caller considers those docs as nested docs. - if (primaryTermDV.advanceExact(segmentDocId) == false) { - return -1; - } - return primaryTermDV.longValue(); - } - - boolean isTombstone(int segmentDocId) throws IOException { - if (tombstoneDV == null) { - return false; - } - assert tombstoneDV.docID() < segmentDocId; - return tombstoneDV.advanceExact(segmentDocId) && tombstoneDV.longValue() > 0; - } - - boolean hasRecoverySource(int segmentDocId) throws IOException { - if (recoverySource == null) { - return false; - } - assert recoverySource.docID() < segmentDocId; - return recoverySource.advanceExact(segmentDocId); - } - } } diff --git a/server/src/main/java/org/elasticsearch/index/engine/NoOpEngine.java b/server/src/main/java/org/elasticsearch/index/engine/NoOpEngine.java index a41f07e994b93..2244dd57037cb 100644 --- a/server/src/main/java/org/elasticsearch/index/engine/NoOpEngine.java +++ b/server/src/main/java/org/elasticsearch/index/engine/NoOpEngine.java @@ -27,16 +27,24 @@ import org.apache.lucene.index.SegmentReader; import org.apache.lucene.store.Directory; import org.elasticsearch.common.lucene.Lucene; +import org.elasticsearch.common.util.concurrent.ReleasableLock; +import org.elasticsearch.index.store.Store; +import org.elasticsearch.index.translog.Translog; +import org.elasticsearch.index.translog.TranslogConfig; +import org.elasticsearch.index.translog.TranslogDeletionPolicy; import java.io.IOException; import java.io.UncheckedIOException; import java.util.List; +import java.util.Map; import java.util.function.Function; /** * NoOpEngine is an engine implementation that does nothing but the bare minimum * required in order to have an engine. All attempts to do something (search, - * index, get), throw {@link UnsupportedOperationException}. + * index, get), throw {@link UnsupportedOperationException}. However, NoOpEngine + * allows to trim any existing translog files through the usage of the + * {{@link #trimUnreferencedTranslogFiles()}} method. */ public final class NoOpEngine extends ReadOnlyEngine { @@ -46,7 +54,7 @@ public NoOpEngine(EngineConfig config) { super(config, null, null, true, Function.identity()); this.stats = new SegmentsStats(); Directory directory = store.directory(); - try (DirectoryReader reader = DirectoryReader.open(directory)) { + try (DirectoryReader reader = DirectoryReader.open(directory, OFF_HEAP_READER_ATTRIBUTES)) { for (LeafReaderContext ctx : reader.getContext().leaves()) { SegmentReader segmentReader = Lucene.segmentReader(ctx.reader()); fillSegmentStats(segmentReader, true, stats); @@ -116,4 +124,54 @@ public SegmentsStats segmentsStats(boolean includeSegmentFileSizes, boolean incl return super.segmentsStats(includeSegmentFileSizes, includeUnloadedSegments); } } + + /** + * This implementation will trim existing translog files using a {@link TranslogDeletionPolicy} + * that retains nothing but the last translog generation from safe commit. + */ + @Override + public void trimUnreferencedTranslogFiles() { + final Store store = this.engineConfig.getStore(); + store.incRef(); + try (ReleasableLock lock = readLock.acquire()) { + ensureOpen(); + final List commits = DirectoryReader.listCommits(store.directory()); + if (commits.size() == 1) { + final Map commitUserData = getLastCommittedSegmentInfos().getUserData(); + final String translogUuid = commitUserData.get(Translog.TRANSLOG_UUID_KEY); + if (translogUuid == null) { + throw new IllegalStateException("commit doesn't contain translog unique id"); + } + if (commitUserData.containsKey(Translog.TRANSLOG_GENERATION_KEY) == false) { + throw new IllegalStateException("commit doesn't contain translog generation id"); + } + final long lastCommitGeneration = Long.parseLong(commitUserData.get(Translog.TRANSLOG_GENERATION_KEY)); + final TranslogConfig translogConfig = engineConfig.getTranslogConfig(); + final long minTranslogGeneration = Translog.readMinTranslogGeneration(translogConfig.getTranslogPath(), translogUuid); + + if (minTranslogGeneration < lastCommitGeneration) { + // a translog deletion policy that retains nothing but the last translog generation from safe commit + final TranslogDeletionPolicy translogDeletionPolicy = new TranslogDeletionPolicy(-1, -1); + translogDeletionPolicy.setTranslogGenerationOfLastCommit(lastCommitGeneration); + translogDeletionPolicy.setMinTranslogGenerationForRecovery(lastCommitGeneration); + + try (Translog translog = new Translog(translogConfig, translogUuid, translogDeletionPolicy, + engineConfig.getGlobalCheckpointSupplier(), engineConfig.getPrimaryTermSupplier(), seqNo -> {})) { + translog.trimUnreferencedReaders(); + // refresh the translog stats + this.translogStats = translog.stats(); + } + } + } + } catch (final Exception e) { + try { + failEngine("translog trimming failed", e); + } catch (Exception inner) { + e.addSuppressed(inner); + } + throw new EngineException(shardId, "failed to trim translog", e); + } finally { + store.decRef(); + } + } } diff --git a/server/src/main/java/org/elasticsearch/index/engine/PrunePostingsMergePolicy.java b/server/src/main/java/org/elasticsearch/index/engine/PrunePostingsMergePolicy.java new file mode 100644 index 0000000000000..0c973ba93f65d --- /dev/null +++ b/server/src/main/java/org/elasticsearch/index/engine/PrunePostingsMergePolicy.java @@ -0,0 +1,224 @@ +/* + * 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.engine; + +import org.apache.lucene.codecs.FieldsProducer; +import org.apache.lucene.index.CodecReader; +import org.apache.lucene.index.FilterCodecReader; +import org.apache.lucene.index.FilterLeafReader; +import org.apache.lucene.index.FilteredTermsEnum; +import org.apache.lucene.index.ImpactsEnum; +import org.apache.lucene.index.MergePolicy; +import org.apache.lucene.index.OneMergeWrappingMergePolicy; +import org.apache.lucene.index.PostingsEnum; +import org.apache.lucene.index.Terms; +import org.apache.lucene.index.TermsEnum; +import org.apache.lucene.search.DocIdSetIterator; +import org.apache.lucene.util.Bits; +import org.apache.lucene.util.BytesRef; + +import java.io.IOException; +import java.util.Iterator; + +/** + * This merge policy drops id field postings for all delete documents this can be + * useful to guarantee consistent update performance even if a large number of deleted / updated documents + * are retained. Merging postings away is efficient since lucene visits postings term by term and + * with the original live-docs being available we are adding a negotiable overhead such that we can + * prune soft-deletes by default. Yet, using this merge policy will cause loosing all search capabilities on top of + * soft deleted documents independent of the retention policy. Note, in order for this merge policy to be effective it needs to be added + * before the {@link org.apache.lucene.index.SoftDeletesRetentionMergePolicy} because otherwise only documents that are deleted / removed + * anyways will be pruned. + */ +final class PrunePostingsMergePolicy extends OneMergeWrappingMergePolicy { + + PrunePostingsMergePolicy(MergePolicy in, String idField) { + super(in, toWrap -> new OneMerge(toWrap.segments) { + @Override + public CodecReader wrapForMerge(CodecReader reader) throws IOException { + CodecReader wrapped = toWrap.wrapForMerge(reader); + return wrapReader(wrapped, idField); + } + }); + } + + private static CodecReader wrapReader(CodecReader reader, String idField) { + Bits liveDocs = reader.getLiveDocs(); + if (liveDocs == null) { + return reader; // no deleted docs - we are good! + } + final boolean fullyDeletedSegment = reader.numDocs() == 0; + return new FilterCodecReader(reader) { + + @Override + public FieldsProducer getPostingsReader() { + FieldsProducer postingsReader = super.getPostingsReader(); + if (postingsReader == null) { + return null; + } + return new FieldsProducer() { + @Override + public void close() throws IOException { + postingsReader.close(); + } + + @Override + public void checkIntegrity() throws IOException { + postingsReader.checkIntegrity(); + } + + @Override + public Iterator iterator() { + return postingsReader.iterator(); + } + + @Override + public Terms terms(String field) throws IOException { + Terms in = postingsReader.terms(field); + if (idField.equals(field) && in != null) { + return new FilterLeafReader.FilterTerms(in) { + @Override + public TermsEnum iterator() throws IOException { + TermsEnum iterator = super.iterator(); + return new FilteredTermsEnum(iterator, false) { + private PostingsEnum internal; + + @Override + protected AcceptStatus accept(BytesRef term) throws IOException { + if (fullyDeletedSegment) { + return AcceptStatus.END; // short-cut this if we don't match anything + } + internal = postings(internal, PostingsEnum.NONE); + if (internal.nextDoc() != DocIdSetIterator.NO_MORE_DOCS) { + return AcceptStatus.YES; + } + return AcceptStatus.NO; + } + + @Override + public PostingsEnum postings(PostingsEnum reuse, int flags) throws IOException { + if (reuse instanceof OnlyLiveDocsPostingsEnum) { + OnlyLiveDocsPostingsEnum reuseInstance = (OnlyLiveDocsPostingsEnum) reuse; + reuseInstance.reset(super.postings(reuseInstance.in, flags)); + return reuseInstance; + } + return new OnlyLiveDocsPostingsEnum(super.postings(null, flags), liveDocs); + } + + @Override + public ImpactsEnum impacts(int flags) throws IOException { + throw new UnsupportedOperationException(); + } + }; + } + }; + } else { + return in; + } + } + + @Override + public int size() { + return postingsReader.size(); + } + + @Override + public long ramBytesUsed() { + return postingsReader.ramBytesUsed(); + } + }; + } + + @Override + public CacheHelper getCoreCacheHelper() { + return null; + } + + @Override + public CacheHelper getReaderCacheHelper() { + return null; + } + }; + } + + private static final class OnlyLiveDocsPostingsEnum extends PostingsEnum { + + private final Bits liveDocs; + private PostingsEnum in; + + OnlyLiveDocsPostingsEnum(PostingsEnum in, Bits liveDocs) { + this.liveDocs = liveDocs; + reset(in); + } + + void reset(PostingsEnum in) { + this.in = in; + } + + @Override + public int docID() { + return in.docID(); + } + + @Override + public int nextDoc() throws IOException { + int docId; + do { + docId = in.nextDoc(); + } while (docId != DocIdSetIterator.NO_MORE_DOCS && liveDocs.get(docId) == false); + return docId; + } + + @Override + public int advance(int target) { + throw new UnsupportedOperationException(); + } + + @Override + public long cost() { + return in.cost(); + } + + @Override + public int freq() throws IOException { + return in.freq(); + } + + @Override + public int nextPosition() throws IOException { + return in.nextPosition(); + } + + @Override + public int startOffset() throws IOException { + return in.startOffset(); + } + + @Override + public int endOffset() throws IOException { + return in.endOffset(); + } + + @Override + public BytesRef getPayload() throws IOException { + return in.getPayload(); + } + } +} diff --git a/server/src/main/java/org/elasticsearch/index/engine/ReadOnlyEngine.java b/server/src/main/java/org/elasticsearch/index/engine/ReadOnlyEngine.java index 9d5f6054243e4..ecd611859b578 100644 --- a/server/src/main/java/org/elasticsearch/index/engine/ReadOnlyEngine.java +++ b/server/src/main/java/org/elasticsearch/index/engine/ReadOnlyEngine.java @@ -18,6 +18,7 @@ */ package org.elasticsearch.index.engine; +import org.apache.lucene.codecs.blocktree.BlockTreeTermsReader; import org.apache.lucene.index.DirectoryReader; import org.apache.lucene.index.IndexCommit; import org.apache.lucene.index.IndexReader; @@ -41,13 +42,17 @@ import org.elasticsearch.index.shard.DocsStats; import org.elasticsearch.index.store.Store; import org.elasticsearch.index.translog.Translog; +import org.elasticsearch.index.translog.TranslogConfig; +import org.elasticsearch.index.translog.TranslogDeletionPolicy; import org.elasticsearch.index.translog.TranslogStats; import java.io.Closeable; import java.io.IOException; import java.io.UncheckedIOException; import java.util.Arrays; +import java.util.Collections; import java.util.List; +import java.util.Map; import java.util.concurrent.CountDownLatch; import java.util.function.BiFunction; import java.util.function.Function; @@ -62,15 +67,22 @@ */ public class ReadOnlyEngine extends Engine { + /** + * Reader attributes used for read only engines. These attributes prevent loading term dictionaries on-heap even if the field is an + * ID field if we are reading form memory maps. + */ + public static final Map OFF_HEAP_READER_ATTRIBUTES = Collections.singletonMap(BlockTreeTermsReader.FST_MODE_KEY, + BlockTreeTermsReader.FSTLoadMode.AUTO.name()); private final SegmentInfos lastCommittedSegmentInfos; private final SeqNoStats seqNoStats; - private final TranslogStats translogStats; private final SearcherManager searcherManager; private final IndexCommit indexCommit; private final Lock indexWriterLock; private final DocsStats docsStats; private final RamAccountingSearcherFactory searcherFactory; + protected volatile TranslogStats translogStats; + /** * Creates a new ReadOnlyEngine. This ctor can also be used to open a read-only engine on top of an already opened * read-write engine. It allows to optionally obtain the writer locks for the shard which would time-out if another @@ -99,7 +111,6 @@ public ReadOnlyEngine(EngineConfig config, SeqNoStats seqNoStats, TranslogStats // yet this makes sure nobody else does. including some testing tools that try to be messy indexWriterLock = obtainLock ? directory.obtainLock(IndexWriter.WRITE_LOCK_NAME) : null; this.lastCommittedSegmentInfos = Lucene.readSegmentInfos(directory); - this.translogStats = translogStats == null ? new TranslogStats(0, 0, 0, 0, 0) : translogStats; if (seqNoStats == null) { seqNoStats = buildSeqNoStats(config, lastCommittedSegmentInfos); ensureMaxSeqNoEqualsToGlobalCheckpoint(seqNoStats); @@ -110,6 +121,8 @@ public ReadOnlyEngine(EngineConfig config, SeqNoStats seqNoStats, TranslogStats reader = wrapReader(reader, readerWrapperFunction); searcherManager = new SearcherManager(reader, searcherFactory); this.docsStats = docsStats(lastCommittedSegmentInfos); + assert translogStats != null || obtainLock : "mutiple translogs instances should not be opened at the same time"; + this.translogStats = translogStats != null ? translogStats : translogStats(config, lastCommittedSegmentInfos); this.indexWriterLock = indexWriterLock; success = true; } finally { @@ -132,12 +145,12 @@ protected void ensureMaxSeqNoEqualsToGlobalCheckpoint(final SeqNoStats seqNoStat final Version indexVersionCreated = engineConfig.getIndexSettings().getIndexVersionCreated(); if (indexVersionCreated.onOrAfter(Version.V_7_2_0) || (seqNoStats.getGlobalCheckpoint() != SequenceNumbers.UNASSIGNED_SEQ_NO)) { + assert assertMaxSeqNoEqualsToGlobalCheckpoint(seqNoStats.getMaxSeqNo(), seqNoStats.getGlobalCheckpoint()); if (seqNoStats.getMaxSeqNo() != seqNoStats.getGlobalCheckpoint()) { throw new IllegalStateException("Maximum sequence number [" + seqNoStats.getMaxSeqNo() + "] from last commit does not match global checkpoint [" + seqNoStats.getGlobalCheckpoint() + "]"); } } - assert assertMaxSeqNoEqualsToGlobalCheckpoint(seqNoStats.getMaxSeqNo(), seqNoStats.getGlobalCheckpoint()); } protected boolean assertMaxSeqNoEqualsToGlobalCheckpoint(final long maxSeqNo, final long globalCheckpoint) { @@ -165,7 +178,7 @@ protected final DirectoryReader wrapReader(DirectoryReader reader, } protected DirectoryReader open(IndexCommit commit) throws IOException { - return DirectoryReader.open(commit); + return DirectoryReader.open(commit, OFF_HEAP_READER_ATTRIBUTES); } private DocsStats docsStats(final SegmentInfos lastCommittedSegmentInfos) { @@ -207,6 +220,26 @@ private static SeqNoStats buildSeqNoStats(EngineConfig config, SegmentInfos info return new SeqNoStats(maxSeqNo, localCheckpoint, config.getGlobalCheckpointSupplier().getAsLong()); } + private static TranslogStats translogStats(final EngineConfig config, final SegmentInfos infos) throws IOException { + final String translogUuid = infos.getUserData().get(Translog.TRANSLOG_UUID_KEY); + if (translogUuid == null) { + throw new IllegalStateException("commit doesn't contain translog unique id"); + } + final long translogGenOfLastCommit = Long.parseLong(infos.getUserData().get(Translog.TRANSLOG_GENERATION_KEY)); + final TranslogConfig translogConfig = config.getTranslogConfig(); + final TranslogDeletionPolicy translogDeletionPolicy = new TranslogDeletionPolicy( + config.getIndexSettings().getTranslogRetentionSize().getBytes(), + config.getIndexSettings().getTranslogRetentionAge().getMillis() + ); + translogDeletionPolicy.setTranslogGenerationOfLastCommit(translogGenOfLastCommit); + + try (Translog translog = new Translog(translogConfig, translogUuid, translogDeletionPolicy, config.getGlobalCheckpointSupplier(), + config.getPrimaryTermSupplier(), seqNo -> {}) + ) { + return translog.stats(); + } + } + @Override public GetResult get(Get get, BiFunction searcherFactory) throws EngineException { return getFromSearcher(get, searcherFactory, SearcherScope.EXTERNAL); @@ -320,7 +353,7 @@ public Translog.Location getTranslogLastWriteLocation() { } @Override - public long getLocalCheckpoint() { + public long getPersistedLocalCheckpoint() { return seqNoStats.getLocalCheckpoint(); } diff --git a/server/src/main/java/org/elasticsearch/index/fielddata/IndexFieldData.java b/server/src/main/java/org/elasticsearch/index/fielddata/IndexFieldData.java index bb3388bc943c9..11388190c3092 100644 --- a/server/src/main/java/org/elasticsearch/index/fielddata/IndexFieldData.java +++ b/server/src/main/java/org/elasticsearch/index/fielddata/IndexFieldData.java @@ -47,6 +47,7 @@ import org.elasticsearch.search.sort.NestedSortBuilder; import java.io.IOException; +import java.util.function.Function; /** * Thread-safe utility class that allows to get per-segment values via the @@ -114,11 +115,14 @@ public static class Nested { private final BitSetProducer rootFilter; private final Query innerQuery; private final NestedSortBuilder nestedSort; + private final Function searcherFactory; - public Nested(BitSetProducer rootFilter, Query innerQuery, NestedSortBuilder nestedSort) { + public Nested(BitSetProducer rootFilter, Query innerQuery, NestedSortBuilder nestedSort, + Function searcherFactory) { this.rootFilter = rootFilter; this.innerQuery = innerQuery; this.nestedSort = nestedSort; + this.searcherFactory = searcherFactory; } public Query getInnerQuery() { @@ -143,7 +147,7 @@ public BitSet rootDocs(LeafReaderContext ctx) throws IOException { */ public DocIdSetIterator innerDocs(LeafReaderContext ctx) throws IOException { final IndexReaderContext topLevelCtx = ReaderUtil.getTopLevelContext(ctx); - IndexSearcher indexSearcher = new IndexSearcher(topLevelCtx); + IndexSearcher indexSearcher = searcherFactory.apply(topLevelCtx); Weight weight = indexSearcher.createWeight(indexSearcher.rewrite(innerQuery), ScoreMode.COMPLETE_NO_SCORES, 1f); Scorer s = weight.scorer(ctx); return s == null ? null : s.iterator(); diff --git a/server/src/main/java/org/elasticsearch/index/fielddata/IndexOrdinalsFieldData.java b/server/src/main/java/org/elasticsearch/index/fielddata/IndexOrdinalsFieldData.java index 8a9fabc9e1354..9d4dbaef21125 100644 --- a/server/src/main/java/org/elasticsearch/index/fielddata/IndexOrdinalsFieldData.java +++ b/server/src/main/java/org/elasticsearch/index/fielddata/IndexOrdinalsFieldData.java @@ -47,4 +47,11 @@ public interface IndexOrdinalsFieldData extends IndexFieldData.Global> scriptFunction; - - protected GlobalOrdinalsIndexFieldData(IndexSettings indexSettings, String fieldName, AtomicOrdinalsFieldData[] segmentAfd, - OrdinalMap ordinalMap, long memorySizeInBytes, Function> scriptFunction) { + protected GlobalOrdinalsIndexFieldData(IndexSettings indexSettings, + String fieldName, + AtomicOrdinalsFieldData[] segmentAfd, + OrdinalMap ordinalMap, + long memorySizeInBytes, + Function> scriptFunction) { super(indexSettings); this.fieldName = fieldName; this.memorySizeInBytes = memorySizeInBytes; this.ordinalMap = ordinalMap; - this.atomicReaders = new Atomic[segmentAfd.length]; - for (int i = 0; i < segmentAfd.length; i++) { - atomicReaders[i] = new Atomic(segmentAfd[i], ordinalMap, i); - } + this.segmentAfd = segmentAfd; this.scriptFunction = scriptFunction; } + public IndexOrdinalsFieldData newConsumer(DirectoryReader source) { + return new Consumer(source, indexSettings); + } + @Override public AtomicOrdinalsFieldData loadDirect(LeafReaderContext context) throws Exception { - return load(context); + throw new IllegalStateException("loadDirect(LeafReaderContext) should not be called in this context"); } @Override @@ -92,9 +103,7 @@ public SortField sortField(@Nullable Object missingValue, MultiValueMode sortMod } @Override - public void clear() { - // no need to clear, because this is cached and cleared in AbstractBytesIndexFieldData - } + public void clear() {} @Override public long ramBytesUsed() { @@ -109,7 +118,7 @@ public Collection getChildResources() { @Override public AtomicOrdinalsFieldData load(LeafReaderContext context) { - return atomicReaders[context.ord]; + throw new IllegalStateException("load(LeafReaderContext) should not be called in this context"); } @Override @@ -117,45 +126,118 @@ public OrdinalMap getOrdinalMap() { return ordinalMap; } - private final class Atomic extends AbstractAtomicOrdinalsFieldData { + @Override + public boolean supportsGlobalOrdinalsMapping() { + return true; + } + + /** + * A non-thread safe {@link IndexOrdinalsFieldData} for global ordinals that creates the {@link TermsEnum} of each + * segment once and use them to provide a single lookup per segment. + */ + public class Consumer extends AbstractIndexComponent implements IndexOrdinalsFieldData, Accountable { + private final DirectoryReader source; + private TermsEnum[] lookups; + + Consumer(DirectoryReader source, IndexSettings settings) { + super(settings); + this.source = source; + } - private final AtomicOrdinalsFieldData afd; - private final OrdinalMap ordinalMap; - private final int segmentIndex; + /** + * Lazy creation of the {@link TermsEnum} for each segment present in this reader + */ + private TermsEnum[] getOrLoadTermsEnums() { + if (lookups == null) { + lookups = new TermsEnum[segmentAfd.length]; + for (int i = 0; i < lookups.length; i++) { + try { + lookups[i] = segmentAfd[i].getOrdinalsValues().termsEnum(); + } catch (IOException e) { + throw new UncheckedIOException("Failed to load terms enum", e); + } + } + } + return lookups; + } - private Atomic(AtomicOrdinalsFieldData afd, OrdinalMap ordinalMap, int segmentIndex) { - super(scriptFunction); - this.afd = afd; - this.ordinalMap = ordinalMap; - this.segmentIndex = segmentIndex; + @Override + public AtomicOrdinalsFieldData loadDirect(LeafReaderContext context) throws Exception { + return load(context); } @Override - public SortedSetDocValues getOrdinalsValues() { - final SortedSetDocValues values = afd.getOrdinalsValues(); - if (values.getValueCount() == ordinalMap.getValueCount()) { - // segment ordinals match global ordinals - return values; - } - final SortedSetDocValues[] bytesValues = new SortedSetDocValues[atomicReaders.length]; - for (int i = 0; i < bytesValues.length; i++) { - bytesValues[i] = atomicReaders[i].afd.getOrdinalsValues(); - } - return new GlobalOrdinalMapping(ordinalMap, bytesValues, segmentIndex); + public IndexOrdinalsFieldData loadGlobal(DirectoryReader indexReader) { + return this; } + @Override + public IndexOrdinalsFieldData localGlobalDirect(DirectoryReader indexReader) throws Exception { + return this; + } + + @Override + public String getFieldName() { + return fieldName; + } + + @Override + public SortField sortField(@Nullable Object missingValue, MultiValueMode sortMode, Nested nested, boolean reverse) { + throw new UnsupportedOperationException("no global ordinals sorting yet"); + } + + @Override + public void clear() {} + @Override public long ramBytesUsed() { - return afd.ramBytesUsed(); + return memorySizeInBytes; } @Override public Collection getChildResources() { - return afd.getChildResources(); + return Collections.emptyList(); + } + + @Override + public AtomicOrdinalsFieldData load(LeafReaderContext context) { + assert source.getReaderCacheHelper().getKey() == context.parent.reader().getReaderCacheHelper().getKey(); + return new AbstractAtomicOrdinalsFieldData(scriptFunction) { + @Override + public SortedSetDocValues getOrdinalsValues() { + final SortedSetDocValues values = segmentAfd[context.ord].getOrdinalsValues(); + if (values.getValueCount() == ordinalMap.getValueCount()) { + // segment ordinals match global ordinals + return values; + } + final TermsEnum[] atomicLookups = getOrLoadTermsEnums(); + return new GlobalOrdinalMapping(ordinalMap, values, atomicLookups, context.ord); + } + + @Override + public long ramBytesUsed() { + return segmentAfd[context.ord].ramBytesUsed(); + } + + + @Override + public Collection getChildResources() { + return segmentAfd[context.ord].getChildResources(); + } + + @Override + public void close() {} + }; + } + + @Override + public boolean supportsGlobalOrdinalsMapping() { + return true; } @Override - public void close() { + public OrdinalMap getOrdinalMap() { + return ordinalMap; } } diff --git a/server/src/main/java/org/elasticsearch/index/fielddata/plain/AbstractIndexOrdinalsFieldData.java b/server/src/main/java/org/elasticsearch/index/fielddata/plain/AbstractIndexOrdinalsFieldData.java index 0dc0de838a3e5..081875ae1e233 100644 --- a/server/src/main/java/org/elasticsearch/index/fielddata/plain/AbstractIndexOrdinalsFieldData.java +++ b/server/src/main/java/org/elasticsearch/index/fielddata/plain/AbstractIndexOrdinalsFieldData.java @@ -32,6 +32,7 @@ import org.elasticsearch.index.fielddata.IndexFieldDataCache; import org.elasticsearch.index.fielddata.IndexOrdinalsFieldData; import org.elasticsearch.index.fielddata.ordinals.GlobalOrdinalsBuilder; +import org.elasticsearch.index.fielddata.ordinals.GlobalOrdinalsIndexFieldData; import org.elasticsearch.indices.breaker.CircuitBreakerService; import java.io.IOException; @@ -60,6 +61,17 @@ public OrdinalMap getOrdinalMap() { @Override public IndexOrdinalsFieldData loadGlobal(DirectoryReader indexReader) { + IndexOrdinalsFieldData fieldData = loadGlobalInternal(indexReader); + if (fieldData instanceof GlobalOrdinalsIndexFieldData) { + // we create a new instance of the cached value for each consumer in order + // to avoid creating new TermsEnums for each segment in the cached instance + return ((GlobalOrdinalsIndexFieldData) fieldData).newConsumer(indexReader); + } else { + return fieldData; + } + } + + private IndexOrdinalsFieldData loadGlobalInternal(DirectoryReader indexReader) { if (indexReader.leaves().size() <= 1) { // ordinals are already global return this; @@ -126,6 +138,11 @@ protected TermsEnum filter(Terms terms, TermsEnum iterator, LeafReader reader) t return iterator; } + @Override + public boolean supportsGlobalOrdinalsMapping() { + return false; + } + private static final class FrequencyFilter extends FilteredTermsEnum { private int minFreq; diff --git a/server/src/main/java/org/elasticsearch/index/fielddata/plain/SortedSetDVOrdinalsIndexFieldData.java b/server/src/main/java/org/elasticsearch/index/fielddata/plain/SortedSetDVOrdinalsIndexFieldData.java index b71dcc7593417..4aaddc7b8dca6 100644 --- a/server/src/main/java/org/elasticsearch/index/fielddata/plain/SortedSetDVOrdinalsIndexFieldData.java +++ b/server/src/main/java/org/elasticsearch/index/fielddata/plain/SortedSetDVOrdinalsIndexFieldData.java @@ -37,6 +37,7 @@ import org.elasticsearch.index.fielddata.IndexOrdinalsFieldData; import org.elasticsearch.index.fielddata.ScriptDocValues; import org.elasticsearch.index.fielddata.fieldcomparator.BytesRefFieldComparatorSource; +import org.elasticsearch.index.fielddata.ordinals.GlobalOrdinalsIndexFieldData; import org.elasticsearch.index.fielddata.ordinals.GlobalOrdinalsBuilder; import org.elasticsearch.indices.breaker.CircuitBreakerService; import org.elasticsearch.search.MultiValueMode; @@ -92,6 +93,17 @@ public AtomicOrdinalsFieldData loadDirect(LeafReaderContext context) throws Exce @Override public IndexOrdinalsFieldData loadGlobal(DirectoryReader indexReader) { + IndexOrdinalsFieldData fieldData = loadGlobalInternal(indexReader); + if (fieldData instanceof GlobalOrdinalsIndexFieldData) { + // we create a new instance of the cached value for each consumer in order + // to avoid creating new TermsEnums for each segment in the cached instance + return ((GlobalOrdinalsIndexFieldData) fieldData).newConsumer(indexReader); + } else { + return fieldData; + } + } + + private IndexOrdinalsFieldData loadGlobalInternal(DirectoryReader indexReader) { if (indexReader.leaves().size() <= 1) { // ordinals are already global return this; @@ -134,4 +146,9 @@ public IndexOrdinalsFieldData localGlobalDirect(DirectoryReader indexReader) thr public OrdinalMap getOrdinalMap() { return null; } + + @Override + public boolean supportsGlobalOrdinalsMapping() { + return true; + } } diff --git a/server/src/main/java/org/elasticsearch/index/fieldvisitor/IdOnlyFieldVisitor.java b/server/src/main/java/org/elasticsearch/index/fieldvisitor/IdOnlyFieldVisitor.java new file mode 100644 index 0000000000000..13566b841c832 --- /dev/null +++ b/server/src/main/java/org/elasticsearch/index/fieldvisitor/IdOnlyFieldVisitor.java @@ -0,0 +1,60 @@ +/* + * 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.fieldvisitor; + +import org.apache.lucene.index.FieldInfo; +import org.apache.lucene.index.StoredFieldVisitor; +import org.elasticsearch.index.mapper.IdFieldMapper; +import org.elasticsearch.index.mapper.Uid; + +public final class IdOnlyFieldVisitor extends StoredFieldVisitor { + private String id = null; + private boolean visited = false; + + @Override + public Status needsField(FieldInfo fieldInfo) { + if (visited) { + return Status.STOP; + } + if (IdFieldMapper.NAME.equals(fieldInfo.name)) { + visited = true; + return Status.YES; + } else { + return Status.NO; + } + } + + @Override + public void binaryField(FieldInfo fieldInfo, byte[] value) { + assert IdFieldMapper.NAME.equals(fieldInfo.name) : fieldInfo; + if (IdFieldMapper.NAME.equals(fieldInfo.name)) { + id = Uid.decodeId(value); + } + } + + public String getId() { + return id; + } + + public void reset() { + id = null; + visited = false; + } +} diff --git a/server/src/main/java/org/elasticsearch/index/mapper/BaseGeoShapeFieldMapper.java b/server/src/main/java/org/elasticsearch/index/mapper/BaseGeoShapeFieldMapper.java index 74892bf7d516c..20151f301d791 100644 --- a/server/src/main/java/org/elasticsearch/index/mapper/BaseGeoShapeFieldMapper.java +++ b/server/src/main/java/org/elasticsearch/index/mapper/BaseGeoShapeFieldMapper.java @@ -190,7 +190,7 @@ public Mapper.Builder parse(String name, Map node, ParserContext } } final Builder builder; - if (parsedDeprecatedParams || parserContext.indexVersionCreated().before(Version.V_6_6_0)) { + if (parsedDeprecatedParams) { // Legacy index-based shape builder = new LegacyGeoShapeFieldMapper.Builder(name, deprecatedParameters); } else { diff --git a/server/src/main/java/org/elasticsearch/index/mapper/ContentPath.java b/server/src/main/java/org/elasticsearch/index/mapper/ContentPath.java index 3c67d3ee7f31c..7f3e26312ad87 100644 --- a/server/src/main/java/org/elasticsearch/index/mapper/ContentPath.java +++ b/server/src/main/java/org/elasticsearch/index/mapper/ContentPath.java @@ -66,4 +66,8 @@ public String pathAsText(String name) { sb.append(name); return sb.toString(); } + + public int length() { + return index; + } } diff --git a/server/src/main/java/org/elasticsearch/index/mapper/DocumentParser.java b/server/src/main/java/org/elasticsearch/index/mapper/DocumentParser.java index 9f5293e7da97a..07231759113e9 100644 --- a/server/src/main/java/org/elasticsearch/index/mapper/DocumentParser.java +++ b/server/src/main/java/org/elasticsearch/index/mapper/DocumentParser.java @@ -764,13 +764,17 @@ private static Mapper.Builder createBuilderFromDynamicValue(final ParseCont return builder; } else if (token == XContentParser.Token.VALUE_NUMBER) { XContentParser.NumberType numberType = context.parser().numberType(); - if (numberType == XContentParser.NumberType.INT || numberType == XContentParser.NumberType.LONG) { + if (numberType == XContentParser.NumberType.INT + || numberType == XContentParser.NumberType.LONG + || numberType == XContentParser.NumberType.BIG_INTEGER) { Mapper.Builder builder = context.root().findTemplateBuilder(context, currentFieldName, XContentFieldType.LONG); if (builder == null) { builder = newLongBuilder(currentFieldName); } return builder; - } else if (numberType == XContentParser.NumberType.FLOAT || numberType == XContentParser.NumberType.DOUBLE) { + } else if (numberType == XContentParser.NumberType.FLOAT + || numberType == XContentParser.NumberType.DOUBLE + || numberType == XContentParser.NumberType.BIG_DECIMAL) { Mapper.Builder builder = context.root().findTemplateBuilder(context, currentFieldName, XContentFieldType.DOUBLE); if (builder == null) { // no templates are defined, we use float by default instead of double diff --git a/server/src/main/java/org/elasticsearch/index/mapper/DynamicKeyFieldMapper.java b/server/src/main/java/org/elasticsearch/index/mapper/DynamicKeyFieldMapper.java new file mode 100644 index 0000000000000..0bbd383079c76 --- /dev/null +++ b/server/src/main/java/org/elasticsearch/index/mapper/DynamicKeyFieldMapper.java @@ -0,0 +1,54 @@ +/* + * 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.mapper; + +import org.elasticsearch.common.settings.Settings; + +/** + * A field mapper that supports lookup of dynamic sub-keys. If the field mapper is named 'my_field', + * then a user is able to search on the field in both of the following ways: + * - Using the field name 'my_field', which will delegate to the field type + * {@link DynamicKeyFieldMapper#fieldType()} as usual. + * - Using any sub-key, for example 'my_field.some_key'. In this case, the search is delegated + * to {@link DynamicKeyFieldMapper#keyedFieldType(String)}, with 'some_key' passed as the + * argument. The field mapper is allowed to create a new field type dynamically in order + * to handle the search. + * + * To prevent conflicts between these dynamic sub-keys and multi-fields, any field mappers + * implementing this interface should explicitly disallow multi-fields. The constructor makes + * sure to passes an empty multi-fields list to help prevent conflicting sub-keys from being + * registered. + * + * Note: we anticipate that 'flattened' fields will be the only implementation of this + * interface. Flattened object fields live in the 'mapper-flattened' module. + */ +public abstract class DynamicKeyFieldMapper extends FieldMapper { + + public DynamicKeyFieldMapper(String simpleName, + MappedFieldType fieldType, + MappedFieldType defaultFieldType, + Settings indexSettings, + CopyTo copyTo) { + super(simpleName, fieldType, defaultFieldType, indexSettings, MultiFields.empty(), copyTo); + } + + public abstract MappedFieldType keyedFieldType(String key); + +} diff --git a/server/src/main/java/org/elasticsearch/index/mapper/FieldMapper.java b/server/src/main/java/org/elasticsearch/index/mapper/FieldMapper.java index 29f22d8dc2cf0..23753b881f20c 100644 --- a/server/src/main/java/org/elasticsearch/index/mapper/FieldMapper.java +++ b/server/src/main/java/org/elasticsearch/index/mapper/FieldMapper.java @@ -33,6 +33,8 @@ import org.elasticsearch.common.settings.Setting.Property; import org.elasticsearch.common.settings.Settings; import org.elasticsearch.common.xcontent.XContentBuilder; +import org.elasticsearch.common.xcontent.XContentParser; +import org.elasticsearch.common.xcontent.support.AbstractXContentParser; import org.elasticsearch.index.analysis.NamedAnalyzer; import org.elasticsearch.index.mapper.FieldNamesFieldMapper.FieldNamesFieldType; import org.elasticsearch.index.similarity.SimilarityProvider; @@ -46,6 +48,7 @@ import java.util.Iterator; import java.util.List; import java.util.Map; +import java.util.HashMap; import java.util.Objects; import java.util.stream.StreamSupport; @@ -190,7 +193,7 @@ public Builder nullValue(Object nullValue) { return this; } - public T addMultiField(Mapper.Builder mapperBuilder) { + public T addMultiField(Mapper.Builder mapperBuilder) { multiFieldsBuilder.add(mapperBuilder); return builder; } @@ -276,14 +279,33 @@ public void parse(ParseContext context) throws IOException { context.doc().add(field); } } catch (Exception e) { - throw new MapperParsingException("failed to parse field [{}] of type [{}] in document with id '{}'", e, fieldType().name(), - fieldType().typeName(), context.sourceToParse().id()); + String valuePreview = ""; + try { + XContentParser parser = context.parser(); + Object complexValue = AbstractXContentParser.readValue(parser, ()-> new HashMap()); + if (complexValue == null) { + valuePreview = "null"; + } else { + valuePreview = complexValue.toString(); + } + } catch (Exception innerException) { + throw new MapperParsingException("failed to parse field [{}] of type [{}] in document with id '{}'. " + + "Could not parse field value preview,", + e, fieldType().name(), fieldType().typeName(), context.sourceToParse().id()); + } + + throw new MapperParsingException("failed to parse field [{}] of type [{}] in document with id '{}'. " + + "Preview of field's value: '{}'", e, fieldType().name(), fieldType().typeName(), + context.sourceToParse().id(), valuePreview); } multiFields.parse(this, context); } /** * Parse the field value and populate fields. + * + * Implementations of this method should ensure that on failing to parse parser.currentToken() must be the + * current failing token */ protected abstract void parseCreateField(ParseContext context, List fields) throws IOException; diff --git a/server/src/main/java/org/elasticsearch/index/mapper/FieldTypeLookup.java b/server/src/main/java/org/elasticsearch/index/mapper/FieldTypeLookup.java index 27d061d8c2788..7d8d5ef943de7 100644 --- a/server/src/main/java/org/elasticsearch/index/mapper/FieldTypeLookup.java +++ b/server/src/main/java/org/elasticsearch/index/mapper/FieldTypeLookup.java @@ -20,11 +20,13 @@ package org.elasticsearch.index.mapper; import org.elasticsearch.common.collect.CopyOnWriteHashMap; +import org.elasticsearch.common.collect.Iterators; import org.elasticsearch.common.regex.Regex; import java.util.Collection; import java.util.HashSet; import java.util.Iterator; +import java.util.Map; import java.util.Objects; import java.util.Set; @@ -36,15 +38,30 @@ class FieldTypeLookup implements Iterable { final CopyOnWriteHashMap fullNameToFieldType; private final CopyOnWriteHashMap aliasToConcreteName; + private final CopyOnWriteHashMap dynamicKeyMappers; + + /** + * The maximum field depth of any mapper that implements {@link DynamicKeyFieldMapper}. + * Allows us stop searching for a 'dynamic key' mapper as soon as we've passed the maximum + * possible field depth. + */ + private final int maxDynamicKeyDepth; + FieldTypeLookup() { fullNameToFieldType = new CopyOnWriteHashMap<>(); aliasToConcreteName = new CopyOnWriteHashMap<>(); + dynamicKeyMappers = new CopyOnWriteHashMap<>(); + maxDynamicKeyDepth = 0; } private FieldTypeLookup(CopyOnWriteHashMap fullNameToFieldType, - CopyOnWriteHashMap aliasToConcreteName) { + CopyOnWriteHashMap aliasToConcreteName, + CopyOnWriteHashMap dynamicKeyMappers, + int maxDynamicKeyDepth) { this.fullNameToFieldType = fullNameToFieldType; this.aliasToConcreteName = aliasToConcreteName; + this.dynamicKeyMappers = dynamicKeyMappers; + this.maxDynamicKeyDepth = maxDynamicKeyDepth; } /** @@ -63,14 +80,21 @@ public FieldTypeLookup copyAndAddAll(String type, CopyOnWriteHashMap fullName = this.fullNameToFieldType; CopyOnWriteHashMap aliases = this.aliasToConcreteName; + CopyOnWriteHashMap dynamicKeyMappers = this.dynamicKeyMappers; for (FieldMapper fieldMapper : fieldMappers) { + String fieldName = fieldMapper.name(); MappedFieldType fieldType = fieldMapper.fieldType(); MappedFieldType fullNameFieldType = fullName.get(fieldType.name()); if (Objects.equals(fieldType, fullNameFieldType) == false) { fullName = fullName.copyAndPut(fieldType.name(), fieldType); } + + if (fieldMapper instanceof DynamicKeyFieldMapper) { + DynamicKeyFieldMapper dynamicKeyMapper = (DynamicKeyFieldMapper) fieldMapper; + dynamicKeyMappers = dynamicKeyMappers.copyAndPut(fieldName, dynamicKeyMapper); + } } for (FieldAliasMapper fieldAliasMapper : fieldAliasMappers) { @@ -83,20 +107,97 @@ public FieldTypeLookup copyAndAddAll(String type, } } - return new FieldTypeLookup(fullName, aliases); + int maxDynamicKeyDepth = getMaxDynamicKeyDepth(aliases, dynamicKeyMappers); + + return new FieldTypeLookup(fullName, aliases, dynamicKeyMappers, maxDynamicKeyDepth); + } + + private static int getMaxDynamicKeyDepth(CopyOnWriteHashMap aliases, + CopyOnWriteHashMap dynamicKeyMappers) { + int maxFieldDepth = 0; + for (Map.Entry entry : aliases.entrySet()) { + String aliasName = entry.getKey(); + String path = entry.getValue(); + if (dynamicKeyMappers.containsKey(path)) { + maxFieldDepth = Math.max(maxFieldDepth, fieldDepth(aliasName)); + } + } + + for (String fieldName : dynamicKeyMappers.keySet()) { + if (dynamicKeyMappers.containsKey(fieldName)) { + maxFieldDepth = Math.max(maxFieldDepth, fieldDepth(fieldName)); + } + } + + return maxFieldDepth; } + /** + * Computes the total depth of this field by counting the number of parent fields + * in its path. As an example, the field 'parent1.parent2.field' has depth 3. + */ + private static int fieldDepth(String field) { + int numDots = 0; + int dotIndex = -1; + while (true) { + dotIndex = field.indexOf('.', dotIndex + 1); + if (dotIndex < 0) { + break; + } + numDots++; + } + return numDots + 1; + } - /** Returns the field for the given field */ + /** + * Returns the mapped field type for the given field name. + */ public MappedFieldType get(String field) { String concreteField = aliasToConcreteName.getOrDefault(field, field); - return fullNameToFieldType.get(concreteField); + MappedFieldType fieldType = fullNameToFieldType.get(concreteField); + if (fieldType != null) { + return fieldType; + } + + // If the mapping contains fields that support dynamic sub-key lookup, check + // if this could correspond to a keyed field of the form 'path_to_field.path_to_key'. + return !dynamicKeyMappers.isEmpty() ? getKeyedFieldType(field) : null; + } + + /** + * Check if the given field corresponds to a dynamic lookup mapper of the + * form 'path_to_field.path_to_key'. If so, returns a field type that + * can be used to perform searches on this field. + */ + private MappedFieldType getKeyedFieldType(String field) { + int dotIndex = -1; + int fieldDepth = 0; + + while (true) { + if (++fieldDepth > maxDynamicKeyDepth) { + return null; + } + + dotIndex = field.indexOf('.', dotIndex + 1); + if (dotIndex < 0) { + return null; + } + + String parentField = field.substring(0, dotIndex); + String concreteField = aliasToConcreteName.getOrDefault(parentField, parentField); + DynamicKeyFieldMapper mapper = dynamicKeyMappers.get(concreteField); + + if (mapper != null) { + String key = field.substring(dotIndex + 1); + return mapper.keyedFieldType(key); + } + } } /** * Returns a list of the full names of a simple match regex like pattern against full name and index name. */ - public Collection simpleMatchToFullName(String pattern) { + public Set simpleMatchToFullName(String pattern) { Set fields = new HashSet<>(); for (MappedFieldType fieldType : this) { if (Regex.simpleMatch(pattern, fieldType.name())) { @@ -113,6 +214,20 @@ public Collection simpleMatchToFullName(String pattern) { @Override public Iterator iterator() { - return fullNameToFieldType.values().iterator(); + Iterator concreteFieldTypes = fullNameToFieldType.values().iterator(); + + if (dynamicKeyMappers.isEmpty()) { + return concreteFieldTypes; + } else { + Iterator keyedFieldTypes = dynamicKeyMappers.values().stream() + .map(mapper -> mapper.keyedFieldType("")) + .iterator(); + return Iterators.concat(concreteFieldTypes, keyedFieldTypes); + } + } + + // Visible for testing. + int maxKeyedLookupDepth() { + return maxDynamicKeyDepth; } } diff --git a/server/src/main/java/org/elasticsearch/index/mapper/LegacyGeoShapeFieldMapper.java b/server/src/main/java/org/elasticsearch/index/mapper/LegacyGeoShapeFieldMapper.java index 9c8726a498a3c..c4996eab901dd 100644 --- a/server/src/main/java/org/elasticsearch/index/mapper/LegacyGeoShapeFieldMapper.java +++ b/server/src/main/java/org/elasticsearch/index/mapper/LegacyGeoShapeFieldMapper.java @@ -208,8 +208,6 @@ private void setupFieldTypeDeprecatedParameters(BuilderContext context) { } if (deprecatedParameters.tree != null) { ft.setTree(deprecatedParameters.tree); - } else if (context.indexCreatedVersion().before(Version.V_6_6_0)) { - ft.setTree(DeprecatedParameters.PrefixTrees.GEOHASH); } if (deprecatedParameters.treeLevels != null) { ft.setTreeLevels(deprecatedParameters.treeLevels); @@ -527,8 +525,7 @@ protected void doXContentBody(XContentBuilder builder, boolean includeDefaults, super.doXContentBody(builder, includeDefaults, params); if (includeDefaults - || (fieldType().tree().equals(indexCreatedVersion.onOrAfter(Version.V_6_6_0) ? - DeprecatedParameters.Defaults.TREE : DeprecatedParameters.PrefixTrees.GEOHASH)) == false) { + || (fieldType().tree().equals(DeprecatedParameters.Defaults.TREE)) == false) { builder.field(DeprecatedParameters.Names.TREE.getPreferredName(), fieldType().tree()); } diff --git a/server/src/main/java/org/elasticsearch/index/mapper/MappedFieldType.java b/server/src/main/java/org/elasticsearch/index/mapper/MappedFieldType.java index 5ef689709400d..411045abaf796 100644 --- a/server/src/main/java/org/elasticsearch/index/mapper/MappedFieldType.java +++ b/server/src/main/java/org/elasticsearch/index/mapper/MappedFieldType.java @@ -390,7 +390,8 @@ public SpanQuery spanPrefixQuery(String value, SpanMultiTermQueryWrapper.SpanRew /** * Create an {@link IntervalsSource} to be used for proximity queries */ - public IntervalsSource intervals(String query, int max_gaps, boolean ordered, NamedAnalyzer analyzer) throws IOException { + public IntervalsSource intervals(String query, int max_gaps, boolean ordered, + NamedAnalyzer analyzer, boolean prefix) throws IOException { throw new IllegalArgumentException("Can only use interval queries on text fields - not on [" + name + "] which is of type [" + typeName() + "]"); } diff --git a/server/src/main/java/org/elasticsearch/index/mapper/MapperService.java b/server/src/main/java/org/elasticsearch/index/mapper/MapperService.java index 487a6ac4789e3..6002348050700 100644 --- a/server/src/main/java/org/elasticsearch/index/mapper/MapperService.java +++ b/server/src/main/java/org/elasticsearch/index/mapper/MapperService.java @@ -37,6 +37,7 @@ import org.elasticsearch.common.regex.Regex; import org.elasticsearch.common.settings.Setting; import org.elasticsearch.common.settings.Setting.Property; +import org.elasticsearch.common.settings.Settings; import org.elasticsearch.common.xcontent.LoggingDeprecationHandler; import org.elasticsearch.common.xcontent.NamedXContentRegistry; import org.elasticsearch.common.xcontent.XContentFactory; @@ -46,8 +47,13 @@ import org.elasticsearch.index.AbstractIndexComponent; import org.elasticsearch.index.IndexSettings; import org.elasticsearch.index.IndexSortConfig; +import org.elasticsearch.index.analysis.AnalysisRegistry; +import org.elasticsearch.index.analysis.CharFilterFactory; import org.elasticsearch.index.analysis.IndexAnalyzers; import org.elasticsearch.index.analysis.NamedAnalyzer; +import org.elasticsearch.index.analysis.ReloadableCustomAnalyzer; +import org.elasticsearch.index.analysis.TokenFilterFactory; +import org.elasticsearch.index.analysis.TokenizerFactory; import org.elasticsearch.index.mapper.Mapper.BuilderContext; import org.elasticsearch.index.query.QueryShardContext; import org.elasticsearch.index.similarity.SimilarityService; @@ -260,9 +266,7 @@ private void assertMappingVersion( final IndexMetaData currentIndexMetaData, final IndexMetaData newIndexMetaData, final Map updatedEntries) { - if (Assertions.ENABLED - && currentIndexMetaData != null - && currentIndexMetaData.getCreationVersion().onOrAfter(Version.V_6_5_0)) { + if (Assertions.ENABLED && currentIndexMetaData != null) { if (currentIndexMetaData.getMappingVersion() == newIndexMetaData.getMappingVersion()) { // if the mapping version is unchanged, then there should not be any updates and all mappings should be the same assert updatedEntries.isEmpty() : updatedEntries; @@ -744,10 +748,10 @@ public MappedFieldType fullName(String fullName) { * Returns all the fields that match the given pattern. If the pattern is prefixed with a type * then the fields will be returned with a type prefix. */ - public Collection simpleMatchToFullName(String pattern) { + public Set simpleMatchToFullName(String pattern) { if (Regex.isSimpleMatchPattern(pattern) == false) { // no wildcards - return Collections.singletonList(pattern); + return Collections.singleton(pattern); } return fieldTypes.simpleMatchToFullName(pattern); } @@ -843,4 +847,24 @@ protected Analyzer getWrappedAnalyzer(String fieldName) { return defaultAnalyzer; } } + + public synchronized List reloadSearchAnalyzers(AnalysisRegistry registry) throws IOException { + logger.info("reloading search analyzers"); + // refresh indexAnalyzers and search analyzers + final Map tokenizerFactories = registry.buildTokenizerFactories(indexSettings); + final Map charFilterFactories = registry.buildCharFilterFactories(indexSettings); + final Map tokenFilterFactories = registry.buildTokenFilterFactories(indexSettings); + final Map settings = indexSettings.getSettings().getGroups("index.analysis.analyzer"); + final List reloadedAnalyzers = new ArrayList<>(); + for (NamedAnalyzer namedAnalyzer : indexAnalyzers.getAnalyzers().values()) { + if (namedAnalyzer.analyzer() instanceof ReloadableCustomAnalyzer) { + ReloadableCustomAnalyzer analyzer = (ReloadableCustomAnalyzer) namedAnalyzer.analyzer(); + String analyzerName = namedAnalyzer.name(); + Settings analyzerSettings = settings.get(analyzerName); + analyzer.reload(analyzerName, analyzerSettings, tokenizerFactories, charFilterFactories, tokenFilterFactories); + reloadedAnalyzers.add(analyzerName); + } + } + return reloadedAnalyzers; + } } diff --git a/server/src/main/java/org/elasticsearch/index/mapper/ParseContext.java b/server/src/main/java/org/elasticsearch/index/mapper/ParseContext.java index 4cfd5be2afed3..b2c536a900e33 100644 --- a/server/src/main/java/org/elasticsearch/index/mapper/ParseContext.java +++ b/server/src/main/java/org/elasticsearch/index/mapper/ParseContext.java @@ -24,7 +24,6 @@ import org.apache.lucene.document.Field; import org.apache.lucene.index.IndexableField; import org.apache.lucene.util.BytesRef; -import org.elasticsearch.Version; import org.elasticsearch.common.xcontent.XContentParser; import org.elasticsearch.index.IndexSettings; @@ -458,18 +457,10 @@ public Iterable nonRootDocuments() { void postParse() { if (documents.size() > 1) { docsReversed = true; - if (indexSettings.getIndexVersionCreated().onOrAfter(Version.V_6_5_0)) { - /** - * For indices created on or after {@link Version#V_6_5_0} we preserve the order - * of the children while ensuring that parents appear after them. - */ - List newDocs = reorderParent(documents); - documents.clear(); - documents.addAll(newDocs); - } else { - // reverse the order of docs for nested docs support, parent should be last - Collections.reverse(documents); - } + // We preserve the order of the children while ensuring that parents appear after them. + List newDocs = reorderParent(documents); + documents.clear(); + documents.addAll(newDocs); } } diff --git a/server/src/main/java/org/elasticsearch/index/mapper/TextFieldMapper.java b/server/src/main/java/org/elasticsearch/index/mapper/TextFieldMapper.java index 8a3203ad8e7e0..d1ae07a7f0b9d 100644 --- a/server/src/main/java/org/elasticsearch/index/mapper/TextFieldMapper.java +++ b/server/src/main/java/org/elasticsearch/index/mapper/TextFieldMapper.java @@ -44,6 +44,7 @@ import org.apache.lucene.search.Query; import org.apache.lucene.search.SynonymQuery; import org.apache.lucene.search.TermQuery; +import org.apache.lucene.search.intervals.Intervals; import org.apache.lucene.search.intervals.IntervalsSource; import org.apache.lucene.search.spans.FieldMaskingSpanQuery; import org.apache.lucene.search.spans.SpanMultiTermQueryWrapper; @@ -51,6 +52,7 @@ import org.apache.lucene.search.spans.SpanOrQuery; import org.apache.lucene.search.spans.SpanQuery; import org.apache.lucene.search.spans.SpanTermQuery; +import org.apache.lucene.util.BytesRef; import org.apache.lucene.util.automaton.Automata; import org.apache.lucene.util.automaton.Automaton; import org.apache.lucene.util.automaton.Operations; @@ -109,7 +111,8 @@ public static class Defaults { public static class Builder extends FieldMapper.Builder { private int positionIncrementGap = POSITION_INCREMENT_GAP_USE_ANALYZER; - private PrefixFieldType prefixFieldType; + private int minPrefixChars = -1; + private int maxPrefixChars = -1; public Builder(String name) { super(name, Defaults.FIELD_TYPE, Defaults.FIELD_TYPE); @@ -160,6 +163,7 @@ public Builder fielddataFrequencyFilter(double minFreq, double maxFreq, int minS } public Builder indexPrefixes(int minChars, int maxChars) { + if (minChars > maxChars) { throw new IllegalArgumentException("min_chars [" + minChars + "] must be less than max_chars [" + maxChars + "]"); } @@ -169,8 +173,8 @@ public Builder indexPrefixes(int minChars, int maxChars) { if (maxChars >= 20) { throw new IllegalArgumentException("max_chars [" + maxChars + "] must be less than 20"); } - this.prefixFieldType = new PrefixFieldType(name(), name() + "._index_prefix", minChars, maxChars); - fieldType().setPrefixFieldType(this.prefixFieldType); + this.minPrefixChars = minChars; + this.maxPrefixChars = maxChars; return this; } @@ -187,21 +191,28 @@ public TextFieldMapper build(BuilderContext context) { } setupFieldType(context); PrefixFieldMapper prefixMapper = null; - if (prefixFieldType != null) { + if (minPrefixChars != -1) { + /** + * Mappings before v7.2.1 use {@link Builder#name} instead of {@link Builder#fullName} + * to build prefix field names so we preserve the name that was used at creation time + * even if it is different from the expected one (in case the field is nested under an object + * or a multi-field). This way search will continue to work on old indices and new indices + * will use the expected full name. + **/ + String fullName = context.indexCreatedVersion().before(Version.V_7_2_1) ? name() : buildFullName(context); + PrefixFieldType prefixFieldType = + new PrefixFieldType(fullName, fullName + "._index_prefix", minPrefixChars, maxPrefixChars); + fieldType().setPrefixFieldType(prefixFieldType); if (fieldType().isSearchable() == false) { throw new IllegalArgumentException("Cannot set index_prefixes on unindexed field [" + name() + "]"); } // Copy the index options of the main field to allow phrase queries on // the prefix field. - if (context.indexCreatedVersion().onOrAfter(Version.V_6_4_0)) { - if (fieldType.indexOptions() == IndexOptions.DOCS_AND_FREQS) { - // frequencies are not needed because prefix queries always use a constant score - prefixFieldType.setIndexOptions(IndexOptions.DOCS); - } else { - prefixFieldType.setIndexOptions(fieldType.indexOptions()); - } - } else if (fieldType.indexOptions() == IndexOptions.DOCS_AND_FREQS_AND_POSITIONS_AND_OFFSETS) { - prefixFieldType.setIndexOptions(IndexOptions.DOCS_AND_FREQS_AND_POSITIONS_AND_OFFSETS); + if (fieldType.indexOptions() == IndexOptions.DOCS_AND_FREQS) { + // frequencies are not needed because prefix queries always use a constant score + prefixFieldType.setIndexOptions(IndexOptions.DOCS); + } else { + prefixFieldType.setIndexOptions(fieldType.indexOptions()); } if (fieldType.storeTermVectorOffsets()) { prefixFieldType.setStoreTermVectorOffsets(true); @@ -408,6 +419,17 @@ public Query prefixQuery(String value, MultiTermQuery.RewriteMethod method, Quer .build(); } + public IntervalsSource intervals(BytesRef term) { + if (term.length > maxChars) { + return Intervals.prefix(term.utf8ToString()); + } + if (term.length >= minChars) { + return Intervals.fixField(name(), Intervals.term(term)); + } + String wildcardTerm = term.utf8ToString() + "?".repeat(Math.max(0, minChars - term.length)); + return Intervals.or(Intervals.fixField(name(), Intervals.wildcard(wildcardTerm)), Intervals.term(term)); + } + @Override public PrefixFieldType clone() { return new PrefixFieldType(parentField, name(), minChars, maxChars); @@ -636,10 +658,21 @@ public Query existsQuery(QueryShardContext context) { } @Override - public IntervalsSource intervals(String text, int maxGaps, boolean ordered, NamedAnalyzer analyzer) throws IOException { + public IntervalsSource intervals(String text, int maxGaps, boolean ordered, + NamedAnalyzer analyzer, boolean prefix) throws IOException { if (indexOptions().compareTo(IndexOptions.DOCS_AND_FREQS_AND_POSITIONS) < 0) { throw new IllegalArgumentException("Cannot create intervals over field [" + name() + "] with no positions indexed"); } + if (analyzer == null) { + analyzer = searchAnalyzer(); + } + if (prefix) { + BytesRef normalizedTerm = analyzer.normalize(name(), text); + if (prefixFieldType != null) { + return prefixFieldType.intervals(normalizedTerm); + } + return Intervals.prefix(normalizedTerm.utf8ToString()); // TODO make Intervals.prefix() take a BytesRef + } IntervalBuilder builder = new IntervalBuilder(name(), analyzer == null ? searchAnalyzer() : analyzer); return builder.analyzeText(text, maxGaps, ordered); } diff --git a/server/src/main/java/org/elasticsearch/index/mapper/TypeParsers.java b/server/src/main/java/org/elasticsearch/index/mapper/TypeParsers.java index 9848a23cac11b..12c80361a855c 100644 --- a/server/src/main/java/org/elasticsearch/index/mapper/TypeParsers.java +++ b/server/src/main/java/org/elasticsearch/index/mapper/TypeParsers.java @@ -22,6 +22,7 @@ import org.apache.logging.log4j.LogManager; import org.apache.lucene.index.IndexOptions; import org.elasticsearch.ElasticsearchParseException; +import org.elasticsearch.Version; import org.elasticsearch.common.logging.DeprecationLogger; import org.elasticsearch.common.time.DateFormatter; import org.elasticsearch.common.xcontent.support.XContentMapValues; @@ -219,11 +220,18 @@ public static boolean parseMultiField(FieldMapper.Builder builder, String name, String propName, Object propNode) { if (propName.equals("fields")) { if (parserContext.isWithinMultiField()) { - deprecationLogger.deprecatedAndMaybeLog("multifield_within_multifield", "At least one multi-field, [" + name + "], was " + - "encountered that itself contains a multi-field. Defining multi-fields within a multi-field is deprecated and will " + - "no longer be supported in 8.0. To resolve the issue, all instances of [fields] that occur within a [fields] block " + - "should be removed from the mappings, either by flattening the chained [fields] blocks into a single level, or " + - "switching to [copy_to] if appropriate."); + // For indices created prior to 8.0, we only emit a deprecation warning and do not fail type parsing. This is to + // maintain the backwards-compatibility guarantee that we can always load indexes from the previous major version. + if (parserContext.indexVersionCreated().before(Version.V_8_0_0)) { + deprecationLogger.deprecatedAndMaybeLog("multifield_within_multifield", "At least one multi-field, [" + name + "], " + + "was encountered that itself contains a multi-field. Defining multi-fields within a multi-field is deprecated " + + "and is not supported for indices created in 8.0 and later. To migrate the mappings, all instances of [fields] " + + "that occur within a [fields] block should be removed from the mappings, either by flattening the chained " + + "[fields] blocks into a single level, or switching to [copy_to] if appropriate."); + } else { + throw new IllegalArgumentException("Encountered a multi-field [" + name + "] which itself contains a multi-field. " + + "Defining chained multi-fields is not supported."); + } } parserContext = parserContext.createMultiFieldContext(parserContext); diff --git a/server/src/main/java/org/elasticsearch/index/query/CommonTermsQueryBuilder.java b/server/src/main/java/org/elasticsearch/index/query/CommonTermsQueryBuilder.java deleted file mode 100644 index d646dc4bb4b07..0000000000000 --- a/server/src/main/java/org/elasticsearch/index/query/CommonTermsQueryBuilder.java +++ /dev/null @@ -1,406 +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.query; - -import org.apache.lucene.analysis.Analyzer; -import org.apache.lucene.analysis.TokenStream; -import org.apache.lucene.analysis.tokenattributes.CharTermAttribute; -import org.apache.lucene.index.Term; -import org.apache.lucene.queries.ExtendedCommonTermsQuery; -import org.apache.lucene.search.BooleanClause.Occur; -import org.apache.lucene.search.Query; -import org.apache.lucene.util.BytesRefBuilder; -import org.elasticsearch.common.ParseField; -import org.elasticsearch.common.ParsingException; -import org.elasticsearch.common.Strings; -import org.elasticsearch.common.io.stream.StreamInput; -import org.elasticsearch.common.io.stream.StreamOutput; -import org.elasticsearch.common.xcontent.XContentBuilder; -import org.elasticsearch.common.xcontent.XContentParser; -import org.elasticsearch.index.mapper.MappedFieldType; - -import java.io.IOException; -import java.util.Objects; - -/** - * CommonTermsQuery query is a query that executes high-frequency terms in a - * optional sub-query to prevent slow queries due to "common" terms like - * stopwords. This query basically builds 2 queries off the {@code #add(Term) - * added} terms where low-frequency terms are added to a required boolean clause - * and high-frequency terms are added to an optional boolean clause. The - * optional clause is only executed if the required "low-frequency' clause - * matches. - */ -public class CommonTermsQueryBuilder extends AbstractQueryBuilder { - - public static final String NAME = "common"; - - public static final float DEFAULT_CUTOFF_FREQ = 0.01f; - public static final Operator DEFAULT_HIGH_FREQ_OCCUR = Operator.OR; - public static final Operator DEFAULT_LOW_FREQ_OCCUR = Operator.OR; - - private static final ParseField CUTOFF_FREQUENCY_FIELD = new ParseField("cutoff_frequency"); - private static final ParseField MINIMUM_SHOULD_MATCH_FIELD = new ParseField("minimum_should_match"); - private static final ParseField LOW_FREQ_OPERATOR_FIELD = new ParseField("low_freq_operator"); - private static final ParseField HIGH_FREQ_OPERATOR_FIELD = new ParseField("high_freq_operator"); - private static final ParseField DISABLE_COORD_FIELD = new ParseField("disable_coord") - .withAllDeprecated("disable_coord has been removed"); - private static final ParseField ANALYZER_FIELD = new ParseField("analyzer"); - private static final ParseField QUERY_FIELD = new ParseField("query"); - private static final ParseField HIGH_FREQ_FIELD = new ParseField("high_freq"); - private static final ParseField LOW_FREQ_FIELD = new ParseField("low_freq"); - - private final String fieldName; - - private final Object text; - - private Operator highFreqOperator = DEFAULT_HIGH_FREQ_OCCUR; - - private Operator lowFreqOperator = DEFAULT_LOW_FREQ_OCCUR; - - private String analyzer = null; - - private String lowFreqMinimumShouldMatch = null; - - private String highFreqMinimumShouldMatch = null; - - private float cutoffFrequency = DEFAULT_CUTOFF_FREQ; - - /** - * Constructs a new common terms query. - */ - public CommonTermsQueryBuilder(String fieldName, Object text) { - if (Strings.isEmpty(fieldName)) { - throw new IllegalArgumentException("field name is null or empty"); - } - if (text == null) { - throw new IllegalArgumentException("text cannot be null"); - } - this.fieldName = fieldName; - this.text = text; - } - - /** - * Read from a stream. - */ - public CommonTermsQueryBuilder(StreamInput in) throws IOException { - super(in); - fieldName = in.readString(); - text = in.readGenericValue(); - highFreqOperator = Operator.readFromStream(in); - lowFreqOperator = Operator.readFromStream(in); - analyzer = in.readOptionalString(); - lowFreqMinimumShouldMatch = in.readOptionalString(); - highFreqMinimumShouldMatch = in.readOptionalString(); - cutoffFrequency = in.readFloat(); - } - - @Override - protected void doWriteTo(StreamOutput out) throws IOException { - out.writeString(this.fieldName); - out.writeGenericValue(this.text); - highFreqOperator.writeTo(out); - lowFreqOperator.writeTo(out); - out.writeOptionalString(analyzer); - out.writeOptionalString(lowFreqMinimumShouldMatch); - out.writeOptionalString(highFreqMinimumShouldMatch); - out.writeFloat(cutoffFrequency); - } - - public String fieldName() { - return this.fieldName; - } - - public Object value() { - return this.text; - } - - /** - * Sets the operator to use for terms with a high document frequency - * (greater than or equal to {@link #cutoffFrequency(float)}. Defaults to - * {@code AND}. - */ - public CommonTermsQueryBuilder highFreqOperator(Operator operator) { - this.highFreqOperator = (operator == null) ? DEFAULT_HIGH_FREQ_OCCUR : operator; - return this; - } - - public Operator highFreqOperator() { - return highFreqOperator; - } - - /** - * Sets the operator to use for terms with a low document frequency (less - * than {@link #cutoffFrequency(float)}. Defaults to {@code AND}. - */ - public CommonTermsQueryBuilder lowFreqOperator(Operator operator) { - this.lowFreqOperator = (operator == null) ? DEFAULT_LOW_FREQ_OCCUR : operator; - return this; - } - - public Operator lowFreqOperator() { - return lowFreqOperator; - } - - /** - * Explicitly set the analyzer to use. Defaults to use explicit mapping - * config for the field, or, if not set, the default search analyzer. - */ - public CommonTermsQueryBuilder analyzer(String analyzer) { - this.analyzer = analyzer; - return this; - } - - public String analyzer() { - return this.analyzer; - } - - /** - * Sets the cutoff document frequency for high / low frequent terms. A value - * in [0..1] (or absolute number >=1) representing the maximum threshold of - * a terms document frequency to be considered a low frequency term. - * Defaults to - * {@code {@value #DEFAULT_CUTOFF_FREQ}} - */ - public CommonTermsQueryBuilder cutoffFrequency(float cutoffFrequency) { - this.cutoffFrequency = cutoffFrequency; - return this; - } - - public float cutoffFrequency() { - return this.cutoffFrequency; - } - - /** - * Sets the minimum number of high frequent query terms that need to match in order to - * produce a hit when there are no low frequent terms. - */ - public CommonTermsQueryBuilder highFreqMinimumShouldMatch(String highFreqMinimumShouldMatch) { - this.highFreqMinimumShouldMatch = highFreqMinimumShouldMatch; - return this; - } - - public String highFreqMinimumShouldMatch() { - return this.highFreqMinimumShouldMatch; - } - - /** - * Sets the minimum number of low frequent query terms that need to match in order to - * produce a hit. - */ - public CommonTermsQueryBuilder lowFreqMinimumShouldMatch(String lowFreqMinimumShouldMatch) { - this.lowFreqMinimumShouldMatch = lowFreqMinimumShouldMatch; - return this; - } - - public String lowFreqMinimumShouldMatch() { - return this.lowFreqMinimumShouldMatch; - } - - @Override - protected void doXContent(XContentBuilder builder, Params params) throws IOException { - builder.startObject(NAME); - builder.startObject(fieldName); - builder.field(QUERY_FIELD.getPreferredName(), text); - builder.field(HIGH_FREQ_OPERATOR_FIELD.getPreferredName(), highFreqOperator.toString()); - builder.field(LOW_FREQ_OPERATOR_FIELD.getPreferredName(), lowFreqOperator.toString()); - if (analyzer != null) { - builder.field(ANALYZER_FIELD.getPreferredName(), analyzer); - } - builder.field(CUTOFF_FREQUENCY_FIELD.getPreferredName(), cutoffFrequency); - if (lowFreqMinimumShouldMatch != null || highFreqMinimumShouldMatch != null) { - builder.startObject(MINIMUM_SHOULD_MATCH_FIELD.getPreferredName()); - if (lowFreqMinimumShouldMatch != null) { - builder.field(LOW_FREQ_FIELD.getPreferredName(), lowFreqMinimumShouldMatch); - } - if (highFreqMinimumShouldMatch != null) { - builder.field(HIGH_FREQ_FIELD.getPreferredName(), highFreqMinimumShouldMatch); - } - builder.endObject(); - } - printBoostAndQueryName(builder); - builder.endObject(); - builder.endObject(); - } - - public static CommonTermsQueryBuilder fromXContent(XContentParser parser) throws IOException { - String fieldName = null; - Object text = null; - float boost = AbstractQueryBuilder.DEFAULT_BOOST; - String analyzer = null; - String lowFreqMinimumShouldMatch = null; - String highFreqMinimumShouldMatch = null; - Operator highFreqOperator = CommonTermsQueryBuilder.DEFAULT_HIGH_FREQ_OCCUR; - Operator lowFreqOperator = CommonTermsQueryBuilder.DEFAULT_LOW_FREQ_OCCUR; - float cutoffFrequency = CommonTermsQueryBuilder.DEFAULT_CUTOFF_FREQ; - String queryName = null; - XContentParser.Token token; - String currentFieldName = null; - while ((token = parser.nextToken()) != XContentParser.Token.END_OBJECT) { - if (token == XContentParser.Token.FIELD_NAME) { - currentFieldName = parser.currentName(); - } else if (token == XContentParser.Token.START_OBJECT) { - throwParsingExceptionOnMultipleFields(NAME, parser.getTokenLocation(), fieldName, currentFieldName); - fieldName = currentFieldName; - while ((token = parser.nextToken()) != XContentParser.Token.END_OBJECT) { - if (token == XContentParser.Token.FIELD_NAME) { - currentFieldName = parser.currentName(); - } else if (token == XContentParser.Token.START_OBJECT) { - if (MINIMUM_SHOULD_MATCH_FIELD.match(currentFieldName, parser.getDeprecationHandler())) { - String innerFieldName = null; - while ((token = parser.nextToken()) != XContentParser.Token.END_OBJECT) { - if (token == XContentParser.Token.FIELD_NAME) { - innerFieldName = parser.currentName(); - } else if (token.isValue()) { - if (LOW_FREQ_FIELD.match(innerFieldName, parser.getDeprecationHandler())) { - lowFreqMinimumShouldMatch = parser.text(); - } else if (HIGH_FREQ_FIELD.match(innerFieldName, parser.getDeprecationHandler())) { - highFreqMinimumShouldMatch = parser.text(); - } else { - throw new ParsingException(parser.getTokenLocation(), "[" + CommonTermsQueryBuilder.NAME + - "] query does not support [" + innerFieldName - + "] for [" + currentFieldName + "]"); - } - } else { - throw new ParsingException(parser.getTokenLocation(), "[" + CommonTermsQueryBuilder.NAME + - "] unexpected token type [" + token - + "] after [" + innerFieldName + "]"); - } - } - } else { - throw new ParsingException(parser.getTokenLocation(), "[" + CommonTermsQueryBuilder.NAME + - "] query does not support [" + currentFieldName + "]"); - } - } else if (token.isValue()) { - if (QUERY_FIELD.match(currentFieldName, parser.getDeprecationHandler())) { - text = parser.objectText(); - } else if (ANALYZER_FIELD.match(currentFieldName, parser.getDeprecationHandler())) { - analyzer = parser.text(); - } else if (DISABLE_COORD_FIELD.match(currentFieldName, parser.getDeprecationHandler())) { - // ignore - } else if (AbstractQueryBuilder.BOOST_FIELD.match(currentFieldName, parser.getDeprecationHandler())) { - boost = parser.floatValue(); - } else if (HIGH_FREQ_OPERATOR_FIELD.match(currentFieldName, parser.getDeprecationHandler())) { - highFreqOperator = Operator.fromString(parser.text()); - } else if (LOW_FREQ_OPERATOR_FIELD.match(currentFieldName, parser.getDeprecationHandler())) { - lowFreqOperator = Operator.fromString(parser.text()); - } else if (MINIMUM_SHOULD_MATCH_FIELD.match(currentFieldName, parser.getDeprecationHandler())) { - lowFreqMinimumShouldMatch = parser.text(); - } else if (CUTOFF_FREQUENCY_FIELD.match(currentFieldName, parser.getDeprecationHandler())) { - cutoffFrequency = parser.floatValue(); - } else if (AbstractQueryBuilder.NAME_FIELD.match(currentFieldName, parser.getDeprecationHandler())) { - queryName = parser.text(); - } else { - throw new ParsingException(parser.getTokenLocation(), "[" + CommonTermsQueryBuilder.NAME + - "] query does not support [" + currentFieldName + "]"); - } - } - } - } else { - throwParsingExceptionOnMultipleFields(NAME, parser.getTokenLocation(), fieldName, parser.currentName()); - fieldName = parser.currentName(); - text = parser.objectText(); - } - } - - return new CommonTermsQueryBuilder(fieldName, text) - .lowFreqMinimumShouldMatch(lowFreqMinimumShouldMatch) - .highFreqMinimumShouldMatch(highFreqMinimumShouldMatch) - .analyzer(analyzer) - .highFreqOperator(highFreqOperator) - .lowFreqOperator(lowFreqOperator) - .cutoffFrequency(cutoffFrequency) - .boost(boost) - .queryName(queryName); - } - - @Override - public String getWriteableName() { - return NAME; - } - - @Override - protected Query doToQuery(QueryShardContext context) throws IOException { - String field; - MappedFieldType fieldType = context.fieldMapper(fieldName); - if (fieldType != null) { - field = fieldType.name(); - } else { - field = fieldName; - } - - Analyzer analyzerObj; - if (analyzer == null) { - if (fieldType != null) { - analyzerObj = context.getSearchAnalyzer(fieldType); - } else { - analyzerObj = context.getMapperService().searchAnalyzer(); - } - } else { - analyzerObj = context.getMapperService().getIndexAnalyzers().get(analyzer); - if (analyzerObj == null) { - throw new QueryShardException(context, "[common] analyzer [" + analyzer + "] not found"); - } - } - - Occur highFreqOccur = highFreqOperator.toBooleanClauseOccur(); - Occur lowFreqOccur = lowFreqOperator.toBooleanClauseOccur(); - - ExtendedCommonTermsQuery commonsQuery = new ExtendedCommonTermsQuery(highFreqOccur, lowFreqOccur, cutoffFrequency); - return parseQueryString(commonsQuery, text, field, analyzerObj, lowFreqMinimumShouldMatch, highFreqMinimumShouldMatch); - } - - private static Query parseQueryString(ExtendedCommonTermsQuery query, Object queryString, String field, Analyzer analyzer, - String lowFreqMinimumShouldMatch, String highFreqMinimumShouldMatch) throws IOException { - // Logic similar to QueryParser#getFieldQuery - try (TokenStream source = analyzer.tokenStream(field, queryString.toString())) { - source.reset(); - CharTermAttribute termAtt = source.addAttribute(CharTermAttribute.class); - BytesRefBuilder builder = new BytesRefBuilder(); - while (source.incrementToken()) { - // UTF-8 - builder.copyChars(termAtt); - query.add(new Term(field, builder.toBytesRef())); - } - } - - query.setLowFreqMinimumNumberShouldMatch(lowFreqMinimumShouldMatch); - query.setHighFreqMinimumNumberShouldMatch(highFreqMinimumShouldMatch); - return query; - } - - @Override - protected int doHashCode() { - return Objects.hash(fieldName, text, highFreqOperator, lowFreqOperator, analyzer, - lowFreqMinimumShouldMatch, highFreqMinimumShouldMatch, cutoffFrequency); - } - - @Override - protected boolean doEquals(CommonTermsQueryBuilder other) { - return Objects.equals(fieldName, other.fieldName) && - Objects.equals(text, other.text) && - Objects.equals(highFreqOperator, other.highFreqOperator) && - Objects.equals(lowFreqOperator, other.lowFreqOperator) && - Objects.equals(analyzer, other.analyzer) && - Objects.equals(lowFreqMinimumShouldMatch, other.lowFreqMinimumShouldMatch) && - Objects.equals(highFreqMinimumShouldMatch, other.highFreqMinimumShouldMatch) && - Objects.equals(cutoffFrequency, other.cutoffFrequency); - } -} diff --git a/server/src/main/java/org/elasticsearch/index/query/IdsQueryBuilder.java b/server/src/main/java/org/elasticsearch/index/query/IdsQueryBuilder.java index 358a2fccff108..e09d71938add4 100644 --- a/server/src/main/java/org/elasticsearch/index/query/IdsQueryBuilder.java +++ b/server/src/main/java/org/elasticsearch/index/query/IdsQueryBuilder.java @@ -22,7 +22,7 @@ import org.apache.logging.log4j.LogManager; import org.apache.lucene.search.MatchNoDocsQuery; import org.apache.lucene.search.Query; -import org.elasticsearch.cluster.metadata.MetaData; +import org.elasticsearch.Version; import org.elasticsearch.common.ParseField; import org.elasticsearch.common.ParsingException; import org.elasticsearch.common.Strings; @@ -33,14 +33,12 @@ import org.elasticsearch.common.xcontent.ObjectParser; import org.elasticsearch.common.xcontent.XContentBuilder; import org.elasticsearch.common.xcontent.XContentParser; -import org.elasticsearch.index.mapper.DocumentMapper; import org.elasticsearch.index.mapper.IdFieldMapper; import org.elasticsearch.index.mapper.MappedFieldType; import java.io.IOException; import java.util.ArrayList; import java.util.Arrays; -import java.util.Collection; import java.util.Collections; import java.util.HashSet; import java.util.Objects; @@ -62,8 +60,6 @@ public class IdsQueryBuilder extends AbstractQueryBuilder { private final Set ids = new HashSet<>(); - private String[] types = Strings.EMPTY_ARRAY; - /** * Creates a new IdsQueryBuilder with no types specified upfront */ @@ -76,38 +72,23 @@ public IdsQueryBuilder() { */ public IdsQueryBuilder(StreamInput in) throws IOException { super(in); - types = in.readStringArray(); + if (in.getVersion().before(Version.V_8_0_0)) { + // types no longer relevant so ignore + String[] types = in.readStringArray(); + if (types.length > 0) { + throw new IllegalStateException("types are no longer supported in ids query but found [" + Arrays.toString(types) + "]"); + } + } Collections.addAll(ids, in.readStringArray()); } @Override protected void doWriteTo(StreamOutput out) throws IOException { - out.writeStringArray(types); - out.writeStringArray(ids.toArray(new String[ids.size()])); - } - - /** - * Add types to query - * - * @deprecated Types are in the process of being removed, prefer to filter on a field instead. - */ - @Deprecated - public IdsQueryBuilder types(String... types) { - if (types == null) { - throw new IllegalArgumentException("[" + NAME + "] types cannot be null"); + if (out.getVersion().before(Version.V_8_0_0)) { + // types not supported so send an empty array to previous versions + out.writeStringArray(Strings.EMPTY_ARRAY); } - this.types = types; - return this; - } - - /** - * Returns the types used in this query - * - * @deprecated Types are in the process of being removed, prefer to filter on a field instead. - */ - @Deprecated - public String[] types() { - return this.types; + out.writeStringArray(ids.toArray(new String[ids.size()])); } /** @@ -131,9 +112,6 @@ public Set ids() { @Override protected void doXContent(XContentBuilder builder, Params params) throws IOException { builder.startObject(NAME); - if (types.length > 0) { - builder.array(TYPE_FIELD.getPreferredName(), types); - } builder.startArray(VALUES_FIELD.getPreferredName()); for (String value : ids) { builder.value(value); @@ -147,18 +125,13 @@ protected void doXContent(XContentBuilder builder, Params params) throws IOExcep () -> new IdsQueryBuilder()); static { - PARSER.declareStringArray(fromList(String.class, IdsQueryBuilder::types), IdsQueryBuilder.TYPE_FIELD); PARSER.declareStringArray(fromList(String.class, IdsQueryBuilder::addIds), IdsQueryBuilder.VALUES_FIELD); declareStandardFields(PARSER); } public static IdsQueryBuilder fromXContent(XContentParser parser) { try { - IdsQueryBuilder builder = PARSER.apply(parser, null); - if (builder.types().length > 0) { - deprecationLogger.deprecatedAndMaybeLog("ids_query_with_types", TYPES_DEPRECATION_MESSAGE); - } - return builder; + return PARSER.apply(parser, null); } catch (IllegalArgumentException e) { throw new ParsingException(parser.getTokenLocation(), e.getMessage(), e); } @@ -179,33 +152,17 @@ protected Query doToQuery(QueryShardContext context) throws IOException { if (this.ids.isEmpty()) { return Queries.newMatchNoDocsQuery("Missing ids in \"" + this.getName() + "\" query."); } else { - final DocumentMapper mapper = context.getMapperService().documentMapper(); - Collection typesForQuery; - if (types.length == 0) { - typesForQuery = context.queryTypes(); - } else if (types.length == 1 && MetaData.ALL.equals(types[0])) { - typesForQuery = Collections.singleton(mapper.type()); - } else { - typesForQuery = new HashSet<>(Arrays.asList(types)); - } - - if (typesForQuery.contains(mapper.type())) { - return idField.termsQuery(new ArrayList<>(ids), context); - } else { - return new MatchNoDocsQuery("Type mismatch"); - } - + return idField.termsQuery(new ArrayList<>(ids), context); } } @Override protected int doHashCode() { - return Objects.hash(ids, Arrays.hashCode(types)); + return Objects.hash(ids); } @Override protected boolean doEquals(IdsQueryBuilder other) { - return Objects.equals(ids, other.ids) && - Arrays.equals(types, other.types); + return Objects.equals(ids, other.ids); } } diff --git a/server/src/main/java/org/elasticsearch/index/query/IntervalBuilder.java b/server/src/main/java/org/elasticsearch/index/query/IntervalBuilder.java index 92b4fa664193c..5e1047684840f 100644 --- a/server/src/main/java/org/elasticsearch/index/query/IntervalBuilder.java +++ b/server/src/main/java/org/elasticsearch/index/query/IntervalBuilder.java @@ -96,7 +96,7 @@ protected IntervalsSource analyzeText(CachingTokenFilter stream, int maxGaps, bo // formulate a single term, boolean, or phrase. if (numTokens == 0) { - return null; + return NO_INTERVALS; } else if (numTokens == 1) { // single term return analyzeTerm(stream); @@ -231,7 +231,7 @@ protected List analyzeGraph(TokenStream source) throws IOExcept return clauses; } - private static final IntervalsSource NO_INTERVALS = new IntervalsSource() { + static final IntervalsSource NO_INTERVALS = new IntervalsSource() { @Override public IntervalIterator intervals(String field, LeafReaderContext ctx) { diff --git a/server/src/main/java/org/elasticsearch/index/query/IntervalsSourceProvider.java b/server/src/main/java/org/elasticsearch/index/query/IntervalsSourceProvider.java index e551654af9a76..d4d28057c12d0 100644 --- a/server/src/main/java/org/elasticsearch/index/query/IntervalsSourceProvider.java +++ b/server/src/main/java/org/elasticsearch/index/query/IntervalsSourceProvider.java @@ -19,10 +19,12 @@ package org.elasticsearch.index.query; +import org.apache.lucene.index.IndexOptions; import org.apache.lucene.search.intervals.FilteredIntervalsSource; import org.apache.lucene.search.intervals.IntervalIterator; import org.apache.lucene.search.intervals.Intervals; import org.apache.lucene.search.intervals.IntervalsSource; +import org.apache.lucene.util.BytesRef; import org.elasticsearch.Version; import org.elasticsearch.common.ParseField; import org.elasticsearch.common.ParsingException; @@ -78,9 +80,13 @@ public static IntervalsSourceProvider fromXContent(XContentParser parser) throws return Disjunction.fromXContent(parser); case "all_of": return Combine.fromXContent(parser); + case "prefix": + return Prefix.fromXContent(parser); + case "wildcard": + return Wildcard.fromXContent(parser); } throw new ParsingException(parser.getTokenLocation(), - "Unknown interval type [" + parser.currentName() + "], expecting one of [match, any_of, all_of]"); + "Unknown interval type [" + parser.currentName() + "], expecting one of [match, any_of, all_of, prefix]"); } private static IntervalsSourceProvider parseInnerIntervals(XContentParser parser) throws IOException { @@ -138,10 +144,10 @@ public IntervalsSource getSource(QueryShardContext context, MappedFieldType fiel if (useField != null) { fieldType = context.fieldMapper(useField); assert fieldType != null; - source = Intervals.fixField(useField, fieldType.intervals(query, maxGaps, ordered, analyzer)); + source = Intervals.fixField(useField, fieldType.intervals(query, maxGaps, ordered, analyzer, false)); } else { - source = fieldType.intervals(query, maxGaps, ordered, analyzer); + source = fieldType.intervals(query, maxGaps, ordered, analyzer, false); } if (filter != null) { return filter.filter(source, context, fieldType); @@ -440,6 +446,226 @@ public static Combine fromXContent(XContentParser parser) { } } + public static class Prefix extends IntervalsSourceProvider { + + public static final String NAME = "prefix"; + + private final String prefix; + private final String analyzer; + private final String useField; + + public Prefix(String prefix, String analyzer, String useField) { + this.prefix = prefix; + this.analyzer = analyzer; + this.useField = useField; + } + + public Prefix(StreamInput in) throws IOException { + this.prefix = in.readString(); + this.analyzer = in.readOptionalString(); + this.useField = in.readOptionalString(); + } + + @Override + public IntervalsSource getSource(QueryShardContext context, MappedFieldType fieldType) throws IOException { + NamedAnalyzer analyzer = null; + if (this.analyzer != null) { + analyzer = context.getMapperService().getIndexAnalyzers().get(this.analyzer); + } + IntervalsSource source; + if (useField != null) { + fieldType = context.fieldMapper(useField); + assert fieldType != null; + source = Intervals.fixField(useField, fieldType.intervals(prefix, 0, false, analyzer, true)); + } + else { + source = fieldType.intervals(prefix, 0, false, analyzer, true); + } + return source; + } + + @Override + public void extractFields(Set fields) { + if (useField != null) { + fields.add(useField); + } + } + + @Override + public boolean equals(Object o) { + if (this == o) return true; + if (o == null || getClass() != o.getClass()) return false; + Prefix prefix = (Prefix) o; + return Objects.equals(this.prefix, prefix.prefix) && + Objects.equals(analyzer, prefix.analyzer) && + Objects.equals(useField, prefix.useField); + } + + @Override + public int hashCode() { + return Objects.hash(prefix, analyzer, useField); + } + + @Override + public String getWriteableName() { + return NAME; + } + + @Override + public void writeTo(StreamOutput out) throws IOException { + out.writeString(prefix); + out.writeOptionalString(analyzer); + out.writeOptionalString(useField); + } + + @Override + public XContentBuilder toXContent(XContentBuilder builder, Params params) throws IOException { + builder.startObject(NAME); + builder.field("prefix", prefix); + if (analyzer != null) { + builder.field("analyzer", analyzer); + } + if (useField != null) { + builder.field("use_field", useField); + } + builder.endObject(); + return builder; + } + + private static final ConstructingObjectParser PARSER = new ConstructingObjectParser<>(NAME, args -> { + String term = (String) args[0]; + String analyzer = (String) args[1]; + String useField = (String) args[2]; + return new Prefix(term, analyzer, useField); + }); + static { + PARSER.declareString(constructorArg(), new ParseField("prefix")); + PARSER.declareString(optionalConstructorArg(), new ParseField("analyzer")); + PARSER.declareString(optionalConstructorArg(), new ParseField("use_field")); + } + + public static Prefix fromXContent(XContentParser parser) throws IOException { + return PARSER.parse(parser, null); + } + } + + public static class Wildcard extends IntervalsSourceProvider { + + public static final String NAME = "wildcard"; + + private final String pattern; + private final String analyzer; + private final String useField; + + public Wildcard(String pattern, String analyzer, String useField) { + this.pattern = pattern; + this.analyzer = analyzer; + this.useField = useField; + } + + public Wildcard(StreamInput in) throws IOException { + this.pattern = in.readString(); + this.analyzer = in.readOptionalString(); + this.useField = in.readOptionalString(); + } + + @Override + public IntervalsSource getSource(QueryShardContext context, MappedFieldType fieldType) { + NamedAnalyzer analyzer = fieldType.searchAnalyzer(); + if (this.analyzer != null) { + analyzer = context.getMapperService().getIndexAnalyzers().get(this.analyzer); + } + IntervalsSource source; + if (useField != null) { + fieldType = context.fieldMapper(useField); + assert fieldType != null; + checkPositions(fieldType); + if (this.analyzer == null) { + analyzer = fieldType.searchAnalyzer(); + } + BytesRef normalizedTerm = analyzer.normalize(useField, pattern); + // TODO Intervals.wildcard() should take BytesRef + source = Intervals.fixField(useField, Intervals.wildcard(normalizedTerm.utf8ToString())); + } + else { + checkPositions(fieldType); + BytesRef normalizedTerm = analyzer.normalize(fieldType.name(), pattern); + source = Intervals.wildcard(normalizedTerm.utf8ToString()); + } + return source; + } + + private void checkPositions(MappedFieldType type) { + if (type.indexOptions().compareTo(IndexOptions.DOCS_AND_FREQS_AND_POSITIONS) < 0) { + throw new IllegalArgumentException("Cannot create intervals over field [" + type.name() + "] with no positions indexed"); + } + } + + @Override + public void extractFields(Set fields) { + if (useField != null) { + fields.add(useField); + } + } + + @Override + public boolean equals(Object o) { + if (this == o) return true; + if (o == null || getClass() != o.getClass()) return false; + Prefix prefix = (Prefix) o; + return Objects.equals(pattern, prefix.prefix) && + Objects.equals(analyzer, prefix.analyzer) && + Objects.equals(useField, prefix.useField); + } + + @Override + public int hashCode() { + return Objects.hash(pattern, analyzer, useField); + } + + @Override + public String getWriteableName() { + return NAME; + } + + @Override + public void writeTo(StreamOutput out) throws IOException { + out.writeString(pattern); + out.writeOptionalString(analyzer); + out.writeOptionalString(useField); + } + + @Override + public XContentBuilder toXContent(XContentBuilder builder, Params params) throws IOException { + builder.startObject(NAME); + builder.field("pattern", pattern); + if (analyzer != null) { + builder.field("analyzer", analyzer); + } + if (useField != null) { + builder.field("use_field", useField); + } + builder.endObject(); + return builder; + } + + private static final ConstructingObjectParser PARSER = new ConstructingObjectParser<>(NAME, args -> { + String term = (String) args[0]; + String analyzer = (String) args[1]; + String useField = (String) args[2]; + return new Wildcard(term, analyzer, useField); + }); + static { + PARSER.declareString(constructorArg(), new ParseField("pattern")); + PARSER.declareString(optionalConstructorArg(), new ParseField("analyzer")); + PARSER.declareString(optionalConstructorArg(), new ParseField("use_field")); + } + + public static Wildcard fromXContent(XContentParser parser) throws IOException { + return PARSER.parse(parser, null); + } + } + static class ScriptFilterSource extends FilteredIntervalsSource { final IntervalFilterScript script; diff --git a/server/src/main/java/org/elasticsearch/index/query/MatchQueryBuilder.java b/server/src/main/java/org/elasticsearch/index/query/MatchQueryBuilder.java index 0d451bd86f264..8d5b6e71927e2 100644 --- a/server/src/main/java/org/elasticsearch/index/query/MatchQueryBuilder.java +++ b/server/src/main/java/org/elasticsearch/index/query/MatchQueryBuilder.java @@ -21,6 +21,7 @@ import org.apache.lucene.search.FuzzyQuery; import org.apache.lucene.search.Query; +import org.elasticsearch.Version; import org.elasticsearch.common.ParseField; import org.elasticsearch.common.ParsingException; import org.elasticsearch.common.io.stream.StreamInput; @@ -42,8 +43,8 @@ * result of the analysis. */ public class MatchQueryBuilder extends AbstractQueryBuilder { + public static final ParseField ZERO_TERMS_QUERY_FIELD = new ParseField("zero_terms_query"); - public static final ParseField CUTOFF_FREQUENCY_FIELD = new ParseField("cutoff_frequency"); public static final ParseField LENIENT_FIELD = new ParseField("lenient"); public static final ParseField FUZZY_TRANSPOSITIONS_FIELD = new ParseField("fuzzy_transpositions"); public static final ParseField FUZZY_REWRITE_FIELD = new ParseField("fuzzy_rewrite"); @@ -85,8 +86,6 @@ public class MatchQueryBuilder extends AbstractQueryBuilder { private MatchQuery.ZeroTermsQuery zeroTermsQuery = MatchQuery.DEFAULT_ZERO_TERMS_QUERY; - private Float cutoffFrequency = null; - private boolean autoGenerateSynonymsPhraseQuery = true; /** @@ -121,7 +120,10 @@ public MatchQueryBuilder(StreamInput in) throws IOException { minimumShouldMatch = in.readOptionalString(); fuzzyRewrite = in.readOptionalString(); fuzziness = in.readOptionalWriteable(Fuzziness::new); - cutoffFrequency = in.readOptionalFloat(); + // cutoff_frequency has been removed + if (in.getVersion().before(Version.V_8_0_0)) { + in.readOptionalFloat(); + } autoGenerateSynonymsPhraseQuery = in.readBoolean(); } @@ -140,7 +142,10 @@ protected void doWriteTo(StreamOutput out) throws IOException { out.writeOptionalString(minimumShouldMatch); out.writeOptionalString(fuzzyRewrite); out.writeOptionalWriteable(fuzziness); - out.writeOptionalFloat(cutoffFrequency); + // cutoff_frequency has been removed + if (out.getVersion().before(Version.V_8_0_0)) { + out.writeOptionalFloat(null); + } out.writeBoolean(autoGenerateSynonymsPhraseQuery); } @@ -231,21 +236,6 @@ public int maxExpansions() { return this.maxExpansions; } - /** - * Set a cutoff value in [0..1] (or absolute number >=1) representing the - * maximum threshold of a terms document frequency to be considered a low - * frequency term. - */ - public MatchQueryBuilder cutoffFrequency(float cutoff) { - this.cutoffFrequency = cutoff; - return this; - } - - /** Gets the optional cutoff value, can be {@code null} if not set previously */ - public Float cutoffFrequency() { - return this.cutoffFrequency; - } - /** Sets optional minimumShouldMatch value to apply to the query */ public MatchQueryBuilder minimumShouldMatch(String minimumShouldMatch) { this.minimumShouldMatch = minimumShouldMatch; @@ -362,9 +352,6 @@ public void doXContent(XContentBuilder builder, Params params) throws IOExceptio builder.field(FUZZY_TRANSPOSITIONS_FIELD.getPreferredName(), fuzzyTranspositions); builder.field(LENIENT_FIELD.getPreferredName(), lenient); builder.field(ZERO_TERMS_QUERY_FIELD.getPreferredName(), zeroTermsQuery.toString()); - if (cutoffFrequency != null) { - builder.field(CUTOFF_FREQUENCY_FIELD.getPreferredName(), cutoffFrequency); - } builder.field(GENERATE_SYNONYMS_PHRASE_QUERY.getPreferredName(), autoGenerateSynonymsPhraseQuery); printBoostAndQueryName(builder); builder.endObject(); @@ -389,7 +376,6 @@ protected Query doToQuery(QueryShardContext context) throws IOException { matchQuery.setTranspositions(fuzzyTranspositions); matchQuery.setFuzzyRewriteMethod(QueryParsers.parseRewriteMethod(fuzzyRewrite, null, LoggingDeprecationHandler.INSTANCE)); matchQuery.setLenient(lenient); - matchQuery.setCommonTermsCutoff(cutoffFrequency); matchQuery.setZeroTermsQuery(zeroTermsQuery); matchQuery.setAutoGenerateSynonymsPhraseQuery(autoGenerateSynonymsPhraseQuery); @@ -411,7 +397,6 @@ protected boolean doEquals(MatchQueryBuilder other) { Objects.equals(lenient, other.lenient) && Objects.equals(fuzzyTranspositions, other.fuzzyTranspositions) && Objects.equals(zeroTermsQuery, other.zeroTermsQuery) && - Objects.equals(cutoffFrequency, other.cutoffFrequency) && Objects.equals(autoGenerateSynonymsPhraseQuery, other.autoGenerateSynonymsPhraseQuery); } @@ -419,7 +404,7 @@ protected boolean doEquals(MatchQueryBuilder other) { protected int doHashCode() { return Objects.hash(fieldName, value, operator, analyzer, fuzziness, prefixLength, maxExpansions, minimumShouldMatch, - fuzzyRewrite, lenient, fuzzyTranspositions, zeroTermsQuery, cutoffFrequency, autoGenerateSynonymsPhraseQuery); + fuzzyRewrite, lenient, fuzzyTranspositions, zeroTermsQuery, autoGenerateSynonymsPhraseQuery); } @Override @@ -440,7 +425,6 @@ public static MatchQueryBuilder fromXContent(XContentParser parser) throws IOExc boolean fuzzyTranspositions = FuzzyQuery.defaultTranspositions; String fuzzyRewrite = null; boolean lenient = MatchQuery.DEFAULT_LENIENCY; - Float cutOffFrequency = null; ZeroTermsQuery zeroTermsQuery = MatchQuery.DEFAULT_ZERO_TERMS_QUERY; boolean autoGenerateSynonymsPhraseQuery = true; String queryName = null; @@ -478,8 +462,6 @@ public static MatchQueryBuilder fromXContent(XContentParser parser) throws IOExc fuzzyTranspositions = parser.booleanValue(); } else if (LENIENT_FIELD.match(currentFieldName, parser.getDeprecationHandler())) { lenient = parser.booleanValue(); - } else if (CUTOFF_FREQUENCY_FIELD.match(currentFieldName, parser.getDeprecationHandler())) { - cutOffFrequency = parser.floatValue(); } else if (ZERO_TERMS_QUERY_FIELD.match(currentFieldName, parser.getDeprecationHandler())) { String zeroTermsValue = parser.text(); if ("none".equalsIgnoreCase(zeroTermsValue)) { @@ -526,14 +508,10 @@ public static MatchQueryBuilder fromXContent(XContentParser parser) throws IOExc matchQuery.fuzzyTranspositions(fuzzyTranspositions); matchQuery.maxExpansions(maxExpansion); matchQuery.lenient(lenient); - if (cutOffFrequency != null) { - matchQuery.cutoffFrequency(cutOffFrequency); - } matchQuery.zeroTermsQuery(zeroTermsQuery); matchQuery.autoGenerateSynonymsPhraseQuery(autoGenerateSynonymsPhraseQuery); matchQuery.queryName(queryName); matchQuery.boost(boost); return matchQuery; } - } diff --git a/server/src/main/java/org/elasticsearch/index/query/MoreLikeThisQueryBuilder.java b/server/src/main/java/org/elasticsearch/index/query/MoreLikeThisQueryBuilder.java index 11530ce5f30b3..0d71829830b85 100644 --- a/server/src/main/java/org/elasticsearch/index/query/MoreLikeThisQueryBuilder.java +++ b/server/src/main/java/org/elasticsearch/index/query/MoreLikeThisQueryBuilder.java @@ -27,6 +27,7 @@ import org.apache.lucene.search.Query; import org.elasticsearch.ElasticsearchParseException; import org.elasticsearch.ExceptionsHelper; +import org.elasticsearch.Version; import org.elasticsearch.action.RoutingMissingException; import org.elasticsearch.action.termvectors.MultiTermVectorsItemResponse; import org.elasticsearch.action.termvectors.MultiTermVectorsRequest; @@ -55,7 +56,6 @@ import org.elasticsearch.index.mapper.IdFieldMapper; import org.elasticsearch.index.mapper.KeywordFieldMapper.KeywordFieldType; import org.elasticsearch.index.mapper.MappedFieldType; -import org.elasticsearch.index.mapper.MapperService; import org.elasticsearch.index.mapper.TextFieldMapper.TextFieldType; import java.io.IOException; @@ -69,7 +69,6 @@ import java.util.Objects; import java.util.Optional; import java.util.Set; -import java.util.stream.Stream; import static org.elasticsearch.common.xcontent.XContentFactory.jsonBuilder; @@ -117,7 +116,6 @@ public class MoreLikeThisQueryBuilder extends AbstractQueryBuilder item.type == null); - } - private static void parseLikeField(XContentParser parser, List texts, List items) throws IOException { if (parser.currentToken().isValue()) { texts.add(parser.text()); @@ -1132,9 +1066,6 @@ private static void setDefaultIndexTypeFields(QueryShardContext context, Item it if (item.index() == null) { item.index(context.index().getName()); } - if (item.type() == null) { - item.type(MapperService.SINGLE_MAPPING_NAME); - } // default fields if not present but don't override for artificial docs if ((item.fields() == null || item.fields().length == 0) && item.doc() == null) { if (useDefaultField) { diff --git a/server/src/main/java/org/elasticsearch/index/query/MultiMatchQueryBuilder.java b/server/src/main/java/org/elasticsearch/index/query/MultiMatchQueryBuilder.java index 7827c032ea0d7..07f7ae4b79398 100644 --- a/server/src/main/java/org/elasticsearch/index/query/MultiMatchQueryBuilder.java +++ b/server/src/main/java/org/elasticsearch/index/query/MultiMatchQueryBuilder.java @@ -22,6 +22,7 @@ import org.apache.lucene.search.FuzzyQuery; import org.apache.lucene.search.Query; import org.elasticsearch.ElasticsearchParseException; +import org.elasticsearch.Version; import org.elasticsearch.common.ParseField; import org.elasticsearch.common.ParsingException; import org.elasticsearch.common.Strings; @@ -50,6 +51,7 @@ * Same as {@link MatchQueryBuilder} but supports multiple fields. */ public class MultiMatchQueryBuilder extends AbstractQueryBuilder { + public static final String NAME = "multi_match"; public static final MultiMatchQueryBuilder.Type DEFAULT_TYPE = MultiMatchQueryBuilder.Type.BEST_FIELDS; @@ -63,7 +65,6 @@ public class MultiMatchQueryBuilder extends AbstractQueryBuilder searcherFactory; private final BiFunction> indexFieldDataService; private final int shardId; private final IndexReader reader; - private String[] types = Strings.EMPTY_ARRAY; private boolean cacheable = true; private final SetOnce frozen = new SetOnce<>(); private final Index fullyQualifiedIndex; - public void setTypes(String... types) { - this.types = types; - } - - public String[] getTypes() { - return types; - } - private final Map namedQueries = new HashMap<>(); private boolean allowUnmappedFields; private boolean mapUnmappedFieldAsString; private NestedScope nestedScope; public QueryShardContext(int shardId, IndexSettings indexSettings, BitsetFilterCache bitsetFilterCache, + Function searcherFactory, BiFunction> indexFieldDataLookup, MapperService mapperService, SimilarityService similarityService, ScriptService scriptService, NamedXContentRegistry xContentRegistry, NamedWriteableRegistry namedWriteableRegistry, Client client, IndexReader reader, LongSupplier nowInMillis, String clusterAlias) { - this(shardId, indexSettings, bitsetFilterCache, indexFieldDataLookup, mapperService, similarityService, scriptService, - xContentRegistry, namedWriteableRegistry, client, reader, nowInMillis, new Index(RemoteClusterAware.buildRemoteIndexName( - clusterAlias, indexSettings.getIndex().getName()), indexSettings.getIndex().getUUID())); + this(shardId, indexSettings, bitsetFilterCache, searcherFactory, indexFieldDataLookup, mapperService, similarityService, + scriptService, xContentRegistry, namedWriteableRegistry, client, reader, nowInMillis, + new Index(RemoteClusterAware.buildRemoteIndexName(clusterAlias, indexSettings.getIndex().getName()), + indexSettings.getIndex().getUUID())); } public QueryShardContext(QueryShardContext source) { - this(source.shardId, source.indexSettings, source.bitsetFilterCache, source.indexFieldDataService, source.mapperService, - source.similarityService, source.scriptService, source.getXContentRegistry(), source.getWriteableRegistry(), - source.client, source.reader, source.nowInMillis, source.fullyQualifiedIndex); - this.types = source.getTypes(); + this(source.shardId, source.indexSettings, source.bitsetFilterCache, source.searcherFactory, source.indexFieldDataService, + source.mapperService, source.similarityService, source.scriptService, source.getXContentRegistry(), + source.getWriteableRegistry(), source.client, source.reader, source.nowInMillis, source.fullyQualifiedIndex); } private QueryShardContext(int shardId, IndexSettings indexSettings, BitsetFilterCache bitsetFilterCache, + Function searcherFactory, BiFunction> indexFieldDataLookup, MapperService mapperService, SimilarityService similarityService, ScriptService scriptService, NamedXContentRegistry xContentRegistry, NamedWriteableRegistry namedWriteableRegistry, Client client, IndexReader reader, LongSupplier nowInMillis, Index fullyQualifiedIndex) { - super(xContentRegistry, namedWriteableRegistry,client, nowInMillis); + super(xContentRegistry, namedWriteableRegistry, client, nowInMillis); this.shardId = shardId; this.similarityService = similarityService; this.mapperService = mapperService; this.bitsetFilterCache = bitsetFilterCache; + this.searcherFactory = searcherFactory; this.indexFieldDataService = indexFieldDataLookup; this.allowUnmappedFields = indexSettings.isDefaultAllowUnmappedFields(); this.nestedScope = new NestedScope(); @@ -173,6 +168,10 @@ public BitSetProducer bitsetFilter(Query filter) { return bitsetFilterCache.getBitSetProducer(filter); } + public IndexSearcher newCachedSearcher(IndexReaderContext context) { + return searcherFactory.apply(context); + } + public > IFD getForField(MappedFieldType fieldType) { return (IFD) indexFieldDataService.apply(fieldType, fullyQualifiedIndex.getName()); } @@ -192,7 +191,7 @@ public Map copyNamedQueries() { * Returns all the fields that match a given pattern. If prefixed with a * type then the fields will be returned with a type prefix. */ - public Collection simpleMatchToIndexNames(String pattern) { + public Set simpleMatchToIndexNames(String pattern) { return mapperService.simpleMatchToFullName(pattern); } @@ -256,24 +255,12 @@ MappedFieldType failIfFieldMappingNotFound(String name, MappedFieldType fieldMap } } - /** - * Returns the narrowed down explicit types, or, if not set, all types. - */ - public Collection queryTypes() { - String[] types = getTypes(); - if (types == null || types.length == 0 || (types.length == 1 && types[0].equals("_all"))) { - DocumentMapper mapper = getMapperService().documentMapper(); - return mapper == null ? Collections.emptyList() : Collections.singleton(mapper.type()); - } - return Arrays.asList(types); - } - private SearchLookup lookup = null; public SearchLookup lookup() { if (lookup == null) { lookup = new SearchLookup(getMapperService(), - mappedFieldType -> indexFieldDataService.apply(mappedFieldType, fullyQualifiedIndex.getName()), types); + mappedFieldType -> indexFieldDataService.apply(mappedFieldType, fullyQualifiedIndex.getName())); } return lookup; } diff --git a/server/src/main/java/org/elasticsearch/index/query/functionscore/ScriptScoreQueryBuilder.java b/server/src/main/java/org/elasticsearch/index/query/functionscore/ScriptScoreQueryBuilder.java index fb53f1c9560cc..b5cb15b3d00aa 100644 --- a/server/src/main/java/org/elasticsearch/index/query/functionscore/ScriptScoreQueryBuilder.java +++ b/server/src/main/java/org/elasticsearch/index/query/functionscore/ScriptScoreQueryBuilder.java @@ -23,17 +23,17 @@ import org.elasticsearch.common.ParseField; import org.elasticsearch.common.io.stream.StreamInput; import org.elasticsearch.common.io.stream.StreamOutput; +import org.elasticsearch.common.lucene.search.function.ScriptScoreFunction; +import org.elasticsearch.common.lucene.search.function.ScriptScoreQuery; import org.elasticsearch.common.xcontent.ConstructingObjectParser; import org.elasticsearch.common.xcontent.XContentBuilder; import org.elasticsearch.common.xcontent.XContentParser; import org.elasticsearch.index.query.AbstractQueryBuilder; -import org.elasticsearch.index.query.QueryBuilder; -import org.elasticsearch.script.Script; -import org.elasticsearch.common.lucene.search.function.ScriptScoreFunction; -import org.elasticsearch.common.lucene.search.function.ScriptScoreQuery; import org.elasticsearch.index.query.InnerHitContextBuilder; +import org.elasticsearch.index.query.QueryBuilder; import org.elasticsearch.index.query.QueryRewriteContext; import org.elasticsearch.index.query.QueryShardContext; +import org.elasticsearch.script.Script; import java.io.IOException; import java.util.Map; diff --git a/server/src/main/java/org/elasticsearch/index/reindex/AbstractBulkByScrollRequest.java b/server/src/main/java/org/elasticsearch/index/reindex/AbstractBulkByScrollRequest.java index add8eb6b668ae..0f27fb4753e9b 100644 --- a/server/src/main/java/org/elasticsearch/index/reindex/AbstractBulkByScrollRequest.java +++ b/server/src/main/java/org/elasticsearch/index/reindex/AbstractBulkByScrollRequest.java @@ -42,7 +42,7 @@ public abstract class AbstractBulkByScrollRequest> extends ActionRequest { - public static final int SIZE_ALL_MATCHES = -1; + public static final int MAX_DOCS_ALL_MATCHES = -1; public static final TimeValue DEFAULT_SCROLL_TIMEOUT = timeValueMinutes(5); public static final int DEFAULT_SCROLL_SIZE = 1000; @@ -59,7 +59,7 @@ public abstract class AbstractBulkByScrollRequest 0)) { + if (false == (maxDocs == -1 || maxDocs > 0)) { e = addValidationError( - "size should be greater than 0 if the request is limited to some number of documents or -1 if it isn't but it was [" - + size + "]", + "maxDocs should be greater than 0 if the request is limited to some number of documents or -1 if it isn't but it was [" + + maxDocs + "]", e); } if (searchRequest.source().slice() != null && slices != DEFAULT_SLICES) { @@ -167,19 +167,19 @@ public ActionRequestValidationException validate() { * Maximum number of processed documents. Defaults to -1 meaning process all * documents. */ - public int getSize() { - return size; + public int getMaxDocs() { + return maxDocs; } /** * Maximum number of processed documents. Defaults to -1 meaning process all * documents. */ - public Self setSize(int size) { - if (size < 0) { - throw new IllegalArgumentException("[size] parameter cannot be negative, found [" + size + "]"); + public Self setMaxDocs(int maxDocs) { + if (maxDocs < 0) { + throw new IllegalArgumentException("[max_docs] parameter cannot be negative, found [" + maxDocs + "]"); } - this.size = size; + this.maxDocs = maxDocs; return self(); } @@ -404,10 +404,10 @@ protected Self doForSlice(Self request, TaskId slicingTask, int totalSlices) { .setRequestsPerSecond(requestsPerSecond / totalSlices) // Sub requests don't have workers .setSlices(1); - if (size != -1) { - // Size is split between workers. This means the size might round + if (maxDocs != MAX_DOCS_ALL_MATCHES) { + // maxDocs is split between workers. This means the maxDocs might round // down! - request.setSize(size == SIZE_ALL_MATCHES ? SIZE_ALL_MATCHES : size / totalSlices); + request.setMaxDocs(maxDocs / totalSlices); } // Set the parent task so this task is cancelled if we cancel the parent request.setParentTask(slicingTask); @@ -425,7 +425,7 @@ public void readFrom(StreamInput in) throws IOException { super.readFrom(in); searchRequest = new SearchRequest(in); abortOnVersionConflict = in.readBoolean(); - size = in.readVInt(); + maxDocs = in.readVInt(); refresh = in.readBoolean(); timeout = in.readTimeValue(); activeShardCount = ActiveShardCount.readFrom(in); @@ -440,7 +440,7 @@ public void writeTo(StreamOutput out) throws IOException { super.writeTo(out); searchRequest.writeTo(out); out.writeBoolean(abortOnVersionConflict); - out.writeVInt(size); + out.writeVInt(maxDocs); out.writeBoolean(refresh); out.writeTimeValue(timeout); activeShardCount.writeTo(out); @@ -460,9 +460,6 @@ protected void searchToString(StringBuilder b) { } else { b.append("[all indices]"); } - if (searchRequest.types() != null && searchRequest.types().length != 0) { - b.append(Arrays.toString(searchRequest.types())); - } } @Override diff --git a/server/src/main/java/org/elasticsearch/index/reindex/AbstractBulkByScrollRequestBuilder.java b/server/src/main/java/org/elasticsearch/index/reindex/AbstractBulkByScrollRequestBuilder.java index a14ef850c5079..f6b025c8265a5 100644 --- a/server/src/main/java/org/elasticsearch/index/reindex/AbstractBulkByScrollRequestBuilder.java +++ b/server/src/main/java/org/elasticsearch/index/reindex/AbstractBulkByScrollRequestBuilder.java @@ -19,7 +19,7 @@ package org.elasticsearch.index.reindex; -import org.elasticsearch.action.Action; +import org.elasticsearch.action.ActionType; import org.elasticsearch.action.ActionRequestBuilder; import org.elasticsearch.action.search.SearchRequestBuilder; import org.elasticsearch.action.support.ActiveShardCount; @@ -35,7 +35,7 @@ public abstract class AbstractBulkByScrollRequestBuilder< private final SearchRequestBuilder source; protected AbstractBulkByScrollRequestBuilder(ElasticsearchClient client, - Action action, SearchRequestBuilder source, Request request) { + ActionType action, SearchRequestBuilder source, Request request) { super(client, action, request); this.source = source; } @@ -67,10 +67,22 @@ public Self filter(QueryBuilder filter) { } /** - * The maximum number of documents to attempt. + * Maximum number of processed documents. Defaults to processing all + * documents. + * @deprecated please use maxDocs(int) instead. */ + @Deprecated public Self size(int size) { - request.setSize(size); + return maxDocs(size); + } + + + /** + * Maximum number of processed documents. Defaults to processing all + * documents. + */ + public Self maxDocs(int maxDocs) { + request.setMaxDocs(maxDocs); return self(); } diff --git a/server/src/main/java/org/elasticsearch/index/reindex/AbstractBulkIndexByScrollRequestBuilder.java b/server/src/main/java/org/elasticsearch/index/reindex/AbstractBulkIndexByScrollRequestBuilder.java index 519c6e062d565..0e0382e146da4 100644 --- a/server/src/main/java/org/elasticsearch/index/reindex/AbstractBulkIndexByScrollRequestBuilder.java +++ b/server/src/main/java/org/elasticsearch/index/reindex/AbstractBulkIndexByScrollRequestBuilder.java @@ -19,7 +19,7 @@ package org.elasticsearch.index.reindex; -import org.elasticsearch.action.Action; +import org.elasticsearch.action.ActionType; import org.elasticsearch.action.search.SearchRequestBuilder; import org.elasticsearch.client.ElasticsearchClient; import org.elasticsearch.script.Script; @@ -29,8 +29,8 @@ public abstract class AbstractBulkIndexByScrollRequestBuilder< Self extends AbstractBulkIndexByScrollRequestBuilder> extends AbstractBulkByScrollRequestBuilder { - protected AbstractBulkIndexByScrollRequestBuilder(ElasticsearchClient client, - Action action, SearchRequestBuilder search, Request request) { + protected AbstractBulkIndexByScrollRequestBuilder(ElasticsearchClient client, ActionType action, + SearchRequestBuilder search, Request request) { super(client, action, search, request); } diff --git a/server/src/main/java/org/elasticsearch/index/reindex/ClientScrollableHitSource.java b/server/src/main/java/org/elasticsearch/index/reindex/ClientScrollableHitSource.java index 454983ba7942a..bbc12fb2c2aea 100644 --- a/server/src/main/java/org/elasticsearch/index/reindex/ClientScrollableHitSource.java +++ b/server/src/main/java/org/elasticsearch/index/reindex/ClientScrollableHitSource.java @@ -70,9 +70,8 @@ public ClientScrollableHitSource(Logger logger, BackoffPolicy backoffPolicy, Thr @Override public void doStart(Consumer onResponse) { if (logger.isDebugEnabled()) { - logger.debug("executing initial scroll against {}{}", - isEmpty(firstSearchRequest.indices()) ? "all indices" : firstSearchRequest.indices(), - isEmpty(firstSearchRequest.types()) ? "" : firstSearchRequest.types()); + logger.debug("executing initial scroll against {}", + isEmpty(firstSearchRequest.indices()) ? "all indices" : firstSearchRequest.indices()); } searchWithRetry(listener -> client.search(firstSearchRequest, listener), r -> consume(r, onResponse)); } diff --git a/server/src/main/java/org/elasticsearch/index/reindex/DeleteByQueryAction.java b/server/src/main/java/org/elasticsearch/index/reindex/DeleteByQueryAction.java index c7cfe28e2c0be..092f93ed2e54e 100644 --- a/server/src/main/java/org/elasticsearch/index/reindex/DeleteByQueryAction.java +++ b/server/src/main/java/org/elasticsearch/index/reindex/DeleteByQueryAction.java @@ -19,9 +19,9 @@ package org.elasticsearch.index.reindex; -import org.elasticsearch.action.Action; +import org.elasticsearch.action.StreamableResponseActionType; -public class DeleteByQueryAction extends Action { +public class DeleteByQueryAction extends StreamableResponseActionType { public static final DeleteByQueryAction INSTANCE = new DeleteByQueryAction(); public static final String NAME = "indices:data/write/delete/byquery"; diff --git a/server/src/main/java/org/elasticsearch/index/reindex/DeleteByQueryRequest.java b/server/src/main/java/org/elasticsearch/index/reindex/DeleteByQueryRequest.java index 227ddd489779c..c1e2f011a99de 100644 --- a/server/src/main/java/org/elasticsearch/index/reindex/DeleteByQueryRequest.java +++ b/server/src/main/java/org/elasticsearch/index/reindex/DeleteByQueryRequest.java @@ -87,19 +87,6 @@ public DeleteByQueryRequest setQuery(QueryBuilder query) { return this; } - /** - * Set the document types for the delete - * @deprecated Types are in the process of being removed. Instead of - * using a type, prefer to filter on a field of the document. - */ - @Deprecated - public DeleteByQueryRequest setDocTypes(String... types) { - if (types != null) { - getSearchRequest().types(types); - } - return this; - } - /** * Set routing limiting the process to the shards that match that routing value */ @@ -140,21 +127,6 @@ public String getRouting() { return getSearchRequest().routing(); } - /** - * Gets the document types on which this request would be executed. Returns an empty array if all - * types are to be processed. - * @deprecated Types are in the process of being removed. Instead of - * using a type, prefer to filter on a field of the document. - */ - @Deprecated - public String[] getDocTypes() { - if (getSearchRequest().types() != null) { - return getSearchRequest().types(); - } else { - return new String[0]; - } - } - @Override protected DeleteByQueryRequest self() { return this; @@ -208,29 +180,6 @@ public IndicesOptions indicesOptions() { return getSearchRequest().indicesOptions(); } - /** - * Gets the document types on which this request would be executed. - * @deprecated Types are in the process of being removed. Instead of - * using a type, prefer to filter on a field of the document. - */ - @Deprecated - public String[] types() { - assert getSearchRequest() != null; - return getSearchRequest().types(); - } - - /** - * Set the document types for the delete - * @deprecated Types are in the process of being removed. Instead of - * using a type, prefer to filter on a field of the document. - */ - @Deprecated - public DeleteByQueryRequest types(String... types) { - assert getSearchRequest() != null; - getSearchRequest().types(types); - return this; - } - @Override public XContentBuilder toXContent(XContentBuilder builder, Params params) throws IOException { builder.startObject(); diff --git a/server/src/main/java/org/elasticsearch/index/reindex/DeleteByQueryRequestBuilder.java b/server/src/main/java/org/elasticsearch/index/reindex/DeleteByQueryRequestBuilder.java index 202aa6e93f19d..77d304f4c17af 100644 --- a/server/src/main/java/org/elasticsearch/index/reindex/DeleteByQueryRequestBuilder.java +++ b/server/src/main/java/org/elasticsearch/index/reindex/DeleteByQueryRequestBuilder.java @@ -19,7 +19,7 @@ package org.elasticsearch.index.reindex; -import org.elasticsearch.action.Action; +import org.elasticsearch.action.ActionType; import org.elasticsearch.action.search.SearchAction; import org.elasticsearch.action.search.SearchRequestBuilder; import org.elasticsearch.client.ElasticsearchClient; @@ -27,12 +27,12 @@ public class DeleteByQueryRequestBuilder extends AbstractBulkByScrollRequestBuilder { - public DeleteByQueryRequestBuilder(ElasticsearchClient client, Action action) { + public DeleteByQueryRequestBuilder(ElasticsearchClient client, ActionType action) { this(client, action, new SearchRequestBuilder(client, SearchAction.INSTANCE)); } private DeleteByQueryRequestBuilder(ElasticsearchClient client, - Action action, + ActionType action, SearchRequestBuilder search) { super(client, action, search, new DeleteByQueryRequest(search.request())); } diff --git a/server/src/main/java/org/elasticsearch/index/reindex/ReindexAction.java b/server/src/main/java/org/elasticsearch/index/reindex/ReindexAction.java index 86d0c96602a3c..9ad9baa5abb90 100644 --- a/server/src/main/java/org/elasticsearch/index/reindex/ReindexAction.java +++ b/server/src/main/java/org/elasticsearch/index/reindex/ReindexAction.java @@ -19,9 +19,9 @@ package org.elasticsearch.index.reindex; -import org.elasticsearch.action.Action; +import org.elasticsearch.action.StreamableResponseActionType; -public class ReindexAction extends Action { +public class ReindexAction extends StreamableResponseActionType { public static final ReindexAction INSTANCE = new ReindexAction(); public static final String NAME = "indices:data/write/reindex"; diff --git a/server/src/main/java/org/elasticsearch/index/reindex/ReindexRequest.java b/server/src/main/java/org/elasticsearch/index/reindex/ReindexRequest.java index de171e88fbca1..bce848998159c 100644 --- a/server/src/main/java/org/elasticsearch/index/reindex/ReindexRequest.java +++ b/server/src/main/java/org/elasticsearch/index/reindex/ReindexRequest.java @@ -19,24 +19,42 @@ package org.elasticsearch.index.reindex; +import org.apache.logging.log4j.LogManager; import org.elasticsearch.action.ActionRequestValidationException; import org.elasticsearch.action.CompositeIndicesRequest; import org.elasticsearch.action.index.IndexRequest; import org.elasticsearch.action.search.SearchRequest; +import org.elasticsearch.common.ParseField; +import org.elasticsearch.common.Strings; +import org.elasticsearch.common.bytes.BytesReference; import org.elasticsearch.common.io.stream.StreamInput; import org.elasticsearch.common.io.stream.StreamOutput; +import org.elasticsearch.common.logging.DeprecationLogger; import org.elasticsearch.common.lucene.uid.Versions; +import org.elasticsearch.common.unit.TimeValue; +import org.elasticsearch.common.xcontent.ObjectParser; import org.elasticsearch.common.xcontent.ToXContentObject; import org.elasticsearch.common.xcontent.XContentBuilder; +import org.elasticsearch.common.xcontent.XContentFactory; +import org.elasticsearch.common.xcontent.XContentParser; import org.elasticsearch.index.VersionType; import org.elasticsearch.index.mapper.MapperService; import org.elasticsearch.index.query.QueryBuilder; +import org.elasticsearch.script.Script; import org.elasticsearch.search.sort.SortOrder; import org.elasticsearch.tasks.TaskId; import java.io.IOException; - +import java.io.InputStream; +import java.net.URI; +import java.net.URISyntaxException; +import java.util.List; +import java.util.Map; + +import static java.util.Collections.emptyMap; +import static java.util.Objects.requireNonNull; import static org.elasticsearch.action.ValidateActions.addValidationError; +import static org.elasticsearch.common.unit.TimeValue.parseTimeValue; import static org.elasticsearch.index.VersionType.INTERNAL; /** @@ -138,16 +156,6 @@ public ReindexRequest setSourceIndices(String... sourceIndices) { return this; } - /** - * Set the document types which need to be copied from the source indices - */ - public ReindexRequest setSourceDocTypes(String... docTypes) { - if (docTypes != null) { - this.getSearchRequest().types(docTypes); - } - return this; - } - /** * Sets the scroll size for setting how many documents are to be processed in one batch during reindex */ @@ -292,13 +300,9 @@ public XContentBuilder toXContent(XContentBuilder builder, Params params) throws builder.startObject("source"); if (remoteInfo != null) { builder.field("remote", remoteInfo); - builder.rawField("query", remoteInfo.getQuery().streamInput(), builder.contentType()); + builder.rawField("query", remoteInfo.getQuery().streamInput(), RemoteInfo.QUERY_CONTENT_TYPE.type()); } builder.array("index", getSearchRequest().indices()); - String[] types = getSearchRequest().types(); - if (types.length > 0) { - builder.array("type", types); - } getSearchRequest().source().innerToXContent(builder, params); builder.endObject(); } @@ -322,8 +326,8 @@ public XContentBuilder toXContent(XContentBuilder builder, Params params) throws } { // Other fields - if (getSize() != -1 || getSize() > 0) { - builder.field("size", getSize()); + if (getMaxDocs() != -1) { + builder.field("max_docs", getMaxDocs()); } if (getScript() != null) { builder.field("script", getScript()); @@ -335,4 +339,163 @@ public XContentBuilder toXContent(XContentBuilder builder, Params params) throws builder.endObject(); return builder; } + + static final ObjectParser PARSER = new ObjectParser<>("reindex"); + + static final String TYPES_DEPRECATION_MESSAGE = "[types removal] Specifying types in reindex requests is deprecated."; + private static final DeprecationLogger deprecationLogger = new DeprecationLogger(LogManager.getLogger(ReindexRequest.class)); + + static { + ObjectParser.Parser sourceParser = (parser, request, context) -> { + // Funky hack to work around Search not having a proper ObjectParser and us wanting to extract query if using remote. + Map source = parser.map(); + String[] indices = extractStringArray(source, "index"); + if (indices != null) { + request.getSearchRequest().indices(indices); + } + request.setRemoteInfo(buildRemoteInfo(source)); + XContentBuilder builder = XContentFactory.contentBuilder(parser.contentType()); + builder.map(source); + try (InputStream stream = BytesReference.bytes(builder).streamInput(); + XContentParser innerParser = parser.contentType().xContent() + .createParser(parser.getXContentRegistry(), parser.getDeprecationHandler(), stream)) { + request.getSearchRequest().source().parseXContent(innerParser, false); + } + }; + + ObjectParser destParser = new ObjectParser<>("dest"); + destParser.declareString(IndexRequest::index, new ParseField("index")); + destParser.declareString((request, type) -> { + deprecationLogger.deprecatedAndMaybeLog("reindex_with_types", TYPES_DEPRECATION_MESSAGE); + request.type(type); + }, new ParseField("type")); + destParser.declareString(IndexRequest::routing, new ParseField("routing")); + destParser.declareString(IndexRequest::opType, new ParseField("op_type")); + destParser.declareString(IndexRequest::setPipeline, new ParseField("pipeline")); + destParser.declareString((s, i) -> s.versionType(VersionType.fromString(i)), new ParseField("version_type")); + + PARSER.declareField(sourceParser::parse, new ParseField("source"), ObjectParser.ValueType.OBJECT); + PARSER.declareField((p, v, c) -> destParser.parse(p, v.getDestination(), c), new ParseField("dest"), ObjectParser.ValueType.OBJECT); + PARSER.declareInt(ReindexRequest::setMaxDocsValidateIdentical, new ParseField("max_docs")); + // avoid silently accepting an ignored size. + PARSER.declareInt((r,s) -> failOnSizeSpecified(), new ParseField("size")); + PARSER.declareField((p, v, c) -> v.setScript(Script.parse(p)), new ParseField("script"), + ObjectParser.ValueType.OBJECT); + PARSER.declareString(ReindexRequest::setConflicts, new ParseField("conflicts")); + } + + public static ReindexRequest fromXContent(XContentParser parser) throws IOException { + ReindexRequest reindexRequest = new ReindexRequest(); + PARSER.parse(parser, reindexRequest, null); + return reindexRequest; + } + + /** + * Yank a string array from a map. Emulates XContent's permissive String to + * String array conversions. + */ + private static String[] extractStringArray(Map source, String name) { + Object value = source.remove(name); + if (value == null) { + return null; + } + if (value instanceof List) { + @SuppressWarnings("unchecked") + List list = (List) value; + return list.toArray(new String[list.size()]); + } else if (value instanceof String) { + return new String[] {(String) value}; + } else { + throw new IllegalArgumentException("Expected [" + name + "] to be a list of a string but was [" + value + ']'); + } + } + + static RemoteInfo buildRemoteInfo(Map source) throws IOException { + @SuppressWarnings("unchecked") + Map remote = (Map) source.remove("remote"); + if (remote == null) { + return null; + } + String username = extractString(remote, "username"); + String password = extractString(remote, "password"); + String hostInRequest = requireNonNull(extractString(remote, "host"), "[host] must be specified to reindex from a remote cluster"); + URI uri; + try { + uri = new URI(hostInRequest); + // URI has less stringent URL parsing than our code. We want to fail if all values are not provided. + if (uri.getPort() == -1) { + throw new URISyntaxException(hostInRequest, "The port was not defined in the [host]"); + } + } catch (URISyntaxException ex) { + throw new IllegalArgumentException("[host] must be of the form [scheme]://[host]:[port](/[pathPrefix])? but was [" + + hostInRequest + "]", ex); + } + + String scheme = uri.getScheme(); + String host = uri.getHost(); + int port = uri.getPort(); + + String pathPrefix = null; + if (uri.getPath().isEmpty() == false) { + pathPrefix = uri.getPath(); + } + + Map headers = extractStringStringMap(remote, "headers"); + TimeValue socketTimeout = extractTimeValue(remote, "socket_timeout", RemoteInfo.DEFAULT_SOCKET_TIMEOUT); + TimeValue connectTimeout = extractTimeValue(remote, "connect_timeout", RemoteInfo.DEFAULT_CONNECT_TIMEOUT); + if (false == remote.isEmpty()) { + throw new IllegalArgumentException( + "Unsupported fields in [remote]: [" + Strings.collectionToCommaDelimitedString(remote.keySet()) + "]"); + } + return new RemoteInfo(scheme, host, port, pathPrefix, RemoteInfo.queryForRemote(source), + username, password, headers, socketTimeout, connectTimeout); + } + + private static String extractString(Map source, String name) { + Object value = source.remove(name); + if (value == null) { + return null; + } + if (value instanceof String) { + return (String) value; + } + throw new IllegalArgumentException("Expected [" + name + "] to be a string but was [" + value + "]"); + } + + private static Map extractStringStringMap(Map source, String name) { + Object value = source.remove(name); + if (value == null) { + return emptyMap(); + } + if (false == value instanceof Map) { + throw new IllegalArgumentException("Expected [" + name + "] to be an object containing strings but was [" + value + "]"); + } + Map map = (Map) value; + for (Map.Entry entry : map.entrySet()) { + if (false == entry.getKey() instanceof String || false == entry.getValue() instanceof String) { + throw new IllegalArgumentException("Expected [" + name + "] to be an object containing strings but has [" + entry + "]"); + } + } + @SuppressWarnings("unchecked") // We just checked.... + Map safe = (Map) map; + return safe; + } + + private static TimeValue extractTimeValue(Map source, String name, TimeValue defaultValue) { + String string = extractString(source, name); + return string == null ? defaultValue : parseTimeValue(string, name); + } + + static void setMaxDocsValidateIdentical(AbstractBulkByScrollRequest request, int maxDocs) { + if (request.getMaxDocs() != AbstractBulkByScrollRequest.MAX_DOCS_ALL_MATCHES && request.getMaxDocs() != maxDocs) { + throw new IllegalArgumentException("[max_docs] set to two different values [" + request.getMaxDocs() + "]" + + " and [" + maxDocs + "]"); + } else { + request.setMaxDocs(maxDocs); + } + } + + private static void failOnSizeSpecified() { + throw new IllegalArgumentException("invalid parameter [size], use [max_docs] instead"); + } } diff --git a/server/src/main/java/org/elasticsearch/index/reindex/ReindexRequestBuilder.java b/server/src/main/java/org/elasticsearch/index/reindex/ReindexRequestBuilder.java index c775ae197db93..a934584bc9724 100644 --- a/server/src/main/java/org/elasticsearch/index/reindex/ReindexRequestBuilder.java +++ b/server/src/main/java/org/elasticsearch/index/reindex/ReindexRequestBuilder.java @@ -19,7 +19,7 @@ package org.elasticsearch.index.reindex; -import org.elasticsearch.action.Action; +import org.elasticsearch.action.ActionType; import org.elasticsearch.action.index.IndexAction; import org.elasticsearch.action.index.IndexRequestBuilder; import org.elasticsearch.action.search.SearchAction; @@ -31,13 +31,13 @@ public class ReindexRequestBuilder extends private final IndexRequestBuilder destination; public ReindexRequestBuilder(ElasticsearchClient client, - Action action) { + ActionType action) { this(client, action, new SearchRequestBuilder(client, SearchAction.INSTANCE), new IndexRequestBuilder(client, IndexAction.INSTANCE)); } private ReindexRequestBuilder(ElasticsearchClient client, - Action action, + ActionType action, SearchRequestBuilder search, IndexRequestBuilder destination) { super(client, action, search, new ReindexRequest(search.request(), destination.request())); this.destination = destination; diff --git a/server/src/main/java/org/elasticsearch/index/reindex/RemoteInfo.java b/server/src/main/java/org/elasticsearch/index/reindex/RemoteInfo.java index 2bca878df8873..10d6138a5338c 100644 --- a/server/src/main/java/org/elasticsearch/index/reindex/RemoteInfo.java +++ b/server/src/main/java/org/elasticsearch/index/reindex/RemoteInfo.java @@ -25,16 +25,24 @@ import org.elasticsearch.common.io.stream.StreamOutput; import org.elasticsearch.common.io.stream.Writeable; import org.elasticsearch.common.unit.TimeValue; +import org.elasticsearch.common.xcontent.DeprecationHandler; +import org.elasticsearch.common.xcontent.NamedXContentRegistry; +import org.elasticsearch.common.xcontent.ToXContent; import org.elasticsearch.common.xcontent.ToXContentObject; +import org.elasticsearch.common.xcontent.XContent; import org.elasticsearch.common.xcontent.XContentBuilder; +import org.elasticsearch.common.xcontent.XContentParser; +import org.elasticsearch.common.xcontent.json.JsonXContent; import java.io.IOException; import java.util.HashMap; import java.util.Map; +import java.util.Objects; import static java.util.Collections.unmodifiableMap; import static java.util.Objects.requireNonNull; import static org.elasticsearch.common.unit.TimeValue.timeValueSeconds; +import static org.elasticsearch.index.query.QueryBuilders.matchAllQuery; public class RemoteInfo implements Writeable, ToXContentObject { /** @@ -46,6 +54,8 @@ public class RemoteInfo implements Writeable, ToXContentObject { */ public static final TimeValue DEFAULT_CONNECT_TIMEOUT = timeValueSeconds(30); + public static final XContent QUERY_CONTENT_TYPE = JsonXContent.jsonXContent; + private final String scheme; private final String host; private final int port; @@ -65,6 +75,7 @@ public class RemoteInfo implements Writeable, ToXContentObject { public RemoteInfo(String scheme, String host, int port, String pathPrefix, BytesReference query, String username, String password, Map headers, TimeValue socketTimeout, TimeValue connectTimeout) { + assert isQueryJson(query) : "Query does not appear to be JSON"; this.scheme = requireNonNull(scheme, "[scheme] must be specified to reindex from a remote cluster"); this.host = requireNonNull(host, "[host] must be specified to reindex from a remote cluster"); this.port = port; @@ -205,4 +216,50 @@ public XContentBuilder toXContent(XContentBuilder builder, Params params) throws builder.endObject(); return builder; } + + @Override + public boolean equals(Object o) { + if (this == o) return true; + if (o == null || getClass() != o.getClass()) return false; + RemoteInfo that = (RemoteInfo) o; + return port == that.port && + Objects.equals(scheme, that.scheme) && + Objects.equals(host, that.host) && + Objects.equals(pathPrefix, that.pathPrefix) && + Objects.equals(query, that.query) && + Objects.equals(username, that.username) && + Objects.equals(password, that.password) && + Objects.equals(headers, that.headers) && + Objects.equals(socketTimeout, that.socketTimeout) && + Objects.equals(connectTimeout, that.connectTimeout); + } + + @Override + public int hashCode() { + return Objects.hash(scheme, host, port, pathPrefix, query, username, password, headers, socketTimeout, connectTimeout); + } + + static BytesReference queryForRemote(Map source) throws IOException { + XContentBuilder builder = XContentBuilder.builder(QUERY_CONTENT_TYPE).prettyPrint(); + Object query = source.remove("query"); + if (query == null) { + return BytesReference.bytes(matchAllQuery().toXContent(builder, ToXContent.EMPTY_PARAMS)); + } + if (!(query instanceof Map)) { + throw new IllegalArgumentException("Expected [query] to be an object but was [" + query + "]"); + } + @SuppressWarnings("unchecked") + Map map = (Map) query; + return BytesReference.bytes(builder.map(map)); + } + + private static boolean isQueryJson(BytesReference bytesReference) { + try (XContentParser parser = QUERY_CONTENT_TYPE.createParser(NamedXContentRegistry.EMPTY, + DeprecationHandler.THROW_UNSUPPORTED_OPERATION, bytesReference.streamInput())) { + Map query = parser.map(); + return true; + } catch (IOException e) { + throw new AssertionError("Could not parse JSON", e); + } + } } diff --git a/server/src/main/java/org/elasticsearch/index/reindex/UpdateByQueryAction.java b/server/src/main/java/org/elasticsearch/index/reindex/UpdateByQueryAction.java index 250a267ea255d..3c0523d34ee56 100644 --- a/server/src/main/java/org/elasticsearch/index/reindex/UpdateByQueryAction.java +++ b/server/src/main/java/org/elasticsearch/index/reindex/UpdateByQueryAction.java @@ -19,9 +19,9 @@ package org.elasticsearch.index.reindex; -import org.elasticsearch.action.Action; +import org.elasticsearch.action.StreamableResponseActionType; -public class UpdateByQueryAction extends Action { +public class UpdateByQueryAction extends StreamableResponseActionType { public static final UpdateByQueryAction INSTANCE = new UpdateByQueryAction(); public static final String NAME = "indices:data/write/update/byquery"; diff --git a/server/src/main/java/org/elasticsearch/index/reindex/UpdateByQueryRequest.java b/server/src/main/java/org/elasticsearch/index/reindex/UpdateByQueryRequest.java index 03922ccc79bd4..138af524bf9c9 100644 --- a/server/src/main/java/org/elasticsearch/index/reindex/UpdateByQueryRequest.java +++ b/server/src/main/java/org/elasticsearch/index/reindex/UpdateByQueryRequest.java @@ -83,19 +83,6 @@ public UpdateByQueryRequest setQuery(QueryBuilder query) { return this; } - /** - * Set the document types for the update - * @deprecated Types are in the process of being removed. Instead of - * using a type, prefer to filter on a field of the document. - */ - @Deprecated - public UpdateByQueryRequest setDocTypes(String... types) { - if (types != null) { - getSearchRequest().types(types); - } - return this; - } - /** * Set routing limiting the process to the shards that match that routing value */ @@ -136,21 +123,6 @@ public String getRouting() { return getSearchRequest().routing(); } - /** - * Gets the document types on which this request would be executed. Returns an empty array if all - * types are to be processed. - * @deprecated Types are in the process of being removed. Instead of - * using a type, prefer to filter on a field of the document. - */ - @Deprecated - public String[] getDocTypes() { - if (getSearchRequest().types() != null) { - return getSearchRequest().types(); - } else { - return new String[0]; - } - } - /** * Ingest pipeline to set on index requests made by this action. */ diff --git a/server/src/main/java/org/elasticsearch/index/reindex/UpdateByQueryRequestBuilder.java b/server/src/main/java/org/elasticsearch/index/reindex/UpdateByQueryRequestBuilder.java index 6b9600dea5eae..91a77e988a881 100644 --- a/server/src/main/java/org/elasticsearch/index/reindex/UpdateByQueryRequestBuilder.java +++ b/server/src/main/java/org/elasticsearch/index/reindex/UpdateByQueryRequestBuilder.java @@ -19,7 +19,7 @@ package org.elasticsearch.index.reindex; -import org.elasticsearch.action.Action; +import org.elasticsearch.action.ActionType; import org.elasticsearch.action.search.SearchAction; import org.elasticsearch.action.search.SearchRequestBuilder; import org.elasticsearch.client.ElasticsearchClient; @@ -27,12 +27,12 @@ public class UpdateByQueryRequestBuilder extends AbstractBulkIndexByScrollRequestBuilder { - public UpdateByQueryRequestBuilder(ElasticsearchClient client, Action action) { + public UpdateByQueryRequestBuilder(ElasticsearchClient client, ActionType action) { this(client, action, new SearchRequestBuilder(client, SearchAction.INSTANCE)); } private UpdateByQueryRequestBuilder(ElasticsearchClient client, - Action action, + ActionType action, SearchRequestBuilder search) { super(client, action, search, new UpdateByQueryRequest(search.request())); } diff --git a/server/src/main/java/org/elasticsearch/index/search/MatchQuery.java b/server/src/main/java/org/elasticsearch/index/search/MatchQuery.java index da7273aa66303..8a43dfbdb3f96 100644 --- a/server/src/main/java/org/elasticsearch/index/search/MatchQuery.java +++ b/server/src/main/java/org/elasticsearch/index/search/MatchQuery.java @@ -29,14 +29,11 @@ import org.apache.lucene.analysis.tokenattributes.TermToBytesRefAttribute; import org.apache.lucene.index.IndexOptions; import org.apache.lucene.index.Term; -import org.apache.lucene.queries.ExtendedCommonTermsQuery; import org.apache.lucene.search.BooleanClause; -import org.apache.lucene.search.BooleanClause.Occur; import org.apache.lucene.search.BooleanQuery; import org.apache.lucene.search.FuzzyQuery; import org.apache.lucene.search.MultiTermQuery; import org.apache.lucene.search.Query; -import org.apache.lucene.search.TermQuery; import org.apache.lucene.search.spans.SpanMultiTermQueryWrapper; import org.apache.lucene.search.spans.SpanNearQuery; import org.apache.lucene.search.spans.SpanOrQuery; @@ -171,8 +168,6 @@ public void writeTo(StreamOutput out) throws IOException { protected ZeroTermsQuery zeroTermsQuery = DEFAULT_ZERO_TERMS_QUERY; - protected Float commonTermsCutoff = null; - protected boolean autoGenerateSynonymsPhraseQuery = true; public MatchQuery(QueryShardContext context) { @@ -194,10 +189,6 @@ public void setOccur(BooleanClause.Occur occur) { this.occur = occur; } - public void setCommonTermsCutoff(Float cutoff) { - this.commonTermsCutoff = cutoff; - } - public void setEnablePositionIncrements(boolean enablePositionIncrements) { this.enablePositionIncrements = enablePositionIncrements; } @@ -247,7 +238,7 @@ public Query parse(Type type, String fieldName, Object value) throws IOException Analyzer analyzer = getAnalyzer(fieldType, type == Type.PHRASE || type == Type.PHRASE_PREFIX); assert analyzer != null; - MatchQueryBuilder builder = new MatchQueryBuilder(analyzer, fieldType); + MatchQueryBuilder builder = new MatchQueryBuilder(analyzer, fieldType, enablePositionIncrements, autoGenerateSynonymsPhraseQuery); /* * If a keyword analyzer is used, we know that further analysis isn't @@ -272,11 +263,7 @@ protected final Query parseInternal(Type type, String fieldName, MatchQueryBuild final Query query; switch (type) { case BOOLEAN: - if (commonTermsCutoff == null) { - query = builder.createBooleanQuery(fieldName, value.toString(), occur); - } else { - query = createCommonTermsQuery(builder, fieldName, value.toString(), occur, occur, commonTermsCutoff); - } + query = builder.createBooleanQuery(fieldName, value.toString(), occur); break; case BOOLEAN_PREFIX: @@ -298,30 +285,6 @@ protected final Query parseInternal(Type type, String fieldName, MatchQueryBuild return query == null ? zeroTermsQuery() : query; } - private Query createCommonTermsQuery(MatchQueryBuilder builder, String field, String queryText, - Occur highFreqOccur, Occur lowFreqOccur, float maxTermFrequency) { - Query booleanQuery = builder.createBooleanQuery(field, queryText, lowFreqOccur); - if (booleanQuery != null && booleanQuery instanceof BooleanQuery) { - BooleanQuery bq = (BooleanQuery) booleanQuery; - return boolToExtendedCommonTermsQuery(bq, highFreqOccur, lowFreqOccur, maxTermFrequency); - } - return booleanQuery; - } - - private Query boolToExtendedCommonTermsQuery(BooleanQuery bq, - Occur highFreqOccur, - Occur lowFreqOccur, - float maxTermFrequency) { - ExtendedCommonTermsQuery query = new ExtendedCommonTermsQuery(highFreqOccur, lowFreqOccur, maxTermFrequency); - for (BooleanClause clause : bq.clauses()) { - if ((clause.getQuery() instanceof TermQuery) == false) { - return bq; - } - query.add(((TermQuery) clause.getQuery()).getTerm()); - } - return query; - } - protected Analyzer getAnalyzer(MappedFieldType fieldType, boolean quoted) { if (analyzer == null) { return quoted ? context.getSearchQuoteAnalyzer(fieldType) : context.getSearchAnalyzer(fieldType); @@ -353,15 +316,16 @@ class MatchQueryBuilder extends QueryBuilder { /** * Creates a new QueryBuilder using the given analyzer. */ - MatchQueryBuilder(Analyzer analyzer, MappedFieldType fieldType) { + MatchQueryBuilder(Analyzer analyzer, MappedFieldType fieldType, + boolean enablePositionIncrements, boolean autoGenerateSynonymsPhraseQuery) { super(analyzer); this.fieldType = fieldType; + setEnablePositionIncrements(enablePositionIncrements); if (hasPositions(fieldType)) { setAutoGenerateMultiTermSynonymsPhraseQuery(autoGenerateSynonymsPhraseQuery); } else { setAutoGenerateMultiTermSynonymsPhraseQuery(false); } - setEnablePositionIncrements(enablePositionIncrements); } @Override diff --git a/server/src/main/java/org/elasticsearch/index/search/MultiMatchQuery.java b/server/src/main/java/org/elasticsearch/index/search/MultiMatchQuery.java index b9943870df727..cbc06a6ff081d 100644 --- a/server/src/main/java/org/elasticsearch/index/search/MultiMatchQuery.java +++ b/server/src/main/java/org/elasticsearch/index/search/MultiMatchQuery.java @@ -138,9 +138,11 @@ private List buildCrossFieldQuery(MultiMatchQueryBuilder.Type type, Map> group : groups.entrySet()) { final MatchQueryBuilder builder; if (group.getValue().size() == 1) { - builder = new MatchQueryBuilder(group.getKey(), group.getValue().get(0).fieldType); + builder = new MatchQueryBuilder(group.getKey(), group.getValue().get(0).fieldType, + enablePositionIncrements, autoGenerateSynonymsPhraseQuery); } else { - builder = new BlendedQueryBuilder(group.getKey(), group.getValue(), tieBreaker); + builder = new BlendedQueryBuilder(group.getKey(), group.getValue(), tieBreaker, + enablePositionIncrements, autoGenerateSynonymsPhraseQuery); } /* @@ -170,8 +172,9 @@ private class BlendedQueryBuilder extends MatchQueryBuilder { private final List blendedFields; private final float tieBreaker; - BlendedQueryBuilder(Analyzer analyzer, List blendedFields, float tieBreaker) { - super(analyzer, blendedFields.get(0).fieldType); + BlendedQueryBuilder(Analyzer analyzer, List blendedFields, float tieBreaker, + boolean enablePositionIncrements, boolean autoGenerateSynonymsPhraseQuery) { + super(analyzer, blendedFields.get(0).fieldType, enablePositionIncrements, autoGenerateSynonymsPhraseQuery); this.blendedFields = blendedFields; this.tieBreaker = tieBreaker; } @@ -182,12 +185,12 @@ protected Query newSynonymQuery(Term[] terms) { for (int i = 0; i < terms.length; i++) { values[i] = terms[i].bytes(); } - return blendTerms(context, values, commonTermsCutoff, tieBreaker, lenient, blendedFields); + return blendTerms(context, values, tieBreaker, lenient, blendedFields); } @Override protected Query newTermQuery(Term term) { - return blendTerm(context, term.bytes(), commonTermsCutoff, tieBreaker, lenient, blendedFields); + return blendTerm(context, term.bytes(), tieBreaker, lenient, blendedFields); } @Override @@ -230,13 +233,13 @@ protected Query analyzeMultiPhrase(String field, TokenStream stream, int slop) t } } - static Query blendTerm(QueryShardContext context, BytesRef value, Float commonTermsCutoff, float tieBreaker, + static Query blendTerm(QueryShardContext context, BytesRef value, float tieBreaker, boolean lenient, List blendedFields) { - return blendTerms(context, new BytesRef[] {value}, commonTermsCutoff, tieBreaker, lenient, blendedFields); + return blendTerms(context, new BytesRef[] {value}, tieBreaker, lenient, blendedFields); } - static Query blendTerms(QueryShardContext context, BytesRef[] values, Float commonTermsCutoff, float tieBreaker, + static Query blendTerms(QueryShardContext context, BytesRef[] values, float tieBreaker, boolean lenient, List blendedFields) { List queries = new ArrayList<>(); @@ -276,11 +279,7 @@ static Query blendTerms(QueryShardContext context, BytesRef[] values, Float comm if (i > 0) { terms = Arrays.copyOf(terms, i); blendedBoost = Arrays.copyOf(blendedBoost, i); - if (commonTermsCutoff != null) { - queries.add(BlendedTermQuery.commonTermsBlendedQuery(terms, blendedBoost, commonTermsCutoff)); - } else { - queries.add(BlendedTermQuery.dismaxBlendedQuery(terms, blendedBoost, tieBreaker)); - } + queries.add(BlendedTermQuery.dismaxBlendedQuery(terms, blendedBoost, tieBreaker)); } if (queries.size() == 1) { return queries.get(0); diff --git a/server/src/main/java/org/elasticsearch/index/search/NestedHelper.java b/server/src/main/java/org/elasticsearch/index/search/NestedHelper.java index 1c17fa0cb935f..ab2ae044bdbaf 100644 --- a/server/src/main/java/org/elasticsearch/index/search/NestedHelper.java +++ b/server/src/main/java/org/elasticsearch/index/search/NestedHelper.java @@ -192,7 +192,7 @@ boolean mightMatchNonNestedDocs(String field, String nestedPath) { return true; // the field is not a sub field of the nested path } - private static String parentObject(String field) { + public static String parentObject(String field) { int lastDot = field.lastIndexOf('.'); if (lastDot == -1) { return null; diff --git a/server/src/main/java/org/elasticsearch/index/search/QueryParserHelper.java b/server/src/main/java/org/elasticsearch/index/search/QueryParserHelper.java index 3acf2929687c5..8d6198e17e2cc 100644 --- a/server/src/main/java/org/elasticsearch/index/search/QueryParserHelper.java +++ b/server/src/main/java/org/elasticsearch/index/search/QueryParserHelper.java @@ -30,6 +30,7 @@ import java.util.HashMap; import java.util.List; import java.util.Map; +import java.util.Set; /** * Helpers to extract and expand field names and boosts @@ -55,6 +56,10 @@ public static Map parseFieldsAndWeights(List fields) { } else { fieldName = field; } + // handle duplicates + if (fieldsAndWeights.containsKey(field)) { + boost *= fieldsAndWeights.get(field); + } fieldsAndWeights.put(fieldName, boost); } return fieldsAndWeights; @@ -84,7 +89,13 @@ public static Map resolveMappingFields(QueryShardContext context, float weight = fieldEntry.getValue() == null ? 1.0f : fieldEntry.getValue(); Map fieldMap = resolveMappingField(context, fieldEntry.getKey(), weight, !multiField, !allField, fieldSuffix); - resolvedFields.putAll(fieldMap); + for (Map.Entry field : fieldMap.entrySet()) { + float boost = field.getValue(); + if (resolvedFields.containsKey(field.getKey())) { + boost *= resolvedFields.get(field.getKey()); + } + resolvedFields.put(field.getKey(), boost); + } } checkForTooManyFields(resolvedFields, context); return resolvedFields; @@ -120,8 +131,9 @@ public static Map resolveMappingField(QueryShardContext context, */ public static Map resolveMappingField(QueryShardContext context, String fieldOrPattern, float weight, boolean acceptAllTypes, boolean acceptMetadataField, String fieldSuffix) { - Collection allFields = context.simpleMatchToIndexNames(fieldOrPattern); + Set allFields = context.simpleMatchToIndexNames(fieldOrPattern); Map fields = new HashMap<>(); + for (String fieldName : allFields) { if (fieldSuffix != null && context.fieldMapper(fieldName + fieldSuffix) != null) { fieldName = fieldName + fieldSuffix; @@ -149,8 +161,17 @@ public static Map resolveMappingField(QueryShardContext context, // other exceptions are parsing errors or not indexed fields: keep } } - fields.put(fieldName, weight); + + // Deduplicate aliases and their concrete fields. + String resolvedFieldName = fieldType.name(); + if (allFields.contains(resolvedFieldName)) { + fieldName = resolvedFieldName; + } + + float w = fields.getOrDefault(fieldName, 1.0F); + fields.put(fieldName, w * weight); } + checkForTooManyFields(fields, context); return fields; } diff --git a/server/src/main/java/org/elasticsearch/index/search/QueryStringQueryParser.java b/server/src/main/java/org/elasticsearch/index/search/QueryStringQueryParser.java index 7bec4fc87f1a9..22be2131e3347 100644 --- a/server/src/main/java/org/elasticsearch/index/search/QueryStringQueryParser.java +++ b/server/src/main/java/org/elasticsearch/index/search/QueryStringQueryParser.java @@ -153,6 +153,12 @@ private QueryStringQueryParser(QueryShardContext context, String defaultField, this.lenient = lenient; } + @Override + public void setEnablePositionIncrements(boolean enable) { + super.setEnablePositionIncrements(enable); + queryBuilder.setEnablePositionIncrements(enable); + } + @Override public void setDefaultOperator(Operator op) { super.setDefaultOperator(op); @@ -423,7 +429,8 @@ protected Query handleBareFuzzy(String field, Token fuzzySlop, String termImage) if (fuzzySlop.image.length() == 1) { return getFuzzyQuery(field, termImage, fuzziness.asDistance(termImage)); } - return getFuzzyQuery(field, termImage, Fuzziness.fromString(fuzzySlop.image.substring(1)).asFloat()); + float distance = Fuzziness.fromString(fuzzySlop.image.substring(1)).asDistance(termImage); + return getFuzzyQuery(field, termImage, distance); } @Override diff --git a/server/src/main/java/org/elasticsearch/index/seqno/GlobalCheckpointSyncAction.java b/server/src/main/java/org/elasticsearch/index/seqno/GlobalCheckpointSyncAction.java index d67cbc833d666..a7f29d14b49b8 100644 --- a/server/src/main/java/org/elasticsearch/index/seqno/GlobalCheckpointSyncAction.java +++ b/server/src/main/java/org/elasticsearch/index/seqno/GlobalCheckpointSyncAction.java @@ -19,10 +19,9 @@ package org.elasticsearch.index.seqno; -import org.apache.logging.log4j.message.ParameterizedMessage; -import org.apache.lucene.store.AlreadyClosedException; -import org.elasticsearch.ExceptionsHelper; import org.elasticsearch.action.ActionListener; +import org.elasticsearch.action.ActionType; +import org.elasticsearch.action.StreamableResponseActionType; import org.elasticsearch.action.support.ActionFilters; import org.elasticsearch.action.support.replication.ReplicationRequest; import org.elasticsearch.action.support.replication.ReplicationResponse; @@ -33,9 +32,7 @@ import org.elasticsearch.common.inject.Inject; import org.elasticsearch.common.io.stream.StreamInput; import org.elasticsearch.common.settings.Settings; -import org.elasticsearch.common.util.concurrent.ThreadContext; import org.elasticsearch.index.shard.IndexShard; -import org.elasticsearch.index.shard.IndexShardClosedException; import org.elasticsearch.index.shard.ShardId; import org.elasticsearch.index.translog.Translog; import org.elasticsearch.indices.IndicesService; @@ -55,6 +52,12 @@ public class GlobalCheckpointSyncAction extends TransportReplicationAction< ReplicationResponse> { public static String ACTION_NAME = "indices:admin/seq_no/global_checkpoint_sync"; + public static ActionType TYPE = new StreamableResponseActionType<>(ACTION_NAME) { + @Override + public ReplicationResponse newResponse() { + return new ReplicationResponse(); + } + }; @Inject public GlobalCheckpointSyncAction( @@ -81,21 +84,6 @@ public GlobalCheckpointSyncAction( ThreadPool.Names.MANAGEMENT); } - public void updateGlobalCheckpointForShard(final ShardId shardId) { - final ThreadContext threadContext = threadPool.getThreadContext(); - try (ThreadContext.StoredContext ignore = threadContext.stashContext()) { - threadContext.markAsSystemContext(); - execute( - new Request(shardId), - ActionListener.wrap(r -> { - }, e -> { - if (ExceptionsHelper.unwrap(e, AlreadyClosedException.class, IndexShardClosedException.class) == null) { - logger.info(new ParameterizedMessage("{} global checkpoint sync failed", shardId), e); - } - })); - } - } - @Override protected ReplicationResponse newResponseInstance() { return new ReplicationResponse(); @@ -118,7 +106,7 @@ protected ReplicaResult shardOperationOnReplica(final Request request, final Ind private void maybeSyncTranslog(final IndexShard indexShard) throws IOException { if (indexShard.getTranslogDurability() == Translog.Durability.REQUEST && - indexShard.getLastSyncedGlobalCheckpoint() < indexShard.getGlobalCheckpoint()) { + indexShard.getLastSyncedGlobalCheckpoint() < indexShard.getLastKnownGlobalCheckpoint()) { indexShard.sync(); } } diff --git a/server/src/main/java/org/elasticsearch/index/seqno/LocalCheckpointTracker.java b/server/src/main/java/org/elasticsearch/index/seqno/LocalCheckpointTracker.java index a19d9ac4abb94..185d3b2ad258c 100644 --- a/server/src/main/java/org/elasticsearch/index/seqno/LocalCheckpointTracker.java +++ b/server/src/main/java/org/elasticsearch/index/seqno/LocalCheckpointTracker.java @@ -22,6 +22,8 @@ import com.carrotsearch.hppc.LongObjectHashMap; import org.elasticsearch.common.SuppressForbidden; +import java.util.concurrent.atomic.AtomicLong; + /** * This class generates sequences numbers and keeps track of the so-called "local checkpoint" which is the highest number for which all * previous sequence numbers have been processed (inclusive). @@ -35,20 +37,31 @@ public class LocalCheckpointTracker { static final short BIT_SET_SIZE = 1024; /** - * A collection of bit sets representing pending sequence numbers. Each sequence number is mapped to a bit set by dividing by the + * A collection of bit sets representing processed sequence numbers. Each sequence number is mapped to a bit set by dividing by the * bit set size. */ final LongObjectHashMap processedSeqNo = new LongObjectHashMap<>(); /** - * The current local checkpoint, i.e., all sequence numbers no more than this number have been completed. + * A collection of bit sets representing durably persisted sequence numbers. Each sequence number is mapped to a bit set by dividing by + * the bit set size. + */ + final LongObjectHashMap persistedSeqNo = new LongObjectHashMap<>(); + + /** + * The current local checkpoint, i.e., all sequence numbers no more than this number have been processed. + */ + final AtomicLong processedCheckpoint = new AtomicLong(); + + /** + * The current persisted local checkpoint, i.e., all sequence numbers no more than this number have been durably persisted. */ - volatile long checkpoint; + final AtomicLong persistedCheckpoint = new AtomicLong(); /** * The next available sequence number. */ - private volatile long nextSeqNo; + final AtomicLong nextSeqNo = new AtomicLong(); /** * Initialize the local checkpoint service. The {@code maxSeqNo} should be set to the last sequence number assigned, or @@ -68,8 +81,9 @@ public LocalCheckpointTracker(final long maxSeqNo, final long localCheckpoint) { throw new IllegalArgumentException( "max seq. no. must be non-negative or [" + SequenceNumbers.NO_OPS_PERFORMED + "] but was [" + maxSeqNo + "]"); } - nextSeqNo = maxSeqNo == SequenceNumbers.NO_OPS_PERFORMED ? 0 : maxSeqNo + 1; - checkpoint = localCheckpoint; + nextSeqNo.set(maxSeqNo + 1); + processedCheckpoint.set(localCheckpoint); + persistedCheckpoint.set(localCheckpoint); } /** @@ -77,48 +91,67 @@ public LocalCheckpointTracker(final long maxSeqNo, final long localCheckpoint) { * * @return the next assigned sequence number */ - public synchronized long generateSeqNo() { - return nextSeqNo++; + public long generateSeqNo() { + return nextSeqNo.getAndIncrement(); } /** * Marks the provided sequence number as seen and updates the max_seq_no if needed. */ - public synchronized void advanceMaxSeqNo(long seqNo) { - if (seqNo >= nextSeqNo) { - nextSeqNo = seqNo + 1; - } + public void advanceMaxSeqNo(final long seqNo) { + nextSeqNo.accumulateAndGet(seqNo + 1, Math::max); } /** - * Marks the processing of the provided sequence number as completed as updates the checkpoint if possible. + * Marks the provided sequence number as processed and updates the processed checkpoint if possible. * - * @param seqNo the sequence number to mark as completed + * @param seqNo the sequence number to mark as processed */ - public synchronized void markSeqNoAsCompleted(final long seqNo) { + public synchronized void markSeqNoAsProcessed(final long seqNo) { + markSeqNo(seqNo, processedCheckpoint, processedSeqNo); + } + + /** + * Marks the provided sequence number as persisted and updates the checkpoint if possible. + * + * @param seqNo the sequence number to mark as persisted + */ + public synchronized void markSeqNoAsPersisted(final long seqNo) { + markSeqNo(seqNo, persistedCheckpoint, persistedSeqNo); + } + + private void markSeqNo(final long seqNo, final AtomicLong checkPoint, final LongObjectHashMap bitSetMap) { + assert Thread.holdsLock(this); // make sure we track highest seen sequence number - if (seqNo >= nextSeqNo) { - nextSeqNo = seqNo + 1; - } - if (seqNo <= checkpoint) { + advanceMaxSeqNo(seqNo); + if (seqNo <= checkPoint.get()) { // this is possible during recovery where we might replay an operation that was also replicated return; } - final CountedBitSet bitSet = getBitSetForSeqNo(seqNo); + final CountedBitSet bitSet = getBitSetForSeqNo(bitSetMap, seqNo); final int offset = seqNoToBitSetOffset(seqNo); bitSet.set(offset); - if (seqNo == checkpoint + 1) { - updateCheckpoint(); + if (seqNo == checkPoint.get() + 1) { + updateCheckpoint(checkPoint, bitSetMap); } } /** - * The current checkpoint which can be advanced by {@link #markSeqNoAsCompleted(long)}. + * The current checkpoint which can be advanced by {@link #markSeqNoAsProcessed(long)}. * * @return the current checkpoint */ - public long getCheckpoint() { - return checkpoint; + public long getProcessedCheckpoint() { + return processedCheckpoint.get(); + } + + /** + * The current persisted checkpoint which can be advanced by {@link #markSeqNoAsPersisted(long)}. + * + * @return the current persisted checkpoint + */ + public long getPersistedCheckpoint() { + return persistedCheckpoint.get(); } /** @@ -127,17 +160,17 @@ public long getCheckpoint() { * @return the maximum sequence number */ public long getMaxSeqNo() { - return nextSeqNo - 1; + return nextSeqNo.get() - 1; } /** * constructs a {@link SeqNoStats} object, using local state and the supplied global checkpoint * - * This is needed to make sure the local checkpoint and max seq no are consistent + * This is needed to make sure the persisted local checkpoint and max seq no are consistent */ public synchronized SeqNoStats getStats(final long globalCheckpoint) { - return new SeqNoStats(getMaxSeqNo(), getCheckpoint(), globalCheckpoint); + return new SeqNoStats(getMaxSeqNo(), getPersistedCheckpoint(), globalCheckpoint); } /** @@ -147,70 +180,74 @@ public synchronized SeqNoStats getStats(final long globalCheckpoint) { * @throws InterruptedException if the thread was interrupted while blocking on the condition */ @SuppressForbidden(reason = "Object#wait") - public synchronized void waitForOpsToComplete(final long seqNo) throws InterruptedException { - while (checkpoint < seqNo) { + public synchronized void waitForProcessedOpsToComplete(final long seqNo) throws InterruptedException { + while (processedCheckpoint.get() < seqNo) { // notified by updateCheckpoint this.wait(); } } /** - * Checks if the given sequence number was marked as completed in this tracker. + * Checks if the given sequence number was marked as processed in this tracker. */ - public boolean contains(final long seqNo) { + public boolean hasProcessed(final long seqNo) { assert seqNo >= 0 : "invalid seq_no=" + seqNo; - if (seqNo >= nextSeqNo) { + if (seqNo >= nextSeqNo.get()) { return false; } - if (seqNo <= checkpoint) { + if (seqNo <= processedCheckpoint.get()) { return true; } final long bitSetKey = getBitSetKey(seqNo); final int bitSetOffset = seqNoToBitSetOffset(seqNo); synchronized (this) { + // check again under lock + if (seqNo <= processedCheckpoint.get()) { + return true; + } final CountedBitSet bitSet = processedSeqNo.get(bitSetKey); return bitSet != null && bitSet.get(bitSetOffset); } } /** - * Moves the checkpoint to the last consecutively processed sequence number. This method assumes that the sequence number following the - * current checkpoint is processed. + * Moves the checkpoint to the last consecutively processed sequence number. This method assumes that the sequence number + * following the current checkpoint is processed. */ @SuppressForbidden(reason = "Object#notifyAll") - private void updateCheckpoint() { + private void updateCheckpoint(AtomicLong checkPoint, LongObjectHashMap bitSetMap) { assert Thread.holdsLock(this); - assert getBitSetForSeqNo(checkpoint + 1).get(seqNoToBitSetOffset(checkpoint + 1)) : + assert getBitSetForSeqNo(bitSetMap, checkPoint.get() + 1).get(seqNoToBitSetOffset(checkPoint.get() + 1)) : "updateCheckpoint is called but the bit following the checkpoint is not set"; try { // keep it simple for now, get the checkpoint one by one; in the future we can optimize and read words - long bitSetKey = getBitSetKey(checkpoint); - CountedBitSet current = processedSeqNo.get(bitSetKey); + long bitSetKey = getBitSetKey(checkPoint.get()); + CountedBitSet current = bitSetMap.get(bitSetKey); if (current == null) { // the bit set corresponding to the checkpoint has already been removed, set ourselves up for the next bit set - assert checkpoint % BIT_SET_SIZE == BIT_SET_SIZE - 1; - current = processedSeqNo.get(++bitSetKey); + assert checkPoint.get() % BIT_SET_SIZE == BIT_SET_SIZE - 1; + current = bitSetMap.get(++bitSetKey); } do { - checkpoint++; + checkPoint.incrementAndGet(); /* * The checkpoint always falls in the current bit set or we have already cleaned it; if it falls on the last bit of the * current bit set, we can clean it. */ - if (checkpoint == lastSeqNoInBitSet(bitSetKey)) { + if (checkPoint.get() == lastSeqNoInBitSet(bitSetKey)) { assert current != null; - final CountedBitSet removed = processedSeqNo.remove(bitSetKey); + final CountedBitSet removed = bitSetMap.remove(bitSetKey); assert removed == current; - current = processedSeqNo.get(++bitSetKey); + current = bitSetMap.get(++bitSetKey); } - } while (current != null && current.get(seqNoToBitSetOffset(checkpoint + 1))); + } while (current != null && current.get(seqNoToBitSetOffset(checkPoint.get() + 1))); } finally { - // notifies waiters in waitForOpsToComplete + // notifies waiters in waitForProcessedOpsToComplete this.notifyAll(); } } - private long lastSeqNoInBitSet(final long bitSetKey) { + private static long lastSeqNoInBitSet(final long bitSetKey) { return (1 + bitSetKey) * BIT_SET_SIZE - 1; } @@ -220,32 +257,32 @@ private long lastSeqNoInBitSet(final long bitSetKey) { * @param seqNo the sequence number to obtain the bit set for * @return the bit set corresponding to the provided sequence number */ - private long getBitSetKey(final long seqNo) { + private static long getBitSetKey(final long seqNo) { return seqNo / BIT_SET_SIZE; } - private CountedBitSet getBitSetForSeqNo(final long seqNo) { + private CountedBitSet getBitSetForSeqNo(final LongObjectHashMap bitSetMap, final long seqNo) { assert Thread.holdsLock(this); final long bitSetKey = getBitSetKey(seqNo); - final int index = processedSeqNo.indexOf(bitSetKey); + final int index = bitSetMap.indexOf(bitSetKey); final CountedBitSet bitSet; - if (processedSeqNo.indexExists(index)) { - bitSet = processedSeqNo.indexGet(index); + if (bitSetMap.indexExists(index)) { + bitSet = bitSetMap.indexGet(index); } else { bitSet = new CountedBitSet(BIT_SET_SIZE); - processedSeqNo.indexInsert(index, bitSetKey, bitSet); + bitSetMap.indexInsert(index, bitSetKey, bitSet); } return bitSet; } /** * Obtain the position in the bit set corresponding to the provided sequence number. The bit set corresponding to the sequence number - * can be obtained via {@link #getBitSetForSeqNo(long)}. + * can be obtained via {@link #getBitSetForSeqNo(LongObjectHashMap, long)}. * * @param seqNo the sequence number to obtain the position for * @return the position in the bit set corresponding to the provided sequence number */ - private int seqNoToBitSetOffset(final long seqNo) { + private static int seqNoToBitSetOffset(final long seqNo) { return Math.toIntExact(seqNo % BIT_SET_SIZE); } diff --git a/server/src/main/java/org/elasticsearch/index/seqno/ReplicationTracker.java b/server/src/main/java/org/elasticsearch/index/seqno/ReplicationTracker.java index 892056674019f..20e998b1e12a6 100644 --- a/server/src/main/java/org/elasticsearch/index/seqno/ReplicationTracker.java +++ b/server/src/main/java/org/elasticsearch/index/seqno/ReplicationTracker.java @@ -105,12 +105,13 @@ public class ReplicationTracker extends AbstractIndexShardComponent implements L private volatile long operationPrimaryTerm; /** - * Boolean flag that indicates if a relocation handoff is in progress. A handoff is started by calling {@link #startRelocationHandoff} - * and is finished by either calling {@link #completeRelocationHandoff} or {@link #abortRelocationHandoff}, depending on whether the - * handoff was successful or not. During the handoff, which has as main objective to transfer the internal state of the global - * checkpoint tracker from the relocation source to the target, the list of in-sync shard copies cannot grow, otherwise the relocation - * target might miss this information and increase the global checkpoint to eagerly. As consequence, some of the methods in this class - * are not allowed to be called while a handoff is in progress, in particular {@link #markAllocationIdAsInSync}. + * Boolean flag that indicates if a relocation handoff is in progress. A handoff is started by calling + * {@link #startRelocationHandoff(String)} and is finished by either calling {@link #completeRelocationHandoff} or + * {@link #abortRelocationHandoff}, depending on whether the handoff was successful or not. During the handoff, which has as main + * objective to transfer the internal state of the global checkpoint tracker from the relocation source to the target, the list of + * in-sync shard copies cannot grow, otherwise the relocation target might miss this information and increase the global checkpoint + * to eagerly. As consequence, some of the methods in this class are not allowed to be called while a handoff is in progress, + * in particular {@link #markAllocationIdAsInSync}. * * A notable exception to this is the method {@link #updateFromMaster}, which is still allowed to be called during a relocation handoff. * The reason for this is that the handoff might fail and can be aborted (using {@link #abortRelocationHandoff}), in which case @@ -146,9 +147,15 @@ public class ReplicationTracker extends AbstractIndexShardComponent implements L final Map checkpoints; /** - * A callback invoked when the global checkpoint is updated. For primary mode this occurs if the computed global checkpoint advances on - * the basis of state changes tracked here. For non-primary mode this occurs if the local knowledge of the global checkpoint advances - * due to an update from the primary. + * The current in-memory global checkpoint. In primary mode, this is a cached version of the checkpoint computed from the local + * checkpoints. In replica mode, this is the in-memory global checkpoint that's communicated by the primary. + */ + volatile long globalCheckpoint; + + /** + * A callback invoked when the in-memory global checkpoint is updated. For primary mode this occurs if the computed global checkpoint + * advances on the basis of state changes tracked here. For non-primary mode this occurs if the local knowledge of the global checkpoint + * advances due to an update from the primary. */ private final LongConsumer onGlobalCheckpointUpdated; @@ -221,12 +228,15 @@ public synchronized Tuple getRetentionLeases(final boo .leases() .stream() .collect(Collectors.groupingBy(lease -> currentTimeMillis - lease.timestamp() > retentionLeaseMillis)); - if (partitionByExpiration.get(true) == null) { + final Collection expiredLeases = partitionByExpiration.get(true); + if (expiredLeases == null) { // early out as no retention leases have expired + logger.debug("no retention leases are expired from current retention leases [{}]", retentionLeases); return Tuple.tuple(false, retentionLeases); } final Collection nonExpiredLeases = partitionByExpiration.get(false) != null ? partitionByExpiration.get(false) : Collections.emptyList(); + logger.debug("expiring retention leases [{}] from current retention leases [{}]", expiredLeases, retentionLeases); retentionLeases = new RetentionLeases(operationPrimaryTerm, retentionLeases.version() + 1, nonExpiredLeases); return Tuple.tuple(true, retentionLeases); } @@ -255,6 +265,7 @@ public RetentionLease addRetentionLease( throw new RetentionLeaseAlreadyExistsException(id); } retentionLease = new RetentionLease(id, retainingSequenceNumber, currentTimeMillisSupplier.getAsLong(), source); + logger.debug("adding new retention lease [{}] to current retention leases [{}]", retentionLease, retentionLeases); retentionLeases = new RetentionLeases( operationPrimaryTerm, retentionLeases.version() + 1, @@ -312,6 +323,7 @@ public void removeRetentionLease(final String id, final ActionListener getInSyncGlobalCheckpoints() { assert primaryMode; @@ -530,24 +545,11 @@ public boolean isRelocated() { * as a logical operator, many of the invariants are written under the form (!A || B), they should be read as (A implies B) however. */ private boolean invariant() { - assert checkpoints.get(shardAllocationId) != null : - "checkpoints map should always have an entry for the current shard"; - // local checkpoints only set during primary mode - assert primaryMode || checkpoints.values().stream() - .allMatch(lcps -> lcps.localCheckpoint == SequenceNumbers.UNASSIGNED_SEQ_NO || - lcps.localCheckpoint == SequenceNumbers.PRE_60_NODE_CHECKPOINT); + assert primaryMode || checkpoints.values().stream().allMatch(lcps -> lcps.localCheckpoint == SequenceNumbers.UNASSIGNED_SEQ_NO); - // global checkpoints for other shards only set during primary mode - assert primaryMode - || checkpoints - .entrySet() - .stream() - .filter(e -> e.getKey().equals(shardAllocationId) == false) - .map(Map.Entry::getValue) - .allMatch(cps -> - (cps.globalCheckpoint == SequenceNumbers.UNASSIGNED_SEQ_NO - || cps.globalCheckpoint == SequenceNumbers.PRE_60_NODE_CHECKPOINT)); + // global checkpoints only set during primary mode + assert primaryMode || checkpoints.values().stream().allMatch(cps -> cps.globalCheckpoint == SequenceNumbers.UNASSIGNED_SEQ_NO); // relocation handoff can only occur in primary mode assert !handoffInProgress || primaryMode; @@ -576,14 +578,14 @@ private boolean invariant() { // the computed global checkpoint is always up-to-date assert !primaryMode - || getGlobalCheckpoint() == computeGlobalCheckpoint(pendingInSync, checkpoints.values(), getGlobalCheckpoint()) + || globalCheckpoint == computeGlobalCheckpoint(pendingInSync, checkpoints.values(), globalCheckpoint) : "global checkpoint is not up-to-date, expected: " + - computeGlobalCheckpoint(pendingInSync, checkpoints.values(), getGlobalCheckpoint()) + " but was: " + getGlobalCheckpoint(); + computeGlobalCheckpoint(pendingInSync, checkpoints.values(), globalCheckpoint) + " but was: " + globalCheckpoint; // when in primary mode, the global checkpoint is at most the minimum local checkpoint on all in-sync shard copies assert !primaryMode - || getGlobalCheckpoint() <= inSyncCheckpointStates(checkpoints, CheckpointState::getLocalCheckpoint, LongStream::min) - : "global checkpoint [" + getGlobalCheckpoint() + "] " + || globalCheckpoint <= inSyncCheckpointStates(checkpoints, CheckpointState::getLocalCheckpoint, LongStream::min) + : "global checkpoint [" + globalCheckpoint + "] " + "for primary mode allocation ID [" + shardAllocationId + "] " + "more than in-sync local checkpoints [" + checkpoints + "]"; @@ -626,7 +628,7 @@ private static long inSyncCheckpointStates( .stream() .filter(cps -> cps.inSync) .mapToLong(function) - .filter(v -> v != SequenceNumbers.PRE_60_NODE_CHECKPOINT && v != SequenceNumbers.UNASSIGNED_SEQ_NO)); + .filter(v -> v != SequenceNumbers.UNASSIGNED_SEQ_NO)); return value.isPresent() ? value.getAsLong() : SequenceNumbers.UNASSIGNED_SEQ_NO; } @@ -657,8 +659,8 @@ public ReplicationTracker( this.operationPrimaryTerm = operationPrimaryTerm; this.handoffInProgress = false; this.appliedClusterStateVersion = -1L; + this.globalCheckpoint = globalCheckpoint; this.checkpoints = new HashMap<>(1 + indexSettings.getNumberOfReplicas()); - checkpoints.put(allocationId, new CheckpointState(SequenceNumbers.UNASSIGNED_SEQ_NO, globalCheckpoint, false, false)); this.onGlobalCheckpointUpdated = Objects.requireNonNull(onGlobalCheckpointUpdated); this.currentTimeMillisSupplier = Objects.requireNonNull(currentTimeMillisSupplier); this.onSyncRetentionLeases = Objects.requireNonNull(onSyncRetentionLeases); @@ -685,28 +687,26 @@ private ReplicationGroup calculateReplicationGroup() { } /** - * Returns the global checkpoint for the shard. + * Returns the in-memory global checkpoint for the shard. * * @return the global checkpoint */ - public synchronized long getGlobalCheckpoint() { - final CheckpointState cps = checkpoints.get(shardAllocationId); - assert cps != null; - return cps.globalCheckpoint; + public long getGlobalCheckpoint() { + return globalCheckpoint; } @Override public long getAsLong() { - return getGlobalCheckpoint(); + return globalCheckpoint; } /** * Updates the global checkpoint on a replica shard after it has been updated by the primary. * - * @param globalCheckpoint the global checkpoint - * @param reason the reason the global checkpoint was updated + * @param newGlobalCheckpoint the new global checkpoint + * @param reason the reason the global checkpoint was updated */ - public synchronized void updateGlobalCheckpointOnReplica(final long globalCheckpoint, final String reason) { + public synchronized void updateGlobalCheckpointOnReplica(final long newGlobalCheckpoint, final String reason) { assert invariant(); assert primaryMode == false; /* @@ -715,18 +715,17 @@ public synchronized void updateGlobalCheckpointOnReplica(final long globalCheckp * replica shards). In these cases, the local knowledge of the global checkpoint could be higher than the sync from the lagging * primary. */ - updateGlobalCheckpoint( - shardAllocationId, - globalCheckpoint, - current -> { - logger.trace("updated global checkpoint from [{}] to [{}] due to [{}]", current, globalCheckpoint, reason); - onGlobalCheckpointUpdated.accept(globalCheckpoint); - }); + final long previousGlobalCheckpoint = globalCheckpoint; + if (newGlobalCheckpoint > previousGlobalCheckpoint) { + globalCheckpoint = newGlobalCheckpoint; + logger.trace("updated global checkpoint from [{}] to [{}] due to [{}]", previousGlobalCheckpoint, globalCheckpoint, reason); + onGlobalCheckpointUpdated.accept(globalCheckpoint); + } assert invariant(); } /** - * Update the local knowledge of the global checkpoint for the specified allocation ID. + * Update the local knowledge of the persisted global checkpoint for the specified allocation ID. * * @param allocationId the allocation ID to update the global checkpoint for * @param globalCheckpoint the global checkpoint @@ -735,24 +734,15 @@ public synchronized void updateGlobalCheckpointForShard(final String allocationI assert primaryMode; assert handoffInProgress == false; assert invariant(); - updateGlobalCheckpoint( - allocationId, - globalCheckpoint, - current -> logger.trace( - "updated local knowledge for [{}] on the primary of the global checkpoint from [{}] to [{}]", - allocationId, - current, - globalCheckpoint)); - assert invariant(); - } - - private void updateGlobalCheckpoint(final String allocationId, final long globalCheckpoint, LongConsumer ifUpdated) { final CheckpointState cps = checkpoints.get(allocationId); assert !this.shardAllocationId.equals(allocationId) || cps != null; if (cps != null && globalCheckpoint > cps.globalCheckpoint) { + final long previousGlobalCheckpoint = cps.globalCheckpoint; cps.globalCheckpoint = globalCheckpoint; - ifUpdated.accept(cps.globalCheckpoint); + logger.trace("updated local knowledge for [{}] on the primary of the global checkpoint from [{}] to [{}]", + allocationId, previousGlobalCheckpoint, globalCheckpoint); } + assert invariant(); } /** @@ -810,23 +800,14 @@ public synchronized void updateFromMaster(final long applyingClusterStateVersion } } else { for (String initializingId : initializingAllocationIds) { - if (shardAllocationId.equals(initializingId) == false) { - final long localCheckpoint = SequenceNumbers.UNASSIGNED_SEQ_NO; - final long globalCheckpoint = localCheckpoint; - checkpoints.put(initializingId, new CheckpointState(localCheckpoint, globalCheckpoint, false, false)); - } + final long localCheckpoint = SequenceNumbers.UNASSIGNED_SEQ_NO; + final long globalCheckpoint = localCheckpoint; + checkpoints.put(initializingId, new CheckpointState(localCheckpoint, globalCheckpoint, false, false)); } for (String inSyncId : inSyncAllocationIds) { - if (shardAllocationId.equals(inSyncId)) { - // current shard is initially marked as not in-sync because we don't know better at that point - CheckpointState checkpointState = checkpoints.get(shardAllocationId); - checkpointState.inSync = true; - checkpointState.tracked = true; - } else { - final long localCheckpoint = SequenceNumbers.UNASSIGNED_SEQ_NO; - final long globalCheckpoint = localCheckpoint; - checkpoints.put(inSyncId, new CheckpointState(localCheckpoint, globalCheckpoint, true, true)); - } + final long localCheckpoint = SequenceNumbers.UNASSIGNED_SEQ_NO; + final long globalCheckpoint = localCheckpoint; + checkpoints.put(inSyncId, new CheckpointState(localCheckpoint, globalCheckpoint, true, true)); } } appliedClusterStateVersion = applyingClusterStateVersion; @@ -911,13 +892,9 @@ public synchronized void markAllocationIdAsInSync(final String allocationId, fin } private boolean updateLocalCheckpoint(String allocationId, CheckpointState cps, long localCheckpoint) { - // a local checkpoint of PRE_60_NODE_CHECKPOINT cannot be overridden - assert cps.localCheckpoint != SequenceNumbers.PRE_60_NODE_CHECKPOINT || - localCheckpoint == SequenceNumbers.PRE_60_NODE_CHECKPOINT : - "pre-6.0 shard copy " + allocationId + " unexpected to send valid local checkpoint " + localCheckpoint; - // a local checkpoint for a shard copy should be a valid sequence number or the pre-6.0 sequence number indicator - assert localCheckpoint != SequenceNumbers.UNASSIGNED_SEQ_NO : - "invalid local checkpoint for shard copy [" + allocationId + "]"; + // a local checkpoint for a shard copy should be a valid sequence number + assert localCheckpoint >= SequenceNumbers.NO_OPS_PERFORMED : + "invalid local checkpoint [" + localCheckpoint + "] for shard copy [" + allocationId + "]"; if (localCheckpoint > cps.localCheckpoint) { logger.trace("updated local checkpoint of [{}] from [{}] to [{}]", allocationId, cps.localCheckpoint, localCheckpoint); cps.localCheckpoint = localCheckpoint; @@ -976,8 +953,6 @@ private static long computeGlobalCheckpoint(final Set pendingInSync, fin if (cps.localCheckpoint == SequenceNumbers.UNASSIGNED_SEQ_NO) { // unassigned in-sync replica return fallback; - } else if (cps.localCheckpoint == SequenceNumbers.PRE_60_NODE_CHECKPOINT) { - // 5.x replica, ignore for global checkpoint calculation } else { minLocalCheckpoint = Math.min(cps.localCheckpoint, minLocalCheckpoint); } @@ -992,13 +967,11 @@ private static long computeGlobalCheckpoint(final Set pendingInSync, fin */ private synchronized void updateGlobalCheckpointOnPrimary() { assert primaryMode; - final CheckpointState cps = checkpoints.get(shardAllocationId); - final long globalCheckpoint = cps.globalCheckpoint; final long computedGlobalCheckpoint = computeGlobalCheckpoint(pendingInSync, checkpoints.values(), getGlobalCheckpoint()); assert computedGlobalCheckpoint >= globalCheckpoint : "new global checkpoint [" + computedGlobalCheckpoint + "] is lower than previous one [" + globalCheckpoint + "]"; if (globalCheckpoint != computedGlobalCheckpoint) { - cps.globalCheckpoint = computedGlobalCheckpoint; + globalCheckpoint = computedGlobalCheckpoint; logger.trace("updated global checkpoint to [{}]", computedGlobalCheckpoint); onGlobalCheckpointUpdated.accept(computedGlobalCheckpoint); } @@ -1007,11 +980,15 @@ private synchronized void updateGlobalCheckpointOnPrimary() { /** * Initiates a relocation handoff and returns the corresponding primary context. */ - public synchronized PrimaryContext startRelocationHandoff() { + public synchronized PrimaryContext startRelocationHandoff(String targetAllocationId) { assert invariant(); assert primaryMode; assert handoffInProgress == false; assert pendingInSync.isEmpty() : "relocation handoff started while there are still shard copies pending in-sync: " + pendingInSync; + if (checkpoints.containsKey(targetAllocationId) == false) { + // can happen if the relocation target was removed from cluster but the recovery process isn't aware of that. + throw new IllegalStateException("relocation target [" + targetAllocationId + "] is no longer part of the replication group"); + } handoffInProgress = true; // copy clusterStateVersion and checkpoints and return // all the entries from checkpoints that are inSync: the reason we don't need to care about initializing non-insync entries @@ -1048,20 +1025,10 @@ public synchronized void completeRelocationHandoff() { primaryMode = false; handoffInProgress = false; relocated = true; - // forget all checkpoint information except for global checkpoint of current shard - checkpoints.entrySet().stream().forEach(e -> { - final CheckpointState cps = e.getValue(); - if (cps.localCheckpoint != SequenceNumbers.UNASSIGNED_SEQ_NO && - cps.localCheckpoint != SequenceNumbers.PRE_60_NODE_CHECKPOINT) { - cps.localCheckpoint = SequenceNumbers.UNASSIGNED_SEQ_NO; - } - if (e.getKey().equals(shardAllocationId) == false) { - // don't throw global checkpoint information of current shard away - if (cps.globalCheckpoint != SequenceNumbers.UNASSIGNED_SEQ_NO && - cps.globalCheckpoint != SequenceNumbers.PRE_60_NODE_CHECKPOINT) { - cps.globalCheckpoint = SequenceNumbers.UNASSIGNED_SEQ_NO; - } - } + // forget all checkpoint information + checkpoints.forEach((key, cps) -> { + cps.localCheckpoint = SequenceNumbers.UNASSIGNED_SEQ_NO; + cps.globalCheckpoint = SequenceNumbers.UNASSIGNED_SEQ_NO; }); assert invariant(); } @@ -1107,7 +1074,7 @@ private Runnable getMasterUpdateOperationFromCurrentState() { } /** - * Whether the are shards blocking global checkpoint advancement. Used by tests. + * Whether the are shards blocking global checkpoint advancement. */ public synchronized boolean pendingInSync() { assert primaryMode; diff --git a/server/src/main/java/org/elasticsearch/index/seqno/RetentionLeaseActions.java b/server/src/main/java/org/elasticsearch/index/seqno/RetentionLeaseActions.java index 74c98bf3dca19..bcb1eeeed1e73 100644 --- a/server/src/main/java/org/elasticsearch/index/seqno/RetentionLeaseActions.java +++ b/server/src/main/java/org/elasticsearch/index/seqno/RetentionLeaseActions.java @@ -19,10 +19,10 @@ package org.elasticsearch.index.seqno; -import org.elasticsearch.action.Action; import org.elasticsearch.action.ActionListener; import org.elasticsearch.action.ActionRequestValidationException; import org.elasticsearch.action.ActionResponse; +import org.elasticsearch.action.StreamableResponseActionType; import org.elasticsearch.action.support.ActionFilters; import org.elasticsearch.action.support.single.shard.SingleShardRequest; import org.elasticsearch.action.support.single.shard.TransportSingleShardAction; @@ -44,7 +44,6 @@ import java.io.IOException; import java.util.Objects; -import java.util.function.Supplier; /** * This class holds all actions related to retention leases. Note carefully that these actions are executed under a primary permit. Care is @@ -70,7 +69,7 @@ abstract static class TransportRetentionLeaseAction> extend final ActionFilters actionFilters, final IndexNameExpressionResolver indexNameExpressionResolver, final IndicesService indicesService, - final Supplier requestSupplier) { + final Writeable.Reader requestSupplier) { super( name, threadPool, @@ -124,7 +123,7 @@ protected boolean resolveIndex(final T request) { } - public static class Add extends Action { + public static class Add extends StreamableResponseActionType { public static final Add INSTANCE = new Add(); public static final String ACTION_NAME = "indices:admin/seq_no/add_retention_lease"; @@ -177,7 +176,7 @@ public Response newResponse() { } - public static class Renew extends Action { + public static class Renew extends StreamableResponseActionType { public static final Renew INSTANCE = new Renew(); public static final String ACTION_NAME = "indices:admin/seq_no/renew_retention_lease"; @@ -223,7 +222,7 @@ public Response newResponse() { } - public static class Remove extends Action { + public static class Remove extends StreamableResponseActionType { public static final Remove INSTANCE = new Remove(); public static final String ACTION_NAME = "indices:admin/seq_no/remove_retention_lease"; @@ -272,19 +271,22 @@ public Response newResponse() { private abstract static class Request> extends SingleShardRequest { - private ShardId shardId; + private final ShardId shardId; public ShardId getShardId() { return shardId; } - private String id; + private final String id; public String getId() { return id; } - Request() { + Request(StreamInput in) throws IOException { + super(in); + shardId = new ShardId(in); + id = in.readString(); } Request(final ShardId shardId, final String id) { @@ -298,13 +300,6 @@ public ActionRequestValidationException validate() { return null; } - @Override - public void readFrom(final StreamInput in) throws IOException { - super.readFrom(in); - shardId = new ShardId(in); - id = in.readString(); - } - @Override public void writeTo(final StreamOutput out) throws IOException { super.writeTo(out); @@ -316,19 +311,22 @@ public void writeTo(final StreamOutput out) throws IOException { private abstract static class AddOrRenewRequest> extends Request { - private long retainingSequenceNumber; + private final long retainingSequenceNumber; public long getRetainingSequenceNumber() { return retainingSequenceNumber; } - private String source; + private final String source; public String getSource() { return source; } - AddOrRenewRequest() { + AddOrRenewRequest(StreamInput in) throws IOException { + super(in); + retainingSequenceNumber = in.readZLong(); + source = in.readString(); } AddOrRenewRequest(final ShardId shardId, final String id, final long retainingSequenceNumber, final String source) { @@ -340,13 +338,6 @@ public String getSource() { this.source = Objects.requireNonNull(source); } - @Override - public void readFrom(final StreamInput in) throws IOException { - super.readFrom(in); - retainingSequenceNumber = in.readZLong(); - source = in.readString(); - } - @Override public void writeTo(final StreamOutput out) throws IOException { super.writeTo(out); @@ -358,7 +349,8 @@ public void writeTo(final StreamOutput out) throws IOException { public static class AddRequest extends AddOrRenewRequest { - public AddRequest() { + AddRequest(StreamInput in) throws IOException { + super(in); } public AddRequest(final ShardId shardId, final String id, final long retainingSequenceNumber, final String source) { @@ -369,7 +361,8 @@ public AddRequest(final ShardId shardId, final String id, final long retainingSe public static class RenewRequest extends AddOrRenewRequest { - public RenewRequest() { + RenewRequest(StreamInput in) throws IOException { + super(in); } public RenewRequest(final ShardId shardId, final String id, final long retainingSequenceNumber, final String source) { @@ -380,7 +373,8 @@ public RenewRequest(final ShardId shardId, final String id, final long retaining public static class RemoveRequest extends Request { - public RemoveRequest() { + RemoveRequest(StreamInput in) throws IOException { + super(in); } public RemoveRequest(final ShardId shardId, final String id) { diff --git a/server/src/main/java/org/elasticsearch/index/seqno/RetentionLeaseBackgroundSyncAction.java b/server/src/main/java/org/elasticsearch/index/seqno/RetentionLeaseBackgroundSyncAction.java index 9be2e55057331..184b936784c0c 100644 --- a/server/src/main/java/org/elasticsearch/index/seqno/RetentionLeaseBackgroundSyncAction.java +++ b/server/src/main/java/org/elasticsearch/index/seqno/RetentionLeaseBackgroundSyncAction.java @@ -21,10 +21,9 @@ import org.apache.logging.log4j.LogManager; import org.apache.logging.log4j.Logger; -import org.apache.logging.log4j.message.ParameterizedMessage; -import org.apache.lucene.store.AlreadyClosedException; -import org.elasticsearch.ExceptionsHelper; import org.elasticsearch.action.ActionListener; +import org.elasticsearch.action.ActionType; +import org.elasticsearch.action.StreamableResponseActionType; import org.elasticsearch.action.support.ActionFilters; import org.elasticsearch.action.support.ActiveShardCount; import org.elasticsearch.action.support.replication.ReplicationRequest; @@ -37,10 +36,8 @@ 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.gateway.WriteStateException; import org.elasticsearch.index.shard.IndexShard; -import org.elasticsearch.index.shard.IndexShardClosedException; import org.elasticsearch.index.shard.ShardId; import org.elasticsearch.indices.IndicesService; import org.elasticsearch.threadpool.ThreadPool; @@ -61,6 +58,12 @@ public class RetentionLeaseBackgroundSyncAction extends TransportReplicationActi ReplicationResponse> { public static String ACTION_NAME = "indices:admin/seq_no/retention_lease_background_sync"; + public static ActionType TYPE = new StreamableResponseActionType<>(ACTION_NAME) { + @Override + public ReplicationResponse newResponse() { + return new ReplicationResponse(); + } + }; private static final Logger LOGGER = LogManager.getLogger(RetentionLeaseSyncAction.class); @@ -93,39 +96,6 @@ public RetentionLeaseBackgroundSyncAction( ThreadPool.Names.MANAGEMENT); } - /** - * Background sync the specified retention leases for the specified shard. - * - * @param shardId the shard to sync - * @param retentionLeases the retention leases to sync - */ - public void backgroundSync( - final ShardId shardId, - final RetentionLeases retentionLeases) { - Objects.requireNonNull(shardId); - Objects.requireNonNull(retentionLeases); - final ThreadContext threadContext = threadPool.getThreadContext(); - try (ThreadContext.StoredContext ignore = threadContext.stashContext()) { - // we have to execute under the system context so that if security is enabled the sync is authorized - threadContext.markAsSystemContext(); - execute( - new Request(shardId, retentionLeases), - ActionListener.wrap( - r -> {}, - e -> { - if (ExceptionsHelper.isTransportStoppedForAction(e, ACTION_NAME + "[p]")) { - // we are likely shutting down - return; - } - if (ExceptionsHelper.unwrap(e, AlreadyClosedException.class, IndexShardClosedException.class) != null) { - // the shard is closed - return; - } - getLogger().warn(new ParameterizedMessage("{} retention lease background sync failed", shardId), e); - })); - } - } - @Override protected void shardOperationOnPrimary( final Request request, diff --git a/server/src/main/java/org/elasticsearch/index/seqno/RetentionLeaseSyncAction.java b/server/src/main/java/org/elasticsearch/index/seqno/RetentionLeaseSyncAction.java index 6129affe493ae..1c4f1b74cbccf 100644 --- a/server/src/main/java/org/elasticsearch/index/seqno/RetentionLeaseSyncAction.java +++ b/server/src/main/java/org/elasticsearch/index/seqno/RetentionLeaseSyncAction.java @@ -21,10 +21,9 @@ import org.apache.logging.log4j.LogManager; import org.apache.logging.log4j.Logger; -import org.apache.logging.log4j.message.ParameterizedMessage; -import org.apache.lucene.store.AlreadyClosedException; -import org.elasticsearch.ExceptionsHelper; import org.elasticsearch.action.ActionListener; +import org.elasticsearch.action.ActionType; +import org.elasticsearch.action.StreamableResponseActionType; import org.elasticsearch.action.support.ActionFilters; import org.elasticsearch.action.support.ActiveShardCount; import org.elasticsearch.action.support.WriteResponse; @@ -39,10 +38,8 @@ 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.gateway.WriteStateException; import org.elasticsearch.index.shard.IndexShard; -import org.elasticsearch.index.shard.IndexShardClosedException; import org.elasticsearch.index.shard.ShardId; import org.elasticsearch.indices.IndicesService; import org.elasticsearch.threadpool.ThreadPool; @@ -59,6 +56,12 @@ public class RetentionLeaseSyncAction extends TransportWriteAction { public static String ACTION_NAME = "indices:admin/seq_no/retention_lease_sync"; + public static ActionType TYPE = new StreamableResponseActionType<>(ACTION_NAME) { + @Override + public Response newResponse() { + return new Response(); + } + }; private static final Logger LOGGER = LogManager.getLogger(RetentionLeaseSyncAction.class); @@ -91,37 +94,6 @@ public RetentionLeaseSyncAction( ThreadPool.Names.MANAGEMENT, false); } - /** - * Sync the specified retention leases for the specified shard. The callback is invoked when the sync succeeds or fails. - * - * @param shardId the shard to sync - * @param retentionLeases the retention leases to sync - * @param listener the callback to invoke when the sync completes normally or abnormally - */ - public void sync( - final ShardId shardId, - final RetentionLeases retentionLeases, - final ActionListener listener) { - Objects.requireNonNull(shardId); - Objects.requireNonNull(retentionLeases); - Objects.requireNonNull(listener); - final ThreadContext threadContext = threadPool.getThreadContext(); - try (ThreadContext.StoredContext ignore = threadContext.stashContext()) { - // we have to execute under the system context so that if security is enabled the sync is authorized - threadContext.markAsSystemContext(); - execute( - new RetentionLeaseSyncAction.Request(shardId, retentionLeases), - ActionListener.wrap( - listener::onResponse, - e -> { - if (ExceptionsHelper.unwrap(e, AlreadyClosedException.class, IndexShardClosedException.class) == null) { - getLogger().warn(new ParameterizedMessage("{} retention lease sync failed", shardId), e); - } - listener.onFailure(e); - })); - } - } - @Override protected void shardOperationOnPrimary(Request request, IndexShard primary, ActionListener> listener) { diff --git a/server/src/main/java/org/elasticsearch/index/seqno/SeqNoStats.java b/server/src/main/java/org/elasticsearch/index/seqno/SeqNoStats.java index a56f8670c23b1..e1b992643fac8 100644 --- a/server/src/main/java/org/elasticsearch/index/seqno/SeqNoStats.java +++ b/server/src/main/java/org/elasticsearch/index/seqno/SeqNoStats.java @@ -58,7 +58,7 @@ public long getMaxSeqNo() { return maxSeqNo; } - /** the maximum sequence number for which all previous operations (including) have been completed */ + /** the maximum sequence number for which all previous operations (including) have been persisted */ public long getLocalCheckpoint() { return localCheckpoint; } diff --git a/server/src/main/java/org/elasticsearch/index/seqno/SequenceNumbers.java b/server/src/main/java/org/elasticsearch/index/seqno/SequenceNumbers.java index 6336e83338f8c..87257a97076da 100644 --- a/server/src/main/java/org/elasticsearch/index/seqno/SequenceNumbers.java +++ b/server/src/main/java/org/elasticsearch/index/seqno/SequenceNumbers.java @@ -28,10 +28,6 @@ public class SequenceNumbers { public static final String LOCAL_CHECKPOINT_KEY = "local_checkpoint"; public static final String MAX_SEQ_NO = "max_seq_no"; - /** - * Represents a checkpoint coming from a pre-6.0 node - */ - public static final long PRE_60_NODE_CHECKPOINT = -3L; /** * Represents an unassigned sequence number (e.g., can be used on primary operations before they are executed). */ diff --git a/server/src/main/java/org/elasticsearch/index/shard/IndexSearcherWrapper.java b/server/src/main/java/org/elasticsearch/index/shard/IndexSearcherWrapper.java deleted file mode 100644 index 3a6df72a740e9..0000000000000 --- a/server/src/main/java/org/elasticsearch/index/shard/IndexSearcherWrapper.java +++ /dev/null @@ -1,138 +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.shard; - -import org.apache.lucene.index.DirectoryReader; -import org.apache.lucene.index.FilterDirectoryReader; -import org.apache.lucene.index.IndexReader; -import org.apache.lucene.index.LeafReader; -import org.apache.lucene.search.IndexSearcher; -import org.elasticsearch.common.lucene.index.ElasticsearchDirectoryReader; -import org.elasticsearch.core.internal.io.IOUtils; -import org.elasticsearch.index.engine.Engine; - -import java.io.IOException; - -/** - * Extension point to add custom functionality at request time to the {@link DirectoryReader} - * and {@link IndexSearcher} managed by the {@link IndexShard}. - */ -public class IndexSearcherWrapper { - - /** - * Wraps the given {@link DirectoryReader}. The wrapped reader can filter out document just like delete documents etc. but - * must not change any term or document content. - *

    - * NOTE: The wrapper has a per-request lifecycle, must delegate {@link IndexReader#getReaderCacheHelper()}, - * {@link LeafReader#getCoreCacheHelper()} and must be an instance of {@link FilterDirectoryReader} that - * eventually exposes the original reader via {@link FilterDirectoryReader#getDelegate()}. - * The returned reader is closed once it goes out of scope. - *

    - * @param reader The provided directory reader to be wrapped to add custom functionality - * @return a new directory reader wrapping the provided directory reader or if no wrapping was performed - * the provided directory reader - */ - protected DirectoryReader wrap(DirectoryReader reader) throws IOException { - return reader; - } - - /** - * @param searcher The provided index searcher to be wrapped to add custom functionality - * @return a new index searcher wrapping the provided index searcher or if no wrapping was performed - * the provided index searcher - */ - protected IndexSearcher wrap(IndexSearcher searcher) throws IOException { - return searcher; - } - /** - * If there are configured {@link IndexSearcherWrapper} instances, the {@link IndexSearcher} of the provided engine searcher - * gets wrapped and a new {@link Engine.Searcher} instances is returned, otherwise the provided {@link Engine.Searcher} is returned. - * - * This is invoked each time a {@link Engine.Searcher} is requested to do an operation. (for example search) - */ - public final Engine.Searcher wrap(Engine.Searcher engineSearcher) throws IOException { - final ElasticsearchDirectoryReader elasticsearchDirectoryReader = - ElasticsearchDirectoryReader.getElasticsearchDirectoryReader(engineSearcher.getDirectoryReader()); - if (elasticsearchDirectoryReader == null) { - throw new IllegalStateException("Can't wrap non elasticsearch directory reader"); - } - NonClosingReaderWrapper nonClosingReaderWrapper = new NonClosingReaderWrapper(engineSearcher.getDirectoryReader()); - DirectoryReader reader = wrap(nonClosingReaderWrapper); - if (reader != nonClosingReaderWrapper) { - if (reader.getReaderCacheHelper() != elasticsearchDirectoryReader.getReaderCacheHelper()) { - throw new IllegalStateException("wrapped directory reader doesn't delegate IndexReader#getCoreCacheKey," + - " wrappers must override this method and delegate to the original readers core cache key. Wrapped readers can't be " + - "used as cache keys since their are used only per request which would lead to subtle bugs"); - } - if (ElasticsearchDirectoryReader.getElasticsearchDirectoryReader(reader) != elasticsearchDirectoryReader) { - // prevent that somebody wraps with a non-filter reader - throw new IllegalStateException("wrapped directory reader hides actual ElasticsearchDirectoryReader but shouldn't"); - } - } - - final IndexSearcher origIndexSearcher = engineSearcher.searcher(); - final IndexSearcher innerIndexSearcher = new IndexSearcher(reader); - innerIndexSearcher.setQueryCache(origIndexSearcher.getQueryCache()); - innerIndexSearcher.setQueryCachingPolicy(origIndexSearcher.getQueryCachingPolicy()); - innerIndexSearcher.setSimilarity(origIndexSearcher.getSimilarity()); - // TODO: Right now IndexSearcher isn't wrapper friendly, when it becomes wrapper friendly we should revise this extension point - // For example if IndexSearcher#rewrite() is overwritten than also IndexSearcher#createNormalizedWeight needs to be overwritten - // This needs to be fixed before we can allow the IndexSearcher from Engine to be wrapped multiple times - final IndexSearcher indexSearcher = wrap(innerIndexSearcher); - if (reader == nonClosingReaderWrapper && indexSearcher == innerIndexSearcher) { - return engineSearcher; - } else { - // we close the reader to make sure wrappers can release resources if needed.... - // our NonClosingReaderWrapper makes sure that our reader is not closed - return new Engine.Searcher(engineSearcher.source(), indexSearcher, () -> - IOUtils.close(indexSearcher.getIndexReader(), // this will close the wrappers excluding the NonClosingReaderWrapper - engineSearcher)); // this will run the closeable on the wrapped engine searcher - } - } - - private static final class NonClosingReaderWrapper extends FilterDirectoryReader { - - private NonClosingReaderWrapper(DirectoryReader in) throws IOException { - super(in, new SubReaderWrapper() { - @Override - public LeafReader wrap(LeafReader reader) { - return reader; - } - }); - } - - @Override - protected DirectoryReader doWrapDirectoryReader(DirectoryReader in) throws IOException { - return new NonClosingReaderWrapper(in); - } - - @Override - protected void doClose() throws IOException { - // don't close here - mimic the MultiReader#doClose = false behavior that FilterDirectoryReader doesn't have - } - - @Override - public CacheHelper getReaderCacheHelper() { - return in.getReaderCacheHelper(); - } - - } - -} diff --git a/server/src/main/java/org/elasticsearch/index/shard/IndexShard.java b/server/src/main/java/org/elasticsearch/index/shard/IndexShard.java index fdd95614756b7..d869ae934becd 100644 --- a/server/src/main/java/org/elasticsearch/index/shard/IndexShard.java +++ b/server/src/main/java/org/elasticsearch/index/shard/IndexShard.java @@ -23,9 +23,13 @@ import org.apache.logging.log4j.Logger; import org.apache.logging.log4j.message.ParameterizedMessage; import org.apache.lucene.index.CheckIndex; +import org.apache.lucene.index.DirectoryReader; +import org.apache.lucene.index.FilterDirectoryReader; import org.apache.lucene.index.IndexCommit; +import org.apache.lucene.index.LeafReader; import org.apache.lucene.index.SegmentInfos; import org.apache.lucene.index.Term; +import org.apache.lucene.search.IndexSearcher; import org.apache.lucene.search.Query; import org.apache.lucene.search.QueryCachingPolicy; import org.apache.lucene.search.ReferenceManager; @@ -50,6 +54,7 @@ import org.elasticsearch.cluster.routing.RecoverySource.SnapshotRecoverySource; import org.elasticsearch.cluster.routing.ShardRouting; import org.elasticsearch.common.Booleans; +import org.elasticsearch.common.CheckedFunction; import org.elasticsearch.common.CheckedRunnable; import org.elasticsearch.common.Nullable; import org.elasticsearch.common.collect.Tuple; @@ -66,6 +71,7 @@ import org.elasticsearch.common.util.concurrent.AbstractRunnable; import org.elasticsearch.common.util.concurrent.AsyncIOProcessor; import org.elasticsearch.common.util.concurrent.RunOnce; +import org.elasticsearch.common.util.concurrent.ThreadContext; import org.elasticsearch.common.xcontent.XContentHelper; import org.elasticsearch.core.internal.io.IOUtils; import org.elasticsearch.gateway.WriteStateException; @@ -243,7 +249,7 @@ Runnable getGlobalCheckpointSyncer() { private static final EnumSet writeAllowedStates = EnumSet.of(IndexShardState.RECOVERING, IndexShardState.POST_RECOVERY, IndexShardState.STARTED); - private final IndexSearcherWrapper searcherWrapper; + private final CheckedFunction readerWrapper; /** * True if this shard is still indexing (recently) and false if we've been idle for long enough (as periodically checked by {@link @@ -269,7 +275,7 @@ public IndexShard( final SimilarityService similarityService, final @Nullable EngineFactory engineFactory, final IndexEventListener indexEventListener, - final IndexSearcherWrapper indexSearcherWrapper, + final CheckedFunction indexReaderWrapper, final ThreadPool threadPool, final BigArrays bigArrays, final Engine.Warmer warmer, @@ -291,6 +297,7 @@ public IndexShard( this.indexSortSupplier = indexSortSupplier; this.indexEventListener = indexEventListener; this.threadPool = threadPool; + this.translogSyncProcessor = createTranslogSyncProcessor(logger, threadPool.getThreadContext(), this::getEngine); this.mapperService = mapperService; this.indexCache = indexCache; this.internalIndexingStats = new InternalIndexingStats(); @@ -349,7 +356,7 @@ public boolean shouldCache(Query query) { cachingPolicy = new UsageTrackingQueryCachingPolicy(); } indexShardOperationPermits = new IndexShardOperationPermits(shardId, threadPool); - searcherWrapper = indexSearcherWrapper; + readerWrapper = indexReaderWrapper; refreshListeners = buildRefreshListeners(); lastSearcherAccess.set(threadPool.relativeTimeInMillis()); persistMetadata(path, indexSettings, shardRouting, null, logger); @@ -540,7 +547,8 @@ public void updateShardState(final ShardRouting newRouting, */ engine.rollTranslogGeneration(); engine.fillSeqNoGaps(newPrimaryTerm); - replicationTracker.updateLocalCheckpoint(currentRouting.allocationId().getId(), getLocalCheckpoint()); + replicationTracker.updateLocalCheckpoint(currentRouting.allocationId().getId(), + getLocalCheckpoint()); primaryReplicaSyncer.accept(this, new ActionListener() { @Override public void onResponse(ResyncTask resyncTask) { @@ -618,10 +626,11 @@ public IndexShardState markAsRecovering(String reason, RecoveryState recoverySta * * @param consumer a {@link Runnable} that is executed after operations are blocked * @throws IllegalIndexShardStateException if the shard is not relocating due to concurrent cancellation + * @throws IllegalStateException if the relocation target is no longer part of the replication group * @throws InterruptedException if blocking operations is interrupted */ - public void relocated(final Consumer consumer) - throws IllegalIndexShardStateException, InterruptedException { + public void relocated(final String targetAllocationId, final Consumer consumer) + throws IllegalIndexShardStateException, IllegalStateException, InterruptedException { assert shardRouting.primary() : "only primaries can be marked as relocated: " + shardRouting; final Releasable forceRefreshes = refreshListeners.forceRefreshes(); try { @@ -635,7 +644,7 @@ public void relocated(final Consumer consumer * network operation. Doing this under the mutex can implicitly block the cluster state update thread on network operations. */ verifyRelocatingState(); - final ReplicationTracker.PrimaryContext primaryContext = replicationTracker.startRelocationHandoff(); + final ReplicationTracker.PrimaryContext primaryContext = replicationTracker.startRelocationHandoff(targetAllocationId); try { consumer.accept(primaryContext); synchronized (mutex) { @@ -1228,7 +1237,7 @@ private Engine.Searcher acquireSearcher(String source, Engine.SearcherScope scop != null : "DirectoryReader must be an instance or ElasticsearchDirectoryReader"; boolean success = false; try { - final Engine.Searcher wrappedSearcher = searcherWrapper == null ? searcher : searcherWrapper.wrap(searcher); + final Engine.Searcher wrappedSearcher = readerWrapper == null ? searcher : wrapSearcher(searcher, readerWrapper); assert wrappedSearcher != null; success = true; return wrappedSearcher; @@ -1241,6 +1250,72 @@ private Engine.Searcher acquireSearcher(String source, Engine.SearcherScope scop } } + static Engine.Searcher wrapSearcher(Engine.Searcher engineSearcher, + CheckedFunction readerWrapper) throws IOException { + assert readerWrapper != null; + final ElasticsearchDirectoryReader elasticsearchDirectoryReader = + ElasticsearchDirectoryReader.getElasticsearchDirectoryReader(engineSearcher.getDirectoryReader()); + if (elasticsearchDirectoryReader == null) { + throw new IllegalStateException("Can't wrap non elasticsearch directory reader"); + } + NonClosingReaderWrapper nonClosingReaderWrapper = new NonClosingReaderWrapper(engineSearcher.getDirectoryReader()); + DirectoryReader reader = readerWrapper.apply(nonClosingReaderWrapper); + if (reader != nonClosingReaderWrapper) { + if (reader.getReaderCacheHelper() != elasticsearchDirectoryReader.getReaderCacheHelper()) { + throw new IllegalStateException("wrapped directory reader doesn't delegate IndexReader#getCoreCacheKey," + + " wrappers must override this method and delegate to the original readers core cache key. Wrapped readers can't be " + + "used as cache keys since their are used only per request which would lead to subtle bugs"); + } + if (ElasticsearchDirectoryReader.getElasticsearchDirectoryReader(reader) != elasticsearchDirectoryReader) { + // prevent that somebody wraps with a non-filter reader + throw new IllegalStateException("wrapped directory reader hides actual ElasticsearchDirectoryReader but shouldn't"); + } + } + + if (reader == nonClosingReaderWrapper) { + return engineSearcher; + } else { + final IndexSearcher origIndexSearcher = engineSearcher.searcher(); + final IndexSearcher newIndexSearcher = new IndexSearcher(reader); + newIndexSearcher.setQueryCache(origIndexSearcher.getQueryCache()); + newIndexSearcher.setQueryCachingPolicy(origIndexSearcher.getQueryCachingPolicy()); + newIndexSearcher.setSimilarity(origIndexSearcher.getSimilarity()); + // we close the reader to make sure wrappers can release resources if needed.... + // our NonClosingReaderWrapper makes sure that our reader is not closed + return new Engine.Searcher(engineSearcher.source(), newIndexSearcher, () -> + IOUtils.close(newIndexSearcher.getIndexReader(), // this will close the wrappers excluding the NonClosingReaderWrapper + engineSearcher)); // this will run the closeable on the wrapped engine searcher + } + } + + private static final class NonClosingReaderWrapper extends FilterDirectoryReader { + + private NonClosingReaderWrapper(DirectoryReader in) throws IOException { + super(in, new SubReaderWrapper() { + @Override + public LeafReader wrap(LeafReader reader) { + return reader; + } + }); + } + + @Override + protected DirectoryReader doWrapDirectoryReader(DirectoryReader in) throws IOException { + return new NonClosingReaderWrapper(in); + } + + @Override + protected void doClose() throws IOException { + // don't close here - mimic the MultiReader#doClose = false behavior that FilterDirectoryReader doesn't have + } + + @Override + public CacheHelper getReaderCacheHelper() { + return in.getReaderCacheHelper(); + } + + } + public void close(String reason, boolean flushEngine) throws IOException { synchronized (mutex) { try { @@ -1865,7 +1940,7 @@ public void updateLocalCheckpointForShard(final String allocationId, final long } /** - * Update the local knowledge of the global checkpoint for the specified allocation ID. + * Update the local knowledge of the persisted global checkpoint for the specified allocation ID. * * @param allocationId the allocation ID to update the global checkpoint for * @param globalCheckpoint the global checkpoint @@ -2079,12 +2154,12 @@ public void markAllocationIdAsInSync(final String allocationId, final long local } /** - * Returns the local checkpoint for the shard. + * Returns the persisted local checkpoint for the shard. * * @return the local checkpoint */ public long getLocalCheckpoint() { - return getEngine().getLocalCheckpoint(); + return getEngine().getPersistedLocalCheckpoint(); } /** @@ -2092,7 +2167,7 @@ public long getLocalCheckpoint() { * * @return the global checkpoint */ - public long getGlobalCheckpoint() { + public long getLastKnownGlobalCheckpoint() { return replicationTracker.getGlobalCheckpoint(); } @@ -2125,15 +2200,21 @@ public void maybeSyncGlobalCheckpoint(final String reason) { return; } assert assertPrimaryMode(); - // only sync if there are not operations in flight + // only sync if there are no operations in flight, or when using async durability final SeqNoStats stats = getEngine().getSeqNoStats(replicationTracker.getGlobalCheckpoint()); - if (stats.getMaxSeqNo() == stats.getGlobalCheckpoint()) { + final boolean asyncDurability = indexSettings().getTranslogDurability() == Translog.Durability.ASYNC; + if (stats.getMaxSeqNo() == stats.getGlobalCheckpoint() || asyncDurability) { final ObjectLongMap globalCheckpoints = getInSyncGlobalCheckpoints(); - final String allocationId = routingEntry().allocationId().getId(); - assert globalCheckpoints.containsKey(allocationId); - final long globalCheckpoint = globalCheckpoints.get(allocationId); + final long globalCheckpoint = replicationTracker.getGlobalCheckpoint(); + // async durability means that the local checkpoint might lag (as it is only advanced on fsync) + // periodically ask for the newest local checkpoint by syncing the global checkpoint, so that ultimately the global + // checkpoint can be synced. Also take into account that a shard might be pending sync, which means that it isn't + // in the in-sync set just yet but might be blocked on waiting for its persisted local checkpoint to catch up to + // the global checkpoint. final boolean syncNeeded = - StreamSupport + (asyncDurability && (stats.getGlobalCheckpoint() < stats.getMaxSeqNo() || replicationTracker.pendingInSync())) + // check if the persisted global checkpoint + || StreamSupport .stream(globalCheckpoints.values().spliterator(), false) .anyMatch(v -> v.value < globalCheckpoint); // only sync if index is not closed and there is a shard lagging the primary @@ -2192,7 +2273,8 @@ public void activateWithPrimaryContext(final ReplicationTracker.PrimaryContext p assert shardRouting.primary() && shardRouting.isRelocationTarget() : "only primary relocation target can update allocation IDs from primary context: " + shardRouting; assert primaryContext.getCheckpointStates().containsKey(routingEntry().allocationId().getId()) && - getLocalCheckpoint() == primaryContext.getCheckpointStates().get(routingEntry().allocationId().getId()).getLocalCheckpoint(); + getLocalCheckpoint() == primaryContext.getCheckpointStates().get(routingEntry().allocationId().getId()) + .getLocalCheckpoint() || indexSettings().getTranslogDurability() == Translog.Durability.ASYNC; synchronized (mutex) { replicationTracker.activateWithPrimaryContext(primaryContext); // make changes to primaryMode flag only under mutex } @@ -2733,7 +2815,7 @@ private void innerAcquireReplicaOperationPermit(final long opPrimaryTerm, bumpPrimaryTerm(opPrimaryTerm, () -> { updateGlobalCheckpointOnReplica(globalCheckpoint, "primary term transition"); - final long currentGlobalCheckpoint = getGlobalCheckpoint(); + final long currentGlobalCheckpoint = getLastKnownGlobalCheckpoint(); final long maxSeqNo = seqNoStats().getMaxSeqNo(); logger.info("detected new primary with primary term [{}], global checkpoint [{}], max_seq_no [{}]", opPrimaryTerm, currentGlobalCheckpoint, maxSeqNo); @@ -2780,19 +2862,24 @@ public List getActiveOperations() { return indexShardOperationPermits.getActiveOperations(); } - private final AsyncIOProcessor translogSyncProcessor = new AsyncIOProcessor(logger, 1024) { - @Override - protected void write(List>> candidates) throws IOException { - try { - getEngine().ensureTranslogSynced(candidates.stream().map(Tuple::v1)); - } catch (AlreadyClosedException ex) { - // that's fine since we already synced everything on engine close - this also is conform with the methods - // documentation - } catch (IOException ex) { // if this fails we are in deep shit - fail the request - logger.debug("failed to sync translog", ex); - throw ex; + private final AsyncIOProcessor translogSyncProcessor; + + private static AsyncIOProcessor createTranslogSyncProcessor(Logger logger, ThreadContext threadContext, + Supplier engineSupplier) { + return new AsyncIOProcessor<>(logger, 1024, threadContext) { + @Override + protected void write(List>> candidates) throws IOException { + try { + engineSupplier.get().ensureTranslogSynced(candidates.stream().map(Tuple::v1)); + } catch (AlreadyClosedException ex) { + // that's fine since we already synced everything on engine close - this also is conform with the methods + // documentation + } catch (IOException ex) { // if this fails we are in deep shit - fail the request + logger.debug("failed to sync translog", ex); + throw ex; + } } - } + }; }; /** @@ -3103,7 +3190,8 @@ assert getActiveOperationsCount() == OPERATIONS_BLOCKED flush(new FlushRequest().waitIfOngoing(true)); SetOnce newEngineReference = new SetOnce<>(); - final long globalCheckpoint = getGlobalCheckpoint(); + final long globalCheckpoint = getLastKnownGlobalCheckpoint(); + assert globalCheckpoint == getLastSyncedGlobalCheckpoint(); synchronized (mutex) { verifyNotClosed(); // we must create both new read-only engine and new read-write engine under mutex to ensure snapshotStoreMetadata, diff --git a/server/src/main/java/org/elasticsearch/index/shard/PrimaryReplicaSyncer.java b/server/src/main/java/org/elasticsearch/index/shard/PrimaryReplicaSyncer.java index 07aade952923b..17ef424185d1f 100644 --- a/server/src/main/java/org/elasticsearch/index/shard/PrimaryReplicaSyncer.java +++ b/server/src/main/java/org/elasticsearch/index/shard/PrimaryReplicaSyncer.java @@ -84,7 +84,7 @@ void setChunkSize(ByteSizeValue chunkSize) { // only settable for tests public void resync(final IndexShard indexShard, final ActionListener listener) { Translog.Snapshot snapshot = null; try { - final long startingSeqNo = indexShard.getGlobalCheckpoint() + 1; + final long startingSeqNo = indexShard.getLastKnownGlobalCheckpoint() + 1; final long maxSeqNo = indexShard.seqNoStats().getMaxSeqNo(); final ShardId shardId = indexShard.shardId(); // Wrap translog snapshot to make it synchronized as it is accessed by different threads through SnapshotSender. diff --git a/server/src/main/java/org/elasticsearch/index/shard/RemoveCorruptedShardDataCommand.java b/server/src/main/java/org/elasticsearch/index/shard/RemoveCorruptedShardDataCommand.java index 16db596515b4c..5fc3ba57980bf 100644 --- a/server/src/main/java/org/elasticsearch/index/shard/RemoveCorruptedShardDataCommand.java +++ b/server/src/main/java/org/elasticsearch/index/shard/RemoveCorruptedShardDataCommand.java @@ -140,17 +140,14 @@ protected void findAndProcessShardPath(OptionSet options, Environment environmen IndexMetaData.FORMAT.loadLatestState(logger, namedXContentRegistry, shardParent); final String shardIdFileName = path.getFileName().toString(); - final String nodeIdFileName = shardParentParent.getParent().getFileName().toString(); if (Files.isDirectory(path) && shardIdFileName.chars().allMatch(Character::isDigit) // SHARD-ID path element check && NodeEnvironment.INDICES_FOLDER.equals(shardParentParent.getFileName().toString()) // `indices` check - && nodeIdFileName.chars().allMatch(Character::isDigit) // NODE-ID check - && NodeEnvironment.NODES_FOLDER.equals(shardParentParent.getParent().getParent().getFileName().toString()) // `nodes` check ) { shardId = Integer.parseInt(shardIdFileName); indexName = indexMetaData.getIndex().getName(); } else { throw new ElasticsearchException("Unable to resolve shard id. Wrong folder structure at [ " + path.toString() - + " ], expected .../nodes/[NODE-ID]/indices/[INDEX-UUID]/[SHARD-ID]"); + + " ], expected .../indices/[INDEX-UUID]/[SHARD-ID]"); } } else { // otherwise resolve shardPath based on the index name and shard id diff --git a/server/src/main/java/org/elasticsearch/index/similarity/SimilarityService.java b/server/src/main/java/org/elasticsearch/index/similarity/SimilarityService.java index 57cbc961aacc0..3fe20a1f3b26e 100644 --- a/server/src/main/java/org/elasticsearch/index/similarity/SimilarityService.java +++ b/server/src/main/java/org/elasticsearch/index/similarity/SimilarityService.java @@ -212,9 +212,8 @@ private static void validateScoresArePositive(Version indexCreatedVersion, Simil for (int freq = 1; freq <= 10; ++freq) { float score = scorer.score(freq, norm); if (score < 0) { - fail(indexCreatedVersion, "Similarities should not return negative scores:\n" + - scorer.explain(Explanation.match(freq, "term freq"), norm)); - break; + throw new IllegalArgumentException("Similarities should not return negative scores:\n" + + scorer.explain(Explanation.match(freq, "term freq"), norm)); } } } @@ -230,10 +229,9 @@ private static void validateScoresDoNotDecreaseWithFreq(Version indexCreatedVers for (int freq = 1; freq <= 10; ++freq) { float score = scorer.score(freq, norm); if (score < previousScore) { - fail(indexCreatedVersion, "Similarity scores should not decrease when term frequency increases:\n" + - scorer.explain(Explanation.match(freq - 1, "term freq"), norm) + "\n" + - scorer.explain(Explanation.match(freq, "term freq"), norm)); - break; + throw new IllegalArgumentException("Similarity scores should not decrease when term frequency increases:\n" + + scorer.explain(Explanation.match(freq - 1, "term freq"), norm) + "\n" + + scorer.explain(Explanation.match(freq, "term freq"), norm)); } previousScore = score; } @@ -256,22 +254,12 @@ private static void validateScoresDoNotIncreaseWithNorm(Version indexCreatedVers } float score = scorer.score(1, norm); if (score > previousScore) { - fail(indexCreatedVersion, "Similarity scores should not increase when norm increases:\n" + - scorer.explain(Explanation.match(1, "term freq"), norm - 1) + "\n" + - scorer.explain(Explanation.match(1, "term freq"), norm)); - break; + throw new IllegalArgumentException("Similarity scores should not increase when norm increases:\n" + + scorer.explain(Explanation.match(1, "term freq"), norm - 1) + "\n" + + scorer.explain(Explanation.match(1, "term freq"), norm)); } previousScore = score; previousNorm = norm; } } - - private static void fail(Version indexCreatedVersion, String message) { - if (indexCreatedVersion.onOrAfter(Version.V_7_0_0)) { - throw new IllegalArgumentException(message); - } else if (indexCreatedVersion.onOrAfter(Version.V_6_5_0)) { - deprecationLogger.deprecated(message); - } - } - } diff --git a/server/src/main/java/org/elasticsearch/index/store/FsDirectoryFactory.java b/server/src/main/java/org/elasticsearch/index/store/FsDirectoryFactory.java index 84bb4c49b27d4..310f3de949527 100644 --- a/server/src/main/java/org/elasticsearch/index/store/FsDirectoryFactory.java +++ b/server/src/main/java/org/elasticsearch/index/store/FsDirectoryFactory.java @@ -22,6 +22,7 @@ import org.apache.lucene.store.Directory; import org.apache.lucene.store.FSDirectory; import org.apache.lucene.store.FileSwitchDirectory; +import org.apache.lucene.store.FilterDirectory; import org.apache.lucene.store.IOContext; import org.apache.lucene.store.IndexInput; import org.apache.lucene.store.LockFactory; @@ -63,11 +64,7 @@ public Directory newDirectory(IndexSettings indexSettings, ShardPath path) throw final Path location = path.resolveIndex(); final LockFactory lockFactory = indexSettings.getValue(INDEX_LOCK_FACTOR_SETTING); Files.createDirectories(location); - Directory wrapped = newFSDirectory(location, lockFactory, indexSettings); - Set preLoadExtensions = new HashSet<>( - indexSettings.getValue(IndexModule.INDEX_STORE_PRE_LOAD_SETTING)); - wrapped = setPreload(wrapped, location, lockFactory, preLoadExtensions); - return wrapped; + return newFSDirectory(location, lockFactory, indexSettings); } protected Directory newFSDirectory(Path location, LockFactory lockFactory, IndexSettings indexSettings) throws IOException { @@ -79,17 +76,20 @@ protected Directory newFSDirectory(Path location, LockFactory lockFactory, Index } else { type = IndexModule.Type.fromSettingsKey(storeType); } + Set preLoadExtensions = new HashSet<>( + indexSettings.getValue(IndexModule.INDEX_STORE_PRE_LOAD_SETTING)); switch (type) { case HYBRIDFS: // Use Lucene defaults final FSDirectory primaryDirectory = FSDirectory.open(location, lockFactory); if (primaryDirectory instanceof MMapDirectory) { - return new HybridDirectory(location, lockFactory, primaryDirectory); + MMapDirectory mMapDirectory = (MMapDirectory) primaryDirectory; + return new HybridDirectory(lockFactory, setPreload(mMapDirectory, lockFactory, preLoadExtensions)); } else { return primaryDirectory; } case MMAPFS: - return new MMapDirectory(location, lockFactory); + return setPreload(new MMapDirectory(location, lockFactory), lockFactory, preLoadExtensions); case SIMPLEFS: return new SimpleFSDirectory(location, lockFactory); case NIOFS: @@ -99,38 +99,57 @@ protected Directory newFSDirectory(Path location, LockFactory lockFactory, Index } } - private static Directory setPreload(Directory directory, Path location, LockFactory lockFactory, + public static MMapDirectory setPreload(MMapDirectory mMapDirectory, LockFactory lockFactory, Set preLoadExtensions) throws IOException { - if (preLoadExtensions.isEmpty() == false - && directory instanceof MMapDirectory - && ((MMapDirectory) directory).getPreload() == false) { + assert mMapDirectory.getPreload() == false; + if (preLoadExtensions.isEmpty() == false) { if (preLoadExtensions.contains("*")) { - ((MMapDirectory) directory).setPreload(true); - return directory; + mMapDirectory.setPreload(true); + } else { + return new PreLoadMMapDirectory(mMapDirectory, lockFactory, preLoadExtensions); } - MMapDirectory primary = new MMapDirectory(location, lockFactory); - primary.setPreload(true); - return new FileSwitchDirectory(preLoadExtensions, primary, directory, true) { - @Override - public String[] listAll() throws IOException { - // avoid listing twice - return primary.listAll(); - } - }; } - return directory; + return mMapDirectory; + } + + /** + * Returns true iff the directory is a hybrid fs directory + */ + public static boolean isHybridFs(Directory directory) { + Directory unwrap = FilterDirectory.unwrap(directory); + return unwrap instanceof HybridDirectory; } static final class HybridDirectory extends NIOFSDirectory { - private final FSDirectory randomAccessDirectory; + private final MMapDirectory delegate; - HybridDirectory(Path location, LockFactory lockFactory, FSDirectory randomAccessDirectory) throws IOException { - super(location, lockFactory); - this.randomAccessDirectory = randomAccessDirectory; + HybridDirectory(LockFactory lockFactory, MMapDirectory delegate) throws IOException { + super(delegate.getDirectory(), lockFactory); + this.delegate = delegate; } @Override public IndexInput openInput(String name, IOContext context) throws IOException { + if (useDelegate(name)) { + // we need to do these checks on the outer directory since the inner doesn't know about pending deletes + ensureOpen(); + ensureCanRead(name); + // we only use the mmap to open inputs. Everything else is managed by the NIOFSDirectory otherwise + // we might run into trouble with files that are pendingDelete in one directory but still + // listed in listAll() from the other. We on the other hand don't want to list files from both dirs + // and intersect for perf reasons. + return delegate.openInput(name, context); + } else { + return super.openInput(name, context); + } + } + + @Override + public void close() throws IOException { + IOUtils.close(super::close, delegate); + } + + boolean useDelegate(String name) { String extension = FileSwitchDirectory.getExtension(name); switch(extension) { // We are mmapping norms, docvalues as well as term dictionaries, all other files are served through NIOFS @@ -138,27 +157,61 @@ public IndexInput openInput(String name, IOContext context) throws IOException { case "nvd": case "dvd": case "tim": + case "tip": case "cfs": - // we need to do these checks on the outer directory since the inner doesn't know about pending deletes - ensureOpen(); - ensureCanRead(name); - // we only use the mmap to open inputs. Everything else is managed by the NIOFSDirectory otherwise - // we might run into trouble with files that are pendingDelete in one directory but still - // listed in listAll() from the other. We on the other hand don't want to list files from both dirs - // and intersect for perf reasons. - return randomAccessDirectory.openInput(name, context); + return true; default: - return super.openInput(name, context); + return false; + } + } + + MMapDirectory getDelegate() { + return delegate; + } + } + // TODO it would be nice to share code between PreLoadMMapDirectory and HybridDirectory but due to the nesting aspect of + // directories here makes it tricky. It would be nice to allow MMAPDirectory to pre-load on a per IndexInput basis. + static final class PreLoadMMapDirectory extends MMapDirectory { + private final MMapDirectory delegate; + private final Set preloadExtensions; + + PreLoadMMapDirectory(MMapDirectory delegate, LockFactory lockFactory, Set preload) throws IOException { + super(delegate.getDirectory(), lockFactory); + super.setPreload(false); + this.delegate = delegate; + this.delegate.setPreload(true); + this.preloadExtensions = preload; + assert getPreload() == false; + } + + @Override + public void setPreload(boolean preload) { + throw new IllegalArgumentException("can't set preload on a preload-wrapper"); + } + + @Override + public IndexInput openInput(String name, IOContext context) throws IOException { + if (useDelegate(name)) { + // we need to do these checks on the outer directory since the inner doesn't know about pending deletes + ensureOpen(); + ensureCanRead(name); + return delegate.openInput(name, context); } + return super.openInput(name, context); } @Override - public void close() throws IOException { - IOUtils.close(super::close, randomAccessDirectory); + public synchronized void close() throws IOException { + IOUtils.close(super::close, delegate); + } + + boolean useDelegate(String name) { + final String extension = FileSwitchDirectory.getExtension(name); + return preloadExtensions.contains(extension); } - Directory getRandomAccessDirectory() { - return randomAccessDirectory; + MMapDirectory getDelegate() { + return delegate; } } } diff --git a/server/src/main/java/org/elasticsearch/index/store/Store.java b/server/src/main/java/org/elasticsearch/index/store/Store.java index 5f1f7d23a8c6a..410774114bd78 100644 --- a/server/src/main/java/org/elasticsearch/index/store/Store.java +++ b/server/src/main/java/org/elasticsearch/index/store/Store.java @@ -38,7 +38,6 @@ import org.apache.lucene.store.AlreadyClosedException; import org.apache.lucene.store.BufferedChecksum; import org.apache.lucene.store.ByteArrayDataInput; -import org.apache.lucene.store.ByteBufferIndexInput; import org.apache.lucene.store.ChecksumIndexInput; import org.apache.lucene.store.Directory; import org.apache.lucene.store.FilterDirectory; @@ -46,7 +45,6 @@ import org.apache.lucene.store.IndexInput; import org.apache.lucene.store.IndexOutput; import org.apache.lucene.store.Lock; -import org.apache.lucene.store.RandomAccessInput; import org.apache.lucene.store.SimpleFSDirectory; import org.apache.lucene.util.ArrayUtil; import org.apache.lucene.util.BytesRef; @@ -98,7 +96,6 @@ import java.util.Iterator; import java.util.List; import java.util.Map; -import java.util.Set; import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicBoolean; import java.util.concurrent.locks.ReentrantReadWriteLock; @@ -137,7 +134,7 @@ public class Store extends AbstractIndexShardComponent implements Closeable, Ref * this by exploiting lucene internals and wrapping the IndexInput in a simple delegate. */ public static final Setting FORCE_RAM_TERM_DICT = Setting.boolSetting("index.force_memory_term_dictionary", false, - Property.IndexScope); + Property.IndexScope, Property.Deprecated); static final String CODEC = "store"; static final int VERSION_WRITE_THROWABLE= 2; // we write throwable since 2.0 static final int VERSION_STACK_TRACE = 1; // we write the stack trace too since 1.4.0 @@ -172,8 +169,7 @@ public Store(ShardId shardId, IndexSettings indexSettings, Directory directory, final TimeValue refreshInterval = indexSettings.getValue(INDEX_STORE_STATS_REFRESH_INTERVAL_SETTING); logger.debug("store stats are refreshed with refresh_interval [{}]", refreshInterval); ByteSizeCachingDirectory sizeCachingDir = new ByteSizeCachingDirectory(directory, refreshInterval); - this.directory = new StoreDirectory(sizeCachingDir, Loggers.getLogger("index.store.deletes", shardId), - indexSettings.getValue(FORCE_RAM_TERM_DICT)); + this.directory = new StoreDirectory(sizeCachingDir, Loggers.getLogger("index.store.deletes", shardId)); this.shardLock = shardLock; this.onClose = onClose; @@ -712,12 +708,10 @@ public int refCount() { static final class StoreDirectory extends FilterDirectory { private final Logger deletesLogger; - private final boolean forceRamTermDict; - StoreDirectory(ByteSizeCachingDirectory delegateDirectory, Logger deletesLogger, boolean forceRamTermDict) { + StoreDirectory(ByteSizeCachingDirectory delegateDirectory, Logger deletesLogger) { super(delegateDirectory); this.deletesLogger = deletesLogger; - this.forceRamTermDict = forceRamTermDict; } /** Estimate the cumulative size of all files in this directory in bytes. */ @@ -744,18 +738,6 @@ private void innerClose() throws IOException { super.close(); } - @Override - public IndexInput openInput(String name, IOContext context) throws IOException { - IndexInput input = super.openInput(name, context); - if (name.endsWith(".tip") || name.endsWith(".cfs")) { - // only do this if we are reading cfs or tip file - all other files don't need this. - if (forceRamTermDict && input instanceof ByteBufferIndexInput) { - return new DeoptimizingIndexInput(input.toString(), input); - } - } - return input; - } - @Override public String toString() { return "store(" + in.toString() + ")"; @@ -1614,127 +1596,4 @@ private static IndexWriterConfig newIndexWriterConfig() { // we also don't specify a codec here and merges should use the engines for this index .setMergePolicy(NoMergePolicy.INSTANCE); } - - /** - * see {@link #FORCE_RAM_TERM_DICT} for details - */ - private static final class DeoptimizingIndexInput extends IndexInput { - - private final IndexInput in; - - private DeoptimizingIndexInput(String resourceDescription, IndexInput in) { - super(resourceDescription); - this.in = in; - } - - @Override - public IndexInput clone() { - return new DeoptimizingIndexInput(toString(), in.clone()); - } - - @Override - public void close() throws IOException { - in.close(); - } - - @Override - public long getFilePointer() { - return in.getFilePointer(); - } - - @Override - public void seek(long pos) throws IOException { - in.seek(pos); - } - - @Override - public long length() { - return in.length(); - } - - @Override - public String toString() { - return in.toString(); - } - - @Override - public IndexInput slice(String sliceDescription, long offset, long length) throws IOException { - return new DeoptimizingIndexInput(sliceDescription, in.slice(sliceDescription, offset, length)); - } - - @Override - public RandomAccessInput randomAccessSlice(long offset, long length) throws IOException { - return in.randomAccessSlice(offset, length); - } - - @Override - public byte readByte() throws IOException { - return in.readByte(); - } - - @Override - public void readBytes(byte[] b, int offset, int len) throws IOException { - in.readBytes(b, offset, len); - } - - @Override - public void readBytes(byte[] b, int offset, int len, boolean useBuffer) throws IOException { - in.readBytes(b, offset, len, useBuffer); - } - - @Override - public short readShort() throws IOException { - return in.readShort(); - } - - @Override - public int readInt() throws IOException { - return in.readInt(); - } - - @Override - public int readVInt() throws IOException { - return in.readVInt(); - } - - @Override - public int readZInt() throws IOException { - return in.readZInt(); - } - - @Override - public long readLong() throws IOException { - return in.readLong(); - } - - @Override - public long readVLong() throws IOException { - return in.readVLong(); - } - - @Override - public long readZLong() throws IOException { - return in.readZLong(); - } - - @Override - public String readString() throws IOException { - return in.readString(); - } - - @Override - public Map readMapOfStrings() throws IOException { - return in.readMapOfStrings(); - } - - @Override - public Set readSetOfStrings() throws IOException { - return in.readSetOfStrings(); - } - - @Override - public void skipBytes(long numBytes) throws IOException { - in.skipBytes(numBytes); - } - } } diff --git a/server/src/main/java/org/elasticsearch/index/termvectors/TermVectorsService.java b/server/src/main/java/org/elasticsearch/index/termvectors/TermVectorsService.java index b53c3d8da427c..e5809460c0400 100644 --- a/server/src/main/java/org/elasticsearch/index/termvectors/TermVectorsService.java +++ b/server/src/main/java/org/elasticsearch/index/termvectors/TermVectorsService.java @@ -80,8 +80,7 @@ public static TermVectorsResponse getTermVectors(IndexShard indexShard, TermVect static TermVectorsResponse getTermVectors(IndexShard indexShard, TermVectorsRequest request, LongSupplier nanoTimeSupplier) { final long startTime = nanoTimeSupplier.getAsLong(); - final TermVectorsResponse termVectorsResponse = new TermVectorsResponse(indexShard.shardId().getIndex().getName(), - request.type(), request.id()); + final TermVectorsResponse termVectorsResponse = new TermVectorsResponse(indexShard.shardId().getIndex().getName(), request.id()); final Term uidTerm = new Term(IdFieldMapper.NAME, Uid.encodeId(request.id())); Fields termVectorsByField = null; @@ -93,7 +92,7 @@ static TermVectorsResponse getTermVectors(IndexShard indexShard, TermVectorsRequ handleFieldWildcards(indexShard, request); } - try (Engine.GetResult get = indexShard.get(new Engine.Get(request.realtime(), false, request.type(), + try (Engine.GetResult get = indexShard.get(new Engine.Get(request.realtime(), false, MapperService.SINGLE_MAPPING_NAME, request.id(), uidTerm) .version(request.version()).versionType(request.versionType())); Engine.Searcher searcher = indexShard.acquireSearcher("term_vector")) { @@ -215,7 +214,7 @@ private static Fields addGeneratedTermVectors(IndexShard indexShard, Engine.GetR String[] getFields = validFields.toArray(new String[validFields.size() + 1]); getFields[getFields.length - 1] = SourceFieldMapper.NAME; GetResult getResult = indexShard.getService().get( - get, request.id(), request.type(), getFields, null); + get, request.id(), MapperService.SINGLE_MAPPING_NAME, getFields, null); Fields generatedTermVectors = generateTermVectors(indexShard, getResult.sourceAsMap(), getResult.getFields().values(), request.offsets(), request.perFieldAnalyzer(), validFields); @@ -295,7 +294,7 @@ private static Fields generateTermVectors(IndexShard indexShard, private static Fields generateTermVectorsFromDoc(IndexShard indexShard, TermVectorsRequest request) throws IOException { // parse the document, at the moment we do update the mapping, just like percolate - ParsedDocument parsedDocument = parseDocument(indexShard, indexShard.shardId().getIndexName(), request.type(), request.doc(), + ParsedDocument parsedDocument = parseDocument(indexShard, indexShard.shardId().getIndexName(), request.doc(), request.xContentType(), request.routing()); // select the right fields and generate term vectors @@ -324,12 +323,12 @@ private static Fields generateTermVectorsFromDoc(IndexShard indexShard, TermVect request.offsets(), request.perFieldAnalyzer(), seenFields); } - private static ParsedDocument parseDocument(IndexShard indexShard, String index, String type, BytesReference doc, + private static ParsedDocument parseDocument(IndexShard indexShard, String index, BytesReference doc, XContentType xContentType, String routing) { MapperService mapperService = indexShard.mapperService(); - DocumentMapperForType docMapper = mapperService.documentMapperWithAutoCreate(type); + DocumentMapperForType docMapper = mapperService.documentMapperWithAutoCreate(MapperService.SINGLE_MAPPING_NAME); ParsedDocument parsedDocument = docMapper.getDocumentMapper().parse( - new SourceToParse(index, type, "_id_for_tv_api", doc, xContentType, routing)); + new SourceToParse(index, MapperService.SINGLE_MAPPING_NAME, "_id_for_tv_api", doc, xContentType, routing)); if (docMapper.getMapping() != null) { parsedDocument.addDynamicMappingsUpdate(docMapper.getMapping()); } diff --git a/server/src/main/java/org/elasticsearch/index/translog/BaseTranslogReader.java b/server/src/main/java/org/elasticsearch/index/translog/BaseTranslogReader.java index bfd9e31abcd47..8f5884d02758e 100644 --- a/server/src/main/java/org/elasticsearch/index/translog/BaseTranslogReader.java +++ b/server/src/main/java/org/elasticsearch/index/translog/BaseTranslogReader.java @@ -148,7 +148,7 @@ public long getLastModifiedTime() throws IOException { } /** - * Reads a single opertation from the given location. + * Reads a single operation from the given location. */ Translog.Operation read(Translog.Location location) throws IOException { assert location.generation == this.generation : "generation mismatch expected: " + generation + " got: " + location.generation; diff --git a/server/src/main/java/org/elasticsearch/index/translog/Translog.java b/server/src/main/java/org/elasticsearch/index/translog/Translog.java index 7626270b6cdc5..82ad2046c510b 100644 --- a/server/src/main/java/org/elasticsearch/index/translog/Translog.java +++ b/server/src/main/java/org/elasticsearch/index/translog/Translog.java @@ -63,6 +63,7 @@ import java.util.concurrent.atomic.AtomicBoolean; import java.util.concurrent.locks.ReadWriteLock; import java.util.concurrent.locks.ReentrantReadWriteLock; +import java.util.function.LongConsumer; import java.util.function.LongSupplier; import java.util.regex.Matcher; import java.util.regex.Pattern; @@ -129,6 +130,7 @@ public class Translog extends AbstractIndexShardComponent implements IndexShardC private final LongSupplier primaryTermSupplier; private final String translogUUID; private final TranslogDeletionPolicy deletionPolicy; + private final LongConsumer persistedSequenceNumberConsumer; /** * Creates a new Translog instance. This method will create a new transaction log unless the given {@link TranslogGeneration} is @@ -146,14 +148,18 @@ public class Translog extends AbstractIndexShardComponent implements IndexShardC * examined and stored in the header whenever a new generation is rolled. It's guaranteed from outside * that a new generation is rolled when the term is increased. This guarantee allows to us to validate * and reject operation whose term is higher than the primary term stored in the translog header. + * @param persistedSequenceNumberConsumer a callback that's called whenever an operation with a given sequence number is successfully + * persisted. */ public Translog( final TranslogConfig config, final String translogUUID, TranslogDeletionPolicy deletionPolicy, - final LongSupplier globalCheckpointSupplier, final LongSupplier primaryTermSupplier) throws IOException { + final LongSupplier globalCheckpointSupplier, final LongSupplier primaryTermSupplier, + final LongConsumer persistedSequenceNumberConsumer) throws IOException { super(config.getShardId(), config.getIndexSettings()); this.config = config; this.globalCheckpointSupplier = globalCheckpointSupplier; this.primaryTermSupplier = primaryTermSupplier; + this.persistedSequenceNumberConsumer = persistedSequenceNumberConsumer; this.deletionPolicy = deletionPolicy; this.translogUUID = translogUUID; bigArrays = config.getBigArrays(); @@ -190,7 +196,8 @@ public Translog( boolean success = false; current = null; try { - current = createWriter(checkpoint.generation + 1, getMinFileGeneration(), checkpoint.globalCheckpoint); + current = createWriter(checkpoint.generation + 1, getMinFileGeneration(), checkpoint.globalCheckpoint, + persistedSequenceNumberConsumer); success = true; } finally { // we have to close all the recovered ones otherwise we leak file handles here @@ -471,7 +478,8 @@ public long sizeInBytesByMinGen(long minGeneration) { * @throws IOException if creating the translog failed */ TranslogWriter createWriter(long fileGeneration) throws IOException { - final TranslogWriter writer = createWriter(fileGeneration, getMinFileGeneration(), globalCheckpointSupplier.getAsLong()); + final TranslogWriter writer = createWriter(fileGeneration, getMinFileGeneration(), globalCheckpointSupplier.getAsLong(), + persistedSequenceNumberConsumer); assert writer.sizeInBytes() == DEFAULT_HEADER_SIZE_IN_BYTES : "Mismatch translog header size; " + "empty translog size [" + writer.sizeInBytes() + ", header size [" + DEFAULT_HEADER_SIZE_IN_BYTES + "]"; return writer; @@ -486,7 +494,8 @@ TranslogWriter createWriter(long fileGeneration) throws IOException { * With no readers and no current, a call to {@link #getMinFileGeneration()} would not work. * @param initialGlobalCheckpoint the global checkpoint to be written in the first checkpoint. */ - TranslogWriter createWriter(long fileGeneration, long initialMinTranslogGen, long initialGlobalCheckpoint) throws IOException { + TranslogWriter createWriter(long fileGeneration, long initialMinTranslogGen, long initialGlobalCheckpoint, + LongConsumer persistedSequenceNumberConsumer) throws IOException { final TranslogWriter newFile; try { newFile = TranslogWriter.create( @@ -497,7 +506,8 @@ TranslogWriter createWriter(long fileGeneration, long initialMinTranslogGen, lon getChannelFactory(), config.getBufferSize(), initialMinTranslogGen, initialGlobalCheckpoint, - globalCheckpointSupplier, this::getMinFileGeneration, primaryTermSupplier.getAsLong(), tragedy); + globalCheckpointSupplier, this::getMinFileGeneration, primaryTermSupplier.getAsLong(), tragedy, + persistedSequenceNumberConsumer); } catch (final IOException e) { throw new TranslogException(shardId, "failed to create new translog file", e); } @@ -1868,7 +1878,7 @@ static String createEmptyTranslog(Path location, long initialGlobalCheckpoint, S location.resolve(getFilename(1)), channelFactory, new ByteSizeValue(10), 1, initialGlobalCheckpoint, () -> { throw new UnsupportedOperationException(); }, () -> { throw new UnsupportedOperationException(); }, primaryTerm, - new TragicExceptionHolder()); + new TragicExceptionHolder(), seqNo -> { throw new UnsupportedOperationException(); }); writer.close(); return translogUUID; } diff --git a/server/src/main/java/org/elasticsearch/index/translog/TranslogHeader.java b/server/src/main/java/org/elasticsearch/index/translog/TranslogHeader.java index d47f0d9d7fdfd..7f95b0ad9871a 100644 --- a/server/src/main/java/org/elasticsearch/index/translog/TranslogHeader.java +++ b/server/src/main/java/org/elasticsearch/index/translog/TranslogHeader.java @@ -30,12 +30,11 @@ import org.elasticsearch.common.io.stream.InputStreamStreamInput; import org.elasticsearch.common.io.stream.OutputStreamStreamOutput; +import java.io.EOFException; import java.io.IOException; import java.nio.channels.FileChannel; import java.nio.file.Path; -import static org.elasticsearch.index.seqno.SequenceNumbers.UNASSIGNED_PRIMARY_TERM; - /** * Each translog file is started with a translog header then followed by translog operations. */ @@ -108,56 +107,57 @@ private static int headerSizeInBytes(int version, int uuidLength) { * Read a translog header from the given path and file channel */ static TranslogHeader read(final String translogUUID, final Path path, final FileChannel channel) throws IOException { - // This input is intentionally not closed because closing it will close the FileChannel. - final BufferedChecksumStreamInput in = - new BufferedChecksumStreamInput( + try { + // This input is intentionally not closed because closing it will close the FileChannel. + final BufferedChecksumStreamInput in = + new BufferedChecksumStreamInput( new InputStreamStreamInput(java.nio.channels.Channels.newInputStream(channel), channel.size()), path.toString()); - final int version; - try { - version = CodecUtil.checkHeader(new InputStreamDataInput(in), TRANSLOG_CODEC, VERSION_CHECKSUMS, VERSION_PRIMARY_TERM); - } catch (CorruptIndexException | IndexFormatTooOldException | IndexFormatTooNewException e) { - tryReportOldVersionError(path, channel); - throw new TranslogCorruptedException(path.toString(), "translog header corrupted", e); - } - if (version == VERSION_CHECKSUMS) { - throw new IllegalStateException("pre-2.0 translog found [" + path + "]"); - } - // Read the translogUUID - final int uuidLen = in.readInt(); - if (uuidLen > channel.size()) { - throw new TranslogCorruptedException( - path.toString(), - "UUID length can't be larger than the translog"); - } - final BytesRef uuid = new BytesRef(uuidLen); - uuid.length = uuidLen; - in.read(uuid.bytes, uuid.offset, uuid.length); - final BytesRef expectedUUID = new BytesRef(translogUUID); - if (uuid.bytesEquals(expectedUUID) == false) { - throw new TranslogCorruptedException( + final int version; + try { + version = CodecUtil.checkHeader(new InputStreamDataInput(in), TRANSLOG_CODEC, VERSION_CHECKSUMS, VERSION_PRIMARY_TERM); + } catch (CorruptIndexException | IndexFormatTooOldException | IndexFormatTooNewException e) { + tryReportOldVersionError(path, channel); + throw new TranslogCorruptedException(path.toString(), "translog header corrupted", e); + } + if (version == VERSION_CHECKSUMS) { + throw new IllegalStateException("pre-2.0 translog found [" + path + "]"); + } + if (version == VERSION_CHECKPOINTS) { + throw new IllegalStateException("pre-6.3 translog found [" + path + "]"); + } + // Read the translogUUID + final int uuidLen = in.readInt(); + if (uuidLen > channel.size()) { + throw new TranslogCorruptedException(path.toString(), "UUID length can't be larger than the translog"); + } + if (uuidLen <= 0) { + throw new TranslogCorruptedException(path.toString(), "UUID length must be positive"); + } + final BytesRef uuid = new BytesRef(uuidLen); + uuid.length = uuidLen; + in.read(uuid.bytes, uuid.offset, uuid.length); + final BytesRef expectedUUID = new BytesRef(translogUUID); + if (uuid.bytesEquals(expectedUUID) == false) { + throw new TranslogCorruptedException( path.toString(), "expected shard UUID " + expectedUUID + " but got: " + uuid + - " this translog file belongs to a different translog"); - } - // Read the primary term - final long primaryTerm; - if (version == VERSION_PRIMARY_TERM) { - primaryTerm = in.readLong(); - } else { - assert version == VERSION_CHECKPOINTS : "Unknown header version [" + version + "]"; - primaryTerm = UNASSIGNED_PRIMARY_TERM; - } - // Verify the checksum - if (version >= VERSION_PRIMARY_TERM) { + " this translog file belongs to a different translog"); + } + // Read the primary term + assert version == VERSION_PRIMARY_TERM; + final long primaryTerm = in.readLong(); + // Verify the checksum Translog.verifyChecksum(in); + assert primaryTerm >= 0 : "Primary term must be non-negative [" + primaryTerm + "]; translog path [" + path + "]"; + + final int headerSizeInBytes = headerSizeInBytes(version, uuid.length); + assert channel.position() == headerSizeInBytes : + "Header is not fully read; header size [" + headerSizeInBytes + "], position [" + channel.position() + "]"; + return new TranslogHeader(translogUUID, primaryTerm, headerSizeInBytes); + } catch (EOFException e) { + throw new TranslogCorruptedException(path.toString(), "translog header truncated", e); } - assert primaryTerm >= 0 : "Primary term must be non-negative [" + primaryTerm + "]; translog path [" + path + "]"; - - final int headerSizeInBytes = headerSizeInBytes(version, uuid.length); - assert channel.position() == headerSizeInBytes : - "Header is not fully read; header size [" + headerSizeInBytes + "], position [" + channel.position() + "]"; - return new TranslogHeader(translogUUID, primaryTerm, headerSizeInBytes); } private static void tryReportOldVersionError(final Path path, final FileChannel channel) throws IOException { diff --git a/server/src/main/java/org/elasticsearch/index/translog/TranslogSnapshot.java b/server/src/main/java/org/elasticsearch/index/translog/TranslogSnapshot.java index bff3e4eb2f540..4cfc886219f60 100644 --- a/server/src/main/java/org/elasticsearch/index/translog/TranslogSnapshot.java +++ b/server/src/main/java/org/elasticsearch/index/translog/TranslogSnapshot.java @@ -76,7 +76,7 @@ public Translog.Operation next() throws IOException { return null; } - protected Translog.Operation readOperation() throws IOException { + private Translog.Operation readOperation() throws IOException { final int opSize = readSize(reusableBuffer, position); reuse = checksummedStream(reusableBuffer, position, opSize, reuse); Translog.Operation op = read(reuse); @@ -93,15 +93,19 @@ public long sizeInBytes() { * reads an operation at the given position into the given buffer. */ protected void readBytes(ByteBuffer buffer, long position) throws IOException { - if (position >= length) { - throw new EOFException("read requested past EOF. pos [" + position + "] end: [" + length + "], generation: [" + - getGeneration() + "], path: [" + path + "]"); - } - if (position < getFirstOperationOffset()) { - throw new IOException("read requested before position of first ops. pos [" + position + "] first op on: [" + - getFirstOperationOffset() + "], generation: [" + getGeneration() + "], path: [" + path + "]"); + try { + if (position >= length) { + throw new EOFException("read requested past EOF. pos [" + position + "] end: [" + length + "], generation: [" + + getGeneration() + "], path: [" + path + "]"); + } + if (position < getFirstOperationOffset()) { + throw new IOException("read requested before position of first ops. pos [" + position + "] first op on: [" + + getFirstOperationOffset() + "], generation: [" + getGeneration() + "], path: [" + path + "]"); + } + Channels.readFromFileChannelWithEofException(channel, position, buffer); + } catch (EOFException e) { + throw new TranslogCorruptedException(path.toString(), "translog truncated", e); } - Channels.readFromFileChannelWithEofException(channel, position, buffer); } @Override diff --git a/server/src/main/java/org/elasticsearch/index/translog/TranslogWriter.java b/server/src/main/java/org/elasticsearch/index/translog/TranslogWriter.java index 6b00b0c5db3ff..0695a2bf65010 100644 --- a/server/src/main/java/org/elasticsearch/index/translog/TranslogWriter.java +++ b/server/src/main/java/org/elasticsearch/index/translog/TranslogWriter.java @@ -19,6 +19,8 @@ package org.elasticsearch.index.translog; +import com.carrotsearch.hppc.LongArrayList; +import com.carrotsearch.hppc.procedures.LongProcedure; import org.apache.lucene.store.AlreadyClosedException; import org.elasticsearch.core.internal.io.IOUtils; import org.elasticsearch.Assertions; @@ -42,6 +44,7 @@ import java.util.Map; import java.util.Objects; import java.util.concurrent.atomic.AtomicBoolean; +import java.util.function.LongConsumer; import java.util.function.LongSupplier; public class TranslogWriter extends BaseTranslogReader implements Closeable { @@ -64,10 +67,15 @@ public class TranslogWriter extends BaseTranslogReader implements Closeable { private final LongSupplier globalCheckpointSupplier; private final LongSupplier minTranslogGenerationSupplier; + // callback that's called whenever an operation with a given sequence number is successfully persisted. + private final LongConsumer persistedSequenceNumberConsumer; + protected final AtomicBoolean closed = new AtomicBoolean(false); // lock order synchronized(syncLock) -> synchronized(this) private final Object syncLock = new Object(); + private LongArrayList nonFsyncedSequenceNumbers; + private final Map> seenSequenceNumbers; private TranslogWriter( @@ -78,7 +86,8 @@ private TranslogWriter( final Path path, final ByteSizeValue bufferSize, final LongSupplier globalCheckpointSupplier, LongSupplier minTranslogGenerationSupplier, TranslogHeader header, - TragicExceptionHolder tragedy) + TragicExceptionHolder tragedy, + final LongConsumer persistedSequenceNumberConsumer) throws IOException { super(initialCheckpoint.generation, channel, path, header); @@ -97,6 +106,8 @@ private TranslogWriter( this.maxSeqNo = initialCheckpoint.maxSeqNo; assert initialCheckpoint.trimmedAboveSeqNo == SequenceNumbers.UNASSIGNED_SEQ_NO : initialCheckpoint.trimmedAboveSeqNo; this.globalCheckpointSupplier = globalCheckpointSupplier; + this.nonFsyncedSequenceNumbers = new LongArrayList(64); + this.persistedSequenceNumberConsumer = persistedSequenceNumberConsumer; this.seenSequenceNumbers = Assertions.ENABLED ? new HashMap<>() : null; this.tragedy = tragedy; } @@ -104,7 +115,7 @@ private TranslogWriter( public static TranslogWriter create(ShardId shardId, String translogUUID, long fileGeneration, Path file, ChannelFactory channelFactory, ByteSizeValue bufferSize, final long initialMinTranslogGen, long initialGlobalCheckpoint, final LongSupplier globalCheckpointSupplier, final LongSupplier minTranslogGenerationSupplier, - final long primaryTerm, TragicExceptionHolder tragedy) + final long primaryTerm, TragicExceptionHolder tragedy, LongConsumer persistedSequenceNumberConsumer) throws IOException { final FileChannel channel = channelFactory.open(file); try { @@ -125,7 +136,7 @@ public static TranslogWriter create(ShardId shardId, String translogUUID, long f writerGlobalCheckpointSupplier = globalCheckpointSupplier; } return new TranslogWriter(channelFactory, shardId, checkpoint, channel, file, bufferSize, - writerGlobalCheckpointSupplier, minTranslogGenerationSupplier, header, tragedy); + writerGlobalCheckpointSupplier, minTranslogGenerationSupplier, header, tragedy, persistedSequenceNumberConsumer); } catch (Exception exception) { // if we fail to bake the file-generation into the checkpoint we stick with the file and once we recover and that // file exists we remove it. We only apply this logic to the checkpoint.generation+1 any other file with a higher generation @@ -177,6 +188,8 @@ public synchronized Translog.Location add(final BytesReference data, final long minSeqNo = SequenceNumbers.min(minSeqNo, seqNo); maxSeqNo = SequenceNumbers.max(maxSeqNo, seqNo); + nonFsyncedSequenceNumbers.add(seqNo); + operationCounter++; assert assertNoSeqNumberConflict(seqNo, data); @@ -338,7 +351,9 @@ private long getWrittenOffset() throws IOException { * @return true if this call caused an actual sync operation */ public boolean syncUpTo(long offset) throws IOException { + boolean synced = false; if (lastSyncedCheckpoint.offset < offset && syncNeeded()) { + LongArrayList flushedSequenceNumbers = null; synchronized (syncLock) { // only one sync/checkpoint should happen concurrently but we wait if (lastSyncedCheckpoint.offset < offset && syncNeeded()) { // double checked locking - we don't want to fsync unless we have to and now that we have @@ -349,6 +364,8 @@ public boolean syncUpTo(long offset) throws IOException { try { outputStream.flush(); checkpointToSync = getCheckpoint(); + flushedSequenceNumbers = nonFsyncedSequenceNumbers; + nonFsyncedSequenceNumbers = new LongArrayList(64); } catch (final Exception ex) { closeWithTragicEvent(ex); throw ex; @@ -366,11 +383,14 @@ public boolean syncUpTo(long offset) throws IOException { assert lastSyncedCheckpoint.offset <= checkpointToSync.offset : "illegal state: " + lastSyncedCheckpoint.offset + " <= " + checkpointToSync.offset; lastSyncedCheckpoint = checkpointToSync; // write protected by syncLock - return true; + synced = true; } } + if (flushedSequenceNumbers != null) { + flushedSequenceNumbers.forEach((LongProcedure) persistedSequenceNumberConsumer::accept); + } } - return false; + return synced; } @Override diff --git a/server/src/main/java/org/elasticsearch/index/translog/TruncateTranslogAction.java b/server/src/main/java/org/elasticsearch/index/translog/TruncateTranslogAction.java index e99128fd3e0a2..7cf165a5b112d 100644 --- a/server/src/main/java/org/elasticsearch/index/translog/TruncateTranslogAction.java +++ b/server/src/main/java/org/elasticsearch/index/translog/TruncateTranslogAction.java @@ -168,7 +168,6 @@ public void execute(Terminal terminal, ShardPath shardPath, Directory indexDirec private boolean isTranslogClean(ShardPath shardPath, String translogUUID) throws IOException { // perform clean check of translog instead of corrupted marker file - boolean clean = true; try { final Path translogPath = shardPath.resolveTranslog(); final long translogGlobalCheckpoint = Translog.readGlobalCheckpoint(translogPath, translogUUID); @@ -182,20 +181,21 @@ private boolean isTranslogClean(ShardPath shardPath, String translogUUID) throws new TranslogDeletionPolicy(indexSettings.getTranslogRetentionSize().getBytes(), indexSettings.getTranslogRetentionAge().getMillis()); try (Translog translog = new Translog(translogConfig, translogUUID, - translogDeletionPolicy, () -> translogGlobalCheckpoint, () -> primaryTerm); + translogDeletionPolicy, () -> translogGlobalCheckpoint, () -> primaryTerm, seqNo -> {}); Translog.Snapshot snapshot = translog.newSnapshot()) { + //noinspection StatementWithEmptyBody we are just checking that we can iterate through the whole snapshot while (snapshot.next() != null) { - // just iterate over snapshot } } + return true; } catch (TranslogCorruptedException e) { - clean = false; + return false; } - return clean; } /** Write a checkpoint file to the given location with the given generation */ - static void writeEmptyCheckpoint(Path filename, int translogLength, long translogGeneration, long globalCheckpoint) throws IOException { + private static void writeEmptyCheckpoint(Path filename, int translogLength, long translogGeneration, long globalCheckpoint) + throws IOException { Checkpoint emptyCheckpoint = Checkpoint.emptyTranslogCheckpoint(translogLength, translogGeneration, globalCheckpoint, translogGeneration); Checkpoint.write(FileChannel::open, filename, emptyCheckpoint, @@ -234,7 +234,7 @@ private String deletingFilesDetails(Path translogPath, Set files) { } /** Return a Set of all files in a given directory */ - public static Set filesInDirectory(Path directory) throws IOException { + private static Set filesInDirectory(Path directory) throws IOException { Set files = new TreeSet<>(); try (DirectoryStream stream = Files.newDirectoryStream(directory)) { for (Path file : stream) { diff --git a/server/src/main/java/org/elasticsearch/indices/IndicesModule.java b/server/src/main/java/org/elasticsearch/indices/IndicesModule.java index bef05ecda9fd8..fa1fa35588ec0 100644 --- a/server/src/main/java/org/elasticsearch/indices/IndicesModule.java +++ b/server/src/main/java/org/elasticsearch/indices/IndicesModule.java @@ -55,7 +55,6 @@ import org.elasticsearch.index.mapper.TextFieldMapper; import org.elasticsearch.index.mapper.TypeFieldMapper; import org.elasticsearch.index.mapper.VersionFieldMapper; -import org.elasticsearch.index.seqno.GlobalCheckpointSyncAction; import org.elasticsearch.index.shard.PrimaryReplicaSyncer; import org.elasticsearch.indices.cluster.IndicesClusterStateService; import org.elasticsearch.indices.flush.SyncedFlushService; @@ -244,7 +243,6 @@ protected void configure() { bind(IndicesClusterStateService.class).asEagerSingleton(); bind(SyncedFlushService.class).asEagerSingleton(); bind(TransportNodesListShardStoreMetaData.class).asEagerSingleton(); - bind(GlobalCheckpointSyncAction.class).asEagerSingleton(); bind(TransportResyncReplicationAction.class).asEagerSingleton(); bind(PrimaryReplicaSyncer.class).asEagerSingleton(); } diff --git a/server/src/main/java/org/elasticsearch/indices/IndicesService.java b/server/src/main/java/org/elasticsearch/indices/IndicesService.java index 16382d15cd325..79f4de40aeafc 100644 --- a/server/src/main/java/org/elasticsearch/indices/IndicesService.java +++ b/server/src/main/java/org/elasticsearch/indices/IndicesService.java @@ -66,7 +66,6 @@ import org.elasticsearch.common.util.Maps; import org.elasticsearch.common.util.concurrent.AbstractRefCounted; import org.elasticsearch.common.util.concurrent.EsExecutors; -import org.elasticsearch.common.util.concurrent.EsRejectedExecutionException; import org.elasticsearch.common.util.iterable.Iterables; import org.elasticsearch.common.xcontent.LoggingDeprecationHandler; import org.elasticsearch.common.xcontent.NamedXContentRegistry; @@ -1220,13 +1219,7 @@ public void run() { } // Reschedule itself to run again if not closed if (closed.get() == false) { - try { - threadPool.schedule(this, interval, ThreadPool.Names.SAME); - } catch (EsRejectedExecutionException e) { - if (closed.get() == false) { - throw e; - } - } + threadPool.scheduleUnlessShuttingDown(interval, ThreadPool.Names.SAME, this); } } diff --git a/server/src/main/java/org/elasticsearch/indices/cluster/IndicesClusterStateService.java b/server/src/main/java/org/elasticsearch/indices/cluster/IndicesClusterStateService.java index 1ce39e283b1da..25a74d8e7773a 100644 --- a/server/src/main/java/org/elasticsearch/indices/cluster/IndicesClusterStateService.java +++ b/server/src/main/java/org/elasticsearch/indices/cluster/IndicesClusterStateService.java @@ -22,10 +22,13 @@ import org.apache.logging.log4j.LogManager; import org.apache.logging.log4j.Logger; import org.apache.logging.log4j.message.ParameterizedMessage; +import org.apache.lucene.store.AlreadyClosedException; import org.apache.lucene.store.LockObtainFailedException; +import org.elasticsearch.ExceptionsHelper; import org.elasticsearch.ResourceAlreadyExistsException; import org.elasticsearch.action.ActionListener; import org.elasticsearch.action.support.replication.ReplicationResponse; +import org.elasticsearch.client.node.NodeClient; import org.elasticsearch.cluster.ClusterChangedEvent; import org.elasticsearch.cluster.ClusterState; import org.elasticsearch.cluster.ClusterStateApplier; @@ -47,6 +50,7 @@ import org.elasticsearch.common.unit.TimeValue; import org.elasticsearch.common.util.concurrent.AbstractRunnable; import org.elasticsearch.common.util.concurrent.ConcurrentCollections; +import org.elasticsearch.common.util.concurrent.ThreadContext; import org.elasticsearch.env.ShardLockObtainFailedException; import org.elasticsearch.gateway.GatewayService; import org.elasticsearch.index.Index; @@ -61,6 +65,7 @@ import org.elasticsearch.index.seqno.RetentionLeases; import org.elasticsearch.index.shard.IndexEventListener; import org.elasticsearch.index.shard.IndexShard; +import org.elasticsearch.index.shard.IndexShardClosedException; import org.elasticsearch.index.shard.IndexShardRelocatedException; import org.elasticsearch.index.shard.IndexShardState; import org.elasticsearch.index.shard.PrimaryReplicaSyncer; @@ -99,7 +104,7 @@ import static org.elasticsearch.indices.cluster.IndicesClusterStateService.AllocatedIndices.IndexRemovalReason.NO_LONGER_ASSIGNED; import static org.elasticsearch.indices.cluster.IndicesClusterStateService.AllocatedIndices.IndexRemovalReason.REOPENED; -public class IndicesClusterStateService extends AbstractLifecycleComponent implements ClusterStateApplier { +public class IndicesClusterStateService extends AbstractLifecycleComponent implements ClusterStateApplier, RetentionLeaseSyncer { private static final Logger logger = LogManager.getLogger(IndicesClusterStateService.class); final AllocatedIndices> indicesService; @@ -122,8 +127,7 @@ public class IndicesClusterStateService extends AbstractLifecycleComponent imple private final boolean sendRefreshMapping; private final List buildInIndexListener; private final PrimaryReplicaSyncer primaryReplicaSyncer; - private final Consumer globalCheckpointSyncer; - private final RetentionLeaseSyncer retentionLeaseSyncer; + private final NodeClient client; @Inject public IndicesClusterStateService( @@ -140,12 +144,10 @@ public IndicesClusterStateService( final PeerRecoverySourceService peerRecoverySourceService, final SnapshotShardsService snapshotShardsService, final PrimaryReplicaSyncer primaryReplicaSyncer, - final GlobalCheckpointSyncAction globalCheckpointSyncAction, - final RetentionLeaseSyncAction retentionLeaseSyncAction, - final RetentionLeaseBackgroundSyncAction retentionLeaseBackgroundSyncAction) { + final NodeClient client) { this( settings, - indicesService, + (AllocatedIndices>) indicesService, clusterService, threadPool, recoveryTargetService, @@ -157,21 +159,7 @@ public IndicesClusterStateService( peerRecoverySourceService, snapshotShardsService, primaryReplicaSyncer, - globalCheckpointSyncAction::updateGlobalCheckpointForShard, - new RetentionLeaseSyncer() { - @Override - public void sync( - final ShardId shardId, - final RetentionLeases retentionLeases, - final ActionListener listener) { - Objects.requireNonNull(retentionLeaseSyncAction).sync(shardId, retentionLeases, listener); - } - - @Override - public void backgroundSync(final ShardId shardId, final RetentionLeases retentionLeases) { - Objects.requireNonNull(retentionLeaseBackgroundSyncAction).backgroundSync(shardId, retentionLeases); - } - }); + client); } // for tests @@ -189,8 +177,7 @@ public void backgroundSync(final ShardId shardId, final RetentionLeases retentio final PeerRecoverySourceService peerRecoverySourceService, final SnapshotShardsService snapshotShardsService, final PrimaryReplicaSyncer primaryReplicaSyncer, - final Consumer globalCheckpointSyncer, - final RetentionLeaseSyncer retentionLeaseSyncer) { + final NodeClient client) { this.settings = settings; this.buildInIndexListener = Arrays.asList( @@ -207,9 +194,8 @@ public void backgroundSync(final ShardId shardId, final RetentionLeases retentio this.nodeMappingRefreshAction = nodeMappingRefreshAction; this.repositoriesService = repositoriesService; this.primaryReplicaSyncer = primaryReplicaSyncer; - this.globalCheckpointSyncer = globalCheckpointSyncer; - this.retentionLeaseSyncer = Objects.requireNonNull(retentionLeaseSyncer); this.sendRefreshMapping = settings.getAsBoolean("indices.cluster.send_refresh_mapping", true); + this.client = client; } @Override @@ -299,6 +285,74 @@ private void updateFailedShardsCache(final ClusterState state) { } } + // overrideable by tests + protected void updateGlobalCheckpointForShard(final ShardId shardId) { + final ThreadContext threadContext = threadPool.getThreadContext(); + try (ThreadContext.StoredContext ignore = threadContext.stashContext()) { + threadContext.markAsSystemContext(); + client.executeLocally(GlobalCheckpointSyncAction.TYPE, new GlobalCheckpointSyncAction.Request(shardId), + ActionListener.wrap(r -> { + }, e -> { + if (ExceptionsHelper.unwrap(e, AlreadyClosedException.class, IndexShardClosedException.class) == null) { + getLogger().info(new ParameterizedMessage("{} global checkpoint sync failed", shardId), e); + } + })); + } + } + + @Override + public void sync(ShardId shardId, RetentionLeases retentionLeases, ActionListener listener) { + Objects.requireNonNull(shardId); + Objects.requireNonNull(retentionLeases); + Objects.requireNonNull(listener); + final ThreadContext threadContext = threadPool.getThreadContext(); + try (ThreadContext.StoredContext ignore = threadContext.stashContext()) { + // we have to execute under the system context so that if security is enabled the sync is authorized + threadContext.markAsSystemContext(); + client.executeLocally(RetentionLeaseSyncAction.TYPE, + new RetentionLeaseSyncAction.Request(shardId, retentionLeases), + ActionListener.wrap( + listener::onResponse, + e -> { + if (ExceptionsHelper.unwrap(e, AlreadyClosedException.class, IndexShardClosedException.class) == null) { + getLogger().warn(new ParameterizedMessage("{} retention lease sync failed", shardId), e); + } + listener.onFailure(e); + })); + } + } + + @Override + public void backgroundSync(ShardId shardId, RetentionLeases retentionLeases) { + Objects.requireNonNull(shardId); + Objects.requireNonNull(retentionLeases); + final ThreadContext threadContext = threadPool.getThreadContext(); + try (ThreadContext.StoredContext ignore = threadContext.stashContext()) { + // we have to execute under the system context so that if security is enabled the sync is authorized + threadContext.markAsSystemContext(); + client.executeLocally(RetentionLeaseBackgroundSyncAction.TYPE, + new RetentionLeaseBackgroundSyncAction.Request(shardId, retentionLeases), + ActionListener.wrap( + r -> {}, + e -> { + if (ExceptionsHelper.isTransportStoppedForAction(e, RetentionLeaseBackgroundSyncAction.ACTION_NAME + "[p]")) { + // we are likely shutting down + return; + } + if (ExceptionsHelper.unwrap(e, AlreadyClosedException.class, IndexShardClosedException.class) != null) { + // the shard is closed + return; + } + getLogger().warn(new ParameterizedMessage("{} retention lease background sync failed", shardId), e); + })); + } + } + + // overrideable by tests + Logger getLogger() { + return logger; + } + /** * Deletes indices (with shard data). * @@ -606,8 +660,8 @@ private void createShard(DiscoveryNodes nodes, RoutingTable routingTable, ShardR new RecoveryListener(shardRouting, primaryTerm), repositoriesService, failedShardHandler, - globalCheckpointSyncer, - retentionLeaseSyncer); + this::updateGlobalCheckpointForShard, + this); } catch (Exception e) { failAndRemoveShard(shardRouting, true, "failed to create shard", e, state); } diff --git a/server/src/main/java/org/elasticsearch/indices/recovery/PeerRecoveryTargetService.java b/server/src/main/java/org/elasticsearch/indices/recovery/PeerRecoveryTargetService.java index 6b1a893667f2c..92f1558d71eca 100644 --- a/server/src/main/java/org/elasticsearch/indices/recovery/PeerRecoveryTargetService.java +++ b/server/src/main/java/org/elasticsearch/indices/recovery/PeerRecoveryTargetService.java @@ -531,11 +531,11 @@ class FilesInfoRequestHandler implements TransportRequestHandler listener = new ChannelActionListener<>(channel, Actions.FILES_INFO, request); + recoveryRef.target().receiveFileInfo( + request.phase1FileNames, request.phase1FileSizes, request.phase1ExistingFileNames, request.phase1ExistingFileSizes, + request.totalTranslogOps, ActionListener.map(listener, nullVal -> TransportResponse.Empty.INSTANCE)); } } } @@ -544,10 +544,10 @@ class CleanFilesRequestHandler implements TransportRequestHandler listener = new ChannelActionListener<>(channel, Actions.CLEAN_FILES, request); + recoveryRef.target().cleanFiles(request.totalTranslogOps(), request.getGlobalCheckpoint(), request.sourceMetaSnapshot(), + ActionListener.map(listener, nullVal -> TransportResponse.Empty.INSTANCE)); } } } diff --git a/server/src/main/java/org/elasticsearch/indices/recovery/RecoverySourceHandler.java b/server/src/main/java/org/elasticsearch/indices/recovery/RecoverySourceHandler.java index 4b89e75691a76..3ae9598124b05 100644 --- a/server/src/main/java/org/elasticsearch/indices/recovery/RecoverySourceHandler.java +++ b/server/src/main/java/org/elasticsearch/indices/recovery/RecoverySourceHandler.java @@ -39,6 +39,7 @@ import org.elasticsearch.common.bytes.BytesArray; import org.elasticsearch.common.collect.Tuple; import org.elasticsearch.common.lease.Releasable; +import org.elasticsearch.common.lease.Releasables; import org.elasticsearch.common.logging.Loggers; import org.elasticsearch.common.lucene.store.InputStreamIndexInput; import org.elasticsearch.common.unit.ByteSizeValue; @@ -74,7 +75,7 @@ import java.util.concurrent.atomic.AtomicInteger; import java.util.concurrent.atomic.AtomicReference; import java.util.function.Consumer; -import java.util.function.Supplier; +import java.util.function.IntSupplier; import java.util.stream.StreamSupport; import static org.elasticsearch.index.seqno.SequenceNumbers.NO_OPS_PERFORMED; @@ -158,15 +159,21 @@ public void recoverToTarget(ActionListener listener) { final long startingSeqNo; final boolean isSequenceNumberBasedRecovery = request.startingSeqNo() != SequenceNumbers.UNASSIGNED_SEQ_NO && isTargetSameHistory() && shard.hasCompleteHistoryOperations("peer-recovery", request.startingSeqNo()); - final SendFileResult sendFileResult; + + final StepListener sendFileStep = new StepListener<>(); + final StepListener prepareEngineStep = new StepListener<>(); + final StepListener sendSnapshotStep = new StepListener<>(); + final StepListener finalizeStep = new StepListener<>(); + if (isSequenceNumberBasedRecovery) { logger.trace("performing sequence numbers based recovery. starting at [{}]", request.startingSeqNo()); startingSeqNo = request.startingSeqNo(); - sendFileResult = SendFileResult.EMPTY; + sendFileStep.onResponse(SendFileResult.EMPTY); } else { - final Engine.IndexCommitRef phase1Snapshot; + final Engine.IndexCommitRef safeCommitRef; try { - phase1Snapshot = shard.acquireSafeIndexCommit(); + safeCommitRef = shard.acquireSafeIndexCommit(); + resources.add(safeCommitRef); } catch (final Exception e) { throw new RecoveryEngineException(shard.shardId(), 1, "snapshot failed", e); } @@ -175,24 +182,29 @@ public void recoverToTarget(ActionListener listener) { startingSeqNo = 0; try { final int estimateNumOps = shard.estimateNumberOfHistoryOperations("peer-recovery", startingSeqNo); - sendFileResult = phase1(phase1Snapshot.getIndexCommit(), shard.getGlobalCheckpoint(), () -> estimateNumOps); + shard.store().incRef(); + final Releasable releaseStore = Releasables.releaseOnce(shard.store()::decRef); + resources.add(releaseStore); + sendFileStep.whenComplete(r -> IOUtils.close(safeCommitRef, releaseStore), e -> { + try { + IOUtils.close(safeCommitRef, releaseStore); + } catch (final IOException ex) { + logger.warn("releasing snapshot caused exception", ex); + } + }); + phase1(safeCommitRef.getIndexCommit(), shard.getLastKnownGlobalCheckpoint(), () -> estimateNumOps, sendFileStep); } catch (final Exception e) { - throw new RecoveryEngineException(shard.shardId(), 1, "phase1 failed", e); - } finally { - try { - IOUtils.close(phase1Snapshot); - } catch (final IOException ex) { - logger.warn("releasing snapshot caused exception", ex); - } + throw new RecoveryEngineException(shard.shardId(), 1, "sendFileStep failed", e); } } assert startingSeqNo >= 0 : "startingSeqNo must be non negative. got: " + startingSeqNo; - final StepListener prepareEngineStep = new StepListener<>(); - // For a sequence based recovery, the target can keep its local translog - prepareTargetForTranslog(isSequenceNumberBasedRecovery == false, - shard.estimateNumberOfHistoryOperations("peer-recovery", startingSeqNo), prepareEngineStep); - final StepListener sendSnapshotStep = new StepListener<>(); + sendFileStep.whenComplete(r -> { + // For a sequence based recovery, the target can keep its local translog + prepareTargetForTranslog(isSequenceNumberBasedRecovery == false, + shard.estimateNumberOfHistoryOperations("peer-recovery", startingSeqNo), prepareEngineStep); + }, onFailure); + prepareEngineStep.whenComplete(prepareEngineTime -> { /* * add shard to replication group (shard will receive replication requests from this point on) now that engine is open. @@ -229,12 +241,12 @@ public void recoverToTarget(ActionListener listener) { }, onFailure); - final StepListener finalizeStep = new StepListener<>(); sendSnapshotStep.whenComplete(r -> finalizeRecovery(r.targetLocalCheckpoint, finalizeStep), onFailure); finalizeStep.whenComplete(r -> { final long phase1ThrottlingWaitTime = 0L; // TODO: return the actual throttle time final SendSnapshotResult sendSnapshotResult = sendSnapshotStep.result(); + final SendFileResult sendFileResult = sendFileStep.result(); final RecoveryResponse response = new RecoveryResponse(sendFileResult.phase1FileNames, sendFileResult.phase1FileSizes, sendFileResult.phase1ExistingFileNames, sendFileResult.phase1ExistingFileSizes, sendFileResult.totalSize, sendFileResult.existingTotalSize, sendFileResult.took.millis(), phase1ThrottlingWaitTime, @@ -330,18 +342,17 @@ static final class SendFileResult { * segments that are missing. Only segments that have the same size and * checksum can be reused */ - public SendFileResult phase1(final IndexCommit snapshot, final long globalCheckpoint, final Supplier translogOps) { + void phase1(IndexCommit snapshot, long globalCheckpoint, IntSupplier translogOps, ActionListener listener) { cancellableThreads.checkForCancel(); // Total size of segment files that are recovered - long totalSize = 0; + long totalSizeInBytes = 0; // Total size of segment files that were able to be re-used - long existingTotalSize = 0; + long existingTotalSizeInBytes = 0; final List phase1FileNames = new ArrayList<>(); final List phase1FileSizes = new ArrayList<>(); final List phase1ExistingFileNames = new ArrayList<>(); final List phase1ExistingFileSizes = new ArrayList<>(); final Store store = shard.store(); - store.incRef(); try { StopWatch stopWatch = new StopWatch().start(); final Store.MetadataSnapshot recoverySourceMetadata; @@ -367,12 +378,12 @@ public SendFileResult phase1(final IndexCommit snapshot, final long globalCheckp for (StoreFileMetaData md : diff.identical) { phase1ExistingFileNames.add(md.name()); phase1ExistingFileSizes.add(md.length()); - existingTotalSize += md.length(); + existingTotalSizeInBytes += md.length(); if (logger.isTraceEnabled()) { logger.trace("recovery [phase1]: not recovering [{}], exist in local store and has checksum [{}]," + " size [{}]", md.name(), md.checksum(), md.length()); } - totalSize += md.length(); + totalSizeInBytes += md.length(); } List phase1Files = new ArrayList<>(diff.different.size() + diff.missing.size()); phase1Files.addAll(diff.different); @@ -386,75 +397,41 @@ public SendFileResult phase1(final IndexCommit snapshot, final long globalCheckp } phase1FileNames.add(md.name()); phase1FileSizes.add(md.length()); - totalSize += md.length(); + totalSizeInBytes += md.length(); } logger.trace("recovery [phase1]: recovering_files [{}] with total_size [{}], reusing_files [{}] with total_size [{}]", - phase1FileNames.size(), new ByteSizeValue(totalSize), - phase1ExistingFileNames.size(), new ByteSizeValue(existingTotalSize)); - cancellableThreads.execute(() -> recoveryTarget.receiveFileInfo( - phase1FileNames, phase1FileSizes, phase1ExistingFileNames, phase1ExistingFileSizes, translogOps.get())); - sendFiles(store, phase1Files.toArray(new StoreFileMetaData[0]), translogOps); - // Send the CLEAN_FILES request, which takes all of the files that - // were transferred and renames them from their temporary file - // names to the actual file names. It also writes checksums for - // the files after they have been renamed. - // - // Once the files have been renamed, any other files that are not - // related to this recovery (out of date segments, for example) - // are deleted - try { - cancellableThreads.executeIO(() -> - recoveryTarget.cleanFiles(translogOps.get(), globalCheckpoint, recoverySourceMetadata)); - } catch (RemoteTransportException | IOException targetException) { - final IOException corruptIndexException; - // we realized that after the index was copied and we wanted to finalize the recovery - // the index was corrupted: - // - maybe due to a broken segments file on an empty index (transferred with no checksum) - // - maybe due to old segments without checksums or length only checks - if ((corruptIndexException = ExceptionsHelper.unwrapCorruption(targetException)) != null) { - try { - final Store.MetadataSnapshot recoverySourceMetadata1 = store.getMetadata(snapshot); - StoreFileMetaData[] metadata = - StreamSupport.stream(recoverySourceMetadata1.spliterator(), false).toArray(StoreFileMetaData[]::new); - ArrayUtil.timSort(metadata, Comparator.comparingLong(StoreFileMetaData::length)); // check small files first - for (StoreFileMetaData md : metadata) { - cancellableThreads.checkForCancel(); - logger.debug("checking integrity for file {} after remove corruption exception", md); - if (store.checkIntegrityNoException(md) == false) { // we are corrupted on the primary -- fail! - shard.failShard("recovery", corruptIndexException); - logger.warn("Corrupted file detected {} checksum mismatch", md); - throw corruptIndexException; - } - } - } catch (IOException ex) { - targetException.addSuppressed(ex); - throw targetException; - } - // corruption has happened on the way to replica - RemoteTransportException exception = new RemoteTransportException("File corruption occurred on recovery but " + - "checksums are ok", null); - exception.addSuppressed(targetException); - logger.warn(() -> new ParameterizedMessage( - "{} Remote file corruption during finalization of recovery on node {}. local checksum OK", - shard.shardId(), request.targetNode()), corruptIndexException); - throw exception; - } else { - throw targetException; - } - } + phase1FileNames.size(), new ByteSizeValue(totalSizeInBytes), + phase1ExistingFileNames.size(), new ByteSizeValue(existingTotalSizeInBytes)); + final StepListener sendFileInfoStep = new StepListener<>(); + final StepListener cleanFilesStep = new StepListener<>(); + cancellableThreads.execute(() -> + recoveryTarget.receiveFileInfo(phase1FileNames, phase1FileSizes, phase1ExistingFileNames, + phase1ExistingFileSizes, translogOps.getAsInt(), sendFileInfoStep)); + + sendFileInfoStep.whenComplete(r -> { + sendFiles(store, phase1Files.toArray(new StoreFileMetaData[0]), translogOps); + cleanFiles(store, recoverySourceMetadata, translogOps, globalCheckpoint, cleanFilesStep); + }, listener::onFailure); + + final long totalSize = totalSizeInBytes; + final long existingTotalSize = existingTotalSizeInBytes; + cleanFilesStep.whenComplete(r -> { + final TimeValue took = stopWatch.totalTime(); + logger.trace("recovery [phase1]: took [{}]", took); + listener.onResponse(new SendFileResult(phase1FileNames, phase1FileSizes, totalSize, phase1ExistingFileNames, + phase1ExistingFileSizes, existingTotalSize, took)); + }, listener::onFailure); } else { logger.trace("skipping [phase1]- identical sync id [{}] found on both source and target", recoverySourceMetadata.getSyncId()); + final TimeValue took = stopWatch.totalTime(); + logger.trace("recovery [phase1]: took [{}]", took); + listener.onResponse(new SendFileResult(phase1FileNames, phase1FileSizes, totalSizeInBytes, phase1ExistingFileNames, + phase1ExistingFileSizes, existingTotalSizeInBytes, took)); } - final TimeValue took = stopWatch.totalTime(); - logger.trace("recovery [phase1]: took [{}]", took); - return new SendFileResult(phase1FileNames, phase1FileSizes, totalSize, phase1ExistingFileNames, - phase1ExistingFileSizes, existingTotalSize, took); } catch (Exception e) { - throw new RecoverFilesRecoveryException(request.shardId(), phase1FileNames.size(), new ByteSizeValue(totalSize), e); - } finally { - store.decRef(); + throw new RecoverFilesRecoveryException(request.shardId(), phase1FileNames.size(), new ByteSizeValue(totalSizeInBytes), e); } } @@ -641,7 +618,7 @@ void finalizeRecovery(final long targetLocalCheckpoint, final ActionListener shard.markAllocationIdAsInSync(request.targetAllocationId(), targetLocalCheckpoint), shardId + " marking " + request.targetAllocationId() + " as in sync", shard, cancellableThreads, logger); - final long globalCheckpoint = shard.getGlobalCheckpoint(); + final long globalCheckpoint = shard.getLastKnownGlobalCheckpoint(); // this global checkpoint is persisted in finalizeRecovery final StepListener finalizeListener = new StepListener<>(); cancellableThreads.executeIO(() -> recoveryTarget.finalizeRecovery(globalCheckpoint, finalizeListener)); finalizeListener.whenComplete(r -> { @@ -652,7 +629,7 @@ void finalizeRecovery(final long targetLocalCheckpoint, final ActionListener shard.relocated(recoveryTarget::handoffPrimaryContext)); + cancellableThreads.execute(() -> shard.relocated(request.targetAllocationId(), recoveryTarget::handoffPrimaryContext)); /* * if the recovery process fails after disabling primary mode on the source shard, both relocation source and * target are failed (see {@link IndexShard#updateRoutingEntry}). @@ -692,7 +669,7 @@ public String toString() { '}'; } - void sendFiles(Store store, StoreFileMetaData[] files, Supplier translogOps) throws Exception { + void sendFiles(Store store, StoreFileMetaData[] files, IntSupplier translogOps) throws Exception { ArrayUtil.timSort(files, Comparator.comparingLong(StoreFileMetaData::length)); // send smallest first final LocalCheckpointTracker requestSeqIdTracker = new LocalCheckpointTracker(NO_OPS_PERFORMED, NO_OPS_PERFORMED); final AtomicReference> error = new AtomicReference<>(); @@ -709,19 +686,20 @@ void sendFiles(Store store, StoreFileMetaData[] files, Supplier translo final BytesArray content = new BytesArray(buffer, 0, bytesRead); final boolean lastChunk = position + content.length() == md.length(); final long requestSeqId = requestSeqIdTracker.generateSeqNo(); - cancellableThreads.execute(() -> requestSeqIdTracker.waitForOpsToComplete(requestSeqId - maxConcurrentFileChunks)); + cancellableThreads.execute( + () -> requestSeqIdTracker.waitForProcessedOpsToComplete(requestSeqId - maxConcurrentFileChunks)); cancellableThreads.checkForCancel(); if (error.get() != null) { break; } final long requestFilePosition = position; cancellableThreads.executeIO(() -> - recoveryTarget.writeFileChunk(md, requestFilePosition, content, lastChunk, translogOps.get(), + recoveryTarget.writeFileChunk(md, requestFilePosition, content, lastChunk, translogOps.getAsInt(), ActionListener.wrap( - r -> requestSeqIdTracker.markSeqNoAsCompleted(requestSeqId), + r -> requestSeqIdTracker.markSeqNoAsProcessed(requestSeqId), e -> { error.compareAndSet(null, Tuple.tuple(md, e)); - requestSeqIdTracker.markSeqNoAsCompleted(requestSeqId); + requestSeqIdTracker.markSeqNoAsProcessed(requestSeqId); } ))); position += content.length(); @@ -734,27 +712,56 @@ void sendFiles(Store store, StoreFileMetaData[] files, Supplier translo // When we terminate exceptionally, we don't wait for the outstanding requests as we don't use their results anyway. // This allows us to end quickly and eliminate the complexity of handling requestSeqIds in case of error. if (error.get() == null) { - cancellableThreads.execute(() -> requestSeqIdTracker.waitForOpsToComplete(requestSeqIdTracker.getMaxSeqNo())); + cancellableThreads.execute(() -> requestSeqIdTracker.waitForProcessedOpsToComplete(requestSeqIdTracker.getMaxSeqNo())); } if (error.get() != null) { - handleErrorOnSendFiles(store, error.get().v1(), error.get().v2()); + handleErrorOnSendFiles(store, error.get().v2(), new StoreFileMetaData[]{error.get().v1()}); } } - private void handleErrorOnSendFiles(Store store, StoreFileMetaData md, Exception e) throws Exception { - final IOException corruptIndexException; - if ((corruptIndexException = ExceptionsHelper.unwrapCorruption(e)) != null) { - if (store.checkIntegrityNoException(md) == false) { // we are corrupted on the primary -- fail! - logger.warn("{} Corrupted file detected {} checksum mismatch", shardId, md); - failEngine(corruptIndexException); - throw corruptIndexException; + private void cleanFiles(Store store, Store.MetadataSnapshot sourceMetadata, IntSupplier translogOps, + long globalCheckpoint, ActionListener listener) { + // Send the CLEAN_FILES request, which takes all of the files that + // were transferred and renames them from their temporary file + // names to the actual file names. It also writes checksums for + // the files after they have been renamed. + // + // Once the files have been renamed, any other files that are not + // related to this recovery (out of date segments, for example) + // are deleted + cancellableThreads.execute(() -> recoveryTarget.cleanFiles(translogOps.getAsInt(), globalCheckpoint, sourceMetadata, + ActionListener.delegateResponse(listener, (l, e) -> ActionListener.completeWith(l, () -> { + StoreFileMetaData[] mds = StreamSupport.stream(sourceMetadata.spliterator(), false).toArray(StoreFileMetaData[]::new); + ArrayUtil.timSort(mds, Comparator.comparingLong(StoreFileMetaData::length)); // check small files first + handleErrorOnSendFiles(store, e, mds); + throw e; + })))); + } + + private void handleErrorOnSendFiles(Store store, Exception e, StoreFileMetaData[] mds) throws Exception { + final IOException corruptIndexException = ExceptionsHelper.unwrapCorruption(e); + if (corruptIndexException != null) { + Exception localException = null; + for (StoreFileMetaData md : mds) { + cancellableThreads.checkForCancel(); + logger.debug("checking integrity for file {} after remove corruption exception", md); + if (store.checkIntegrityNoException(md) == false) { // we are corrupted on the primary -- fail! + logger.warn("{} Corrupted file detected {} checksum mismatch", shardId, md); + if (localException == null) { + localException = corruptIndexException; + } + failEngine(corruptIndexException); + } + } + if (localException != null) { + throw localException; } else { // corruption has happened on the way to replica - RemoteTransportException exception = new RemoteTransportException( + RemoteTransportException remoteException = new RemoteTransportException( "File corruption occurred on recovery but checksums are ok", null); - exception.addSuppressed(e); + remoteException.addSuppressed(e); logger.warn(() -> new ParameterizedMessage("{} Remote file corruption on node {}, recovering {}. local checksum OK", - shardId, request.targetNode(), md), corruptIndexException); - throw exception; + shardId, request.targetNode(), mds), corruptIndexException); + throw remoteException; } } else { throw e; diff --git a/server/src/main/java/org/elasticsearch/indices/recovery/RecoveryState.java b/server/src/main/java/org/elasticsearch/indices/recovery/RecoveryState.java index 18018fc7db054..d29456ec42bed 100644 --- a/server/src/main/java/org/elasticsearch/indices/recovery/RecoveryState.java +++ b/server/src/main/java/org/elasticsearch/indices/recovery/RecoveryState.java @@ -927,7 +927,7 @@ public synchronized XContentBuilder toXContent(XContentBuilder builder, Params p builder.field(Fields.REUSED, reusedFileCount()); builder.field(Fields.RECOVERED, recoveredFileCount()); builder.field(Fields.PERCENT, String.format(Locale.ROOT, "%1.1f%%", recoveredFilesPercent())); - if (params.paramAsBoolean("details", false)) { + if (params.paramAsBoolean("detailed", false)) { builder.startArray(Fields.DETAILS); for (File file : fileDetails.values()) { file.toXContent(builder, params); @@ -954,7 +954,7 @@ public synchronized String toString() { } } - public File getFileDetails(String dest) { + public synchronized File getFileDetails(String dest) { return fileDetails.get(dest); } } diff --git a/server/src/main/java/org/elasticsearch/indices/recovery/RecoveryTarget.java b/server/src/main/java/org/elasticsearch/indices/recovery/RecoveryTarget.java index b3c6d12ab96e3..0b3f95b2ec4a2 100644 --- a/server/src/main/java/org/elasticsearch/indices/recovery/RecoveryTarget.java +++ b/server/src/main/java/org/elasticsearch/indices/recovery/RecoveryTarget.java @@ -377,68 +377,75 @@ public void receiveFileInfo(List phase1FileNames, List phase1FileSizes, List phase1ExistingFileNames, List phase1ExistingFileSizes, - int totalTranslogOps) { - final RecoveryState.Index index = state().getIndex(); - for (int i = 0; i < phase1ExistingFileNames.size(); i++) { - index.addFileDetail(phase1ExistingFileNames.get(i), phase1ExistingFileSizes.get(i), true); - } - for (int i = 0; i < phase1FileNames.size(); i++) { - index.addFileDetail(phase1FileNames.get(i), phase1FileSizes.get(i), false); - } - state().getTranslog().totalOperations(totalTranslogOps); - state().getTranslog().totalOperationsOnStart(totalTranslogOps); - + int totalTranslogOps, + ActionListener listener) { + ActionListener.completeWith(listener, () -> { + final RecoveryState.Index index = state().getIndex(); + for (int i = 0; i < phase1ExistingFileNames.size(); i++) { + index.addFileDetail(phase1ExistingFileNames.get(i), phase1ExistingFileSizes.get(i), true); + } + for (int i = 0; i < phase1FileNames.size(); i++) { + index.addFileDetail(phase1FileNames.get(i), phase1FileSizes.get(i), false); + } + state().getTranslog().totalOperations(totalTranslogOps); + state().getTranslog().totalOperationsOnStart(totalTranslogOps); + return null; + }); } @Override - public void cleanFiles(int totalTranslogOps, long globalCheckpoint, Store.MetadataSnapshot sourceMetaData) throws IOException { - state().getTranslog().totalOperations(totalTranslogOps); - // first, we go and move files that were created with the recovery id suffix to - // the actual names, its ok if we have a corrupted index here, since we have replicas - // to recover from in case of a full cluster shutdown just when this code executes... - multiFileWriter.renameAllTempFiles(); - final Store store = store(); - store.incRef(); - try { - store.cleanupAndVerify("recovery CleanFilesRequestHandler", sourceMetaData); - final String translogUUID = Translog.createEmptyTranslog( - indexShard.shardPath().resolveTranslog(), globalCheckpoint, shardId, indexShard.getPendingPrimaryTerm()); - store.associateIndexWithNewTranslog(translogUUID); - - if (indexShard.getRetentionLeases().leases().isEmpty()) { - // if empty, may be a fresh IndexShard, so write an empty leases file to disk - indexShard.persistRetentionLeases(); - assert indexShard.loadRetentionLeases().leases().isEmpty(); - } else { - assert indexShard.assertRetentionLeasesPersisted(); - } - - } catch (CorruptIndexException | IndexFormatTooNewException | IndexFormatTooOldException ex) { - // this is a fatal exception at this stage. - // this means we transferred files from the remote that have not be checksummed and they are - // broken. We have to clean up this shard entirely, remove all files and bubble it up to the - // source shard since this index might be broken there as well? The Source can handle this and checks - // its content on disk if possible. + public void cleanFiles(int totalTranslogOps, long globalCheckpoint, Store.MetadataSnapshot sourceMetaData, + ActionListener listener) { + ActionListener.completeWith(listener, () -> { + state().getTranslog().totalOperations(totalTranslogOps); + // first, we go and move files that were created with the recovery id suffix to + // the actual names, its ok if we have a corrupted index here, since we have replicas + // to recover from in case of a full cluster shutdown just when this code executes... + multiFileWriter.renameAllTempFiles(); + final Store store = store(); + store.incRef(); try { + store.cleanupAndVerify("recovery CleanFilesRequestHandler", sourceMetaData); + final String translogUUID = Translog.createEmptyTranslog( + indexShard.shardPath().resolveTranslog(), globalCheckpoint, shardId, indexShard.getPendingPrimaryTerm()); + store.associateIndexWithNewTranslog(translogUUID); + + if (indexShard.getRetentionLeases().leases().isEmpty()) { + // if empty, may be a fresh IndexShard, so write an empty leases file to disk + indexShard.persistRetentionLeases(); + assert indexShard.loadRetentionLeases().leases().isEmpty(); + } else { + assert indexShard.assertRetentionLeasesPersisted(); + } + + } catch (CorruptIndexException | IndexFormatTooNewException | IndexFormatTooOldException ex) { + // this is a fatal exception at this stage. + // this means we transferred files from the remote that have not be checksummed and they are + // broken. We have to clean up this shard entirely, remove all files and bubble it up to the + // source shard since this index might be broken there as well? The Source can handle this and checks + // its content on disk if possible. try { - store.removeCorruptionMarker(); - } finally { - Lucene.cleanLuceneIndex(store.directory()); // clean up and delete all files + try { + store.removeCorruptionMarker(); + } finally { + Lucene.cleanLuceneIndex(store.directory()); // clean up and delete all files + } + } catch (Exception e) { + logger.debug("Failed to clean lucene index", e); + ex.addSuppressed(e); } - } catch (Exception e) { - logger.debug("Failed to clean lucene index", e); - ex.addSuppressed(e); + RecoveryFailedException rfe = new RecoveryFailedException(state(), "failed to clean after recovery", ex); + fail(rfe, true); + throw rfe; + } catch (Exception ex) { + RecoveryFailedException rfe = new RecoveryFailedException(state(), "failed to clean after recovery", ex); + fail(rfe, true); + throw rfe; + } finally { + store.decRef(); } - RecoveryFailedException rfe = new RecoveryFailedException(state(), "failed to clean after recovery", ex); - fail(rfe, true); - throw rfe; - } catch (Exception ex) { - RecoveryFailedException rfe = new RecoveryFailedException(state(), "failed to clean after recovery", ex); - fail(rfe, true); - throw rfe; - } finally { - store.decRef(); - } + return null; + }); } @Override diff --git a/server/src/main/java/org/elasticsearch/indices/recovery/RecoveryTargetHandler.java b/server/src/main/java/org/elasticsearch/indices/recovery/RecoveryTargetHandler.java index d03fe42d90146..9be076bc73215 100644 --- a/server/src/main/java/org/elasticsearch/indices/recovery/RecoveryTargetHandler.java +++ b/server/src/main/java/org/elasticsearch/indices/recovery/RecoveryTargetHandler.java @@ -26,7 +26,6 @@ import org.elasticsearch.index.store.StoreFileMetaData; import org.elasticsearch.index.translog.Translog; -import java.io.IOException; import java.util.List; public interface RecoveryTargetHandler { @@ -89,7 +88,8 @@ void receiveFileInfo(List phase1FileNames, List phase1FileSizes, List phase1ExistingFileNames, List phase1ExistingFileSizes, - int totalTranslogOps); + int totalTranslogOps, + ActionListener listener); /** * After all source files has been sent over, this command is sent to the target so it can clean any local @@ -99,7 +99,7 @@ void receiveFileInfo(List phase1FileNames, * @param globalCheckpoint the global checkpoint on the primary * @param sourceMetaData meta data of the source store */ - void cleanFiles(int totalTranslogOps, long globalCheckpoint, Store.MetadataSnapshot sourceMetaData) throws IOException; + void cleanFiles(int totalTranslogOps, long globalCheckpoint, Store.MetadataSnapshot sourceMetaData, ActionListener listener); /** writes a partial file chunk to the target store */ void writeFileChunk(StoreFileMetaData fileMetaData, long position, BytesReference content, diff --git a/server/src/main/java/org/elasticsearch/indices/recovery/RemoteRecoveryTargetHandler.java b/server/src/main/java/org/elasticsearch/indices/recovery/RemoteRecoveryTargetHandler.java index 230f53515754e..bb5457c1a3dca 100644 --- a/server/src/main/java/org/elasticsearch/indices/recovery/RemoteRecoveryTargetHandler.java +++ b/server/src/main/java/org/elasticsearch/indices/recovery/RemoteRecoveryTargetHandler.java @@ -70,7 +70,6 @@ public RemoteRecoveryTargetHandler(long recoveryId, ShardId shardId, TransportSe .withTimeout(recoverySettings.internalActionLongTimeout()) .build(); this.fileChunkRequestOptions = TransportRequestOptions.builder() - // we are saving the cpu for other things .withType(TransportRequestOptions.Type.RECOVERY) .withTimeout(recoverySettings.internalActionTimeout()) .build(); @@ -130,22 +129,23 @@ public void indexTranslogOperations( @Override public void receiveFileInfo(List phase1FileNames, List phase1FileSizes, List phase1ExistingFileNames, - List phase1ExistingFileSizes, int totalTranslogOps) { - + List phase1ExistingFileSizes, int totalTranslogOps, ActionListener listener) { RecoveryFilesInfoRequest recoveryInfoFilesRequest = new RecoveryFilesInfoRequest(recoveryId, shardId, - phase1FileNames, phase1FileSizes, phase1ExistingFileNames, phase1ExistingFileSizes, totalTranslogOps); + phase1FileNames, phase1FileSizes, phase1ExistingFileNames, phase1ExistingFileSizes, totalTranslogOps); transportService.submitRequest(targetNode, PeerRecoveryTargetService.Actions.FILES_INFO, recoveryInfoFilesRequest, - TransportRequestOptions.builder().withTimeout(recoverySettings.internalActionTimeout()).build(), - EmptyTransportResponseHandler.INSTANCE_SAME).txGet(); - + TransportRequestOptions.builder().withTimeout(recoverySettings.internalActionTimeout()).build(), + new ActionListenerResponseHandler<>(ActionListener.map(listener, r -> null), + in -> TransportResponse.Empty.INSTANCE, ThreadPool.Names.GENERIC)); } @Override - public void cleanFiles(int totalTranslogOps, long globalCheckpoint, Store.MetadataSnapshot sourceMetaData) throws IOException { + public void cleanFiles(int totalTranslogOps, long globalCheckpoint, Store.MetadataSnapshot sourceMetaData, + ActionListener listener) { transportService.submitRequest(targetNode, PeerRecoveryTargetService.Actions.CLEAN_FILES, new RecoveryCleanFilesRequest(recoveryId, shardId, sourceMetaData, totalTranslogOps, globalCheckpoint), TransportRequestOptions.builder().withTimeout(recoverySettings.internalActionTimeout()).build(), - EmptyTransportResponseHandler.INSTANCE_SAME).txGet(); + new ActionListenerResponseHandler<>(ActionListener.map(listener, r -> null), + in -> TransportResponse.Empty.INSTANCE, ThreadPool.Names.GENERIC)); } @Override diff --git a/server/src/main/java/org/elasticsearch/indices/store/TransportNodesListShardStoreMetaData.java b/server/src/main/java/org/elasticsearch/indices/store/TransportNodesListShardStoreMetaData.java index 20307af32f4ed..20633d24ba7eb 100644 --- a/server/src/main/java/org/elasticsearch/indices/store/TransportNodesListShardStoreMetaData.java +++ b/server/src/main/java/org/elasticsearch/indices/store/TransportNodesListShardStoreMetaData.java @@ -50,6 +50,7 @@ import org.elasticsearch.index.store.Store; import org.elasticsearch.index.store.StoreFileMetaData; import org.elasticsearch.indices.IndicesService; +import org.elasticsearch.tasks.Task; import org.elasticsearch.threadpool.ThreadPool; import org.elasticsearch.transport.TransportService; @@ -91,8 +92,8 @@ public void list(ShardId shardId, DiscoveryNode[] nodes, ActionListener pipelines = new HashMap<>(); + private volatile Map pipelines = Map.of(); private final ThreadPool threadPool; private final IngestMetric totalMetrics = new IngestMetric(); @@ -236,7 +236,12 @@ public ClusterState execute(ClusterState currentState) { * Returns the pipeline by the specified id */ public Pipeline getPipeline(String id) { - return pipelines.get(id); + PipelineHolder holder = pipelines.get(id); + if (holder != null) { + return holder.pipeline; + } else { + return null; + } } public Map getProcessorFactories() { @@ -252,52 +257,10 @@ public IngestInfo info() { return new IngestInfo(processorInfoList); } - Map pipelines() { + Map pipelines() { return pipelines; } - @Override - public void applyClusterState(final ClusterChangedEvent event) { - ClusterState state = event.state(); - Map originalPipelines = pipelines; - try { - innerUpdatePipelines(event.previousState(), state); - } catch (ElasticsearchParseException e) { - logger.warn("failed to update ingest pipelines", e); - } - //pipelines changed, so add the old metrics to the new metrics - if (originalPipelines != pipelines) { - pipelines.forEach((id, pipeline) -> { - Pipeline originalPipeline = originalPipelines.get(id); - if (originalPipeline != null) { - pipeline.getMetrics().add(originalPipeline.getMetrics()); - List> oldPerProcessMetrics = new ArrayList<>(); - List> newPerProcessMetrics = new ArrayList<>(); - getProcessorMetrics(originalPipeline.getCompoundProcessor(), oldPerProcessMetrics); - getProcessorMetrics(pipeline.getCompoundProcessor(), newPerProcessMetrics); - //Best attempt to populate new processor metrics using a parallel array of the old metrics. This is not ideal since - //the per processor metrics may get reset when the arrays don't match. However, to get to an ideal model, unique and - //consistent id's per processor and/or semantic equals for each processor will be needed. - if (newPerProcessMetrics.size() == oldPerProcessMetrics.size()) { - Iterator> oldMetricsIterator = oldPerProcessMetrics.iterator(); - for (Tuple compositeMetric : newPerProcessMetrics) { - String type = compositeMetric.v1().getType(); - IngestMetric metric = compositeMetric.v2(); - if (oldMetricsIterator.hasNext()) { - Tuple oldCompositeMetric = oldMetricsIterator.next(); - String oldType = oldCompositeMetric.v1().getType(); - IngestMetric oldMetric = oldCompositeMetric.v2(); - if (type.equals(oldType)) { - metric.add(oldMetric); - } - } - } - } - } - }); - } - } - /** * Recursive method to obtain all of the non-failure processors for given compoundProcessor. Since conditionals are implemented as * wrappers to the actual processor, always prefer the actual processor's metric over the conditional processor's metric. @@ -324,25 +287,6 @@ private static List> getProcessorMetrics(Compound return processorMetrics; } - private static Pipeline substitutePipeline(String id, ElasticsearchParseException e) { - String tag = e.getHeaderKeys().contains("processor_tag") ? e.getHeader("processor_tag").get(0) : null; - String type = e.getHeaderKeys().contains("processor_type") ? e.getHeader("processor_type").get(0) : "unknown"; - String errorMessage = "pipeline with id [" + id + "] could not be loaded, caused by [" + e.getDetailedMessage() + "]"; - Processor failureProcessor = new AbstractProcessor(tag) { - @Override - public IngestDocument execute(IngestDocument ingestDocument) { - throw new IllegalStateException(errorMessage); - } - - @Override - public String getType() { - return type; - } - }; - String description = "this is a place holder pipeline, because pipeline with id [" + id + "] could not be loaded"; - return new Pipeline(id, description, null, new CompoundProcessor(failureProcessor)); - } - static ClusterState innerPut(PutPipelineRequest request, ClusterState currentState) { IngestMetadata currentIngestMetadata = currentState.metaData().custom(IngestMetadata.TYPE); Map pipelines; @@ -403,10 +347,11 @@ protected void doRun() { String pipelineId = indexRequest.getPipeline(); if (NOOP_PIPELINE_NAME.equals(pipelineId) == false) { try { - Pipeline pipeline = pipelines.get(pipelineId); - if (pipeline == null) { + PipelineHolder holder = pipelines.get(pipelineId); + if (holder == null) { throw new IllegalArgumentException("pipeline with id [" + pipelineId + "] does not exist"); } + Pipeline pipeline = holder.pipeline; innerExecute(indexRequest, pipeline, itemDroppedHandler); //this shouldn't be needed here but we do it for consistency with index api // which requires it to prevent double execution @@ -424,7 +369,8 @@ protected void doRun() { public IngestStats stats() { IngestStats.Builder statsBuilder = new IngestStats.Builder(); statsBuilder.addTotalMetrics(totalMetrics); - pipelines.forEach((id, pipeline) -> { + pipelines.forEach((id, holder) -> { + Pipeline pipeline = holder.pipeline; CompoundProcessor rootProcessor = pipeline.getCompoundProcessor(); statsBuilder.addPipelineMetrics(id, pipeline.getMetrics()); List> processorMetrics = new ArrayList<>(); @@ -503,37 +449,146 @@ private void innerExecute(IndexRequest indexRequest, Pipeline pipeline, Consumer } } - private void innerUpdatePipelines(ClusterState previousState, ClusterState state) { + @Override + public void applyClusterState(final ClusterChangedEvent event) { + ClusterState state = event.state(); if (state.blocks().hasGlobalBlock(GatewayService.STATE_NOT_RECOVERED_BLOCK)) { return; } - IngestMetadata ingestMetadata = state.getMetaData().custom(IngestMetadata.TYPE); - IngestMetadata previousIngestMetadata = previousState.getMetaData().custom(IngestMetadata.TYPE); - if (Objects.equals(ingestMetadata, previousIngestMetadata)) { + IngestMetadata newIngestMetadata = state.getMetaData().custom(IngestMetadata.TYPE); + if (newIngestMetadata == null) { return; } - Map pipelines = new HashMap<>(); - List exceptions = new ArrayList<>(); - for (PipelineConfiguration pipeline : ingestMetadata.getPipelines().values()) { + try { + innerUpdatePipelines(newIngestMetadata); + } catch (ElasticsearchParseException e) { + logger.warn("failed to update ingest pipelines", e); + } + } + + void innerUpdatePipelines(IngestMetadata newIngestMetadata) { + Map existingPipelines = this.pipelines; + + // Lazy initialize these variables in order to favour the most like scenario that there are no pipeline changes: + Map newPipelines = null; + List exceptions = null; + // Iterate over pipeline configurations in ingest metadata and constructs a new pipeline if there is no pipeline + // or the pipeline configuration has been modified + for (PipelineConfiguration newConfiguration : newIngestMetadata.getPipelines().values()) { + PipelineHolder previous = existingPipelines.get(newConfiguration.getId()); + if (previous != null && previous.configuration.equals(newConfiguration)) { + continue; + } + + if (newPipelines == null) { + newPipelines = new HashMap<>(existingPipelines); + } try { - pipelines.put( - pipeline.getId(), - Pipeline.create(pipeline.getId(), pipeline.getConfigAsMap(), processorFactories, scriptService) + Pipeline newPipeline = + Pipeline.create(newConfiguration.getId(), newConfiguration.getConfigAsMap(), processorFactories, scriptService); + newPipelines.put( + newConfiguration.getId(), + new PipelineHolder(newConfiguration, newPipeline) ); + + if (previous == null) { + continue; + } + Pipeline oldPipeline = previous.pipeline; + newPipeline.getMetrics().add(oldPipeline.getMetrics()); + List> oldPerProcessMetrics = new ArrayList<>(); + List> newPerProcessMetrics = new ArrayList<>(); + getProcessorMetrics(oldPipeline.getCompoundProcessor(), oldPerProcessMetrics); + getProcessorMetrics(newPipeline.getCompoundProcessor(), newPerProcessMetrics); + //Best attempt to populate new processor metrics using a parallel array of the old metrics. This is not ideal since + //the per processor metrics may get reset when the arrays don't match. However, to get to an ideal model, unique and + //consistent id's per processor and/or semantic equals for each processor will be needed. + if (newPerProcessMetrics.size() == oldPerProcessMetrics.size()) { + Iterator> oldMetricsIterator = oldPerProcessMetrics.iterator(); + for (Tuple compositeMetric : newPerProcessMetrics) { + String type = compositeMetric.v1().getType(); + IngestMetric metric = compositeMetric.v2(); + if (oldMetricsIterator.hasNext()) { + Tuple oldCompositeMetric = oldMetricsIterator.next(); + String oldType = oldCompositeMetric.v1().getType(); + IngestMetric oldMetric = oldCompositeMetric.v2(); + if (type.equals(oldType)) { + metric.add(oldMetric); + } + } + } + } } catch (ElasticsearchParseException e) { - pipelines.put(pipeline.getId(), substitutePipeline(pipeline.getId(), e)); + Pipeline pipeline = substitutePipeline(newConfiguration.getId(), e); + newPipelines.put(newConfiguration.getId(), new PipelineHolder(newConfiguration, pipeline)); + if (exceptions == null) { + exceptions = new ArrayList<>(); + } exceptions.add(e); } catch (Exception e) { ElasticsearchParseException parseException = new ElasticsearchParseException( - "Error updating pipeline with id [" + pipeline.getId() + "]", e); - pipelines.put(pipeline.getId(), substitutePipeline(pipeline.getId(), parseException)); + "Error updating pipeline with id [" + newConfiguration.getId() + "]", e); + Pipeline pipeline = substitutePipeline(newConfiguration.getId(), parseException); + newPipelines.put(newConfiguration.getId(), new PipelineHolder(newConfiguration, pipeline)); + if (exceptions == null) { + exceptions = new ArrayList<>(); + } exceptions.add(parseException); } } - this.pipelines = Collections.unmodifiableMap(pipelines); - ExceptionsHelper.rethrowAndSuppress(exceptions); + + // Iterate over the current active pipelines and check whether they are missing in the pipeline configuration and + // if so delete the pipeline from new Pipelines map: + for (Map.Entry entry : existingPipelines.entrySet()) { + if (newIngestMetadata.getPipelines().get(entry.getKey()) == null) { + if (newPipelines == null) { + newPipelines = new HashMap<>(existingPipelines); + } + newPipelines.remove(entry.getKey()); + } + } + + if (newPipelines != null) { + // Update the pipelines: + this.pipelines = Map.copyOf(newPipelines); + + // Rethrow errors that may have occurred during creating new pipeline instances: + if (exceptions != null) { + ExceptionsHelper.rethrowAndSuppress(exceptions); + } + } + } + + private static Pipeline substitutePipeline(String id, ElasticsearchParseException e) { + String tag = e.getHeaderKeys().contains("processor_tag") ? e.getHeader("processor_tag").get(0) : null; + String type = e.getHeaderKeys().contains("processor_type") ? e.getHeader("processor_type").get(0) : "unknown"; + String errorMessage = "pipeline with id [" + id + "] could not be loaded, caused by [" + e.getDetailedMessage() + "]"; + Processor failureProcessor = new AbstractProcessor(tag) { + @Override + public IngestDocument execute(IngestDocument ingestDocument) { + throw new IllegalStateException(errorMessage); + } + + @Override + public String getType() { + return type; + } + }; + String description = "this is a place holder pipeline, because pipeline with id [" + id + "] could not be loaded"; + return new Pipeline(id, description, null, new CompoundProcessor(failureProcessor)); + } + + static class PipelineHolder { + + final PipelineConfiguration configuration; + final Pipeline pipeline; + + PipelineHolder(PipelineConfiguration configuration, Pipeline pipeline) { + this.configuration = Objects.requireNonNull(configuration); + this.pipeline = Objects.requireNonNull(pipeline); + } } } diff --git a/server/src/main/java/org/elasticsearch/monitor/os/OsProbe.java b/server/src/main/java/org/elasticsearch/monitor/os/OsProbe.java index 18173dd275a46..320bc15fda1f4 100644 --- a/server/src/main/java/org/elasticsearch/monitor/os/OsProbe.java +++ b/server/src/main/java/org/elasticsearch/monitor/os/OsProbe.java @@ -42,6 +42,24 @@ import java.util.regex.Pattern; import java.util.stream.Collectors; +/** + * The {@link OsProbe} class retrieves information about the physical and swap size of the machine + * memory, as well as the system load average and cpu load. + * + * In some exceptional cases, it's possible the underlying native method used by + * {@link #getFreePhysicalMemorySize()} and {@link #getTotalPhysicalMemorySize()} can return a + * negative value. Because of this, we prevent those methods from returning negative values, + * returning 0 instead. + * + * The OS can report a negative number in a number of cases: + * - Non-supported OSes (HP-UX, or AIX) + * - A failure of macOS to initialize host statistics + * - An OS that does not support the {@code _SC_PHYS_PAGES} or {@code _SC_PAGE_SIZE} flags for the {@code sysconf()} linux kernel call + * - An overflow of the product of {@code _SC_PHYS_PAGES} and {@code _SC_PAGE_SIZE} + * - An error case retrieving these values from a linux kernel + * - A non-standard libc implementation not implementing the required values + * For a more exhaustive explanation, see https://github.com/elastic/elasticsearch/pull/42725 + */ public class OsProbe { private static final OperatingSystemMXBean osMxBean = ManagementFactory.getOperatingSystemMXBean(); @@ -67,12 +85,19 @@ public class OsProbe { */ public long getFreePhysicalMemorySize() { if (getFreePhysicalMemorySize == null) { - return -1; + logger.warn("getFreePhysicalMemorySize is not available"); + return 0; } try { - return (long) getFreePhysicalMemorySize.invoke(osMxBean); + final long freeMem = (long) getFreePhysicalMemorySize.invoke(osMxBean); + if (freeMem < 0) { + logger.warn("OS reported a negative free memory value [{}]", freeMem); + return 0; + } + return freeMem; } catch (Exception e) { - return -1; + logger.warn("exception retrieving free physical memory", e); + return 0; } } @@ -81,12 +106,19 @@ public long getFreePhysicalMemorySize() { */ public long getTotalPhysicalMemorySize() { if (getTotalPhysicalMemorySize == null) { - return -1; + logger.warn("getTotalPhysicalMemorySize is not available"); + return 0; } try { - return (long) getTotalPhysicalMemorySize.invoke(osMxBean); + final long totalMem = (long) getTotalPhysicalMemorySize.invoke(osMxBean); + if (totalMem < 0) { + logger.warn("OS reported a negative total memory value [{}]", totalMem); + return 0; + } + return totalMem; } catch (Exception e) { - return -1; + logger.warn("exception retrieving total physical memory", e); + return 0; } } diff --git a/server/src/main/java/org/elasticsearch/monitor/os/OsStats.java b/server/src/main/java/org/elasticsearch/monitor/os/OsStats.java index 86047281a22fb..9dbd9e4365a53 100644 --- a/server/src/main/java/org/elasticsearch/monitor/os/OsStats.java +++ b/server/src/main/java/org/elasticsearch/monitor/os/OsStats.java @@ -228,13 +228,17 @@ public static class Mem implements Writeable, ToXContentFragment { private final long free; public Mem(long total, long free) { + assert total >= 0 : "expected total memory to be positive, got: " + total; + assert free >= 0 : "expected free memory to be positive, got: " + total; this.total = total; this.free = free; } public Mem(StreamInput in) throws IOException { this.total = in.readLong(); + assert total >= 0 : "expected total memory to be positive, got: " + total; this.free = in.readLong(); + assert free >= 0 : "expected free memory to be positive, got: " + total; } @Override diff --git a/server/src/main/java/org/elasticsearch/node/InternalSettingsPreparer.java b/server/src/main/java/org/elasticsearch/node/InternalSettingsPreparer.java index b73268d9985b5..2aee13a5b4a40 100644 --- a/server/src/main/java/org/elasticsearch/node/InternalSettingsPreparer.java +++ b/server/src/main/java/org/elasticsearch/node/InternalSettingsPreparer.java @@ -36,10 +36,7 @@ public class InternalSettingsPreparer { - /** - * Prepares settings for the transport client by gathering all - * elasticsearch system properties and setting defaults. - */ + // TODO: refactor this method out, it used to exist for the transport client public static Settings prepareSettings(Settings input) { Settings.Builder output = Settings.builder(); initializeSettings(output, input, Collections.emptyMap()); diff --git a/server/src/main/java/org/elasticsearch/node/Node.java b/server/src/main/java/org/elasticsearch/node/Node.java index 6e592ba324fb4..53990ad0c4c1e 100644 --- a/server/src/main/java/org/elasticsearch/node/Node.java +++ b/server/src/main/java/org/elasticsearch/node/Node.java @@ -26,7 +26,7 @@ import org.elasticsearch.Build; import org.elasticsearch.ElasticsearchException; import org.elasticsearch.ElasticsearchTimeoutException; -import org.elasticsearch.action.Action; +import org.elasticsearch.action.ActionType; import org.elasticsearch.action.ActionModule; import org.elasticsearch.action.admin.cluster.snapshots.status.TransportNodesSnapshotsStatus; import org.elasticsearch.action.search.SearchExecutionStatsCollector; @@ -55,7 +55,10 @@ import org.elasticsearch.cluster.metadata.MetaDataIndexUpgradeService; import org.elasticsearch.cluster.metadata.TemplateUpgradeService; import org.elasticsearch.cluster.node.DiscoveryNode; -import org.elasticsearch.cluster.routing.RoutingService; +import org.elasticsearch.cluster.node.DiscoveryNodeRole; +import org.elasticsearch.cluster.routing.BatchedRerouteService; +import org.elasticsearch.cluster.routing.LazilyInitializedRerouteService; +import org.elasticsearch.cluster.routing.RerouteService; import org.elasticsearch.cluster.routing.allocation.DiskThresholdMonitor; import org.elasticsearch.cluster.service.ClusterService; import org.elasticsearch.common.StopWatch; @@ -64,7 +67,6 @@ import org.elasticsearch.common.component.LifecycleComponent; import org.elasticsearch.common.inject.Injector; import org.elasticsearch.common.inject.Key; -import org.elasticsearch.common.inject.Module; import org.elasticsearch.common.inject.ModulesBuilder; import org.elasticsearch.common.io.stream.NamedWriteableRegistry; import org.elasticsearch.common.lease.Releasables; @@ -74,6 +76,7 @@ import org.elasticsearch.common.network.NetworkModule; import org.elasticsearch.common.network.NetworkService; import org.elasticsearch.common.settings.ClusterSettings; +import org.elasticsearch.common.settings.ConsistentSettingsService; import org.elasticsearch.common.settings.Setting; import org.elasticsearch.common.settings.Setting.Property; import org.elasticsearch.common.settings.SettingUpgrader; @@ -311,6 +314,14 @@ protected Node( this.pluginsService = new PluginsService(tmpSettings, environment.configFile(), environment.modulesFile(), environment.pluginsFile(), classpathPlugins); final Settings settings = pluginsService.updatedSettings(); + final Set possibleRoles = Stream.concat( + DiscoveryNodeRole.BUILT_IN_ROLES.stream(), + pluginsService.filterPlugins(Plugin.class) + .stream() + .map(Plugin::getRoles) + .flatMap(Set::stream)) + .collect(Collectors.toSet()); + DiscoveryNode.setPossibleRoles(possibleRoles); localNodeFactory = new LocalNodeFactory(settings, nodeEnvironment.nodeId()); // create the environment based on the finalized (processed) view of the settings @@ -355,26 +366,26 @@ protected Node( final ClusterService clusterService = new ClusterService(settings, settingsModule.getClusterSettings(), threadPool); clusterService.addStateApplier(scriptModule.getScriptService()); resourcesToClose.add(clusterService); + clusterService.addLocalNodeMasterListener( + new ConsistentSettingsService(settings, clusterService, settingsModule.getConsistentSettings()) + .newHashPublisher()); final IngestService ingestService = new IngestService(clusterService, threadPool, this.environment, scriptModule.getScriptService(), analysisModule.getAnalysisRegistry(), pluginsService.filterPlugins(IngestPlugin.class)); - final DiskThresholdMonitor listener = new DiskThresholdMonitor(settings, clusterService::state, - clusterService.getClusterSettings(), client); + final LazilyInitializedRerouteService lazilyInitializedRerouteService = new LazilyInitializedRerouteService(); + final DiskThresholdMonitor diskThresholdMonitor = new DiskThresholdMonitor(settings, clusterService::state, + clusterService.getClusterSettings(), client, threadPool::relativeTimeInMillis, lazilyInitializedRerouteService); final ClusterInfoService clusterInfoService = newClusterInfoService(settings, clusterService, threadPool, client, - listener::onNewInfo); + diskThresholdMonitor::onNewInfo); final UsageService usageService = new UsageService(); ModulesBuilder modules = new ModulesBuilder(); - // plugin modules must be added here, before others or we can get crazy injection errors... - for (Module pluginModule : pluginsService.createGuiceModules()) { - modules.add(pluginModule); - } final MonitorService monitorService = new MonitorService(settings, nodeEnvironment, threadPool, clusterInfoService); ClusterModule clusterModule = new ClusterModule(settings, clusterService, clusterPlugins, clusterInfoService); modules.add(clusterModule); IndicesModule indicesModule = new IndicesModule(pluginsService.filterPlugins(MapperPlugin.class)); modules.add(indicesModule); - SearchModule searchModule = new SearchModule(settings, false, pluginsService.filterPlugins(SearchPlugin.class)); + SearchModule searchModule = new SearchModule(settings, pluginsService.filterPlugins(SearchPlugin.class)); CircuitBreakerService circuitBreakerService = createCircuitBreakerService(settingsModule.getSettings(), settingsModule.getClusterSettings()); resourcesToClose.add(circuitBreakerService); @@ -445,13 +456,13 @@ protected Node( namedWriteableRegistry).stream()) .collect(Collectors.toList()); - ActionModule actionModule = new ActionModule(false, settings, clusterModule.getIndexNameExpressionResolver(), + ActionModule actionModule = new ActionModule(settings, clusterModule.getIndexNameExpressionResolver(), settingsModule.getIndexScopedSettings(), settingsModule.getClusterSettings(), settingsModule.getSettingsFilter(), threadPool, pluginsService.filterPlugins(ActionPlugin.class), client, circuitBreakerService, usageService); modules.add(actionModule); final RestController restController = actionModule.getRestController(); - final NetworkModule networkModule = new NetworkModule(settings, false, pluginsService.filterPlugins(NetworkPlugin.class), + final NetworkModule networkModule = new NetworkModule(settings, pluginsService.filterPlugins(NetworkPlugin.class), threadPool, bigArrays, pageCacheRecycler, circuitBreakerService, namedWriteableRegistry, xContentRegistry, networkService, restController); Collection>> customMetaDataUpgraders = @@ -495,10 +506,13 @@ protected Node( RestoreService restoreService = new RestoreService(clusterService, repositoryService, clusterModule.getAllocationService(), metaDataCreateIndexService, metaDataIndexUpgradeService, clusterService.getClusterSettings()); + final RerouteService rerouteService + = new BatchedRerouteService(clusterService, clusterModule.getAllocationService()::reroute); + lazilyInitializedRerouteService.setRerouteService(rerouteService); final DiscoveryModule discoveryModule = new DiscoveryModule(settings, threadPool, transportService, namedWriteableRegistry, networkService, clusterService.getMasterService(), clusterService.getClusterApplierService(), clusterService.getClusterSettings(), pluginsService.filterPlugins(DiscoveryPlugin.class), - clusterModule.getAllocationService(), environment.configFile(), gatewayMetaState); + clusterModule.getAllocationService(), environment.configFile(), gatewayMetaState, rerouteService); this.nodeService = new NodeService(settings, threadPool, monitorService, discoveryModule.getDiscovery(), transportService, indicesService, pluginsService, circuitBreakerService, scriptModule.getScriptService(), httpServerTransport, ingestService, clusterService, settingsModule.getSettingsFilter(), responseCollectorService, @@ -573,6 +587,7 @@ protected Node( b.bind(SnapshotShardsService.class).toInstance(snapshotShardsService); b.bind(TransportNodesSnapshotsStatus.class).toInstance(nodesSnapshotsStatus); b.bind(RestoreService.class).toInstance(restoreService); + b.bind(RerouteService.class).toInstance(rerouteService); } ); injector = modules.createInjector(); @@ -583,11 +598,9 @@ protected Node( List pluginLifecycleComponents = pluginComponents.stream() .filter(p -> p instanceof LifecycleComponent) .map(p -> (LifecycleComponent) p).collect(Collectors.toList()); - pluginLifecycleComponents.addAll(pluginsService.getGuiceServiceClasses().stream() - .map(injector::getInstance).collect(Collectors.toList())); resourcesToClose.addAll(pluginLifecycleComponents); this.pluginLifecycleComponents = Collections.unmodifiableList(pluginLifecycleComponents); - client.initialize(injector.getInstance(new Key>() {}), + client.initialize(injector.getInstance(new Key>() {}), () -> clusterService.localNode().getId(), transportService.getRemoteClusterService()); this.namedWriteableRegistry = namedWriteableRegistry; @@ -661,7 +674,6 @@ public Node start() throws NodeValidationException { injector.getInstance(IndicesClusterStateService.class).start(); injector.getInstance(SnapshotsService.class).start(); injector.getInstance(SnapshotShardsService.class).start(); - injector.getInstance(RoutingService.class).start(); injector.getInstance(SearchService.class).start(); nodeService.getMonitorService().start(); @@ -779,7 +791,6 @@ private Node stop() { // This can confuse other nodes and delay things - mostly if we're the master and we're running tests. injector.getInstance(Discovery.class).stop(); // we close indices first, so operations won't be allowed on it - injector.getInstance(RoutingService.class).stop(); injector.getInstance(ClusterService.class).stop(); injector.getInstance(NodeConnectionsService.class).stop(); nodeService.getMonitorService().stop(); @@ -829,8 +840,6 @@ public synchronized void close() throws IOException { toClose.add(injector.getInstance(IndicesService.class)); // close filter/fielddata caches after indices toClose.add(injector.getInstance(IndicesStore.class)); - toClose.add(() -> stopWatch.stop().start("routing")); - toClose.add(injector.getInstance(RoutingService.class)); toClose.add(() -> stopWatch.stop().start("cluster")); toClose.add(injector.getInstance(ClusterService.class)); toClose.add(() -> stopWatch.stop().start("node_connections_service")); diff --git a/server/src/main/java/org/elasticsearch/persistent/CompletionPersistentTaskAction.java b/server/src/main/java/org/elasticsearch/persistent/CompletionPersistentTaskAction.java index b09c33b59d2f5..ef216961489f9 100644 --- a/server/src/main/java/org/elasticsearch/persistent/CompletionPersistentTaskAction.java +++ b/server/src/main/java/org/elasticsearch/persistent/CompletionPersistentTaskAction.java @@ -18,9 +18,9 @@ */ package org.elasticsearch.persistent; -import org.elasticsearch.action.Action; import org.elasticsearch.action.ActionListener; import org.elasticsearch.action.ActionRequestValidationException; +import org.elasticsearch.action.StreamableResponseActionType; import org.elasticsearch.action.support.ActionFilters; import org.elasticsearch.action.support.master.MasterNodeOperationRequestBuilder; import org.elasticsearch.action.support.master.MasterNodeRequest; @@ -34,6 +34,7 @@ import org.elasticsearch.common.inject.Inject; import org.elasticsearch.common.io.stream.StreamInput; import org.elasticsearch.common.io.stream.StreamOutput; +import org.elasticsearch.tasks.Task; import org.elasticsearch.threadpool.ThreadPool; import org.elasticsearch.transport.TransportService; @@ -43,10 +44,10 @@ import static org.elasticsearch.action.ValidateActions.addValidationError; /** - * Action that is used by executor node to indicate that the persistent action finished or failed on the node and needs to be + * ActionType that is used by executor node to indicate that the persistent action finished or failed on the node and needs to be * removed from the cluster state in case of successful completion or restarted on some other node in case of failure. */ -public class CompletionPersistentTaskAction extends Action { +public class CompletionPersistentTaskAction extends StreamableResponseActionType { public static final CompletionPersistentTaskAction INSTANCE = new CompletionPersistentTaskAction(); public static final String NAME = "cluster:admin/persistent/completion"; @@ -161,7 +162,7 @@ protected ClusterBlockException checkBlock(Request request, ClusterState state) } @Override - protected final void masterOperation(final Request request, ClusterState state, + protected final void masterOperation(Task ignoredTask, final Request request, ClusterState state, final ActionListener listener) { persistentTasksClusterService.completePersistentTask(request.taskId, request.allocationId, request.exception, ActionListener.delegateFailure(listener, diff --git a/server/src/main/java/org/elasticsearch/persistent/PersistentTasksNodeService.java b/server/src/main/java/org/elasticsearch/persistent/PersistentTasksNodeService.java index 260fabc67cdca..9a9a39cd32e04 100644 --- a/server/src/main/java/org/elasticsearch/persistent/PersistentTasksNodeService.java +++ b/server/src/main/java/org/elasticsearch/persistent/PersistentTasksNodeService.java @@ -47,7 +47,7 @@ /** * This component is responsible for coordination of execution of persistent tasks on individual nodes. It runs on all - * non-transport client nodes in the cluster and monitors cluster state changes to detect started commands. + * nodes in the cluster and monitors cluster state changes to detect started commands. */ public class PersistentTasksNodeService implements ClusterStateListener { diff --git a/server/src/main/java/org/elasticsearch/persistent/PersistentTasksService.java b/server/src/main/java/org/elasticsearch/persistent/PersistentTasksService.java index 9664be255da17..f10560a807de1 100644 --- a/server/src/main/java/org/elasticsearch/persistent/PersistentTasksService.java +++ b/server/src/main/java/org/elasticsearch/persistent/PersistentTasksService.java @@ -20,7 +20,7 @@ import org.apache.logging.log4j.LogManager; import org.apache.logging.log4j.Logger; -import org.elasticsearch.action.Action; +import org.elasticsearch.action.ActionType; import org.elasticsearch.action.ActionListener; import org.elasticsearch.action.ActionRequest; import org.elasticsearch.action.admin.cluster.node.tasks.cancel.CancelTasksRequest; @@ -129,7 +129,7 @@ public void sendRemoveRequest(final String taskId, final ActionListener - void execute(final Req request, final Action action, final ActionListener> listener) { + void execute(final Req request, final ActionType action, final ActionListener> listener) { try { client.execute(action, request, ActionListener.map(listener, PersistentTaskResponse::getTask)); } catch (Exception e) { diff --git a/server/src/main/java/org/elasticsearch/persistent/RemovePersistentTaskAction.java b/server/src/main/java/org/elasticsearch/persistent/RemovePersistentTaskAction.java index 877033fe4f33a..8002be6626991 100644 --- a/server/src/main/java/org/elasticsearch/persistent/RemovePersistentTaskAction.java +++ b/server/src/main/java/org/elasticsearch/persistent/RemovePersistentTaskAction.java @@ -18,9 +18,9 @@ */ package org.elasticsearch.persistent; -import org.elasticsearch.action.Action; import org.elasticsearch.action.ActionListener; import org.elasticsearch.action.ActionRequestValidationException; +import org.elasticsearch.action.StreamableResponseActionType; import org.elasticsearch.action.support.ActionFilters; import org.elasticsearch.action.support.master.MasterNodeOperationRequestBuilder; import org.elasticsearch.action.support.master.MasterNodeRequest; @@ -34,13 +34,14 @@ import org.elasticsearch.common.inject.Inject; import org.elasticsearch.common.io.stream.StreamInput; import org.elasticsearch.common.io.stream.StreamOutput; +import org.elasticsearch.tasks.Task; import org.elasticsearch.threadpool.ThreadPool; import org.elasticsearch.transport.TransportService; import java.io.IOException; import java.util.Objects; -public class RemovePersistentTaskAction extends Action { +public class RemovePersistentTaskAction extends StreamableResponseActionType { public static final RemovePersistentTaskAction INSTANCE = new RemovePersistentTaskAction(); public static final String NAME = "cluster:admin/persistent/remove"; @@ -146,7 +147,7 @@ protected ClusterBlockException checkBlock(Request request, ClusterState state) } @Override - protected final void masterOperation(final Request request, ClusterState state, + protected final void masterOperation(Task ignoredTask, final Request request, ClusterState state, final ActionListener listener) { persistentTasksClusterService.removePersistentTask( request.taskId, ActionListener.delegateFailure(listener, diff --git a/server/src/main/java/org/elasticsearch/persistent/StartPersistentTaskAction.java b/server/src/main/java/org/elasticsearch/persistent/StartPersistentTaskAction.java index 977239ee2b990..01fa8aa5edb1b 100644 --- a/server/src/main/java/org/elasticsearch/persistent/StartPersistentTaskAction.java +++ b/server/src/main/java/org/elasticsearch/persistent/StartPersistentTaskAction.java @@ -18,9 +18,9 @@ */ package org.elasticsearch.persistent; -import org.elasticsearch.action.Action; import org.elasticsearch.action.ActionListener; import org.elasticsearch.action.ActionRequestValidationException; +import org.elasticsearch.action.StreamableResponseActionType; import org.elasticsearch.action.support.ActionFilters; import org.elasticsearch.action.support.master.MasterNodeOperationRequestBuilder; import org.elasticsearch.action.support.master.MasterNodeRequest; @@ -35,6 +35,7 @@ import org.elasticsearch.common.inject.Inject; import org.elasticsearch.common.io.stream.StreamInput; import org.elasticsearch.common.io.stream.StreamOutput; +import org.elasticsearch.tasks.Task; import org.elasticsearch.threadpool.ThreadPool; import org.elasticsearch.transport.TransportService; @@ -46,7 +47,7 @@ /** * This action can be used to add the record for the persistent action to the cluster state. */ -public class StartPersistentTaskAction extends Action { +public class StartPersistentTaskAction extends StreamableResponseActionType { public static final StartPersistentTaskAction INSTANCE = new StartPersistentTaskAction(); public static final String NAME = "cluster:admin/persistent/start"; @@ -213,7 +214,7 @@ protected ClusterBlockException checkBlock(Request request, ClusterState state) } @Override - protected final void masterOperation(final Request request, ClusterState state, + protected final void masterOperation(Task ignoredTask, final Request request, ClusterState state, final ActionListener listener) { persistentTasksClusterService.createPersistentTask(request.taskId, request.taskName, request.params, ActionListener.delegateFailure(listener, diff --git a/server/src/main/java/org/elasticsearch/persistent/UpdatePersistentTaskStatusAction.java b/server/src/main/java/org/elasticsearch/persistent/UpdatePersistentTaskStatusAction.java index 218154d37c9b7..e3514406d7080 100644 --- a/server/src/main/java/org/elasticsearch/persistent/UpdatePersistentTaskStatusAction.java +++ b/server/src/main/java/org/elasticsearch/persistent/UpdatePersistentTaskStatusAction.java @@ -18,9 +18,9 @@ */ package org.elasticsearch.persistent; -import org.elasticsearch.action.Action; import org.elasticsearch.action.ActionListener; import org.elasticsearch.action.ActionRequestValidationException; +import org.elasticsearch.action.StreamableResponseActionType; import org.elasticsearch.action.support.ActionFilters; import org.elasticsearch.action.support.master.MasterNodeOperationRequestBuilder; import org.elasticsearch.action.support.master.MasterNodeRequest; @@ -34,6 +34,7 @@ import org.elasticsearch.common.inject.Inject; import org.elasticsearch.common.io.stream.StreamInput; import org.elasticsearch.common.io.stream.StreamOutput; +import org.elasticsearch.tasks.Task; import org.elasticsearch.threadpool.ThreadPool; import org.elasticsearch.transport.TransportService; @@ -42,7 +43,7 @@ import static org.elasticsearch.action.ValidateActions.addValidationError; -public class UpdatePersistentTaskStatusAction extends Action { +public class UpdatePersistentTaskStatusAction extends StreamableResponseActionType { public static final UpdatePersistentTaskStatusAction INSTANCE = new UpdatePersistentTaskStatusAction(); public static final String NAME = "cluster:admin/persistent/update_status"; @@ -176,7 +177,7 @@ protected ClusterBlockException checkBlock(Request request, ClusterState state) } @Override - protected final void masterOperation(final Request request, + protected final void masterOperation(Task ignoredTask, final Request request, final ClusterState state, final ActionListener listener) { persistentTasksClusterService.updatePersistentTaskState(request.taskId, request.allocationId, request.state, diff --git a/server/src/main/java/org/elasticsearch/plugins/ActionPlugin.java b/server/src/main/java/org/elasticsearch/plugins/ActionPlugin.java index adc2fa8f0b282..ee6e026d5a535 100644 --- a/server/src/main/java/org/elasticsearch/plugins/ActionPlugin.java +++ b/server/src/main/java/org/elasticsearch/plugins/ActionPlugin.java @@ -19,17 +19,16 @@ package org.elasticsearch.plugins; -import org.elasticsearch.action.Action; +import org.elasticsearch.action.ActionType; import org.elasticsearch.action.ActionRequest; import org.elasticsearch.action.ActionResponse; -import org.elasticsearch.action.admin.indices.mapping.put.MappingRequestValidator; -import org.elasticsearch.action.admin.indices.mapping.put.TransportPutMappingAction; +import org.elasticsearch.action.RequestValidators; +import org.elasticsearch.action.admin.indices.alias.IndicesAliasesRequest; +import org.elasticsearch.action.admin.indices.mapping.put.PutMappingRequest; import org.elasticsearch.action.support.ActionFilter; import org.elasticsearch.action.support.TransportAction; -import org.elasticsearch.action.support.TransportActions; import org.elasticsearch.cluster.metadata.IndexNameExpressionResolver; import org.elasticsearch.cluster.node.DiscoveryNodes; -import org.elasticsearch.common.Strings; import org.elasticsearch.common.settings.ClusterSettings; import org.elasticsearch.common.settings.IndexScopedSettings; import org.elasticsearch.common.settings.Settings; @@ -67,15 +66,15 @@ public interface ActionPlugin { } /** - * Client actions added by this plugin. This defaults to all of the {@linkplain Action} in + * Client actions added by this plugin. This defaults to all of the {@linkplain ActionType} in * {@linkplain ActionPlugin#getActions()}. */ - default List> getClientActions() { + default List> getClientActions() { return getActions().stream().map(a -> a.action).collect(Collectors.toList()); } /** - * Action filters added by this plugin. + * ActionType filters added by this plugin. */ default List getActionFilters() { return Collections.emptyList(); @@ -129,22 +128,18 @@ default UnaryOperator getRestHandlerWrapper(ThreadContext threadCon } final class ActionHandler { - private final Action action; + private final ActionType action; private final Class> transportAction; - private final Class[] supportTransportActions; /** - * Create a record of an action, the {@linkplain TransportAction} that handles it, and any supporting {@linkplain TransportActions} - * that are needed by that {@linkplain TransportAction}. + * Create a record of an action, the {@linkplain TransportAction} that handles it. */ - public ActionHandler(Action action, Class> transportAction, - Class... supportTransportActions) { + public ActionHandler(ActionType action, Class> transportAction) { this.action = action; this.transportAction = transportAction; - this.supportTransportActions = supportTransportActions; } - public Action getAction() { + public ActionType getAction() { return action; } @@ -152,17 +147,9 @@ public Class> getTransportAction() return transportAction; } - public Class[] getSupportTransportActions() { - return supportTransportActions; - } - @Override public String toString() { - StringBuilder b = new StringBuilder().append(action.name()).append(" is handled by ").append(transportAction.getName()); - if (supportTransportActions.length > 0) { - b.append('[').append(Strings.arrayToCommaDelimitedString(supportTransportActions)).append(']'); - } - return b.toString(); + return action.name() + " is handled by " + transportAction.getName(); } @Override @@ -172,21 +159,25 @@ public boolean equals(Object obj) { } ActionHandler other = (ActionHandler) obj; return Objects.equals(action, other.action) - && Objects.equals(transportAction, other.transportAction) - && Objects.deepEquals(supportTransportActions, other.supportTransportActions); + && Objects.equals(transportAction, other.transportAction); } @Override public int hashCode() { - return Objects.hash(action, transportAction, supportTransportActions); + return Objects.hash(action, transportAction); } } /** - * Returns a collection of validators that are used by {@link TransportPutMappingAction.RequestValidators} to - * validate a {@link org.elasticsearch.action.admin.indices.mapping.put.PutMappingRequest} before the executing it. + * Returns a collection of validators that are used by {@link RequestValidators} to validate a + * {@link org.elasticsearch.action.admin.indices.mapping.put.PutMappingRequest} before the executing it. */ - default Collection mappingRequestValidators() { + default Collection> mappingRequestValidators() { return Collections.emptyList(); } + + default Collection> indicesAliasesRequestValidators() { + return Collections.emptyList(); + } + } diff --git a/server/src/main/java/org/elasticsearch/plugins/DiscoveryPlugin.java b/server/src/main/java/org/elasticsearch/plugins/DiscoveryPlugin.java index 21cfbd0e99e8d..4757386efc5e8 100644 --- a/server/src/main/java/org/elasticsearch/plugins/DiscoveryPlugin.java +++ b/server/src/main/java/org/elasticsearch/plugins/DiscoveryPlugin.java @@ -20,6 +20,7 @@ package org.elasticsearch.plugins; import org.elasticsearch.cluster.ClusterState; +import org.elasticsearch.cluster.coordination.ElectionStrategy; import org.elasticsearch.cluster.node.DiscoveryNode; import org.elasticsearch.common.network.NetworkService; import org.elasticsearch.common.settings.Settings; @@ -85,4 +86,11 @@ default Map> getSeedHostProviders(TransportS * {@link IllegalStateException} if the node and the cluster-state are incompatible. */ default BiConsumer getJoinValidator() { return null; } + + /** + * Allows plugging in election strategies (see {@link ElectionStrategy}) that define a customized notion of an election quorum. + */ + default Map getElectionStrategies() { + return Collections.emptyMap(); + } } diff --git a/server/src/main/java/org/elasticsearch/plugins/Plugin.java b/server/src/main/java/org/elasticsearch/plugins/Plugin.java index dcba40b58c21a..5bc8e9267a515 100644 --- a/server/src/main/java/org/elasticsearch/plugins/Plugin.java +++ b/server/src/main/java/org/elasticsearch/plugins/Plugin.java @@ -25,9 +25,9 @@ import org.elasticsearch.cluster.metadata.IndexMetaData; import org.elasticsearch.cluster.metadata.IndexTemplateMetaData; import org.elasticsearch.cluster.metadata.MetaData; +import org.elasticsearch.cluster.node.DiscoveryNodeRole; import org.elasticsearch.cluster.service.ClusterService; import org.elasticsearch.common.component.LifecycleComponent; -import org.elasticsearch.common.inject.Module; import org.elasticsearch.common.io.stream.NamedWriteable; import org.elasticsearch.common.io.stream.NamedWriteableRegistry; import org.elasticsearch.common.settings.Setting; @@ -50,6 +50,7 @@ import java.util.List; import java.util.Map; import java.util.Optional; +import java.util.Set; import java.util.function.UnaryOperator; /** @@ -82,21 +83,6 @@ protected Optional getFeature() { return Optional.empty(); } - /** - * Node level guice modules. - */ - public Collection createGuiceModules() { - return Collections.emptyList(); - } - - /** - * Node level services that will be automatically started/stopped/closed. This classes must be constructed - * by injection with guice. - */ - public Collection> getGuiceServiceClasses() { - return Collections.emptyList(); - } - /** * Returns components added by this plugin. * @@ -236,6 +222,10 @@ public List> getExecutorBuilders(Settings settings) { */ public List getBootstrapChecks() { return Collections.emptyList(); } + public Set getRoles() { + return Set.of(); + } + /** * Close the resources opened by this plugin. * diff --git a/server/src/main/java/org/elasticsearch/plugins/PluginLoaderIndirection.java b/server/src/main/java/org/elasticsearch/plugins/PluginLoaderIndirection.java index 7a07e4f08e849..f4c8f7794ea91 100644 --- a/server/src/main/java/org/elasticsearch/plugins/PluginLoaderIndirection.java +++ b/server/src/main/java/org/elasticsearch/plugins/PluginLoaderIndirection.java @@ -21,6 +21,7 @@ import java.util.List; +// TODO: remove this indirection now that transport client is gone /** * This class exists solely as an intermediate layer to avoid causing PluginsService * to load ExtendedPluginsClassLoader when used in the transport client. diff --git a/server/src/main/java/org/elasticsearch/plugins/PluginsService.java b/server/src/main/java/org/elasticsearch/plugins/PluginsService.java index 240837077e1c3..feb18c61e3365 100644 --- a/server/src/main/java/org/elasticsearch/plugins/PluginsService.java +++ b/server/src/main/java/org/elasticsearch/plugins/PluginsService.java @@ -33,8 +33,6 @@ import org.elasticsearch.bootstrap.JarHell; import org.elasticsearch.common.Strings; import org.elasticsearch.common.collect.Tuple; -import org.elasticsearch.common.component.LifecycleComponent; -import org.elasticsearch.common.inject.Module; import org.elasticsearch.common.io.FileSystemUtils; import org.elasticsearch.common.settings.Setting; import org.elasticsearch.common.settings.Setting.Property; @@ -114,7 +112,7 @@ public PluginsService( List pluginsList = new ArrayList<>(); // we need to build a List of plugins for checking mandatory plugins final List pluginsNames = new ArrayList<>(); - // first we load plugins that are on the classpath. this is for tests and transport clients + // first we load plugins that are on the classpath. this is for tests for (Class pluginClass : classpathPlugins) { Plugin plugin = loadPlugin(pluginClass, settings, configPath); PluginInfo pluginInfo = new PluginInfo(pluginClass.getName(), "classpath plugin", "NA", Version.CURRENT, "1.8", @@ -237,14 +235,6 @@ public Settings updatedSettings() { return builder.put(this.settings).build(); } - public Collection createGuiceModules() { - List modules = new ArrayList<>(); - for (Tuple plugin : plugins) { - modules.addAll(plugin.v2().createGuiceModules()); - } - return modules; - } - public List> getExecutorBuilders(Settings settings) { final ArrayList> builders = new ArrayList<>(); for (final Tuple plugin : plugins) { @@ -253,15 +243,6 @@ public List> getExecutorBuilders(Settings settings) { return builders; } - /** Returns all classes injected into guice by plugins which extend {@link LifecycleComponent}. */ - public Collection> getGuiceServiceClasses() { - List> services = new ArrayList<>(); - for (Tuple plugin : plugins) { - services.addAll(plugin.v2().getGuiceServiceClasses()); - } - return services; - } - public void onIndexModule(IndexModule indexModule) { for (Tuple plugin : plugins) { plugin.v2().onIndexModule(indexModule); diff --git a/server/src/main/java/org/elasticsearch/repositories/FilterRepository.java b/server/src/main/java/org/elasticsearch/repositories/FilterRepository.java index 1fa42579617e1..8c9eff0698835 100644 --- a/server/src/main/java/org/elasticsearch/repositories/FilterRepository.java +++ b/server/src/main/java/org/elasticsearch/repositories/FilterRepository.java @@ -38,6 +38,7 @@ import java.io.IOException; import java.util.List; +import java.util.Map; public class FilterRepository implements Repository { @@ -79,9 +80,10 @@ public void initializeSnapshot(SnapshotId snapshotId, List indices, Met @Override public SnapshotInfo finalizeSnapshot(SnapshotId snapshotId, List indices, long startTime, String failure, int totalShards, - List shardFailures, long repositoryStateId, boolean includeGlobalState) { + List shardFailures, long repositoryStateId, boolean includeGlobalState, + Map userMetadata) { return in.finalizeSnapshot(snapshotId, indices, startTime, failure, totalShards, shardFailures, repositoryStateId, - includeGlobalState); + includeGlobalState, userMetadata); } @Override diff --git a/server/src/main/java/org/elasticsearch/repositories/Repository.java b/server/src/main/java/org/elasticsearch/repositories/Repository.java index 0eca92039fbf8..788459b16c540 100644 --- a/server/src/main/java/org/elasticsearch/repositories/Repository.java +++ b/server/src/main/java/org/elasticsearch/repositories/Repository.java @@ -37,6 +37,7 @@ import java.io.IOException; import java.util.List; +import java.util.Map; import java.util.function.Function; /** @@ -134,7 +135,8 @@ default Repository create(RepositoryMetaData metaData, Function indices, long startTime, String failure, int totalShards, - List shardFailures, long repositoryStateId, boolean includeGlobalState); + List shardFailures, long repositoryStateId, boolean includeGlobalState, + Map userMetadata); /** * Deletes snapshot diff --git a/server/src/main/java/org/elasticsearch/repositories/RepositoryData.java b/server/src/main/java/org/elasticsearch/repositories/RepositoryData.java index 589e0432c032d..3d0daff98936e 100644 --- a/server/src/main/java/org/elasticsearch/repositories/RepositoryData.java +++ b/server/src/main/java/org/elasticsearch/repositories/RepositoryData.java @@ -442,8 +442,7 @@ public static RepositoryData snapshotsFromXContent(final XContentParser parser, /** * Writes the incompatible snapshot ids to x-content. */ - public void incompatibleSnapshotsToXContent(final XContentBuilder builder) throws IOException { - + public XContentBuilder incompatibleSnapshotsToXContent(XContentBuilder builder) throws IOException { builder.startObject(); // write the incompatible snapshots list builder.startArray(INCOMPATIBLE_SNAPSHOTS); @@ -452,6 +451,7 @@ public void incompatibleSnapshotsToXContent(final XContentBuilder builder) throw } builder.endArray(); builder.endObject(); + return builder; } /** diff --git a/server/src/main/java/org/elasticsearch/repositories/blobstore/BlobStoreRepository.java b/server/src/main/java/org/elasticsearch/repositories/blobstore/BlobStoreRepository.java index 49b551b26b796..25a3bf0974f33 100644 --- a/server/src/main/java/org/elasticsearch/repositories/blobstore/BlobStoreRepository.java +++ b/server/src/main/java/org/elasticsearch/repositories/blobstore/BlobStoreRepository.java @@ -50,9 +50,8 @@ import org.elasticsearch.common.collect.Tuple; import org.elasticsearch.common.component.AbstractLifecycleComponent; import org.elasticsearch.common.compress.NotXContentException; +import org.elasticsearch.common.io.Streams; import org.elasticsearch.common.io.stream.BytesStreamOutput; -import org.elasticsearch.common.io.stream.OutputStreamStreamOutput; -import org.elasticsearch.common.io.stream.StreamOutput; import org.elasticsearch.common.lucene.Lucene; import org.elasticsearch.common.lucene.store.InputStreamIndexInput; import org.elasticsearch.common.metrics.CounterMetric; @@ -60,15 +59,11 @@ import org.elasticsearch.common.settings.Settings; import org.elasticsearch.common.unit.ByteSizeUnit; import org.elasticsearch.common.unit.ByteSizeValue; -import org.elasticsearch.common.util.set.Sets; import org.elasticsearch.common.xcontent.LoggingDeprecationHandler; import org.elasticsearch.common.xcontent.NamedXContentRegistry; -import org.elasticsearch.common.xcontent.XContentBuilder; import org.elasticsearch.common.xcontent.XContentFactory; -import org.elasticsearch.common.xcontent.XContentHelper; import org.elasticsearch.common.xcontent.XContentParser; import org.elasticsearch.common.xcontent.XContentType; -import org.elasticsearch.core.internal.io.Streams; import org.elasticsearch.index.mapper.MapperService; import org.elasticsearch.index.shard.ShardId; import org.elasticsearch.index.snapshots.IndexShardRestoreFailedException; @@ -128,7 +123,7 @@ public abstract class BlobStoreRepository extends AbstractLifecycleComponent imp protected final RepositoryMetaData metadata; - private final ThreadPool threadPool; + protected final ThreadPool threadPool; private static final int BUFFER_SIZE = 4096; @@ -197,6 +192,8 @@ public abstract class BlobStoreRepository extends AbstractLifecycleComponent imp private final SetOnce blobStore = new SetOnce<>(); + private final BlobPath basePath; + /** * Constructs new BlobStoreRepository * @param metadata The metadata for this repository including name and settings @@ -204,7 +201,7 @@ public abstract class BlobStoreRepository extends AbstractLifecycleComponent imp * @param threadPool Threadpool to run long running repository manipulations on asynchronously */ protected BlobStoreRepository(RepositoryMetaData metadata, Settings settings, NamedXContentRegistry namedXContentRegistry, - ThreadPool threadPool) { + ThreadPool threadPool, BlobPath basePath) { this.settings = settings; this.metadata = metadata; this.threadPool = threadPool; @@ -212,6 +209,7 @@ protected BlobStoreRepository(RepositoryMetaData metadata, Settings settings, Na snapshotRateLimiter = getRateLimiter(metadata.settings(), "max_snapshot_bytes_per_sec", new ByteSizeValue(40, ByteSizeUnit.MB)); restoreRateLimiter = getRateLimiter(metadata.settings(), "max_restore_bytes_per_sec", new ByteSizeValue(40, ByteSizeUnit.MB)); readOnly = metadata.settings().getAsBoolean("readonly", false); + this.basePath = basePath; indexShardSnapshotFormat = new ChecksumBlobStoreFormat<>(SNAPSHOT_CODEC, SNAPSHOT_NAME_FORMAT, BlobStoreIndexShardSnapshot::fromXContent, namedXContentRegistry, compress); @@ -252,6 +250,10 @@ protected void doClose() { } } + public ThreadPool threadPool() { + return threadPool; + } + // package private, only use for testing BlobContainer getBlobContainer() { return blobContainer.get(); @@ -283,9 +285,10 @@ protected BlobContainer blobContainer() { } /** - * maintains single lazy instance of {@link BlobStore} + * Maintains single lazy instance of {@link BlobStore}. + * Public for testing. */ - protected BlobStore blobStore() { + public BlobStore blobStore() { assertSnapshotOrGenericThread(); BlobStore store = blobStore.get(); @@ -317,8 +320,11 @@ protected BlobStore blobStore() { /** * Returns base path of the repository + * Public for testing. */ - protected abstract BlobPath basePath(); + public BlobPath basePath() { + return basePath; + } /** * Returns true if metadata and snapshot files should be compressed @@ -363,10 +369,7 @@ public void initializeSnapshot(SnapshotId snapshotId, List indices, Met // write the index metadata for each index in the snapshot for (IndexId index : indices) { - final IndexMetaData indexMetaData = clusterMetaData.index(index.getName()); - final BlobPath indexPath = basePath().add("indices").add(index.getId()); - final BlobContainer indexMetaDataBlobContainer = blobStore().blobContainer(indexPath); - indexMetaDataFormat.write(indexMetaData, indexMetaDataBlobContainer, snapshotId.getUUID()); + indexMetaDataFormat.write(clusterMetaData.index(index.getName()), indexContainer(index), snapshotId.getUUID()); } } catch (IOException ex) { throw new SnapshotCreationException(metadata.name(), snapshotId, ex); @@ -388,46 +391,68 @@ public void deleteSnapshot(SnapshotId snapshotId, long repositoryStateId, Action logger.warn(() -> new ParameterizedMessage("cannot read snapshot file [{}]", snapshotId), ex); } // Delete snapshot from the index file, since it is the maintainer of truth of active snapshots - final RepositoryData repositoryData; final RepositoryData updatedRepositoryData; + final Map foundIndices; try { - repositoryData = getRepositoryData(); + final RepositoryData repositoryData = getRepositoryData(); updatedRepositoryData = repositoryData.removeSnapshot(snapshotId); + // Cache the indices that were found before writing out the new index-N blob so that a stuck master will never + // delete an index that was created by another master node after writing this index-N blob. + foundIndices = blobStore().blobContainer(basePath().add("indices")).children(); writeIndexGen(updatedRepositoryData, repositoryStateId); } catch (Exception ex) { listener.onFailure(new RepositoryException(metadata.name(), "failed to delete snapshot [" + snapshotId + "]", ex)); return; } final SnapshotInfo finalSnapshotInfo = snapshot; - final Collection unreferencedIndices = Sets.newHashSet(repositoryData.getIndices().values()); - unreferencedIndices.removeAll(updatedRepositoryData.getIndices().values()); try { blobContainer().deleteBlobsIgnoringIfNotExists( Arrays.asList(snapshotFormat.blobName(snapshotId.getUUID()), globalMetaDataFormat.blobName(snapshotId.getUUID()))); } catch (IOException e) { logger.warn(() -> new ParameterizedMessage("[{}] Unable to delete global metadata files", snapshotId), e); } + final var survivingIndices = updatedRepositoryData.getIndices(); deleteIndices( Optional.ofNullable(finalSnapshotInfo) - .map(info -> info.indices().stream().map(repositoryData::resolveIndexId).collect(Collectors.toList())) + .map(info -> info.indices().stream().filter(survivingIndices::containsKey) + .map(updatedRepositoryData::resolveIndexId).collect(Collectors.toList())) .orElse(Collections.emptyList()), snapshotId, ActionListener.map(listener, v -> { - try { - blobStore().blobContainer(basePath().add("indices")).deleteBlobsIgnoringIfNotExists( - unreferencedIndices.stream().map(IndexId::getId).collect(Collectors.toList())); - } catch (IOException e) { - logger.warn(() -> - new ParameterizedMessage( - "[{}] indices {} are no longer part of any snapshots in the repository, " + - "but failed to clean up their index folders.", metadata.name(), unreferencedIndices), e); - } + cleanupStaleIndices(foundIndices, survivingIndices); return null; }) ); } } + private void cleanupStaleIndices(Map foundIndices, Map survivingIndices) { + try { + final Set survivingIndexIds = survivingIndices.values().stream() + .map(IndexId::getId).collect(Collectors.toSet()); + for (Map.Entry indexEntry : foundIndices.entrySet()) { + final String indexSnId = indexEntry.getKey(); + try { + if (survivingIndexIds.contains(indexSnId) == false) { + logger.debug("[{}] Found stale index [{}]. Cleaning it up", metadata.name(), indexSnId); + indexEntry.getValue().delete(); + logger.debug("[{}] Cleaned up stale index [{}]", metadata.name(), indexSnId); + } + } catch (IOException e) { + logger.warn(() -> new ParameterizedMessage( + "[{}] index {} is no longer part of any snapshots in the repository, " + + "but failed to clean up their index folders", metadata.name(), indexSnId), e); + } + } + } catch (Exception e) { + // TODO: We shouldn't be blanket catching and suppressing all exceptions here and instead handle them safely upstream. + // Currently this catch exists as a stop gap solution to tackle unexpected runtime exceptions from implementations + // bubbling up and breaking the snapshot functionality. + assert false : e; + logger.warn(new ParameterizedMessage("[{}] Exception during cleanup of stale indices", metadata.name()), e); + } + } + private void deleteIndices(List indices, SnapshotId snapshotId, ActionListener listener) { if (indices.isEmpty()) { listener.onResponse(null); @@ -450,8 +475,7 @@ protected void doRun() { if (indexMetaData != null) { for (int shardId = 0; shardId < indexMetaData.getNumberOfShards(); shardId++) { try { - final ShardId sid = new ShardId(indexMetaData.getIndex(), shardId); - new Context(snapshotId, indexId, sid, sid).delete(); + deleteShardSnapshot(indexId, new ShardId(indexMetaData.getIndex(), shardId), snapshotId); } catch (SnapshotException ex) { final int finalShardId = shardId; logger.warn(() -> new ParameterizedMessage("[{}] failed to delete shard data for shard [{}][{}]", @@ -466,9 +490,8 @@ protected void doRun() { } private void deleteIndexMetaDataBlobIgnoringErrors(SnapshotId snapshotId, IndexId indexId) { - BlobContainer indexMetaDataBlobContainer = blobStore().blobContainer(basePath().add("indices").add(indexId.getId())); try { - indexMetaDataFormat.delete(indexMetaDataBlobContainer, snapshotId.getUUID()); + indexMetaDataFormat.delete(indexContainer(indexId), snapshotId.getUUID()); } catch (IOException ex) { logger.warn(() -> new ParameterizedMessage("[{}] failed to delete metadata for index [{}]", snapshotId, indexId.getName()), ex); @@ -483,15 +506,16 @@ public SnapshotInfo finalizeSnapshot(final SnapshotId snapshotId, final int totalShards, final List shardFailures, final long repositoryStateId, - final boolean includeGlobalState) { + final boolean includeGlobalState, + final Map userMetadata) { SnapshotInfo blobStoreSnapshot = new SnapshotInfo(snapshotId, indices.stream().map(IndexId::getName).collect(Collectors.toList()), startTime, failure, System.currentTimeMillis(), totalShards, shardFailures, - includeGlobalState); + includeGlobalState, userMetadata); try { + final RepositoryData updatedRepositoryData = getRepositoryData().addSnapshot(snapshotId, blobStoreSnapshot.state(), indices); snapshotFormat.write(blobStoreSnapshot, blobContainer(), snapshotId.getUUID()); - final RepositoryData repositoryData = getRepositoryData(); - writeIndexGen(repositoryData.addSnapshot(snapshotId, blobStoreSnapshot.state(), indices), repositoryStateId); + writeIndexGen(updatedRepositoryData, repositoryStateId); } catch (FileAlreadyExistsException ex) { // if another master was elected and took over finalizing the snapshot, it is possible // that both nodes try to finalize the snapshot and write to the same blobs, so we just @@ -528,8 +552,19 @@ public MetaData getSnapshotGlobalMetaData(final SnapshotId snapshotId) { @Override public IndexMetaData getSnapshotIndexMetaData(final SnapshotId snapshotId, final IndexId index) throws IOException { - final BlobPath indexPath = basePath().add("indices").add(index.getId()); - return indexMetaDataFormat.read(blobStore().blobContainer(indexPath), snapshotId.getUUID()); + return indexMetaDataFormat.read(indexContainer(index), snapshotId.getUUID()); + } + + private BlobPath indicesPath() { + return basePath().add("indices"); + } + + private BlobContainer indexContainer(IndexId indexId) { + return blobStore().blobContainer(indicesPath().add(indexId.getId())); + } + + private BlobContainer shardContainer(IndexId indexId, ShardId shardId) { + return blobStore().blobContainer(indicesPath().add(indexId.getId()).add(Integer.toString(shardId.getId()))); } /** @@ -577,10 +612,9 @@ public String startVerification() { String seed = UUIDs.randomBase64UUID(); byte[] testBytes = Strings.toUTF8Bytes(seed); BlobContainer testContainer = blobStore().blobContainer(basePath().add(testBlobPrefix(seed))); - String blobName = "master.dat"; BytesArray bytes = new BytesArray(testBytes); try (InputStream stream = bytes.streamInput()) { - testContainer.writeBlobAtomic(blobName, stream, bytes.length(), true); + testContainer.writeBlobAtomic("master.dat", stream, bytes.length(), true); } return seed; } @@ -610,32 +644,23 @@ public RepositoryData getRepositoryData() { final String snapshotsIndexBlobName = INDEX_FILE_PREFIX + Long.toString(indexGen); RepositoryData repositoryData; - try (InputStream blob = blobContainer().readBlob(snapshotsIndexBlobName)) { - BytesStreamOutput out = new BytesStreamOutput(); - Streams.copy(blob, out); - // EMPTY is safe here because RepositoryData#fromXContent calls namedObject - try (XContentParser parser = XContentHelper.createParser(NamedXContentRegistry.EMPTY, - LoggingDeprecationHandler.INSTANCE, out.bytes(), XContentType.JSON)) { - repositoryData = RepositoryData.snapshotsFromXContent(parser, indexGen); - } catch (NotXContentException e) { - logger.warn("[{}] index blob is not valid x-content [{} bytes]", snapshotsIndexBlobName, out.bytes().length()); - throw e; - } + // EMPTY is safe here because RepositoryData#fromXContent calls namedObject + try (InputStream blob = blobContainer().readBlob(snapshotsIndexBlobName); + XContentParser parser = XContentType.JSON.xContent().createParser(NamedXContentRegistry.EMPTY, + LoggingDeprecationHandler.INSTANCE, blob)) { + repositoryData = RepositoryData.snapshotsFromXContent(parser, indexGen); } // now load the incompatible snapshot ids, if they exist - try (InputStream blob = blobContainer().readBlob(INCOMPATIBLE_SNAPSHOTS_BLOB)) { - BytesStreamOutput out = new BytesStreamOutput(); - Streams.copy(blob, out); - try (XContentParser parser = XContentHelper.createParser(NamedXContentRegistry.EMPTY, - LoggingDeprecationHandler.INSTANCE, out.bytes(), XContentType.JSON)) { - repositoryData = repositoryData.incompatibleSnapshotsFromXContent(parser); - } + try (InputStream blob = blobContainer().readBlob(INCOMPATIBLE_SNAPSHOTS_BLOB); + XContentParser parser = XContentType.JSON.xContent().createParser(NamedXContentRegistry.EMPTY, + LoggingDeprecationHandler.INSTANCE, blob)) { + repositoryData = repositoryData.incompatibleSnapshotsFromXContent(parser); } catch (NoSuchFileException e) { if (isReadOnly()) { logger.debug("[{}] Incompatible snapshots blob [{}] does not exist, the likely " + - "reason is that there are no incompatible snapshots in the repository", - metadata.name(), INCOMPATIBLE_SNAPSHOTS_BLOB); + "reason is that there are no incompatible snapshots in the repository", + metadata.name(), INCOMPATIBLE_SNAPSHOTS_BLOB); } else { // write an empty incompatible-snapshots blob - we do this so that there // is a blob present, which helps speed up some cloud-based repositories @@ -653,7 +678,7 @@ public RepositoryData getRepositoryData() { } } - public static String testBlobPrefix(String seed) { + private static String testBlobPrefix(String seed) { return TESTS_FILE + seed; } @@ -673,25 +698,10 @@ protected void writeIndexGen(final RepositoryData repositoryData, final long rep "] - possibly due to simultaneous snapshot deletion requests"); } final long newGen = currentGen + 1; - final BytesReference snapshotsBytes; - try (BytesStreamOutput bStream = new BytesStreamOutput()) { - try (StreamOutput stream = new OutputStreamStreamOutput(bStream)) { - XContentBuilder builder = XContentFactory.contentBuilder(XContentType.JSON, stream); - repositoryData.snapshotsToXContent(builder); - builder.close(); - } - snapshotsBytes = bStream.bytes(); - } // write the index file final String indexBlob = INDEX_FILE_PREFIX + Long.toString(newGen); logger.debug("Repository [{}] writing new index generational blob [{}]", metadata.name(), indexBlob); - writeAtomic(indexBlob, snapshotsBytes, true); - // delete the N-2 index file if it exists, keep the previous one around as a backup - if (isReadOnly() == false && newGen - 2 >= 0) { - final String oldSnapshotIndexFile = INDEX_FILE_PREFIX + Long.toString(newGen - 2); - blobContainer().deleteBlobIgnoringIfNotExists(oldSnapshotIndexFile); - } - + writeAtomic(indexBlob, BytesReference.bytes(repositoryData.snapshotsToXContent(XContentFactory.jsonBuilder())), true); // write the current generation to the index-latest file final BytesReference genBytes; try (BytesStreamOutput bStream = new BytesStreamOutput()) { @@ -700,6 +710,15 @@ protected void writeIndexGen(final RepositoryData repositoryData, final long rep } logger.debug("Repository [{}] updating index.latest with generation [{}]", metadata.name(), newGen); writeAtomic(INDEX_LATEST_BLOB, genBytes, false); + // delete the N-2 index file if it exists, keep the previous one around as a backup + if (newGen - 2 >= 0) { + final String oldSnapshotIndexFile = INDEX_FILE_PREFIX + Long.toString(newGen - 2); + try { + blobContainer().deleteBlobIgnoringIfNotExists(oldSnapshotIndexFile); + } catch (IOException e) { + logger.warn("Failed to clean up old index blob [{}]", oldSnapshotIndexFile); + } + } } /** @@ -709,16 +728,9 @@ protected void writeIndexGen(final RepositoryData repositoryData, final long rep */ void writeIncompatibleSnapshots(RepositoryData repositoryData) throws IOException { assert isReadOnly() == false; // can not write to a read only repository - final BytesReference bytes; - try (BytesStreamOutput bStream = new BytesStreamOutput()) { - try (StreamOutput stream = new OutputStreamStreamOutput(bStream); - XContentBuilder builder = XContentFactory.contentBuilder(XContentType.JSON, stream)) { - repositoryData.incompatibleSnapshotsToXContent(builder); - } - bytes = bStream.bytes(); - } // write the incompatible snapshots blob - writeAtomic(INCOMPATIBLE_SNAPSHOTS_BLOB, bytes, false); + writeAtomic(INCOMPATIBLE_SNAPSHOTS_BLOB, + BytesReference.bytes(repositoryData.incompatibleSnapshotsToXContent(XContentFactory.jsonBuilder())), false); } /** @@ -756,11 +768,7 @@ long latestIndexBlobId() throws IOException { // package private for testing long readSnapshotIndexLatestBlob() throws IOException { - try (InputStream blob = blobContainer().readBlob(INDEX_LATEST_BLOB)) { - BytesStreamOutput out = new BytesStreamOutput(); - Streams.copy(blob, out); - return Numbers.bytesToLong(out.bytes().toBytesRef()); - } + return Numbers.bytesToLong(Streams.readFully(blobContainer().readBlob(INDEX_LATEST_BLOB)).toBytesRef()); } private long listBlobsToGetLatestIndexId() throws IOException { @@ -793,306 +801,20 @@ private void writeAtomic(final String blobName, final BytesReference bytesRef, b @Override public void snapshotShard(Store store, MapperService mapperService, SnapshotId snapshotId, IndexId indexId, IndexCommit snapshotIndexCommit, IndexShardSnapshotStatus snapshotStatus) { - SnapshotContext snapshotContext = new SnapshotContext(store, snapshotId, indexId, snapshotStatus, System.currentTimeMillis()); + final ShardId shardId = store.shardId(); + final long startTime = threadPool.relativeTimeInMillis(); try { - snapshotContext.snapshot(snapshotIndexCommit); - } catch (Exception e) { - snapshotStatus.moveToFailed(System.currentTimeMillis(), ExceptionsHelper.detailedMessage(e)); - if (e instanceof IndexShardSnapshotFailedException) { - throw (IndexShardSnapshotFailedException) e; - } else { - throw new IndexShardSnapshotFailedException(store.shardId(), e); - } - } - } - - @Override - public void restoreShard(Store store, SnapshotId snapshotId, - Version version, IndexId indexId, ShardId snapshotShardId, RecoveryState recoveryState) { - ShardId shardId = store.shardId(); - final Context context = new Context(snapshotId, indexId, shardId, snapshotShardId); - BlobPath path = basePath().add("indices").add(indexId.getId()).add(Integer.toString(snapshotShardId.getId())); - BlobContainer blobContainer = blobStore().blobContainer(path); - final RestoreContext snapshotContext = new RestoreContext(shardId, snapshotId, recoveryState, blobContainer); - try { - BlobStoreIndexShardSnapshot snapshot = context.loadSnapshot(); - SnapshotFiles snapshotFiles = new SnapshotFiles(snapshot.snapshot(), snapshot.indexFiles()); - snapshotContext.restore(snapshotFiles, store); - } catch (Exception e) { - throw new IndexShardRestoreFailedException(shardId, "failed to restore snapshot [" + snapshotId + "]", e); - } - } - - @Override - public IndexShardSnapshotStatus getShardSnapshotStatus(SnapshotId snapshotId, Version version, IndexId indexId, ShardId shardId) { - Context context = new Context(snapshotId, indexId, shardId); - BlobStoreIndexShardSnapshot snapshot = context.loadSnapshot(); - return IndexShardSnapshotStatus.newDone(snapshot.startTime(), snapshot.time(), - snapshot.incrementalFileCount(), snapshot.totalFileCount(), - snapshot.incrementalSize(), snapshot.totalSize()); - } - - @Override - public void verify(String seed, DiscoveryNode localNode) { - assertSnapshotOrGenericThread(); - if (isReadOnly()) { - try { - latestIndexBlobId(); - } catch (IOException e) { - throw new RepositoryVerificationException(metadata.name(), "path " + basePath() + - " is not accessible on node " + localNode, e); - } - } else { - BlobContainer testBlobContainer = blobStore().blobContainer(basePath().add(testBlobPrefix(seed))); - if (testBlobContainer.blobExists("master.dat")) { - try { - BytesArray bytes = new BytesArray(seed); - try (InputStream stream = bytes.streamInput()) { - testBlobContainer.writeBlob("data-" + localNode.getId() + ".dat", stream, bytes.length(), true); - } - } catch (IOException exp) { - throw new RepositoryVerificationException(metadata.name(), "store location [" + blobStore() + - "] is not accessible on the node [" + localNode + "]", exp); - } - } else { - throw new RepositoryVerificationException(metadata.name(), "a file written by master to the store [" + blobStore() + - "] cannot be accessed on the node [" + localNode + "]. " + - "This might indicate that the store [" + blobStore() + "] is not shared between this node and the master node or " + - "that permissions on the store don't allow reading files written by the master node"); - } - } - } - - @Override - public String toString() { - return "BlobStoreRepository[" + - "[" + metadata.name() + - "], [" + blobStore() + ']' + - ']'; - } - - /** - * Context for snapshot/restore operations - */ - private class Context { - - protected final SnapshotId snapshotId; - - protected final ShardId shardId; - - protected final BlobContainer blobContainer; - - Context(SnapshotId snapshotId, IndexId indexId, ShardId shardId) { - this(snapshotId, indexId, shardId, shardId); - } - - Context(SnapshotId snapshotId, IndexId indexId, ShardId shardId, ShardId snapshotShardId) { - this.snapshotId = snapshotId; - this.shardId = shardId; - blobContainer = blobStore().blobContainer(basePath().add("indices").add(indexId.getId()) - .add(Integer.toString(snapshotShardId.getId()))); - } - - /** - * Delete shard snapshot - */ - public void delete() { - final Map blobs; - try { - blobs = blobContainer.listBlobs(); - } catch (IOException e) { - throw new IndexShardSnapshotException(shardId, "Failed to list content of gateway", e); - } - - Tuple tuple = buildBlobStoreIndexShardSnapshots(blobs); - BlobStoreIndexShardSnapshots snapshots = tuple.v1(); - int fileListGeneration = tuple.v2(); - - try { - indexShardSnapshotFormat.delete(blobContainer, snapshotId.getUUID()); - } catch (IOException e) { - logger.warn(new ParameterizedMessage("[{}] [{}] failed to delete shard snapshot file", shardId, snapshotId), e); - } - - // Build a list of snapshots that should be preserved - List newSnapshotsList = new ArrayList<>(); - for (SnapshotFiles point : snapshots) { - if (!point.snapshot().equals(snapshotId.getName())) { - newSnapshotsList.add(point); - } - } - // finalize the snapshot and rewrite the snapshot index with the next sequential snapshot index - finalize(newSnapshotsList, fileListGeneration + 1, blobs, "snapshot deletion [" + snapshotId + "]"); - } - - /** - * Loads information about shard snapshot - */ - BlobStoreIndexShardSnapshot loadSnapshot() { - try { - return indexShardSnapshotFormat.read(blobContainer, snapshotId.getUUID()); - } catch (IOException ex) { - throw new SnapshotException(metadata.name(), snapshotId, "failed to read shard snapshot file for " + shardId, ex); - } - } - - /** - * Writes a new index file for the shard and removes all unreferenced files from the repository. - * - * We need to be really careful in handling index files in case of failures to make sure we don't - * have index file that points to files that were deleted. - * - * @param snapshots list of active snapshots in the container - * @param fileListGeneration the generation number of the snapshot index file - * @param blobs list of blobs in the container - * @param reason a reason explaining why the shard index file is written - */ - protected void finalize(final List snapshots, - final int fileListGeneration, - final Map blobs, - final String reason) { - final String indexGeneration = Integer.toString(fileListGeneration); - final BlobStoreIndexShardSnapshots updatedSnapshots = new BlobStoreIndexShardSnapshots(snapshots); - try { - // Delete temporary index files first, as we might otherwise fail in the next step creating the new index file if an earlier - // attempt to write an index file with this generation failed mid-way after creating the temporary file. - final List blobNames = - blobs.keySet().stream().filter(FsBlobContainer::isTempBlobName).collect(Collectors.toList()); - try { - blobContainer.deleteBlobsIgnoringIfNotExists(blobNames); - } catch (IOException e) { - logger.warn(() -> new ParameterizedMessage("[{}][{}] failed to delete index blobs during finalization", - snapshotId, shardId), e); - throw e; - } - - // If we deleted all snapshots, we don't need to create a new index file - if (snapshots.size() > 0) { - indexShardSnapshotsFormat.writeAtomic(updatedSnapshots, blobContainer, indexGeneration); - } - - // Delete old index files - final List indexBlobs = - blobs.keySet().stream().filter(blob -> blob.startsWith(SNAPSHOT_INDEX_PREFIX)).collect(Collectors.toList()); - try { - blobContainer.deleteBlobsIgnoringIfNotExists(indexBlobs); - } catch (IOException e) { - logger.warn(() -> new ParameterizedMessage("[{}][{}] failed to delete index blobs during finalization", - snapshotId, shardId), e); - throw e; - } - - // Delete all blobs that don't exist in a snapshot - final List orphanedBlobs = blobs.keySet().stream() - .filter(blobName -> - blobName.startsWith(DATA_BLOB_PREFIX) && updatedSnapshots.findNameFile(canonicalName(blobName)) == null) - .collect(Collectors.toList()); - try { - blobContainer.deleteBlobsIgnoringIfNotExists(orphanedBlobs); - } catch (IOException e) { - logger.warn(() -> new ParameterizedMessage("[{}][{}] failed to delete data blobs during finalization", - snapshotId, shardId), e); - } - } catch (IOException e) { - String message = - "Failed to finalize " + reason + " with shard index [" + indexShardSnapshotsFormat.blobName(indexGeneration) + "]"; - throw new IndexShardSnapshotFailedException(shardId, message, e); - } - } - - /** - * Loads all available snapshots in the repository - * - * @param blobs list of blobs in repository - * @return tuple of BlobStoreIndexShardSnapshots and the last snapshot index generation - */ - protected Tuple buildBlobStoreIndexShardSnapshots(Map blobs) { - int latest = -1; - Set blobKeys = blobs.keySet(); - for (String name : blobKeys) { - if (name.startsWith(SNAPSHOT_INDEX_PREFIX)) { - try { - int gen = Integer.parseInt(name.substring(SNAPSHOT_INDEX_PREFIX.length())); - if (gen > latest) { - latest = gen; - } - } catch (NumberFormatException ex) { - logger.warn("failed to parse index file name [{}]", name); - } - } - } - if (latest >= 0) { - try { - final BlobStoreIndexShardSnapshots shardSnapshots = - indexShardSnapshotsFormat.read(blobContainer, Integer.toString(latest)); - return new Tuple<>(shardSnapshots, latest); - } catch (IOException e) { - final String file = SNAPSHOT_INDEX_PREFIX + latest; - logger.warn(() -> new ParameterizedMessage("failed to read index file [{}]", file), e); - } - } else if (blobKeys.isEmpty() == false) { - logger.warn("Could not find a readable index-N file in a non-empty shard snapshot directory [{}]", blobContainer.path()); - } - - // We couldn't load the index file - falling back to loading individual snapshots - List snapshots = new ArrayList<>(); - for (String name : blobKeys) { - try { - BlobStoreIndexShardSnapshot snapshot = null; - if (name.startsWith(SNAPSHOT_PREFIX)) { - snapshot = indexShardSnapshotFormat.readBlob(blobContainer, name); - } - if (snapshot != null) { - snapshots.add(new SnapshotFiles(snapshot.snapshot(), snapshot.indexFiles())); - } - } catch (IOException e) { - logger.warn(() -> new ParameterizedMessage("failed to read commit point [{}]", name), e); - } - } - return new Tuple<>(new BlobStoreIndexShardSnapshots(snapshots), latest); - } - } - - /** - * Context for snapshot operations - */ - private class SnapshotContext extends Context { - - private final Store store; - private final IndexShardSnapshotStatus snapshotStatus; - private final long startTime; - - /** - * Constructs new context - * - * @param store store to be snapshotted - * @param snapshotId snapshot id - * @param indexId the id of the index being snapshotted - * @param snapshotStatus snapshot status to report progress - */ - SnapshotContext(Store store, SnapshotId snapshotId, IndexId indexId, IndexShardSnapshotStatus snapshotStatus, long startTime) { - super(snapshotId, indexId, store.shardId()); - this.snapshotStatus = snapshotStatus; - this.store = store; - this.startTime = startTime; - } - - /** - * Create snapshot from index commit point - * - * @param snapshotIndexCommit snapshot commit point - */ - public void snapshot(final IndexCommit snapshotIndexCommit) { logger.debug("[{}] [{}] snapshot to [{}] ...", shardId, snapshotId, metadata.name()); + final BlobContainer shardContainer = shardContainer(indexId, shardId); final Map blobs; try { - blobs = blobContainer.listBlobs(); + blobs = shardContainer.listBlobs(); } catch (IOException e) { throw new IndexShardSnapshotFailedException(shardId, "failed to list blobs", e); } - Tuple tuple = buildBlobStoreIndexShardSnapshots(blobs); + Tuple tuple = buildBlobStoreIndexShardSnapshots(blobs, shardContainer); BlobStoreIndexShardSnapshots snapshots = tuple.v1(); int fileListGeneration = tuple.v2(); @@ -1102,24 +824,24 @@ public void snapshot(final IndexCommit snapshotIndexCommit) { } final List indexCommitPointFiles = new ArrayList<>(); - store.incRef(); - int indexIncrementalFileCount = 0; - int indexTotalNumberOfFiles = 0; - long indexIncrementalSize = 0; - long indexTotalFileCount = 0; try { ArrayList filesToSnapshot = new ArrayList<>(); final Store.MetadataSnapshot metadata; // TODO apparently we don't use the MetadataSnapshot#.recoveryDiff(...) here but we should final Collection fileNames; try { - logger.trace("[{}] [{}] Loading store metadata using index commit [{}]", shardId, snapshotId, snapshotIndexCommit); + logger.trace( + "[{}] [{}] Loading store metadata using index commit [{}]", shardId, snapshotId, snapshotIndexCommit); metadata = store.getMetadata(snapshotIndexCommit); fileNames = snapshotIndexCommit.getFileNames(); } catch (IOException e) { throw new IndexShardSnapshotFailedException(shardId, "Failed to get store file metadata", e); } + int indexIncrementalFileCount = 0; + int indexTotalNumberOfFiles = 0; + long indexIncrementalSize = 0; + long indexTotalFileCount = 0; for (String fileName : fileNames) { if (snapshotStatus.isAborted()) { logger.debug("[{}] [{}] Aborted on the file [{}], exiting", shardId, snapshotId, fileName); @@ -1162,7 +884,7 @@ public void snapshot(final IndexCommit snapshotIndexCommit) { for (BlobStoreIndexShardSnapshot.FileInfo snapshotFileInfo : filesToSnapshot) { try { - snapshotFile(snapshotFileInfo); + snapshotFile(snapshotFileInfo, indexId, shardId, snapshotId, snapshotStatus, store); } catch (IOException e) { throw new IndexShardSnapshotFailedException(shardId, "Failed to perform snapshot (index files)", e); } @@ -1175,20 +897,20 @@ public void snapshot(final IndexCommit snapshotIndexCommit) { // now create and write the commit point final BlobStoreIndexShardSnapshot snapshot = new BlobStoreIndexShardSnapshot(snapshotId.getName(), - lastSnapshotStatus.getIndexVersion(), - indexCommitPointFiles, - lastSnapshotStatus.getStartTime(), - // snapshotStatus.startTime() is assigned on the same machine, - // so it's safe to use with VLong - System.currentTimeMillis() - lastSnapshotStatus.getStartTime(), - lastSnapshotStatus.getIncrementalFileCount(), - lastSnapshotStatus.getIncrementalSize() + lastSnapshotStatus.getIndexVersion(), + indexCommitPointFiles, + lastSnapshotStatus.getStartTime(), + // snapshotStatus.startTime() is assigned on the same machine, + // so it's safe to use the relative time in millis + threadPool.relativeTimeInMillis() - lastSnapshotStatus.getStartTime(), + lastSnapshotStatus.getIncrementalFileCount(), + lastSnapshotStatus.getIncrementalSize() ); //TODO: The time stored in snapshot doesn't include cleanup time. logger.trace("[{}] [{}] writing shard snapshot file", shardId, snapshotId); try { - indexShardSnapshotFormat.write(snapshot, blobContainer, snapshotId.getUUID()); + indexShardSnapshotFormat.write(snapshot, shardContainer, snapshotId.getUUID()); } catch (IOException e) { throw new IndexShardSnapshotFailedException(shardId, "Failed to write commit point", e); } @@ -1201,151 +923,317 @@ public void snapshot(final IndexCommit snapshotIndexCommit) { newSnapshotsList.add(point); } // finalize the snapshot and rewrite the snapshot index with the next sequential snapshot index - finalize(newSnapshotsList, fileListGeneration + 1, blobs, "snapshot creation [" + snapshotId + "]"); - snapshotStatus.moveToDone(System.currentTimeMillis()); + finalizeShard(newSnapshotsList, fileListGeneration, blobs, "snapshot creation [" + snapshotId + "]", shardContainer, + shardId, snapshotId); + snapshotStatus.moveToDone(threadPool.relativeTimeInMillis()); + } catch (Exception e) { + snapshotStatus.moveToFailed(threadPool.relativeTimeInMillis(), ExceptionsHelper.detailedMessage(e)); + if (e instanceof IndexShardSnapshotFailedException) { + throw (IndexShardSnapshotFailedException) e; + } else { + throw new IndexShardSnapshotFailedException(store.shardId(), e); + } } + } - /** - * Snapshot individual file - * - * @param fileInfo file to be snapshotted - */ - private void snapshotFile(final BlobStoreIndexShardSnapshot.FileInfo fileInfo) throws IOException { - final String file = fileInfo.physicalName(); - try (IndexInput indexInput = store.openVerifyingInput(file, IOContext.READONCE, fileInfo.metadata())) { - for (int i = 0; i < fileInfo.numberOfParts(); i++) { - final long partBytes = fileInfo.partBytes(i); - - final InputStreamIndexInput inputStreamIndexInput = new InputStreamIndexInput(indexInput, partBytes); - InputStream inputStream = inputStreamIndexInput; - if (snapshotRateLimiter != null) { - inputStream = new RateLimitingInputStream(inputStreamIndexInput, snapshotRateLimiter, - snapshotRateLimitingTimeInNanos::inc); - } - inputStream = new AbortableInputStream(inputStream, fileInfo.physicalName()); - blobContainer.writeBlob(fileInfo.partName(i), inputStream, partBytes, true); + @Override + public void restoreShard(Store store, SnapshotId snapshotId, Version version, IndexId indexId, ShardId snapshotShardId, + RecoveryState recoveryState) { + ShardId shardId = store.shardId(); + try { + final BlobContainer container = shardContainer(indexId, snapshotShardId); + BlobStoreIndexShardSnapshot snapshot = loadShardSnapshot(container, snapshotId); + SnapshotFiles snapshotFiles = new SnapshotFiles(snapshot.snapshot(), snapshot.indexFiles()); + new FileRestoreContext(metadata.name(), shardId, snapshotId, recoveryState, BUFFER_SIZE) { + @Override + protected InputStream fileInputStream(BlobStoreIndexShardSnapshot.FileInfo fileInfo) { + final InputStream dataBlobCompositeStream = new SlicedInputStream(fileInfo.numberOfParts()) { + @Override + protected InputStream openSlice(long slice) throws IOException { + return container.readBlob(fileInfo.partName(slice)); + } + }; + return restoreRateLimiter == null ? dataBlobCompositeStream + : new RateLimitingInputStream(dataBlobCompositeStream, restoreRateLimiter, restoreRateLimitingTimeInNanos::inc); } - Store.verify(indexInput); - snapshotStatus.addProcessedFile(fileInfo.length()); - } catch (Exception t) { - failStoreIfCorrupted(t); - snapshotStatus.addProcessedFile(0); - throw t; - } + }.restore(snapshotFiles, store); + } catch (Exception e) { + throw new IndexShardRestoreFailedException(shardId, "failed to restore snapshot [" + snapshotId + "]", e); } + } + + @Override + public IndexShardSnapshotStatus getShardSnapshotStatus(SnapshotId snapshotId, Version version, IndexId indexId, ShardId shardId) { + BlobStoreIndexShardSnapshot snapshot = loadShardSnapshot(shardContainer(indexId, shardId), snapshotId); + return IndexShardSnapshotStatus.newDone(snapshot.startTime(), snapshot.time(), + snapshot.incrementalFileCount(), snapshot.totalFileCount(), + snapshot.incrementalSize(), snapshot.totalSize()); + } - private void failStoreIfCorrupted(Exception e) { - if (Lucene.isCorruptionException(e)) { + @Override + public void verify(String seed, DiscoveryNode localNode) { + assertSnapshotOrGenericThread(); + if (isReadOnly()) { + try { + latestIndexBlobId(); + } catch (IOException e) { + throw new RepositoryVerificationException(metadata.name(), "path " + basePath() + + " is not accessible on node " + localNode, e); + } + } else { + BlobContainer testBlobContainer = blobStore().blobContainer(basePath().add(testBlobPrefix(seed))); + if (testBlobContainer.blobExists("master.dat")) { try { - store.markStoreCorrupted((IOException) e); - } catch (IOException inner) { - inner.addSuppressed(e); - logger.warn("store cannot be marked as corrupted", inner); + BytesArray bytes = new BytesArray(seed); + try (InputStream stream = bytes.streamInput()) { + testBlobContainer.writeBlob("data-" + localNode.getId() + ".dat", stream, bytes.length(), true); + } + } catch (IOException exp) { + throw new RepositoryVerificationException(metadata.name(), "store location [" + blobStore() + + "] is not accessible on the node [" + localNode + "]", exp); } + } else { + throw new RepositoryVerificationException(metadata.name(), "a file written by master to the store [" + blobStore() + + "] cannot be accessed on the node [" + localNode + "]. " + + "This might indicate that the store [" + blobStore() + "] is not shared between this node and the master node or " + + "that permissions on the store don't allow reading files written by the master node"); } } + } - /** - * Checks if snapshot file already exists in the list of blobs - * - * @param fileInfo file to check - * @param blobs list of blobs - * @return true if file exists in the list of blobs - */ - private boolean snapshotFileExistsInBlobs(BlobStoreIndexShardSnapshot.FileInfo fileInfo, Map blobs) { - BlobMetaData blobMetaData = blobs.get(fileInfo.name()); - if (blobMetaData != null) { - return blobMetaData.length() == fileInfo.length(); - } else if (blobs.containsKey(fileInfo.partName(0))) { - // multi part file sum up the size and check - int part = 0; - long totalSize = 0; - while (true) { - blobMetaData = blobs.get(fileInfo.partName(part++)); - if (blobMetaData == null) { - break; - } - totalSize += blobMetaData.length(); - } - return totalSize == fileInfo.length(); - } - // no file, not exact and not multipart - return false; + @Override + public String toString() { + return "BlobStoreRepository[" + + "[" + metadata.name() + + "], [" + blobStore() + ']' + + ']'; + } + + /** + * Delete shard snapshot + */ + private void deleteShardSnapshot(IndexId indexId, ShardId snapshotShardId, SnapshotId snapshotId) { + final BlobContainer shardContainer = shardContainer(indexId, snapshotShardId); + final Map blobs; + try { + blobs = shardContainer.listBlobs(); + } catch (IOException e) { + throw new IndexShardSnapshotException(snapshotShardId, "Failed to list content of shard directory", e); } - private class AbortableInputStream extends FilterInputStream { - private final String fileName; + Tuple tuple = buildBlobStoreIndexShardSnapshots(blobs, shardContainer); + BlobStoreIndexShardSnapshots snapshots = tuple.v1(); + int fileListGeneration = tuple.v2(); + + try { + indexShardSnapshotFormat.delete(shardContainer, snapshotId.getUUID()); + } catch (IOException e) { + logger.warn(new ParameterizedMessage("[{}] [{}] failed to delete shard snapshot file", snapshotShardId, snapshotId), e); + } - AbortableInputStream(InputStream delegate, String fileName) { - super(delegate); - this.fileName = fileName; + // Build a list of snapshots that should be preserved + List newSnapshotsList = new ArrayList<>(); + for (SnapshotFiles point : snapshots) { + if (!point.snapshot().equals(snapshotId.getName())) { + newSnapshotsList.add(point); } + } + // finalize the snapshot and rewrite the snapshot index with the next sequential snapshot index + finalizeShard(newSnapshotsList, fileListGeneration, blobs, "snapshot deletion [" + snapshotId + "]", shardContainer, + snapshotShardId, snapshotId); + } - @Override - public int read() throws IOException { - checkAborted(); - return in.read(); + /** + * Loads information about shard snapshot + */ + private BlobStoreIndexShardSnapshot loadShardSnapshot(BlobContainer shardContainer, SnapshotId snapshotId) { + try { + return indexShardSnapshotFormat.read(shardContainer, snapshotId.getUUID()); + } catch (IOException ex) { + throw new SnapshotException(metadata.name(), snapshotId, + "failed to read shard snapshot file for [" + shardContainer.path() + ']', ex); + } + } + + /** + * Writes a new index file for the shard and removes all unreferenced files from the repository. + * + * We need to be really careful in handling index files in case of failures to make sure we don't + * have index file that points to files that were deleted. + * + * @param snapshots list of active snapshots in the container + * @param fileListGeneration the generation number of the current snapshot index file + * @param blobs list of blobs in the container + * @param reason a reason explaining why the shard index file is written + */ + private void finalizeShard(List snapshots, int fileListGeneration, Map blobs, + String reason, BlobContainer shardContainer, ShardId shardId, SnapshotId snapshotId) { + final String indexGeneration = Integer.toString(fileListGeneration + 1); + try { + final List blobsToDelete; + if (snapshots.isEmpty()) { + // If we deleted all snapshots, we don't need to create a new index file and simply delete all the blobs we found + blobsToDelete = List.copyOf(blobs.keySet()); + } else { + final BlobStoreIndexShardSnapshots updatedSnapshots = new BlobStoreIndexShardSnapshots(snapshots); + indexShardSnapshotsFormat.writeAtomic(updatedSnapshots, shardContainer, indexGeneration); + // Delete all previous index-N, data-blobs that are not referenced by the new index-N and temporary blobs + blobsToDelete = blobs.keySet().stream().filter(blob -> + blob.startsWith(SNAPSHOT_INDEX_PREFIX) + || blob.startsWith(DATA_BLOB_PREFIX) && updatedSnapshots.findNameFile(canonicalName(blob)) == null + || FsBlobContainer.isTempBlobName(blob)).collect(Collectors.toList()); + } + try { + shardContainer.deleteBlobsIgnoringIfNotExists(blobsToDelete); + } catch (IOException e) { + logger.warn(() -> new ParameterizedMessage("[{}][{}] failed to delete blobs during finalization", + snapshotId, shardId), e); } + } catch (IOException e) { + String message = + "Failed to finalize " + reason + " with shard index [" + indexShardSnapshotsFormat.blobName(indexGeneration) + "]"; + throw new IndexShardSnapshotFailedException(shardId, message, e); + } + } - @Override - public int read(byte[] b, int off, int len) throws IOException { - checkAborted(); - return in.read(b, off, len); + /** + * Loads all available snapshots in the repository + * + * @param blobs list of blobs in repository + * @return tuple of BlobStoreIndexShardSnapshots and the last snapshot index generation + */ + private Tuple buildBlobStoreIndexShardSnapshots(Map blobs, + BlobContainer shardContainer) { + int latest = -1; + Set blobKeys = blobs.keySet(); + for (String name : blobKeys) { + if (name.startsWith(SNAPSHOT_INDEX_PREFIX)) { + try { + int gen = Integer.parseInt(name.substring(SNAPSHOT_INDEX_PREFIX.length())); + if (gen > latest) { + latest = gen; + } + } catch (NumberFormatException ex) { + logger.warn("failed to parse index file name [{}]", name); + } } + } + if (latest >= 0) { + try { + final BlobStoreIndexShardSnapshots shardSnapshots = + indexShardSnapshotsFormat.read(shardContainer, Integer.toString(latest)); + return new Tuple<>(shardSnapshots, latest); + } catch (IOException e) { + final String file = SNAPSHOT_INDEX_PREFIX + latest; + logger.warn(() -> new ParameterizedMessage("failed to read index file [{}]", file), e); + } + } else if (blobKeys.isEmpty() == false) { + logger.warn("Could not find a readable index-N file in a non-empty shard snapshot directory [{}]", shardContainer.path()); + } - private void checkAborted() { - if (snapshotStatus.isAborted()) { - logger.debug("[{}] [{}] Aborted on the file [{}], exiting", shardId, snapshotId, fileName); - throw new IndexShardSnapshotFailedException(shardId, "Aborted"); + // We couldn't load the index file - falling back to loading individual snapshots + List snapshots = new ArrayList<>(); + for (String name : blobKeys) { + try { + BlobStoreIndexShardSnapshot snapshot = null; + if (name.startsWith(SNAPSHOT_PREFIX)) { + snapshot = indexShardSnapshotFormat.readBlob(shardContainer, name); + } + if (snapshot != null) { + snapshots.add(new SnapshotFiles(snapshot.snapshot(), snapshot.indexFiles())); } + } catch (IOException e) { + logger.warn(() -> new ParameterizedMessage("Failed to read blob [{}]", name), e); } } + return new Tuple<>(new BlobStoreIndexShardSnapshots(snapshots), latest); } - private static final class PartSliceStream extends SlicedInputStream { + /** + * Snapshot individual file + * @param fileInfo file to be snapshotted + */ + private void snapshotFile(BlobStoreIndexShardSnapshot.FileInfo fileInfo, IndexId indexId, ShardId shardId, SnapshotId snapshotId, + IndexShardSnapshotStatus snapshotStatus, Store store) throws IOException { + final BlobContainer shardContainer = shardContainer(indexId, shardId); + final String file = fileInfo.physicalName(); + try (IndexInput indexInput = store.openVerifyingInput(file, IOContext.READONCE, fileInfo.metadata())) { + for (int i = 0; i < fileInfo.numberOfParts(); i++) { + final long partBytes = fileInfo.partBytes(i); + + InputStream inputStream = new InputStreamIndexInput(indexInput, partBytes); + if (snapshotRateLimiter != null) { + inputStream = new RateLimitingInputStream(inputStream, snapshotRateLimiter, + snapshotRateLimitingTimeInNanos::inc); + } + // Make reads abortable by mutating the snapshotStatus object + inputStream = new FilterInputStream(inputStream) { + @Override + public int read() throws IOException { + checkAborted(); + return super.read(); + } - private final BlobContainer container; - private final BlobStoreIndexShardSnapshot.FileInfo info; + @Override + public int read(byte[] b, int off, int len) throws IOException { + checkAborted(); + return super.read(b, off, len); + } - PartSliceStream(BlobContainer container, BlobStoreIndexShardSnapshot.FileInfo info) { - super(info.numberOfParts()); - this.info = info; - this.container = container; + private void checkAborted() { + if (snapshotStatus.isAborted()) { + logger.debug("[{}] [{}] Aborted on the file [{}], exiting", shardId, + snapshotId, fileInfo.physicalName()); + throw new IndexShardSnapshotFailedException(shardId, "Aborted"); + } + } + }; + shardContainer.writeBlob(fileInfo.partName(i), inputStream, partBytes, true); + } + Store.verify(indexInput); + snapshotStatus.addProcessedFile(fileInfo.length()); + } catch (Exception t) { + failStoreIfCorrupted(store, t); + snapshotStatus.addProcessedFile(0); + throw t; } + } - @Override - protected InputStream openSlice(long slice) throws IOException { - return container.readBlob(info.partName(slice)); + private static void failStoreIfCorrupted(Store store, Exception e) { + if (Lucene.isCorruptionException(e)) { + try { + store.markStoreCorrupted((IOException) e); + } catch (IOException inner) { + inner.addSuppressed(e); + logger.warn("store cannot be marked as corrupted", inner); + } } } /** - * Context for restore operations + * Checks if snapshot file already exists in the list of blobs + * @param fileInfo file to check + * @param blobs list of blobs + * @return true if file exists in the list of blobs */ - private class RestoreContext extends FileRestoreContext { - - private final BlobContainer blobContainer; - - /** - * Constructs new restore context - * @param shardId shard id to restore into - * @param snapshotId snapshot id - * @param recoveryState recovery state to report progress - * @param blobContainer the blob container to read the files from - */ - RestoreContext(ShardId shardId, SnapshotId snapshotId, RecoveryState recoveryState, BlobContainer blobContainer) { - super(metadata.name(), shardId, snapshotId, recoveryState, BUFFER_SIZE); - this.blobContainer = blobContainer; - } - - @Override - protected InputStream fileInputStream(BlobStoreIndexShardSnapshot.FileInfo fileInfo) { - if (restoreRateLimiter == null) { - return new PartSliceStream(blobContainer, fileInfo); - } else { - RateLimitingInputStream.Listener listener = restoreRateLimitingTimeInNanos::inc; - return new RateLimitingInputStream(new PartSliceStream(blobContainer, fileInfo), restoreRateLimiter, listener); + private static boolean snapshotFileExistsInBlobs(BlobStoreIndexShardSnapshot.FileInfo fileInfo, Map blobs) { + BlobMetaData blobMetaData = blobs.get(fileInfo.name()); + if (blobMetaData != null) { + return blobMetaData.length() == fileInfo.length(); + } else if (blobs.containsKey(fileInfo.partName(0))) { + // multi part file sum up the size and check + int part = 0; + long totalSize = 0; + while (true) { + blobMetaData = blobs.get(fileInfo.partName(part++)); + if (blobMetaData == null) { + break; + } + totalSize += blobMetaData.length(); } + return totalSize == fileInfo.length(); } + // no file, not exact and not multipart + return false; } } diff --git a/server/src/main/java/org/elasticsearch/repositories/blobstore/ChecksumBlobStoreFormat.java b/server/src/main/java/org/elasticsearch/repositories/blobstore/ChecksumBlobStoreFormat.java index d216fe3234e83..0f059a89e6c8c 100644 --- a/server/src/main/java/org/elasticsearch/repositories/blobstore/ChecksumBlobStoreFormat.java +++ b/server/src/main/java/org/elasticsearch/repositories/blobstore/ChecksumBlobStoreFormat.java @@ -30,6 +30,7 @@ import org.elasticsearch.common.bytes.BytesArray; import org.elasticsearch.common.bytes.BytesReference; import org.elasticsearch.common.compress.CompressorFactory; +import org.elasticsearch.common.io.Streams; import org.elasticsearch.common.io.stream.BytesStreamOutput; import org.elasticsearch.common.io.stream.StreamOutput; import org.elasticsearch.common.lucene.store.ByteArrayIndexInput; @@ -42,7 +43,6 @@ import org.elasticsearch.common.xcontent.XContentHelper; import org.elasticsearch.common.xcontent.XContentParser; import org.elasticsearch.common.xcontent.XContentType; -import org.elasticsearch.core.internal.io.Streams; import org.elasticsearch.gateway.CorruptStateException; import org.elasticsearch.snapshots.SnapshotInfo; @@ -149,24 +149,21 @@ public String blobName(String name) { * @param blobName blob name */ public T readBlob(BlobContainer blobContainer, String blobName) throws IOException { - try (InputStream inputStream = blobContainer.readBlob(blobName)) { - ByteArrayOutputStream out = new ByteArrayOutputStream(); - Streams.copy(inputStream, out); - final byte[] bytes = out.toByteArray(); - final String resourceDesc = "ChecksumBlobStoreFormat.readBlob(blob=\"" + blobName + "\")"; - try (ByteArrayIndexInput indexInput = new ByteArrayIndexInput(resourceDesc, bytes)) { - CodecUtil.checksumEntireFile(indexInput); - CodecUtil.checkHeader(indexInput, codec, VERSION, VERSION); - long filePointer = indexInput.getFilePointer(); - long contentSize = indexInput.length() - CodecUtil.footerLength() - filePointer; - try (XContentParser parser = XContentHelper.createParser(namedXContentRegistry, LoggingDeprecationHandler.INSTANCE, - new BytesArray(bytes, (int) filePointer, (int) contentSize))) { - return reader.apply(parser); - } - } catch (CorruptIndexException | IndexFormatTooOldException | IndexFormatTooNewException ex) { - // we trick this into a dedicated exception with the original stacktrace - throw new CorruptStateException(ex); + final BytesReference bytes = Streams.readFully(blobContainer.readBlob(blobName)); + final String resourceDesc = "ChecksumBlobStoreFormat.readBlob(blob=\"" + blobName + "\")"; + try (ByteArrayIndexInput indexInput = + new ByteArrayIndexInput(resourceDesc, BytesReference.toBytes(bytes))) { + CodecUtil.checksumEntireFile(indexInput); + CodecUtil.checkHeader(indexInput, codec, VERSION, VERSION); + long filePointer = indexInput.getFilePointer(); + long contentSize = indexInput.length() - CodecUtil.footerLength() - filePointer; + try (XContentParser parser = XContentHelper.createParser(namedXContentRegistry, LoggingDeprecationHandler.INSTANCE, + bytes.slice((int) filePointer, (int) contentSize))) { + return reader.apply(parser); } + } catch (CorruptIndexException | IndexFormatTooOldException | IndexFormatTooNewException ex) { + // we trick this into a dedicated exception with the original stacktrace + throw new CorruptStateException(ex); } } diff --git a/server/src/main/java/org/elasticsearch/repositories/blobstore/package-info.java b/server/src/main/java/org/elasticsearch/repositories/blobstore/package-info.java index 9d6d72f0458c9..0a66df3cf8521 100644 --- a/server/src/main/java/org/elasticsearch/repositories/blobstore/package-info.java +++ b/server/src/main/java/org/elasticsearch/repositories/blobstore/package-info.java @@ -20,7 +20,7 @@ /** *

    This package exposes the blobstore repository used by Elasticsearch Snapshots.

    * - *

    Preliminaries

    + *

    Preliminaries

    * *

    The {@link org.elasticsearch.repositories.blobstore.BlobStoreRepository} forms the basis of implementations of * {@link org.elasticsearch.repositories.Repository} on top of a blob store. A blobstore can be used as the basis for an implementation @@ -84,7 +84,7 @@ * } *

    * - *

    Getting the Repository's RepositoryData

    + *

    Getting the Repository's RepositoryData

    * *

    Loading the {@link org.elasticsearch.repositories.RepositoryData} that holds the list of all snapshots as well as the mapping of * indices' names to their repository {@link org.elasticsearch.repositories.IndexId} is done by invoking @@ -103,7 +103,7 @@ *

  • First, find the most recent {@code RepositoryData} by getting a list of all index-N blobs through listing all blobs with prefix * "index-" under the repository root and then selecting the one with the highest value for N.
  • *
  • If this operation fails because the repository's {@code BlobContainer} does not support list operations (in the case of read-only - * repositories), read the highest value of N from the the index.latest blob.
  • + * repositories), read the highest value of N from the index.latest blob. * * *
  • @@ -114,11 +114,11 @@ * *
  • * - *

    Creating a Snapshot

    + *

    Creating a Snapshot

    * *

    Creating a snapshot in the repository happens in the three steps described in detail below.

    * - *

    Initializing a Snapshot in the Repository

    + *

    Initializing a Snapshot in the Repository

    * *

    Creating a snapshot in the repository starts with a call to {@link org.elasticsearch.repositories.Repository#initializeSnapshot} which * the blob store repository implements via the following actions:

    @@ -130,7 +130,7 @@ * * TODO: This behavior is problematic, adjust these docs once https://github.com/elastic/elasticsearch/issues/41581 is fixed * - *

    Writing Shard Data (Segments)

    + *

    Writing Shard Data (Segments)

    * *

    Once all the metadata has been written by the snapshot initialization, the snapshot process moves on to writing the actual shard data * to the repository by invoking {@link org.elasticsearch.repositories.Repository#snapshotShard} on the data-nodes that hold the primaries @@ -158,7 +158,7 @@ * {@code BlobStoreIndexShardSnapshots} blob is written to the shard's path with name {@code index-${N+1}}. * * - *

    Finalizing the Snapshot

    + *

    Finalizing the Snapshot

    * *

    After all primaries have finished writing the necessary segment files to the blob store in the previous step, the master node moves on * to finalizing the snapshot by invoking {@link org.elasticsearch.repositories.Repository#finalizeSnapshot}. This method executes the @@ -173,7 +173,7 @@ *

  • Write the updated {@code /index.latest} blob containing the new repository generation {@code N + 1}.
  • * * - *

    Deleting a Snapshot

    + *

    Deleting a Snapshot

    * *

    Deleting a snapshot is an operation that is exclusively executed on the master node that runs through the following sequence of * action when {@link org.elasticsearch.repositories.blobstore.BlobStoreRepository#deleteSnapshot} is invoked:

    diff --git a/server/src/main/java/org/elasticsearch/repositories/fs/FsRepository.java b/server/src/main/java/org/elasticsearch/repositories/fs/FsRepository.java index 8f495f2d4842a..f338e0ee4cb08 100644 --- a/server/src/main/java/org/elasticsearch/repositories/fs/FsRepository.java +++ b/server/src/main/java/org/elasticsearch/repositories/fs/FsRepository.java @@ -66,14 +66,12 @@ public class FsRepository extends BlobStoreRepository { private final ByteSizeValue chunkSize; - private final BlobPath basePath; - /** * Constructs a shared file system repository. */ public FsRepository(RepositoryMetaData metadata, Environment environment, NamedXContentRegistry namedXContentRegistry, ThreadPool threadPool) { - super(metadata, environment.settings(), namedXContentRegistry, threadPool); + super(metadata, environment.settings(), namedXContentRegistry, threadPool, BlobPath.cleanPath()); this.environment = environment; String location = REPOSITORIES_LOCATION_SETTING.get(metadata.settings()); if (location.isEmpty()) { @@ -101,7 +99,6 @@ public FsRepository(RepositoryMetaData metadata, Environment environment, NamedX } else { this.chunkSize = REPOSITORIES_CHUNK_SIZE_SETTING.get(environment.settings()); } - this.basePath = BlobPath.cleanPath(); } @Override @@ -115,9 +112,4 @@ protected BlobStore createBlobStore() throws Exception { protected ByteSizeValue chunkSize() { return chunkSize; } - - @Override - protected BlobPath basePath() { - return basePath; - } } diff --git a/server/src/main/java/org/elasticsearch/rest/RestController.java b/server/src/main/java/org/elasticsearch/rest/RestController.java index a55005454f629..3f32d281918a3 100644 --- a/server/src/main/java/org/elasticsearch/rest/RestController.java +++ b/server/src/main/java/org/elasticsearch/rest/RestController.java @@ -53,13 +53,12 @@ import java.util.concurrent.atomic.AtomicBoolean; import java.util.function.UnaryOperator; +import static org.elasticsearch.rest.BytesRestResponse.TEXT_CONTENT_TYPE; import static org.elasticsearch.rest.RestStatus.BAD_REQUEST; -import static org.elasticsearch.rest.RestStatus.METHOD_NOT_ALLOWED; -import static org.elasticsearch.rest.RestStatus.FORBIDDEN; import static org.elasticsearch.rest.RestStatus.INTERNAL_SERVER_ERROR; +import static org.elasticsearch.rest.RestStatus.METHOD_NOT_ALLOWED; import static org.elasticsearch.rest.RestStatus.NOT_ACCEPTABLE; import static org.elasticsearch.rest.RestStatus.OK; -import static org.elasticsearch.rest.BytesRestResponse.TEXT_CONTENT_TYPE; public class RestController implements HttpServerTransport.Dispatcher { @@ -253,7 +252,7 @@ boolean dispatchRequest(final RestRequest request, final RestChannel channel, fi // If an alternative handler for an explicit path is registered to a // different HTTP method than the one supplied - return a 405 Method // Not Allowed error. - handleUnsupportedHttpMethod(request, channel, validMethodSet); + handleUnsupportedHttpMethod(request, channel, validMethodSet, null); requestHandled = true; } else if (validMethodSet.contains(request.method()) == false && (request.method() == RestRequest.Method.OPTIONS)) { @@ -330,16 +329,28 @@ void tryAllHandlers(final RestRequest request, final RestChannel channel, final return; } - // Loop through all possible handlers, attempting to dispatch the request - Iterator allHandlers = getAllHandlers(request); - for (Iterator it = allHandlers; it.hasNext(); ) { - final Optional mHandler = Optional.ofNullable(it.next()).flatMap(mh -> mh.getHandler(request.method())); - requestHandled = dispatchRequest(request, channel, client, mHandler); - if (requestHandled) { - break; + try { + // Resolves the HTTP method and fails if the method is invalid + final RestRequest.Method requestMethod = request.method(); + + // Loop through all possible handlers, attempting to dispatch the request + Iterator allHandlers = getAllHandlers(request); + for (Iterator it = allHandlers; it.hasNext(); ) { + Optional mHandler = Optional.empty(); + if (requestMethod != null) { + mHandler = Optional.ofNullable(it.next()).flatMap(mh -> mh.getHandler(requestMethod)); + } + requestHandled = dispatchRequest(request, channel, client, mHandler); + if (requestHandled) { + break; + } } + } catch (final IllegalArgumentException e) { + handleUnsupportedHttpMethod(request, channel, getValidHandlerMethodSet(request), e); + requestHandled = true; } + // If request has not been handled, fallback to a bad request error. if (requestHandled == false) { handleBadRequest(request, channel); @@ -365,11 +376,25 @@ Iterator getAllHandlers(final RestRequest request) { * HTTP/1.1 - * 10.4.6 - 405 Method Not Allowed). */ - private void handleUnsupportedHttpMethod(RestRequest request, RestChannel channel, Set validMethodSet) { + private void handleUnsupportedHttpMethod(final RestRequest request, + final RestChannel channel, + final Set validMethodSet, + @Nullable final IllegalArgumentException exception) { try { - BytesRestResponse bytesRestResponse = BytesRestResponse.createSimpleErrorResponse(channel, METHOD_NOT_ALLOWED, - "Incorrect HTTP method for uri [" + request.uri() + "] and method [" + request.method() + "], allowed: " + validMethodSet); - bytesRestResponse.addHeader("Allow", Strings.collectionToDelimitedString(validMethodSet, ",")); + final StringBuilder msg = new StringBuilder(); + if (exception != null) { + msg.append(exception.getMessage()); + } else { + msg.append("Incorrect HTTP method for uri [").append(request.uri()); + msg.append("] and method [").append(request.method()).append("]"); + } + if (validMethodSet.isEmpty() == false) { + msg.append(", allowed: ").append(validMethodSet); + } + BytesRestResponse bytesRestResponse = BytesRestResponse.createSimpleErrorResponse(channel, METHOD_NOT_ALLOWED, msg.toString()); + if (validMethodSet.isEmpty() == false) { + bytesRestResponse.addHeader("Allow", Strings.collectionToDelimitedString(validMethodSet, ",")); + } channel.sendResponse(bytesRestResponse); } catch (final IOException e) { logger.warn("failed to send bad request response", e); @@ -385,11 +410,12 @@ private void handleUnsupportedHttpMethod(RestRequest request, RestChannel channe * - Options). */ private void handleOptionsRequest(RestRequest request, RestChannel channel, Set validMethodSet) { - if (request.method() == RestRequest.Method.OPTIONS && validMethodSet.size() > 0) { + assert request.method() == RestRequest.Method.OPTIONS; + if (validMethodSet.isEmpty() == false) { BytesRestResponse bytesRestResponse = new BytesRestResponse(OK, TEXT_CONTENT_TYPE, BytesArray.EMPTY); bytesRestResponse.addHeader("Allow", Strings.collectionToDelimitedString(validMethodSet, ",")); channel.sendResponse(bytesRestResponse); - } else if (request.method() == RestRequest.Method.OPTIONS && validMethodSet.size() == 0) { + } else { /* * When we have an OPTIONS HTTP request and no valid handlers, * simply send OK by default (with the Access Control Origin header @@ -433,20 +459,25 @@ private String getPath(RestRequest request) { return request.rawPath(); } - void handleFavicon(RestRequest request, RestChannel channel) { - if (request.method() == RestRequest.Method.GET) { - try { - try (InputStream stream = getClass().getResourceAsStream("/config/favicon.ico")) { - ByteArrayOutputStream out = new ByteArrayOutputStream(); - Streams.copy(stream, out); - BytesRestResponse restResponse = new BytesRestResponse(RestStatus.OK, "image/x-icon", out.toByteArray()); - channel.sendResponse(restResponse); + private void handleFavicon(final RestRequest request, final RestChannel channel) { + try { + if (request.method() != RestRequest.Method.GET) { + handleUnsupportedHttpMethod(request, channel, Set.of(RestRequest.Method.GET), null); + } else { + try { + try (InputStream stream = getClass().getResourceAsStream("/config/favicon.ico")) { + ByteArrayOutputStream out = new ByteArrayOutputStream(); + Streams.copy(stream, out); + BytesRestResponse restResponse = new BytesRestResponse(RestStatus.OK, "image/x-icon", out.toByteArray()); + channel.sendResponse(restResponse); + } + } catch (IOException e) { + channel.sendResponse( + new BytesRestResponse(INTERNAL_SERVER_ERROR, BytesRestResponse.TEXT_CONTENT_TYPE, BytesArray.EMPTY)); } - } catch (IOException e) { - channel.sendResponse(new BytesRestResponse(INTERNAL_SERVER_ERROR, BytesRestResponse.TEXT_CONTENT_TYPE, BytesArray.EMPTY)); } - } else { - channel.sendResponse(new BytesRestResponse(FORBIDDEN, BytesRestResponse.TEXT_CONTENT_TYPE, BytesArray.EMPTY)); + } catch (final IllegalArgumentException e) { + handleUnsupportedHttpMethod(request, channel, Set.of(RestRequest.Method.GET), e); } } @@ -512,5 +543,4 @@ private static CircuitBreaker inFlightRequestsBreaker(CircuitBreakerService circ // We always obtain a fresh breaker to reflect changes to the breaker configuration. return circuitBreakerService.getBreaker(CircuitBreaker.IN_FLIGHT_REQUESTS); } - } diff --git a/server/src/main/java/org/elasticsearch/rest/RestRequest.java b/server/src/main/java/org/elasticsearch/rest/RestRequest.java index fe976ee4ddce6..4fd8515caba03 100644 --- a/server/src/main/java/org/elasticsearch/rest/RestRequest.java +++ b/server/src/main/java/org/elasticsearch/rest/RestRequest.java @@ -153,6 +153,12 @@ public enum Method { GET, POST, PUT, DELETE, OPTIONS, HEAD, PATCH, TRACE, CONNECT } + /** + * Returns the HTTP method used in the REST request. + * + * @return the {@link Method} used in the REST request + * @throws IllegalArgumentException if the HTTP method is invalid + */ public Method method() { return httpRequest.method(); } diff --git a/server/src/main/java/org/elasticsearch/rest/action/admin/cluster/RestGetSnapshotsAction.java b/server/src/main/java/org/elasticsearch/rest/action/admin/cluster/RestGetSnapshotsAction.java index f42180b5029b8..1346757520dfb 100644 --- a/server/src/main/java/org/elasticsearch/rest/action/admin/cluster/RestGetSnapshotsAction.java +++ b/server/src/main/java/org/elasticsearch/rest/action/admin/cluster/RestGetSnapshotsAction.java @@ -49,10 +49,10 @@ public String getName() { @Override public RestChannelConsumer prepareRequest(final RestRequest request, final NodeClient client) throws IOException { - String repository = request.param("repository"); + String[] repositories = request.paramAsStringArray("repository", Strings.EMPTY_ARRAY); String[] snapshots = request.paramAsStringArray("snapshot", Strings.EMPTY_ARRAY); - GetSnapshotsRequest getSnapshotsRequest = getSnapshotsRequest(repository).snapshots(snapshots); + GetSnapshotsRequest getSnapshotsRequest = getSnapshotsRequest(repositories).snapshots(snapshots); getSnapshotsRequest.ignoreUnavailable(request.paramAsBoolean("ignore_unavailable", getSnapshotsRequest.ignoreUnavailable())); getSnapshotsRequest.verbose(request.paramAsBoolean("verbose", getSnapshotsRequest.verbose())); getSnapshotsRequest.masterNodeTimeout(request.paramAsTime("master_timeout", getSnapshotsRequest.masterNodeTimeout())); diff --git a/server/src/main/java/org/elasticsearch/rest/action/admin/indices/RestAnalyzeAction.java b/server/src/main/java/org/elasticsearch/rest/action/admin/indices/RestAnalyzeAction.java index d9d6bbcfee98d..99c8598106843 100644 --- a/server/src/main/java/org/elasticsearch/rest/action/admin/indices/RestAnalyzeAction.java +++ b/server/src/main/java/org/elasticsearch/rest/action/admin/indices/RestAnalyzeAction.java @@ -18,7 +18,7 @@ */ package org.elasticsearch.rest.action.admin.indices; -import org.elasticsearch.action.admin.indices.analyze.AnalyzeRequest; +import org.elasticsearch.action.admin.indices.analyze.AnalyzeAction; import org.elasticsearch.client.node.NodeClient; import org.elasticsearch.common.ParseField; import org.elasticsearch.common.settings.Settings; @@ -29,8 +29,6 @@ import org.elasticsearch.rest.action.RestToXContentListener; import java.io.IOException; -import java.util.ArrayList; -import java.util.List; import static org.elasticsearch.rest.RestRequest.Method.GET; import static org.elasticsearch.rest.RestRequest.Method.POST; @@ -64,106 +62,10 @@ public String getName() { @Override public RestChannelConsumer prepareRequest(final RestRequest request, final NodeClient client) throws IOException { - - AnalyzeRequest analyzeRequest = new AnalyzeRequest(request.param("index")); - try (XContentParser parser = request.contentOrSourceParamParser()) { - buildFromContent(parser, analyzeRequest); - } catch (IOException e) { - throw new IllegalArgumentException("Failed to parse request body", e); + AnalyzeAction.Request analyzeRequest = AnalyzeAction.Request.fromXContent(parser, request.param("index")); + return channel -> client.admin().indices().analyze(analyzeRequest, new RestToXContentListener<>(channel)); } - - return channel -> client.admin().indices().analyze(analyzeRequest, new RestToXContentListener<>(channel)); } - static void buildFromContent(XContentParser parser, AnalyzeRequest analyzeRequest) - throws IOException { - if (parser.nextToken() != XContentParser.Token.START_OBJECT) { - throw new IllegalArgumentException("Malformed content, must start with an object"); - } else { - XContentParser.Token token; - String currentFieldName = null; - while ((token = parser.nextToken()) != XContentParser.Token.END_OBJECT) { - if (token == XContentParser.Token.FIELD_NAME) { - currentFieldName = parser.currentName(); - } else if (Fields.TEXT.match(currentFieldName, parser.getDeprecationHandler()) && - token == XContentParser.Token.VALUE_STRING) { - analyzeRequest.text(parser.text()); - } else if (Fields.TEXT.match(currentFieldName, parser.getDeprecationHandler()) && - token == XContentParser.Token.START_ARRAY) { - List texts = new ArrayList<>(); - while ((token = parser.nextToken()) != XContentParser.Token.END_ARRAY) { - if (token.isValue() == false) { - throw new IllegalArgumentException(currentFieldName + " array element should only contain text"); - } - texts.add(parser.text()); - } - analyzeRequest.text(texts.toArray(new String[texts.size()])); - } else if (Fields.ANALYZER.match(currentFieldName, parser.getDeprecationHandler()) - && token == XContentParser.Token.VALUE_STRING) { - analyzeRequest.analyzer(parser.text()); - } else if (Fields.FIELD.match(currentFieldName, parser.getDeprecationHandler()) && - token == XContentParser.Token.VALUE_STRING) { - analyzeRequest.field(parser.text()); - } else if (Fields.TOKENIZER.match(currentFieldName, parser.getDeprecationHandler())) { - if (token == XContentParser.Token.VALUE_STRING) { - analyzeRequest.tokenizer(parser.text()); - } else if (token == XContentParser.Token.START_OBJECT) { - analyzeRequest.tokenizer(parser.map()); - } else { - throw new IllegalArgumentException(currentFieldName + " should be tokenizer's name or setting"); - } - } else if (Fields.TOKEN_FILTERS.match(currentFieldName, parser.getDeprecationHandler()) - && token == XContentParser.Token.START_ARRAY) { - while ((token = parser.nextToken()) != XContentParser.Token.END_ARRAY) { - if (token == XContentParser.Token.VALUE_STRING) { - analyzeRequest.addTokenFilter(parser.text()); - } else if (token == XContentParser.Token.START_OBJECT) { - analyzeRequest.addTokenFilter(parser.map()); - } else { - throw new IllegalArgumentException(currentFieldName - + " array element should contain filter's name or setting"); - } - } - } else if (Fields.CHAR_FILTERS.match(currentFieldName, parser.getDeprecationHandler()) - && token == XContentParser.Token.START_ARRAY) { - while ((token = parser.nextToken()) != XContentParser.Token.END_ARRAY) { - if (token == XContentParser.Token.VALUE_STRING) { - analyzeRequest.addCharFilter(parser.text()); - } else if (token == XContentParser.Token.START_OBJECT) { - analyzeRequest.addCharFilter(parser.map()); - } else { - throw new IllegalArgumentException(currentFieldName - + " array element should contain char filter's name or setting"); - } - } - } else if (Fields.EXPLAIN.match(currentFieldName, parser.getDeprecationHandler())) { - if (parser.isBooleanValue()) { - analyzeRequest.explain(parser.booleanValue()); - } else { - throw new IllegalArgumentException(currentFieldName + " must be either 'true' or 'false'"); - } - } else if (Fields.ATTRIBUTES.match(currentFieldName, parser.getDeprecationHandler()) && - token == XContentParser.Token.START_ARRAY) { - List attributes = new ArrayList<>(); - while ((token = parser.nextToken()) != XContentParser.Token.END_ARRAY) { - if (token.isValue() == false) { - throw new IllegalArgumentException(currentFieldName + " array element should only contain attribute name"); - } - attributes.add(parser.text()); - } - analyzeRequest.attributes(attributes.toArray(new String[attributes.size()])); - } else if (Fields.NORMALIZER.match(currentFieldName, parser.getDeprecationHandler())) { - if (token == XContentParser.Token.VALUE_STRING) { - analyzeRequest.normalizer(parser.text()); - } else { - throw new IllegalArgumentException(currentFieldName + " should be normalizer's name"); - } - } else { - throw new IllegalArgumentException("Unknown parameter [" - + currentFieldName + "] in request body or parameter is of the wrong type[" + token + "] "); - } - } - } - } } diff --git a/server/src/main/java/org/elasticsearch/rest/action/cat/RestIndicesAction.java b/server/src/main/java/org/elasticsearch/rest/action/cat/RestIndicesAction.java index 788de8e715480..8c7d34c1ca1ff 100644 --- a/server/src/main/java/org/elasticsearch/rest/action/cat/RestIndicesAction.java +++ b/server/src/main/java/org/elasticsearch/rest/action/cat/RestIndicesAction.java @@ -19,54 +19,58 @@ package org.elasticsearch.rest.action.cat; +import org.elasticsearch.action.ActionListener; +import org.elasticsearch.action.ActionResponse; import org.elasticsearch.action.admin.cluster.health.ClusterHealthRequest; import org.elasticsearch.action.admin.cluster.health.ClusterHealthResponse; import org.elasticsearch.action.admin.cluster.state.ClusterStateRequest; import org.elasticsearch.action.admin.cluster.state.ClusterStateResponse; +import org.elasticsearch.action.admin.indices.settings.get.GetSettingsRequest; +import org.elasticsearch.action.admin.indices.settings.get.GetSettingsResponse; import org.elasticsearch.action.admin.indices.stats.CommonStats; import org.elasticsearch.action.admin.indices.stats.IndexStats; import org.elasticsearch.action.admin.indices.stats.IndicesStatsRequest; import org.elasticsearch.action.admin.indices.stats.IndicesStatsResponse; +import org.elasticsearch.action.support.GroupedActionListener; import org.elasticsearch.action.support.IndicesOptions; -import org.elasticsearch.client.Requests; import org.elasticsearch.client.node.NodeClient; -import org.elasticsearch.cluster.ClusterState; import org.elasticsearch.cluster.health.ClusterHealthStatus; import org.elasticsearch.cluster.health.ClusterIndexHealth; import org.elasticsearch.cluster.metadata.IndexMetaData; -import org.elasticsearch.cluster.metadata.IndexNameExpressionResolver; import org.elasticsearch.common.Strings; import org.elasticsearch.common.Table; -import org.elasticsearch.common.collect.ImmutableOpenMap; import org.elasticsearch.common.settings.Settings; import org.elasticsearch.common.time.DateFormatter; -import org.elasticsearch.index.Index; +import org.elasticsearch.common.unit.TimeValue; import org.elasticsearch.index.IndexSettings; import org.elasticsearch.rest.RestController; import org.elasticsearch.rest.RestRequest; import org.elasticsearch.rest.RestResponse; -import org.elasticsearch.rest.action.RestActionListener; import org.elasticsearch.rest.action.RestResponseListener; import java.time.Instant; import java.time.ZoneOffset; import java.time.ZonedDateTime; import java.util.Arrays; +import java.util.Collection; import java.util.Collections; import java.util.HashSet; import java.util.Locale; +import java.util.Map; import java.util.Set; +import java.util.function.Function; +import java.util.stream.Collectors; +import java.util.stream.StreamSupport; +import static org.elasticsearch.action.support.master.MasterNodeRequest.DEFAULT_MASTER_NODE_TIMEOUT; import static org.elasticsearch.rest.RestRequest.Method.GET; public class RestIndicesAction extends AbstractCatAction { private static final DateFormatter STRICT_DATE_TIME_FORMATTER = DateFormatter.forPattern("strict_date_time"); - private final IndexNameExpressionResolver indexNameExpressionResolver; - public RestIndicesAction(Settings settings, RestController controller, IndexNameExpressionResolver indexNameExpressionResolver) { + public RestIndicesAction(Settings settings, RestController controller) { super(settings); - this.indexNameExpressionResolver = indexNameExpressionResolver; controller.registerHandler(GET, "/_cat/indices", this); controller.registerHandler(GET, "/_cat/indices/{index}", this); } @@ -85,50 +89,147 @@ protected void documentation(StringBuilder sb) { @Override public RestChannelConsumer doCatRequest(final RestRequest request, final NodeClient client) { final String[] indices = Strings.splitStringByCommaToArray(request.param("index")); - final ClusterStateRequest clusterStateRequest = new ClusterStateRequest(); - clusterStateRequest.clear().indices(indices).metaData(true); - clusterStateRequest.local(request.paramAsBoolean("local", clusterStateRequest.local())); - clusterStateRequest.masterNodeTimeout(request.paramAsTime("master_timeout", clusterStateRequest.masterNodeTimeout())); - final IndicesOptions strictExpandIndicesOptions = IndicesOptions.strictExpand(); - clusterStateRequest.indicesOptions(strictExpandIndicesOptions); - - return channel -> client.admin().cluster().state(clusterStateRequest, new RestActionListener(channel) { + final IndicesOptions indicesOptions = IndicesOptions.strictExpand(); + final boolean local = request.paramAsBoolean("local", false); + final TimeValue masterNodeTimeout = request.paramAsTime("master_timeout", DEFAULT_MASTER_NODE_TIMEOUT); + final boolean includeUnloadedSegments = request.paramAsBoolean("include_unloaded_segments", false); + + return channel -> { + final ActionListener listener = ActionListener.notifyOnce(new RestResponseListener<>(channel) { + @Override + public RestResponse buildResponse(final Table table) throws Exception { + return RestTable.buildResponse(table, channel); + } + }); + + sendGetSettingsRequest(indices, indicesOptions, local, masterNodeTimeout, client, new ActionListener<>() { + @Override + public void onResponse(final GetSettingsResponse getSettingsResponse) { + final GroupedActionListener groupedListener = createGroupedListener(request, 4, listener); + groupedListener.onResponse(getSettingsResponse); + + // Indices that were successfully resolved during the get settings request might be deleted when the subsequent cluster + // state, cluster health and indices stats requests execute. We have to distinguish two cases: + // 1) the deleted index was explicitly passed as parameter to the /_cat/indices request. In this case we want the + // subsequent requests to fail. + // 2) the deleted index was resolved as part of a wildcard or _all. In this case, we want the subsequent requests not to + // fail on the deleted index (as we want to ignore wildcards that cannot be resolved). + // This behavior can be ensured by letting the cluster state, cluster health and indices stats requests re-resolve the + // index names with the same indices options that we used for the initial cluster state request (strictExpand). + sendIndicesStatsRequest(indices, indicesOptions, includeUnloadedSegments, client, + ActionListener.wrap(groupedListener::onResponse, groupedListener::onFailure)); + sendClusterStateRequest(indices, indicesOptions, local, masterNodeTimeout, client, + ActionListener.wrap(groupedListener::onResponse, groupedListener::onFailure)); + sendClusterHealthRequest(indices, indicesOptions, local, masterNodeTimeout, client, + ActionListener.wrap(groupedListener::onResponse, groupedListener::onFailure)); + } + + @Override + public void onFailure(final Exception e) { + listener.onFailure(e); + } + }); + }; + } + + /** + * We're using the Get Settings API here to resolve the authorized indices for the user. + * This is because the Cluster State and Cluster Health APIs do not filter output based + * on index privileges, so they can't be used to determine which indices are authorized + * or not. On top of this, the Indices Stats API cannot be used either to resolve indices + * as it does not provide information for all existing indices (for example recovering + * indices or non replicated closed indices are not reported in indices stats response). + */ + private void sendGetSettingsRequest(final String[] indices, + final IndicesOptions indicesOptions, + final boolean local, + final TimeValue masterNodeTimeout, + final NodeClient client, + final ActionListener listener) { + final GetSettingsRequest request = new GetSettingsRequest(); + request.indices(indices); + request.indicesOptions(indicesOptions); + request.local(local); + request.masterNodeTimeout(masterNodeTimeout); + request.names(IndexSettings.INDEX_SEARCH_THROTTLED.getKey()); + + client.admin().indices().getSettings(request, listener); + } + + private void sendClusterStateRequest(final String[] indices, + final IndicesOptions indicesOptions, + final boolean local, + final TimeValue masterNodeTimeout, + final NodeClient client, + final ActionListener listener) { + + final ClusterStateRequest request = new ClusterStateRequest(); + request.indices(indices); + request.indicesOptions(indicesOptions); + request.local(local); + request.masterNodeTimeout(masterNodeTimeout); + + client.admin().cluster().state(request, listener); + } + + private void sendClusterHealthRequest(final String[] indices, + final IndicesOptions indicesOptions, + final boolean local, + final TimeValue masterNodeTimeout, + final NodeClient client, + final ActionListener listener) { + + final ClusterHealthRequest request = new ClusterHealthRequest(); + request.indices(indices); + request.indicesOptions(indicesOptions); + request.local(local); + request.masterNodeTimeout(masterNodeTimeout); + + client.admin().cluster().health(request, listener); + } + + private void sendIndicesStatsRequest(final String[] indices, + final IndicesOptions indicesOptions, + final boolean includeUnloadedSegments, + final NodeClient client, + final ActionListener listener) { + + final IndicesStatsRequest request = new IndicesStatsRequest(); + request.indices(indices); + request.indicesOptions(indicesOptions); + request.all(); + request.includeUnloadedSegments(includeUnloadedSegments); + + client.admin().indices().stats(request, listener); + } + + private GroupedActionListener createGroupedListener(final RestRequest request, final int size, + final ActionListener
    listener) { + return new GroupedActionListener<>(new ActionListener<>() { @Override - public void processResponse(final ClusterStateResponse clusterStateResponse) { - final ClusterState clusterState = clusterStateResponse.getState(); - final IndexMetaData[] indicesMetaData = getOrderedIndexMetaData(indices, clusterState, strictExpandIndicesOptions); - // Indices that were successfully resolved during the cluster state request might be deleted when the subsequent cluster - // health and indices stats requests execute. We have to distinguish two cases: - // 1) the deleted index was explicitly passed as parameter to the /_cat/indices request. In this case we want the subsequent - // requests to fail. - // 2) the deleted index was resolved as part of a wildcard or _all. In this case, we want the subsequent requests not to - // fail on the deleted index (as we want to ignore wildcards that cannot be resolved). - // This behavior can be ensured by letting the cluster health and indices stats requests re-resolve the index names with the - // same indices options that we used for the initial cluster state request (strictExpand). - final ClusterHealthRequest clusterHealthRequest = Requests.clusterHealthRequest(indices); - clusterHealthRequest.indicesOptions(strictExpandIndicesOptions); - clusterHealthRequest.local(request.paramAsBoolean("local", clusterHealthRequest.local())); - - client.admin().cluster().health(clusterHealthRequest, new RestActionListener(channel) { - @Override - public void processResponse(final ClusterHealthResponse clusterHealthResponse) { - final IndicesStatsRequest indicesStatsRequest = new IndicesStatsRequest(); - indicesStatsRequest.indices(indices); - indicesStatsRequest.indicesOptions(strictExpandIndicesOptions); - indicesStatsRequest.all(); - indicesStatsRequest.includeUnloadedSegments(request.paramAsBoolean("include_unloaded_segments", false)); - - client.admin().indices().stats(indicesStatsRequest, new RestResponseListener(channel) { - @Override - public RestResponse buildResponse(IndicesStatsResponse indicesStatsResponse) throws Exception { - final Table tab = buildTable(request, indicesMetaData, clusterHealthResponse, indicesStatsResponse); - return RestTable.buildResponse(tab, channel); - } - }); - } - }); + public void onResponse(final Collection responses) { + GetSettingsResponse settingsResponse = extractResponse(responses, GetSettingsResponse.class); + Map indicesSettings = StreamSupport.stream(settingsResponse.getIndexToSettings().spliterator(), false) + .collect(Collectors.toMap(cursor -> cursor.key, cursor -> cursor.value)); + + ClusterStateResponse stateResponse = extractResponse(responses, ClusterStateResponse.class); + Map indicesStates = StreamSupport.stream(stateResponse.getState().getMetaData().spliterator(), false) + .collect(Collectors.toMap(indexMetaData -> indexMetaData.getIndex().getName(), Function.identity())); + + ClusterHealthResponse healthResponse = extractResponse(responses, ClusterHealthResponse.class); + Map indicesHealths = healthResponse.getIndices(); + + IndicesStatsResponse statsResponse = extractResponse(responses, IndicesStatsResponse.class); + Map indicesStats = statsResponse.getIndices(); + + listener.onResponse(buildTable(request, indicesSettings, indicesHealths, indicesStats, indicesStates)); } - }); + + @Override + public void onFailure(final Exception e) { + listener.onFailure(e); + } + }, size); } private static final Set RESPONSE_PARAMS; @@ -393,18 +494,35 @@ protected Table getTableWithHeader(final RestRequest request) { // package private for testing Table buildTable(final RestRequest request, - final IndexMetaData[] indicesMetaData, - final ClusterHealthResponse clusterHealthResponse, - final IndicesStatsResponse indicesStatsResponse) { - final String healthParam = request.param("health"); + final Map indicesSettings, + final Map indicesHealths, + final Map indicesStats, + final Map indicesMetaDatas) { + final String healthParam = request.param("health"); final Table table = getTableWithHeader(request); - for (IndexMetaData indexMetaData : indicesMetaData) { - final String indexName = indexMetaData.getIndex().getName(); - final ClusterIndexHealth indexHealth = clusterHealthResponse.getIndices().get(indexName); - final IndexStats indexStats = indicesStatsResponse.getIndices().get(indexName); + + indicesSettings.forEach((indexName, settings) -> { + if (indicesMetaDatas.containsKey(indexName) == false) { + // the index exists in the Get Indices response but is not present in the cluster state: + // it is likely that the index was deleted in the meanwhile, so we ignore it. + return; + } + + final IndexMetaData indexMetaData = indicesMetaDatas.get(indexName); final IndexMetaData.State indexState = indexMetaData.getState(); - final boolean searchThrottled = IndexSettings.INDEX_SEARCH_THROTTLED.get(indexMetaData.getSettings()); + final IndexStats indexStats = indicesStats.get(indexName); + final boolean searchThrottled = IndexSettings.INDEX_SEARCH_THROTTLED.get(settings); + + final String health; + final ClusterIndexHealth indexHealth = indicesHealths.get(indexName); + if (indexHealth != null) { + health = indexHealth.getStatus().toString().toLowerCase(Locale.ROOT); + } else if (indexStats != null) { + health = "red*"; + } else { + health = ""; + } if (healthParam != null) { final ClusterHealthStatus healthStatusFilter = ClusterHealthStatus.fromString(healthParam); @@ -413,44 +531,26 @@ Table buildTable(final RestRequest request, // index health is known but does not match the one requested skip = indexHealth.getStatus() != healthStatusFilter; } else { - // index health is unknown, skip if we don't explicitly request RED health or if the index is closed but not replicated - skip = ClusterHealthStatus.RED != healthStatusFilter || indexState == IndexMetaData.State.CLOSE; + // index health is unknown, skip if we don't explicitly request RED health + skip = ClusterHealthStatus.RED != healthStatusFilter; } if (skip) { - continue; + return; } } - // the open index is present in the cluster state but is not returned in the indices stats API - if (indexStats == null && indexState != IndexMetaData.State.CLOSE) { - // the index stats API is called last, after cluster state and cluster health. If the index stats - // has not resolved the same open indices as the initial cluster state call, then the indices might - // have been removed in the meantime or, more likely, are unauthorized. This is because the cluster - // state exposes everything, even unauthorized indices, which are not exposed in APIs. - // We ignore such an index instead of displaying it with an empty stats. - continue; - } - final CommonStats primaryStats; final CommonStats totalStats; - if (indexState == IndexMetaData.State.CLOSE) { - // empty stats for closed indices, but their names are displayed + if (indexStats == null || indexState == IndexMetaData.State.CLOSE) { + // TODO: expose docs stats for replicated closed indices primaryStats = new CommonStats(); totalStats = new CommonStats(); } else { primaryStats = indexStats.getPrimaries(); totalStats = indexStats.getTotal(); } - table.startRow(); - - String health = null; - if (indexHealth != null) { - health = indexHealth.getStatus().toString().toLowerCase(Locale.ROOT); - } else if (indexStats != null) { - health = "red*"; - } table.addCell(health); table.addCell(indexState.toString().toLowerCase(Locale.ROOT)); table.addCell(indexName); @@ -648,25 +748,13 @@ Table buildTable(final RestRequest request, table.addCell(searchThrottled); table.endRow(); - } + }); return table; } - // package private for testing - IndexMetaData[] getOrderedIndexMetaData(String[] indicesExpression, ClusterState clusterState, IndicesOptions indicesOptions) { - final Index[] concreteIndices = indexNameExpressionResolver.concreteIndices(clusterState, indicesOptions, indicesExpression); - // concreteIndices should contain exactly the indices in state.metaData() that were selected by clusterStateRequest using the - // same indices option (IndicesOptions.strictExpand()). We select the indices again here so that they can be displayed in the - // resulting table in the requesting order. - assert concreteIndices.length == clusterState.metaData().getIndices().size(); - final ImmutableOpenMap indexMetaDataMap = clusterState.metaData().getIndices(); - final IndexMetaData[] indicesMetaData = new IndexMetaData[concreteIndices.length]; - // select the index metadata in the requested order, so that the response can display the indices in the resulting table - // in the requesting order. - for (int i = 0; i < concreteIndices.length; i++) { - indicesMetaData[i] = indexMetaDataMap.get(concreteIndices[i].getName()); - } - return indicesMetaData; + @SuppressWarnings("unchecked") + private static A extractResponse(final Collection responses, Class c) { + return (A) responses.stream().filter(c::isInstance).findFirst().get(); } } diff --git a/server/src/main/java/org/elasticsearch/rest/action/cat/RestNodesAction.java b/server/src/main/java/org/elasticsearch/rest/action/cat/RestNodesAction.java index 74a103dcb540c..f623408597382 100644 --- a/server/src/main/java/org/elasticsearch/rest/action/cat/RestNodesAction.java +++ b/server/src/main/java/org/elasticsearch/rest/action/cat/RestNodesAction.java @@ -29,6 +29,7 @@ import org.elasticsearch.action.admin.cluster.state.ClusterStateResponse; import org.elasticsearch.client.node.NodeClient; import org.elasticsearch.cluster.node.DiscoveryNode; +import org.elasticsearch.cluster.node.DiscoveryNodeRole; import org.elasticsearch.cluster.node.DiscoveryNodes; import org.elasticsearch.common.Strings; import org.elasticsearch.common.Table; @@ -324,7 +325,7 @@ Table buildTable(boolean fullId, RestRequest req, ClusterStateResponse state, No if (node.getRoles().isEmpty()) { roles = "-"; } else { - roles = node.getRoles().stream().map(DiscoveryNode.Role::getAbbreviation).collect(Collectors.joining()); + roles = node.getRoles().stream().map(DiscoveryNodeRole::roleNameAbbreviation).sorted().collect(Collectors.joining()); } table.addCell(roles); table.addCell(masterId == null ? "x" : masterId.equals(node.getId()) ? "*" : "-"); diff --git a/server/src/main/java/org/elasticsearch/rest/action/cat/RestSnapshotAction.java b/server/src/main/java/org/elasticsearch/rest/action/cat/RestSnapshotAction.java index 22258ce2d8878..4b679cbf475eb 100644 --- a/server/src/main/java/org/elasticsearch/rest/action/cat/RestSnapshotAction.java +++ b/server/src/main/java/org/elasticsearch/rest/action/cat/RestSnapshotAction.java @@ -20,9 +20,11 @@ package org.elasticsearch.rest.action.cat; +import org.elasticsearch.ElasticsearchException; import org.elasticsearch.action.admin.cluster.snapshots.get.GetSnapshotsRequest; import org.elasticsearch.action.admin.cluster.snapshots.get.GetSnapshotsResponse; import org.elasticsearch.client.node.NodeClient; +import org.elasticsearch.common.Strings; import org.elasticsearch.common.Table; import org.elasticsearch.common.settings.Settings; import org.elasticsearch.common.time.DateFormatter; @@ -36,6 +38,8 @@ import java.time.Instant; import java.time.ZoneOffset; +import java.util.List; +import java.util.Map; import java.util.concurrent.TimeUnit; import static org.elasticsearch.rest.RestRequest.Method.GET; @@ -58,7 +62,7 @@ public String getName() { @Override protected RestChannelConsumer doCatRequest(final RestRequest request, NodeClient client) { GetSnapshotsRequest getSnapshotsRequest = new GetSnapshotsRequest() - .repository(request.param("repository")) + .repositories(request.paramAsStringArray("repository", new String[]{"_all"})) .snapshots(new String[]{GetSnapshotsRequest.ALL_SNAPSHOTS}); getSnapshotsRequest.ignoreUnavailable(request.paramAsBoolean("ignore_unavailable", getSnapshotsRequest.ignoreUnavailable())); @@ -66,7 +70,7 @@ protected RestChannelConsumer doCatRequest(final RestRequest request, NodeClient getSnapshotsRequest.masterNodeTimeout(request.paramAsTime("master_timeout", getSnapshotsRequest.masterNodeTimeout())); return channel -> - client.admin().cluster().getSnapshots(getSnapshotsRequest, new RestResponseListener(channel) { + client.admin().cluster().getSnapshots(getSnapshotsRequest, new RestResponseListener<>(channel) { @Override public RestResponse buildResponse(GetSnapshotsResponse getSnapshotsResponse) throws Exception { return RestTable.buildResponse(buildTable(request, getSnapshotsResponse), channel); @@ -84,6 +88,7 @@ protected Table getTableWithHeader(RestRequest request) { return new Table() .startHeaders() .addCell("id", "alias:id,snapshot;desc:unique snapshot") + .addCell("repository", "alias:re,repo;desc:repository name") .addCell("status", "alias:s,status;text-align:right;desc:snapshot name") .addCell("start_epoch", "alias:ste,startEpoch;desc:start time in seconds since 1970-01-01 00:00:00") .addCell("start_time", "alias:sti,startTime;desc:start time in HH:MM:SS") @@ -102,29 +107,50 @@ protected Table getTableWithHeader(RestRequest request) { private Table buildTable(RestRequest req, GetSnapshotsResponse getSnapshotsResponse) { Table table = getTableWithHeader(req); - for (SnapshotInfo snapshotStatus : getSnapshotsResponse.getSnapshots()) { - table.startRow(); - - table.addCell(snapshotStatus.snapshotId().getName()); - table.addCell(snapshotStatus.state()); - table.addCell(TimeUnit.SECONDS.convert(snapshotStatus.startTime(), TimeUnit.MILLISECONDS)); - table.addCell(FORMATTER.format(Instant.ofEpochMilli(snapshotStatus.startTime()))); - table.addCell(TimeUnit.SECONDS.convert(snapshotStatus.endTime(), TimeUnit.MILLISECONDS)); - table.addCell(FORMATTER.format(Instant.ofEpochMilli(snapshotStatus.endTime()))); - final long durationMillis; - if (snapshotStatus.state() == SnapshotState.IN_PROGRESS) { - durationMillis = System.currentTimeMillis() - snapshotStatus.startTime(); - } else { - durationMillis = snapshotStatus.endTime() - snapshotStatus.startTime(); + + if (getSnapshotsResponse.isFailed()) { + ElasticsearchException causes = null; + + for (ElasticsearchException e : getSnapshotsResponse.getFailedResponses().values()) { + if (causes == null) { + causes = e; + } else { + causes.addSuppressed(e); + } + } + throw new ElasticsearchException( + "Repositories [" + + Strings.collectionToCommaDelimitedString(getSnapshotsResponse.getFailedResponses().keySet()) + + "] failed to retrieve snapshots", causes); + } + + for (Map.Entry> response : getSnapshotsResponse.getSuccessfulResponses().entrySet()) { + String repository = response.getKey(); + for (SnapshotInfo snapshotStatus : response.getValue()) { + table.startRow(); + + table.addCell(snapshotStatus.snapshotId().getName()); + table.addCell(repository); + table.addCell(snapshotStatus.state()); + table.addCell(TimeUnit.SECONDS.convert(snapshotStatus.startTime(), TimeUnit.MILLISECONDS)); + table.addCell(FORMATTER.format(Instant.ofEpochMilli(snapshotStatus.startTime()))); + table.addCell(TimeUnit.SECONDS.convert(snapshotStatus.endTime(), TimeUnit.MILLISECONDS)); + table.addCell(FORMATTER.format(Instant.ofEpochMilli(snapshotStatus.endTime()))); + final long durationMillis; + if (snapshotStatus.state() == SnapshotState.IN_PROGRESS) { + durationMillis = System.currentTimeMillis() - snapshotStatus.startTime(); + } else { + durationMillis = snapshotStatus.endTime() - snapshotStatus.startTime(); + } + table.addCell(TimeValue.timeValueMillis(durationMillis)); + table.addCell(snapshotStatus.indices().size()); + table.addCell(snapshotStatus.successfulShards()); + table.addCell(snapshotStatus.failedShards()); + table.addCell(snapshotStatus.totalShards()); + table.addCell(snapshotStatus.reason()); + + table.endRow(); } - table.addCell(TimeValue.timeValueMillis(durationMillis)); - table.addCell(snapshotStatus.indices().size()); - table.addCell(snapshotStatus.successfulShards()); - table.addCell(snapshotStatus.failedShards()); - table.addCell(snapshotStatus.totalShards()); - table.addCell(snapshotStatus.reason()); - - table.endRow(); } return table; diff --git a/server/src/main/java/org/elasticsearch/rest/action/document/RestMultiTermVectorsAction.java b/server/src/main/java/org/elasticsearch/rest/action/document/RestMultiTermVectorsAction.java index 075167e97ccb2..1896314aa4a06 100644 --- a/server/src/main/java/org/elasticsearch/rest/action/document/RestMultiTermVectorsAction.java +++ b/server/src/main/java/org/elasticsearch/rest/action/document/RestMultiTermVectorsAction.java @@ -24,7 +24,6 @@ import org.elasticsearch.client.node.NodeClient; import org.elasticsearch.common.Strings; import org.elasticsearch.common.settings.Settings; -import org.elasticsearch.index.mapper.MapperService; import org.elasticsearch.rest.BaseRestHandler; import org.elasticsearch.rest.RestController; import org.elasticsearch.rest.RestRequest; @@ -56,9 +55,6 @@ public RestChannelConsumer prepareRequest(final RestRequest request, final NodeC TermVectorsRequest template = new TermVectorsRequest() .index(request.param("index")); - - template.type(MapperService.SINGLE_MAPPING_NAME); - RestTermVectorsAction.readURIParameters(template, request); multiTermVectorsRequest.ids(Strings.commaDelimitedListToStringArray(request.param("ids"))); request.withContentOrSourceParamParserOrNull(p -> multiTermVectorsRequest.add(template, p)); diff --git a/server/src/main/java/org/elasticsearch/rest/action/document/RestTermVectorsAction.java b/server/src/main/java/org/elasticsearch/rest/action/document/RestTermVectorsAction.java index 3e136b368280d..2a77a2b1afe19 100644 --- a/server/src/main/java/org/elasticsearch/rest/action/document/RestTermVectorsAction.java +++ b/server/src/main/java/org/elasticsearch/rest/action/document/RestTermVectorsAction.java @@ -25,7 +25,6 @@ import org.elasticsearch.common.settings.Settings; import org.elasticsearch.common.xcontent.XContentParser; import org.elasticsearch.index.VersionType; -import org.elasticsearch.index.mapper.MapperService; import org.elasticsearch.rest.BaseRestHandler; import org.elasticsearch.rest.RestController; import org.elasticsearch.rest.RestRequest; @@ -63,7 +62,7 @@ public String getName() { @Override public RestChannelConsumer prepareRequest(final RestRequest request, final NodeClient client) throws IOException { TermVectorsRequest termVectorsRequest; - termVectorsRequest = new TermVectorsRequest(request.param("index"), MapperService.SINGLE_MAPPING_NAME, request.param("id")); + termVectorsRequest = new TermVectorsRequest(request.param("index"), request.param("id")); if (request.hasContentOrSourceParam()) { try (XContentParser parser = request.contentOrSourceParamParser()) { diff --git a/server/src/main/java/org/elasticsearch/rest/action/search/RestCountAction.java b/server/src/main/java/org/elasticsearch/rest/action/search/RestCountAction.java index ecdd34ca07c88..df744978ab198 100644 --- a/server/src/main/java/org/elasticsearch/rest/action/search/RestCountAction.java +++ b/server/src/main/java/org/elasticsearch/rest/action/search/RestCountAction.java @@ -19,13 +19,11 @@ package org.elasticsearch.rest.action.search; -import org.apache.logging.log4j.LogManager; import org.elasticsearch.action.search.SearchRequest; import org.elasticsearch.action.search.SearchResponse; import org.elasticsearch.action.support.IndicesOptions; import org.elasticsearch.client.node.NodeClient; import org.elasticsearch.common.Strings; -import org.elasticsearch.common.logging.DeprecationLogger; import org.elasticsearch.common.settings.Settings; import org.elasticsearch.common.xcontent.XContentBuilder; import org.elasticsearch.index.query.QueryBuilder; @@ -46,10 +44,6 @@ import static org.elasticsearch.search.internal.SearchContext.DEFAULT_TERMINATE_AFTER; public class RestCountAction extends BaseRestHandler { - private static final DeprecationLogger deprecationLogger = new DeprecationLogger( - LogManager.getLogger(RestCountAction.class)); - static final String TYPES_DEPRECATION_MESSAGE = "[types removal]" + - " Specifying types in count requests is deprecated."; public RestCountAction(Settings settings, RestController controller) { super(settings); @@ -57,10 +51,6 @@ public RestCountAction(Settings settings, RestController controller) { controller.registerHandler(GET, "/_count", this); controller.registerHandler(POST, "/{index}/_count", this); controller.registerHandler(GET, "/{index}/_count", this); - - // Deprecated typed endpoints. - controller.registerHandler(POST, "/{index}/{type}/_count", this); - controller.registerHandler(GET, "/{index}/{type}/_count", this); } @Override @@ -90,11 +80,6 @@ public RestChannelConsumer prepareRequest(final RestRequest request, final NodeC searchSourceBuilder.minScore(minScore); } - if (request.hasParam("type")) { - deprecationLogger.deprecatedAndMaybeLog("count_with_types", TYPES_DEPRECATION_MESSAGE); - countRequest.types(Strings.splitStringByCommaToArray(request.param("type"))); - } - countRequest.preference(request.param("preference")); final int terminateAfter = request.paramAsInt("terminate_after", DEFAULT_TERMINATE_AFTER); diff --git a/server/src/main/java/org/elasticsearch/rest/action/search/RestMultiSearchAction.java b/server/src/main/java/org/elasticsearch/rest/action/search/RestMultiSearchAction.java index d9beba089857f..1aef4aa5254a4 100644 --- a/server/src/main/java/org/elasticsearch/rest/action/search/RestMultiSearchAction.java +++ b/server/src/main/java/org/elasticsearch/rest/action/search/RestMultiSearchAction.java @@ -134,7 +134,7 @@ public static void parseMultiLineRequest(RestRequest request, IndicesOptions ind final Tuple sourceTuple = request.contentOrSourceParam(); final XContent xContent = sourceTuple.v1().xContent(); final BytesReference data = sourceTuple.v2(); - MultiSearchRequest.readMultiLineFormat(data, xContent, consumer, indices, indicesOptions, Strings.EMPTY_ARRAY, routing, + MultiSearchRequest.readMultiLineFormat(data, xContent, consumer, indices, indicesOptions, routing, searchType, ccsMinimizeRoundtrips, request.getXContentRegistry(), allowExplicitIndex); } diff --git a/server/src/main/java/org/elasticsearch/search/DefaultSearchContext.java b/server/src/main/java/org/elasticsearch/search/DefaultSearchContext.java index f0eaa0d51dadf..c2c089ffe31ab 100644 --- a/server/src/main/java/org/elasticsearch/search/DefaultSearchContext.java +++ b/server/src/main/java/org/elasticsearch/search/DefaultSearchContext.java @@ -43,7 +43,6 @@ import org.elasticsearch.index.mapper.MappedFieldType; import org.elasticsearch.index.mapper.MapperService; import org.elasticsearch.index.mapper.ObjectMapper; -import org.elasticsearch.index.mapper.TypeFieldMapper; import org.elasticsearch.index.query.AbstractQueryBuilder; import org.elasticsearch.index.query.ParsedQuery; import org.elasticsearch.index.query.QueryBuilder; @@ -76,7 +75,6 @@ import java.io.IOException; import java.io.UncheckedIOException; import java.util.ArrayList; -import java.util.Arrays; import java.util.Collections; import java.util.HashMap; import java.util.List; @@ -175,13 +173,13 @@ final class DefaultSearchContext extends SearchContext { this.indexShard = indexShard; this.indexService = indexService; this.clusterService = clusterService; - this.searcher = new ContextIndexSearcher(engineSearcher, indexService.cache().query(), indexShard.getQueryCachingPolicy()); + this.searcher = new ContextIndexSearcher(engineSearcher.reader(), engineSearcher.searcher().getSimilarity(), + indexService.cache().query(), indexShard.getQueryCachingPolicy()); this.relativeTimeSupplier = relativeTimeSupplier; this.timeout = timeout; this.minNodeVersion = minNodeVersion; queryShardContext = indexService.newQueryShardContext(request.shardId().id(), searcher.getIndexReader(), request::nowInMillis, shardTarget.getClusterAlias()); - queryShardContext.setTypes(request.types()); queryBoost = request.indexBoost(); } @@ -269,13 +267,8 @@ public void preProcess(boolean rewrite) { @Override public Query buildFilteredQuery(Query query) { List filters = new ArrayList<>(); - Query typeFilter = createTypeFilter(queryShardContext.getTypes()); - if (typeFilter != null) { - filters.add(typeFilter); - } if (mapperService().hasNested() - && typeFilter == null // when a _type filter is set, it will automatically exclude nested docs && new NestedHelper(mapperService()).mightMatchNestedDocs(query) && (aliasFilter == null || new NestedHelper(mapperService()).mightMatchNestedDocs(aliasFilter))) { filters.add(Queries.newNonNestedFilter()); @@ -301,17 +294,6 @@ && new NestedHelper(mapperService()).mightMatchNestedDocs(query) } } - private Query createTypeFilter(String[] types) { - if (types != null && types.length >= 1) { - MappedFieldType ft = mapperService().fullName(TypeFieldMapper.NAME); - if (ft != null) { - // ft might be null if no documents have been indexed yet - return ft.termsQuery(Arrays.asList(types), queryShardContext); - } - } - return null; - } - @Override public long id() { return this.id; diff --git a/server/src/main/java/org/elasticsearch/search/SearchHits.java b/server/src/main/java/org/elasticsearch/search/SearchHits.java index e5021b47a3e36..7d884c028b548 100644 --- a/server/src/main/java/org/elasticsearch/search/SearchHits.java +++ b/server/src/main/java/org/elasticsearch/search/SearchHits.java @@ -47,7 +47,7 @@ public static SearchHits empty() { } public static SearchHits empty(boolean withTotalHits) { - // We shouldn't use static final instance, since that could directly be returned by native transport clients + // TODO: consider using static final instance return new SearchHits(EMPTY, withTotalHits ? new TotalHits(0, Relation.EQUAL_TO) : null, 0); } diff --git a/server/src/main/java/org/elasticsearch/search/SearchModule.java b/server/src/main/java/org/elasticsearch/search/SearchModule.java index 4c6ba07c631af..01a4aa66810c8 100644 --- a/server/src/main/java/org/elasticsearch/search/SearchModule.java +++ b/server/src/main/java/org/elasticsearch/search/SearchModule.java @@ -32,9 +32,7 @@ import org.elasticsearch.common.xcontent.ParseFieldRegistry; import org.elasticsearch.common.xcontent.XContentParser; import org.elasticsearch.index.query.BoolQueryBuilder; -import org.elasticsearch.index.query.MatchBoolPrefixQueryBuilder; import org.elasticsearch.index.query.BoostingQueryBuilder; -import org.elasticsearch.index.query.CommonTermsQueryBuilder; import org.elasticsearch.index.query.ConstantScoreQueryBuilder; import org.elasticsearch.index.query.DisMaxQueryBuilder; import org.elasticsearch.index.query.DistanceFeatureQueryBuilder; @@ -49,6 +47,7 @@ import org.elasticsearch.index.query.IntervalQueryBuilder; import org.elasticsearch.index.query.IntervalsSourceProvider; import org.elasticsearch.index.query.MatchAllQueryBuilder; +import org.elasticsearch.index.query.MatchBoolPrefixQueryBuilder; import org.elasticsearch.index.query.MatchNoneQueryBuilder; import org.elasticsearch.index.query.MatchPhrasePrefixQueryBuilder; import org.elasticsearch.index.query.MatchPhraseQueryBuilder; @@ -154,9 +153,13 @@ import org.elasticsearch.search.aggregations.bucket.significant.heuristics.SignificanceHeuristic; import org.elasticsearch.search.aggregations.bucket.significant.heuristics.SignificanceHeuristicParser; import org.elasticsearch.search.aggregations.bucket.terms.DoubleTerms; +import org.elasticsearch.search.aggregations.bucket.terms.LongRareTerms; import org.elasticsearch.search.aggregations.bucket.terms.LongTerms; +import org.elasticsearch.search.aggregations.bucket.terms.RareTermsAggregationBuilder; +import org.elasticsearch.search.aggregations.bucket.terms.StringRareTerms; import org.elasticsearch.search.aggregations.bucket.terms.StringTerms; import org.elasticsearch.search.aggregations.bucket.terms.TermsAggregationBuilder; +import org.elasticsearch.search.aggregations.bucket.terms.UnmappedRareTerms; import org.elasticsearch.search.aggregations.bucket.terms.UnmappedTerms; import org.elasticsearch.search.aggregations.metrics.AvgAggregationBuilder; import org.elasticsearch.search.aggregations.metrics.CardinalityAggregationBuilder; @@ -280,7 +283,6 @@ public class SearchModule { public static final Setting INDICES_MAX_CLAUSE_COUNT_SETTING = Setting.intSetting("indices.query.bool.max_clause_count", 1024, 1, Integer.MAX_VALUE, Setting.Property.NodeScope); - private final boolean transportClient; private final Map highlighters; private final ParseFieldRegistry significanceHeuristicParserRegistry = new ParseFieldRegistry<>( "significance_heuristic"); @@ -296,14 +298,11 @@ public class SearchModule { * * NOTE: This constructor should not be called in production unless an accurate {@link Settings} object is provided. * When constructed, a static flag is set in Lucene {@link BooleanQuery#setMaxClauseCount} according to the settings. - * - * @param settings Current settings - * @param transportClient Is this being constructed in the TransportClient or not + * @param settings Current settings * @param plugins List of included {@link SearchPlugin} objects. */ - public SearchModule(Settings settings, boolean transportClient, List plugins) { + public SearchModule(Settings settings, List plugins) { this.settings = settings; - this.transportClient = transportClient; registerSuggesters(plugins); highlighters = setupHighlighters(settings, plugins); registerScoreFunctions(plugins); @@ -395,6 +394,11 @@ private void registerAggregations(List plugins) { .addResultReader(UnmappedTerms.NAME, UnmappedTerms::new) .addResultReader(LongTerms.NAME, LongTerms::new) .addResultReader(DoubleTerms.NAME, DoubleTerms::new)); + registerAggregation(new AggregationSpec(RareTermsAggregationBuilder.NAME, RareTermsAggregationBuilder::new, + RareTermsAggregationBuilder::parse) + .addResultReader(StringRareTerms.NAME, StringRareTerms::new) + .addResultReader(UnmappedRareTerms.NAME, UnmappedRareTerms::new) + .addResultReader(LongRareTerms.NAME, LongRareTerms::new)); registerAggregation(new AggregationSpec(SignificantTermsAggregationBuilder.NAME, SignificantTermsAggregationBuilder::new, SignificantTermsAggregationBuilder.getParser(significanceHeuristicParserRegistry)) .addResultReader(SignificantStringTerms.NAME, SignificantStringTerms::new) @@ -438,12 +442,10 @@ private void registerAggregations(List plugins) { } private void registerAggregation(AggregationSpec spec) { - if (false == transportClient) { - namedXContents.add(new NamedXContentRegistry.Entry(BaseAggregationBuilder.class, spec.getName(), (p, c) -> { - AggregatorFactories.AggParseContext context = (AggregatorFactories.AggParseContext) c; - return spec.getParser().parse(context.name, p); - })); - } + namedXContents.add(new NamedXContentRegistry.Entry(BaseAggregationBuilder.class, spec.getName(), (p, c) -> { + AggregatorFactories.AggParseContext context = (AggregatorFactories.AggParseContext) c; + return spec.getParser().parse(context.name, p); + })); namedWriteables.add( new NamedWriteableRegistry.Entry(AggregationBuilder.class, spec.getName().getPreferredName(), spec.getReader())); for (Map.Entry> t : spec.getResultReaders().entrySet()) { @@ -539,12 +541,10 @@ private void registerPipelineAggregations(List plugins) { } private void registerPipelineAggregation(PipelineAggregationSpec spec) { - if (false == transportClient) { - namedXContents.add(new NamedXContentRegistry.Entry(BaseAggregationBuilder.class, spec.getName(), (p, c) -> { - AggregatorFactories.AggParseContext context = (AggregatorFactories.AggParseContext) c; - return spec.getParser().parse(context.name, p); - })); - } + namedXContents.add(new NamedXContentRegistry.Entry(BaseAggregationBuilder.class, spec.getName(), (p, c) -> { + AggregatorFactories.AggParseContext context = (AggregatorFactories.AggParseContext) c; + return spec.getParser().parse(context.name, p); + })); namedWriteables.add( new NamedWriteableRegistry.Entry(PipelineAggregationBuilder.class, spec.getName().getPreferredName(), spec.getReader())); namedWriteables.add( @@ -567,9 +567,7 @@ private void registerRescorers(List plugins) { } private void registerRescorer(RescorerSpec spec) { - if (false == transportClient) { - namedXContents.add(new NamedXContentRegistry.Entry(RescorerBuilder.class, spec.getName(), (p, c) -> spec.getParser().apply(p))); - } + namedXContents.add(new NamedXContentRegistry.Entry(RescorerBuilder.class, spec.getName(), (p, c) -> spec.getParser().apply(p))); namedWriteables.add(new NamedWriteableRegistry.Entry(RescorerBuilder.class, spec.getName().getPreferredName(), spec.getReader())); } @@ -767,7 +765,6 @@ private void registerQueryParsers(List plugins) { registerQuery(new QuerySpec<>(MoreLikeThisQueryBuilder.NAME, MoreLikeThisQueryBuilder::new, MoreLikeThisQueryBuilder::fromXContent)); registerQuery(new QuerySpec<>(WrapperQueryBuilder.NAME, WrapperQueryBuilder::new, WrapperQueryBuilder::fromXContent)); - registerQuery(new QuerySpec<>(CommonTermsQueryBuilder.NAME, CommonTermsQueryBuilder::new, CommonTermsQueryBuilder::fromXContent)); registerQuery( new QuerySpec<>(SpanMultiTermQueryBuilder.NAME, SpanMultiTermQueryBuilder::new, SpanMultiTermQueryBuilder::fromXContent)); registerQuery(new QuerySpec<>(FunctionScoreQueryBuilder.NAME, FunctionScoreQueryBuilder::new, @@ -804,6 +801,10 @@ private void registerIntervalsSourceProviders() { IntervalsSourceProvider.Combine.NAME, IntervalsSourceProvider.Combine::new)); namedWriteables.add(new NamedWriteableRegistry.Entry(IntervalsSourceProvider.class, IntervalsSourceProvider.Disjunction.NAME, IntervalsSourceProvider.Disjunction::new)); + namedWriteables.add(new NamedWriteableRegistry.Entry(IntervalsSourceProvider.class, + IntervalsSourceProvider.Prefix.NAME, IntervalsSourceProvider.Prefix::new)); + namedWriteables.add(new NamedWriteableRegistry.Entry(IntervalsSourceProvider.class, + IntervalsSourceProvider.Wildcard.NAME, IntervalsSourceProvider.Wildcard::new)); } private void registerQuery(QuerySpec spec) { diff --git a/server/src/main/java/org/elasticsearch/search/SearchService.java b/server/src/main/java/org/elasticsearch/search/SearchService.java index daf8e1faf7bb8..19ae52cb51099 100644 --- a/server/src/main/java/org/elasticsearch/search/SearchService.java +++ b/server/src/main/java/org/elasticsearch/search/SearchService.java @@ -548,19 +548,35 @@ final SearchContext createAndPutContext(ShardSearchRequest request) throws IOExc } SearchContext context = createContext(request); + onNewContext(context); boolean success = false; try { putContext(context); - if (request.scroll() != null) { + success = true; + return context; + } finally { + if (success == false) { + freeContext(context.id()); + } + } + } + + private void onNewContext(SearchContext context) { + boolean success = false; + try { + if (context.scrollContext() != null) { openScrollContexts.incrementAndGet(); context.indexShard().getSearchOperationListener().onNewScrollContext(context); } context.indexShard().getSearchOperationListener().onNewContext(context); success = true; - return context; } finally { - if (!success) { - freeContext(context.id()); + // currently, the concrete listener is CompositeListener, which swallows exceptions, but here we anyway try to do the + // right thing by closing and notifying onFreeXXX in case one of the listeners fails with an exception in the future. + if (success == false) { + try (context) { + onFreeContext(context); + } } } } @@ -648,18 +664,23 @@ private void freeAllContextForIndex(Index index) { public boolean freeContext(long id) { try (SearchContext context = removeContext(id)) { if (context != null) { - assert context.refCount() > 0 : " refCount must be > 0: " + context.refCount(); - context.indexShard().getSearchOperationListener().onFreeContext(context); - if (context.scrollContext() != null) { - openScrollContexts.decrementAndGet(); - context.indexShard().getSearchOperationListener().onFreeScrollContext(context); - } + onFreeContext(context); return true; } return false; } } + private void onFreeContext(SearchContext context) { + assert context.refCount() > 0 : " refCount must be > 0: " + context.refCount(); + assert activeContexts.containsKey(context.id()) == false; + context.indexShard().getSearchOperationListener().onFreeContext(context); + if (context.scrollContext() != null) { + openScrollContexts.decrementAndGet(); + context.indexShard().getSearchOperationListener().onFreeScrollContext(context); + } + } + public void freeAllScrollContexts() { for (SearchContext searchContext : activeContexts.values()) { if (searchContext.scrollContext() != null) { diff --git a/server/src/main/java/org/elasticsearch/search/aggregations/AbstractAggregationBuilder.java b/server/src/main/java/org/elasticsearch/search/aggregations/AbstractAggregationBuilder.java index f07d9abfcb035..117f3a0772a71 100644 --- a/server/src/main/java/org/elasticsearch/search/aggregations/AbstractAggregationBuilder.java +++ b/server/src/main/java/org/elasticsearch/search/aggregations/AbstractAggregationBuilder.java @@ -135,12 +135,12 @@ public final String getWriteableName() { } @Override - public final AggregatorFactory build(SearchContext context, AggregatorFactory parent) throws IOException { - AggregatorFactory factory = doBuild(context, parent, factoriesBuilder); + public final AggregatorFactory build(SearchContext context, AggregatorFactory parent) throws IOException { + AggregatorFactory factory = doBuild(context, parent, factoriesBuilder); return factory; } - protected abstract AggregatorFactory doBuild(SearchContext context, AggregatorFactory parent, + protected abstract AggregatorFactory doBuild(SearchContext context, AggregatorFactory parent, AggregatorFactories.Builder subfactoriesBuilder) throws IOException; @Override @@ -164,28 +164,17 @@ public final XContentBuilder toXContent(XContentBuilder builder, Params params) @Override public int hashCode() { - return Objects.hash(factoriesBuilder, metaData, name, doHashCode()); + return Objects.hash(factoriesBuilder, metaData, name); } - protected abstract int doHashCode(); - @Override public boolean equals(Object obj) { - if (obj == null) - return false; - if (getClass() != obj.getClass()) - return false; - @SuppressWarnings("unchecked") + if (this == obj) return true; + if (obj == null || getClass() != obj.getClass()) return false; AbstractAggregationBuilder other = (AbstractAggregationBuilder) obj; - if (!Objects.equals(name, other.name)) - return false; - if (!Objects.equals(metaData, other.metaData)) - return false; - if (!Objects.equals(factoriesBuilder, other.factoriesBuilder)) - return false; - return doEquals(obj); - } - - protected abstract boolean doEquals(Object obj); + return Objects.equals(name, other.name) + && Objects.equals(metaData, other.metaData) + && Objects.equals(factoriesBuilder, other.factoriesBuilder); + } } diff --git a/server/src/main/java/org/elasticsearch/search/aggregations/AggregationBuilder.java b/server/src/main/java/org/elasticsearch/search/aggregations/AggregationBuilder.java index 359c8dd571e25..6a7b955542480 100644 --- a/server/src/main/java/org/elasticsearch/search/aggregations/AggregationBuilder.java +++ b/server/src/main/java/org/elasticsearch/search/aggregations/AggregationBuilder.java @@ -63,7 +63,7 @@ public String getName() { } /** Internal: build an {@link AggregatorFactory} based on the configuration of this builder. */ - protected abstract AggregatorFactory build(SearchContext context, AggregatorFactory parent) throws IOException; + protected abstract AggregatorFactory build(SearchContext context, AggregatorFactory parent) throws IOException; /** Associate metadata with this {@link AggregationBuilder}. */ @Override diff --git a/server/src/main/java/org/elasticsearch/search/aggregations/AggregatorFactories.java b/server/src/main/java/org/elasticsearch/search/aggregations/AggregatorFactories.java index 7f296daa5d18c..5e818fe69c7ad 100644 --- a/server/src/main/java/org/elasticsearch/search/aggregations/AggregatorFactories.java +++ b/server/src/main/java/org/elasticsearch/search/aggregations/AggregatorFactories.java @@ -161,16 +161,16 @@ public AggParseContext(String name) { } } - public static final AggregatorFactories EMPTY = new AggregatorFactories(new AggregatorFactory[0], new ArrayList<>()); + public static final AggregatorFactories EMPTY = new AggregatorFactories(new AggregatorFactory[0], new ArrayList<>()); - private AggregatorFactory[] factories; + private AggregatorFactory[] factories; private List pipelineAggregatorFactories; public static Builder builder() { return new Builder(); } - private AggregatorFactories(AggregatorFactory[] factories, List pipelineAggregators) { + private AggregatorFactories(AggregatorFactory[] factories, List pipelineAggregators) { this.factories = factories; this.pipelineAggregatorFactories = pipelineAggregators; } @@ -314,7 +314,7 @@ Builder skipResolveOrder() { return this; } - public AggregatorFactories build(SearchContext context, AggregatorFactory parent) throws IOException { + public AggregatorFactories build(SearchContext context, AggregatorFactory parent) throws IOException { if (aggregationBuilders.isEmpty() && pipelineAggregatorBuilders.isEmpty()) { return EMPTY; } @@ -325,7 +325,7 @@ public AggregatorFactories build(SearchContext context, AggregatorFactory par orderedpipelineAggregators = resolvePipelineAggregatorOrder(this.pipelineAggregatorBuilders, this.aggregationBuilders, parent); } - AggregatorFactory[] aggFactories = new AggregatorFactory[aggregationBuilders.size()]; + AggregatorFactory[] aggFactories = new AggregatorFactory[aggregationBuilders.size()]; int i = 0; for (AggregationBuilder agg : aggregationBuilders) { @@ -337,7 +337,7 @@ public AggregatorFactories build(SearchContext context, AggregatorFactory par private List resolvePipelineAggregatorOrder( Collection pipelineAggregatorBuilders, Collection aggregationBuilders, - AggregatorFactory parent) { + AggregatorFactory parent) { Map pipelineAggregatorBuildersMap = new HashMap<>(); for (PipelineAggregationBuilder builder : pipelineAggregatorBuilders) { pipelineAggregatorBuildersMap.put(builder.getName(), builder); diff --git a/server/src/main/java/org/elasticsearch/search/aggregations/AggregatorFactory.java b/server/src/main/java/org/elasticsearch/search/aggregations/AggregatorFactory.java index d6e7aca46a63b..196f4faea3e2d 100644 --- a/server/src/main/java/org/elasticsearch/search/aggregations/AggregatorFactory.java +++ b/server/src/main/java/org/elasticsearch/search/aggregations/AggregatorFactory.java @@ -33,17 +33,17 @@ import java.util.List; import java.util.Map; -public abstract class AggregatorFactory> { +public abstract class AggregatorFactory { public static final class MultiBucketAggregatorWrapper extends Aggregator { private final BigArrays bigArrays; private final Aggregator parent; - private final AggregatorFactory factory; + private final AggregatorFactory factory; private final Aggregator first; ObjectArray aggregators; ObjectArray collectors; - MultiBucketAggregatorWrapper(BigArrays bigArrays, SearchContext context, Aggregator parent, AggregatorFactory factory, + MultiBucketAggregatorWrapper(BigArrays bigArrays, SearchContext context, Aggregator parent, AggregatorFactory factory, Aggregator first) { this.bigArrays = bigArrays; this.parent = parent; @@ -167,7 +167,7 @@ public void close() { } protected final String name; - protected final AggregatorFactory parent; + protected final AggregatorFactory parent; protected final AggregatorFactories factories; protected final Map metaData; protected final SearchContext context; @@ -180,7 +180,7 @@ public void close() { * @throws IOException * if an error occurs creating the factory */ - public AggregatorFactory(String name, SearchContext context, AggregatorFactory parent, + public AggregatorFactory(String name, SearchContext context, AggregatorFactory parent, AggregatorFactories.Builder subFactoriesBuilder, Map metaData) throws IOException { this.name = name; this.context = context; @@ -217,7 +217,7 @@ public final Aggregator create(Aggregator parent, boolean collectsFromSingleBuck return createInternal(parent, collectsFromSingleBucket, this.factories.createPipelineAggregators(), this.metaData); } - public AggregatorFactory getParent() { + public AggregatorFactory getParent() { return parent; } @@ -226,7 +226,7 @@ public AggregatorFactory getParent() { * {@link Aggregator}s that only know how to collect bucket {@code 0}, this * returns an aggregator that can collect any bucket. */ - protected static Aggregator asMultiBucketAggregator(final AggregatorFactory factory, final SearchContext context, + protected static Aggregator asMultiBucketAggregator(final AggregatorFactory factory, final SearchContext context, final Aggregator parent) throws IOException { final Aggregator first = factory.create(parent, true); final BigArrays bigArrays = context.bigArrays(); diff --git a/server/src/main/java/org/elasticsearch/search/aggregations/InternalAggregation.java b/server/src/main/java/org/elasticsearch/search/aggregations/InternalAggregation.java index b525fd32d918a..d664b10cfa16e 100644 --- a/server/src/main/java/org/elasticsearch/search/aggregations/InternalAggregation.java +++ b/server/src/main/java/org/elasticsearch/search/aggregations/InternalAggregation.java @@ -218,40 +218,22 @@ public final XContentBuilder toXContent(XContentBuilder builder, Params params) @Override public int hashCode() { - return Objects.hash(name, metaData, pipelineAggregators, doHashCode()); + return Objects.hash(name, metaData, pipelineAggregators); } - /** - * Opportunity for subclasses to the {@link #hashCode()} for this - * class. - **/ - protected abstract int doHashCode(); - @Override public boolean equals(Object obj) { - if (obj == null) { - return false; - } - if (obj.getClass() != getClass()) { + if (obj == null || getClass() != obj.getClass()) { return false; } + if (obj == this) { return true; } + InternalAggregation other = (InternalAggregation) obj; return Objects.equals(name, other.name) && Objects.equals(pipelineAggregators, other.pipelineAggregators) && - Objects.equals(metaData, other.metaData) && - doEquals(obj); + Objects.equals(metaData, other.metaData); } - /** - * Opportunity for subclasses to add criteria to the {@link #equals(Object)} - * method for this class. - * - * This method can safely cast obj to the subclass since the - * {@link #equals(Object)} method checks that obj is the same - * class as this - */ - protected abstract boolean doEquals(Object obj); - @Override public String toString() { return Strings.toString(this); diff --git a/server/src/main/java/org/elasticsearch/search/aggregations/PipelineAggregationBuilder.java b/server/src/main/java/org/elasticsearch/search/aggregations/PipelineAggregationBuilder.java index 08938a5b9b9fe..85b9a7dcb0ca5 100644 --- a/server/src/main/java/org/elasticsearch/search/aggregations/PipelineAggregationBuilder.java +++ b/server/src/main/java/org/elasticsearch/search/aggregations/PipelineAggregationBuilder.java @@ -67,7 +67,7 @@ public final String[] getBucketsPaths() { * Internal: Validates the state of this factory (makes sure the factory is properly * configured) */ - protected abstract void validate(AggregatorFactory parent, Collection aggregationBuilders, + protected abstract void validate(AggregatorFactory parent, Collection aggregationBuilders, Collection pipelineAggregatorBuilders); /** diff --git a/server/src/main/java/org/elasticsearch/search/aggregations/bucket/BestBucketsDeferringCollector.java b/server/src/main/java/org/elasticsearch/search/aggregations/bucket/BestBucketsDeferringCollector.java index 32695ac69a88e..5e5e60760d790 100644 --- a/server/src/main/java/org/elasticsearch/search/aggregations/bucket/BestBucketsDeferringCollector.java +++ b/server/src/main/java/org/elasticsearch/search/aggregations/bucket/BestBucketsDeferringCollector.java @@ -40,6 +40,7 @@ import java.io.IOException; import java.util.ArrayList; import java.util.List; +import java.util.Objects; /** * A specialization of {@link DeferringBucketCollector} that collects all @@ -48,28 +49,28 @@ * this collector. */ public class BestBucketsDeferringCollector extends DeferringBucketCollector { - private static class Entry { + static class Entry { final LeafReaderContext context; final PackedLongValues docDeltas; final PackedLongValues buckets; Entry(LeafReaderContext context, PackedLongValues docDeltas, PackedLongValues buckets) { - this.context = context; - this.docDeltas = docDeltas; - this.buckets = buckets; + this.context = Objects.requireNonNull(context); + this.docDeltas = Objects.requireNonNull(docDeltas); + this.buckets = Objects.requireNonNull(buckets); } } - final List entries = new ArrayList<>(); - BucketCollector collector; - final SearchContext searchContext; - final boolean isGlobal; - LeafReaderContext context; - PackedLongValues.Builder docDeltas; - PackedLongValues.Builder buckets; - long maxBucket = -1; - boolean finished = false; - LongHash selectedBuckets; + protected List entries = new ArrayList<>(); + protected BucketCollector collector; + protected final SearchContext searchContext; + protected final boolean isGlobal; + protected LeafReaderContext context; + protected PackedLongValues.Builder docDeltasBuilder; + protected PackedLongValues.Builder bucketsBuilder; + protected long maxBucket = -1; + protected boolean finished = false; + protected LongHash selectedBuckets; /** * Sole constructor. @@ -97,28 +98,32 @@ public void setDeferredCollector(Iterable deferredCollectors) { private void finishLeaf() { if (context != null) { - entries.add(new Entry(context, docDeltas.build(), buckets.build())); + assert docDeltasBuilder != null && bucketsBuilder != null; + entries.add(new Entry(context, docDeltasBuilder.build(), bucketsBuilder.build())); } - context = null; - docDeltas = null; - buckets = null; } @Override public LeafBucketCollector getLeafCollector(LeafReaderContext ctx) throws IOException { finishLeaf(); - context = ctx; - docDeltas = PackedLongValues.packedBuilder(PackedInts.DEFAULT); - buckets = PackedLongValues.packedBuilder(PackedInts.DEFAULT); + context = null; + // allocates the builder lazily in case this segment doesn't contain any match + docDeltasBuilder = null; + bucketsBuilder = null; return new LeafBucketCollector() { int lastDoc = 0; @Override public void collect(int doc, long bucket) throws IOException { - docDeltas.add(doc - lastDoc); - buckets.add(bucket); + if (context == null) { + context = ctx; + docDeltasBuilder = PackedLongValues.packedBuilder(PackedInts.DEFAULT); + bucketsBuilder = PackedLongValues.packedBuilder(PackedInts.DEFAULT); + } + docDeltasBuilder.add(doc - lastDoc); + bucketsBuilder.add(bucket); lastDoc = doc; maxBucket = Math.max(maxBucket, bucket); } @@ -141,7 +146,7 @@ public void postCollection() throws IOException { */ @Override public void prepareSelectedBuckets(long... selectedBuckets) throws IOException { - if (!finished) { + if (finished == false) { throw new IllegalStateException("Cannot replay yet, collection is not finished: postCollect() has not been called"); } if (this.selectedBuckets != null) { @@ -160,14 +165,16 @@ public void prepareSelectedBuckets(long... selectedBuckets) throws IOException { Query query = isGlobal ? new MatchAllDocsQuery() : searchContext.query(); weight = searchContext.searcher().createWeight(searchContext.searcher().rewrite(query), ScoreMode.COMPLETE, 1f); } + for (Entry entry : entries) { + assert entry.docDeltas.size() > 0 : "segment should have at least one document to replay, got 0"; final LeafBucketCollector leafCollector = collector.getLeafCollector(entry.context); - DocIdSetIterator docIt = null; - if (needsScores && entry.docDeltas.size() > 0) { + DocIdSetIterator scoreIt = null; + if (needsScores) { Scorer scorer = weight.scorer(entry.context); // We don't need to check if the scorer is null // since we are sure that there are documents to replay (entry.docDeltas it not empty). - docIt = scorer.iterator(); + scoreIt = scorer.iterator(); leafCollector.setScorer(scorer); } final PackedLongValues.Iterator docDeltaIterator = entry.docDeltas.iterator(); @@ -179,17 +186,16 @@ public void prepareSelectedBuckets(long... selectedBuckets) throws IOException { final long rebasedBucket = hash.find(bucket); if (rebasedBucket != -1) { if (needsScores) { - if (docIt.docID() < doc) { - docIt.advance(doc); + if (scoreIt.docID() < doc) { + scoreIt.advance(doc); } // aggregations should only be replayed on matching documents - assert docIt.docID() == doc; + assert scoreIt.docID() == doc; } leafCollector.collect(doc, rebasedBucket); } } } - collector.postCollection(); } diff --git a/server/src/main/java/org/elasticsearch/search/aggregations/bucket/BucketsAggregator.java b/server/src/main/java/org/elasticsearch/search/aggregations/bucket/BucketsAggregator.java index 71dacc698bee6..a4ef4286447c1 100644 --- a/server/src/main/java/org/elasticsearch/search/aggregations/bucket/BucketsAggregator.java +++ b/server/src/main/java/org/elasticsearch/search/aggregations/bucket/BucketsAggregator.java @@ -90,7 +90,9 @@ public final void mergeBuckets(long[] mergeMap, long newNumBuckets) { docCounts.fill(0, newNumBuckets, 0); for (int i = 0; i < oldDocCounts.size(); i++) { int docCount = oldDocCounts.get(i); - if (docCount != 0) { + + // Skip any in the map which have been "removed", signified with -1 + if (docCount != 0 && mergeMap[i] != -1) { docCounts.increment(mergeMap[i], docCount); } } diff --git a/server/src/main/java/org/elasticsearch/search/aggregations/bucket/InternalSingleBucketAggregation.java b/server/src/main/java/org/elasticsearch/search/aggregations/bucket/InternalSingleBucketAggregation.java index 5626c8f21859d..cf5ddc54884d0 100644 --- a/server/src/main/java/org/elasticsearch/search/aggregations/bucket/InternalSingleBucketAggregation.java +++ b/server/src/main/java/org/elasticsearch/search/aggregations/bucket/InternalSingleBucketAggregation.java @@ -137,14 +137,18 @@ public XContentBuilder doXContentBody(XContentBuilder builder, Params params) th } @Override - protected boolean doEquals(Object obj) { + public boolean equals(Object obj) { + if (this == obj) return true; + if (obj == null || getClass() != obj.getClass()) return false; + if (super.equals(obj) == false) return false; + InternalSingleBucketAggregation other = (InternalSingleBucketAggregation) obj; return Objects.equals(docCount, other.docCount) && Objects.equals(aggregations, other.aggregations); } @Override - protected int doHashCode() { - return Objects.hash(docCount, aggregations); + public int hashCode() { + return Objects.hash(super.hashCode(), docCount, aggregations); } } diff --git a/server/src/main/java/org/elasticsearch/search/aggregations/bucket/MergingBucketsDeferringCollector.java b/server/src/main/java/org/elasticsearch/search/aggregations/bucket/MergingBucketsDeferringCollector.java index 53049d0301c2d..bff5015846951 100644 --- a/server/src/main/java/org/elasticsearch/search/aggregations/bucket/MergingBucketsDeferringCollector.java +++ b/server/src/main/java/org/elasticsearch/search/aggregations/bucket/MergingBucketsDeferringCollector.java @@ -19,222 +19,106 @@ package org.elasticsearch.search.aggregations.bucket; -import org.apache.lucene.index.LeafReaderContext; -import org.apache.lucene.search.DocIdSetIterator; -import org.apache.lucene.search.ScoreMode; -import org.apache.lucene.search.Scorer; -import org.apache.lucene.search.Weight; import org.apache.lucene.util.packed.PackedInts; import org.apache.lucene.util.packed.PackedLongValues; -import org.elasticsearch.common.util.BigArrays; -import org.elasticsearch.common.util.LongHash; -import org.elasticsearch.search.aggregations.Aggregator; -import org.elasticsearch.search.aggregations.BucketCollector; -import org.elasticsearch.search.aggregations.InternalAggregation; -import org.elasticsearch.search.aggregations.LeafBucketCollector; -import org.elasticsearch.search.aggregations.MultiBucketCollector; import org.elasticsearch.search.internal.SearchContext; -import java.io.IOException; import java.util.ArrayList; import java.util.List; /** - * A specialization of {@link DeferringBucketCollector} that collects all + * A specialization of {@link BestBucketsDeferringCollector} that collects all * matches and then is able to replay a given subset of buckets. Exposes * mergeBuckets, which can be invoked by the aggregator when increasing the * rounding interval. */ -public class MergingBucketsDeferringCollector extends DeferringBucketCollector { - - List entries = new ArrayList<>(); - BucketCollector collector; - final SearchContext searchContext; - LeafReaderContext context; - PackedLongValues.Builder docDeltas; - PackedLongValues.Builder buckets; - long maxBucket = -1; - boolean finished = false; - LongHash selectedBuckets; - - public MergingBucketsDeferringCollector(SearchContext context) { - this.searchContext = context; - } - - @Override - public void setDeferredCollector(Iterable deferredCollectors) { - this.collector = MultiBucketCollector.wrap(deferredCollectors); - } - - @Override - public ScoreMode scoreMode() { - if (collector == null) { - throw new IllegalStateException(); - } - return collector.scoreMode(); - } - - @Override - public void preCollection() throws IOException { - collector.preCollection(); - } - - private void finishLeaf() { - if (context != null) { - entries.add(new Entry(context, docDeltas.build(), buckets.build())); - } - context = null; - docDeltas = null; - buckets = null; - } - - @Override - public LeafBucketCollector getLeafCollector(LeafReaderContext ctx) throws IOException { - finishLeaf(); - - context = ctx; - docDeltas = PackedLongValues.packedBuilder(PackedInts.DEFAULT); - buckets = PackedLongValues.packedBuilder(PackedInts.DEFAULT); - - return new LeafBucketCollector() { - int lastDoc = 0; - - @Override - public void collect(int doc, long bucket) { - docDeltas.add(doc - lastDoc); - buckets.add(bucket); - lastDoc = doc; - maxBucket = Math.max(maxBucket, bucket); - } - }; +public class MergingBucketsDeferringCollector extends BestBucketsDeferringCollector { + public MergingBucketsDeferringCollector(SearchContext context, boolean isGlobal) { + super(context, isGlobal); } +/** + * Merges/prunes the existing bucket ordinals and docDeltas according to the provided mergeMap. + * + * The mergeMap is an array where the index position represents the current bucket ordinal, and + * the value at that position represents the ordinal the bucket should be merged with. If + * the value is set to -1 it is removed entirely. + * + * For example, if the mergeMap [1,1,3,-1,3] is provided: + * - Buckets `0` and `1` will be merged to bucket ordinal `1` + * - Bucket `2` and `4` will be merged to ordinal `3` + * - Bucket `3` will be removed entirely + * + * This process rebuilds the ordinals and docDeltas according to the mergeMap, so it should + * not be called unless there are actually changes to be made, to avoid unnecessary work. + */ public void mergeBuckets(long[] mergeMap) { - List newEntries = new ArrayList<>(entries.size()); for (Entry sourceEntry : entries) { PackedLongValues.Builder newBuckets = PackedLongValues.packedBuilder(PackedInts.DEFAULT); + PackedLongValues.Builder newDocDeltas = PackedLongValues.packedBuilder(PackedInts.DEFAULT); + PackedLongValues.Iterator docDeltasItr = sourceEntry.docDeltas.iterator(); + + long lastGoodDelta = 0; for (PackedLongValues.Iterator itr = sourceEntry.buckets.iterator(); itr.hasNext();) { long bucket = itr.next(); - newBuckets.add(mergeMap[Math.toIntExact(bucket)]); + assert docDeltasItr.hasNext(); + long delta = docDeltasItr.next(); + + // Only merge in the ordinal if it hasn't been "removed", signified with -1 + long ordinal = mergeMap[Math.toIntExact(bucket)]; + + if (ordinal != -1) { + newBuckets.add(ordinal); + newDocDeltas.add(delta + lastGoodDelta); + lastGoodDelta = 0; + } else { + // we are skipping this ordinal, which means we need to accumulate the + // doc delta's since the last "good" delta + lastGoodDelta += delta; + } + } + // Only create an entry if this segment has buckets after merging + if (newBuckets.size() > 0) { + assert newDocDeltas.size() > 0 : "docDeltas was empty but we had buckets"; + newEntries.add(new Entry(sourceEntry.context, newDocDeltas.build(), newBuckets.build())); } - newEntries.add(new Entry(sourceEntry.context, sourceEntry.docDeltas, newBuckets.build())); } entries = newEntries; // if there are buckets that have been collected in the current segment // we need to update the bucket ordinals there too - if (buckets.size() > 0) { - PackedLongValues currentBuckets = buckets.build(); + if (bucketsBuilder != null && bucketsBuilder.size() > 0) { + PackedLongValues currentBuckets = bucketsBuilder.build(); PackedLongValues.Builder newBuckets = PackedLongValues.packedBuilder(PackedInts.DEFAULT); - for (PackedLongValues.Iterator itr = currentBuckets.iterator(); itr.hasNext();) { - long bucket = itr.next(); - newBuckets.add(mergeMap[Math.toIntExact(bucket)]); - } - buckets = newBuckets; - } - } - - @Override - public void postCollection() { - finishLeaf(); - finished = true; - } - - /** - * Replay the wrapped collector, but only on a selection of buckets. - */ - @Override - public void prepareSelectedBuckets(long... selectedBuckets) throws IOException { - if (finished == false) { - throw new IllegalStateException("Cannot replay yet, collection is not finished: postCollect() has not been called"); - } - if (this.selectedBuckets != null) { - throw new IllegalStateException("Already been replayed"); - } + PackedLongValues.Builder newDocDeltas = PackedLongValues.packedBuilder(PackedInts.DEFAULT); - final LongHash hash = new LongHash(selectedBuckets.length, BigArrays.NON_RECYCLING_INSTANCE); - for (long bucket : selectedBuckets) { - hash.add(bucket); - } - this.selectedBuckets = hash; - - boolean needsScores = collector.scoreMode().needsScores(); - Weight weight = null; - if (needsScores) { - weight = searchContext.searcher().createWeight( - searchContext.searcher().rewrite(searchContext.query()), - ScoreMode.COMPLETE, 1f); - } - for (Entry entry : entries) { - final LeafBucketCollector leafCollector = collector.getLeafCollector(entry.context); - DocIdSetIterator docIt = null; - if (needsScores && entry.docDeltas.size() > 0) { - Scorer scorer = weight.scorer(entry.context); - // We don't need to check if the scorer is null - // since we are sure that there are documents to replay - // (entry.docDeltas it not empty). - docIt = scorer.iterator(); - leafCollector.setScorer(scorer); - } - final PackedLongValues.Iterator docDeltaIterator = entry.docDeltas.iterator(); - final PackedLongValues.Iterator buckets = entry.buckets.iterator(); - int doc = 0; - for (long i = 0, end = entry.docDeltas.size(); i < end; ++i) { - doc += docDeltaIterator.next(); - final long bucket = buckets.next(); - final long rebasedBucket = hash.find(bucket); - if (rebasedBucket != -1) { - if (needsScores) { - if (docIt.docID() < doc) { - docIt.advance(doc); - } - // aggregations should only be replayed on matching - // documents - assert docIt.docID() == doc; - } - leafCollector.collect(doc, rebasedBucket); - } - } - } + // The current segment's deltas aren't built yet, so build to a temp object + PackedLongValues currentDeltas = docDeltasBuilder.build(); + PackedLongValues.Iterator docDeltasItr = currentDeltas.iterator(); - collector.postCollection(); - } - - /** - * Wrap the provided aggregator so that it behaves (almost) as if it had - * been collected directly. - */ - @Override - public Aggregator wrap(final Aggregator in) { - - return new WrappedAggregator(in) { - - @Override - public InternalAggregation buildAggregation(long bucket) throws IOException { - if (selectedBuckets == null) { - throw new IllegalStateException("Collection has not been replayed yet."); - } - final long rebasedBucket = selectedBuckets.find(bucket); - if (rebasedBucket == -1) { - throw new IllegalStateException("Cannot build for a bucket which has not been collected [" + bucket + "]"); + long lastGoodDelta = 0; + for (PackedLongValues.Iterator itr = currentBuckets.iterator(); itr.hasNext();) { + long bucket = itr.next(); + assert docDeltasItr.hasNext(); + long delta = docDeltasItr.next(); + long ordinal = mergeMap[Math.toIntExact(bucket)]; + + // Only merge in the ordinal if it hasn't been "removed", signified with -1 + if (ordinal != -1) { + newBuckets.add(ordinal); + newDocDeltas.add(delta + lastGoodDelta); + lastGoodDelta = 0; + } else { + // we are skipping this ordinal, which means we need to accumulate the + // doc delta's since the last "good" delta. + // The first is skipped because the original deltas are stored as offsets from first doc, + // not offsets from 0 + lastGoodDelta += delta; } - return in.buildAggregation(rebasedBucket); } - - }; - } - - private static class Entry { - final LeafReaderContext context; - final PackedLongValues docDeltas; - final PackedLongValues buckets; - - Entry(LeafReaderContext context, PackedLongValues docDeltas, PackedLongValues buckets) { - this.context = context; - this.docDeltas = docDeltas; - this.buckets = buckets; + docDeltasBuilder = newDocDeltas; + bucketsBuilder = newBuckets; } } - } diff --git a/server/src/main/java/org/elasticsearch/search/aggregations/bucket/adjacency/AdjacencyMatrixAggregationBuilder.java b/server/src/main/java/org/elasticsearch/search/aggregations/bucket/adjacency/AdjacencyMatrixAggregationBuilder.java index fa33d64a1fb91..6b0530eb62e07 100644 --- a/server/src/main/java/org/elasticsearch/search/aggregations/bucket/adjacency/AdjacencyMatrixAggregationBuilder.java +++ b/server/src/main/java/org/elasticsearch/search/aggregations/bucket/adjacency/AdjacencyMatrixAggregationBuilder.java @@ -196,7 +196,7 @@ public Map filters() { @Override - protected AggregatorFactory doBuild(SearchContext context, AggregatorFactory parent, Builder subFactoriesBuilder) + protected AggregatorFactory doBuild(SearchContext context, AggregatorFactory parent, Builder subFactoriesBuilder) throws IOException { int maxFilters = context.indexShard().indexSettings().getMaxAdjacencyMatrixFilters(); if (filters.size() > maxFilters){ @@ -230,12 +230,15 @@ protected XContentBuilder internalXContent(XContentBuilder builder, Params param } @Override - protected int doHashCode() { - return Objects.hash(filters, separator); + public int hashCode() { + return Objects.hash(super.hashCode(), filters, separator); } @Override - protected boolean doEquals(Object obj) { + public boolean equals(Object obj) { + if (this == obj) return true; + if (obj == null || getClass() != obj.getClass()) return false; + if (super.equals(obj) == false) return false; AdjacencyMatrixAggregationBuilder other = (AdjacencyMatrixAggregationBuilder) obj; return Objects.equals(filters, other.filters) && Objects.equals(separator, other.separator); } diff --git a/server/src/main/java/org/elasticsearch/search/aggregations/bucket/adjacency/AdjacencyMatrixAggregatorFactory.java b/server/src/main/java/org/elasticsearch/search/aggregations/bucket/adjacency/AdjacencyMatrixAggregatorFactory.java index 7be588bcb5607..541a8e482c634 100644 --- a/server/src/main/java/org/elasticsearch/search/aggregations/bucket/adjacency/AdjacencyMatrixAggregatorFactory.java +++ b/server/src/main/java/org/elasticsearch/search/aggregations/bucket/adjacency/AdjacencyMatrixAggregatorFactory.java @@ -34,14 +34,14 @@ import java.util.List; import java.util.Map; -public class AdjacencyMatrixAggregatorFactory extends AggregatorFactory { +public class AdjacencyMatrixAggregatorFactory extends AggregatorFactory { private final String[] keys; private final Weight[] weights; private final String separator; public AdjacencyMatrixAggregatorFactory(String name, List filters, String separator, - SearchContext context, AggregatorFactory parent, AggregatorFactories.Builder subFactories, + SearchContext context, AggregatorFactory parent, AggregatorFactories.Builder subFactories, Map metaData) throws IOException { super(name, context, parent, subFactories, metaData); IndexSearcher contextSearcher = context.searcher(); diff --git a/server/src/main/java/org/elasticsearch/search/aggregations/bucket/adjacency/InternalAdjacencyMatrix.java b/server/src/main/java/org/elasticsearch/search/aggregations/bucket/adjacency/InternalAdjacencyMatrix.java index c862f458939ed..46eac6ce55dda 100644 --- a/server/src/main/java/org/elasticsearch/search/aggregations/bucket/adjacency/InternalAdjacencyMatrix.java +++ b/server/src/main/java/org/elasticsearch/search/aggregations/bucket/adjacency/InternalAdjacencyMatrix.java @@ -239,12 +239,16 @@ public XContentBuilder doXContentBody(XContentBuilder builder, Params params) th } @Override - protected int doHashCode() { - return Objects.hash(buckets); + public int hashCode() { + return Objects.hash(super.hashCode(), buckets); } @Override - protected boolean doEquals(Object obj) { + public boolean equals(Object obj) { + if (this == obj) return true; + if (obj == null || getClass() != obj.getClass()) return false; + if (super.equals(obj) == false) return false; + InternalAdjacencyMatrix that = (InternalAdjacencyMatrix) obj; return Objects.equals(buckets, that.buckets); } diff --git a/server/src/main/java/org/elasticsearch/search/aggregations/bucket/composite/CompositeAggregationBuilder.java b/server/src/main/java/org/elasticsearch/search/aggregations/bucket/composite/CompositeAggregationBuilder.java index 809dec737ff2b..c91e40d4609f8 100644 --- a/server/src/main/java/org/elasticsearch/search/aggregations/bucket/composite/CompositeAggregationBuilder.java +++ b/server/src/main/java/org/elasticsearch/search/aggregations/bucket/composite/CompositeAggregationBuilder.java @@ -162,7 +162,7 @@ public int size() { * this aggregator or the instance of the parent's factory that is incompatible with * the composite aggregation. */ - private AggregatorFactory checkParentIsNullOrNested(AggregatorFactory factory) { + private AggregatorFactory checkParentIsNullOrNested(AggregatorFactory factory) { if (factory == null) { return null; } else if (factory instanceof NestedAggregatorFactory) { @@ -195,9 +195,9 @@ private static void validateSources(List> source } @Override - protected AggregatorFactory doBuild(SearchContext context, AggregatorFactory parent, + protected AggregatorFactory doBuild(SearchContext context, AggregatorFactory parent, AggregatorFactories.Builder subfactoriesBuilder) throws IOException { - AggregatorFactory invalid = checkParentIsNullOrNested(parent); + AggregatorFactory invalid = checkParentIsNullOrNested(parent); if (invalid != null) { throw new IllegalArgumentException("[composite] aggregation cannot be used with a parent aggregation of" + " type: [" + invalid.getClass().getSimpleName() + "]"); @@ -256,12 +256,15 @@ protected XContentBuilder internalXContent(XContentBuilder builder, Params param } @Override - protected int doHashCode() { - return Objects.hash(sources, size, after); + public int hashCode() { + return Objects.hash(super.hashCode(), sources, size, after); } @Override - protected boolean doEquals(Object obj) { + public boolean equals(Object obj) { + if (this == obj) return true; + if (obj == null || getClass() != obj.getClass()) return false; + if (super.equals(obj) == false) return false; CompositeAggregationBuilder other = (CompositeAggregationBuilder) obj; return size == other.size && Objects.equals(sources, other.sources) && diff --git a/server/src/main/java/org/elasticsearch/search/aggregations/bucket/composite/CompositeAggregationFactory.java b/server/src/main/java/org/elasticsearch/search/aggregations/bucket/composite/CompositeAggregationFactory.java index 2b2fa4fb7e3eb..d79a00ce4c011 100644 --- a/server/src/main/java/org/elasticsearch/search/aggregations/bucket/composite/CompositeAggregationFactory.java +++ b/server/src/main/java/org/elasticsearch/search/aggregations/bucket/composite/CompositeAggregationFactory.java @@ -29,12 +29,12 @@ import java.util.List; import java.util.Map; -class CompositeAggregationFactory extends AggregatorFactory { +class CompositeAggregationFactory extends AggregatorFactory { private final int size; private final CompositeValuesSourceConfig[] sources; private final CompositeKey afterKey; - CompositeAggregationFactory(String name, SearchContext context, AggregatorFactory parent, + CompositeAggregationFactory(String name, SearchContext context, AggregatorFactory parent, AggregatorFactories.Builder subFactoriesBuilder, Map metaData, int size, CompositeValuesSourceConfig[] sources, CompositeKey afterKey) throws IOException { super(name, context, parent, subFactoriesBuilder, metaData); diff --git a/server/src/main/java/org/elasticsearch/search/aggregations/bucket/composite/CompositeValuesSourceBuilder.java b/server/src/main/java/org/elasticsearch/search/aggregations/bucket/composite/CompositeValuesSourceBuilder.java index 5c028e8f124b9..dc7331aa58745 100644 --- a/server/src/main/java/org/elasticsearch/search/aggregations/bucket/composite/CompositeValuesSourceBuilder.java +++ b/server/src/main/java/org/elasticsearch/search/aggregations/bucket/composite/CompositeValuesSourceBuilder.java @@ -117,12 +117,10 @@ public final XContentBuilder toXContent(XContentBuilder builder, Params params) } @Override - public final int hashCode() { - return Objects.hash(field, missingBucket, script, valueType, order, format, innerHashCode()); + public int hashCode() { + return Objects.hash(field, missingBucket, script, valueType, order, format); } - protected abstract int innerHashCode(); - @Override public boolean equals(Object o) { if (this == o) return true; @@ -135,12 +133,9 @@ public boolean equals(Object o) { Objects.equals(valueType, that.valueType()) && Objects.equals(missingBucket, that.missingBucket()) && Objects.equals(order, that.order()) && - Objects.equals(format, that.format()) && - innerEquals(that); + Objects.equals(format, that.format()); } - protected abstract boolean innerEquals(AB builder); - public String name() { return name; } diff --git a/server/src/main/java/org/elasticsearch/search/aggregations/bucket/composite/DateHistogramValuesSourceBuilder.java b/server/src/main/java/org/elasticsearch/search/aggregations/bucket/composite/DateHistogramValuesSourceBuilder.java index bb7632278de91..71cc5fcd4b2b0 100644 --- a/server/src/main/java/org/elasticsearch/search/aggregations/bucket/composite/DateHistogramValuesSourceBuilder.java +++ b/server/src/main/java/org/elasticsearch/search/aggregations/bucket/composite/DateHistogramValuesSourceBuilder.java @@ -97,12 +97,16 @@ protected void doXContentBody(XContentBuilder builder, Params params) throws IOE } @Override - protected int innerHashCode() { - return Objects.hash(dateHistogramInterval, timeZone); + public int hashCode() { + return Objects.hash(super.hashCode(), dateHistogramInterval, timeZone); } @Override - protected boolean innerEquals(DateHistogramValuesSourceBuilder other) { + public boolean equals(Object obj) { + if (this == obj) return true; + if (obj == null || getClass() != obj.getClass()) return false; + if (super.equals(obj) == false) return false; + DateHistogramValuesSourceBuilder other = (DateHistogramValuesSourceBuilder) obj; return Objects.equals(dateHistogramInterval, other.dateHistogramInterval) && Objects.equals(timeZone, other.timeZone); } diff --git a/server/src/main/java/org/elasticsearch/search/aggregations/bucket/composite/HistogramValuesSourceBuilder.java b/server/src/main/java/org/elasticsearch/search/aggregations/bucket/composite/HistogramValuesSourceBuilder.java index 75e2b8a3aaede..6ec1c76d459ab 100644 --- a/server/src/main/java/org/elasticsearch/search/aggregations/bucket/composite/HistogramValuesSourceBuilder.java +++ b/server/src/main/java/org/elasticsearch/search/aggregations/bucket/composite/HistogramValuesSourceBuilder.java @@ -73,12 +73,16 @@ protected void doXContentBody(XContentBuilder builder, Params params) throws IOE } @Override - protected int innerHashCode() { - return Objects.hash(interval); + public int hashCode() { + return Objects.hash(super.hashCode(), interval); } @Override - protected boolean innerEquals(HistogramValuesSourceBuilder other) { + public boolean equals(Object obj) { + if (this == obj) return true; + if (obj == null || getClass() != obj.getClass()) return false; + if (super.equals(obj) == false) return false; + HistogramValuesSourceBuilder other = (HistogramValuesSourceBuilder) obj; return Objects.equals(interval, other.interval); } diff --git a/server/src/main/java/org/elasticsearch/search/aggregations/bucket/composite/InternalComposite.java b/server/src/main/java/org/elasticsearch/search/aggregations/bucket/composite/InternalComposite.java index e6a7edd8c217d..ac51301766cb4 100644 --- a/server/src/main/java/org/elasticsearch/search/aggregations/bucket/composite/InternalComposite.java +++ b/server/src/main/java/org/elasticsearch/search/aggregations/bucket/composite/InternalComposite.java @@ -182,7 +182,11 @@ public InternalAggregation doReduce(List aggregations, Redu } @Override - protected boolean doEquals(Object obj) { + public boolean equals(Object obj) { + if (this == obj) return true; + if (obj == null || getClass() != obj.getClass()) return false; + if (super.equals(obj) == false) return false; + InternalComposite that = (InternalComposite) obj; return Objects.equals(size, that.size) && Objects.equals(buckets, that.buckets) && @@ -191,8 +195,8 @@ protected boolean doEquals(Object obj) { } @Override - protected int doHashCode() { - return Objects.hash(size, buckets, afterKey, Arrays.hashCode(reverseMuls)); + public int hashCode() { + return Objects.hash(super.hashCode(), size, buckets, afterKey, Arrays.hashCode(reverseMuls)); } private static class BucketIterator implements Comparable { diff --git a/server/src/main/java/org/elasticsearch/search/aggregations/bucket/composite/TermsValuesSourceBuilder.java b/server/src/main/java/org/elasticsearch/search/aggregations/bucket/composite/TermsValuesSourceBuilder.java index 3ef871f59b93c..061b8c5f5dec3 100644 --- a/server/src/main/java/org/elasticsearch/search/aggregations/bucket/composite/TermsValuesSourceBuilder.java +++ b/server/src/main/java/org/elasticsearch/search/aggregations/bucket/composite/TermsValuesSourceBuilder.java @@ -64,16 +64,6 @@ protected void innerWriteTo(StreamOutput out) throws IOException {} @Override protected void doXContentBody(XContentBuilder builder, Params params) throws IOException {} - @Override - protected int innerHashCode() { - return 0; - } - - @Override - protected boolean innerEquals(TermsValuesSourceBuilder builder) { - return true; - } - @Override public String type() { return TYPE; diff --git a/server/src/main/java/org/elasticsearch/search/aggregations/bucket/filter/FilterAggregationBuilder.java b/server/src/main/java/org/elasticsearch/search/aggregations/bucket/filter/FilterAggregationBuilder.java index 71c1c8fa3deb2..e53e8ed3fc2c4 100644 --- a/server/src/main/java/org/elasticsearch/search/aggregations/bucket/filter/FilterAggregationBuilder.java +++ b/server/src/main/java/org/elasticsearch/search/aggregations/bucket/filter/FilterAggregationBuilder.java @@ -93,7 +93,7 @@ protected AggregationBuilder doRewrite(QueryRewriteContext queryShardContext) th } @Override - protected AggregatorFactory doBuild(SearchContext context, AggregatorFactory parent, + protected AggregatorFactory doBuild(SearchContext context, AggregatorFactory parent, AggregatorFactories.Builder subFactoriesBuilder) throws IOException { return new FilterAggregatorFactory(name, filter, context, parent, subFactoriesBuilder, metaData); } @@ -112,12 +112,15 @@ public static FilterAggregationBuilder parse(String aggregationName, XContentPar } @Override - protected int doHashCode() { - return Objects.hash(filter); + public int hashCode() { + return Objects.hash(super.hashCode(), filter); } @Override - protected boolean doEquals(Object obj) { + public boolean equals(Object obj) { + if (this == obj) return true; + if (obj == null || getClass() != obj.getClass()) return false; + if (super.equals(obj) == false) return false; FilterAggregationBuilder other = (FilterAggregationBuilder) obj; return Objects.equals(filter, other.filter); } diff --git a/server/src/main/java/org/elasticsearch/search/aggregations/bucket/filter/FilterAggregatorFactory.java b/server/src/main/java/org/elasticsearch/search/aggregations/bucket/filter/FilterAggregatorFactory.java index 04dd8d3a53cea..c9773ef632616 100644 --- a/server/src/main/java/org/elasticsearch/search/aggregations/bucket/filter/FilterAggregatorFactory.java +++ b/server/src/main/java/org/elasticsearch/search/aggregations/bucket/filter/FilterAggregatorFactory.java @@ -35,13 +35,13 @@ import java.util.List; import java.util.Map; -public class FilterAggregatorFactory extends AggregatorFactory { +public class FilterAggregatorFactory extends AggregatorFactory { private Weight weight; private Query filter; public FilterAggregatorFactory(String name, QueryBuilder filterBuilder, SearchContext context, - AggregatorFactory parent, AggregatorFactories.Builder subFactoriesBuilder, Map metaData) throws IOException { + AggregatorFactory parent, AggregatorFactories.Builder subFactoriesBuilder, Map metaData) throws IOException { super(name, context, parent, subFactoriesBuilder, metaData); filter = filterBuilder.toQuery(context.getQueryShardContext()); } diff --git a/server/src/main/java/org/elasticsearch/search/aggregations/bucket/filter/FiltersAggregationBuilder.java b/server/src/main/java/org/elasticsearch/search/aggregations/bucket/filter/FiltersAggregationBuilder.java index 54dfc301b2dbc..add11f16483ec 100644 --- a/server/src/main/java/org/elasticsearch/search/aggregations/bucket/filter/FiltersAggregationBuilder.java +++ b/server/src/main/java/org/elasticsearch/search/aggregations/bucket/filter/FiltersAggregationBuilder.java @@ -220,7 +220,7 @@ protected AggregationBuilder doRewrite(QueryRewriteContext queryShardContext) th } @Override - protected AggregatorFactory doBuild(SearchContext context, AggregatorFactory parent, Builder subFactoriesBuilder) + protected AggregatorFactory doBuild(SearchContext context, AggregatorFactory parent, Builder subFactoriesBuilder) throws IOException { return new FiltersAggregatorFactory(name, filters, keyed, otherBucket, otherBucketKey, context, parent, subFactoriesBuilder, metaData); @@ -331,12 +331,15 @@ public static FiltersAggregationBuilder parse(String aggregationName, XContentPa } @Override - protected int doHashCode() { - return Objects.hash(filters, keyed, otherBucket, otherBucketKey); + public int hashCode() { + return Objects.hash(super.hashCode(), filters, keyed, otherBucket, otherBucketKey); } @Override - protected boolean doEquals(Object obj) { + public boolean equals(Object obj) { + if (this == obj) return true; + if (obj == null || getClass() != obj.getClass()) return false; + if (super.equals(obj) == false) return false; FiltersAggregationBuilder other = (FiltersAggregationBuilder) obj; return Objects.equals(filters, other.filters) && Objects.equals(keyed, other.keyed) diff --git a/server/src/main/java/org/elasticsearch/search/aggregations/bucket/filter/FiltersAggregatorFactory.java b/server/src/main/java/org/elasticsearch/search/aggregations/bucket/filter/FiltersAggregatorFactory.java index cc299765621aa..4e9eec14c0089 100644 --- a/server/src/main/java/org/elasticsearch/search/aggregations/bucket/filter/FiltersAggregatorFactory.java +++ b/server/src/main/java/org/elasticsearch/search/aggregations/bucket/filter/FiltersAggregatorFactory.java @@ -35,7 +35,7 @@ import java.util.List; import java.util.Map; -public class FiltersAggregatorFactory extends AggregatorFactory { +public class FiltersAggregatorFactory extends AggregatorFactory { private final String[] keys; private final Query[] filters; @@ -45,7 +45,7 @@ public class FiltersAggregatorFactory extends AggregatorFactory filters, boolean keyed, boolean otherBucket, - String otherBucketKey, SearchContext context, AggregatorFactory parent, AggregatorFactories.Builder subFactories, + String otherBucketKey, SearchContext context, AggregatorFactory parent, AggregatorFactories.Builder subFactories, Map metaData) throws IOException { super(name, context, parent, subFactories, metaData); this.keyed = keyed; diff --git a/server/src/main/java/org/elasticsearch/search/aggregations/bucket/filter/InternalFilters.java b/server/src/main/java/org/elasticsearch/search/aggregations/bucket/filter/InternalFilters.java index f6ebfa459c02d..d866754c5626e 100644 --- a/server/src/main/java/org/elasticsearch/search/aggregations/bucket/filter/InternalFilters.java +++ b/server/src/main/java/org/elasticsearch/search/aggregations/bucket/filter/InternalFilters.java @@ -251,12 +251,16 @@ public XContentBuilder doXContentBody(XContentBuilder builder, Params params) th } @Override - protected int doHashCode() { - return Objects.hash(buckets, keyed); + public int hashCode() { + return Objects.hash(super.hashCode(), buckets, keyed); } @Override - protected boolean doEquals(Object obj) { + public boolean equals(Object obj) { + if (this == obj) return true; + if (obj == null || getClass() != obj.getClass()) return false; + if (super.equals(obj) == false) return false; + InternalFilters that = (InternalFilters) obj; return Objects.equals(buckets, that.buckets) && Objects.equals(keyed, that.keyed); diff --git a/server/src/main/java/org/elasticsearch/search/aggregations/bucket/geogrid/GeoGridAggregationBuilder.java b/server/src/main/java/org/elasticsearch/search/aggregations/bucket/geogrid/GeoGridAggregationBuilder.java index 76fa6621d01cc..bae95c84c00ed 100644 --- a/server/src/main/java/org/elasticsearch/search/aggregations/bucket/geogrid/GeoGridAggregationBuilder.java +++ b/server/src/main/java/org/elasticsearch/search/aggregations/bucket/geogrid/GeoGridAggregationBuilder.java @@ -108,9 +108,9 @@ protected void innerWriteTo(StreamOutput out) throws IOException { /** * Creates a new instance of the {@link ValuesSourceAggregatorFactory}-derived class specific to the geo aggregation. */ - protected abstract ValuesSourceAggregatorFactory createFactory( + protected abstract ValuesSourceAggregatorFactory createFactory( String name, ValuesSourceConfig config, int precision, int requiredSize, int shardSize, - SearchContext context, AggregatorFactory parent, Builder subFactoriesBuilder, Map metaData + SearchContext context, AggregatorFactory parent, Builder subFactoriesBuilder, Map metaData ) throws IOException; public int precision() { @@ -144,8 +144,8 @@ public int shardSize() { } @Override - protected ValuesSourceAggregatorFactory innerBuild(SearchContext context, - ValuesSourceConfig config, AggregatorFactory parent, Builder subFactoriesBuilder) + protected ValuesSourceAggregatorFactory innerBuild(SearchContext context, + ValuesSourceConfig config, AggregatorFactory parent, Builder subFactoriesBuilder) throws IOException { int shardSize = this.shardSize; @@ -180,22 +180,18 @@ protected XContentBuilder doXContentBody(XContentBuilder builder, Params params) } @Override - protected boolean innerEquals(Object obj) { + public boolean equals(Object obj) { + if (this == obj) return true; + if (obj == null || getClass() != obj.getClass()) return false; + if (super.equals(obj) == false) return false; GeoGridAggregationBuilder other = (GeoGridAggregationBuilder) obj; - if (precision != other.precision) { - return false; - } - if (requiredSize != other.requiredSize) { - return false; - } - if (shardSize != other.shardSize) { - return false; - } - return true; + return precision == other.precision + && requiredSize == other.requiredSize + && shardSize == other.shardSize; } @Override - protected int innerHashCode() { - return Objects.hash(precision, requiredSize, shardSize); + public int hashCode() { + return Objects.hash(super.hashCode(), precision, requiredSize, shardSize); } } diff --git a/server/src/main/java/org/elasticsearch/search/aggregations/bucket/geogrid/GeoHashGridAggregationBuilder.java b/server/src/main/java/org/elasticsearch/search/aggregations/bucket/geogrid/GeoHashGridAggregationBuilder.java index 4e560e681c796..cd7e8d41c6419 100644 --- a/server/src/main/java/org/elasticsearch/search/aggregations/bucket/geogrid/GeoHashGridAggregationBuilder.java +++ b/server/src/main/java/org/elasticsearch/search/aggregations/bucket/geogrid/GeoHashGridAggregationBuilder.java @@ -59,9 +59,9 @@ public GeoGridAggregationBuilder precision(int precision) { } @Override - protected ValuesSourceAggregatorFactory createFactory( + protected ValuesSourceAggregatorFactory createFactory( String name, ValuesSourceConfig config, int precision, int requiredSize, int shardSize, - SearchContext context, AggregatorFactory parent, AggregatorFactories.Builder subFactoriesBuilder, + SearchContext context, AggregatorFactory parent, AggregatorFactories.Builder subFactoriesBuilder, Map metaData ) throws IOException { return new GeoHashGridAggregatorFactory(name, config, precision, requiredSize, shardSize, context, parent, diff --git a/server/src/main/java/org/elasticsearch/search/aggregations/bucket/geogrid/GeoHashGridAggregatorFactory.java b/server/src/main/java/org/elasticsearch/search/aggregations/bucket/geogrid/GeoHashGridAggregatorFactory.java index 54253b9dea25c..7f7127d13f041 100644 --- a/server/src/main/java/org/elasticsearch/search/aggregations/bucket/geogrid/GeoHashGridAggregatorFactory.java +++ b/server/src/main/java/org/elasticsearch/search/aggregations/bucket/geogrid/GeoHashGridAggregatorFactory.java @@ -37,14 +37,14 @@ import java.util.List; import java.util.Map; -public class GeoHashGridAggregatorFactory extends ValuesSourceAggregatorFactory { +public class GeoHashGridAggregatorFactory extends ValuesSourceAggregatorFactory { private final int precision; private final int requiredSize; private final int shardSize; GeoHashGridAggregatorFactory(String name, ValuesSourceConfig config, int precision, int requiredSize, - int shardSize, SearchContext context, AggregatorFactory parent, AggregatorFactories.Builder subFactoriesBuilder, + int shardSize, SearchContext context, AggregatorFactory parent, AggregatorFactories.Builder subFactoriesBuilder, Map metaData) throws IOException { super(name, config, context, parent, subFactoriesBuilder, metaData); this.precision = precision; diff --git a/server/src/main/java/org/elasticsearch/search/aggregations/bucket/geogrid/GeoTileGridAggregationBuilder.java b/server/src/main/java/org/elasticsearch/search/aggregations/bucket/geogrid/GeoTileGridAggregationBuilder.java index 33efeeb5d38b6..374b7ddf70735 100644 --- a/server/src/main/java/org/elasticsearch/search/aggregations/bucket/geogrid/GeoTileGridAggregationBuilder.java +++ b/server/src/main/java/org/elasticsearch/search/aggregations/bucket/geogrid/GeoTileGridAggregationBuilder.java @@ -58,9 +58,9 @@ public GeoGridAggregationBuilder precision(int precision) { } @Override - protected ValuesSourceAggregatorFactory createFactory( + protected ValuesSourceAggregatorFactory createFactory( String name, ValuesSourceConfig config, int precision, int requiredSize, int shardSize, - SearchContext context, AggregatorFactory parent, AggregatorFactories.Builder subFactoriesBuilder, + SearchContext context, AggregatorFactory parent, AggregatorFactories.Builder subFactoriesBuilder, Map metaData ) throws IOException { return new GeoTileGridAggregatorFactory(name, config, precision, requiredSize, shardSize, context, parent, diff --git a/server/src/main/java/org/elasticsearch/search/aggregations/bucket/geogrid/GeoTileGridAggregatorFactory.java b/server/src/main/java/org/elasticsearch/search/aggregations/bucket/geogrid/GeoTileGridAggregatorFactory.java index 87077a89d6c23..34a1bdc35ca0c 100644 --- a/server/src/main/java/org/elasticsearch/search/aggregations/bucket/geogrid/GeoTileGridAggregatorFactory.java +++ b/server/src/main/java/org/elasticsearch/search/aggregations/bucket/geogrid/GeoTileGridAggregatorFactory.java @@ -36,14 +36,14 @@ import java.util.List; import java.util.Map; -public class GeoTileGridAggregatorFactory extends ValuesSourceAggregatorFactory { +public class GeoTileGridAggregatorFactory extends ValuesSourceAggregatorFactory { private final int precision; private final int requiredSize; private final int shardSize; GeoTileGridAggregatorFactory(String name, ValuesSourceConfig config, int precision, int requiredSize, - int shardSize, SearchContext context, AggregatorFactory parent, + int shardSize, SearchContext context, AggregatorFactory parent, AggregatorFactories.Builder subFactoriesBuilder, Map metaData ) throws IOException { super(name, config, context, parent, subFactoriesBuilder, metaData); diff --git a/server/src/main/java/org/elasticsearch/search/aggregations/bucket/geogrid/InternalGeoGrid.java b/server/src/main/java/org/elasticsearch/search/aggregations/bucket/geogrid/InternalGeoGrid.java index 36908f51d70de..ae77e69b3e1ec 100644 --- a/server/src/main/java/org/elasticsearch/search/aggregations/bucket/geogrid/InternalGeoGrid.java +++ b/server/src/main/java/org/elasticsearch/search/aggregations/bucket/geogrid/InternalGeoGrid.java @@ -133,15 +133,19 @@ int getRequiredSize() { } @Override - protected int doHashCode() { - return Objects.hash(requiredSize, buckets); + public int hashCode() { + return Objects.hash(super.hashCode(), requiredSize, buckets); } @Override - protected boolean doEquals(Object obj) { + public boolean equals(Object obj) { + if (this == obj) return true; + if (obj == null || getClass() != obj.getClass()) return false; + if (super.equals(obj) == false) return false; + InternalGeoGrid other = (InternalGeoGrid) obj; - return Objects.equals(requiredSize, other.requiredSize) && - Objects.equals(buckets, other.buckets); + return Objects.equals(requiredSize, other.requiredSize) + && Objects.equals(buckets, other.buckets); } } diff --git a/server/src/main/java/org/elasticsearch/search/aggregations/bucket/global/GlobalAggregationBuilder.java b/server/src/main/java/org/elasticsearch/search/aggregations/bucket/global/GlobalAggregationBuilder.java index cae359d9ecb1a..a6e85a031573a 100644 --- a/server/src/main/java/org/elasticsearch/search/aggregations/bucket/global/GlobalAggregationBuilder.java +++ b/server/src/main/java/org/elasticsearch/search/aggregations/bucket/global/GlobalAggregationBuilder.java @@ -61,7 +61,7 @@ protected void doWriteTo(StreamOutput out) throws IOException { } @Override - protected AggregatorFactory doBuild(SearchContext context, AggregatorFactory parent, Builder subFactoriesBuilder) + protected AggregatorFactory doBuild(SearchContext context, AggregatorFactory parent, Builder subFactoriesBuilder) throws IOException { return new GlobalAggregatorFactory(name, context, parent, subFactoriesBuilder, metaData); } @@ -78,16 +78,6 @@ public static GlobalAggregationBuilder parse(String aggregationName, XContentPar return new GlobalAggregationBuilder(aggregationName); } - @Override - protected boolean doEquals(Object obj) { - return true; - } - - @Override - protected int doHashCode() { - return 0; - } - @Override public String getType() { return NAME; diff --git a/server/src/main/java/org/elasticsearch/search/aggregations/bucket/global/GlobalAggregatorFactory.java b/server/src/main/java/org/elasticsearch/search/aggregations/bucket/global/GlobalAggregatorFactory.java index b11ffde6d7bc4..ffe66894d103a 100644 --- a/server/src/main/java/org/elasticsearch/search/aggregations/bucket/global/GlobalAggregatorFactory.java +++ b/server/src/main/java/org/elasticsearch/search/aggregations/bucket/global/GlobalAggregatorFactory.java @@ -30,9 +30,9 @@ import java.util.List; import java.util.Map; -public class GlobalAggregatorFactory extends AggregatorFactory { +public class GlobalAggregatorFactory extends AggregatorFactory { - public GlobalAggregatorFactory(String name, SearchContext context, AggregatorFactory parent, + public GlobalAggregatorFactory(String name, SearchContext context, AggregatorFactory parent, AggregatorFactories.Builder subFactories, Map metaData) throws IOException { super(name, context, parent, subFactories, metaData); } diff --git a/server/src/main/java/org/elasticsearch/search/aggregations/bucket/histogram/AutoDateHistogramAggregationBuilder.java b/server/src/main/java/org/elasticsearch/search/aggregations/bucket/histogram/AutoDateHistogramAggregationBuilder.java index 794ce066ed76e..1219fc24606fd 100644 --- a/server/src/main/java/org/elasticsearch/search/aggregations/bucket/histogram/AutoDateHistogramAggregationBuilder.java +++ b/server/src/main/java/org/elasticsearch/search/aggregations/bucket/histogram/AutoDateHistogramAggregationBuilder.java @@ -19,6 +19,7 @@ package org.elasticsearch.search.aggregations.bucket.histogram; +import org.elasticsearch.Version; import org.elasticsearch.common.ParseField; import org.elasticsearch.common.Rounding; import org.elasticsearch.common.io.stream.StreamInput; @@ -48,42 +49,65 @@ import java.util.Map; import java.util.Objects; +import static java.util.Map.entry; + public class AutoDateHistogramAggregationBuilder extends ValuesSourceAggregationBuilder { public static final String NAME = "auto_date_histogram"; private static final ParseField NUM_BUCKETS_FIELD = new ParseField("buckets"); + private static final ParseField MINIMUM_INTERVAL_FIELD = new ParseField("minimum_interval"); private static final ObjectParser PARSER; static { PARSER = new ObjectParser<>(AutoDateHistogramAggregationBuilder.NAME); ValuesSourceParserHelper.declareNumericFields(PARSER, true, true, true); - PARSER.declareInt(AutoDateHistogramAggregationBuilder::setNumBuckets, NUM_BUCKETS_FIELD); + PARSER.declareStringOrNull(AutoDateHistogramAggregationBuilder::setMinimumIntervalExpression, MINIMUM_INTERVAL_FIELD); } + public static final Map ALLOWED_INTERVALS = Map.ofEntries( + entry(Rounding.DateTimeUnit.YEAR_OF_CENTURY, "year"), + entry(Rounding.DateTimeUnit.MONTH_OF_YEAR, "month"), + entry(Rounding.DateTimeUnit.DAY_OF_MONTH, "day"), + entry( Rounding.DateTimeUnit.HOUR_OF_DAY, "hour"), + entry(Rounding.DateTimeUnit.MINUTES_OF_HOUR, "minute"), + entry(Rounding.DateTimeUnit.SECOND_OF_MINUTE, "second") + ); + /** * * Build roundings, computed dynamically as roundings are time zone dependent. * The current implementation probably should not be invoked in a tight loop. * @return Array of RoundingInfo */ - static RoundingInfo[] buildRoundings(ZoneId timeZone) { + static RoundingInfo[] buildRoundings(ZoneId timeZone, String minimumInterval) { + + int indexToSliceFrom = 0; + RoundingInfo[] roundings = new RoundingInfo[6]; - roundings[0] = new RoundingInfo(createRounding(Rounding.DateTimeUnit.SECOND_OF_MINUTE, timeZone), - 1000L, "s", 1, 5, 10, 30); - roundings[1] = new RoundingInfo(createRounding(Rounding.DateTimeUnit.MINUTES_OF_HOUR, timeZone), + roundings[0] = new RoundingInfo(Rounding.DateTimeUnit.SECOND_OF_MINUTE, + timeZone, 1000L, "s",1, 5, 10, 30); + roundings[1] = new RoundingInfo(Rounding.DateTimeUnit.MINUTES_OF_HOUR, timeZone, 60 * 1000L, "m", 1, 5, 10, 30); - roundings[2] = new RoundingInfo(createRounding(Rounding.DateTimeUnit.HOUR_OF_DAY, timeZone), - 60 * 60 * 1000L, "h",1, 3, 12); - roundings[3] = new RoundingInfo(createRounding(Rounding.DateTimeUnit.DAY_OF_MONTH, timeZone), + roundings[2] = new RoundingInfo(Rounding.DateTimeUnit.HOUR_OF_DAY, timeZone, + 60 * 60 * 1000L, "h", 1, 3, 12); + roundings[3] = new RoundingInfo(Rounding.DateTimeUnit.DAY_OF_MONTH, timeZone, 24 * 60 * 60 * 1000L, "d", 1, 7); - roundings[4] = new RoundingInfo(createRounding(Rounding.DateTimeUnit.MONTH_OF_YEAR, timeZone), + roundings[4] = new RoundingInfo(Rounding.DateTimeUnit.MONTH_OF_YEAR, timeZone, 30 * 24 * 60 * 60 * 1000L, "M", 1, 3); - roundings[5] = new RoundingInfo(createRounding(Rounding.DateTimeUnit.YEAR_OF_CENTURY, timeZone), + roundings[5] = new RoundingInfo(Rounding.DateTimeUnit.YEAR_OF_CENTURY, timeZone, 365 * 24 * 60 * 60 * 1000L, "y", 1, 5, 10, 20, 50, 100); - return roundings; + + for (int i = 0; i < roundings.length; i++) { + RoundingInfo roundingInfo = roundings[i]; + if (roundingInfo.getDateTimeUnit().equals(minimumInterval)) { + indexToSliceFrom = i; + break; + } + } + return Arrays.copyOfRange(roundings, indexToSliceFrom, roundings.length); } public static AutoDateHistogramAggregationBuilder parse(String aggregationName, XContentParser parser) throws IOException { @@ -92,6 +116,22 @@ public static AutoDateHistogramAggregationBuilder parse(String aggregationName, private int numBuckets = 10; + private String minimumIntervalExpression; + + public String getMinimumIntervalExpression() { + return minimumIntervalExpression; + } + + public AutoDateHistogramAggregationBuilder setMinimumIntervalExpression(String minimumIntervalExpression) { + if (minimumIntervalExpression != null && !ALLOWED_INTERVALS.containsValue(minimumIntervalExpression)) { + throw new IllegalArgumentException(MINIMUM_INTERVAL_FIELD.getPreferredName() + + " must be one of [" + ALLOWED_INTERVALS.values().toString() + "]"); + } + this.minimumIntervalExpression = minimumIntervalExpression; + return this; + } + + /** Create a new builder with the given name. */ public AutoDateHistogramAggregationBuilder(String name) { super(name, ValuesSourceType.NUMERIC, ValueType.DATE); @@ -101,12 +141,16 @@ public AutoDateHistogramAggregationBuilder(String name) { public AutoDateHistogramAggregationBuilder(StreamInput in) throws IOException { super(in, ValuesSourceType.NUMERIC, ValueType.DATE); numBuckets = in.readVInt(); + if (in.getVersion().onOrAfter(Version.V_7_3_0)) { + minimumIntervalExpression = in.readOptionalString(); + } } protected AutoDateHistogramAggregationBuilder(AutoDateHistogramAggregationBuilder clone, Builder factoriesBuilder, Map metaData) { super(clone, factoriesBuilder, metaData); this.numBuckets = clone.numBuckets; + this.minimumIntervalExpression = clone.minimumIntervalExpression; } @Override @@ -117,6 +161,9 @@ protected AggregationBuilder shallowCopy(Builder factoriesBuilder, Map innerBuild(SearchContext context, ValuesSourceConfig config, - AggregatorFactory parent, Builder subFactoriesBuilder) throws IOException { - RoundingInfo[] roundings = buildRoundings(timeZone()); + protected ValuesSourceAggregatorFactory innerBuild(SearchContext context, ValuesSourceConfig config, + AggregatorFactory parent, Builder subFactoriesBuilder) throws IOException { + RoundingInfo[] roundings = buildRoundings(timeZone(), getMinimumIntervalExpression()); int maxRoundingInterval = Arrays.stream(roundings,0, roundings.length-1) .map(rounding -> rounding.innerIntervals) .flatMapToInt(Arrays::stream) @@ -152,7 +199,9 @@ public int getNumBuckets() { throw new IllegalArgumentException(NUM_BUCKETS_FIELD.getPreferredName()+ " must be less than " + bucketCeiling); } - return new AutoDateHistogramAggregatorFactory(name, config, numBuckets, roundings, context, parent, subFactoriesBuilder, metaData); + return new AutoDateHistogramAggregatorFactory(name, config, numBuckets, roundings, context, parent, + subFactoriesBuilder, + metaData); } static Rounding createRounding(Rounding.DateTimeUnit interval, ZoneId timeZone) { @@ -167,18 +216,22 @@ static Rounding createRounding(Rounding.DateTimeUnit interval, ZoneId timeZone) @Override protected XContentBuilder doXContentBody(XContentBuilder builder, Params params) throws IOException { builder.field(NUM_BUCKETS_FIELD.getPreferredName(), numBuckets); + builder.field(MINIMUM_INTERVAL_FIELD.getPreferredName(), minimumIntervalExpression); return builder; } @Override - protected int innerHashCode() { - return Objects.hash(numBuckets); + public int hashCode() { + return Objects.hash(super.hashCode(), numBuckets, minimumIntervalExpression); } @Override - protected boolean innerEquals(Object obj) { + public boolean equals(Object obj) { + if (this == obj) return true; + if (obj == null || getClass() != obj.getClass()) return false; + if (super.equals(obj) == false) return false; AutoDateHistogramAggregationBuilder other = (AutoDateHistogramAggregationBuilder) obj; - return Objects.equals(numBuckets, other.numBuckets); + return Objects.equals(numBuckets, other.numBuckets) && Objects.equals(minimumIntervalExpression, other.minimumIntervalExpression); } public static class RoundingInfo implements Writeable { @@ -186,12 +239,22 @@ public static class RoundingInfo implements Writeable { final int[] innerIntervals; final long roughEstimateDurationMillis; final String unitAbbreviation; - - public RoundingInfo(Rounding rounding, long roughEstimateDurationMillis, String unitAbbreviation, int... innerIntervals) { - this.rounding = rounding; + final String dateTimeUnit; + + public RoundingInfo(Rounding.DateTimeUnit dateTimeUnit, + ZoneId timeZone, + long roughEstimateDurationMillis, + String unitAbbreviation, + int... innerIntervals) { + this.rounding = createRounding(dateTimeUnit, timeZone); this.roughEstimateDurationMillis = roughEstimateDurationMillis; this.unitAbbreviation = unitAbbreviation; this.innerIntervals = innerIntervals; + Objects.requireNonNull(dateTimeUnit, "dateTimeUnit cannot be null"); + if (!ALLOWED_INTERVALS.containsKey(dateTimeUnit)) { + throw new IllegalArgumentException("dateTimeUnit must be one of " + ALLOWED_INTERVALS.keySet().toString()); + } + this.dateTimeUnit = ALLOWED_INTERVALS.get(dateTimeUnit); } public RoundingInfo(StreamInput in) throws IOException { @@ -199,6 +262,7 @@ public RoundingInfo(StreamInput in) throws IOException { roughEstimateDurationMillis = in.readVLong(); innerIntervals = in.readIntArray(); unitAbbreviation = in.readString(); + dateTimeUnit = in.readString(); } @Override @@ -207,19 +271,22 @@ public void writeTo(StreamOutput out) throws IOException { out.writeVLong(roughEstimateDurationMillis); out.writeIntArray(innerIntervals); out.writeString(unitAbbreviation); + out.writeString(dateTimeUnit); } public int getMaximumInnerInterval() { return innerIntervals[innerIntervals.length - 1]; } + public String getDateTimeUnit() { return this.dateTimeUnit; } + public long getRoughEstimateDurationMillis() { return roughEstimateDurationMillis; } @Override public int hashCode() { - return Objects.hash(rounding, Arrays.hashCode(innerIntervals)); + return Objects.hash(rounding, Arrays.hashCode(innerIntervals), dateTimeUnit); } @Override @@ -231,8 +298,10 @@ public boolean equals(Object obj) { return false; } RoundingInfo other = (RoundingInfo) obj; - return Objects.equals(rounding, other.rounding) && - Objects.deepEquals(innerIntervals, other.innerIntervals); + return Objects.equals(rounding, other.rounding) + && Objects.deepEquals(innerIntervals, other.innerIntervals) + && Objects.equals(dateTimeUnit, other.dateTimeUnit) + ; } } } diff --git a/server/src/main/java/org/elasticsearch/search/aggregations/bucket/histogram/AutoDateHistogramAggregator.java b/server/src/main/java/org/elasticsearch/search/aggregations/bucket/histogram/AutoDateHistogramAggregator.java index 1b982ea9deca2..b10507cd2ce65 100644 --- a/server/src/main/java/org/elasticsearch/search/aggregations/bucket/histogram/AutoDateHistogramAggregator.java +++ b/server/src/main/java/org/elasticsearch/search/aggregations/bucket/histogram/AutoDateHistogramAggregator.java @@ -93,7 +93,7 @@ protected boolean shouldDefer(Aggregator aggregator) { @Override public DeferringBucketCollector getDeferringCollector() { - deferringCollector = new MergingBucketsDeferringCollector(context); + deferringCollector = new MergingBucketsDeferringCollector(context, descendsFromGlobalAggregator(parent())); return deferringCollector; } diff --git a/server/src/main/java/org/elasticsearch/search/aggregations/bucket/histogram/AutoDateHistogramAggregatorFactory.java b/server/src/main/java/org/elasticsearch/search/aggregations/bucket/histogram/AutoDateHistogramAggregatorFactory.java index 051f2f9f6e7c7..c2ca3b15059df 100644 --- a/server/src/main/java/org/elasticsearch/search/aggregations/bucket/histogram/AutoDateHistogramAggregatorFactory.java +++ b/server/src/main/java/org/elasticsearch/search/aggregations/bucket/histogram/AutoDateHistogramAggregatorFactory.java @@ -35,14 +35,19 @@ import java.util.Map; public final class AutoDateHistogramAggregatorFactory - extends ValuesSourceAggregatorFactory { + extends ValuesSourceAggregatorFactory { private final int numBuckets; private RoundingInfo[] roundingInfos; - public AutoDateHistogramAggregatorFactory(String name, ValuesSourceConfig config, int numBuckets, RoundingInfo[] roundingInfos, - SearchContext context, AggregatorFactory parent, AggregatorFactories.Builder subFactoriesBuilder, - Map metaData) throws IOException { + public AutoDateHistogramAggregatorFactory(String name, + ValuesSourceConfig config, + int numBuckets, + RoundingInfo[] roundingInfos, + SearchContext context, + AggregatorFactory parent, + AggregatorFactories.Builder subFactoriesBuilder, + Map metaData) throws IOException { super(name, config, context, parent, subFactoriesBuilder, metaData); this.numBuckets = numBuckets; this.roundingInfos = roundingInfos; diff --git a/server/src/main/java/org/elasticsearch/search/aggregations/bucket/histogram/DateHistogramAggregationBuilder.java b/server/src/main/java/org/elasticsearch/search/aggregations/bucket/histogram/DateHistogramAggregationBuilder.java index 49b4275f4abc8..1fd2b3b5133a0 100644 --- a/server/src/main/java/org/elasticsearch/search/aggregations/bucket/histogram/DateHistogramAggregationBuilder.java +++ b/server/src/main/java/org/elasticsearch/search/aggregations/bucket/histogram/DateHistogramAggregationBuilder.java @@ -478,8 +478,8 @@ ZoneId rewriteTimeZone(QueryShardContext context) throws IOException { } @Override - protected ValuesSourceAggregatorFactory innerBuild(SearchContext context, ValuesSourceConfig config, - AggregatorFactory parent, Builder subFactoriesBuilder) throws IOException { + protected ValuesSourceAggregatorFactory innerBuild(SearchContext context, ValuesSourceConfig config, + AggregatorFactory parent, Builder subFactoriesBuilder) throws IOException { final ZoneId tz = timeZone(); final Rounding rounding = dateHistogramInterval.createRounding(tz); final ZoneId rewrittenTimeZone = rewriteTimeZone(context.getQueryShardContext()); @@ -500,12 +500,15 @@ ZoneId rewriteTimeZone(QueryShardContext context) throws IOException { } @Override - protected int innerHashCode() { - return Objects.hash(order, keyed, minDocCount, dateHistogramInterval, minDocCount, extendedBounds); + public int hashCode() { + return Objects.hash(super.hashCode(), order, keyed, minDocCount, dateHistogramInterval, minDocCount, extendedBounds); } @Override - protected boolean innerEquals(Object obj) { + public boolean equals(Object obj) { + if (this == obj) return true; + if (obj == null || getClass() != obj.getClass()) return false; + if (super.equals(obj) == false) return false; DateHistogramAggregationBuilder other = (DateHistogramAggregationBuilder) obj; return Objects.equals(order, other.order) && Objects.equals(keyed, other.keyed) diff --git a/server/src/main/java/org/elasticsearch/search/aggregations/bucket/histogram/DateHistogramAggregatorFactory.java b/server/src/main/java/org/elasticsearch/search/aggregations/bucket/histogram/DateHistogramAggregatorFactory.java index 8c025eb34eeb3..419125b5f4e47 100644 --- a/server/src/main/java/org/elasticsearch/search/aggregations/bucket/histogram/DateHistogramAggregatorFactory.java +++ b/server/src/main/java/org/elasticsearch/search/aggregations/bucket/histogram/DateHistogramAggregatorFactory.java @@ -36,7 +36,7 @@ import java.util.Map; public final class DateHistogramAggregatorFactory - extends ValuesSourceAggregatorFactory { + extends ValuesSourceAggregatorFactory { private final long offset; private final BucketOrder order; @@ -49,7 +49,7 @@ public final class DateHistogramAggregatorFactory public DateHistogramAggregatorFactory(String name, ValuesSourceConfig config, long offset, BucketOrder order, boolean keyed, long minDocCount, Rounding rounding, Rounding shardRounding, ExtendedBounds extendedBounds, SearchContext context, - AggregatorFactory parent, AggregatorFactories.Builder subFactoriesBuilder, + AggregatorFactory parent, AggregatorFactories.Builder subFactoriesBuilder, Map metaData) throws IOException { super(name, config, context, parent, subFactoriesBuilder, metaData); this.offset = offset; diff --git a/server/src/main/java/org/elasticsearch/search/aggregations/bucket/histogram/HistogramAggregationBuilder.java b/server/src/main/java/org/elasticsearch/search/aggregations/bucket/histogram/HistogramAggregationBuilder.java index fd38f6eb63a93..ecb65df433a69 100644 --- a/server/src/main/java/org/elasticsearch/search/aggregations/bucket/histogram/HistogramAggregationBuilder.java +++ b/server/src/main/java/org/elasticsearch/search/aggregations/bucket/histogram/HistogramAggregationBuilder.java @@ -295,26 +295,29 @@ public String getType() { } @Override - protected ValuesSourceAggregatorFactory innerBuild(SearchContext context, ValuesSourceConfig config, - AggregatorFactory parent, Builder subFactoriesBuilder) throws IOException { + protected ValuesSourceAggregatorFactory innerBuild(SearchContext context, ValuesSourceConfig config, + AggregatorFactory parent, Builder subFactoriesBuilder) throws IOException { return new HistogramAggregatorFactory(name, config, interval, offset, order, keyed, minDocCount, minBound, maxBound, context, parent, subFactoriesBuilder, metaData); } @Override - protected int innerHashCode() { - return Objects.hash(order, keyed, minDocCount, interval, offset, minBound, maxBound); + public int hashCode() { + return Objects.hash(super.hashCode(), order, keyed, minDocCount, interval, offset, minBound, maxBound); } @Override - protected boolean innerEquals(Object obj) { + public boolean equals(Object obj) { + if (this == obj) return true; + if (obj == null || getClass() != obj.getClass()) return false; + if (super.equals(obj) == false) return false; HistogramAggregationBuilder other = (HistogramAggregationBuilder) obj; return Objects.equals(order, other.order) - && Objects.equals(keyed, other.keyed) - && Objects.equals(minDocCount, other.minDocCount) - && Objects.equals(interval, other.interval) - && Objects.equals(offset, other.offset) - && Objects.equals(minBound, other.minBound) - && Objects.equals(maxBound, other.maxBound); + && Objects.equals(keyed, other.keyed) + && Objects.equals(minDocCount, other.minDocCount) + && Objects.equals(interval, other.interval) + && Objects.equals(offset, other.offset) + && Objects.equals(minBound, other.minBound) + && Objects.equals(maxBound, other.maxBound); } } diff --git a/server/src/main/java/org/elasticsearch/search/aggregations/bucket/histogram/HistogramAggregatorFactory.java b/server/src/main/java/org/elasticsearch/search/aggregations/bucket/histogram/HistogramAggregatorFactory.java index 683ec1987c597..b6828c8e84d1b 100644 --- a/server/src/main/java/org/elasticsearch/search/aggregations/bucket/histogram/HistogramAggregatorFactory.java +++ b/server/src/main/java/org/elasticsearch/search/aggregations/bucket/histogram/HistogramAggregatorFactory.java @@ -34,7 +34,7 @@ import java.util.List; import java.util.Map; -public final class HistogramAggregatorFactory extends ValuesSourceAggregatorFactory { +public final class HistogramAggregatorFactory extends ValuesSourceAggregatorFactory { private final double interval, offset; private final BucketOrder order; @@ -44,7 +44,7 @@ public final class HistogramAggregatorFactory extends ValuesSourceAggregatorFact public HistogramAggregatorFactory(String name, ValuesSourceConfig config, double interval, double offset, BucketOrder order, boolean keyed, long minDocCount, double minBound, double maxBound, - SearchContext context, AggregatorFactory parent, + SearchContext context, AggregatorFactory parent, AggregatorFactories.Builder subFactoriesBuilder, Map metaData) throws IOException { super(name, config, context, parent, subFactoriesBuilder, metaData); this.interval = interval; diff --git a/server/src/main/java/org/elasticsearch/search/aggregations/bucket/histogram/InternalAutoDateHistogram.java b/server/src/main/java/org/elasticsearch/search/aggregations/bucket/histogram/InternalAutoDateHistogram.java index dc0eefec45c82..f10318de02e43 100644 --- a/server/src/main/java/org/elasticsearch/search/aggregations/bucket/histogram/InternalAutoDateHistogram.java +++ b/server/src/main/java/org/elasticsearch/search/aggregations/bucket/histogram/InternalAutoDateHistogram.java @@ -598,7 +598,11 @@ public Bucket createBucket(Number key, long docCount, InternalAggregations aggre } @Override - protected boolean doEquals(Object obj) { + public boolean equals(Object obj) { + if (this == obj) return true; + if (obj == null || getClass() != obj.getClass()) return false; + if (super.equals(obj) == false) return false; + InternalAutoDateHistogram that = (InternalAutoDateHistogram) obj; return Objects.equals(buckets, that.buckets) && Objects.equals(format, that.format) @@ -606,7 +610,7 @@ protected boolean doEquals(Object obj) { } @Override - protected int doHashCode() { - return Objects.hash(buckets, format, bucketInfo); + public int hashCode() { + return Objects.hash(super.hashCode(), buckets, format, bucketInfo); } } diff --git a/server/src/main/java/org/elasticsearch/search/aggregations/bucket/histogram/InternalDateHistogram.java b/server/src/main/java/org/elasticsearch/search/aggregations/bucket/histogram/InternalDateHistogram.java index 162bfde5acf0f..8ac4ce5f27dc8 100644 --- a/server/src/main/java/org/elasticsearch/search/aggregations/bucket/histogram/InternalDateHistogram.java +++ b/server/src/main/java/org/elasticsearch/search/aggregations/bucket/histogram/InternalDateHistogram.java @@ -512,7 +512,11 @@ public Bucket createBucket(Number key, long docCount, InternalAggregations aggre } @Override - protected boolean doEquals(Object obj) { + public boolean equals(Object obj) { + if (this == obj) return true; + if (obj == null || getClass() != obj.getClass()) return false; + if (super.equals(obj) == false) return false; + InternalDateHistogram that = (InternalDateHistogram) obj; return Objects.equals(buckets, that.buckets) && Objects.equals(order, that.order) @@ -524,7 +528,7 @@ protected boolean doEquals(Object obj) { } @Override - protected int doHashCode() { - return Objects.hash(buckets, order, format, keyed, minDocCount, offset, emptyBucketInfo); + public int hashCode() { + return Objects.hash(super.hashCode(), buckets, order, format, keyed, minDocCount, offset, emptyBucketInfo); } } diff --git a/server/src/main/java/org/elasticsearch/search/aggregations/bucket/histogram/InternalHistogram.java b/server/src/main/java/org/elasticsearch/search/aggregations/bucket/histogram/InternalHistogram.java index b324241e9bb6d..0bd18ab753e9f 100644 --- a/server/src/main/java/org/elasticsearch/search/aggregations/bucket/histogram/InternalHistogram.java +++ b/server/src/main/java/org/elasticsearch/search/aggregations/bucket/histogram/InternalHistogram.java @@ -489,7 +489,11 @@ public Bucket createBucket(Number key, long docCount, InternalAggregations aggre } @Override - protected boolean doEquals(Object obj) { + public boolean equals(Object obj) { + if (this == obj) return true; + if (obj == null || getClass() != obj.getClass()) return false; + if (super.equals(obj) == false) return false; + InternalHistogram that = (InternalHistogram) obj; return Objects.equals(buckets, that.buckets) && Objects.equals(emptyBucketInfo, that.emptyBucketInfo) @@ -500,7 +504,7 @@ protected boolean doEquals(Object obj) { } @Override - protected int doHashCode() { - return Objects.hash(buckets, emptyBucketInfo, format, keyed, minDocCount, order); + public int hashCode() { + return Objects.hash(super.hashCode(), buckets, emptyBucketInfo, format, keyed, minDocCount, order); } } diff --git a/server/src/main/java/org/elasticsearch/search/aggregations/bucket/missing/MissingAggregationBuilder.java b/server/src/main/java/org/elasticsearch/search/aggregations/bucket/missing/MissingAggregationBuilder.java index b72c004464427..de729b619dcd4 100644 --- a/server/src/main/java/org/elasticsearch/search/aggregations/bucket/missing/MissingAggregationBuilder.java +++ b/server/src/main/java/org/elasticsearch/search/aggregations/bucket/missing/MissingAggregationBuilder.java @@ -83,8 +83,8 @@ protected boolean serializeTargetValueType() { } @Override - protected ValuesSourceAggregatorFactory innerBuild(SearchContext context, - ValuesSourceConfig config, AggregatorFactory parent, Builder subFactoriesBuilder) throws IOException { + protected ValuesSourceAggregatorFactory innerBuild(SearchContext context, + ValuesSourceConfig config, AggregatorFactory parent, Builder subFactoriesBuilder) throws IOException { return new MissingAggregatorFactory(name, config, context, parent, subFactoriesBuilder, metaData); } @@ -92,17 +92,7 @@ protected boolean serializeTargetValueType() { public XContentBuilder doXContentBody(XContentBuilder builder, Params params) throws IOException { return builder; } - - @Override - protected int innerHashCode() { - return 0; - } - - @Override - protected boolean innerEquals(Object obj) { - return true; - } - + @Override public String getType() { return NAME; diff --git a/server/src/main/java/org/elasticsearch/search/aggregations/bucket/missing/MissingAggregatorFactory.java b/server/src/main/java/org/elasticsearch/search/aggregations/bucket/missing/MissingAggregatorFactory.java index 5d05b79cef2d6..61705c773d1de 100644 --- a/server/src/main/java/org/elasticsearch/search/aggregations/bucket/missing/MissingAggregatorFactory.java +++ b/server/src/main/java/org/elasticsearch/search/aggregations/bucket/missing/MissingAggregatorFactory.java @@ -32,10 +32,10 @@ import java.util.List; import java.util.Map; -public class MissingAggregatorFactory extends ValuesSourceAggregatorFactory { +public class MissingAggregatorFactory extends ValuesSourceAggregatorFactory { public MissingAggregatorFactory(String name, ValuesSourceConfig config, SearchContext context, - AggregatorFactory parent, AggregatorFactories.Builder subFactoriesBuilder, Map metaData) throws IOException { + AggregatorFactory parent, AggregatorFactories.Builder subFactoriesBuilder, Map metaData) throws IOException { super(name, config, context, parent, subFactoriesBuilder, metaData); } diff --git a/server/src/main/java/org/elasticsearch/search/aggregations/bucket/nested/NestedAggregationBuilder.java b/server/src/main/java/org/elasticsearch/search/aggregations/bucket/nested/NestedAggregationBuilder.java index 8429cf79ed903..e27b341003d97 100644 --- a/server/src/main/java/org/elasticsearch/search/aggregations/bucket/nested/NestedAggregationBuilder.java +++ b/server/src/main/java/org/elasticsearch/search/aggregations/bucket/nested/NestedAggregationBuilder.java @@ -87,7 +87,7 @@ public String path() { } @Override - protected AggregatorFactory doBuild(SearchContext context, AggregatorFactory parent, Builder subFactoriesBuilder) + protected AggregatorFactory doBuild(SearchContext context, AggregatorFactory parent, Builder subFactoriesBuilder) throws IOException { ObjectMapper childObjectMapper = context.getObjectMapper(path); if (childObjectMapper == null) { @@ -143,14 +143,16 @@ public static NestedAggregationBuilder parse(String aggregationName, XContentPar return new NestedAggregationBuilder(aggregationName, path); } - @Override - protected int doHashCode() { - return Objects.hash(path); + public int hashCode() { + return Objects.hash(super.hashCode(), path); } @Override - protected boolean doEquals(Object obj) { + public boolean equals(Object obj) { + if (this == obj) return true; + if (obj == null || getClass() != obj.getClass()) return false; + if (super.equals(obj) == false) return false; NestedAggregationBuilder other = (NestedAggregationBuilder) obj; return Objects.equals(path, other.path); } diff --git a/server/src/main/java/org/elasticsearch/search/aggregations/bucket/nested/NestedAggregatorFactory.java b/server/src/main/java/org/elasticsearch/search/aggregations/bucket/nested/NestedAggregatorFactory.java index 6724ee7da30d6..769c5d024a94b 100644 --- a/server/src/main/java/org/elasticsearch/search/aggregations/bucket/nested/NestedAggregatorFactory.java +++ b/server/src/main/java/org/elasticsearch/search/aggregations/bucket/nested/NestedAggregatorFactory.java @@ -32,13 +32,13 @@ import java.util.List; import java.util.Map; -public class NestedAggregatorFactory extends AggregatorFactory { +public class NestedAggregatorFactory extends AggregatorFactory { private final ObjectMapper parentObjectMapper; private final ObjectMapper childObjectMapper; NestedAggregatorFactory(String name, ObjectMapper parentObjectMapper, ObjectMapper childObjectMapper, - SearchContext context, AggregatorFactory parent, AggregatorFactories.Builder subFactories, + SearchContext context, AggregatorFactory parent, AggregatorFactories.Builder subFactories, Map metaData) throws IOException { super(name, context, parent, subFactories, metaData); this.parentObjectMapper = parentObjectMapper; diff --git a/server/src/main/java/org/elasticsearch/search/aggregations/bucket/nested/ReverseNestedAggregationBuilder.java b/server/src/main/java/org/elasticsearch/search/aggregations/bucket/nested/ReverseNestedAggregationBuilder.java index 06c73ef797e28..62dfb4011769b 100644 --- a/server/src/main/java/org/elasticsearch/search/aggregations/bucket/nested/ReverseNestedAggregationBuilder.java +++ b/server/src/main/java/org/elasticsearch/search/aggregations/bucket/nested/ReverseNestedAggregationBuilder.java @@ -92,7 +92,7 @@ public String path() { } @Override - protected AggregatorFactory doBuild(SearchContext context, AggregatorFactory parent, Builder subFactoriesBuilder) + protected AggregatorFactory doBuild(SearchContext context, AggregatorFactory parent, Builder subFactoriesBuilder) throws IOException { if (findNestedAggregatorFactory(parent) == null) { throw new SearchParseException(context, @@ -120,7 +120,7 @@ protected AggregatorFactory doBuild(SearchContext context, AggregatorFactory< } } - private static NestedAggregatorFactory findNestedAggregatorFactory(AggregatorFactory parent) { + private static NestedAggregatorFactory findNestedAggregatorFactory(AggregatorFactory parent) { if (parent == null) { return null; } else if (parent instanceof NestedAggregatorFactory) { @@ -168,14 +168,16 @@ public static ReverseNestedAggregationBuilder parse(String aggregationName, XCon return factory; } - @Override - protected int doHashCode() { - return Objects.hash(path); + public int hashCode() { + return Objects.hash(super.hashCode(), path); } @Override - protected boolean doEquals(Object obj) { + public boolean equals(Object obj) { + if (this == obj) return true; + if (obj == null || getClass() != obj.getClass()) return false; + if (super.equals(obj) == false) return false; ReverseNestedAggregationBuilder other = (ReverseNestedAggregationBuilder) obj; return Objects.equals(path, other.path); } diff --git a/server/src/main/java/org/elasticsearch/search/aggregations/bucket/nested/ReverseNestedAggregatorFactory.java b/server/src/main/java/org/elasticsearch/search/aggregations/bucket/nested/ReverseNestedAggregatorFactory.java index 3ed6f21d99c8d..022a1cec77190 100644 --- a/server/src/main/java/org/elasticsearch/search/aggregations/bucket/nested/ReverseNestedAggregatorFactory.java +++ b/server/src/main/java/org/elasticsearch/search/aggregations/bucket/nested/ReverseNestedAggregatorFactory.java @@ -32,13 +32,13 @@ import java.util.List; import java.util.Map; -public class ReverseNestedAggregatorFactory extends AggregatorFactory { +public class ReverseNestedAggregatorFactory extends AggregatorFactory { private final boolean unmapped; private final ObjectMapper parentObjectMapper; public ReverseNestedAggregatorFactory(String name, boolean unmapped, ObjectMapper parentObjectMapper, - SearchContext context, AggregatorFactory parent, + SearchContext context, AggregatorFactory parent, AggregatorFactories.Builder subFactories, Map metaData) throws IOException { super(name, context, parent, subFactories, metaData); diff --git a/server/src/main/java/org/elasticsearch/search/aggregations/bucket/range/AbstractRangeAggregatorFactory.java b/server/src/main/java/org/elasticsearch/search/aggregations/bucket/range/AbstractRangeAggregatorFactory.java index 156adcdc4f3ea..e1943e6af1170 100644 --- a/server/src/main/java/org/elasticsearch/search/aggregations/bucket/range/AbstractRangeAggregatorFactory.java +++ b/server/src/main/java/org/elasticsearch/search/aggregations/bucket/range/AbstractRangeAggregatorFactory.java @@ -35,15 +35,14 @@ import java.util.List; import java.util.Map; -public class AbstractRangeAggregatorFactory, R extends Range> - extends ValuesSourceAggregatorFactory { +public class AbstractRangeAggregatorFactory extends ValuesSourceAggregatorFactory { private final InternalRange.Factory rangeFactory; private final R[] ranges; private final boolean keyed; public AbstractRangeAggregatorFactory(String name, ValuesSourceConfig config, R[] ranges, boolean keyed, - InternalRange.Factory rangeFactory, SearchContext context, AggregatorFactory parent, + InternalRange.Factory rangeFactory, SearchContext context, AggregatorFactory parent, AggregatorFactories.Builder subFactoriesBuilder, Map metaData) throws IOException { super(name, config, context, parent, subFactoriesBuilder, metaData); this.ranges = ranges; diff --git a/server/src/main/java/org/elasticsearch/search/aggregations/bucket/range/AbstractRangeBuilder.java b/server/src/main/java/org/elasticsearch/search/aggregations/bucket/range/AbstractRangeBuilder.java index e5b62d85ca17c..542929564e66f 100644 --- a/server/src/main/java/org/elasticsearch/search/aggregations/bucket/range/AbstractRangeBuilder.java +++ b/server/src/main/java/org/elasticsearch/search/aggregations/bucket/range/AbstractRangeBuilder.java @@ -140,12 +140,15 @@ protected XContentBuilder doXContentBody(XContentBuilder builder, Params params) } @Override - protected int innerHashCode() { - return Objects.hash(ranges, keyed); + public int hashCode() { + return Objects.hash(super.hashCode(), ranges, keyed); } @Override - protected boolean innerEquals(Object obj) { + public boolean equals(Object obj) { + if (this == obj) return true; + if (obj == null || getClass() != obj.getClass()) return false; + if (super.equals(obj) == false) return false; AbstractRangeBuilder other = (AbstractRangeBuilder) obj; return Objects.equals(ranges, other.ranges) && Objects.equals(keyed, other.keyed); diff --git a/server/src/main/java/org/elasticsearch/search/aggregations/bucket/range/BinaryRangeAggregatorFactory.java b/server/src/main/java/org/elasticsearch/search/aggregations/bucket/range/BinaryRangeAggregatorFactory.java index a8d74a7d83615..41d30cc9bff81 100644 --- a/server/src/main/java/org/elasticsearch/search/aggregations/bucket/range/BinaryRangeAggregatorFactory.java +++ b/server/src/main/java/org/elasticsearch/search/aggregations/bucket/range/BinaryRangeAggregatorFactory.java @@ -32,7 +32,7 @@ import java.util.Map; public class BinaryRangeAggregatorFactory - extends ValuesSourceAggregatorFactory { + extends ValuesSourceAggregatorFactory { private final List ranges; private final boolean keyed; @@ -41,7 +41,7 @@ public BinaryRangeAggregatorFactory(String name, ValuesSourceConfig config, List ranges, boolean keyed, SearchContext context, - AggregatorFactory parent, Builder subFactoriesBuilder, + AggregatorFactory parent, Builder subFactoriesBuilder, Map metaData) throws IOException { super(name, config, context, parent, subFactoriesBuilder, metaData); this.ranges = ranges; diff --git a/server/src/main/java/org/elasticsearch/search/aggregations/bucket/range/DateRangeAggregationBuilder.java b/server/src/main/java/org/elasticsearch/search/aggregations/bucket/range/DateRangeAggregationBuilder.java index 2b5e92ddcb3f9..f0896fb682bfb 100644 --- a/server/src/main/java/org/elasticsearch/search/aggregations/bucket/range/DateRangeAggregationBuilder.java +++ b/server/src/main/java/org/elasticsearch/search/aggregations/bucket/range/DateRangeAggregationBuilder.java @@ -289,7 +289,7 @@ public DateRangeAggregationBuilder addUnboundedFrom(ZonedDateTime from) { @Override protected DateRangeAggregatorFactory innerBuild(SearchContext context, ValuesSourceConfig config, - AggregatorFactory parent, Builder subFactoriesBuilder) throws IOException { + AggregatorFactory parent, Builder subFactoriesBuilder) throws IOException { // We need to call processRanges here so they are parsed and we know whether `now` has been used before we make // the decision of whether to cache the request RangeAggregator.Range[] ranges = processRanges(range -> { diff --git a/server/src/main/java/org/elasticsearch/search/aggregations/bucket/range/DateRangeAggregatorFactory.java b/server/src/main/java/org/elasticsearch/search/aggregations/bucket/range/DateRangeAggregatorFactory.java index 6dd51197ca823..361dcbb4c24d6 100644 --- a/server/src/main/java/org/elasticsearch/search/aggregations/bucket/range/DateRangeAggregatorFactory.java +++ b/server/src/main/java/org/elasticsearch/search/aggregations/bucket/range/DateRangeAggregatorFactory.java @@ -28,10 +28,10 @@ import java.io.IOException; import java.util.Map; -public class DateRangeAggregatorFactory extends AbstractRangeAggregatorFactory { +public class DateRangeAggregatorFactory extends AbstractRangeAggregatorFactory { public DateRangeAggregatorFactory(String name, ValuesSourceConfig config, RangeAggregator.Range[] ranges, boolean keyed, - InternalRange.Factory rangeFactory, SearchContext context, AggregatorFactory parent, + InternalRange.Factory rangeFactory, SearchContext context, AggregatorFactory parent, AggregatorFactories.Builder subFactoriesBuilder, Map metaData) throws IOException { super(name, config, ranges, keyed, rangeFactory, context, parent, subFactoriesBuilder, metaData); } diff --git a/server/src/main/java/org/elasticsearch/search/aggregations/bucket/range/GeoDistanceAggregationBuilder.java b/server/src/main/java/org/elasticsearch/search/aggregations/bucket/range/GeoDistanceAggregationBuilder.java index c612b303fb134..d9e29d0df469e 100644 --- a/server/src/main/java/org/elasticsearch/search/aggregations/bucket/range/GeoDistanceAggregationBuilder.java +++ b/server/src/main/java/org/elasticsearch/search/aggregations/bucket/range/GeoDistanceAggregationBuilder.java @@ -411,8 +411,8 @@ public boolean keyed() { } @Override - protected ValuesSourceAggregatorFactory innerBuild(SearchContext context, - ValuesSourceConfig config, AggregatorFactory parent, Builder subFactoriesBuilder) + protected ValuesSourceAggregatorFactory innerBuild(SearchContext context, + ValuesSourceConfig config, AggregatorFactory parent, Builder subFactoriesBuilder) throws IOException { Range[] ranges = this.ranges.toArray(new Range[this.range().size()]); if (ranges.length == 0) { @@ -433,12 +433,15 @@ protected XContentBuilder doXContentBody(XContentBuilder builder, Params params) } @Override - protected int innerHashCode() { - return Objects.hash(origin, ranges, keyed, distanceType, unit); + public int hashCode() { + return Objects.hash(super.hashCode(), origin, ranges, keyed, distanceType, unit); } @Override - protected boolean innerEquals(Object obj) { + public boolean equals(Object obj) { + if (this == obj) return true; + if (obj == null || getClass() != obj.getClass()) return false; + if (super.equals(obj) == false) return false; GeoDistanceAggregationBuilder other = (GeoDistanceAggregationBuilder) obj; return Objects.equals(origin, other.origin) && Objects.equals(ranges, other.ranges) diff --git a/server/src/main/java/org/elasticsearch/search/aggregations/bucket/range/GeoDistanceRangeAggregatorFactory.java b/server/src/main/java/org/elasticsearch/search/aggregations/bucket/range/GeoDistanceRangeAggregatorFactory.java index 26847e1ba10c1..b99ae657aaee8 100644 --- a/server/src/main/java/org/elasticsearch/search/aggregations/bucket/range/GeoDistanceRangeAggregatorFactory.java +++ b/server/src/main/java/org/elasticsearch/search/aggregations/bucket/range/GeoDistanceRangeAggregatorFactory.java @@ -43,7 +43,7 @@ import java.util.Map; public class GeoDistanceRangeAggregatorFactory - extends ValuesSourceAggregatorFactory { + extends ValuesSourceAggregatorFactory { private final InternalRange.Factory rangeFactory = InternalGeoDistance.FACTORY; private final GeoPoint origin; @@ -54,7 +54,7 @@ public class GeoDistanceRangeAggregatorFactory public GeoDistanceRangeAggregatorFactory(String name, ValuesSourceConfig config, GeoPoint origin, Range[] ranges, DistanceUnit unit, GeoDistance distanceType, boolean keyed, SearchContext context, - AggregatorFactory parent, AggregatorFactories.Builder subFactoriesBuilder, Map metaData) throws IOException { + AggregatorFactory parent, AggregatorFactories.Builder subFactoriesBuilder, Map metaData) throws IOException { super(name, config, context, parent, subFactoriesBuilder, metaData); this.origin = origin; this.ranges = ranges; 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 35ee592d6c9ae..7051403b302a7 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 @@ -275,16 +275,20 @@ public XContentBuilder doXContentBody(XContentBuilder builder, return builder; } + @Override - public boolean doEquals(Object obj) { + public boolean equals(Object obj) { + if (this == obj) return true; + if (obj == null || getClass() != obj.getClass()) return false; + if (super.equals(obj) == false) return false; + InternalBinaryRange that = (InternalBinaryRange) obj; return Objects.equals(buckets, that.buckets) && Objects.equals(format, that.format) && Objects.equals(keyed, that.keyed); } - @Override - public int doHashCode() { - return Objects.hash(buckets, format, keyed); + public int hashCode() { + return Objects.hash(super.hashCode(), buckets, format, keyed); } } diff --git a/server/src/main/java/org/elasticsearch/search/aggregations/bucket/range/InternalRange.java b/server/src/main/java/org/elasticsearch/search/aggregations/bucket/range/InternalRange.java index 747cb22c87071..94b8d23617036 100644 --- a/server/src/main/java/org/elasticsearch/search/aggregations/bucket/range/InternalRange.java +++ b/server/src/main/java/org/elasticsearch/search/aggregations/bucket/range/InternalRange.java @@ -338,12 +338,16 @@ public XContentBuilder doXContentBody(XContentBuilder builder, Params params) th } @Override - protected int doHashCode() { - return Objects.hash(ranges, format, keyed); + public int hashCode() { + return Objects.hash(super.hashCode(), ranges, format, keyed); } @Override - protected boolean doEquals(Object obj) { + public boolean equals(Object obj) { + if (this == obj) return true; + if (obj == null || getClass() != obj.getClass()) return false; + if (super.equals(obj) == false) return false; + InternalRange that = (InternalRange) obj; return Objects.equals(ranges, that.ranges) && Objects.equals(format, that.format) diff --git a/server/src/main/java/org/elasticsearch/search/aggregations/bucket/range/IpRangeAggregationBuilder.java b/server/src/main/java/org/elasticsearch/search/aggregations/bucket/range/IpRangeAggregationBuilder.java index 1a3eb4957e2c5..9ee76784d207e 100644 --- a/server/src/main/java/org/elasticsearch/search/aggregations/bucket/range/IpRangeAggregationBuilder.java +++ b/server/src/main/java/org/elasticsearch/search/aggregations/bucket/range/IpRangeAggregationBuilder.java @@ -364,9 +364,9 @@ private static BytesRef toBytesRef(String ip) { } @Override - protected ValuesSourceAggregatorFactory innerBuild( + protected ValuesSourceAggregatorFactory innerBuild( SearchContext context, ValuesSourceConfig config, - AggregatorFactory parent, Builder subFactoriesBuilder) + AggregatorFactory parent, Builder subFactoriesBuilder) throws IOException { List ranges = new ArrayList<>(); if(this.ranges.size() == 0){ @@ -387,14 +387,17 @@ protected XContentBuilder doXContentBody(XContentBuilder builder, Params params) } @Override - protected int innerHashCode() { - return Objects.hash(keyed, ranges); + public int hashCode() { + return Objects.hash(super.hashCode(), keyed, ranges); } @Override - protected boolean innerEquals(Object obj) { + public boolean equals(Object obj) { + if (this == obj) return true; + if (obj == null || getClass() != obj.getClass()) return false; + if (super.equals(obj) == false) return false; IpRangeAggregationBuilder that = (IpRangeAggregationBuilder) obj; return keyed == that.keyed - && ranges.equals(that.ranges); + && ranges.equals(that.ranges); } } diff --git a/server/src/main/java/org/elasticsearch/search/aggregations/bucket/range/RangeAggregationBuilder.java b/server/src/main/java/org/elasticsearch/search/aggregations/bucket/range/RangeAggregationBuilder.java index 9a7edf7ef95dd..25c072a7a0656 100644 --- a/server/src/main/java/org/elasticsearch/search/aggregations/bucket/range/RangeAggregationBuilder.java +++ b/server/src/main/java/org/elasticsearch/search/aggregations/bucket/range/RangeAggregationBuilder.java @@ -147,7 +147,7 @@ public RangeAggregationBuilder addUnboundedFrom(double from) { @Override protected RangeAggregatorFactory innerBuild(SearchContext context, ValuesSourceConfig config, - AggregatorFactory parent, Builder subFactoriesBuilder) throws IOException { + AggregatorFactory parent, Builder subFactoriesBuilder) throws IOException { // We need to call processRanges here so they are parsed before we make the decision of whether to cache the request Range[] ranges = processRanges(range -> { DocValueFormat parser = config.format(); diff --git a/server/src/main/java/org/elasticsearch/search/aggregations/bucket/range/RangeAggregatorFactory.java b/server/src/main/java/org/elasticsearch/search/aggregations/bucket/range/RangeAggregatorFactory.java index d1dc3e71b5c11..66fde1554b922 100644 --- a/server/src/main/java/org/elasticsearch/search/aggregations/bucket/range/RangeAggregatorFactory.java +++ b/server/src/main/java/org/elasticsearch/search/aggregations/bucket/range/RangeAggregatorFactory.java @@ -30,10 +30,10 @@ import java.io.IOException; import java.util.Map; -public class RangeAggregatorFactory extends AbstractRangeAggregatorFactory { +public class RangeAggregatorFactory extends AbstractRangeAggregatorFactory { public RangeAggregatorFactory(String name, ValuesSourceConfig config, Range[] ranges, boolean keyed, - Factory rangeFactory, SearchContext context, AggregatorFactory parent, + Factory rangeFactory, SearchContext context, AggregatorFactory parent, AggregatorFactories.Builder subFactoriesBuilder, Map metaData) throws IOException { super(name, config, ranges, keyed, rangeFactory, context, parent, subFactoriesBuilder, metaData); } diff --git a/server/src/main/java/org/elasticsearch/search/aggregations/bucket/sampler/DiversifiedAggregationBuilder.java b/server/src/main/java/org/elasticsearch/search/aggregations/bucket/sampler/DiversifiedAggregationBuilder.java index d6ff4c3d18f5d..d4d7d4f8b01bc 100644 --- a/server/src/main/java/org/elasticsearch/search/aggregations/bucket/sampler/DiversifiedAggregationBuilder.java +++ b/server/src/main/java/org/elasticsearch/search/aggregations/bucket/sampler/DiversifiedAggregationBuilder.java @@ -148,8 +148,8 @@ public String executionHint() { } @Override - protected ValuesSourceAggregatorFactory innerBuild(SearchContext context, - ValuesSourceConfig config, AggregatorFactory parent, Builder subFactoriesBuilder) throws IOException { + protected ValuesSourceAggregatorFactory innerBuild(SearchContext context, + ValuesSourceConfig config, AggregatorFactory parent, Builder subFactoriesBuilder) throws IOException { return new DiversifiedAggregatorFactory(name, config, shardSize, maxDocsPerValue, executionHint, context, parent, subFactoriesBuilder, metaData); } @@ -165,16 +165,19 @@ protected XContentBuilder doXContentBody(XContentBuilder builder, Params params) } @Override - protected int innerHashCode() { - return Objects.hash(shardSize, maxDocsPerValue, executionHint); + public int hashCode() { + return Objects.hash(super.hashCode(), shardSize, maxDocsPerValue, executionHint); } @Override - protected boolean innerEquals(Object obj) { + public boolean equals(Object obj) { + if (this == obj) return true; + if (obj == null || getClass() != obj.getClass()) return false; + if (super.equals(obj) == false) return false; DiversifiedAggregationBuilder other = (DiversifiedAggregationBuilder) obj; return Objects.equals(shardSize, other.shardSize) - && Objects.equals(maxDocsPerValue, other.maxDocsPerValue) - && Objects.equals(executionHint, other.executionHint); + && Objects.equals(maxDocsPerValue, other.maxDocsPerValue) + && Objects.equals(executionHint, other.executionHint); } @Override diff --git a/server/src/main/java/org/elasticsearch/search/aggregations/bucket/sampler/DiversifiedAggregatorFactory.java b/server/src/main/java/org/elasticsearch/search/aggregations/bucket/sampler/DiversifiedAggregatorFactory.java index 2bd68296d7b3e..281e6b0e29d9f 100644 --- a/server/src/main/java/org/elasticsearch/search/aggregations/bucket/sampler/DiversifiedAggregatorFactory.java +++ b/server/src/main/java/org/elasticsearch/search/aggregations/bucket/sampler/DiversifiedAggregatorFactory.java @@ -37,14 +37,14 @@ import java.util.List; import java.util.Map; -public class DiversifiedAggregatorFactory extends ValuesSourceAggregatorFactory { +public class DiversifiedAggregatorFactory extends ValuesSourceAggregatorFactory { private final int shardSize; private final int maxDocsPerValue; private final String executionHint; DiversifiedAggregatorFactory(String name, ValuesSourceConfig config, int shardSize, int maxDocsPerValue, - String executionHint, SearchContext context, AggregatorFactory parent, AggregatorFactories.Builder subFactoriesBuilder, + String executionHint, SearchContext context, AggregatorFactory parent, AggregatorFactories.Builder subFactoriesBuilder, Map metaData) throws IOException { super(name, config, context, parent, subFactoriesBuilder, metaData); this.shardSize = shardSize; diff --git a/server/src/main/java/org/elasticsearch/search/aggregations/bucket/sampler/SamplerAggregationBuilder.java b/server/src/main/java/org/elasticsearch/search/aggregations/bucket/sampler/SamplerAggregationBuilder.java index 1aed0a0374aa9..afdb3e5e48cb6 100644 --- a/server/src/main/java/org/elasticsearch/search/aggregations/bucket/sampler/SamplerAggregationBuilder.java +++ b/server/src/main/java/org/elasticsearch/search/aggregations/bucket/sampler/SamplerAggregationBuilder.java @@ -84,7 +84,7 @@ public int shardSize() { } @Override - protected SamplerAggregatorFactory doBuild(SearchContext context, AggregatorFactory parent, Builder subFactoriesBuilder) + protected SamplerAggregatorFactory doBuild(SearchContext context, AggregatorFactory parent, Builder subFactoriesBuilder) throws IOException { return new SamplerAggregatorFactory(name, shardSize, context, parent, subFactoriesBuilder, metaData); } @@ -126,12 +126,15 @@ public static SamplerAggregationBuilder parse(String aggregationName, XContentPa } @Override - protected int doHashCode() { - return Objects.hash(shardSize); + public int hashCode() { + return Objects.hash(super.hashCode(), shardSize); } @Override - protected boolean doEquals(Object obj) { + public boolean equals(Object obj) { + if (this == obj) return true; + if (obj == null || getClass() != obj.getClass()) return false; + if (super.equals(obj) == false) return false; SamplerAggregationBuilder other = (SamplerAggregationBuilder) obj; return Objects.equals(shardSize, other.shardSize); } diff --git a/server/src/main/java/org/elasticsearch/search/aggregations/bucket/sampler/SamplerAggregatorFactory.java b/server/src/main/java/org/elasticsearch/search/aggregations/bucket/sampler/SamplerAggregatorFactory.java index c122cc95fa117..d78a2bc275a04 100644 --- a/server/src/main/java/org/elasticsearch/search/aggregations/bucket/sampler/SamplerAggregatorFactory.java +++ b/server/src/main/java/org/elasticsearch/search/aggregations/bucket/sampler/SamplerAggregatorFactory.java @@ -29,11 +29,11 @@ import java.util.List; import java.util.Map; -public class SamplerAggregatorFactory extends AggregatorFactory { +public class SamplerAggregatorFactory extends AggregatorFactory { private final int shardSize; - SamplerAggregatorFactory(String name, int shardSize, SearchContext context, AggregatorFactory parent, + SamplerAggregatorFactory(String name, int shardSize, SearchContext context, AggregatorFactory parent, AggregatorFactories.Builder subFactories, Map metaData) throws IOException { super(name, context, parent, subFactories, metaData); this.shardSize = shardSize; diff --git a/server/src/main/java/org/elasticsearch/search/aggregations/bucket/significant/InternalMappedSignificantTerms.java b/server/src/main/java/org/elasticsearch/search/aggregations/bucket/significant/InternalMappedSignificantTerms.java index e1bd4defd3e08..849015163dc23 100644 --- a/server/src/main/java/org/elasticsearch/search/aggregations/bucket/significant/InternalMappedSignificantTerms.java +++ b/server/src/main/java/org/elasticsearch/search/aggregations/bucket/significant/InternalMappedSignificantTerms.java @@ -109,10 +109,13 @@ protected SignificanceHeuristic getSignificanceHeuristic() { } @Override - protected boolean doEquals(Object obj) { + public boolean equals(Object obj) { + if (this == obj) return true; + if (obj == null || getClass() != obj.getClass()) return false; + if (super.equals(obj) == false) return false; + InternalMappedSignificantTerms that = (InternalMappedSignificantTerms) obj; - return super.doEquals(obj) - && Objects.equals(format, that.format) + return Objects.equals(format, that.format) && subsetSize == that.subsetSize && supersetSize == that.supersetSize && Objects.equals(significanceHeuristic, that.significanceHeuristic) @@ -121,8 +124,8 @@ protected boolean doEquals(Object obj) { } @Override - protected int doHashCode() { - return Objects.hash(super.doHashCode(), format, subsetSize, supersetSize, significanceHeuristic, buckets, bucketMap); + public int hashCode() { + return Objects.hash(super.hashCode(), format, subsetSize, supersetSize, significanceHeuristic, buckets, bucketMap); } @Override diff --git a/server/src/main/java/org/elasticsearch/search/aggregations/bucket/significant/InternalSignificantTerms.java b/server/src/main/java/org/elasticsearch/search/aggregations/bucket/significant/InternalSignificantTerms.java index 42a3c6a849bc1..5d8bc893a2604 100644 --- a/server/src/main/java/org/elasticsearch/search/aggregations/bucket/significant/InternalSignificantTerms.java +++ b/server/src/main/java/org/elasticsearch/search/aggregations/bucket/significant/InternalSignificantTerms.java @@ -272,12 +272,16 @@ public InternalAggregation doReduce(List aggregations, Redu protected abstract SignificanceHeuristic getSignificanceHeuristic(); @Override - protected int doHashCode() { - return Objects.hash(minDocCount, requiredSize); + public int hashCode() { + return Objects.hash(super.hashCode(), minDocCount, requiredSize); } @Override - protected boolean doEquals(Object obj) { + public boolean equals(Object obj) { + if (this == obj) return true; + if (obj == null || getClass() != obj.getClass()) return false; + if (super.equals(obj) == false) return false; + InternalSignificantTerms that = (InternalSignificantTerms) obj; return Objects.equals(minDocCount, that.minDocCount) && Objects.equals(requiredSize, that.requiredSize); diff --git a/server/src/main/java/org/elasticsearch/search/aggregations/bucket/significant/SignificantStringTerms.java b/server/src/main/java/org/elasticsearch/search/aggregations/bucket/significant/SignificantStringTerms.java index 551ecd6a9f23e..a2adbcffab2f8 100644 --- a/server/src/main/java/org/elasticsearch/search/aggregations/bucket/significant/SignificantStringTerms.java +++ b/server/src/main/java/org/elasticsearch/search/aggregations/bucket/significant/SignificantStringTerms.java @@ -103,6 +103,10 @@ protected XContentBuilder keyToXContent(XContentBuilder builder) throws IOExcept @Override public boolean equals(Object obj) { + if (this == obj) return true; + if (obj == null || getClass() != obj.getClass()) return false; + if (super.equals(obj) == false) return false; + return super.equals(obj) && Objects.equals(termBytes, ((SignificantStringTerms.Bucket) obj).termBytes); } diff --git a/server/src/main/java/org/elasticsearch/search/aggregations/bucket/significant/SignificantTermsAggregationBuilder.java b/server/src/main/java/org/elasticsearch/search/aggregations/bucket/significant/SignificantTermsAggregationBuilder.java index 02216c5a5cb35..75b32f8abe062 100644 --- a/server/src/main/java/org/elasticsearch/search/aggregations/bucket/significant/SignificantTermsAggregationBuilder.java +++ b/server/src/main/java/org/elasticsearch/search/aggregations/bucket/significant/SignificantTermsAggregationBuilder.java @@ -283,8 +283,8 @@ public SignificanceHeuristic significanceHeuristic() { } @Override - protected ValuesSourceAggregatorFactory innerBuild(SearchContext context, ValuesSourceConfig config, - AggregatorFactory parent, Builder subFactoriesBuilder) throws IOException { + protected ValuesSourceAggregatorFactory innerBuild(SearchContext context, ValuesSourceConfig config, + AggregatorFactory parent, Builder subFactoriesBuilder) throws IOException { SignificanceHeuristic executionHeuristic = this.significanceHeuristic.rewrite(context); return new SignificantTermsAggregatorFactory(name, config, includeExclude, executionHint, filterBuilder, bucketCountThresholds, executionHeuristic, context, parent, subFactoriesBuilder, metaData); @@ -307,12 +307,15 @@ protected XContentBuilder doXContentBody(XContentBuilder builder, Params params) } @Override - protected int innerHashCode() { - return Objects.hash(bucketCountThresholds, executionHint, filterBuilder, includeExclude, significanceHeuristic); + public int hashCode() { + return Objects.hash(super.hashCode(), bucketCountThresholds, executionHint, filterBuilder, includeExclude, significanceHeuristic); } @Override - protected boolean innerEquals(Object obj) { + public boolean equals(Object obj) { + if (this == obj) return true; + if (obj == null || getClass() != obj.getClass()) return false; + if (super.equals(obj) == false) return false; SignificantTermsAggregationBuilder other = (SignificantTermsAggregationBuilder) obj; return Objects.equals(bucketCountThresholds, other.bucketCountThresholds) && Objects.equals(executionHint, other.executionHint) diff --git a/server/src/main/java/org/elasticsearch/search/aggregations/bucket/significant/SignificantTermsAggregatorFactory.java b/server/src/main/java/org/elasticsearch/search/aggregations/bucket/significant/SignificantTermsAggregatorFactory.java index 09fd5877344f6..dbe8873f88ad4 100644 --- a/server/src/main/java/org/elasticsearch/search/aggregations/bucket/significant/SignificantTermsAggregatorFactory.java +++ b/server/src/main/java/org/elasticsearch/search/aggregations/bucket/significant/SignificantTermsAggregatorFactory.java @@ -58,7 +58,7 @@ import java.util.List; import java.util.Map; -public class SignificantTermsAggregatorFactory extends ValuesSourceAggregatorFactory +public class SignificantTermsAggregatorFactory extends ValuesSourceAggregatorFactory implements Releasable { private static final DeprecationLogger deprecationLogger = new DeprecationLogger( LogManager.getLogger(SignificantTermsAggregatorFactory.class)); @@ -82,7 +82,7 @@ public SignificantTermsAggregatorFactory(String name, TermsAggregator.BucketCountThresholds bucketCountThresholds, SignificanceHeuristic significanceHeuristic, SearchContext context, - AggregatorFactory parent, + AggregatorFactory parent, AggregatorFactories.Builder subFactoriesBuilder, Map metaData) throws IOException { super(name, config, context, parent, subFactoriesBuilder, metaData); diff --git a/server/src/main/java/org/elasticsearch/search/aggregations/bucket/significant/SignificantTextAggregationBuilder.java b/server/src/main/java/org/elasticsearch/search/aggregations/bucket/significant/SignificantTextAggregationBuilder.java index f0b85f979c233..7d53776e4fe7d 100644 --- a/server/src/main/java/org/elasticsearch/search/aggregations/bucket/significant/SignificantTextAggregationBuilder.java +++ b/server/src/main/java/org/elasticsearch/search/aggregations/bucket/significant/SignificantTextAggregationBuilder.java @@ -340,7 +340,7 @@ protected void doWriteTo(StreamOutput out) throws IOException { } @Override - protected AggregatorFactory doBuild(SearchContext context, AggregatorFactory parent, + protected AggregatorFactory doBuild(SearchContext context, AggregatorFactory parent, Builder subFactoriesBuilder) throws IOException { SignificanceHeuristic executionHeuristic = this.significanceHeuristic.rewrite(context); @@ -378,13 +378,17 @@ protected XContentBuilder internalXContent(XContentBuilder builder, Params param } @Override - protected int doHashCode() { - return Objects.hash(bucketCountThresholds, fieldName, filterDuplicateText, filterBuilder, - includeExclude, significanceHeuristic, Arrays.hashCode(sourceFieldNames)); + public int hashCode() { + return Objects.hash(super.hashCode(), bucketCountThresholds, fieldName, + filterDuplicateText, filterBuilder, + includeExclude, significanceHeuristic, Arrays.hashCode(sourceFieldNames)); } @Override - protected boolean doEquals(Object obj) { + public boolean equals(Object obj) { + if (this == obj) return true; + if (obj == null || getClass() != obj.getClass()) return false; + if (super.equals(obj) == false) return false; SignificantTextAggregationBuilder other = (SignificantTextAggregationBuilder) obj; return Objects.equals(bucketCountThresholds, other.bucketCountThresholds) && Objects.equals(fieldName, other.fieldName) diff --git a/server/src/main/java/org/elasticsearch/search/aggregations/bucket/significant/SignificantTextAggregatorFactory.java b/server/src/main/java/org/elasticsearch/search/aggregations/bucket/significant/SignificantTextAggregatorFactory.java index 92a136960395d..eb33c6dbc4493 100644 --- a/server/src/main/java/org/elasticsearch/search/aggregations/bucket/significant/SignificantTextAggregatorFactory.java +++ b/server/src/main/java/org/elasticsearch/search/aggregations/bucket/significant/SignificantTextAggregatorFactory.java @@ -49,7 +49,7 @@ import java.util.List; import java.util.Map; -public class SignificantTextAggregatorFactory extends AggregatorFactory +public class SignificantTextAggregatorFactory extends AggregatorFactory implements Releasable { private final IncludeExclude includeExclude; @@ -67,7 +67,7 @@ public class SignificantTextAggregatorFactory extends AggregatorFactory parent, + SignificanceHeuristic significanceHeuristic, SearchContext context, AggregatorFactory parent, AggregatorFactories.Builder subFactoriesBuilder, String fieldName, String [] sourceFieldNames, boolean filterDuplicateText, Map metaData) throws IOException { super(name, context, parent, subFactoriesBuilder, metaData); diff --git a/server/src/main/java/org/elasticsearch/search/aggregations/bucket/terms/AbstractRareTermsAggregator.java b/server/src/main/java/org/elasticsearch/search/aggregations/bucket/terms/AbstractRareTermsAggregator.java new file mode 100644 index 0000000000000..2bbe3c01988df --- /dev/null +++ b/server/src/main/java/org/elasticsearch/search/aggregations/bucket/terms/AbstractRareTermsAggregator.java @@ -0,0 +1,134 @@ +/* + * 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.search.aggregations.bucket.terms; + +import org.elasticsearch.common.util.SetBackedScalingCuckooFilter; +import org.elasticsearch.search.DocValueFormat; +import org.elasticsearch.search.aggregations.Aggregator; +import org.elasticsearch.search.aggregations.AggregatorFactories; +import org.elasticsearch.search.aggregations.BucketOrder; +import org.elasticsearch.search.aggregations.LeafBucketCollector; +import org.elasticsearch.search.aggregations.bucket.DeferableBucketAggregator; +import org.elasticsearch.search.aggregations.bucket.DeferringBucketCollector; +import org.elasticsearch.search.aggregations.bucket.MergingBucketsDeferringCollector; +import org.elasticsearch.search.aggregations.bucket.nested.NestedAggregator; +import org.elasticsearch.search.aggregations.pipeline.PipelineAggregator; +import org.elasticsearch.search.aggregations.support.ValuesSource; +import org.elasticsearch.search.internal.SearchContext; + +import java.io.IOException; +import java.util.List; +import java.util.Map; +import java.util.Random; + +public abstract class AbstractRareTermsAggregator extends DeferableBucketAggregator { + + static final BucketOrder ORDER = BucketOrder.compound(BucketOrder.count(true), BucketOrder.key(true)); // sort by count ascending + + protected final long maxDocCount; + protected final double precision; + protected final DocValueFormat format; + protected final T valuesSource; + protected final U includeExclude; + + MergingBucketsDeferringCollector deferringCollector; + LeafBucketCollector subCollectors; + final SetBackedScalingCuckooFilter filter; + + AbstractRareTermsAggregator(String name, AggregatorFactories factories, SearchContext context, + Aggregator parent, List pipelineAggregators, + Map metaData, long maxDocCount, double precision, + DocValueFormat format, T valuesSource, U includeExclude) throws IOException { + super(name, factories, context, parent, pipelineAggregators, metaData); + + // We seed the rng with the ShardID so results are deterministic and don't change randomly + this.filter = new SetBackedScalingCuckooFilter(10000, new Random(context.indexShard().shardId().hashCode()), precision); + this.filter.registerBreaker(this::addRequestCircuitBreakerBytes); + + this.maxDocCount = maxDocCount; + this.precision = precision; + this.format = format; + this.valuesSource = valuesSource; + this.includeExclude = includeExclude; + String scoringAgg = subAggsNeedScore(); + String nestedAgg = descendsFromNestedAggregator(parent); + if (scoringAgg != null && nestedAgg != null) { + /* + * Terms agg would force the collect mode to depth_first here, because + * we need to access the score of nested documents in a sub-aggregation + * and we are not able to generate this score while replaying deferred documents. + * + * But the RareTerms agg _must_ execute in breadth first since it relies on + * deferring execution, so we just have to throw up our hands and refuse + */ + throw new IllegalStateException("RareTerms agg [" + name() + "] is the child of the nested agg [" + nestedAgg + + "], and also has a scoring child agg [" + scoringAgg + "]. This combination is not supported because " + + "it requires executing in [depth_first] mode, which the RareTerms agg cannot do."); + } + } + + @Override + protected boolean shouldDefer(Aggregator aggregator) { + return true; + } + + @Override + public DeferringBucketCollector getDeferringCollector() { + deferringCollector = new MergingBucketsDeferringCollector(context, descendsFromGlobalAggregator(parent())); + return deferringCollector; + } + + private String subAggsNeedScore() { + for (Aggregator subAgg : subAggregators) { + if (subAgg.scoreMode().needsScores()) { + return subAgg.name(); + } + } + return null; + } + + private String descendsFromNestedAggregator(Aggregator parent) { + while (parent != null) { + if (parent.getClass() == NestedAggregator.class) { + return parent.name(); + } + parent = parent.parent(); + } + return null; + } + + protected void doCollect(V val, int docId) throws IOException { + long bucketOrdinal = addValueToOrds(val); + + if (bucketOrdinal < 0) { // already seen + bucketOrdinal = -1 - bucketOrdinal; + collectExistingBucket(subCollectors, docId, bucketOrdinal); + } else { + collectBucket(subCollectors, docId, bucketOrdinal); + } + } + + /** + * Add's the value to the ordinal map. Return the newly allocated id if it wasn't in the ordinal map yet, + * or -1-id if it was already present + */ + abstract long addValueToOrds(V value); +} diff --git a/server/src/main/java/org/elasticsearch/search/aggregations/bucket/terms/IncludeExclude.java b/server/src/main/java/org/elasticsearch/search/aggregations/bucket/terms/IncludeExclude.java index 8154108f9f0bc..30653f04a355a 100644 --- a/server/src/main/java/org/elasticsearch/search/aggregations/bucket/terms/IncludeExclude.java +++ b/server/src/main/java/org/elasticsearch/search/aggregations/bucket/terms/IncludeExclude.java @@ -137,10 +137,12 @@ public static IncludeExclude parseExclude(XContentParser parser) throws IOExcept } } + public abstract static class Filter {} + // The includeValue and excludeValue ByteRefs which are the result of the parsing // process are converted into a LongFilter when used on numeric fields // in the index. - public abstract static class LongFilter { + public abstract static class LongFilter extends Filter { public abstract boolean accept(long value); } @@ -183,7 +185,7 @@ private void addReject(long val) { } // Only used for the 'map' execution mode (ie. scripts) - public abstract static class StringFilter { + public abstract static class StringFilter extends Filter { public abstract boolean accept(BytesRef value); } @@ -231,7 +233,7 @@ public boolean accept(BytesRef value) { } } - public abstract static class OrdinalsFilter { + public abstract static class OrdinalsFilter extends Filter { public abstract LongBitSet acceptedGlobalOrdinals(SortedSetDocValues globalOrdinals) throws IOException; } diff --git a/server/src/main/java/org/elasticsearch/search/aggregations/bucket/terms/InternalMappedRareTerms.java b/server/src/main/java/org/elasticsearch/search/aggregations/bucket/terms/InternalMappedRareTerms.java new file mode 100644 index 0000000000000..d774d09fa1862 --- /dev/null +++ b/server/src/main/java/org/elasticsearch/search/aggregations/bucket/terms/InternalMappedRareTerms.java @@ -0,0 +1,182 @@ +/* + * 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.search.aggregations.bucket.terms; + +import org.apache.logging.log4j.LogManager; +import org.apache.logging.log4j.Logger; +import org.apache.lucene.util.CollectionUtil; +import org.elasticsearch.common.Randomness; +import org.elasticsearch.common.io.stream.StreamInput; +import org.elasticsearch.common.io.stream.StreamOutput; +import org.elasticsearch.common.util.SetBackedScalingCuckooFilter; +import org.elasticsearch.common.xcontent.XContentBuilder; +import org.elasticsearch.search.DocValueFormat; +import org.elasticsearch.search.aggregations.AggregationExecutionException; +import org.elasticsearch.search.aggregations.BucketOrder; +import org.elasticsearch.search.aggregations.InternalAggregation; +import org.elasticsearch.search.aggregations.pipeline.PipelineAggregator; + +import java.io.IOException; +import java.util.ArrayList; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.Objects; +import java.util.function.Function; +import java.util.stream.Collectors; + +public abstract class InternalMappedRareTerms, B extends InternalRareTerms.Bucket> + extends InternalRareTerms { + + protected DocValueFormat format; + protected List buckets; + protected Map bucketMap; + + final SetBackedScalingCuckooFilter filter; + + protected final Logger logger = LogManager.getLogger(getClass()); + + InternalMappedRareTerms(String name, BucketOrder order, List pipelineAggregators, + Map metaData, DocValueFormat format, + List buckets, long maxDocCount, SetBackedScalingCuckooFilter filter) { + super(name, order, maxDocCount, pipelineAggregators, metaData); + this.format = format; + this.buckets = buckets; + this.filter = filter; + } + + public long getMaxDocCount() { + return maxDocCount; + } + + SetBackedScalingCuckooFilter getFilter() { + return filter; + } + + /** + * Read from a stream. + */ + InternalMappedRareTerms(StreamInput in, Bucket.Reader bucketReader) throws IOException { + super(in); + format = in.readNamedWriteable(DocValueFormat.class); + buckets = in.readList(stream -> bucketReader.read(stream, format)); + filter = new SetBackedScalingCuckooFilter(in, Randomness.get()); + } + + @Override + protected void writeTermTypeInfoTo(StreamOutput out) throws IOException { + out.writeNamedWriteable(format); + out.writeList(buckets); + filter.writeTo(out); + } + + @Override + public InternalAggregation doReduce(List aggregations, ReduceContext reduceContext) { + Map> buckets = new HashMap<>(); + InternalRareTerms referenceTerms = null; + SetBackedScalingCuckooFilter filter = null; + + for (InternalAggregation aggregation : aggregations) { + // Unmapped rare terms don't have a cuckoo filter so we'll skip all this work + // and save some type casting headaches later. + if (aggregation.isMapped() == false) { + continue; + } + + @SuppressWarnings("unchecked") + InternalRareTerms terms = (InternalRareTerms) aggregation; + if (referenceTerms == null && aggregation.getClass().equals(UnmappedRareTerms.class) == false) { + referenceTerms = terms; + } + if (referenceTerms != null && + referenceTerms.getClass().equals(terms.getClass()) == false && + terms.getClass().equals(UnmappedRareTerms.class) == false) { + // control gets into this loop when the same field name against which the query is executed + // is of different types in different indices. + throw new AggregationExecutionException("Merging/Reducing the aggregations failed when computing the aggregation [" + + referenceTerms.getName() + "] because the field you gave in the aggregation query existed as two different " + + "types in two different indices"); + } + for (B bucket : terms.getBuckets()) { + List bucketList = buckets.computeIfAbsent(bucket.getKey(), k -> new ArrayList<>()); + bucketList.add(bucket); + } + + SetBackedScalingCuckooFilter otherFilter = ((InternalMappedRareTerms)aggregation).getFilter(); + if (filter == null) { + filter = new SetBackedScalingCuckooFilter(otherFilter); + } else { + filter.merge(otherFilter); + } + } + + final List rare = new ArrayList<>(); + for (List sameTermBuckets : buckets.values()) { + final B b = sameTermBuckets.get(0).reduce(sameTermBuckets, reduceContext); + if ((b.getDocCount() <= maxDocCount && containsTerm(filter, b) == false)) { + rare.add(b); + reduceContext.consumeBucketsAndMaybeBreak(1); + } else if (b.getDocCount() > maxDocCount) { + // this term has gone over threshold while merging, so add it to the filter. + // Note this may happen during incremental reductions too + addToFilter(filter, b); + } + } + CollectionUtil.introSort(rare, order.comparator(null)); + return createWithFilter(name, rare, filter); + } + + public abstract boolean containsTerm(SetBackedScalingCuckooFilter filter, B bucket); + + public abstract void addToFilter(SetBackedScalingCuckooFilter filter, B bucket); + + @Override + public List getBuckets() { + return buckets; + } + + @Override + public B getBucketByKey(String term) { + if (bucketMap == null) { + bucketMap = buckets.stream().collect(Collectors.toMap(InternalRareTerms.Bucket::getKeyAsString, Function.identity())); + } + return bucketMap.get(term); + } + + @Override + public boolean equals(Object obj) { + if (this == obj) return true; + if (obj == null || getClass() != obj.getClass()) return false; + if (super.equals(obj) == false) return false; + InternalMappedRareTerms that = (InternalMappedRareTerms) obj; + return Objects.equals(buckets, that.buckets) + && Objects.equals(format, that.format) + && Objects.equals(filter, that.filter); + } + + @Override + public int hashCode() { + return Objects.hash(super.hashCode(), buckets, format, filter); + } + + @Override + public final XContentBuilder doXContentBody(XContentBuilder builder, Params params) throws IOException { + return doXContentCommon(builder, params, buckets); + } +} diff --git a/server/src/main/java/org/elasticsearch/search/aggregations/bucket/terms/InternalMappedTerms.java b/server/src/main/java/org/elasticsearch/search/aggregations/bucket/terms/InternalMappedTerms.java index 547c9d0a80ec6..91c8c72cddb8b 100644 --- a/server/src/main/java/org/elasticsearch/search/aggregations/bucket/terms/InternalMappedTerms.java +++ b/server/src/main/java/org/elasticsearch/search/aggregations/bucket/terms/InternalMappedTerms.java @@ -115,10 +115,13 @@ public B getBucketByKey(String term) { } @Override - protected boolean doEquals(Object obj) { + public boolean equals(Object obj) { + if (this == obj) return true; + if (obj == null || getClass() != obj.getClass()) return false; + if (super.equals(obj) == false) return false; + InternalMappedTerms that = (InternalMappedTerms) obj; - return super.doEquals(obj) - && Objects.equals(buckets, that.buckets) + return Objects.equals(buckets, that.buckets) && Objects.equals(format, that.format) && Objects.equals(otherDocCount, that.otherDocCount) && Objects.equals(showTermDocCountError, that.showTermDocCountError) @@ -127,8 +130,8 @@ protected boolean doEquals(Object obj) { } @Override - protected int doHashCode() { - return Objects.hash(super.doHashCode(), buckets, format, otherDocCount, showTermDocCountError, shardSize); + public int hashCode() { + return Objects.hash(super.hashCode(), buckets, format, otherDocCount, showTermDocCountError, shardSize); } @Override diff --git a/server/src/main/java/org/elasticsearch/search/aggregations/bucket/terms/InternalRareTerms.java b/server/src/main/java/org/elasticsearch/search/aggregations/bucket/terms/InternalRareTerms.java new file mode 100644 index 0000000000000..dd1a0c19200cf --- /dev/null +++ b/server/src/main/java/org/elasticsearch/search/aggregations/bucket/terms/InternalRareTerms.java @@ -0,0 +1,205 @@ +/* + * 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.search.aggregations.bucket.terms; + +import org.elasticsearch.common.io.stream.StreamInput; +import org.elasticsearch.common.io.stream.StreamOutput; +import org.elasticsearch.common.util.SetBackedScalingCuckooFilter; +import org.elasticsearch.common.xcontent.XContentBuilder; +import org.elasticsearch.search.DocValueFormat; +import org.elasticsearch.search.aggregations.Aggregations; +import org.elasticsearch.search.aggregations.BucketOrder; +import org.elasticsearch.search.aggregations.InternalAggregation; +import org.elasticsearch.search.aggregations.InternalAggregations; +import org.elasticsearch.search.aggregations.InternalMultiBucketAggregation; +import org.elasticsearch.search.aggregations.InternalOrder; +import org.elasticsearch.search.aggregations.KeyComparable; +import org.elasticsearch.search.aggregations.pipeline.PipelineAggregator; + +import java.io.IOException; +import java.util.ArrayList; +import java.util.List; +import java.util.Map; +import java.util.Objects; + +public abstract class InternalRareTerms, B extends InternalRareTerms.Bucket> + extends InternalMultiBucketAggregation implements RareTerms { + + public abstract static class Bucket> extends InternalMultiBucketAggregation.InternalBucket + implements RareTerms.Bucket, KeyComparable { + /** + * Reads a bucket. Should be a constructor reference. + */ + @FunctionalInterface + public interface Reader> { + B read(StreamInput in, DocValueFormat format) throws IOException; + } + + long bucketOrd; + + protected long docCount; + protected InternalAggregations aggregations; + protected final DocValueFormat format; + + protected Bucket(long docCount, InternalAggregations aggregations, DocValueFormat formatter) { + this.format = formatter; + this.docCount = docCount; + this.aggregations = aggregations; + } + + /** + * Read from a stream. + */ + protected Bucket(StreamInput in, DocValueFormat formatter) throws IOException { + this.format = formatter; + docCount = in.readVLong(); + aggregations = new InternalAggregations(in); + } + + @Override + public final void writeTo(StreamOutput out) throws IOException { + out.writeVLong(getDocCount()); + aggregations.writeTo(out); + writeTermTo(out); + } + + protected abstract void writeTermTo(StreamOutput out) throws IOException; + + @Override + public long getDocCount() { + return docCount; + } + + @Override + public Aggregations getAggregations() { + return aggregations; + } + + abstract B newBucket(long docCount, InternalAggregations aggs); + + public B reduce(List buckets, ReduceContext context) { + long docCount = 0; + List aggregationsList = new ArrayList<>(buckets.size()); + for (B bucket : buckets) { + docCount += bucket.docCount; + aggregationsList.add(bucket.aggregations); + } + InternalAggregations aggs = InternalAggregations.reduce(aggregationsList, context); + return newBucket(docCount, aggs); + } + + @Override + public final XContentBuilder toXContent(XContentBuilder builder, Params params) throws IOException { + builder.startObject(); + keyToXContent(builder); + builder.field(CommonFields.DOC_COUNT.getPreferredName(), getDocCount()); + aggregations.toXContentInternal(builder, params); + builder.endObject(); + return builder; + } + + protected abstract XContentBuilder keyToXContent(XContentBuilder builder) throws IOException; + + @Override + public boolean equals(Object obj) { + if (obj == null || getClass() != obj.getClass()) { + return false; + } + Bucket that = (Bucket) obj; + return Objects.equals(docCount, that.docCount) + && Objects.equals(aggregations, that.aggregations); + } + + @Override + public int hashCode() { + return Objects.hash(getClass(), docCount, aggregations); + } + } + + protected final BucketOrder order; + protected final long maxDocCount; + + protected InternalRareTerms(String name, BucketOrder order, long maxDocCount, + List pipelineAggregators, Map metaData) { + super(name, pipelineAggregators, metaData); + this.order = order; + this.maxDocCount = maxDocCount; + } + + /** + * Read from a stream. + */ + protected InternalRareTerms(StreamInput in) throws IOException { + super(in); + order = InternalOrder.Streams.readOrder(in); + maxDocCount = in.readVLong(); + } + + @Override + protected final void doWriteTo(StreamOutput out) throws IOException { + order.writeTo(out); + out.writeVLong(maxDocCount); + writeTermTypeInfoTo(out); + } + + protected abstract void writeTermTypeInfoTo(StreamOutput out) throws IOException; + + @Override + public abstract List getBuckets(); + + @Override + public abstract B getBucketByKey(String term); + + @Override + public InternalAggregation doReduce(List aggregations, ReduceContext reduceContext) { + throw new UnsupportedOperationException(); + } + + protected abstract A createWithFilter(String name, List buckets, SetBackedScalingCuckooFilter filter); + + /** + * Create an array to hold some buckets. Used in collecting the results. + */ + protected abstract B[] createBucketsArray(int size); + + @Override + public boolean equals(Object obj) { + if (this == obj) return true; + if (obj == null || getClass() != obj.getClass()) return false; + if (super.equals(obj) == false) return false; + InternalRareTerms that = (InternalRareTerms) obj; + return Objects.equals(maxDocCount, that.maxDocCount) + && Objects.equals(order, that.order); + } + + @Override + public int hashCode() { + return Objects.hash(super.hashCode(), maxDocCount, order); + } + + protected static XContentBuilder doXContentCommon(XContentBuilder builder, Params params, + List buckets) throws IOException { + builder.startArray(CommonFields.BUCKETS.getPreferredName()); + for (Bucket bucket : buckets) { + bucket.toXContent(builder, params); + } + builder.endArray(); + return builder; + } +} diff --git a/server/src/main/java/org/elasticsearch/search/aggregations/bucket/terms/InternalTerms.java b/server/src/main/java/org/elasticsearch/search/aggregations/bucket/terms/InternalTerms.java index f40ff84bf2130..fc607621ff7ed 100644 --- a/server/src/main/java/org/elasticsearch/search/aggregations/bucket/terms/InternalTerms.java +++ b/server/src/main/java/org/elasticsearch/search/aggregations/bucket/terms/InternalTerms.java @@ -326,7 +326,11 @@ public InternalAggregation doReduce(List aggregations, Redu protected abstract B[] createBucketsArray(int size); @Override - protected boolean doEquals(Object obj) { + public boolean equals(Object obj) { + if (this == obj) return true; + if (obj == null || getClass() != obj.getClass()) return false; + if (super.equals(obj) == false) return false; + InternalTerms that = (InternalTerms) obj; return Objects.equals(minDocCount, that.minDocCount) && Objects.equals(order, that.order) @@ -334,8 +338,8 @@ protected boolean doEquals(Object obj) { } @Override - protected int doHashCode() { - return Objects.hash(minDocCount, order, requiredSize); + public int hashCode() { + return Objects.hash(super.hashCode(), minDocCount, order, requiredSize); } protected static XContentBuilder doXContentCommon(XContentBuilder builder, Params params, diff --git a/server/src/main/java/org/elasticsearch/search/aggregations/bucket/terms/LongRareTerms.java b/server/src/main/java/org/elasticsearch/search/aggregations/bucket/terms/LongRareTerms.java new file mode 100644 index 0000000000000..29f84fb6030e1 --- /dev/null +++ b/server/src/main/java/org/elasticsearch/search/aggregations/bucket/terms/LongRareTerms.java @@ -0,0 +1,156 @@ +/* + * 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.search.aggregations.bucket.terms; + + +import org.elasticsearch.common.io.stream.StreamInput; +import org.elasticsearch.common.io.stream.StreamOutput; +import org.elasticsearch.common.util.SetBackedScalingCuckooFilter; +import org.elasticsearch.common.xcontent.XContentBuilder; +import org.elasticsearch.search.DocValueFormat; +import org.elasticsearch.search.aggregations.BucketOrder; +import org.elasticsearch.search.aggregations.InternalAggregations; +import org.elasticsearch.search.aggregations.pipeline.PipelineAggregator; + +import java.io.IOException; +import java.util.List; +import java.util.Map; +import java.util.Objects; + +/** + * Result of the RareTerms aggregation when the field is some kind of whole number like a integer, long, or a date. + */ +public class LongRareTerms extends InternalMappedRareTerms { + public static final String NAME = "lrareterms"; + + public static class Bucket extends InternalRareTerms.Bucket { + long term; + + public Bucket(long term, long docCount, InternalAggregations aggregations, DocValueFormat format) { + super(docCount, aggregations, format); + this.term = term; + } + + /** + * Read from a stream. + */ + public Bucket(StreamInput in, DocValueFormat format) throws IOException { + super(in, format); + term = in.readLong(); + } + + @Override + protected void writeTermTo(StreamOutput out) throws IOException { + out.writeLong(term); + } + + @Override + public String getKeyAsString() { + return format.format(term).toString(); + } + + @Override + public Object getKey() { + return term; + } + + @Override + public Number getKeyAsNumber() { + return term; + } + + @Override + public int compareKey(Bucket other) { + return Long.compare(term, other.term); + } + + @Override + Bucket newBucket(long docCount, InternalAggregations aggs) { + return new Bucket(term, docCount, aggs, format); + } + + @Override + protected final XContentBuilder keyToXContent(XContentBuilder builder) throws IOException { + builder.field(CommonFields.KEY.getPreferredName(), term); + if (format != DocValueFormat.RAW) { + builder.field(CommonFields.KEY_AS_STRING.getPreferredName(), format.format(term).toString()); + } + return builder; + } + + @Override + public boolean equals(Object obj) { + return super.equals(obj) && Objects.equals(term, ((Bucket) obj).term); + } + + @Override + public int hashCode() { + return Objects.hash(super.hashCode(), term); + } + } + + LongRareTerms(String name, BucketOrder order, List pipelineAggregators, + Map metaData, DocValueFormat format, + List buckets, long maxDocCount, SetBackedScalingCuckooFilter filter) { + super(name, order, pipelineAggregators, metaData, format, buckets, maxDocCount, filter); + } + + /** + * Read from a stream. + */ + public LongRareTerms(StreamInput in) throws IOException { + super(in, LongRareTerms.Bucket::new); + } + + @Override + public String getWriteableName() { + return NAME; + } + + @Override + public LongRareTerms create(List buckets) { + return new LongRareTerms(name, order, pipelineAggregators(), metaData, format, buckets, maxDocCount, filter); + } + + @Override + public LongRareTerms.Bucket createBucket(InternalAggregations aggregations, LongRareTerms.Bucket prototype) { + return new LongRareTerms.Bucket(prototype.term, prototype.getDocCount(), aggregations, prototype.format); + } + + @Override + protected LongRareTerms createWithFilter(String name, List buckets, SetBackedScalingCuckooFilter filter) { + return new LongRareTerms(name, order, pipelineAggregators(), getMetaData(), format, + buckets, maxDocCount, filter); + } + + @Override + protected LongRareTerms.Bucket[] createBucketsArray(int size) { + return new LongRareTerms.Bucket[size]; + } + + @Override + public boolean containsTerm(SetBackedScalingCuckooFilter filter, LongRareTerms.Bucket bucket) { + return filter.mightContain((long) bucket.getKey()); + } + + @Override + public void addToFilter(SetBackedScalingCuckooFilter filter, LongRareTerms.Bucket bucket) { + filter.add((long) bucket.getKey()); + } +} diff --git a/server/src/main/java/org/elasticsearch/search/aggregations/bucket/terms/LongRareTermsAggregator.java b/server/src/main/java/org/elasticsearch/search/aggregations/bucket/terms/LongRareTermsAggregator.java new file mode 100644 index 0000000000000..b1d294fefdcf6 --- /dev/null +++ b/server/src/main/java/org/elasticsearch/search/aggregations/bucket/terms/LongRareTermsAggregator.java @@ -0,0 +1,169 @@ +/* + * 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.search.aggregations.bucket.terms; + +import org.apache.lucene.index.LeafReaderContext; +import org.apache.lucene.index.SortedNumericDocValues; +import org.apache.lucene.util.CollectionUtil; +import org.elasticsearch.common.lease.Releasables; +import org.elasticsearch.common.util.LongHash; +import org.elasticsearch.search.DocValueFormat; +import org.elasticsearch.search.aggregations.Aggregator; +import org.elasticsearch.search.aggregations.AggregatorFactories; +import org.elasticsearch.search.aggregations.InternalAggregation; +import org.elasticsearch.search.aggregations.LeafBucketCollector; +import org.elasticsearch.search.aggregations.LeafBucketCollectorBase; +import org.elasticsearch.search.aggregations.pipeline.PipelineAggregator; +import org.elasticsearch.search.aggregations.support.ValuesSource; +import org.elasticsearch.search.internal.SearchContext; + +import java.io.IOException; +import java.util.ArrayList; +import java.util.List; +import java.util.Map; + +import static java.util.Collections.emptyList; + +/** + * An aggregator that finds "rare" string values (e.g. terms agg that orders ascending) + */ +public class LongRareTermsAggregator extends AbstractRareTermsAggregator { + + protected LongHash bucketOrds; + + LongRareTermsAggregator(String name, AggregatorFactories factories, ValuesSource.Numeric valuesSource, DocValueFormat format, + SearchContext aggregationContext, Aggregator parent, IncludeExclude.LongFilter longFilter, + int maxDocCount, double precision, List pipelineAggregators, + Map metaData) throws IOException { + super(name, factories, aggregationContext, parent, pipelineAggregators, metaData, maxDocCount, precision, + format, valuesSource, longFilter); + this.bucketOrds = new LongHash(1, aggregationContext.bigArrays()); + } + + protected SortedNumericDocValues getValues(ValuesSource.Numeric valuesSource, LeafReaderContext ctx) throws IOException { + return valuesSource.longValues(ctx); + } + + @Override + public LeafBucketCollector getLeafCollector(LeafReaderContext ctx, + final LeafBucketCollector sub) throws IOException { + final SortedNumericDocValues values = getValues(valuesSource, ctx); + if (subCollectors == null) { + subCollectors = sub; + } + return new LeafBucketCollectorBase(sub, values) { + + @Override + public void collect(int docId, long owningBucketOrdinal) throws IOException { + if (values.advanceExact(docId)) { + final int valuesCount = values.docValueCount(); + long previous = Long.MAX_VALUE; + for (int i = 0; i < valuesCount; ++i) { + final long val = values.nextValue(); + if (previous != val || i == 0) { + if ((includeExclude == null) || (includeExclude.accept(val))) { + doCollect(val, docId); + } + previous = val; + } + } + } + } + }; + } + + @Override + long addValueToOrds(Long value) { + return bucketOrds.add(value); + } + + /** + * Merges the ordinals to a minimal set, populates the CuckooFilter and + * generates a final set of buckets. + * + * If a term is below the maxDocCount, it is turned into a Bucket. Otherwise, + * the term is added to the filter, and pruned from the ordinal map. If + * necessary the ordinal map is merged down to a minimal set to remove deletions + */ + private List buildSketch() { + long deletionCount = 0; + LongHash newBucketOrds = new LongHash(1, context.bigArrays()); + List buckets = new ArrayList<>(); + try (LongHash oldBucketOrds = bucketOrds) { + + long[] mergeMap = new long[(int) oldBucketOrds.size()]; + for (int i = 0; i < oldBucketOrds.size(); i++) { + long oldKey = oldBucketOrds.get(i); + long newBucketOrd = -1; + + long docCount = bucketDocCount(i); + // if the key is below threshold, reinsert into the new ords + if (docCount <= maxDocCount) { + newBucketOrd = newBucketOrds.add(oldKey); + LongRareTerms.Bucket bucket = new LongRareTerms.Bucket(oldKey, docCount, null, format); + bucket.bucketOrd = newBucketOrd; + buckets.add(bucket); + + consumeBucketsAndMaybeBreak(1); + } else { + // Make a note when one of the ords has been deleted + deletionCount += 1; + filter.add(oldKey); + } + mergeMap[i] = newBucketOrd; + } + + // Only merge/delete the ordinals if we have actually deleted one, + // to save on some redundant work + if (deletionCount > 0) { + mergeBuckets(mergeMap, newBucketOrds.size()); + if (deferringCollector != null) { + deferringCollector.mergeBuckets(mergeMap); + } + } + } + bucketOrds = newBucketOrds; + return buckets; + } + + @Override + public InternalAggregation buildAggregation(long owningBucketOrdinal) throws IOException { + assert owningBucketOrdinal == 0; + List buckets = buildSketch(); + runDeferredCollections(buckets.stream().mapToLong(b -> b.bucketOrd).toArray()); + + // Finalize the buckets + for (LongRareTerms.Bucket bucket : buckets) { + bucket.aggregations = bucketAggregations(bucket.bucketOrd); + } + + CollectionUtil.introSort(buckets, ORDER.comparator(this)); + return new LongRareTerms(name, ORDER, pipelineAggregators(), metaData(), format, buckets, maxDocCount, filter); + } + + @Override + public InternalAggregation buildEmptyAggregation() { + return new LongRareTerms(name, ORDER, pipelineAggregators(), metaData(), format, emptyList(), 0, filter); + } + + @Override + public void doClose() { + Releasables.close(bucketOrds); + } +} diff --git a/server/src/main/java/org/elasticsearch/search/aggregations/bucket/terms/RareTerms.java b/server/src/main/java/org/elasticsearch/search/aggregations/bucket/terms/RareTerms.java new file mode 100644 index 0000000000000..2248514783264 --- /dev/null +++ b/server/src/main/java/org/elasticsearch/search/aggregations/bucket/terms/RareTerms.java @@ -0,0 +1,48 @@ +/* + * 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.search.aggregations.bucket.terms; + +import org.elasticsearch.search.aggregations.bucket.MultiBucketsAggregation; + +import java.util.List; + + +public interface RareTerms extends MultiBucketsAggregation { + + /** + * A bucket that is associated with a single term + */ + interface Bucket extends MultiBucketsAggregation.Bucket { + + Number getKeyAsNumber(); + } + + /** + * Return the sorted list of the buckets in this terms aggregation. + */ + @Override + List getBuckets(); + + /** + * Get the bucket for the given term, or null if there is no such bucket. + */ + Bucket getBucketByKey(String term); + +} + diff --git a/server/src/main/java/org/elasticsearch/search/aggregations/bucket/terms/RareTermsAggregationBuilder.java b/server/src/main/java/org/elasticsearch/search/aggregations/bucket/terms/RareTermsAggregationBuilder.java new file mode 100644 index 0000000000000..285869dd2e0cf --- /dev/null +++ b/server/src/main/java/org/elasticsearch/search/aggregations/bucket/terms/RareTermsAggregationBuilder.java @@ -0,0 +1,203 @@ +/* + * 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.search.aggregations.bucket.terms; + +import org.elasticsearch.common.ParseField; +import org.elasticsearch.common.io.stream.StreamInput; +import org.elasticsearch.common.io.stream.StreamOutput; +import org.elasticsearch.common.xcontent.ObjectParser; +import org.elasticsearch.common.xcontent.XContentBuilder; +import org.elasticsearch.common.xcontent.XContentParser; +import org.elasticsearch.search.aggregations.AggregationBuilder; +import org.elasticsearch.search.aggregations.AggregatorFactories.Builder; +import org.elasticsearch.search.aggregations.AggregatorFactory; +import org.elasticsearch.search.aggregations.support.ValueType; +import org.elasticsearch.search.aggregations.support.ValuesSource; +import org.elasticsearch.search.aggregations.support.ValuesSourceAggregationBuilder; +import org.elasticsearch.search.aggregations.support.ValuesSourceAggregatorFactory; +import org.elasticsearch.search.aggregations.support.ValuesSourceConfig; +import org.elasticsearch.search.aggregations.support.ValuesSourceParserHelper; +import org.elasticsearch.search.aggregations.support.ValuesSourceType; +import org.elasticsearch.search.internal.SearchContext; + +import java.io.IOException; +import java.util.Map; +import java.util.Objects; + +public class RareTermsAggregationBuilder extends ValuesSourceAggregationBuilder { + public static final String NAME = "rare_terms"; + + private static final ParseField MAX_DOC_COUNT_FIELD_NAME = new ParseField("max_doc_count"); + private static final ParseField PRECISION = new ParseField("precision"); + + private static final int MAX_MAX_DOC_COUNT = 100; + private static final ObjectParser PARSER; + static { + PARSER = new ObjectParser<>(RareTermsAggregationBuilder.NAME); + ValuesSourceParserHelper.declareAnyFields(PARSER, true, true); + PARSER.declareLong(RareTermsAggregationBuilder::maxDocCount, MAX_DOC_COUNT_FIELD_NAME); + + PARSER.declareField((b, v) -> b.includeExclude(IncludeExclude.merge(v, b.includeExclude())), + IncludeExclude::parseInclude, IncludeExclude.INCLUDE_FIELD, ObjectParser.ValueType.OBJECT_ARRAY_OR_STRING); + + PARSER.declareField((b, v) -> b.includeExclude(IncludeExclude.merge(b.includeExclude(), v)), + IncludeExclude::parseExclude, IncludeExclude.EXCLUDE_FIELD, ObjectParser.ValueType.STRING_ARRAY); + + PARSER.declareDouble(RareTermsAggregationBuilder::setPrecision, PRECISION); + } + + public static AggregationBuilder parse(String aggregationName, XContentParser parser) throws IOException { + return PARSER.parse(parser, new RareTermsAggregationBuilder(aggregationName, null), null); + } + + private IncludeExclude includeExclude = null; + private int maxDocCount = 1; + private double precision = 0.001; + + public RareTermsAggregationBuilder(String name, ValueType valueType) { + super(name, ValuesSourceType.ANY, valueType); + } + + private RareTermsAggregationBuilder(RareTermsAggregationBuilder clone, Builder factoriesBuilder, Map metaData) { + super(clone, factoriesBuilder, metaData); + this.includeExclude = clone.includeExclude; + } + + @Override + protected AggregationBuilder shallowCopy(Builder factoriesBuilder, Map metaData) { + return new RareTermsAggregationBuilder(this, factoriesBuilder, metaData); + } + + /** + * Read from a stream. + */ + public RareTermsAggregationBuilder(StreamInput in) throws IOException { + super(in, ValuesSourceType.ANY); + includeExclude = in.readOptionalWriteable(IncludeExclude::new); + maxDocCount = in.readVInt(); + } + + @Override + protected boolean serializeTargetValueType() { + return true; + } + + @Override + protected void innerWriteTo(StreamOutput out) throws IOException { + out.writeOptionalWriteable(includeExclude); + out.writeVInt(maxDocCount); + } + + /** + * Set the maximum document count terms should have in order to appear in + * the response. + */ + public RareTermsAggregationBuilder maxDocCount(long maxDocCount) { + if (maxDocCount <= 0) { + throw new IllegalArgumentException( + "[" + MAX_DOC_COUNT_FIELD_NAME.getPreferredName() + "] must be greater than 0. Found [" + + maxDocCount + "] in [" + name + "]"); + } + //TODO review: what size cap should we put on this? + if (maxDocCount > MAX_MAX_DOC_COUNT) { + throw new IllegalArgumentException("[" + MAX_DOC_COUNT_FIELD_NAME.getPreferredName() + "] must be smaller" + + "than " + MAX_MAX_DOC_COUNT + "in [" + name + "]"); + } + this.maxDocCount = (int) maxDocCount; + return this; + } + + /** + * Set terms to include and exclude from the aggregation results + */ + public RareTermsAggregationBuilder includeExclude(IncludeExclude includeExclude) { + this.includeExclude = includeExclude; + return this; + } + + /** + * Get terms to include and exclude from the aggregation results + */ + public IncludeExclude includeExclude() { + return includeExclude; + } + + /** + * Get the current false positive rate for individual cuckoo filters. + */ + public double getPrecision() { + return precision; + } + + /** + * Set's the false-positive rate for individual cuckoo filters. Does not dictate the overall fpp rate + * since we use a "scaling" cuckoo filter which adds more filters as required, and the overall + * error rate grows differently than individual filters + * + * This value does, however, affect the overall space usage of the filter. Coarser precisions provide + * more compact filters. The default is 0.01 + */ + public void setPrecision(double precision) { + if (precision < 0.00001) { + throw new IllegalArgumentException("[precision] must be greater than 0.00001"); + } + this.precision = precision; + } + + @Override + protected ValuesSourceAggregatorFactory innerBuild(SearchContext context, + ValuesSourceConfig config, + AggregatorFactory parent, + Builder subFactoriesBuilder) throws IOException { + return new RareTermsAggregatorFactory(name, config, includeExclude, + context, parent, subFactoriesBuilder, metaData, maxDocCount, precision); + } + + @Override + protected XContentBuilder doXContentBody(XContentBuilder builder, Params params) throws IOException { + if (includeExclude != null) { + includeExclude.toXContent(builder, params); + } + builder.field(MAX_DOC_COUNT_FIELD_NAME.getPreferredName(), maxDocCount); + builder.field(PRECISION.getPreferredName(), precision); + return builder; + } + + @Override + public int hashCode() { + return Objects.hash(super.hashCode(), includeExclude, maxDocCount, precision); + } + + @Override + public boolean equals(Object obj) { + if (this == obj) return true; + if (obj == null || getClass() != obj.getClass()) return false; + if (super.equals(obj) == false) return false; + RareTermsAggregationBuilder other = (RareTermsAggregationBuilder) obj; + return Objects.equals(includeExclude, other.includeExclude) + && Objects.equals(maxDocCount, other.maxDocCount) + && Objects.equals(precision, other.precision); + } + + @Override + public String getType() { + return NAME; + } + +} diff --git a/server/src/main/java/org/elasticsearch/search/aggregations/bucket/terms/RareTermsAggregatorFactory.java b/server/src/main/java/org/elasticsearch/search/aggregations/bucket/terms/RareTermsAggregatorFactory.java new file mode 100644 index 0000000000000..ab4c4bc00843a --- /dev/null +++ b/server/src/main/java/org/elasticsearch/search/aggregations/bucket/terms/RareTermsAggregatorFactory.java @@ -0,0 +1,164 @@ +/* + * 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.search.aggregations.bucket.terms; + +import org.elasticsearch.common.ParseField; +import org.elasticsearch.common.logging.DeprecationLogger; +import org.elasticsearch.search.DocValueFormat; +import org.elasticsearch.search.aggregations.AggregationExecutionException; +import org.elasticsearch.search.aggregations.Aggregator; +import org.elasticsearch.search.aggregations.AggregatorFactories; +import org.elasticsearch.search.aggregations.AggregatorFactory; +import org.elasticsearch.search.aggregations.InternalAggregation; +import org.elasticsearch.search.aggregations.NonCollectingAggregator; +import org.elasticsearch.search.aggregations.pipeline.PipelineAggregator; +import org.elasticsearch.search.aggregations.support.ValuesSource; +import org.elasticsearch.search.aggregations.support.ValuesSourceAggregatorFactory; +import org.elasticsearch.search.aggregations.support.ValuesSourceConfig; +import org.elasticsearch.search.internal.SearchContext; + +import java.io.IOException; +import java.util.List; +import java.util.Map; + +public class RareTermsAggregatorFactory extends ValuesSourceAggregatorFactory { + private final IncludeExclude includeExclude; + private final int maxDocCount; + private final double precision; + + RareTermsAggregatorFactory(String name, ValuesSourceConfig config, + IncludeExclude includeExclude, + SearchContext context, + AggregatorFactory parent, AggregatorFactories.Builder subFactoriesBuilder, + Map metaData, int maxDocCount, double precision) throws IOException { + super(name, config, context, parent, subFactoriesBuilder, metaData); + this.includeExclude = includeExclude; + this.maxDocCount = maxDocCount; + this.precision = precision; + } + + @Override + protected Aggregator createUnmapped(Aggregator parent, List pipelineAggregators, Map metaData) + throws IOException { + final InternalAggregation aggregation = new UnmappedRareTerms(name, pipelineAggregators, metaData); + return new NonCollectingAggregator(name, context, parent, factories, pipelineAggregators, metaData) { + @Override + public InternalAggregation buildEmptyAggregation() { + return aggregation; + } + }; + } + + @Override + protected Aggregator doCreateInternal(ValuesSource valuesSource, Aggregator parent, boolean collectsFromSingleBucket, + List pipelineAggregators, Map metaData) throws IOException { + if (collectsFromSingleBucket == false) { + return asMultiBucketAggregator(this, context, parent); + } + if (valuesSource instanceof ValuesSource.Bytes) { + ExecutionMode execution = ExecutionMode.MAP; //TODO global ords not implemented yet, only supports "map" + + DocValueFormat format = config.format(); + if ((includeExclude != null) && (includeExclude.isRegexBased()) && format != DocValueFormat.RAW) { + throw new AggregationExecutionException("Aggregation [" + name + "] cannot support " + + "regular expression style include/exclude settings as they can only be applied to string fields. " + + "Use an array of values for include/exclude clauses"); + } + + return execution.create(name, factories, valuesSource, format, + includeExclude, context, parent, pipelineAggregators, metaData, maxDocCount, precision); + } + + if ((includeExclude != null) && (includeExclude.isRegexBased())) { + throw new AggregationExecutionException("Aggregation [" + name + "] cannot support regular expression style include/exclude " + + "settings as they can only be applied to string fields. Use an array of numeric values for include/exclude clauses " + + "used to filter numeric fields"); + } + + if (valuesSource instanceof ValuesSource.Numeric) { + IncludeExclude.LongFilter longFilter = null; + if (((ValuesSource.Numeric) valuesSource).isFloatingPoint()) { + throw new AggregationExecutionException("RareTerms aggregation does not support floating point fields."); + } + if (includeExclude != null) { + longFilter = includeExclude.convertToLongFilter(config.format()); + } + return new LongRareTermsAggregator(name, factories, (ValuesSource.Numeric) valuesSource, config.format(), + context, parent, longFilter, maxDocCount, precision, pipelineAggregators, metaData); + } + + throw new AggregationExecutionException("RareTerms aggregation cannot be applied to field [" + config.fieldContext().field() + + "]. It can only be applied to numeric or string fields."); + } + + public enum ExecutionMode { + + MAP(new ParseField("map")) { + + @Override + Aggregator create(String name, AggregatorFactories factories, ValuesSource valuesSource, + DocValueFormat format, IncludeExclude includeExclude, + SearchContext context, Aggregator parent, + List pipelineAggregators, + Map metaData, long maxDocCount, double precision) + throws IOException { + final IncludeExclude.StringFilter filter = includeExclude == null ? null : includeExclude.convertToStringFilter(format); + return new StringRareTermsAggregator(name, factories, (ValuesSource.Bytes) valuesSource, format, filter, + context, parent, pipelineAggregators, metaData, maxDocCount, precision); + } + + @Override + boolean needsGlobalOrdinals() { + return false; + } + + }; + + public static ExecutionMode fromString(String value, final DeprecationLogger deprecationLogger) { + switch (value) { + case "map": + return MAP; + default: + throw new IllegalArgumentException("Unknown `execution_hint`: [" + value + "], expected any of [map]"); + } + } + + private final ParseField parseField; + + ExecutionMode(ParseField parseField) { + this.parseField = parseField; + } + + abstract Aggregator create(String name, AggregatorFactories factories, ValuesSource valuesSource, + DocValueFormat format, IncludeExclude includeExclude, + SearchContext context, Aggregator parent, + List pipelineAggregators, Map metaData, + long maxDocCount, double precision) + throws IOException; + + abstract boolean needsGlobalOrdinals(); + + @Override + public String toString() { + return parseField.getPreferredName(); + } + } + +} diff --git a/server/src/main/java/org/elasticsearch/search/aggregations/bucket/terms/StringRareTerms.java b/server/src/main/java/org/elasticsearch/search/aggregations/bucket/terms/StringRareTerms.java new file mode 100644 index 0000000000000..3c3e19664a631 --- /dev/null +++ b/server/src/main/java/org/elasticsearch/search/aggregations/bucket/terms/StringRareTerms.java @@ -0,0 +1,159 @@ +/* + * 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.search.aggregations.bucket.terms; + +import org.apache.lucene.util.BytesRef; +import org.elasticsearch.common.io.stream.StreamInput; +import org.elasticsearch.common.io.stream.StreamOutput; +import org.elasticsearch.common.util.SetBackedScalingCuckooFilter; +import org.elasticsearch.common.xcontent.XContentBuilder; +import org.elasticsearch.search.DocValueFormat; +import org.elasticsearch.search.aggregations.BucketOrder; +import org.elasticsearch.search.aggregations.InternalAggregations; +import org.elasticsearch.search.aggregations.pipeline.PipelineAggregator; + +import java.io.IOException; +import java.util.List; +import java.util.Map; +import java.util.Objects; + +public class StringRareTerms extends InternalMappedRareTerms { + public static final String NAME = "srareterms"; + + public static class Bucket extends InternalRareTerms.Bucket { + BytesRef termBytes; + + public Bucket(BytesRef term, long docCount, InternalAggregations aggregations, DocValueFormat format) { + super(docCount, aggregations, format); + this.termBytes = term; + } + + /** + * Read from a stream. + */ + public Bucket(StreamInput in, DocValueFormat format) throws IOException { + super(in, format); + termBytes = in.readBytesRef(); + } + + @Override + protected void writeTermTo(StreamOutput out) throws IOException { + out.writeBytesRef(termBytes); + } + + @Override + public Object getKey() { + return getKeyAsString(); + } + + // this method is needed for scripted numeric aggs + @Override + public Number getKeyAsNumber() { + /* + * If the term is a long greater than 2^52 then parsing as a double would lose accuracy. Therefore, we first parse as a long and + * if this fails then we attempt to parse the term as a double. + */ + try { + return Long.parseLong(termBytes.utf8ToString()); + } catch (final NumberFormatException ignored) { + return Double.parseDouble(termBytes.utf8ToString()); + } + } + + @Override + public String getKeyAsString() { + return format.format(termBytes).toString(); + } + + @Override + public int compareKey(Bucket other) { + return termBytes.compareTo(other.termBytes); + } + + @Override + Bucket newBucket(long docCount, InternalAggregations aggs) { + return new Bucket(termBytes, docCount, aggs, format); + } + + @Override + protected final XContentBuilder keyToXContent(XContentBuilder builder) throws IOException { + return builder.field(CommonFields.KEY.getPreferredName(), getKeyAsString()); + } + + @Override + public boolean equals(Object obj) { + return super.equals(obj) && Objects.equals(termBytes, ((Bucket) obj).termBytes); + } + + @Override + public int hashCode() { + return Objects.hash(super.hashCode(), termBytes); + } + } + + StringRareTerms(String name, BucketOrder order, List pipelineAggregators, + Map metaData, DocValueFormat format, + List buckets, long maxDocCount, SetBackedScalingCuckooFilter filter) { + super(name, order, pipelineAggregators, metaData, format, buckets, maxDocCount, filter); + } + + /** + * Read from a stream. + */ + public StringRareTerms(StreamInput in) throws IOException { + super(in, StringRareTerms.Bucket::new); + } + + @Override + public String getWriteableName() { + return NAME; + } + + @Override + public StringRareTerms create(List buckets) { + return new StringRareTerms(name, order, pipelineAggregators(), metaData, format, buckets, maxDocCount, filter); + } + + @Override + public StringRareTerms.Bucket createBucket(InternalAggregations aggregations, StringRareTerms.Bucket prototype) { + return new StringRareTerms.Bucket(prototype.termBytes, prototype.getDocCount(), aggregations, prototype.format); + } + + @Override + protected StringRareTerms createWithFilter(String name, List buckets, + SetBackedScalingCuckooFilter filterFilter) { + return new StringRareTerms(name, order, pipelineAggregators(), metaData, format, + buckets, maxDocCount, filterFilter); + } + + @Override + protected StringRareTerms.Bucket[] createBucketsArray(int size) { + return new StringRareTerms.Bucket[size]; + } + + @Override + public boolean containsTerm(SetBackedScalingCuckooFilter filter, StringRareTerms.Bucket bucket) { + return filter.mightContain(bucket.termBytes); + } + + @Override + public void addToFilter(SetBackedScalingCuckooFilter filter, StringRareTerms.Bucket bucket) { + filter.add(bucket.termBytes); + } +} diff --git a/server/src/main/java/org/elasticsearch/search/aggregations/bucket/terms/StringRareTermsAggregator.java b/server/src/main/java/org/elasticsearch/search/aggregations/bucket/terms/StringRareTermsAggregator.java new file mode 100644 index 0000000000000..0c200e96b242c --- /dev/null +++ b/server/src/main/java/org/elasticsearch/search/aggregations/bucket/terms/StringRareTermsAggregator.java @@ -0,0 +1,175 @@ +/* + * 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.search.aggregations.bucket.terms; + +import org.apache.lucene.index.LeafReaderContext; +import org.apache.lucene.util.BytesRef; +import org.apache.lucene.util.BytesRefBuilder; +import org.apache.lucene.util.CollectionUtil; +import org.elasticsearch.common.lease.Releasables; +import org.elasticsearch.common.util.BytesRefHash; +import org.elasticsearch.index.fielddata.SortedBinaryDocValues; +import org.elasticsearch.search.DocValueFormat; +import org.elasticsearch.search.aggregations.Aggregator; +import org.elasticsearch.search.aggregations.AggregatorFactories; +import org.elasticsearch.search.aggregations.InternalAggregation; +import org.elasticsearch.search.aggregations.LeafBucketCollector; +import org.elasticsearch.search.aggregations.LeafBucketCollectorBase; +import org.elasticsearch.search.aggregations.pipeline.PipelineAggregator; +import org.elasticsearch.search.aggregations.support.ValuesSource; +import org.elasticsearch.search.internal.SearchContext; + +import java.io.IOException; +import java.util.ArrayList; +import java.util.List; +import java.util.Map; + +import static java.util.Collections.emptyList; + +/** + * An aggregator that finds "rare" string values (e.g. terms agg that orders ascending) + */ +public class StringRareTermsAggregator extends AbstractRareTermsAggregator { + protected BytesRefHash bucketOrds; + + StringRareTermsAggregator(String name, AggregatorFactories factories, ValuesSource.Bytes valuesSource, + DocValueFormat format, IncludeExclude.StringFilter stringFilter, + SearchContext context, Aggregator parent, List pipelineAggregators, + Map metaData, long maxDocCount, double precision) throws IOException { + super(name, factories, context, parent, pipelineAggregators, metaData, maxDocCount, precision, format, valuesSource, stringFilter); + this.bucketOrds = new BytesRefHash(1, context.bigArrays()); + } + + @Override + public LeafBucketCollector getLeafCollector(LeafReaderContext ctx, + final LeafBucketCollector sub) throws IOException { + final SortedBinaryDocValues values = valuesSource.bytesValues(ctx); + if (subCollectors == null) { + subCollectors = sub; + } + return new LeafBucketCollectorBase(sub, values) { + final BytesRefBuilder previous = new BytesRefBuilder(); + + @Override + public void collect(int docId, long bucket) throws IOException { + assert bucket == 0; + if (values.advanceExact(docId)) { + final int valuesCount = values.docValueCount(); + previous.clear(); + + // SortedBinaryDocValues don't guarantee uniqueness so we + // need to take care of dups + for (int i = 0; i < valuesCount; ++i) { + final BytesRef bytes = values.nextValue(); + if (includeExclude != null && !includeExclude.accept(bytes)) { + continue; + } + if (i > 0 && previous.get().equals(bytes)) { + continue; + } + + doCollect(bytes, docId); + previous.copyBytes(bytes); + } + } + } + }; + } + + @Override + long addValueToOrds(BytesRef value) { + return bucketOrds.add(value); + } + + /** + * Merges the ordinals to a minimal set, populates the CuckooFilter and + * generates a final set of buckets. + * + * If a term is below the maxDocCount, it is turned into a Bucket. Otherwise, + * the term is added to the filter, and pruned from the ordinal map. If + * necessary the ordinal map is merged down to a minimal set to remove deletions + */ + private List buildSketch() { + long deletionCount = 0; + BytesRefHash newBucketOrds = new BytesRefHash(1, context.bigArrays()); + List buckets = new ArrayList<>(); + try (BytesRefHash oldBucketOrds = bucketOrds) { + + long[] mergeMap = new long[(int) oldBucketOrds.size()]; + BytesRef scratch = new BytesRef(); + for (int i = 0; i < oldBucketOrds.size(); i++) { + BytesRef oldKey = oldBucketOrds.get(i, scratch); + long newBucketOrd = -1; + long docCount = bucketDocCount(i); + // if the key is below threshold, reinsert into the new ords + if (docCount <= maxDocCount) { + newBucketOrd = newBucketOrds.add(oldKey); + StringRareTerms.Bucket bucket = new StringRareTerms.Bucket(BytesRef.deepCopyOf(oldKey), docCount, null, format); + bucket.bucketOrd = newBucketOrd; + buckets.add(bucket); + + consumeBucketsAndMaybeBreak(1); + } else { + // Make a note when one of the ords has been deleted + deletionCount += 1; + filter.add(oldKey); + } + mergeMap[i] = newBucketOrd; + } + + // Only merge/delete the ordinals if we have actually deleted one, + // to save on some redundant work + if (deletionCount > 0) { + mergeBuckets(mergeMap, newBucketOrds.size()); + if (deferringCollector != null) { + deferringCollector.mergeBuckets(mergeMap); + } + } + } + bucketOrds = newBucketOrds; + return buckets; + } + + @Override + public InternalAggregation buildAggregation(long owningBucketOrdinal) throws IOException { + assert owningBucketOrdinal == 0; + + List buckets = buildSketch(); + runDeferredCollections(buckets.stream().mapToLong(b -> b.bucketOrd).toArray()); + + // Finalize the buckets + for (StringRareTerms.Bucket bucket : buckets) { + bucket.aggregations = bucketAggregations(bucket.bucketOrd); + } + + CollectionUtil.introSort(buckets, ORDER.comparator(this)); + return new StringRareTerms(name, ORDER, pipelineAggregators(), metaData(), format, buckets, maxDocCount, filter); + } + + @Override + public InternalAggregation buildEmptyAggregation() { + return new StringRareTerms(name, LongRareTermsAggregator.ORDER, pipelineAggregators(), metaData(), format, emptyList(), 0, filter); + } + + @Override + public void doClose() { + Releasables.close(bucketOrds); + } +} + diff --git a/server/src/main/java/org/elasticsearch/search/aggregations/bucket/terms/StringTermsAggregator.java b/server/src/main/java/org/elasticsearch/search/aggregations/bucket/terms/StringTermsAggregator.java index 20162fd1bc78a..446aafa22d36b 100644 --- a/server/src/main/java/org/elasticsearch/search/aggregations/bucket/terms/StringTermsAggregator.java +++ b/server/src/main/java/org/elasticsearch/search/aggregations/bucket/terms/StringTermsAggregator.java @@ -114,8 +114,9 @@ public void collect(int doc, long bucket) throws IOException { public InternalAggregation buildAggregation(long owningBucketOrdinal) throws IOException { assert owningBucketOrdinal == 0; - if (bucketCountThresholds.getMinDocCount() == 0 && (InternalOrder.isCountDesc(order) == false || - bucketOrds.size() < bucketCountThresholds.getRequiredSize())) { + if (bucketCountThresholds.getMinDocCount() == 0 + && (InternalOrder.isCountDesc(order) == false + || bucketOrds.size() < bucketCountThresholds.getRequiredSize())) { // we need to fill-in the blanks for (LeafReaderContext ctx : context.searcher().getTopReaderContext().leaves()) { final SortedBinaryDocValues values = valuesSource.bytesValues(ctx); @@ -168,11 +169,10 @@ public InternalAggregation buildAggregation(long owningBucketOrdinal) throws IOE runDeferredCollections(survivingBucketOrds); // Now build the aggs - for (int i = 0; i < list.length; i++) { - final StringTerms.Bucket bucket = list[i]; - bucket.termBytes = BytesRef.deepCopyOf(bucket.termBytes); - bucket.aggregations = bucketAggregations(bucket.bucketOrd); - bucket.docCountError = 0; + for (final StringTerms.Bucket bucket : list) { + bucket.termBytes = BytesRef.deepCopyOf(bucket.termBytes); + bucket.aggregations = bucketAggregations(bucket.bucketOrd); + bucket.docCountError = 0; } return new StringTerms(name, order, bucketCountThresholds.getRequiredSize(), bucketCountThresholds.getMinDocCount(), diff --git a/server/src/main/java/org/elasticsearch/search/aggregations/bucket/terms/TermsAggregationBuilder.java b/server/src/main/java/org/elasticsearch/search/aggregations/bucket/terms/TermsAggregationBuilder.java index 5887f6b525891..a124feb115b19 100644 --- a/server/src/main/java/org/elasticsearch/search/aggregations/bucket/terms/TermsAggregationBuilder.java +++ b/server/src/main/java/org/elasticsearch/search/aggregations/bucket/terms/TermsAggregationBuilder.java @@ -333,8 +333,8 @@ public TermsAggregationBuilder showTermDocCountError(boolean showTermDocCountErr } @Override - protected ValuesSourceAggregatorFactory innerBuild(SearchContext context, ValuesSourceConfig config, - AggregatorFactory parent, Builder subFactoriesBuilder) throws IOException { + protected ValuesSourceAggregatorFactory innerBuild(SearchContext context, ValuesSourceConfig config, + AggregatorFactory parent, Builder subFactoriesBuilder) throws IOException { return new TermsAggregatorFactory(name, config, order, includeExclude, executionHint, collectMode, bucketCountThresholds, showTermDocCountError, context, parent, subFactoriesBuilder, metaData); } @@ -358,19 +358,23 @@ protected XContentBuilder doXContentBody(XContentBuilder builder, Params params) } @Override - protected int innerHashCode() { - return Objects.hash(bucketCountThresholds, collectMode, executionHint, includeExclude, order, showTermDocCountError); + public int hashCode() { + return Objects.hash(super.hashCode(), bucketCountThresholds, collectMode, + executionHint, includeExclude, order, showTermDocCountError); } @Override - protected boolean innerEquals(Object obj) { + public boolean equals(Object obj) { + if (this == obj) return true; + if (obj == null || getClass() != obj.getClass()) return false; + if (super.equals(obj) == false) return false; TermsAggregationBuilder other = (TermsAggregationBuilder) obj; return Objects.equals(bucketCountThresholds, other.bucketCountThresholds) - && Objects.equals(collectMode, other.collectMode) - && Objects.equals(executionHint, other.executionHint) - && Objects.equals(includeExclude, other.includeExclude) - && Objects.equals(order, other.order) - && Objects.equals(showTermDocCountError, other.showTermDocCountError); + && Objects.equals(collectMode, other.collectMode) + && Objects.equals(executionHint, other.executionHint) + && Objects.equals(includeExclude, other.includeExclude) + && Objects.equals(order, other.order) + && Objects.equals(showTermDocCountError, other.showTermDocCountError); } @Override diff --git a/server/src/main/java/org/elasticsearch/search/aggregations/bucket/terms/TermsAggregatorFactory.java b/server/src/main/java/org/elasticsearch/search/aggregations/bucket/terms/TermsAggregatorFactory.java index 877a8e59bc2d3..96daeeb476a28 100644 --- a/server/src/main/java/org/elasticsearch/search/aggregations/bucket/terms/TermsAggregatorFactory.java +++ b/server/src/main/java/org/elasticsearch/search/aggregations/bucket/terms/TermsAggregatorFactory.java @@ -46,7 +46,7 @@ import java.util.List; import java.util.Map; -public class TermsAggregatorFactory extends ValuesSourceAggregatorFactory { +public class TermsAggregatorFactory extends ValuesSourceAggregatorFactory { private static final DeprecationLogger deprecationLogger = new DeprecationLogger(LogManager.getLogger(TermsAggregatorFactory.class)); static Boolean REMAP_GLOBAL_ORDS, COLLECT_SEGMENT_ORDS; @@ -67,7 +67,7 @@ public class TermsAggregatorFactory extends ValuesSourceAggregatorFactory parent, + AggregatorFactory parent, AggregatorFactories.Builder subFactoriesBuilder, Map metaData) throws IOException { super(name, config, context, parent, subFactoriesBuilder, metaData); @@ -189,7 +189,7 @@ protected Aggregator doCreateInternal(ValuesSource valuesSource, Aggregator pare } throw new AggregationExecutionException("terms aggregation cannot be applied to field [" + config.fieldContext().field() - + "]. It can only be applied to numeric or string fields."); + + "]. It can only be applied to numeric or string fields."); } // return the SubAggCollectionMode that this aggregation should use based on the expected size @@ -262,21 +262,26 @@ Aggregator create(String name, assert maxOrd != -1; final double ratio = maxOrd / ((double) context.searcher().getIndexReader().numDocs()); + assert valuesSource instanceof ValuesSource.Bytes.WithOrdinals; + ValuesSource.Bytes.WithOrdinals ordinalsValuesSource = (ValuesSource.Bytes.WithOrdinals) valuesSource; + if (factories == AggregatorFactories.EMPTY && includeExclude == null && Aggregator.descendsFromBucketAggregator(parent) == false && + ordinalsValuesSource.supportsGlobalOrdinalsMapping() && // we use the static COLLECT_SEGMENT_ORDS to allow tests to force specific optimizations (COLLECT_SEGMENT_ORDS!= null ? COLLECT_SEGMENT_ORDS.booleanValue() : ratio <= 0.5 && maxOrd <= 2048)) { /** * We can use the low cardinality execution mode iff this aggregator: * - has no sub-aggregator AND * - is not a child of a bucket aggregator AND + * - has a values source that can map from segment to global ordinals * - At least we reduce the number of global ordinals look-ups by half (ration <= 0.5) AND * - the maximum global ordinal is less than 2048 (LOW_CARDINALITY has additional memory usage, * which directly linked to maxOrd, so we need to limit). */ return new GlobalOrdinalsStringTermsAggregator.LowCardinality(name, factories, - (ValuesSource.Bytes.WithOrdinals) valuesSource, order, format, bucketCountThresholds, context, parent, false, + ordinalsValuesSource, order, format, bucketCountThresholds, context, parent, false, subAggCollectMode, showTermDocCountError, pipelineAggregators, metaData); } @@ -301,7 +306,7 @@ Aggregator create(String name, remapGlobalOrds = false; } } - return new GlobalOrdinalsStringTermsAggregator(name, factories, (ValuesSource.Bytes.WithOrdinals) valuesSource, order, + return new GlobalOrdinalsStringTermsAggregator(name, factories, ordinalsValuesSource, order, format, bucketCountThresholds, filter, context, parent, remapGlobalOrds, subAggCollectMode, showTermDocCountError, pipelineAggregators, metaData); } diff --git a/server/src/main/java/org/elasticsearch/search/aggregations/bucket/terms/UnmappedRareTerms.java b/server/src/main/java/org/elasticsearch/search/aggregations/bucket/terms/UnmappedRareTerms.java new file mode 100644 index 0000000000000..eff5441a1d7e7 --- /dev/null +++ b/server/src/main/java/org/elasticsearch/search/aggregations/bucket/terms/UnmappedRareTerms.java @@ -0,0 +1,119 @@ +/* + * 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.search.aggregations.bucket.terms; + +import org.elasticsearch.common.io.stream.StreamInput; +import org.elasticsearch.common.io.stream.StreamOutput; +import org.elasticsearch.common.util.SetBackedScalingCuckooFilter; +import org.elasticsearch.common.xcontent.XContentBuilder; +import org.elasticsearch.search.DocValueFormat; +import org.elasticsearch.search.aggregations.InternalAggregation; +import org.elasticsearch.search.aggregations.InternalAggregations; +import org.elasticsearch.search.aggregations.pipeline.PipelineAggregator; + +import java.io.IOException; +import java.util.Collections; +import java.util.List; +import java.util.Map; + +import static java.util.Collections.emptyList; + +/** + * Result of the RareTerms aggregation when the field is unmapped. + */ +public class UnmappedRareTerms extends InternalRareTerms { + public static final String NAME = "umrareterms"; + + protected abstract static class Bucket extends InternalRareTerms.Bucket { + private Bucket(long docCount, InternalAggregations aggregations, DocValueFormat formatter) { + super(docCount, aggregations, formatter); + } + } + + UnmappedRareTerms(String name, List pipelineAggregators, Map metaData) { + super(name, LongRareTermsAggregator.ORDER, 0, pipelineAggregators, metaData); + } + + /** + * Read from a stream. + */ + public UnmappedRareTerms(StreamInput in) throws IOException { + super(in); + } + + @Override + protected void writeTermTypeInfoTo(StreamOutput out) throws IOException { + // Nothing to write + } + + @Override + public String getWriteableName() { + return NAME; + } + + @Override + public String getType() { + return StringTerms.NAME; + } + + @Override + public UnmappedRareTerms create(List buckets) { + return new UnmappedRareTerms(name, pipelineAggregators(), metaData); + } + + @Override + public UnmappedRareTerms.Bucket createBucket(InternalAggregations aggregations, UnmappedRareTerms.Bucket prototype) { + throw new UnsupportedOperationException("not supported for UnmappedRareTerms"); + } + + @Override + protected UnmappedRareTerms createWithFilter(String name, List buckets, SetBackedScalingCuckooFilter filter) { + throw new UnsupportedOperationException("not supported for UnmappedRareTerms"); + } + + @Override + public InternalAggregation doReduce(List aggregations, ReduceContext reduceContext) { + return new UnmappedRareTerms(name, pipelineAggregators(), metaData); + } + + @Override + public boolean isMapped() { + return false; + } + + @Override + public final XContentBuilder doXContentBody(XContentBuilder builder, Params params) throws IOException { + return doXContentCommon(builder, params, Collections.emptyList()); + } + + @Override + public List getBuckets() { + return emptyList(); + } + + @Override + public UnmappedRareTerms.Bucket getBucketByKey(String term) { + return null; + } + + @Override + protected UnmappedRareTerms.Bucket[] createBucketsArray(int size) { + return new UnmappedRareTerms.Bucket[size]; + } +} diff --git a/server/src/main/java/org/elasticsearch/search/aggregations/metrics/AbstractInternalHDRPercentiles.java b/server/src/main/java/org/elasticsearch/search/aggregations/metrics/AbstractInternalHDRPercentiles.java index b3bbe6b911c48..d8e043ee9b5e9 100644 --- a/server/src/main/java/org/elasticsearch/search/aggregations/metrics/AbstractInternalHDRPercentiles.java +++ b/server/src/main/java/org/elasticsearch/search/aggregations/metrics/AbstractInternalHDRPercentiles.java @@ -150,7 +150,11 @@ public XContentBuilder doXContentBody(XContentBuilder builder, Params params) th } @Override - protected boolean doEquals(Object obj) { + public boolean equals(Object obj) { + if (this == obj) return true; + if (obj == null || getClass() != obj.getClass()) return false; + if (super.equals(obj) == false) return false; + AbstractInternalHDRPercentiles that = (AbstractInternalHDRPercentiles) obj; return keyed == that.keyed && Arrays.equals(keys, that.keys) @@ -158,10 +162,14 @@ protected boolean doEquals(Object obj) { } @Override - protected int doHashCode() { + public int hashCode() { // we cannot use state.hashCode at the moment because of: // https://github.com/HdrHistogram/HdrHistogram/issues/81 // TODO: upgrade the HDRHistogram library - return Objects.hash(keyed, Arrays.hashCode(keys), state.getIntegerToDoubleValueConversionRatio(), state.getTotalCount()); + return Objects.hash(super.hashCode(), + keyed, + Arrays.hashCode(keys), + state.getIntegerToDoubleValueConversionRatio(), + state.getTotalCount()); } } diff --git a/server/src/main/java/org/elasticsearch/search/aggregations/metrics/AbstractInternalTDigestPercentiles.java b/server/src/main/java/org/elasticsearch/search/aggregations/metrics/AbstractInternalTDigestPercentiles.java index cc63e5f7a4325..ca03e2aa2b1c9 100644 --- a/server/src/main/java/org/elasticsearch/search/aggregations/metrics/AbstractInternalTDigestPercentiles.java +++ b/server/src/main/java/org/elasticsearch/search/aggregations/metrics/AbstractInternalTDigestPercentiles.java @@ -133,7 +133,11 @@ public XContentBuilder doXContentBody(XContentBuilder builder, Params params) th } @Override - protected boolean doEquals(Object obj) { + public boolean equals(Object obj) { + if (this == obj) return true; + if (obj == null || getClass() != obj.getClass()) return false; + if (super.equals(obj) == false) return false; + AbstractInternalTDigestPercentiles that = (AbstractInternalTDigestPercentiles) obj; return keyed == that.keyed && Arrays.equals(keys, that.keys) @@ -141,7 +145,7 @@ protected boolean doEquals(Object obj) { } @Override - protected int doHashCode() { - return Objects.hash(keyed, Arrays.hashCode(keys), state); + public int hashCode() { + return Objects.hash(super.hashCode(), keyed, Arrays.hashCode(keys), state); } } diff --git a/server/src/main/java/org/elasticsearch/search/aggregations/metrics/AvgAggregationBuilder.java b/server/src/main/java/org/elasticsearch/search/aggregations/metrics/AvgAggregationBuilder.java index 1f57964f667fb..a6a5fb02f8363 100644 --- a/server/src/main/java/org/elasticsearch/search/aggregations/metrics/AvgAggregationBuilder.java +++ b/server/src/main/java/org/elasticsearch/search/aggregations/metrics/AvgAggregationBuilder.java @@ -79,7 +79,7 @@ protected void innerWriteTo(StreamOutput out) { @Override protected AvgAggregatorFactory innerBuild(SearchContext context, ValuesSourceConfig config, - AggregatorFactory parent, Builder subFactoriesBuilder) throws IOException { + AggregatorFactory parent, Builder subFactoriesBuilder) throws IOException { return new AvgAggregatorFactory(name, config, context, parent, subFactoriesBuilder, metaData); } @@ -88,16 +88,6 @@ public XContentBuilder doXContentBody(XContentBuilder builder, Params params) th return builder; } - @Override - protected int innerHashCode() { - return 0; - } - - @Override - protected boolean innerEquals(Object obj) { - return true; - } - @Override public String getType() { return NAME; diff --git a/server/src/main/java/org/elasticsearch/search/aggregations/metrics/AvgAggregatorFactory.java b/server/src/main/java/org/elasticsearch/search/aggregations/metrics/AvgAggregatorFactory.java index 817e40db26e18..2b1c53535c9ac 100644 --- a/server/src/main/java/org/elasticsearch/search/aggregations/metrics/AvgAggregatorFactory.java +++ b/server/src/main/java/org/elasticsearch/search/aggregations/metrics/AvgAggregatorFactory.java @@ -33,10 +33,10 @@ import java.util.List; import java.util.Map; -class AvgAggregatorFactory extends ValuesSourceAggregatorFactory { +class AvgAggregatorFactory extends ValuesSourceAggregatorFactory { AvgAggregatorFactory(String name, ValuesSourceConfig config, SearchContext context, - AggregatorFactory parent, AggregatorFactories.Builder subFactoriesBuilder, Map metaData) throws IOException { + AggregatorFactory parent, AggregatorFactories.Builder subFactoriesBuilder, Map metaData) throws IOException { super(name, config, context, parent, subFactoriesBuilder, metaData); } diff --git a/server/src/main/java/org/elasticsearch/search/aggregations/metrics/CardinalityAggregationBuilder.java b/server/src/main/java/org/elasticsearch/search/aggregations/metrics/CardinalityAggregationBuilder.java index fa8a1118c88ba..8d927e2fa59eb 100644 --- a/server/src/main/java/org/elasticsearch/search/aggregations/metrics/CardinalityAggregationBuilder.java +++ b/server/src/main/java/org/elasticsearch/search/aggregations/metrics/CardinalityAggregationBuilder.java @@ -124,7 +124,7 @@ public Long precisionThreshold() { @Override protected CardinalityAggregatorFactory innerBuild(SearchContext context, ValuesSourceConfig config, - AggregatorFactory parent, Builder subFactoriesBuilder) throws IOException { + AggregatorFactory parent, Builder subFactoriesBuilder) throws IOException { return new CardinalityAggregatorFactory(name, config, precisionThreshold, context, parent, subFactoriesBuilder, metaData); } @@ -137,12 +137,15 @@ public XContentBuilder doXContentBody(XContentBuilder builder, Params params) th } @Override - protected int innerHashCode() { - return Objects.hash(precisionThreshold); + public int hashCode() { + return Objects.hash(super.hashCode(), precisionThreshold); } @Override - protected boolean innerEquals(Object obj) { + public boolean equals(Object obj) { + if (this == obj) return true; + if (obj == null || getClass() != obj.getClass()) return false; + if (super.equals(obj) == false) return false; CardinalityAggregationBuilder other = (CardinalityAggregationBuilder) obj; return Objects.equals(precisionThreshold, other.precisionThreshold); } diff --git a/server/src/main/java/org/elasticsearch/search/aggregations/metrics/CardinalityAggregatorFactory.java b/server/src/main/java/org/elasticsearch/search/aggregations/metrics/CardinalityAggregatorFactory.java index 413c896fbcb3f..8927697b90f41 100644 --- a/server/src/main/java/org/elasticsearch/search/aggregations/metrics/CardinalityAggregatorFactory.java +++ b/server/src/main/java/org/elasticsearch/search/aggregations/metrics/CardinalityAggregatorFactory.java @@ -32,12 +32,12 @@ import java.util.List; import java.util.Map; -class CardinalityAggregatorFactory extends ValuesSourceAggregatorFactory { +class CardinalityAggregatorFactory extends ValuesSourceAggregatorFactory { private final Long precisionThreshold; CardinalityAggregatorFactory(String name, ValuesSourceConfig config, Long precisionThreshold, - SearchContext context, AggregatorFactory parent, AggregatorFactories.Builder subFactoriesBuilder, + SearchContext context, AggregatorFactory parent, AggregatorFactories.Builder subFactoriesBuilder, Map metaData) throws IOException { super(name, config, context, parent, subFactoriesBuilder, metaData); this.precisionThreshold = precisionThreshold; diff --git a/server/src/main/java/org/elasticsearch/search/aggregations/metrics/ExtendedStatsAggregationBuilder.java b/server/src/main/java/org/elasticsearch/search/aggregations/metrics/ExtendedStatsAggregationBuilder.java index 33caa5f840028..af9e8bbca8d56 100644 --- a/server/src/main/java/org/elasticsearch/search/aggregations/metrics/ExtendedStatsAggregationBuilder.java +++ b/server/src/main/java/org/elasticsearch/search/aggregations/metrics/ExtendedStatsAggregationBuilder.java @@ -99,7 +99,7 @@ public double sigma() { @Override protected ExtendedStatsAggregatorFactory innerBuild(SearchContext context, ValuesSourceConfig config, - AggregatorFactory parent, Builder subFactoriesBuilder) throws IOException { + AggregatorFactory parent, Builder subFactoriesBuilder) throws IOException { return new ExtendedStatsAggregatorFactory(name, config, sigma, context, parent, subFactoriesBuilder, metaData); } @@ -110,12 +110,15 @@ public XContentBuilder doXContentBody(XContentBuilder builder, Params params) th } @Override - protected int innerHashCode() { - return Objects.hash(sigma); + public int hashCode() { + return Objects.hash(super.hashCode(), sigma); } @Override - protected boolean innerEquals(Object obj) { + public boolean equals(Object obj) { + if (this == obj) return true; + if (obj == null || getClass() != obj.getClass()) return false; + if (super.equals(obj) == false) return false; ExtendedStatsAggregationBuilder other = (ExtendedStatsAggregationBuilder) obj; return Objects.equals(sigma, other.sigma); } diff --git a/server/src/main/java/org/elasticsearch/search/aggregations/metrics/ExtendedStatsAggregatorFactory.java b/server/src/main/java/org/elasticsearch/search/aggregations/metrics/ExtendedStatsAggregatorFactory.java index 890f3199498b4..31e38a3bfd197 100644 --- a/server/src/main/java/org/elasticsearch/search/aggregations/metrics/ExtendedStatsAggregatorFactory.java +++ b/server/src/main/java/org/elasticsearch/search/aggregations/metrics/ExtendedStatsAggregatorFactory.java @@ -33,12 +33,12 @@ import java.util.List; import java.util.Map; -class ExtendedStatsAggregatorFactory extends ValuesSourceAggregatorFactory { +class ExtendedStatsAggregatorFactory extends ValuesSourceAggregatorFactory { private final double sigma; ExtendedStatsAggregatorFactory(String name, ValuesSourceConfig config, double sigma, - SearchContext context, AggregatorFactory parent, AggregatorFactories.Builder subFactoriesBuilder, + SearchContext context, AggregatorFactory parent, AggregatorFactories.Builder subFactoriesBuilder, Map metaData) throws IOException { super(name, config, context, parent, subFactoriesBuilder, metaData); this.sigma = sigma; diff --git a/server/src/main/java/org/elasticsearch/search/aggregations/metrics/GeoBoundsAggregationBuilder.java b/server/src/main/java/org/elasticsearch/search/aggregations/metrics/GeoBoundsAggregationBuilder.java index 9955f62f80ab7..6f6101fc45ee4 100644 --- a/server/src/main/java/org/elasticsearch/search/aggregations/metrics/GeoBoundsAggregationBuilder.java +++ b/server/src/main/java/org/elasticsearch/search/aggregations/metrics/GeoBoundsAggregationBuilder.java @@ -99,7 +99,7 @@ public boolean wrapLongitude() { @Override protected GeoBoundsAggregatorFactory innerBuild(SearchContext context, ValuesSourceConfig config, - AggregatorFactory parent, Builder subFactoriesBuilder) throws IOException { + AggregatorFactory parent, Builder subFactoriesBuilder) throws IOException { return new GeoBoundsAggregatorFactory(name, config, wrapLongitude, context, parent, subFactoriesBuilder, metaData); } @@ -110,12 +110,15 @@ public XContentBuilder doXContentBody(XContentBuilder builder, Params params) th } @Override - protected int innerHashCode() { - return Objects.hash(wrapLongitude); + public int hashCode() { + return Objects.hash(super.hashCode(), wrapLongitude); } @Override - protected boolean innerEquals(Object obj) { + public boolean equals(Object obj) { + if (this == obj) return true; + if (obj == null || getClass() != obj.getClass()) return false; + if (super.equals(obj) == false) return false; GeoBoundsAggregationBuilder other = (GeoBoundsAggregationBuilder) obj; return Objects.equals(wrapLongitude, other.wrapLongitude); } diff --git a/server/src/main/java/org/elasticsearch/search/aggregations/metrics/GeoBoundsAggregatorFactory.java b/server/src/main/java/org/elasticsearch/search/aggregations/metrics/GeoBoundsAggregatorFactory.java index e6080d16cbfd4..de8936079c236 100644 --- a/server/src/main/java/org/elasticsearch/search/aggregations/metrics/GeoBoundsAggregatorFactory.java +++ b/server/src/main/java/org/elasticsearch/search/aggregations/metrics/GeoBoundsAggregatorFactory.java @@ -32,12 +32,12 @@ import java.util.List; import java.util.Map; -class GeoBoundsAggregatorFactory extends ValuesSourceAggregatorFactory { +class GeoBoundsAggregatorFactory extends ValuesSourceAggregatorFactory { private final boolean wrapLongitude; GeoBoundsAggregatorFactory(String name, ValuesSourceConfig config, boolean wrapLongitude, - SearchContext context, AggregatorFactory parent, AggregatorFactories.Builder subFactoriesBuilder, + SearchContext context, AggregatorFactory parent, AggregatorFactories.Builder subFactoriesBuilder, Map metaData) throws IOException { super(name, config, context, parent, subFactoriesBuilder, metaData); this.wrapLongitude = wrapLongitude; diff --git a/server/src/main/java/org/elasticsearch/search/aggregations/metrics/GeoCentroidAggregationBuilder.java b/server/src/main/java/org/elasticsearch/search/aggregations/metrics/GeoCentroidAggregationBuilder.java index 088483656f8ff..98e8f2e9dbfbe 100644 --- a/server/src/main/java/org/elasticsearch/search/aggregations/metrics/GeoCentroidAggregationBuilder.java +++ b/server/src/main/java/org/elasticsearch/search/aggregations/metrics/GeoCentroidAggregationBuilder.java @@ -79,7 +79,7 @@ protected void innerWriteTo(StreamOutput out) { @Override protected GeoCentroidAggregatorFactory innerBuild(SearchContext context, ValuesSourceConfig config, - AggregatorFactory parent, Builder subFactoriesBuilder) throws IOException { + AggregatorFactory parent, Builder subFactoriesBuilder) throws IOException { return new GeoCentroidAggregatorFactory(name, config, context, parent, subFactoriesBuilder, metaData); } @@ -88,16 +88,6 @@ public XContentBuilder doXContentBody(XContentBuilder builder, Params params) th return builder; } - @Override - protected int innerHashCode() { - return 0; - } - - @Override - protected boolean innerEquals(Object obj) { - return true; - } - @Override public String getType() { return NAME; diff --git a/server/src/main/java/org/elasticsearch/search/aggregations/metrics/GeoCentroidAggregatorFactory.java b/server/src/main/java/org/elasticsearch/search/aggregations/metrics/GeoCentroidAggregatorFactory.java index 2bfb31c49930c..b12ce921b7d52 100644 --- a/server/src/main/java/org/elasticsearch/search/aggregations/metrics/GeoCentroidAggregatorFactory.java +++ b/server/src/main/java/org/elasticsearch/search/aggregations/metrics/GeoCentroidAggregatorFactory.java @@ -32,10 +32,10 @@ import java.util.List; import java.util.Map; -class GeoCentroidAggregatorFactory extends ValuesSourceAggregatorFactory { +class GeoCentroidAggregatorFactory extends ValuesSourceAggregatorFactory { GeoCentroidAggregatorFactory(String name, ValuesSourceConfig config, - SearchContext context, AggregatorFactory parent, AggregatorFactories.Builder subFactoriesBuilder, + SearchContext context, AggregatorFactory parent, AggregatorFactories.Builder subFactoriesBuilder, Map metaData) throws IOException { super(name, config, context, parent, subFactoriesBuilder, metaData); } diff --git a/server/src/main/java/org/elasticsearch/search/aggregations/metrics/HDRPercentileRanksAggregatorFactory.java b/server/src/main/java/org/elasticsearch/search/aggregations/metrics/HDRPercentileRanksAggregatorFactory.java index 1bb96e17da7a5..cfea50bc31224 100644 --- a/server/src/main/java/org/elasticsearch/search/aggregations/metrics/HDRPercentileRanksAggregatorFactory.java +++ b/server/src/main/java/org/elasticsearch/search/aggregations/metrics/HDRPercentileRanksAggregatorFactory.java @@ -34,14 +34,14 @@ import java.util.Map; class HDRPercentileRanksAggregatorFactory - extends ValuesSourceAggregatorFactory { + extends ValuesSourceAggregatorFactory { private final double[] values; private final int numberOfSignificantValueDigits; private final boolean keyed; HDRPercentileRanksAggregatorFactory(String name, ValuesSourceConfig config, double[] values, - int numberOfSignificantValueDigits, boolean keyed, SearchContext context, AggregatorFactory parent, + int numberOfSignificantValueDigits, boolean keyed, SearchContext context, AggregatorFactory parent, AggregatorFactories.Builder subFactoriesBuilder, Map metaData) throws IOException { super(name, config, context, parent, subFactoriesBuilder, metaData); this.values = values; diff --git a/server/src/main/java/org/elasticsearch/search/aggregations/metrics/HDRPercentilesAggregatorFactory.java b/server/src/main/java/org/elasticsearch/search/aggregations/metrics/HDRPercentilesAggregatorFactory.java index fe53f32889a7e..584ff30f53d9d 100644 --- a/server/src/main/java/org/elasticsearch/search/aggregations/metrics/HDRPercentilesAggregatorFactory.java +++ b/server/src/main/java/org/elasticsearch/search/aggregations/metrics/HDRPercentilesAggregatorFactory.java @@ -33,14 +33,14 @@ import java.util.List; import java.util.Map; -class HDRPercentilesAggregatorFactory extends ValuesSourceAggregatorFactory { +class HDRPercentilesAggregatorFactory extends ValuesSourceAggregatorFactory { private final double[] percents; private final int numberOfSignificantValueDigits; private final boolean keyed; HDRPercentilesAggregatorFactory(String name, ValuesSourceConfig config, double[] percents, - int numberOfSignificantValueDigits, boolean keyed, SearchContext context, AggregatorFactory parent, + int numberOfSignificantValueDigits, boolean keyed, SearchContext context, AggregatorFactory parent, AggregatorFactories.Builder subFactoriesBuilder, Map metaData) throws IOException { super(name, config, context, parent, subFactoriesBuilder, metaData); this.percents = percents; diff --git a/server/src/main/java/org/elasticsearch/search/aggregations/metrics/InternalAvg.java b/server/src/main/java/org/elasticsearch/search/aggregations/metrics/InternalAvg.java index 1b30afc087459..089407314de99 100644 --- a/server/src/main/java/org/elasticsearch/search/aggregations/metrics/InternalAvg.java +++ b/server/src/main/java/org/elasticsearch/search/aggregations/metrics/InternalAvg.java @@ -118,12 +118,15 @@ public XContentBuilder doXContentBody(XContentBuilder builder, Params params) th } @Override - protected int doHashCode() { - return Objects.hash(sum, count, format.getWriteableName()); + public int hashCode() { + return Objects.hash(super.hashCode(), sum, count, format.getWriteableName()); } @Override - protected boolean doEquals(Object obj) { + public boolean equals(Object obj) { + if (this == obj) return true; + if (obj == null || getClass() != obj.getClass()) return false; + if (super.equals(obj) == false) return false; InternalAvg other = (InternalAvg) obj; return Objects.equals(sum, other.sum) && Objects.equals(count, other.count) && diff --git a/server/src/main/java/org/elasticsearch/search/aggregations/metrics/InternalCardinality.java b/server/src/main/java/org/elasticsearch/search/aggregations/metrics/InternalCardinality.java index 144777379b5ad..bfe82c6bef659 100644 --- a/server/src/main/java/org/elasticsearch/search/aggregations/metrics/InternalCardinality.java +++ b/server/src/main/java/org/elasticsearch/search/aggregations/metrics/InternalCardinality.java @@ -30,6 +30,7 @@ import java.io.IOException; import java.util.List; import java.util.Map; +import java.util.Objects; public final class InternalCardinality extends InternalNumericMetricsAggregation.SingleValue implements Cardinality { private final HyperLogLogPlusPlus counts; @@ -117,12 +118,16 @@ public XContentBuilder doXContentBody(XContentBuilder builder, Params params) th } @Override - protected int doHashCode() { - return counts.hashCode(0); + public int hashCode() { + return Objects.hash(super.hashCode(), counts.hashCode(0)); } @Override - protected boolean doEquals(Object obj) { + public boolean equals(Object obj) { + if (this == obj) return true; + if (obj == null || getClass() != obj.getClass()) return false; + if (super.equals(obj) == false) return false; + InternalCardinality other = (InternalCardinality) obj; return counts.equals(0, other.counts); } diff --git a/server/src/main/java/org/elasticsearch/search/aggregations/metrics/InternalExtendedStats.java b/server/src/main/java/org/elasticsearch/search/aggregations/metrics/InternalExtendedStats.java index 26a244c8ddfb8..3fe2e75576aa4 100644 --- a/server/src/main/java/org/elasticsearch/search/aggregations/metrics/InternalExtendedStats.java +++ b/server/src/main/java/org/elasticsearch/search/aggregations/metrics/InternalExtendedStats.java @@ -215,15 +215,18 @@ protected XContentBuilder otherStatsToXContent(XContentBuilder builder, Params p } @Override - protected int doHashCode() { - return Objects.hash(super.doHashCode(), sumOfSqrs, sigma); + public int hashCode() { + return Objects.hash(super.hashCode(), sumOfSqrs, sigma); } @Override - protected boolean doEquals(Object obj) { + public boolean equals(Object obj) { + if (this == obj) return true; + if (obj == null || getClass() != obj.getClass()) return false; + if (super.equals(obj) == false) return false; + InternalExtendedStats other = (InternalExtendedStats) obj; - return super.doEquals(obj) && - Double.compare(sumOfSqrs, other.sumOfSqrs) == 0 && + return Double.compare(sumOfSqrs, other.sumOfSqrs) == 0 && Double.compare(sigma, other.sigma) == 0; } } diff --git a/server/src/main/java/org/elasticsearch/search/aggregations/metrics/InternalGeoBounds.java b/server/src/main/java/org/elasticsearch/search/aggregations/metrics/InternalGeoBounds.java index 7f259baca693f..4d48e4ab8966b 100644 --- a/server/src/main/java/org/elasticsearch/search/aggregations/metrics/InternalGeoBounds.java +++ b/server/src/main/java/org/elasticsearch/search/aggregations/metrics/InternalGeoBounds.java @@ -252,7 +252,11 @@ public GeoPoint bottomRight() { } @Override - protected boolean doEquals(Object obj) { + public boolean equals(Object obj) { + if (this == obj) return true; + if (obj == null || getClass() != obj.getClass()) return false; + if (super.equals(obj) == false) return false; + InternalGeoBounds other = (InternalGeoBounds) obj; return top == other.top && bottom == other.bottom && @@ -264,7 +268,7 @@ protected boolean doEquals(Object obj) { } @Override - protected int doHashCode() { - return Objects.hash(bottom, posLeft, posRight, negLeft, negRight, wrapLongitude); + public int hashCode() { + return Objects.hash(super.hashCode(), bottom, posLeft, posRight, negLeft, negRight, wrapLongitude); } } diff --git a/server/src/main/java/org/elasticsearch/search/aggregations/metrics/InternalGeoCentroid.java b/server/src/main/java/org/elasticsearch/search/aggregations/metrics/InternalGeoCentroid.java index 63dd8b50f71ef..2172d15259b85 100644 --- a/server/src/main/java/org/elasticsearch/search/aggregations/metrics/InternalGeoCentroid.java +++ b/server/src/main/java/org/elasticsearch/search/aggregations/metrics/InternalGeoCentroid.java @@ -180,15 +180,18 @@ public XContentBuilder doXContentBody(XContentBuilder builder, Params params) th } @Override - public boolean doEquals(Object o) { - InternalGeoCentroid that = (InternalGeoCentroid) o; + public boolean equals(Object obj) { + if (this == obj) return true; + if (obj == null || getClass() != obj.getClass()) return false; + if (super.equals(obj) == false) return false; + InternalGeoCentroid that = (InternalGeoCentroid) obj; return count == that.count && Objects.equals(centroid, that.centroid); } @Override - protected int doHashCode() { - return Objects.hash(centroid, count); + public int hashCode() { + return Objects.hash(super.hashCode(), centroid, count); } @Override diff --git a/server/src/main/java/org/elasticsearch/search/aggregations/metrics/InternalMax.java b/server/src/main/java/org/elasticsearch/search/aggregations/metrics/InternalMax.java index 300c82710f6d1..9a8458c85a690 100644 --- a/server/src/main/java/org/elasticsearch/search/aggregations/metrics/InternalMax.java +++ b/server/src/main/java/org/elasticsearch/search/aggregations/metrics/InternalMax.java @@ -90,12 +90,15 @@ public XContentBuilder doXContentBody(XContentBuilder builder, Params params) th } @Override - protected int doHashCode() { - return Objects.hash(max); + public int hashCode() { + return Objects.hash(super.hashCode(), max); } @Override - protected boolean doEquals(Object obj) { + public boolean equals(Object obj) { + if (this == obj) return true; + if (obj == null || getClass() != obj.getClass()) return false; + if (super.equals(obj) == false) return false; InternalMax other = (InternalMax) obj; return Objects.equals(max, other.max); } diff --git a/server/src/main/java/org/elasticsearch/search/aggregations/metrics/InternalMedianAbsoluteDeviation.java b/server/src/main/java/org/elasticsearch/search/aggregations/metrics/InternalMedianAbsoluteDeviation.java index ace9edb13f515..871f387638dc3 100644 --- a/server/src/main/java/org/elasticsearch/search/aggregations/metrics/InternalMedianAbsoluteDeviation.java +++ b/server/src/main/java/org/elasticsearch/search/aggregations/metrics/InternalMedianAbsoluteDeviation.java @@ -106,12 +106,15 @@ public XContentBuilder doXContentBody(XContentBuilder builder, Params params) th } @Override - protected int doHashCode() { - return Objects.hash(valuesSketch); + public int hashCode() { + return Objects.hash(super.hashCode(), valuesSketch); } @Override - protected boolean doEquals(Object obj) { + public boolean equals(Object obj) { + if (this == obj) return true; + if (obj == null || getClass() != obj.getClass()) return false; + if (super.equals(obj) == false) return false; InternalMedianAbsoluteDeviation other = (InternalMedianAbsoluteDeviation) obj; return Objects.equals(valuesSketch, other.valuesSketch); } diff --git a/server/src/main/java/org/elasticsearch/search/aggregations/metrics/InternalMin.java b/server/src/main/java/org/elasticsearch/search/aggregations/metrics/InternalMin.java index 60ed785edfe91..f68d5a46860bc 100644 --- a/server/src/main/java/org/elasticsearch/search/aggregations/metrics/InternalMin.java +++ b/server/src/main/java/org/elasticsearch/search/aggregations/metrics/InternalMin.java @@ -90,12 +90,15 @@ public XContentBuilder doXContentBody(XContentBuilder builder, Params params) th } @Override - protected int doHashCode() { - return Objects.hash(min); + public int hashCode() { + return Objects.hash(super.hashCode(), min); } @Override - protected boolean doEquals(Object obj) { + public boolean equals(Object obj) { + if (this == obj) return true; + if (obj == null || getClass() != obj.getClass()) return false; + if (super.equals(obj) == false) return false; InternalMin other = (InternalMin) obj; return Objects.equals(min, other.min); } diff --git a/server/src/main/java/org/elasticsearch/search/aggregations/metrics/InternalNumericMetricsAggregation.java b/server/src/main/java/org/elasticsearch/search/aggregations/metrics/InternalNumericMetricsAggregation.java index b3439671580e7..84f405c78cb11 100644 --- a/server/src/main/java/org/elasticsearch/search/aggregations/metrics/InternalNumericMetricsAggregation.java +++ b/server/src/main/java/org/elasticsearch/search/aggregations/metrics/InternalNumericMetricsAggregation.java @@ -107,19 +107,16 @@ protected InternalNumericMetricsAggregation(StreamInput in) throws IOException { @Override public int hashCode() { - return Objects.hash(format, super.hashCode()); + return Objects.hash(super.hashCode(), format); } @Override public boolean equals(Object obj) { - if (obj == null) { - return false; - } - if (obj.getClass() != getClass()) { - return false; - } + if (this == obj) return true; + if (obj == null || getClass() != obj.getClass()) return false; + if (super.equals(obj) == false) return false; + InternalNumericMetricsAggregation other = (InternalNumericMetricsAggregation) obj; - return super.equals(obj) && - Objects.equals(format, other.format); + return Objects.equals(format, other.format); } } diff --git a/server/src/main/java/org/elasticsearch/search/aggregations/metrics/InternalScriptedMetric.java b/server/src/main/java/org/elasticsearch/search/aggregations/metrics/InternalScriptedMetric.java index a0bb5e74fd5cb..ce051a1691b55 100644 --- a/server/src/main/java/org/elasticsearch/search/aggregations/metrics/InternalScriptedMetric.java +++ b/server/src/main/java/org/elasticsearch/search/aggregations/metrics/InternalScriptedMetric.java @@ -135,15 +135,19 @@ public XContentBuilder doXContentBody(XContentBuilder builder, Params params) th } @Override - protected boolean doEquals(Object obj) { + public boolean equals(Object obj) { + if (this == obj) return true; + if (obj == null || getClass() != obj.getClass()) return false; + if (super.equals(obj) == false) return false; + InternalScriptedMetric other = (InternalScriptedMetric) obj; return Objects.equals(reduceScript, other.reduceScript) && Objects.equals(aggregation, other.aggregation); } @Override - protected int doHashCode() { - return Objects.hash(reduceScript, aggregation); + public int hashCode() { + return Objects.hash(super.hashCode(), reduceScript, aggregation); } } diff --git a/server/src/main/java/org/elasticsearch/search/aggregations/metrics/InternalStats.java b/server/src/main/java/org/elasticsearch/search/aggregations/metrics/InternalStats.java index a05d6db7024e6..3c0aa8c7313bc 100644 --- a/server/src/main/java/org/elasticsearch/search/aggregations/metrics/InternalStats.java +++ b/server/src/main/java/org/elasticsearch/search/aggregations/metrics/InternalStats.java @@ -212,12 +212,16 @@ protected XContentBuilder otherStatsToXContent(XContentBuilder builder, Params p } @Override - protected int doHashCode() { - return Objects.hash(count, min, max, sum); + public int hashCode() { + return Objects.hash(super.hashCode(), count, min, max, sum); } @Override - protected boolean doEquals(Object obj) { + public boolean equals(Object obj) { + if (this == obj) return true; + if (obj == null || getClass() != obj.getClass()) return false; + if (super.equals(obj) == false) return false; + InternalStats other = (InternalStats) obj; return count == other.count && Double.compare(min, other.min) == 0 && diff --git a/server/src/main/java/org/elasticsearch/search/aggregations/metrics/InternalSum.java b/server/src/main/java/org/elasticsearch/search/aggregations/metrics/InternalSum.java index c3bb7173b3f07..2e1c9635aa896 100644 --- a/server/src/main/java/org/elasticsearch/search/aggregations/metrics/InternalSum.java +++ b/server/src/main/java/org/elasticsearch/search/aggregations/metrics/InternalSum.java @@ -100,12 +100,16 @@ public XContentBuilder doXContentBody(XContentBuilder builder, Params params) th } @Override - protected int doHashCode() { - return Objects.hashCode(sum); + public int hashCode() { + return Objects.hash(super.hashCode(), sum); } @Override - protected boolean doEquals(Object obj) { + public boolean equals(Object obj) { + if (this == obj) return true; + if (obj == null || getClass() != obj.getClass()) return false; + if (super.equals(obj) == false) return false; + InternalSum that = (InternalSum) obj; return Objects.equals(sum, that.sum); } diff --git a/server/src/main/java/org/elasticsearch/search/aggregations/metrics/InternalTopHits.java b/server/src/main/java/org/elasticsearch/search/aggregations/metrics/InternalTopHits.java index 4a266ee0703bd..348e98302d2bc 100644 --- a/server/src/main/java/org/elasticsearch/search/aggregations/metrics/InternalTopHits.java +++ b/server/src/main/java/org/elasticsearch/search/aggregations/metrics/InternalTopHits.java @@ -180,7 +180,11 @@ public XContentBuilder doXContentBody(XContentBuilder builder, Params params) th // Equals and hashcode implemented for testing round trips @Override - protected boolean doEquals(Object obj) { + public boolean equals(Object obj) { + if (this == obj) return true; + if (obj == null || getClass() != obj.getClass()) return false; + if (super.equals(obj) == false) return false; + InternalTopHits other = (InternalTopHits) obj; if (from != other.from) return false; if (size != other.size) return false; @@ -207,9 +211,10 @@ protected boolean doEquals(Object obj) { } @Override - protected int doHashCode() { - int hashCode = from; - hashCode = 31 * hashCode + size; + public int hashCode() { + int hashCode = super.hashCode(); + hashCode = 31 * hashCode + Integer.hashCode(from); + hashCode = 31 * hashCode + Integer.hashCode(size); hashCode = 31 * hashCode + Long.hashCode(topDocs.topDocs.totalHits.value); hashCode = 31 * hashCode + topDocs.topDocs.totalHits.relation.hashCode(); for (int d = 0; d < topDocs.topDocs.scoreDocs.length; d++) { diff --git a/server/src/main/java/org/elasticsearch/search/aggregations/metrics/InternalValueCount.java b/server/src/main/java/org/elasticsearch/search/aggregations/metrics/InternalValueCount.java index 36f2749c791f4..32ee8bd36d120 100644 --- a/server/src/main/java/org/elasticsearch/search/aggregations/metrics/InternalValueCount.java +++ b/server/src/main/java/org/elasticsearch/search/aggregations/metrics/InternalValueCount.java @@ -90,12 +90,16 @@ public String toString() { } @Override - protected int doHashCode() { - return Objects.hash(value); + public int hashCode() { + return Objects.hash(super.hashCode(), value); } @Override - protected boolean doEquals(Object obj) { + public boolean equals(Object obj) { + if (this == obj) return true; + if (obj == null || getClass() != obj.getClass()) return false; + if (super.equals(obj) == false) return false; + InternalValueCount that = (InternalValueCount) obj; return Objects.equals(this.value, that.value); } diff --git a/server/src/main/java/org/elasticsearch/search/aggregations/metrics/InternalWeightedAvg.java b/server/src/main/java/org/elasticsearch/search/aggregations/metrics/InternalWeightedAvg.java index e06ffbc7b4a5a..6165803299296 100644 --- a/server/src/main/java/org/elasticsearch/search/aggregations/metrics/InternalWeightedAvg.java +++ b/server/src/main/java/org/elasticsearch/search/aggregations/metrics/InternalWeightedAvg.java @@ -129,12 +129,15 @@ public XContentBuilder doXContentBody(XContentBuilder builder, Params params) th } @Override - protected int doHashCode() { - return Objects.hash(sum, weight, format.getWriteableName()); + public int hashCode() { + return Objects.hash(super.hashCode(), sum, weight, format.getWriteableName()); } @Override - protected boolean doEquals(Object obj) { + public boolean equals(Object obj) { + if (this == obj) return true; + if (obj == null || getClass() != obj.getClass()) return false; + if (super.equals(obj) == false) return false; InternalWeightedAvg other = (InternalWeightedAvg) obj; return Objects.equals(sum, other.sum) && Objects.equals(weight, other.weight) && diff --git a/server/src/main/java/org/elasticsearch/search/aggregations/metrics/MaxAggregationBuilder.java b/server/src/main/java/org/elasticsearch/search/aggregations/metrics/MaxAggregationBuilder.java index 0c3229f08fd51..b5d788cfc853f 100644 --- a/server/src/main/java/org/elasticsearch/search/aggregations/metrics/MaxAggregationBuilder.java +++ b/server/src/main/java/org/elasticsearch/search/aggregations/metrics/MaxAggregationBuilder.java @@ -79,7 +79,7 @@ protected void innerWriteTo(StreamOutput out) { @Override protected MaxAggregatorFactory innerBuild(SearchContext context, ValuesSourceConfig config, - AggregatorFactory parent, Builder subFactoriesBuilder) throws IOException { + AggregatorFactory parent, Builder subFactoriesBuilder) throws IOException { return new MaxAggregatorFactory(name, config, context, parent, subFactoriesBuilder, metaData); } @@ -88,16 +88,6 @@ public XContentBuilder doXContentBody(XContentBuilder builder, Params params) th return builder; } - @Override - protected int innerHashCode() { - return 0; - } - - @Override - protected boolean innerEquals(Object obj) { - return true; - } - @Override public String getType() { return NAME; diff --git a/server/src/main/java/org/elasticsearch/search/aggregations/metrics/MaxAggregatorFactory.java b/server/src/main/java/org/elasticsearch/search/aggregations/metrics/MaxAggregatorFactory.java index d64987d9cde4e..2ae762396818c 100644 --- a/server/src/main/java/org/elasticsearch/search/aggregations/metrics/MaxAggregatorFactory.java +++ b/server/src/main/java/org/elasticsearch/search/aggregations/metrics/MaxAggregatorFactory.java @@ -33,10 +33,10 @@ import java.util.List; import java.util.Map; -class MaxAggregatorFactory extends ValuesSourceAggregatorFactory { +class MaxAggregatorFactory extends ValuesSourceAggregatorFactory { MaxAggregatorFactory(String name, ValuesSourceConfig config, SearchContext context, - AggregatorFactory parent, AggregatorFactories.Builder subFactoriesBuilder, Map metaData) throws IOException { + AggregatorFactory parent, AggregatorFactories.Builder subFactoriesBuilder, Map metaData) throws IOException { super(name, config, context, parent, subFactoriesBuilder, metaData); } diff --git a/server/src/main/java/org/elasticsearch/search/aggregations/metrics/MedianAbsoluteDeviationAggregationBuilder.java b/server/src/main/java/org/elasticsearch/search/aggregations/metrics/MedianAbsoluteDeviationAggregationBuilder.java index b5b119cc38f6a..6218fd3e52b0f 100644 --- a/server/src/main/java/org/elasticsearch/search/aggregations/metrics/MedianAbsoluteDeviationAggregationBuilder.java +++ b/server/src/main/java/org/elasticsearch/search/aggregations/metrics/MedianAbsoluteDeviationAggregationBuilder.java @@ -107,9 +107,9 @@ protected void innerWriteTo(StreamOutput out) throws IOException { } @Override - protected ValuesSourceAggregatorFactory innerBuild(SearchContext context, + protected ValuesSourceAggregatorFactory innerBuild(SearchContext context, ValuesSourceConfig config, - AggregatorFactory parent, + AggregatorFactory parent, AggregatorFactories.Builder subFactoriesBuilder) throws IOException { @@ -123,12 +123,15 @@ protected XContentBuilder doXContentBody(XContentBuilder builder, Params params) } @Override - protected int innerHashCode() { - return Objects.hash(compression); + public int hashCode() { + return Objects.hash(super.hashCode(), compression); } @Override - protected boolean innerEquals(Object obj) { + public boolean equals(Object obj) { + if (this == obj) return true; + if (obj == null || getClass() != obj.getClass()) return false; + if (super.equals(obj) == false) return false; MedianAbsoluteDeviationAggregationBuilder other = (MedianAbsoluteDeviationAggregationBuilder) obj; return Objects.equals(compression, other.compression); } diff --git a/server/src/main/java/org/elasticsearch/search/aggregations/metrics/MedianAbsoluteDeviationAggregatorFactory.java b/server/src/main/java/org/elasticsearch/search/aggregations/metrics/MedianAbsoluteDeviationAggregatorFactory.java index 3e621e396de06..037e743ad32db 100644 --- a/server/src/main/java/org/elasticsearch/search/aggregations/metrics/MedianAbsoluteDeviationAggregatorFactory.java +++ b/server/src/main/java/org/elasticsearch/search/aggregations/metrics/MedianAbsoluteDeviationAggregatorFactory.java @@ -32,15 +32,14 @@ import java.util.List; import java.util.Map; -public class MedianAbsoluteDeviationAggregatorFactory extends - ValuesSourceAggregatorFactory { +public class MedianAbsoluteDeviationAggregatorFactory extends ValuesSourceAggregatorFactory { private final double compression; MedianAbsoluteDeviationAggregatorFactory(String name, ValuesSourceConfig config, SearchContext context, - AggregatorFactory parent, + AggregatorFactory parent, AggregatorFactories.Builder subFactoriesBuilder, Map metaData, double compression) throws IOException { diff --git a/server/src/main/java/org/elasticsearch/search/aggregations/metrics/MinAggregationBuilder.java b/server/src/main/java/org/elasticsearch/search/aggregations/metrics/MinAggregationBuilder.java index 2d23539189dc8..f195e10c65eb8 100644 --- a/server/src/main/java/org/elasticsearch/search/aggregations/metrics/MinAggregationBuilder.java +++ b/server/src/main/java/org/elasticsearch/search/aggregations/metrics/MinAggregationBuilder.java @@ -79,7 +79,7 @@ protected void innerWriteTo(StreamOutput out) { @Override protected MinAggregatorFactory innerBuild(SearchContext context, ValuesSourceConfig config, - AggregatorFactory parent, Builder subFactoriesBuilder) throws IOException { + AggregatorFactory parent, Builder subFactoriesBuilder) throws IOException { return new MinAggregatorFactory(name, config, context, parent, subFactoriesBuilder, metaData); } @@ -88,16 +88,6 @@ public XContentBuilder doXContentBody(XContentBuilder builder, Params params) th return builder; } - @Override - protected int innerHashCode() { - return 0; - } - - @Override - protected boolean innerEquals(Object obj) { - return true; - } - @Override public String getType() { return NAME; diff --git a/server/src/main/java/org/elasticsearch/search/aggregations/metrics/MinAggregatorFactory.java b/server/src/main/java/org/elasticsearch/search/aggregations/metrics/MinAggregatorFactory.java index 240cf2ba715fa..23e55627babfd 100644 --- a/server/src/main/java/org/elasticsearch/search/aggregations/metrics/MinAggregatorFactory.java +++ b/server/src/main/java/org/elasticsearch/search/aggregations/metrics/MinAggregatorFactory.java @@ -33,10 +33,10 @@ import java.util.List; import java.util.Map; -class MinAggregatorFactory extends ValuesSourceAggregatorFactory { +class MinAggregatorFactory extends ValuesSourceAggregatorFactory { MinAggregatorFactory(String name, ValuesSourceConfig config, SearchContext context, - AggregatorFactory parent, AggregatorFactories.Builder subFactoriesBuilder, Map metaData) throws IOException { + AggregatorFactory parent, AggregatorFactories.Builder subFactoriesBuilder, Map metaData) throws IOException { super(name, config, context, parent, subFactoriesBuilder, metaData); } diff --git a/server/src/main/java/org/elasticsearch/search/aggregations/metrics/PercentileRanksAggregationBuilder.java b/server/src/main/java/org/elasticsearch/search/aggregations/metrics/PercentileRanksAggregationBuilder.java index 3bf70d20989d3..d4a600b11b5fc 100644 --- a/server/src/main/java/org/elasticsearch/search/aggregations/metrics/PercentileRanksAggregationBuilder.java +++ b/server/src/main/java/org/elasticsearch/search/aggregations/metrics/PercentileRanksAggregationBuilder.java @@ -240,8 +240,8 @@ public PercentilesMethod method() { } @Override - protected ValuesSourceAggregatorFactory innerBuild(SearchContext context, ValuesSourceConfig config, - AggregatorFactory parent, Builder subFactoriesBuilder) throws IOException { + protected ValuesSourceAggregatorFactory innerBuild(SearchContext context, ValuesSourceConfig config, + AggregatorFactory parent, Builder subFactoriesBuilder) throws IOException { switch (method) { case TDIGEST: return new TDigestPercentileRanksAggregatorFactory(name, config, values, compression, keyed, context, parent, @@ -269,37 +269,40 @@ protected XContentBuilder doXContentBody(XContentBuilder builder, Params params) } @Override - protected boolean innerEquals(Object obj) { + public boolean equals(Object obj) { + if (this == obj) return true; + if (obj == null || getClass() != obj.getClass()) return false; + if (super.equals(obj) == false) return false; PercentileRanksAggregationBuilder other = (PercentileRanksAggregationBuilder) obj; - if (!Objects.equals(method, other.method)) { + if (Objects.equals(method, other.method) == false) { return false; } boolean equalSettings = false; switch (method) { - case HDR: - equalSettings = Objects.equals(numberOfSignificantValueDigits, other.numberOfSignificantValueDigits); - break; - case TDIGEST: - equalSettings = Objects.equals(compression, other.compression); - break; - default: - throw new IllegalStateException("Illegal method [" + method + "]"); + case HDR: + equalSettings = Objects.equals(numberOfSignificantValueDigits, other.numberOfSignificantValueDigits); + break; + case TDIGEST: + equalSettings = Objects.equals(compression, other.compression); + break; + default: + throw new IllegalStateException("Illegal method [" + method + "]"); } return equalSettings - && Objects.deepEquals(values, other.values) - && Objects.equals(keyed, other.keyed) - && Objects.equals(method, other.method); + && Objects.deepEquals(values, other.values) + && Objects.equals(keyed, other.keyed) + && Objects.equals(method, other.method); } @Override - protected int innerHashCode() { + public int hashCode() { switch (method) { - case HDR: - return Objects.hash(Arrays.hashCode(values), keyed, numberOfSignificantValueDigits, method); - case TDIGEST: - return Objects.hash(Arrays.hashCode(values), keyed, compression, method); - default: - throw new IllegalStateException("Illegal method [" + method + "]"); + case HDR: + return Objects.hash(super.hashCode(), Arrays.hashCode(values), keyed, numberOfSignificantValueDigits, method); + case TDIGEST: + return Objects.hash(super.hashCode(), Arrays.hashCode(values), keyed, compression, method); + default: + throw new IllegalStateException("Illegal method [" + method + "]"); } } diff --git a/server/src/main/java/org/elasticsearch/search/aggregations/metrics/PercentilesAggregationBuilder.java b/server/src/main/java/org/elasticsearch/search/aggregations/metrics/PercentilesAggregationBuilder.java index 3a6f5f89622f0..302e05a600a67 100644 --- a/server/src/main/java/org/elasticsearch/search/aggregations/metrics/PercentilesAggregationBuilder.java +++ b/server/src/main/java/org/elasticsearch/search/aggregations/metrics/PercentilesAggregationBuilder.java @@ -263,8 +263,8 @@ public PercentilesMethod method() { } @Override - protected ValuesSourceAggregatorFactory innerBuild(SearchContext context, ValuesSourceConfig config, - AggregatorFactory parent, Builder subFactoriesBuilder) throws IOException { + protected ValuesSourceAggregatorFactory innerBuild(SearchContext context, ValuesSourceConfig config, + AggregatorFactory parent, Builder subFactoriesBuilder) throws IOException { switch (method) { case TDIGEST: return new TDigestPercentilesAggregatorFactory(name, config, percents, compression, keyed, context, parent, @@ -292,35 +292,39 @@ protected XContentBuilder doXContentBody(XContentBuilder builder, Params params) } @Override - protected boolean innerEquals(Object obj) { + public boolean equals(Object obj) { + if (this == obj) return true; + if (obj == null || getClass() != obj.getClass()) return false; + if (super.equals(obj) == false) return false; + PercentilesAggregationBuilder other = (PercentilesAggregationBuilder) obj; - if (!Objects.equals(method, other.method)) { + if (Objects.equals(method, other.method) == false) { return false; } boolean equalSettings = false; switch (method) { - case HDR: - equalSettings = Objects.equals(numberOfSignificantValueDigits, other.numberOfSignificantValueDigits); - break; - case TDIGEST: - equalSettings = Objects.equals(compression, other.compression); - break; - default: - throw new IllegalStateException("Illegal method [" + method.toString() + "]"); + case HDR: + equalSettings = Objects.equals(numberOfSignificantValueDigits, other.numberOfSignificantValueDigits); + break; + case TDIGEST: + equalSettings = Objects.equals(compression, other.compression); + break; + default: + throw new IllegalStateException("Illegal method [" + method.toString() + "]"); } return equalSettings - && Objects.deepEquals(percents, other.percents) - && Objects.equals(keyed, other.keyed) - && Objects.equals(method, other.method); + && Objects.deepEquals(percents, other.percents) + && Objects.equals(keyed, other.keyed) + && Objects.equals(method, other.method); } @Override - protected int innerHashCode() { + public int hashCode() { switch (method) { case HDR: - return Objects.hash(Arrays.hashCode(percents), keyed, numberOfSignificantValueDigits, method); + return Objects.hash(super.hashCode(), Arrays.hashCode(percents), keyed, numberOfSignificantValueDigits, method); case TDIGEST: - return Objects.hash(Arrays.hashCode(percents), keyed, compression, method); + return Objects.hash(super.hashCode(), Arrays.hashCode(percents), keyed, compression, method); default: throw new IllegalStateException("Illegal method [" + method.toString() + "]"); } diff --git a/server/src/main/java/org/elasticsearch/search/aggregations/metrics/ScriptedMetricAggregationBuilder.java b/server/src/main/java/org/elasticsearch/search/aggregations/metrics/ScriptedMetricAggregationBuilder.java index 25fcebc6aa52d..0944ade85f174 100644 --- a/server/src/main/java/org/elasticsearch/search/aggregations/metrics/ScriptedMetricAggregationBuilder.java +++ b/server/src/main/java/org/elasticsearch/search/aggregations/metrics/ScriptedMetricAggregationBuilder.java @@ -193,7 +193,7 @@ public Map params() { } @Override - protected ScriptedMetricAggregatorFactory doBuild(SearchContext context, AggregatorFactory parent, + protected ScriptedMetricAggregatorFactory doBuild(SearchContext context, AggregatorFactory parent, Builder subfactoriesBuilder) throws IOException { if (combineScript == null) { @@ -325,18 +325,21 @@ public String getType() { } @Override - protected int doHashCode() { - return Objects.hash(initScript, mapScript, combineScript, reduceScript, params); + public int hashCode() { + return Objects.hash(super.hashCode(), initScript, mapScript, combineScript, reduceScript, params); } @Override - protected boolean doEquals(Object obj) { + public boolean equals(Object obj) { + if (this == obj) return true; + if (obj == null || getClass() != obj.getClass()) return false; + if (super.equals(obj) == false) return false; ScriptedMetricAggregationBuilder other = (ScriptedMetricAggregationBuilder) obj; return Objects.equals(initScript, other.initScript) - && Objects.equals(mapScript, other.mapScript) - && Objects.equals(combineScript, other.combineScript) - && Objects.equals(reduceScript, other.reduceScript) - && Objects.equals(params, other.params); + && Objects.equals(mapScript, other.mapScript) + && Objects.equals(combineScript, other.combineScript) + && Objects.equals(reduceScript, other.reduceScript) + && Objects.equals(params, other.params); } } diff --git a/server/src/main/java/org/elasticsearch/search/aggregations/metrics/ScriptedMetricAggregatorFactory.java b/server/src/main/java/org/elasticsearch/search/aggregations/metrics/ScriptedMetricAggregatorFactory.java index 01084ee0b7f8b..5c0ceeb1043d7 100644 --- a/server/src/main/java/org/elasticsearch/search/aggregations/metrics/ScriptedMetricAggregatorFactory.java +++ b/server/src/main/java/org/elasticsearch/search/aggregations/metrics/ScriptedMetricAggregatorFactory.java @@ -36,7 +36,7 @@ import java.util.List; import java.util.Map; -class ScriptedMetricAggregatorFactory extends AggregatorFactory { +class ScriptedMetricAggregatorFactory extends AggregatorFactory { private final ScriptedMetricAggContexts.MapScript.Factory mapScript; private final Map mapScriptParams; @@ -53,7 +53,7 @@ class ScriptedMetricAggregatorFactory extends AggregatorFactory initScriptParams, ScriptedMetricAggContexts.CombineScript.Factory combineScript, Map combineScriptParams, Script reduceScript, Map aggParams, - SearchLookup lookup, SearchContext context, AggregatorFactory parent, + SearchLookup lookup, SearchContext context, AggregatorFactory parent, AggregatorFactories.Builder subFactories, Map metaData) throws IOException { super(name, context, parent, subFactories, metaData); this.mapScript = mapScript; diff --git a/server/src/main/java/org/elasticsearch/search/aggregations/metrics/StatsAggregationBuilder.java b/server/src/main/java/org/elasticsearch/search/aggregations/metrics/StatsAggregationBuilder.java index d96bbba447580..34e5ab2864106 100644 --- a/server/src/main/java/org/elasticsearch/search/aggregations/metrics/StatsAggregationBuilder.java +++ b/server/src/main/java/org/elasticsearch/search/aggregations/metrics/StatsAggregationBuilder.java @@ -80,7 +80,7 @@ protected void innerWriteTo(StreamOutput out) { @Override protected StatsAggregatorFactory innerBuild(SearchContext context, ValuesSourceConfig config, - AggregatorFactory parent, Builder subFactoriesBuilder) throws IOException { + AggregatorFactory parent, Builder subFactoriesBuilder) throws IOException { return new StatsAggregatorFactory(name, config, context, parent, subFactoriesBuilder, metaData); } @@ -89,16 +89,6 @@ public XContentBuilder doXContentBody(XContentBuilder builder, Params params) th return builder; } - @Override - protected int innerHashCode() { - return 0; - } - - @Override - protected boolean innerEquals(Object obj) { - return true; - } - @Override public String getType() { return NAME; diff --git a/server/src/main/java/org/elasticsearch/search/aggregations/metrics/StatsAggregatorFactory.java b/server/src/main/java/org/elasticsearch/search/aggregations/metrics/StatsAggregatorFactory.java index 82dce359037c2..bc0e8a44e77fb 100644 --- a/server/src/main/java/org/elasticsearch/search/aggregations/metrics/StatsAggregatorFactory.java +++ b/server/src/main/java/org/elasticsearch/search/aggregations/metrics/StatsAggregatorFactory.java @@ -33,10 +33,10 @@ import java.util.List; import java.util.Map; -class StatsAggregatorFactory extends ValuesSourceAggregatorFactory { +class StatsAggregatorFactory extends ValuesSourceAggregatorFactory { StatsAggregatorFactory(String name, ValuesSourceConfig config, SearchContext context, - AggregatorFactory parent, AggregatorFactories.Builder subFactoriesBuilder, Map metaData) throws IOException { + AggregatorFactory parent, AggregatorFactories.Builder subFactoriesBuilder, Map metaData) throws IOException { super(name, config, context, parent, subFactoriesBuilder, metaData); } diff --git a/server/src/main/java/org/elasticsearch/search/aggregations/metrics/SumAggregationBuilder.java b/server/src/main/java/org/elasticsearch/search/aggregations/metrics/SumAggregationBuilder.java index 8035a3ad671f7..9f320b098108e 100644 --- a/server/src/main/java/org/elasticsearch/search/aggregations/metrics/SumAggregationBuilder.java +++ b/server/src/main/java/org/elasticsearch/search/aggregations/metrics/SumAggregationBuilder.java @@ -79,7 +79,7 @@ protected void innerWriteTo(StreamOutput out) { @Override protected SumAggregatorFactory innerBuild(SearchContext context, ValuesSourceConfig config, - AggregatorFactory parent, Builder subFactoriesBuilder) throws IOException { + AggregatorFactory parent, Builder subFactoriesBuilder) throws IOException { return new SumAggregatorFactory(name, config, context, parent, subFactoriesBuilder, metaData); } @@ -88,16 +88,6 @@ public XContentBuilder doXContentBody(XContentBuilder builder, Params params) th return builder; } - @Override - protected int innerHashCode() { - return 0; - } - - @Override - protected boolean innerEquals(Object obj) { - return true; - } - @Override public String getType() { return NAME; diff --git a/server/src/main/java/org/elasticsearch/search/aggregations/metrics/SumAggregatorFactory.java b/server/src/main/java/org/elasticsearch/search/aggregations/metrics/SumAggregatorFactory.java index d8fa88541cb9b..a59e6dbf1e7e6 100644 --- a/server/src/main/java/org/elasticsearch/search/aggregations/metrics/SumAggregatorFactory.java +++ b/server/src/main/java/org/elasticsearch/search/aggregations/metrics/SumAggregatorFactory.java @@ -33,10 +33,10 @@ import java.util.List; import java.util.Map; -class SumAggregatorFactory extends ValuesSourceAggregatorFactory { +class SumAggregatorFactory extends ValuesSourceAggregatorFactory { SumAggregatorFactory(String name, ValuesSourceConfig config, SearchContext context, - AggregatorFactory parent, AggregatorFactories.Builder subFactoriesBuilder, Map metaData) throws IOException { + AggregatorFactory parent, AggregatorFactories.Builder subFactoriesBuilder, Map metaData) throws IOException { super(name, config, context, parent, subFactoriesBuilder, metaData); } diff --git a/server/src/main/java/org/elasticsearch/search/aggregations/metrics/TDigestPercentileRanksAggregatorFactory.java b/server/src/main/java/org/elasticsearch/search/aggregations/metrics/TDigestPercentileRanksAggregatorFactory.java index 10913bf59d14f..e4b3a722c7cb7 100644 --- a/server/src/main/java/org/elasticsearch/search/aggregations/metrics/TDigestPercentileRanksAggregatorFactory.java +++ b/server/src/main/java/org/elasticsearch/search/aggregations/metrics/TDigestPercentileRanksAggregatorFactory.java @@ -34,14 +34,14 @@ import java.util.Map; class TDigestPercentileRanksAggregatorFactory - extends ValuesSourceAggregatorFactory { + extends ValuesSourceAggregatorFactory { private final double[] percents; private final double compression; private final boolean keyed; TDigestPercentileRanksAggregatorFactory(String name, ValuesSourceConfig config, double[] percents, - double compression, boolean keyed, SearchContext context, AggregatorFactory parent, + double compression, boolean keyed, SearchContext context, AggregatorFactory parent, AggregatorFactories.Builder subFactoriesBuilder, Map metaData) throws IOException { super(name, config, context, parent, subFactoriesBuilder, metaData); this.percents = percents; diff --git a/server/src/main/java/org/elasticsearch/search/aggregations/metrics/TDigestPercentilesAggregatorFactory.java b/server/src/main/java/org/elasticsearch/search/aggregations/metrics/TDigestPercentilesAggregatorFactory.java index 0c1396196fb62..eed8df5011660 100644 --- a/server/src/main/java/org/elasticsearch/search/aggregations/metrics/TDigestPercentilesAggregatorFactory.java +++ b/server/src/main/java/org/elasticsearch/search/aggregations/metrics/TDigestPercentilesAggregatorFactory.java @@ -34,14 +34,14 @@ import java.util.Map; class TDigestPercentilesAggregatorFactory - extends ValuesSourceAggregatorFactory { + extends ValuesSourceAggregatorFactory { private final double[] percents; private final double compression; private final boolean keyed; TDigestPercentilesAggregatorFactory(String name, ValuesSourceConfig config, double[] percents, - double compression, boolean keyed, SearchContext context, AggregatorFactory parent, + double compression, boolean keyed, SearchContext context, AggregatorFactory parent, AggregatorFactories.Builder subFactoriesBuilder, Map metaData) throws IOException { super(name, config, context, parent, subFactoriesBuilder, metaData); this.percents = percents; diff --git a/server/src/main/java/org/elasticsearch/search/aggregations/metrics/TopHitsAggregationBuilder.java b/server/src/main/java/org/elasticsearch/search/aggregations/metrics/TopHitsAggregationBuilder.java index 019fec82d0df4..87aa56fc15510 100644 --- a/server/src/main/java/org/elasticsearch/search/aggregations/metrics/TopHitsAggregationBuilder.java +++ b/server/src/main/java/org/elasticsearch/search/aggregations/metrics/TopHitsAggregationBuilder.java @@ -572,7 +572,7 @@ public TopHitsAggregationBuilder subAggregations(Builder subFactories) { } @Override - protected TopHitsAggregatorFactory doBuild(SearchContext context, AggregatorFactory parent, Builder subfactoriesBuilder) + protected TopHitsAggregatorFactory doBuild(SearchContext context, AggregatorFactory parent, Builder subfactoriesBuilder) throws IOException { long innerResultWindow = from() + size(); int maxInnerResultWindow = context.mapperService().getIndexSettings().getMaxInnerResultWindow(); @@ -770,26 +770,31 @@ public static TopHitsAggregationBuilder parse(String aggregationName, XContentPa } @Override - protected int doHashCode() { - return Objects.hash(explain, fetchSourceContext, docValueFields, storedFieldsContext, from, highlightBuilder, - scriptFields, size, sorts, trackScores, version, seqNoAndPrimaryTerm); + public int hashCode() { + return Objects.hash(super.hashCode(), explain, fetchSourceContext, docValueFields, + storedFieldsContext, from, highlightBuilder, + scriptFields, size, sorts, trackScores, version, + seqNoAndPrimaryTerm); } @Override - protected boolean doEquals(Object obj) { + public boolean equals(Object obj) { + if (this == obj) return true; + if (obj == null || getClass() != obj.getClass()) return false; + if (super.equals(obj) == false) return false; TopHitsAggregationBuilder other = (TopHitsAggregationBuilder) obj; return Objects.equals(explain, other.explain) - && Objects.equals(fetchSourceContext, other.fetchSourceContext) - && Objects.equals(docValueFields, other.docValueFields) - && Objects.equals(storedFieldsContext, other.storedFieldsContext) - && Objects.equals(from, other.from) - && Objects.equals(highlightBuilder, other.highlightBuilder) - && Objects.equals(scriptFields, other.scriptFields) - && Objects.equals(size, other.size) - && Objects.equals(sorts, other.sorts) - && Objects.equals(trackScores, other.trackScores) - && Objects.equals(version, other.version) - && Objects.equals(seqNoAndPrimaryTerm, other.seqNoAndPrimaryTerm); + && Objects.equals(fetchSourceContext, other.fetchSourceContext) + && Objects.equals(docValueFields, other.docValueFields) + && Objects.equals(storedFieldsContext, other.storedFieldsContext) + && Objects.equals(from, other.from) + && Objects.equals(highlightBuilder, other.highlightBuilder) + && Objects.equals(scriptFields, other.scriptFields) + && Objects.equals(size, other.size) + && Objects.equals(sorts, other.sorts) + && Objects.equals(trackScores, other.trackScores) + && Objects.equals(version, other.version) + && Objects.equals(seqNoAndPrimaryTerm, other.seqNoAndPrimaryTerm); } @Override diff --git a/server/src/main/java/org/elasticsearch/search/aggregations/metrics/TopHitsAggregatorFactory.java b/server/src/main/java/org/elasticsearch/search/aggregations/metrics/TopHitsAggregatorFactory.java index 7edaccb66d4bd..72d8fa254c22f 100644 --- a/server/src/main/java/org/elasticsearch/search/aggregations/metrics/TopHitsAggregatorFactory.java +++ b/server/src/main/java/org/elasticsearch/search/aggregations/metrics/TopHitsAggregatorFactory.java @@ -38,7 +38,7 @@ import java.util.Map; import java.util.Optional; -class TopHitsAggregatorFactory extends AggregatorFactory { +class TopHitsAggregatorFactory extends AggregatorFactory { private final int from; private final int size; @@ -56,7 +56,7 @@ class TopHitsAggregatorFactory extends AggregatorFactory sort, HighlightBuilder highlightBuilder, StoredFieldsContext storedFieldsContext, List docValueFields, List scriptFields, FetchSourceContext fetchSourceContext, - SearchContext context, AggregatorFactory parent, AggregatorFactories.Builder subFactories, Map metaData) + SearchContext context, AggregatorFactory parent, AggregatorFactories.Builder subFactories, Map metaData) throws IOException { super(name, context, parent, subFactories, metaData); this.from = from; diff --git a/server/src/main/java/org/elasticsearch/search/aggregations/metrics/ValueCountAggregationBuilder.java b/server/src/main/java/org/elasticsearch/search/aggregations/metrics/ValueCountAggregationBuilder.java index 70243cb8bc47a..ccf8ef8ba3dca 100644 --- a/server/src/main/java/org/elasticsearch/search/aggregations/metrics/ValueCountAggregationBuilder.java +++ b/server/src/main/java/org/elasticsearch/search/aggregations/metrics/ValueCountAggregationBuilder.java @@ -84,7 +84,7 @@ protected boolean serializeTargetValueType() { @Override protected ValueCountAggregatorFactory innerBuild(SearchContext context, ValuesSourceConfig config, - AggregatorFactory parent, AggregatorFactories.Builder subFactoriesBuilder) throws IOException { + AggregatorFactory parent, AggregatorFactories.Builder subFactoriesBuilder) throws IOException { return new ValueCountAggregatorFactory(name, config, context, parent, subFactoriesBuilder, metaData); } @@ -93,16 +93,6 @@ public XContentBuilder doXContentBody(XContentBuilder builder, Params params) th return builder; } - @Override - protected int innerHashCode() { - return 0; - } - - @Override - protected boolean innerEquals(Object obj) { - return true; - } - @Override public String getType() { return NAME; diff --git a/server/src/main/java/org/elasticsearch/search/aggregations/metrics/ValueCountAggregatorFactory.java b/server/src/main/java/org/elasticsearch/search/aggregations/metrics/ValueCountAggregatorFactory.java index 26f1760940d20..b149dc9a7307c 100644 --- a/server/src/main/java/org/elasticsearch/search/aggregations/metrics/ValueCountAggregatorFactory.java +++ b/server/src/main/java/org/elasticsearch/search/aggregations/metrics/ValueCountAggregatorFactory.java @@ -32,10 +32,10 @@ import java.util.List; import java.util.Map; -class ValueCountAggregatorFactory extends ValuesSourceAggregatorFactory { +class ValueCountAggregatorFactory extends ValuesSourceAggregatorFactory { ValueCountAggregatorFactory(String name, ValuesSourceConfig config, SearchContext context, - AggregatorFactory parent, AggregatorFactories.Builder subFactoriesBuilder, Map metaData) throws IOException { + AggregatorFactory parent, AggregatorFactories.Builder subFactoriesBuilder, Map metaData) throws IOException { super(name, config, context, parent, subFactoriesBuilder, metaData); } diff --git a/server/src/main/java/org/elasticsearch/search/aggregations/metrics/WeightedAvgAggregationBuilder.java b/server/src/main/java/org/elasticsearch/search/aggregations/metrics/WeightedAvgAggregationBuilder.java index c3f67fb905254..4cc2ff97c4ce9 100644 --- a/server/src/main/java/org/elasticsearch/search/aggregations/metrics/WeightedAvgAggregationBuilder.java +++ b/server/src/main/java/org/elasticsearch/search/aggregations/metrics/WeightedAvgAggregationBuilder.java @@ -98,10 +98,10 @@ protected void innerWriteTo(StreamOutput out) { } @Override - protected MultiValuesSourceAggregatorFactory innerBuild(SearchContext context, + protected MultiValuesSourceAggregatorFactory innerBuild(SearchContext context, Map> configs, DocValueFormat format, - AggregatorFactory parent, + AggregatorFactory parent, Builder subFactoriesBuilder) throws IOException { return new WeightedAvgAggregatorFactory(name, configs, format, context, parent, subFactoriesBuilder, metaData); } @@ -111,16 +111,6 @@ public XContentBuilder doXContentBody(XContentBuilder builder, ToXContent.Params return builder; } - @Override - protected int innerHashCode() { - return 0; - } - - @Override - protected boolean innerEquals(Object obj) { - return true; - } - @Override public String getType() { return NAME; diff --git a/server/src/main/java/org/elasticsearch/search/aggregations/metrics/WeightedAvgAggregatorFactory.java b/server/src/main/java/org/elasticsearch/search/aggregations/metrics/WeightedAvgAggregatorFactory.java index afdb727c512b0..0e57e000cf116 100644 --- a/server/src/main/java/org/elasticsearch/search/aggregations/metrics/WeightedAvgAggregatorFactory.java +++ b/server/src/main/java/org/elasticsearch/search/aggregations/metrics/WeightedAvgAggregatorFactory.java @@ -34,10 +34,10 @@ import java.util.List; import java.util.Map; -class WeightedAvgAggregatorFactory extends MultiValuesSourceAggregatorFactory { +class WeightedAvgAggregatorFactory extends MultiValuesSourceAggregatorFactory { WeightedAvgAggregatorFactory(String name, Map> configs, - DocValueFormat format, SearchContext context, AggregatorFactory parent, + DocValueFormat format, SearchContext context, AggregatorFactory parent, AggregatorFactories.Builder subFactoriesBuilder, Map metaData) throws IOException { super(name, configs, format, context, parent, subFactoriesBuilder, metaData); diff --git a/server/src/main/java/org/elasticsearch/search/aggregations/pipeline/AbstractPipelineAggregationBuilder.java b/server/src/main/java/org/elasticsearch/search/aggregations/pipeline/AbstractPipelineAggregationBuilder.java index 8a278ead95a2b..fa2f8c8090bcd 100644 --- a/server/src/main/java/org/elasticsearch/search/aggregations/pipeline/AbstractPipelineAggregationBuilder.java +++ b/server/src/main/java/org/elasticsearch/search/aggregations/pipeline/AbstractPipelineAggregationBuilder.java @@ -84,7 +84,7 @@ public String type() { * configured) */ @Override - public final void validate(AggregatorFactory parent, Collection factories, + public final void validate(AggregatorFactory parent, Collection factories, Collection pipelineAggregatorFactories) { doValidate(parent, factories, pipelineAggregatorFactories); } @@ -102,14 +102,14 @@ public final PipelineAggregator create() { return aggregator; } - public void doValidate(AggregatorFactory parent, Collection factories, + public void doValidate(AggregatorFactory parent, Collection factories, Collection pipelineAggregatorFactories) { } /** * Validates pipeline aggregations that need sequentially ordered data. */ - public static void validateSequentiallyOrderedParentAggs(AggregatorFactory parent, String type, String name) { + public static void validateSequentiallyOrderedParentAggs(AggregatorFactory parent, String type, String name) { if ((parent instanceof HistogramAggregatorFactory || parent instanceof DateHistogramAggregatorFactory || parent instanceof AutoDateHistogramAggregatorFactory) == false) { throw new IllegalStateException( @@ -171,32 +171,20 @@ protected boolean overrideBucketsPath() { @Override public int hashCode() { - return Objects.hash(Arrays.hashCode(bucketsPaths), metaData, name, type, doHashCode()); + return Objects.hash(Arrays.hashCode(bucketsPaths), metaData, name, type); } - protected abstract int doHashCode(); - @Override public boolean equals(Object obj) { - if (obj == null) - return false; - if (getClass() != obj.getClass()) - return false; - @SuppressWarnings("unchecked") + if (this == obj) return true; + if (obj == null || getClass() != obj.getClass()) return false; AbstractPipelineAggregationBuilder other = (AbstractPipelineAggregationBuilder) obj; - if (!Objects.equals(name, other.name)) - return false; - if (!Objects.equals(type, other.type)) - return false; - if (!Objects.deepEquals(bucketsPaths, other.bucketsPaths)) - return false; - if (!Objects.equals(metaData, other.metaData)) - return false; - return doEquals(obj); + return Objects.equals(type, other.type) + && Objects.equals(name, other.name) + && Objects.equals(metaData, other.metaData) + && Objects.deepEquals(bucketsPaths, other.bucketsPaths); } - protected abstract boolean doEquals(Object obj); - @Override public String getType() { return type; diff --git a/server/src/main/java/org/elasticsearch/search/aggregations/pipeline/AvgBucketPipelineAggregationBuilder.java b/server/src/main/java/org/elasticsearch/search/aggregations/pipeline/AvgBucketPipelineAggregationBuilder.java index 1495c691c83d7..005c39416ca3b 100644 --- a/server/src/main/java/org/elasticsearch/search/aggregations/pipeline/AvgBucketPipelineAggregationBuilder.java +++ b/server/src/main/java/org/elasticsearch/search/aggregations/pipeline/AvgBucketPipelineAggregationBuilder.java @@ -63,16 +63,6 @@ protected AvgBucketPipelineAggregationBuilder buildFactory(String pipelineAggreg } }; - @Override - protected int innerHashCode() { - return 0; - } - - @Override - protected boolean innerEquals(BucketMetricsPipelineAggregationBuilder other) { - return true; - } - @Override public String getWriteableName() { return NAME; diff --git a/server/src/main/java/org/elasticsearch/search/aggregations/pipeline/BucketMetricsPipelineAggregationBuilder.java b/server/src/main/java/org/elasticsearch/search/aggregations/pipeline/BucketMetricsPipelineAggregationBuilder.java index eddc48c6fdcba..0c4aa6f1359bd 100644 --- a/server/src/main/java/org/elasticsearch/search/aggregations/pipeline/BucketMetricsPipelineAggregationBuilder.java +++ b/server/src/main/java/org/elasticsearch/search/aggregations/pipeline/BucketMetricsPipelineAggregationBuilder.java @@ -107,7 +107,7 @@ public GapPolicy gapPolicy() { protected abstract PipelineAggregator createInternal(Map metaData); @Override - public void doValidate(AggregatorFactory parent, Collection aggBuilders, + public void doValidate(AggregatorFactory parent, Collection aggBuilders, Collection pipelineAggregatorFactories) { if (bucketsPaths.length != 1) { throw new IllegalStateException(PipelineAggregator.Parser.BUCKETS_PATH.getPreferredName() @@ -147,21 +147,19 @@ protected final XContentBuilder internalXContent(XContentBuilder builder, Params protected abstract XContentBuilder doXContentBody(XContentBuilder builder, Params params) throws IOException; @Override - protected final int doHashCode() { - return Objects.hash(format, gapPolicy, innerHashCode()); + public int hashCode() { + return Objects.hash(super.hashCode(), format, gapPolicy); } - protected abstract int innerHashCode(); - @Override - protected final boolean doEquals(Object obj) { + public boolean equals(Object obj) { + if (this == obj) return true; + if (obj == null || getClass() != obj.getClass()) return false; + if (super.equals(obj) == false) return false; @SuppressWarnings("unchecked") BucketMetricsPipelineAggregationBuilder other = (BucketMetricsPipelineAggregationBuilder) obj; return Objects.equals(format, other.format) - && Objects.equals(gapPolicy, other.gapPolicy) - && innerEquals(other); + && Objects.equals(gapPolicy, other.gapPolicy); } - protected abstract boolean innerEquals(BucketMetricsPipelineAggregationBuilder other); - } diff --git a/server/src/main/java/org/elasticsearch/search/aggregations/pipeline/BucketScriptPipelineAggregationBuilder.java b/server/src/main/java/org/elasticsearch/search/aggregations/pipeline/BucketScriptPipelineAggregationBuilder.java index c8ea3553752f8..a0dabefed323f 100644 --- a/server/src/main/java/org/elasticsearch/search/aggregations/pipeline/BucketScriptPipelineAggregationBuilder.java +++ b/server/src/main/java/org/elasticsearch/search/aggregations/pipeline/BucketScriptPipelineAggregationBuilder.java @@ -240,15 +240,20 @@ protected boolean overrideBucketsPath() { } @Override - protected int doHashCode() { - return Objects.hash(bucketsPathsMap, script, format, gapPolicy); + public int hashCode() { + return Objects.hash(super.hashCode(), bucketsPathsMap, script, format, gapPolicy); } @Override - protected boolean doEquals(Object obj) { + public boolean equals(Object obj) { + if (this == obj) return true; + if (obj == null || getClass() != obj.getClass()) return false; + if (super.equals(obj) == false) return false; BucketScriptPipelineAggregationBuilder other = (BucketScriptPipelineAggregationBuilder) obj; - return Objects.equals(bucketsPathsMap, other.bucketsPathsMap) && Objects.equals(script, other.script) - && Objects.equals(format, other.format) && Objects.equals(gapPolicy, other.gapPolicy); + return Objects.equals(bucketsPathsMap, other.bucketsPathsMap) + && Objects.equals(script, other.script) + && Objects.equals(format, other.format) + && Objects.equals(gapPolicy, other.gapPolicy); } @Override diff --git a/server/src/main/java/org/elasticsearch/search/aggregations/pipeline/BucketSelectorPipelineAggregationBuilder.java b/server/src/main/java/org/elasticsearch/search/aggregations/pipeline/BucketSelectorPipelineAggregationBuilder.java index a6627be1cfe6f..6bb83c60dea83 100644 --- a/server/src/main/java/org/elasticsearch/search/aggregations/pipeline/BucketSelectorPipelineAggregationBuilder.java +++ b/server/src/main/java/org/elasticsearch/search/aggregations/pipeline/BucketSelectorPipelineAggregationBuilder.java @@ -199,15 +199,20 @@ protected boolean overrideBucketsPath() { } @Override - protected int doHashCode() { - return Objects.hash(bucketsPathsMap, script, gapPolicy); + public int hashCode() { + return Objects.hash(super.hashCode(), bucketsPathsMap, script, gapPolicy); } @Override - protected boolean doEquals(Object obj) { + public boolean equals(Object obj) { + if (this == obj) return true; + if (obj == null || getClass() != obj.getClass()) return false; + if (super.equals(obj) == false) return false; + BucketSelectorPipelineAggregationBuilder other = (BucketSelectorPipelineAggregationBuilder) obj; - return Objects.equals(bucketsPathsMap, other.bucketsPathsMap) && Objects.equals(script, other.script) - && Objects.equals(gapPolicy, other.gapPolicy); + return Objects.equals(bucketsPathsMap, other.bucketsPathsMap) + && Objects.equals(script, other.script) + && Objects.equals(gapPolicy, other.gapPolicy); } @Override diff --git a/server/src/main/java/org/elasticsearch/search/aggregations/pipeline/BucketSortPipelineAggregationBuilder.java b/server/src/main/java/org/elasticsearch/search/aggregations/pipeline/BucketSortPipelineAggregationBuilder.java index 4dcd42934fc96..1e80c6f78c3c5 100644 --- a/server/src/main/java/org/elasticsearch/search/aggregations/pipeline/BucketSortPipelineAggregationBuilder.java +++ b/server/src/main/java/org/elasticsearch/search/aggregations/pipeline/BucketSortPipelineAggregationBuilder.java @@ -144,7 +144,7 @@ protected PipelineAggregator createInternal(Map metaData) { } @Override - public void doValidate(AggregatorFactory parent, Collection aggFactories, + public void doValidate(AggregatorFactory parent, Collection aggFactories, Collection pipelineAggregatoractories) { if (sorts.isEmpty() && size == null && from == 0) { throw new IllegalStateException("[" + name + "] is configured to perform nothing. Please set either of " @@ -174,12 +174,15 @@ protected boolean overrideBucketsPath() { } @Override - protected int doHashCode() { - return Objects.hash(sorts, from, size, gapPolicy); + public int hashCode() { + return Objects.hash(super.hashCode(), sorts, from, size, gapPolicy); } @Override - protected boolean doEquals(Object obj) { + public boolean equals(Object obj) { + if (this == obj) return true; + if (obj == null || getClass() != obj.getClass()) return false; + if (super.equals(obj) == false) return false; BucketSortPipelineAggregationBuilder other = (BucketSortPipelineAggregationBuilder) obj; return Objects.equals(sorts, other.sorts) && Objects.equals(from, other.from) diff --git a/server/src/main/java/org/elasticsearch/search/aggregations/pipeline/CumulativeSumPipelineAggregationBuilder.java b/server/src/main/java/org/elasticsearch/search/aggregations/pipeline/CumulativeSumPipelineAggregationBuilder.java index a8b51d9c1a5fe..fc9b8caf72769 100644 --- a/server/src/main/java/org/elasticsearch/search/aggregations/pipeline/CumulativeSumPipelineAggregationBuilder.java +++ b/server/src/main/java/org/elasticsearch/search/aggregations/pipeline/CumulativeSumPipelineAggregationBuilder.java @@ -93,7 +93,7 @@ protected PipelineAggregator createInternal(Map metaData) { } @Override - public void doValidate(AggregatorFactory parent, Collection aggFactories, + public void doValidate(AggregatorFactory parent, Collection aggFactories, Collection pipelineAggregatorFactories) { if (bucketsPaths.length != 1) { throw new IllegalStateException(BUCKETS_PATH.getPreferredName() @@ -162,12 +162,15 @@ public static CumulativeSumPipelineAggregationBuilder parse(String pipelineAggre } @Override - protected int doHashCode() { - return Objects.hash(format); + public int hashCode() { + return Objects.hash(super.hashCode(), format); } @Override - protected boolean doEquals(Object obj) { + public boolean equals(Object obj) { + if (this == obj) return true; + if (obj == null || getClass() != obj.getClass()) return false; + if (super.equals(obj) == false) return false; CumulativeSumPipelineAggregationBuilder other = (CumulativeSumPipelineAggregationBuilder) obj; return Objects.equals(format, other.format); } diff --git a/server/src/main/java/org/elasticsearch/search/aggregations/pipeline/DerivativePipelineAggregationBuilder.java b/server/src/main/java/org/elasticsearch/search/aggregations/pipeline/DerivativePipelineAggregationBuilder.java index 25f30bc8343cb..4aa4155ac063f 100644 --- a/server/src/main/java/org/elasticsearch/search/aggregations/pipeline/DerivativePipelineAggregationBuilder.java +++ b/server/src/main/java/org/elasticsearch/search/aggregations/pipeline/DerivativePipelineAggregationBuilder.java @@ -152,7 +152,7 @@ protected PipelineAggregator createInternal(Map metaData) { } @Override - public void doValidate(AggregatorFactory parent, Collection aggFactories, + public void doValidate(AggregatorFactory parent, Collection aggFactories, Collection pipelineAggregatoractories) { if (bucketsPaths.length != 1) { throw new IllegalStateException(PipelineAggregator.Parser.BUCKETS_PATH.getPreferredName() @@ -238,23 +238,19 @@ public static DerivativePipelineAggregationBuilder parse(String pipelineAggregat } @Override - protected boolean doEquals(Object obj) { + public boolean equals(Object obj) { + if (this == obj) return true; + if (obj == null || getClass() != obj.getClass()) return false; + if (super.equals(obj) == false) return false; DerivativePipelineAggregationBuilder other = (DerivativePipelineAggregationBuilder) obj; - if (!Objects.equals(format, other.format)) { - return false; - } - if (!Objects.equals(gapPolicy, other.gapPolicy)) { - return false; - } - if (!Objects.equals(units, other.units)) { - return false; - } - return true; + return Objects.equals(format, other.format) && + gapPolicy == other.gapPolicy && + Objects.equals(units, other.units); } @Override - protected int doHashCode() { - return Objects.hash(format, gapPolicy, units); + public int hashCode() { + return Objects.hash(super.hashCode(), format, gapPolicy, units); } @Override diff --git a/server/src/main/java/org/elasticsearch/search/aggregations/pipeline/ExtendedStatsBucketPipelineAggregationBuilder.java b/server/src/main/java/org/elasticsearch/search/aggregations/pipeline/ExtendedStatsBucketPipelineAggregationBuilder.java index 3d16cf91ee065..89816a1fb22e8 100644 --- a/server/src/main/java/org/elasticsearch/search/aggregations/pipeline/ExtendedStatsBucketPipelineAggregationBuilder.java +++ b/server/src/main/java/org/elasticsearch/search/aggregations/pipeline/ExtendedStatsBucketPipelineAggregationBuilder.java @@ -80,7 +80,7 @@ protected PipelineAggregator createInternal(Map metaData) { } @Override - public void doValidate(AggregatorFactory parent, Collection aggBuilders, + public void doValidate(AggregatorFactory parent, Collection aggBuilders, Collection pipelineAggregatorFactories) { super.doValidate(parent, aggBuilders, pipelineAggregatorFactories); @@ -97,12 +97,15 @@ protected XContentBuilder doXContentBody(XContentBuilder builder, Params params) } @Override - protected int innerHashCode() { - return Objects.hash(sigma); + public int hashCode() { + return Objects.hash(super.hashCode(), sigma); } @Override - protected boolean innerEquals(BucketMetricsPipelineAggregationBuilder obj) { + public boolean equals(Object obj) { + if (this == obj) return true; + if (obj == null || getClass() != obj.getClass()) return false; + if (super.equals(obj) == false) return false; ExtendedStatsBucketPipelineAggregationBuilder other = (ExtendedStatsBucketPipelineAggregationBuilder) obj; return Objects.equals(sigma, other.sigma); } diff --git a/server/src/main/java/org/elasticsearch/search/aggregations/pipeline/InternalBucketMetricValue.java b/server/src/main/java/org/elasticsearch/search/aggregations/pipeline/InternalBucketMetricValue.java index 8a3de634dbfd4..1acdb54080693 100644 --- a/server/src/main/java/org/elasticsearch/search/aggregations/pipeline/InternalBucketMetricValue.java +++ b/server/src/main/java/org/elasticsearch/search/aggregations/pipeline/InternalBucketMetricValue.java @@ -118,12 +118,15 @@ public XContentBuilder doXContentBody(XContentBuilder builder, Params params) th } @Override - protected int doHashCode() { - return Objects.hash(value, Arrays.hashCode(keys)); + public int hashCode() { + return Objects.hash(super.hashCode(), value, Arrays.hashCode(keys)); } @Override - protected boolean doEquals(Object obj) { + public boolean equals(Object obj) { + if (this == obj) return true; + if (obj == null || getClass() != obj.getClass()) return false; + if (super.equals(obj) == false) return false; InternalBucketMetricValue other = (InternalBucketMetricValue) obj; return Objects.equals(value, other.value) && Arrays.equals(keys, other.keys); diff --git a/server/src/main/java/org/elasticsearch/search/aggregations/pipeline/InternalDerivative.java b/server/src/main/java/org/elasticsearch/search/aggregations/pipeline/InternalDerivative.java index b8d5245846aeb..6597689bb6d32 100644 --- a/server/src/main/java/org/elasticsearch/search/aggregations/pipeline/InternalDerivative.java +++ b/server/src/main/java/org/elasticsearch/search/aggregations/pipeline/InternalDerivative.java @@ -98,12 +98,15 @@ public XContentBuilder doXContentBody(XContentBuilder builder, Params params) th } @Override - protected int doHashCode() { - return Objects.hash(normalizationFactor, value); + public int hashCode() { + return Objects.hash(super.hashCode(), normalizationFactor, value); } @Override - protected boolean doEquals(Object obj) { + public boolean equals(Object obj) { + if (this == obj) return true; + if (obj == null || getClass() != obj.getClass()) return false; + if (super.equals(obj) == false) return false; InternalDerivative other = (InternalDerivative) obj; return Objects.equals(value, other.value) && Objects.equals(normalizationFactor, other.normalizationFactor); diff --git a/server/src/main/java/org/elasticsearch/search/aggregations/pipeline/InternalPercentilesBucket.java b/server/src/main/java/org/elasticsearch/search/aggregations/pipeline/InternalPercentilesBucket.java index d9a3908d22136..77493f66d643e 100644 --- a/server/src/main/java/org/elasticsearch/search/aggregations/pipeline/InternalPercentilesBucket.java +++ b/server/src/main/java/org/elasticsearch/search/aggregations/pipeline/InternalPercentilesBucket.java @@ -156,14 +156,18 @@ public XContentBuilder doXContentBody(XContentBuilder builder, Params params) th } @Override - protected boolean doEquals(Object obj) { + public boolean equals(Object obj) { + if (this == obj) return true; + if (obj == null || getClass() != obj.getClass()) return false; + if (super.equals(obj) == false) return false; + InternalPercentilesBucket that = (InternalPercentilesBucket) obj; return Arrays.equals(percents, that.percents) && Arrays.equals(percentiles, that.percentiles); } @Override - protected int doHashCode() { - return Objects.hash(Arrays.hashCode(percents), Arrays.hashCode(percentiles)); + public int hashCode() { + return Objects.hash(super.hashCode(), Arrays.hashCode(percents), Arrays.hashCode(percentiles)); } public static class Iter implements Iterator { diff --git a/server/src/main/java/org/elasticsearch/search/aggregations/pipeline/InternalSimpleValue.java b/server/src/main/java/org/elasticsearch/search/aggregations/pipeline/InternalSimpleValue.java index 2aac262a0076b..4f7b51b6e3b38 100644 --- a/server/src/main/java/org/elasticsearch/search/aggregations/pipeline/InternalSimpleValue.java +++ b/server/src/main/java/org/elasticsearch/search/aggregations/pipeline/InternalSimpleValue.java @@ -91,12 +91,15 @@ public XContentBuilder doXContentBody(XContentBuilder builder, Params params) th } @Override - protected int doHashCode() { - return Objects.hash(value); + public int hashCode() { + return Objects.hash(super.hashCode(), value); } @Override - protected boolean doEquals(Object obj) { + public boolean equals(Object obj) { + if (this == obj) return true; + if (obj == null || getClass() != obj.getClass()) return false; + if (super.equals(obj) == false) return false; InternalSimpleValue other = (InternalSimpleValue) obj; return Objects.equals(value, other.value); } diff --git a/server/src/main/java/org/elasticsearch/search/aggregations/pipeline/MaxBucketPipelineAggregationBuilder.java b/server/src/main/java/org/elasticsearch/search/aggregations/pipeline/MaxBucketPipelineAggregationBuilder.java index b335c15865d70..b37c41d85febc 100644 --- a/server/src/main/java/org/elasticsearch/search/aggregations/pipeline/MaxBucketPipelineAggregationBuilder.java +++ b/server/src/main/java/org/elasticsearch/search/aggregations/pipeline/MaxBucketPipelineAggregationBuilder.java @@ -63,16 +63,6 @@ protected MaxBucketPipelineAggregationBuilder buildFactory(String pipelineAggreg } }; - @Override - protected int innerHashCode() { - return 0; - } - - @Override - protected boolean innerEquals(BucketMetricsPipelineAggregationBuilder other) { - return true; - } - @Override public String getWriteableName() { return NAME; diff --git a/server/src/main/java/org/elasticsearch/search/aggregations/pipeline/MinBucketPipelineAggregationBuilder.java b/server/src/main/java/org/elasticsearch/search/aggregations/pipeline/MinBucketPipelineAggregationBuilder.java index 405285993c0ab..766a9c3849070 100644 --- a/server/src/main/java/org/elasticsearch/search/aggregations/pipeline/MinBucketPipelineAggregationBuilder.java +++ b/server/src/main/java/org/elasticsearch/search/aggregations/pipeline/MinBucketPipelineAggregationBuilder.java @@ -63,16 +63,6 @@ protected MinBucketPipelineAggregationBuilder buildFactory(String pipelineAggreg } }; - @Override - protected int innerHashCode() { - return 0; - } - - @Override - protected boolean innerEquals(BucketMetricsPipelineAggregationBuilder other) { - return true; - } - @Override public String getWriteableName() { return NAME; diff --git a/server/src/main/java/org/elasticsearch/search/aggregations/pipeline/MovFnPipelineAggregationBuilder.java b/server/src/main/java/org/elasticsearch/search/aggregations/pipeline/MovFnPipelineAggregationBuilder.java index 3f589d8a08949..44f26c3c32bf8 100644 --- a/server/src/main/java/org/elasticsearch/search/aggregations/pipeline/MovFnPipelineAggregationBuilder.java +++ b/server/src/main/java/org/elasticsearch/search/aggregations/pipeline/MovFnPipelineAggregationBuilder.java @@ -169,7 +169,7 @@ public void setWindow(int window) { } @Override - public void doValidate(AggregatorFactory parent, Collection aggFactories, + public void doValidate(AggregatorFactory parent, Collection aggFactories, Collection pipelineAggregatoractories) { if (window <= 0) { throw new IllegalArgumentException("[" + WINDOW.getPreferredName() + "] must be a positive, non-zero integer."); @@ -224,12 +224,15 @@ protected boolean overrideBucketsPath() { } @Override - protected int doHashCode() { - return Objects.hash(bucketsPathString, script, format, gapPolicy, window); + public int hashCode() { + return Objects.hash(super.hashCode(), bucketsPathString, script, format, gapPolicy, window); } @Override - protected boolean doEquals(Object obj) { + public boolean equals(Object obj) { + if (this == obj) return true; + if (obj == null || getClass() != obj.getClass()) return false; + if (super.equals(obj) == false) return false; MovFnPipelineAggregationBuilder other = (MovFnPipelineAggregationBuilder) obj; return Objects.equals(bucketsPathString, other.bucketsPathString) && Objects.equals(script, other.script) diff --git a/server/src/main/java/org/elasticsearch/search/aggregations/pipeline/PercentilesBucketPipelineAggregationBuilder.java b/server/src/main/java/org/elasticsearch/search/aggregations/pipeline/PercentilesBucketPipelineAggregationBuilder.java index 18105c4518564..06a940af89c77 100644 --- a/server/src/main/java/org/elasticsearch/search/aggregations/pipeline/PercentilesBucketPipelineAggregationBuilder.java +++ b/server/src/main/java/org/elasticsearch/search/aggregations/pipeline/PercentilesBucketPipelineAggregationBuilder.java @@ -109,7 +109,7 @@ protected PipelineAggregator createInternal(Map metaData) { } @Override - public void doValidate(AggregatorFactory parent, Collection aggFactories, + public void doValidate(AggregatorFactory parent, Collection aggFactories, Collection pipelineAggregatorFactories) { super.doValidate(parent, aggFactories, pipelineAggregatorFactories); @@ -172,14 +172,18 @@ else if (KEYED_FIELD.match(field, parser.getDeprecationHandler()) && token == XC }; @Override - protected int innerHashCode() { - return Objects.hash(Arrays.hashCode(percents), keyed); + public int hashCode() { + return Objects.hash(super.hashCode(), Arrays.hashCode(percents), keyed); } @Override - protected boolean innerEquals(BucketMetricsPipelineAggregationBuilder obj) { + public boolean equals(Object obj) { + if (this == obj) return true; + if (obj == null || getClass() != obj.getClass()) return false; + if (super.equals(obj) == false) return false; PercentilesBucketPipelineAggregationBuilder other = (PercentilesBucketPipelineAggregationBuilder) obj; - return Objects.deepEquals(percents, other.percents) && Objects.equals(keyed, other.keyed); + return Objects.deepEquals(percents, other.percents) + && Objects.equals(keyed, other.keyed); } @Override diff --git a/server/src/main/java/org/elasticsearch/search/aggregations/pipeline/SerialDiffPipelineAggregationBuilder.java b/server/src/main/java/org/elasticsearch/search/aggregations/pipeline/SerialDiffPipelineAggregationBuilder.java index 019026740f813..3b02edf51579a 100644 --- a/server/src/main/java/org/elasticsearch/search/aggregations/pipeline/SerialDiffPipelineAggregationBuilder.java +++ b/server/src/main/java/org/elasticsearch/search/aggregations/pipeline/SerialDiffPipelineAggregationBuilder.java @@ -140,7 +140,7 @@ protected PipelineAggregator createInternal(Map metaData) { } @Override - public void doValidate(AggregatorFactory parent, Collection aggFactories, + public void doValidate(AggregatorFactory parent, Collection aggFactories, Collection pipelineAggregatoractories) { validateSequentiallyOrderedParentAggs(parent, NAME, name); } @@ -228,11 +228,15 @@ public static SerialDiffPipelineAggregationBuilder parse(String reducerName, XCo } @Override - protected int doHashCode() { - return Objects.hash(format, gapPolicy, lag); + public int hashCode() { + return Objects.hash(super.hashCode(), format, gapPolicy, lag); } + @Override - protected boolean doEquals(Object obj) { + public boolean equals(Object obj) { + if (this == obj) return true; + if (obj == null || getClass() != obj.getClass()) return false; + if (super.equals(obj) == false) return false; SerialDiffPipelineAggregationBuilder other = (SerialDiffPipelineAggregationBuilder) obj; return Objects.equals(format, other.format) && Objects.equals(gapPolicy, other.gapPolicy) diff --git a/server/src/main/java/org/elasticsearch/search/aggregations/pipeline/SiblingPipelineAggregator.java b/server/src/main/java/org/elasticsearch/search/aggregations/pipeline/SiblingPipelineAggregator.java index c04bd9fa0dfa4..a131dde9c7673 100644 --- a/server/src/main/java/org/elasticsearch/search/aggregations/pipeline/SiblingPipelineAggregator.java +++ b/server/src/main/java/org/elasticsearch/search/aggregations/pipeline/SiblingPipelineAggregator.java @@ -36,7 +36,7 @@ import java.util.stream.StreamSupport; public abstract class SiblingPipelineAggregator extends PipelineAggregator { - SiblingPipelineAggregator(String name, String[] bucketsPaths, Map metaData) { + protected SiblingPipelineAggregator(String name, String[] bucketsPaths, Map metaData) { super(name, bucketsPaths, metaData); } diff --git a/server/src/main/java/org/elasticsearch/search/aggregations/pipeline/StatsBucketPipelineAggregationBuilder.java b/server/src/main/java/org/elasticsearch/search/aggregations/pipeline/StatsBucketPipelineAggregationBuilder.java index 904cc16c29076..4943349c5463f 100644 --- a/server/src/main/java/org/elasticsearch/search/aggregations/pipeline/StatsBucketPipelineAggregationBuilder.java +++ b/server/src/main/java/org/elasticsearch/search/aggregations/pipeline/StatsBucketPipelineAggregationBuilder.java @@ -64,16 +64,6 @@ protected StatsBucketPipelineAggregationBuilder buildFactory(String pipelineAggr } }; - @Override - protected int innerHashCode() { - return 0; - } - - @Override - protected boolean innerEquals(BucketMetricsPipelineAggregationBuilder other) { - return true; - } - @Override public String getWriteableName() { return NAME; diff --git a/server/src/main/java/org/elasticsearch/search/aggregations/pipeline/SumBucketPipelineAggregationBuilder.java b/server/src/main/java/org/elasticsearch/search/aggregations/pipeline/SumBucketPipelineAggregationBuilder.java index eb075d4368981..eae58ee12f320 100644 --- a/server/src/main/java/org/elasticsearch/search/aggregations/pipeline/SumBucketPipelineAggregationBuilder.java +++ b/server/src/main/java/org/elasticsearch/search/aggregations/pipeline/SumBucketPipelineAggregationBuilder.java @@ -63,16 +63,6 @@ protected SumBucketPipelineAggregationBuilder buildFactory(String pipelineAggreg } }; - @Override - protected int innerHashCode() { - return 0; - } - - @Override - protected boolean innerEquals(BucketMetricsPipelineAggregationBuilder other) { - return true; - } - @Override public String getWriteableName() { return NAME; diff --git a/server/src/main/java/org/elasticsearch/search/aggregations/support/MultiValuesSourceAggregationBuilder.java b/server/src/main/java/org/elasticsearch/search/aggregations/support/MultiValuesSourceAggregationBuilder.java index acc0d2ee20bcb..cd37d1a9969ed 100644 --- a/server/src/main/java/org/elasticsearch/search/aggregations/support/MultiValuesSourceAggregationBuilder.java +++ b/server/src/main/java/org/elasticsearch/search/aggregations/support/MultiValuesSourceAggregationBuilder.java @@ -164,7 +164,7 @@ public AB format(String format) { } @Override - protected final MultiValuesSourceAggregatorFactory doBuild(SearchContext context, AggregatorFactory parent, + protected final MultiValuesSourceAggregatorFactory doBuild(SearchContext context, AggregatorFactory parent, AggregatorFactories.Builder subFactoriesBuilder) throws IOException { ValueType finalValueType = this.valueType != null ? this.valueType : targetValueType; @@ -190,8 +190,8 @@ private static DocValueFormat resolveFormat(@Nullable String format, @Nullable V return valueFormat; } - protected abstract MultiValuesSourceAggregatorFactory innerBuild(SearchContext context, - Map> configs, DocValueFormat format, AggregatorFactory parent, + protected abstract MultiValuesSourceAggregatorFactory innerBuild(SearchContext context, + Map> configs, DocValueFormat format, AggregatorFactory parent, AggregatorFactories.Builder subFactoriesBuilder) throws IOException; @@ -225,28 +225,20 @@ public final XContentBuilder internalXContent(XContentBuilder builder, Params pa protected abstract XContentBuilder doXContentBody(XContentBuilder builder, Params params) throws IOException; @Override - protected final int doHashCode() { - return Objects.hash(fields, format, targetValueType, valueType, innerHashCode()); + public int hashCode() { + return Objects.hash(super.hashCode(), fields, format, targetValueType, valueType); } - protected abstract int innerHashCode(); @Override - protected final boolean doEquals(Object other) { - if (this == other) { - return true; - } - - if (other == null || getClass() != other.getClass()) { - return false; - } - - MultiValuesSourceAggregationBuilder that = (MultiValuesSourceAggregationBuilder) other; - - return Objects.equals(this.fields, that.fields) - && Objects.equals(this.format, that.format) - && Objects.equals(this.valueType, that.valueType); + public boolean equals(Object obj) { + if (this == obj) return true; + if (obj == null || getClass() != obj.getClass()) return false; + if (super.equals(obj) == false) return false; + + MultiValuesSourceAggregationBuilder other = (MultiValuesSourceAggregationBuilder) obj; + return Objects.equals(this.fields, other.fields) + && Objects.equals(this.format, other.format) + && Objects.equals(this.valueType, other.valueType); } - - protected abstract boolean innerEquals(Object obj); } diff --git a/server/src/main/java/org/elasticsearch/search/aggregations/support/MultiValuesSourceAggregatorFactory.java b/server/src/main/java/org/elasticsearch/search/aggregations/support/MultiValuesSourceAggregatorFactory.java index 5de8fbd7561dc..9516c9db87537 100644 --- a/server/src/main/java/org/elasticsearch/search/aggregations/support/MultiValuesSourceAggregatorFactory.java +++ b/server/src/main/java/org/elasticsearch/search/aggregations/support/MultiValuesSourceAggregatorFactory.java @@ -30,15 +30,15 @@ import java.util.List; import java.util.Map; -public abstract class MultiValuesSourceAggregatorFactory> - extends AggregatorFactory { +public abstract class MultiValuesSourceAggregatorFactory + extends AggregatorFactory { protected final Map> configs; protected final DocValueFormat format; public MultiValuesSourceAggregatorFactory(String name, Map> configs, DocValueFormat format, SearchContext context, - AggregatorFactory parent, AggregatorFactories.Builder subFactoriesBuilder, + AggregatorFactory parent, AggregatorFactories.Builder subFactoriesBuilder, Map metaData) throws IOException { super(name, context, parent, subFactoriesBuilder, metaData); this.configs = configs; diff --git a/server/src/main/java/org/elasticsearch/search/aggregations/support/ValuesSource.java b/server/src/main/java/org/elasticsearch/search/aggregations/support/ValuesSource.java index b931cd81da954..7fd38288a821b 100644 --- a/server/src/main/java/org/elasticsearch/search/aggregations/support/ValuesSource.java +++ b/server/src/main/java/org/elasticsearch/search/aggregations/support/ValuesSource.java @@ -111,6 +111,15 @@ public abstract SortedSetDocValues ordinalsValues(LeafReaderContext context) public abstract SortedSetDocValues globalOrdinalsValues(LeafReaderContext context) throws IOException; + /** + * Whether this values source is able to provide a mapping between global and segment ordinals, + * by returning the underlying {@link OrdinalMap}. If this method returns false, then calling + * {@link #globalOrdinalsMapping} will result in an {@link UnsupportedOperationException}. + */ + public boolean supportsGlobalOrdinalsMapping() { + return true; + } + /** Returns a mapping from segment ordinals to global ordinals. */ public abstract LongUnaryOperator globalOrdinalsMapping(LeafReaderContext context) throws IOException; @@ -153,6 +162,11 @@ public SortedSetDocValues globalOrdinalsValues(LeafReaderContext context) { return atomicFieldData.getOrdinalsValues(); } + @Override + public boolean supportsGlobalOrdinalsMapping() { + return indexFieldData.supportsGlobalOrdinalsMapping(); + } + @Override public LongUnaryOperator globalOrdinalsMapping(LeafReaderContext context) throws IOException { final IndexOrdinalsFieldData global = indexFieldData.loadGlobal((DirectoryReader)context.parent.reader()); @@ -452,6 +466,7 @@ public boolean advanceExact(int doc) throws IOException { if (bytesValues.advanceExact(doc)) { count = bytesValues.docValueCount(); grow(); + script.setDocument(doc); for (int i = 0; i < count; ++i) { final BytesRef value = bytesValues.nextValue(); script.setNextAggregationValue(value.utf8ToString()); diff --git a/server/src/main/java/org/elasticsearch/search/aggregations/support/ValuesSourceAggregationBuilder.java b/server/src/main/java/org/elasticsearch/search/aggregations/support/ValuesSourceAggregationBuilder.java index 57f2a214a41fb..11969b3f7dbeb 100644 --- a/server/src/main/java/org/elasticsearch/search/aggregations/support/ValuesSourceAggregationBuilder.java +++ b/server/src/main/java/org/elasticsearch/search/aggregations/support/ValuesSourceAggregationBuilder.java @@ -299,10 +299,10 @@ public ZoneId timeZone() { } @Override - protected final ValuesSourceAggregatorFactory doBuild(SearchContext context, AggregatorFactory parent, + protected final ValuesSourceAggregatorFactory doBuild(SearchContext context, AggregatorFactory parent, AggregatorFactories.Builder subFactoriesBuilder) throws IOException { ValuesSourceConfig config = resolveConfig(context); - ValuesSourceAggregatorFactory factory = innerBuild(context, config, parent, subFactoriesBuilder); + ValuesSourceAggregatorFactory factory = innerBuild(context, config, parent, subFactoriesBuilder); return factory; } @@ -312,8 +312,8 @@ protected ValuesSourceConfig resolveConfig(SearchContext context) { valueType, field, script, missing, timeZone, format); } - protected abstract ValuesSourceAggregatorFactory innerBuild(SearchContext context, ValuesSourceConfig config, - AggregatorFactory parent, AggregatorFactories.Builder subFactoriesBuilder) throws IOException; + protected abstract ValuesSourceAggregatorFactory innerBuild(SearchContext context, ValuesSourceConfig config, + AggregatorFactory parent, AggregatorFactories.Builder subFactoriesBuilder) throws IOException; @Override public final XContentBuilder internalXContent(XContentBuilder builder, Params params) throws IOException { @@ -344,34 +344,24 @@ public final XContentBuilder internalXContent(XContentBuilder builder, Params pa protected abstract XContentBuilder doXContentBody(XContentBuilder builder, Params params) throws IOException; @Override - protected final int doHashCode() { - return Objects.hash(field, format, missing, script, targetValueType, timeZone, valueType, valuesSourceType, - innerHashCode()); + public int hashCode() { + return Objects.hash(super.hashCode(), field, format, missing, script, + targetValueType, timeZone, valueType, valuesSourceType); } - protected abstract int innerHashCode(); - @Override - protected final boolean doEquals(Object obj) { + public boolean equals(Object obj) { + if (this == obj) return true; + if (obj == null || getClass() != obj.getClass()) return false; + if (super.equals(obj) == false) return false; ValuesSourceAggregationBuilder other = (ValuesSourceAggregationBuilder) obj; - if (!Objects.equals(field, other.field)) - return false; - if (!Objects.equals(format, other.format)) - return false; - if (!Objects.equals(missing, other.missing)) - return false; - if (!Objects.equals(script, other.script)) - return false; - if (!Objects.equals(targetValueType, other.targetValueType)) - return false; - if (!Objects.equals(timeZone, other.timeZone)) - return false; - if (!Objects.equals(valueType, other.valueType)) - return false; - if (!Objects.equals(valuesSourceType, other.valuesSourceType)) - return false; - return innerEquals(obj); + return Objects.equals(valuesSourceType, other.valuesSourceType) + && Objects.equals(field, other.field) + && Objects.equals(format, other.format) + && Objects.equals(missing, other.missing) + && Objects.equals(script, other.script) + && Objects.equals(targetValueType, other.targetValueType) + && Objects.equals(timeZone, other.timeZone) + && Objects.equals(valueType, other.valueType); } - - protected abstract boolean innerEquals(Object obj); } diff --git a/server/src/main/java/org/elasticsearch/search/aggregations/support/ValuesSourceAggregatorFactory.java b/server/src/main/java/org/elasticsearch/search/aggregations/support/ValuesSourceAggregatorFactory.java index 37260f9013314..17da14aabf8ec 100644 --- a/server/src/main/java/org/elasticsearch/search/aggregations/support/ValuesSourceAggregatorFactory.java +++ b/server/src/main/java/org/elasticsearch/search/aggregations/support/ValuesSourceAggregatorFactory.java @@ -29,13 +29,12 @@ import java.util.List; import java.util.Map; -public abstract class ValuesSourceAggregatorFactory> - extends AggregatorFactory { +public abstract class ValuesSourceAggregatorFactory extends AggregatorFactory { protected ValuesSourceConfig config; public ValuesSourceAggregatorFactory(String name, ValuesSourceConfig config, SearchContext context, - AggregatorFactory parent, AggregatorFactories.Builder subFactoriesBuilder, Map metaData) throws IOException { + AggregatorFactory parent, AggregatorFactories.Builder subFactoriesBuilder, Map metaData) throws IOException { super(name, context, parent, subFactoriesBuilder, metaData); this.config = config; } @@ -43,13 +42,28 @@ public ValuesSourceAggregatorFactory(String name, ValuesSourceConfig config, @Override public Aggregator createInternal(Aggregator parent, boolean collectsFromSingleBucket, List pipelineAggregators, Map metaData) throws IOException { - VS vs = config.toValuesSource(context.getQueryShardContext()); + VS vs = config.toValuesSource(context.getQueryShardContext(), this::resolveMissingAny); if (vs == null) { return createUnmapped(parent, pipelineAggregators, metaData); } return doCreateInternal(vs, parent, collectsFromSingleBucket, pipelineAggregators, metaData); } + /** + * This method provides a hook for aggregations that need finer grained control over the ValuesSource selected when the user supplies a + * missing value and there is no mapped field to infer the type from. This will only be called for aggregations that specify the + * ValuesSourceType.ANY in their constructors (On the builder class). The user supplied object is passed as a parameter, so its type + * may be inspected as needed. + * + * Generally, only the type of the returned ValuesSource is used, so returning the EMPTY instance of the chosen type is recommended. + * + * @param missing The user supplied missing value + * @return A ValuesSource instance compatible with the supplied parameter + */ + protected ValuesSource resolveMissingAny(Object missing) { + return ValuesSource.Bytes.WithOrdinals.EMPTY; + } + protected abstract Aggregator createUnmapped(Aggregator parent, List pipelineAggregators, Map metaData) throws IOException; diff --git a/server/src/main/java/org/elasticsearch/search/aggregations/support/ValuesSourceConfig.java b/server/src/main/java/org/elasticsearch/search/aggregations/support/ValuesSourceConfig.java index c1c171e3e4fd5..919d1b752e22c 100644 --- a/server/src/main/java/org/elasticsearch/search/aggregations/support/ValuesSourceConfig.java +++ b/server/src/main/java/org/elasticsearch/search/aggregations/support/ValuesSourceConfig.java @@ -36,6 +36,7 @@ import java.time.ZoneId; import java.time.ZoneOffset; +import java.util.function.Function; /** * A configuration that tells aggregations how to retrieve data from the index @@ -223,10 +224,15 @@ public DocValueFormat format() { return format; } + @Nullable + public VS toValuesSource(QueryShardContext context) { + return toValuesSource(context, value -> ValuesSource.Bytes.WithOrdinals.EMPTY); + } + /** Get a value source given its configuration. A return value of null indicates that * no value source could be built. */ @Nullable - public VS toValuesSource(QueryShardContext context) { + public VS toValuesSource(QueryShardContext context, Function resolveMissingAny) { if (!valid()) { throw new IllegalStateException( "value source config is invalid; must have either a field context or a script or marked as unwrapped"); @@ -241,8 +247,10 @@ public VS toValuesSource(QueryShardContext context) { vs = (VS) ValuesSource.Numeric.EMPTY; } else if (valueSourceType() == ValuesSourceType.GEOPOINT) { vs = (VS) ValuesSource.GeoPoint.EMPTY; - } else if (valueSourceType() == ValuesSourceType.ANY || valueSourceType() == ValuesSourceType.BYTES) { + } else if (valueSourceType() == ValuesSourceType.BYTES) { vs = (VS) ValuesSource.Bytes.WithOrdinals.EMPTY; + } else if (valueSourceType() == ValuesSourceType.ANY) { + vs = (VS) resolveMissingAny.apply(missing()); } else { throw new IllegalArgumentException("Can't deal with unmapped ValuesSource type " + valueSourceType()); } diff --git a/server/src/main/java/org/elasticsearch/search/fetch/FetchPhase.java b/server/src/main/java/org/elasticsearch/search/fetch/FetchPhase.java index c23be0f4cb994..196e74e0bbabd 100644 --- a/server/src/main/java/org/elasticsearch/search/fetch/FetchPhase.java +++ b/server/src/main/java/org/elasticsearch/search/fetch/FetchPhase.java @@ -31,7 +31,6 @@ import org.apache.lucene.search.Weight; import org.apache.lucene.util.BitSet; import org.elasticsearch.ExceptionsHelper; -import org.elasticsearch.Version; import org.elasticsearch.common.bytes.BytesReference; import org.elasticsearch.common.collect.Tuple; import org.elasticsearch.common.document.DocumentField; @@ -383,32 +382,19 @@ private SearchHit.NestedIdentity getInternalNestedIdentity(SearchContext context BitSet parentBits = context.bitsetFilterCache().getBitSetProducer(parentFilter).getBitSet(subReaderContext); int offset = 0; - if (indexSettings.getIndexVersionCreated().onOrAfter(Version.V_6_5_0)) { - /** - * Starts from the previous parent and finds the offset of the - * nestedSubDocID within the nested children. Nested documents - * are indexed in the same order than in the source array so the offset - * of the nested child is the number of nested document with the same parent - * that appear before him. - */ - int previousParent = parentBits.prevSetBit(currentParent); - for (int docId = childIter.advance(previousParent + 1); docId < nestedSubDocId && docId != DocIdSetIterator.NO_MORE_DOCS; - docId = childIter.nextDoc()) { - offset++; - } - currentParent = nestedSubDocId; - } else { - /** - * Nested documents are in reverse order in this version so we start from the current nested document - * and find the number of documents with the same parent that appear after it. - */ - int nextParent = parentBits.nextSetBit(currentParent); - for (int docId = childIter.advance(currentParent + 1); docId < nextParent && docId != DocIdSetIterator.NO_MORE_DOCS; - docId = childIter.nextDoc()) { - offset++; - } - currentParent = nextParent; + /** + * Starts from the previous parent and finds the offset of the + * nestedSubDocID within the nested children. Nested documents + * are indexed in the same order than in the source array so the offset + * of the nested child is the number of nested document with the same parent + * that appear before him. + */ + int previousParent = parentBits.prevSetBit(currentParent); + for (int docId = childIter.advance(previousParent + 1); docId < nestedSubDocId && docId != DocIdSetIterator.NO_MORE_DOCS; + docId = childIter.nextDoc()) { + offset++; } + currentParent = nestedSubDocId; current = nestedObjectMapper = nestedParentObjectMapper; int currentPrefix = current == null ? 0 : current.name().length() + 1; nestedIdentity = new SearchHit.NestedIdentity(originalName.substring(currentPrefix), offset, nestedIdentity); diff --git a/server/src/main/java/org/elasticsearch/search/fetch/subphase/highlight/FragmentBuilderHelper.java b/server/src/main/java/org/elasticsearch/search/fetch/subphase/highlight/FragmentBuilderHelper.java index 583516c5cd4c2..d896d3a9d922d 100644 --- a/server/src/main/java/org/elasticsearch/search/fetch/subphase/highlight/FragmentBuilderHelper.java +++ b/server/src/main/java/org/elasticsearch/search/fetch/subphase/highlight/FragmentBuilderHelper.java @@ -26,7 +26,7 @@ import org.apache.lucene.search.vectorhighlight.FieldFragList.WeightedFragInfo.SubInfo; import org.apache.lucene.search.vectorhighlight.FragmentsBuilder; import org.apache.lucene.util.CollectionUtil; -import org.elasticsearch.index.analysis.CustomAnalyzer; +import org.elasticsearch.index.analysis.AnalyzerComponentsProvider; import org.elasticsearch.index.analysis.NamedAnalyzer; import org.elasticsearch.index.analysis.TokenFilterFactory; import org.elasticsearch.index.mapper.MappedFieldType; @@ -81,9 +81,8 @@ private static boolean containsBrokenAnalysis(Analyzer analyzer) { if (analyzer instanceof NamedAnalyzer) { analyzer = ((NamedAnalyzer) analyzer).analyzer(); } - if (analyzer instanceof CustomAnalyzer) { - final CustomAnalyzer a = (CustomAnalyzer) analyzer; - TokenFilterFactory[] tokenFilters = a.tokenFilters(); + if (analyzer instanceof AnalyzerComponentsProvider) { + final TokenFilterFactory[] tokenFilters = ((AnalyzerComponentsProvider) analyzer).getComponents().getTokenFilters(); for (TokenFilterFactory tokenFilterFactory : tokenFilters) { if (tokenFilterFactory.breaksFastVectorHighlighter()) { return true; diff --git a/server/src/main/java/org/elasticsearch/search/internal/ContextIndexSearcher.java b/server/src/main/java/org/elasticsearch/search/internal/ContextIndexSearcher.java index 7c56796f3d24d..794c72ec14cfb 100644 --- a/server/src/main/java/org/elasticsearch/search/internal/ContextIndexSearcher.java +++ b/server/src/main/java/org/elasticsearch/search/internal/ContextIndexSearcher.java @@ -20,14 +20,19 @@ package org.elasticsearch.search.internal; import org.apache.lucene.index.DirectoryReader; +import org.apache.lucene.index.IndexReader; import org.apache.lucene.index.LeafReaderContext; import org.apache.lucene.index.Term; import org.apache.lucene.index.TermStates; import org.apache.lucene.search.BulkScorer; import org.apache.lucene.search.CollectionStatistics; +import org.apache.lucene.search.CollectionTerminatedException; import org.apache.lucene.search.Collector; +import org.apache.lucene.search.ConjunctionDISI; +import org.apache.lucene.search.DocIdSetIterator; import org.apache.lucene.search.Explanation; import org.apache.lucene.search.IndexSearcher; +import org.apache.lucene.search.LeafCollector; import org.apache.lucene.search.Query; import org.apache.lucene.search.QueryCache; import org.apache.lucene.search.QueryCachingPolicy; @@ -35,9 +40,13 @@ import org.apache.lucene.search.Scorer; import org.apache.lucene.search.TermStatistics; import org.apache.lucene.search.Weight; -import org.apache.lucene.search.XIndexSearcher; +import org.apache.lucene.search.similarities.Similarity; +import org.apache.lucene.util.BitSet; +import org.apache.lucene.util.BitSetIterator; +import org.apache.lucene.util.Bits; +import org.apache.lucene.util.CombinedBitSet; +import org.apache.lucene.util.SparseFixedBitSet; import org.elasticsearch.common.lease.Releasable; -import org.elasticsearch.index.engine.Engine; import org.elasticsearch.search.dfs.AggregatedDfs; import org.elasticsearch.search.profile.Timer; import org.elasticsearch.search.profile.query.ProfileWeight; @@ -46,6 +55,7 @@ import org.elasticsearch.search.profile.query.QueryTimingType; import java.io.IOException; +import java.util.Arrays; import java.util.List; import java.util.Set; @@ -53,26 +63,19 @@ * Context-aware extension of {@link IndexSearcher}. */ public class ContextIndexSearcher extends IndexSearcher implements Releasable { - - /** The wrapped {@link IndexSearcher}. The reason why we sometimes prefer delegating to this searcher instead of {@code super} is that - * this instance may have more assertions, for example if it comes from MockInternalEngine which wraps the IndexSearcher into an - * AssertingIndexSearcher. */ - private final XIndexSearcher in; + /** + * The interval at which we check for search cancellation when we cannot use + * a {@link CancellableBulkScorer}. See {@link #intersectScorerAndBitSet}. + */ + private static int CHECK_CANCELLED_SCORER_INTERVAL = 1 << 11; private AggregatedDfs aggregatedDfs; - - private final Engine.Searcher engineSearcher; - - // TODO revisit moving the profiler to inheritance or wrapping model in the future private QueryProfiler profiler; - private Runnable checkCancelled; - public ContextIndexSearcher(Engine.Searcher searcher, QueryCache queryCache, QueryCachingPolicy queryCachingPolicy) { - super(searcher.reader()); - engineSearcher = searcher; - in = new XIndexSearcher(searcher.searcher()); - setSimilarity(searcher.searcher().getSimilarity()); + public ContextIndexSearcher(IndexReader reader, Similarity similarity, QueryCache queryCache, QueryCachingPolicy queryCachingPolicy) { + super(reader); + setSimilarity(similarity); setQueryCache(queryCache); setQueryCachingPolicy(queryCachingPolicy); } @@ -104,7 +107,7 @@ public Query rewrite(Query original) throws IOException { } try { - return in.rewrite(original); + return super.rewrite(original); } finally { if (profiler != null) { profiler.stopAndAddRewriteTime(); @@ -130,7 +133,6 @@ public Weight createWeight(Query query, ScoreMode scoreMode, float boost) throws } return new ProfileWeight(query, weight, profile); } else { - // needs to be 'super', not 'in' in order to use aggregated DFS return super.createWeight(query, scoreMode, boost); } } @@ -152,13 +154,13 @@ public Explanation explain(LeafReaderContext context, int doc) throws IOExceptio } @Override - public Scorer scorer(LeafReaderContext context) throws IOException { + public boolean isCacheable(LeafReaderContext ctx) { throw new UnsupportedOperationException(); } @Override - public boolean isCacheable(LeafReaderContext ctx) { - throw new UnsupportedOperationException(); + public Scorer scorer(LeafReaderContext context) throws IOException { + return weight.scorer(context); } @Override @@ -174,16 +176,75 @@ public BulkScorer bulkScorer(LeafReaderContext context) throws IOException { } else { cancellableWeight = weight; } - in.search(leaves, cancellableWeight, collector); + searchInternal(leaves, cancellableWeight, collector); } - @Override - public Explanation explain(Query query, int doc) throws IOException { - if (aggregatedDfs != null) { - // dfs data is needed to explain the score - return super.explain(createWeight(rewrite(query), ScoreMode.COMPLETE, 1f), doc); + private void searchInternal(List leaves, Weight weight, Collector collector) throws IOException { + for (LeafReaderContext ctx : leaves) { // search each subreader + final LeafCollector leafCollector; + try { + leafCollector = collector.getLeafCollector(ctx); + } catch (CollectionTerminatedException e) { + // there is no doc of interest in this reader context + // continue with the following leaf + continue; + } + Bits liveDocs = ctx.reader().getLiveDocs(); + BitSet liveDocsBitSet = getSparseBitSetOrNull(ctx.reader().getLiveDocs()); + if (liveDocsBitSet == null) { + BulkScorer bulkScorer = weight.bulkScorer(ctx); + if (bulkScorer != null) { + try { + bulkScorer.score(leafCollector, liveDocs); + } catch (CollectionTerminatedException e) { + // collection was terminated prematurely + // continue with the following leaf + } + } + } else { + // if the role query result set is sparse then we should use the SparseFixedBitSet for advancing: + Scorer scorer = weight.scorer(ctx); + if (scorer != null) { + try { + intersectScorerAndBitSet(scorer, liveDocsBitSet, leafCollector, + checkCancelled == null ? () -> {} : checkCancelled); + } catch (CollectionTerminatedException e) { + // collection was terminated prematurely + // continue with the following leaf + } + } + } } - return in.explain(query, doc); + } + + private static BitSet getSparseBitSetOrNull(Bits liveDocs) { + if (liveDocs instanceof SparseFixedBitSet) { + return (BitSet) liveDocs; + } else if (liveDocs instanceof CombinedBitSet + // if the underlying role bitset is sparse + && ((CombinedBitSet) liveDocs).getFirst() instanceof SparseFixedBitSet) { + return (BitSet) liveDocs; + } else { + return null; + } + + } + + static void intersectScorerAndBitSet(Scorer scorer, BitSet acceptDocs, + LeafCollector collector, Runnable checkCancelled) throws IOException { + // ConjunctionDISI uses the DocIdSetIterator#cost() to order the iterators, so if roleBits has the lowest cardinality it should + // be used first: + DocIdSetIterator iterator = ConjunctionDISI.intersectIterators(Arrays.asList(new BitSetIterator(acceptDocs, + acceptDocs.approximateCardinality()), scorer.iterator())); + int seen = 0; + checkCancelled.run(); + for (int docId = iterator.nextDoc(); docId < DocIdSetIterator.NO_MORE_DOCS; docId = iterator.nextDoc()) { + if (++seen % CHECK_CANCELLED_SCORER_INTERVAL == 0) { + checkCancelled.run(); + } + collector.collect(docId); + } + checkCancelled.run(); } @Override @@ -215,10 +276,8 @@ public CollectionStatistics collectionStatistics(String field) throws IOExceptio } public DirectoryReader getDirectoryReader() { - return engineSearcher.getDirectoryReader(); - } - - public Engine.Searcher getEngineSearcher() { - return engineSearcher; + final IndexReader reader = getIndexReader(); + assert reader instanceof DirectoryReader : "expected an instance of DirectoryReader, got " + reader.getClass(); + return (DirectoryReader) reader; } } diff --git a/server/src/main/java/org/elasticsearch/search/internal/ShardSearchLocalRequest.java b/server/src/main/java/org/elasticsearch/search/internal/ShardSearchLocalRequest.java index 12eef5dcf29de..70c2aa6e5ac6d 100644 --- a/server/src/main/java/org/elasticsearch/search/internal/ShardSearchLocalRequest.java +++ b/server/src/main/java/org/elasticsearch/search/internal/ShardSearchLocalRequest.java @@ -19,6 +19,7 @@ package org.elasticsearch.search.internal; +import org.elasticsearch.Version; import org.elasticsearch.action.search.SearchRequest; import org.elasticsearch.action.search.SearchType; import org.elasticsearch.common.Nullable; @@ -35,6 +36,7 @@ import org.elasticsearch.search.builder.SearchSourceBuilder; import java.io.IOException; +import java.util.Arrays; /** * Shard level search request that gets created and consumed on the local node. @@ -61,7 +63,6 @@ public class ShardSearchLocalRequest implements ShardSearchRequest { private final int numberOfShards; private final SearchType searchType; private final Scroll scroll; - private final String[] types; private final float indexBoost; private final Boolean requestCache; private final long nowInMillis; @@ -74,7 +75,7 @@ public class ShardSearchLocalRequest implements ShardSearchRequest { public ShardSearchLocalRequest(SearchRequest searchRequest, ShardId shardId, int numberOfShards, AliasFilter aliasFilter, float indexBoost, long nowInMillis, @Nullable String clusterAlias, String[] indexRoutings) { - this(shardId, numberOfShards, searchRequest.searchType(), searchRequest.source(), searchRequest.types(), + this(shardId, numberOfShards, searchRequest.searchType(), searchRequest.source(), searchRequest.requestCache(), aliasFilter, indexBoost, searchRequest.allowPartialSearchResults(), indexRoutings, searchRequest.preference(), searchRequest.scroll(), nowInMillis, clusterAlias); // If allowPartialSearchResults is unset (ie null), the cluster-level default should have been substituted @@ -82,11 +83,11 @@ public ShardSearchLocalRequest(SearchRequest searchRequest, ShardId shardId, int assert searchRequest.allowPartialSearchResults() != null; } - public ShardSearchLocalRequest(ShardId shardId, String[] types, long nowInMillis, AliasFilter aliasFilter) { - this(shardId, -1, null, null, types, null, aliasFilter, 1.0f, false, Strings.EMPTY_ARRAY, null, null, nowInMillis, null); + public ShardSearchLocalRequest(ShardId shardId, long nowInMillis, AliasFilter aliasFilter) { + this(shardId, -1, null, null, null, aliasFilter, 1.0f, false, Strings.EMPTY_ARRAY, null, null, nowInMillis, null); } - private ShardSearchLocalRequest(ShardId shardId, int numberOfShards, SearchType searchType, SearchSourceBuilder source, String[] types, + private ShardSearchLocalRequest(ShardId shardId, int numberOfShards, SearchType searchType, SearchSourceBuilder source, Boolean requestCache, AliasFilter aliasFilter, float indexBoost, boolean allowPartialSearchResults, String[] indexRoutings, String preference, Scroll scroll, long nowInMillis, @Nullable String clusterAlias) { @@ -94,7 +95,6 @@ private ShardSearchLocalRequest(ShardId shardId, int numberOfShards, SearchType this.numberOfShards = numberOfShards; this.searchType = searchType; this.source = source; - this.types = types; this.requestCache = requestCache; this.aliasFilter = aliasFilter; this.indexBoost = indexBoost; @@ -112,7 +112,14 @@ private ShardSearchLocalRequest(ShardId shardId, int numberOfShards, SearchType numberOfShards = in.readVInt(); scroll = in.readOptionalWriteable(Scroll::new); source = in.readOptionalWriteable(SearchSourceBuilder::new); - types = in.readStringArray(); + if (in.getVersion().before(Version.V_8_0_0)) { + // types no longer relevant so ignore + String[] types = in.readStringArray(); + if (types.length > 0) { + throw new IllegalStateException( + "types are no longer supported in search requests but found [" + Arrays.toString(types) + "]"); + } + } aliasFilter = new AliasFilter(in); indexBoost = in.readFloat(); nowInMillis = in.readVLong(); @@ -131,7 +138,10 @@ protected final void innerWriteTo(StreamOutput out, boolean asKey) throws IOExce } out.writeOptionalWriteable(scroll); out.writeOptionalWriteable(source); - out.writeStringArray(types); + if (out.getVersion().before(Version.V_8_0_0)) { + // types not supported so send an empty array to previous versions + out.writeStringArray(Strings.EMPTY_ARRAY); + } aliasFilter.writeTo(out); out.writeFloat(indexBoost); if (asKey == false) { @@ -151,11 +161,6 @@ public ShardId shardId() { return shardId; } - @Override - public String[] types() { - return types; - } - @Override public SearchSourceBuilder source() { return source; diff --git a/server/src/main/java/org/elasticsearch/search/internal/ShardSearchRequest.java b/server/src/main/java/org/elasticsearch/search/internal/ShardSearchRequest.java index b88bda9009043..0e9d5de9788f3 100644 --- a/server/src/main/java/org/elasticsearch/search/internal/ShardSearchRequest.java +++ b/server/src/main/java/org/elasticsearch/search/internal/ShardSearchRequest.java @@ -49,8 +49,6 @@ public interface ShardSearchRequest { ShardId shardId(); - String[] types(); - SearchSourceBuilder source(); AliasFilter getAliasFilter(); diff --git a/server/src/main/java/org/elasticsearch/search/internal/ShardSearchTransportRequest.java b/server/src/main/java/org/elasticsearch/search/internal/ShardSearchTransportRequest.java index 9aae2df27779f..07557d9459973 100644 --- a/server/src/main/java/org/elasticsearch/search/internal/ShardSearchTransportRequest.java +++ b/server/src/main/java/org/elasticsearch/search/internal/ShardSearchTransportRequest.java @@ -93,11 +93,6 @@ public ShardId shardId() { return shardSearchLocalRequest.shardId(); } - @Override - public String[] types() { - return shardSearchLocalRequest.types(); - } - @Override public SearchSourceBuilder source() { return shardSearchLocalRequest.source(); diff --git a/server/src/main/java/org/elasticsearch/search/lookup/DocLookup.java b/server/src/main/java/org/elasticsearch/search/lookup/DocLookup.java index 9cfe121fe7ebd..0022cdfdc9d99 100644 --- a/server/src/main/java/org/elasticsearch/search/lookup/DocLookup.java +++ b/server/src/main/java/org/elasticsearch/search/lookup/DocLookup.java @@ -19,7 +19,6 @@ package org.elasticsearch.search.lookup; import org.apache.lucene.index.LeafReaderContext; -import org.elasticsearch.common.Nullable; import org.elasticsearch.index.fielddata.IndexFieldData; import org.elasticsearch.index.mapper.MappedFieldType; import org.elasticsearch.index.mapper.MapperService; @@ -31,13 +30,9 @@ public class DocLookup { private final MapperService mapperService; private final Function> fieldDataLookup; - @Nullable - private final String[] types; - - DocLookup(MapperService mapperService, Function> fieldDataLookup, @Nullable String[] types) { + DocLookup(MapperService mapperService, Function> fieldDataLookup) { this.mapperService = mapperService; this.fieldDataLookup = fieldDataLookup; - this.types = types; } public MapperService mapperService() { @@ -49,10 +44,6 @@ public IndexFieldData getForField(MappedFieldType fieldType) { } public LeafDocLookup getLeafDocLookup(LeafReaderContext context) { - return new LeafDocLookup(mapperService, fieldDataLookup, types, context); - } - - public String[] getTypes() { - return types; + return new LeafDocLookup(mapperService, fieldDataLookup, context); } } diff --git a/server/src/main/java/org/elasticsearch/search/lookup/FieldsLookup.java b/server/src/main/java/org/elasticsearch/search/lookup/FieldsLookup.java index feefb1fcb30e4..c089501e38596 100644 --- a/server/src/main/java/org/elasticsearch/search/lookup/FieldsLookup.java +++ b/server/src/main/java/org/elasticsearch/search/lookup/FieldsLookup.java @@ -19,22 +19,18 @@ package org.elasticsearch.search.lookup; import org.apache.lucene.index.LeafReaderContext; -import org.elasticsearch.common.Nullable; import org.elasticsearch.index.mapper.MapperService; public class FieldsLookup { private final MapperService mapperService; - @Nullable - private final String[] types; - FieldsLookup(MapperService mapperService, @Nullable String[] types) { + FieldsLookup(MapperService mapperService) { this.mapperService = mapperService; - this.types = types; } public LeafFieldsLookup getLeafFieldsLookup(LeafReaderContext context) { - return new LeafFieldsLookup(mapperService, types, context.reader()); + return new LeafFieldsLookup(mapperService, context.reader()); } } diff --git a/server/src/main/java/org/elasticsearch/search/lookup/LeafDocLookup.java b/server/src/main/java/org/elasticsearch/search/lookup/LeafDocLookup.java index 04522834579e4..4fd6c78e47513 100644 --- a/server/src/main/java/org/elasticsearch/search/lookup/LeafDocLookup.java +++ b/server/src/main/java/org/elasticsearch/search/lookup/LeafDocLookup.java @@ -21,7 +21,6 @@ import org.apache.logging.log4j.LogManager; import org.apache.lucene.index.LeafReaderContext; import org.elasticsearch.ExceptionsHelper; -import org.elasticsearch.common.Nullable; import org.elasticsearch.common.logging.DeprecationLogger; import org.elasticsearch.index.fielddata.IndexFieldData; import org.elasticsearch.index.fielddata.ScriptDocValues; @@ -31,7 +30,6 @@ import java.io.IOException; import java.security.AccessController; import java.security.PrivilegedAction; -import java.util.Arrays; import java.util.Collection; import java.util.HashMap; import java.util.Map; @@ -51,18 +49,14 @@ public class LeafDocLookup implements Map> { private final MapperService mapperService; private final Function> fieldDataLookup; - @Nullable - private final String[] types; - private final LeafReaderContext reader; private int docId = -1; - LeafDocLookup(MapperService mapperService, Function> fieldDataLookup, @Nullable String[] types, + LeafDocLookup(MapperService mapperService, Function> fieldDataLookup, LeafReaderContext reader) { this.mapperService = mapperService; this.fieldDataLookup = fieldDataLookup; - this.types = types; this.reader = reader; } @@ -90,8 +84,7 @@ public ScriptDocValues get(Object key) { if (scriptValues == null) { final MappedFieldType fieldType = mapperService.fullName(fieldName); if (fieldType == null) { - throw new IllegalArgumentException("No field found for [" + fieldName + "] in mapping with types " + - Arrays.toString(types)); + throw new IllegalArgumentException("No field found for [" + fieldName + "] in mapping"); } // load fielddata on behalf of the script: otherwise it would need additional permissions // to deal with pagedbytes/ramusagestimator/etc diff --git a/server/src/main/java/org/elasticsearch/search/lookup/LeafFieldsLookup.java b/server/src/main/java/org/elasticsearch/search/lookup/LeafFieldsLookup.java index d98a8585ecf6a..f614ce400ef36 100644 --- a/server/src/main/java/org/elasticsearch/search/lookup/LeafFieldsLookup.java +++ b/server/src/main/java/org/elasticsearch/search/lookup/LeafFieldsLookup.java @@ -20,7 +20,6 @@ import org.apache.lucene.index.LeafReader; import org.elasticsearch.ElasticsearchParseException; -import org.elasticsearch.common.Nullable; import org.elasticsearch.index.fieldvisitor.SingleFieldsVisitor; import org.elasticsearch.index.mapper.DocumentMapper; import org.elasticsearch.index.mapper.MappedFieldType; @@ -29,7 +28,6 @@ import java.io.IOException; import java.util.ArrayList; -import java.util.Arrays; import java.util.Collection; import java.util.HashMap; import java.util.List; @@ -42,18 +40,14 @@ public class LeafFieldsLookup implements Map { private final MapperService mapperService; - @Nullable - private final String[] types; - private final LeafReader reader; private int docId = -1; private final Map cachedFieldData = new HashMap<>(); - LeafFieldsLookup(MapperService mapperService, @Nullable String[] types, LeafReader reader) { + LeafFieldsLookup(MapperService mapperService, LeafReader reader) { this.mapperService = mapperService; - this.types = types; this.reader = reader; } @@ -136,7 +130,7 @@ private FieldLookup loadFieldData(String name) { if (data == null) { MappedFieldType fieldType = mapperService.fullName(name); if (fieldType == null) { - throw new IllegalArgumentException("No field found for [" + name + "] in mapping with types " + Arrays.toString(types)); + throw new IllegalArgumentException("No field found for [" + name + "] in mapping"); } data = new FieldLookup(fieldType); cachedFieldData.put(name, data); diff --git a/server/src/main/java/org/elasticsearch/search/lookup/SearchLookup.java b/server/src/main/java/org/elasticsearch/search/lookup/SearchLookup.java index 8f4b5143dc6cd..04aef7d2e8f63 100644 --- a/server/src/main/java/org/elasticsearch/search/lookup/SearchLookup.java +++ b/server/src/main/java/org/elasticsearch/search/lookup/SearchLookup.java @@ -20,7 +20,6 @@ package org.elasticsearch.search.lookup; import org.apache.lucene.index.LeafReaderContext; -import org.elasticsearch.common.Nullable; import org.elasticsearch.index.fielddata.IndexFieldData; import org.elasticsearch.index.mapper.MappedFieldType; import org.elasticsearch.index.mapper.MapperService; @@ -35,11 +34,10 @@ public class SearchLookup { final FieldsLookup fieldsLookup; - public SearchLookup(MapperService mapperService, Function> fieldDataLookup, - @Nullable String[] types) { - docMap = new DocLookup(mapperService, fieldDataLookup, types); + public SearchLookup(MapperService mapperService, Function> fieldDataLookup) { + docMap = new DocLookup(mapperService, fieldDataLookup); sourceLookup = new SourceLookup(); - fieldsLookup = new FieldsLookup(mapperService, types); + fieldsLookup = new FieldsLookup(mapperService); } public LeafSearchLookup getLeafSearchLookup(LeafReaderContext context) { diff --git a/server/src/main/java/org/elasticsearch/search/slice/SliceBuilder.java b/server/src/main/java/org/elasticsearch/search/slice/SliceBuilder.java index 3c86b21a0873d..01924e938dcd0 100644 --- a/server/src/main/java/org/elasticsearch/search/slice/SliceBuilder.java +++ b/server/src/main/java/org/elasticsearch/search/slice/SliceBuilder.java @@ -217,19 +217,15 @@ public Query toFilter(ClusterService clusterService, ShardSearchRequest request, int shardId = request.shardId().id(); int numShards = context.getIndexSettings().getNumberOfShards(); - if (minNodeVersion.onOrAfter(Version.V_6_4_0) && - (request.preference() != null || request.indexRoutings().length > 0)) { + if (request.preference() != null || request.indexRoutings().length > 0) { GroupShardsIterator group = buildShardIterator(clusterService, request); assert group.size() <= numShards : "index routing shards: " + group.size() + " cannot be greater than total number of shards: " + numShards; if (group.size() < numShards) { - /** + /* * The routing of this request targets a subset of the shards of this index so we need to we retrieve * the original {@link GroupShardsIterator} and compute the request shard id and number of * shards from it. - * This behavior has been added in {@link Version#V_6_4_0} so if there is another node in the cluster - * with an older version we use the original shard id and number of shards in order to ensure that all - * slices use the same numbers. */ numShards = group.size(); int ord = 0; diff --git a/server/src/main/java/org/elasticsearch/search/slice/TermsSliceQuery.java b/server/src/main/java/org/elasticsearch/search/slice/TermsSliceQuery.java index 52ac72d01b778..1ec3479424193 100644 --- a/server/src/main/java/org/elasticsearch/search/slice/TermsSliceQuery.java +++ b/server/src/main/java/org/elasticsearch/search/slice/TermsSliceQuery.java @@ -78,6 +78,9 @@ public boolean isCacheable(LeafReaderContext ctx) { private DocIdSet build(LeafReader reader) throws IOException { final DocIdSetBuilder builder = new DocIdSetBuilder(reader.maxDoc()); final Terms terms = reader.terms(getField()); + if (terms == null) { + return DocIdSet.EMPTY; + } final TermsEnum te = terms.iterator(); PostingsEnum docsEnum = null; for (BytesRef term = te.next(); term != null; term = te.next()) { diff --git a/server/src/main/java/org/elasticsearch/search/sort/FieldSortBuilder.java b/server/src/main/java/org/elasticsearch/search/sort/FieldSortBuilder.java index 8abd4b9f40d5c..949a5a3ff441c 100644 --- a/server/src/main/java/org/elasticsearch/search/sort/FieldSortBuilder.java +++ b/server/src/main/java/org/elasticsearch/search/sort/FieldSortBuilder.java @@ -19,13 +19,11 @@ package org.elasticsearch.search.sort; -import org.apache.logging.log4j.LogManager; import org.apache.lucene.search.SortField; import org.elasticsearch.Version; import org.elasticsearch.common.ParseField; import org.elasticsearch.common.io.stream.StreamInput; import org.elasticsearch.common.io.stream.StreamOutput; -import org.elasticsearch.common.logging.DeprecationLogger; import org.elasticsearch.common.xcontent.ObjectParser; import org.elasticsearch.common.xcontent.ObjectParser.ValueType; import org.elasticsearch.common.xcontent.XContentBuilder; @@ -36,6 +34,7 @@ import org.elasticsearch.index.fielddata.IndexNumericFieldData.NumericType; import org.elasticsearch.index.fielddata.plain.SortedNumericDVIndexFieldData; import org.elasticsearch.index.mapper.MappedFieldType; +import org.elasticsearch.index.mapper.ObjectMapper; import org.elasticsearch.index.query.QueryBuilder; import org.elasticsearch.index.query.QueryRewriteContext; import org.elasticsearch.index.query.QueryShardContext; @@ -47,13 +46,13 @@ import java.util.Locale; import java.util.Objects; +import static org.elasticsearch.index.search.NestedHelper.parentObject; import static org.elasticsearch.search.sort.NestedSortBuilder.NESTED_FIELD; /** * A sort builder to sort based on a document field. */ public class FieldSortBuilder extends SortBuilder { - private static final DeprecationLogger deprecationLogger = new DeprecationLogger(LogManager.getLogger(FieldSortBuilder.class)); public static final String NAME = "field_sort"; public static final ParseField MISSING = new ParseField("missing"); @@ -80,10 +79,6 @@ public class FieldSortBuilder extends SortBuilder { private SortMode sortMode; - private QueryBuilder nestedFilter; - - private String nestedPath; - private NestedSortBuilder nestedSort; /** Copy constructor. */ @@ -95,8 +90,6 @@ public FieldSortBuilder(FieldSortBuilder template) { if (template.sortMode != null) { this.sortMode(template.sortMode()); } - this.setNestedFilter(template.getNestedFilter()); - this.setNestedPath(template.getNestedPath()); if (template.getNestedSort() != null) { this.setNestedSort(template.getNestedSort()); } @@ -121,8 +114,12 @@ public FieldSortBuilder(String fieldName) { */ public FieldSortBuilder(StreamInput in) throws IOException { fieldName = in.readString(); - nestedFilter = in.readOptionalNamedWriteable(QueryBuilder.class); - nestedPath = in.readOptionalString(); + if (in.getVersion().before(Version.V_8_0_0)) { + if (in.readOptionalNamedWriteable(QueryBuilder.class) != null || in.readOptionalString() != null) { + throw new IOException("the [sort] options [nested_path] and [nested_filter] are removed in 8.x, " + + "please use [nested] instead"); + } + } missing = in.readGenericValue(); order = in.readOptionalWriteable(SortOrder::readFromStream); sortMode = in.readOptionalWriteable(SortMode::readFromStream); @@ -136,8 +133,10 @@ public FieldSortBuilder(StreamInput in) throws IOException { @Override public void writeTo(StreamOutput out) throws IOException { out.writeString(fieldName); - out.writeOptionalNamedWriteable(nestedFilter); - out.writeOptionalString(nestedPath); + if (out.getVersion().before(Version.V_8_0_0)) { + out.writeOptionalNamedWriteable(null); + out.writeOptionalString(null); + } out.writeGenericValue(missing); out.writeOptionalWriteable(order); out.writeOptionalWriteable(sortMode); @@ -210,58 +209,6 @@ public SortMode sortMode() { return this.sortMode; } - /** - * Sets the nested filter that the nested objects should match with in order - * to be taken into account for sorting. - * - * @deprecated set nested sort with {@link #setNestedSort(NestedSortBuilder)} and retrieve with {@link #getNestedSort()} - */ - @Deprecated - public FieldSortBuilder setNestedFilter(QueryBuilder nestedFilter) { - if (this.nestedSort != null) { - throw new IllegalArgumentException("Setting both nested_path/nested_filter and nested not allowed"); - } - this.nestedFilter = nestedFilter; - return this; - } - - /** - * Returns the nested filter that the nested objects should match with in - * order to be taken into account for sorting. - * - * @deprecated set nested sort with {@link #setNestedSort(NestedSortBuilder)} and retrieve with {@link #getNestedSort()} - */ - @Deprecated - public QueryBuilder getNestedFilter() { - return this.nestedFilter; - } - - /** - * Sets the nested path if sorting occurs on a field that is inside a nested - * object. By default when sorting on a field inside a nested object, the - * nearest upper nested object is selected as nested path. - * - * @deprecated set nested sort with {@link #setNestedSort(NestedSortBuilder)} and retrieve with {@link #getNestedSort()} - */ - @Deprecated - public FieldSortBuilder setNestedPath(String nestedPath) { - if (this.nestedSort != null) { - throw new IllegalArgumentException("Setting both nested_path/nested_filter and nested not allowed"); - } - this.nestedPath = nestedPath; - return this; - } - - /** - * Returns the nested path if sorting occurs in a field that is inside a - * nested object. - * @deprecated set nested sort with {@link #setNestedSort(NestedSortBuilder)} and retrieve with {@link #getNestedSort()} - */ - @Deprecated - public String getNestedPath() { - return this.nestedPath; - } - /** * Returns the {@link NestedSortBuilder} */ @@ -276,9 +223,6 @@ public NestedSortBuilder getNestedSort() { * order to be taken into account for sorting. */ public FieldSortBuilder setNestedSort(final NestedSortBuilder nestedSort) { - if (this.nestedFilter != null || this.nestedPath != null) { - throw new IllegalArgumentException("Setting both nested_path/nested_filter and nested not allowed"); - } this.nestedSort = nestedSort; return this; } @@ -330,12 +274,6 @@ public XContentBuilder toXContent(XContentBuilder builder, Params params) throws if (sortMode != null) { builder.field(SORT_MODE.getPreferredName(), sortMode); } - if (nestedFilter != null) { - builder.field(NESTED_FILTER_FIELD.getPreferredName(), nestedFilter, params); - } - if (nestedPath != null) { - builder.field(NESTED_PATH_FIELD.getPreferredName(), nestedPath); - } if (nestedSort != null) { builder.field(NESTED_FIELD.getPreferredName(), nestedSort); } @@ -367,69 +305,85 @@ private static NumericType resolveNumericType(String value) { @Override public SortFieldAndFormat build(QueryShardContext context) throws IOException { if (DOC_FIELD_NAME.equals(fieldName)) { - if (order == SortOrder.DESC) { - return SORT_DOC_REVERSE; + return order == SortOrder.DESC ? SORT_DOC_REVERSE : SORT_DOC; + } + + boolean isUnmapped = false; + MappedFieldType fieldType = context.fieldMapper(fieldName); + if (fieldType == null) { + isUnmapped = true; + if (unmappedType != null) { + fieldType = context.getMapperService().unmappedFieldType(unmappedType); } else { - return SORT_DOC; - } - } else { - boolean isUnmapped = false; - MappedFieldType fieldType = context.fieldMapper(fieldName); - if (fieldType == null) { - isUnmapped = true; - if (unmappedType != null) { - fieldType = context.getMapperService().unmappedFieldType(unmappedType); - } else { - throw new QueryShardException(context, "No mapping found for [" + fieldName + "] in order to sort on"); - } + throw new QueryShardException(context, "No mapping found for [" + fieldName + "] in order to sort on"); } + } - MultiValueMode localSortMode = null; - if (sortMode != null) { - localSortMode = MultiValueMode.fromString(sortMode.toString()); - } + MultiValueMode localSortMode = null; + if (sortMode != null) { + localSortMode = MultiValueMode.fromString(sortMode.toString()); + } - boolean reverse = (order == SortOrder.DESC); - if (localSortMode == null) { - localSortMode = reverse ? MultiValueMode.MAX : MultiValueMode.MIN; - } + boolean reverse = (order == SortOrder.DESC); + if (localSortMode == null) { + localSortMode = reverse ? MultiValueMode.MAX : MultiValueMode.MIN; + } - Nested nested = null; - if (isUnmapped == false) { - if (nestedSort != null) { - if (context.indexVersionCreated().before(Version.V_6_5_0) && nestedSort.getMaxChildren() != Integer.MAX_VALUE) { - throw new QueryShardException(context, - "max_children is only supported on v6.5.0 or higher"); - } - if (nestedSort.getNestedSort() != null && nestedSort.getMaxChildren() != Integer.MAX_VALUE) { - throw new QueryShardException(context, - "max_children is only supported on last level of nested sort"); - } - // new nested sorts takes priority - nested = resolveNested(context, nestedSort); - } else { - nested = resolveNested(context, nestedPath, nestedFilter); + Nested nested = null; + if (isUnmapped == false) { + if (nestedSort != null) { + if (nestedSort.getNestedSort() != null && nestedSort.getMaxChildren() != Integer.MAX_VALUE) { + throw new QueryShardException(context, + "max_children is only supported on last level of nested sort"); } + nested = resolveNested(context, nestedSort); + } else { + validateMissingNestedPath(context, fieldName); } + } - IndexFieldData fieldData = context.getForField(fieldType); - if (fieldData instanceof IndexNumericFieldData == false - && (sortMode == SortMode.SUM || sortMode == SortMode.AVG || sortMode == SortMode.MEDIAN)) { - throw new QueryShardException(context, "we only support AVG, MEDIAN and SUM on number based fields"); + IndexFieldData fieldData = context.getForField(fieldType); + if (fieldData instanceof IndexNumericFieldData == false + && (sortMode == SortMode.SUM || sortMode == SortMode.AVG || sortMode == SortMode.MEDIAN)) { + throw new QueryShardException(context, "we only support AVG, MEDIAN and SUM on number based fields"); + } + final SortField field; + if (numericType != null) { + if (fieldData instanceof IndexNumericFieldData == false) { + throw new QueryShardException(context, + "[numeric_type] option cannot be set on a non-numeric field, got " + fieldType.typeName()); } - final SortField field; - if (numericType != null) { - if (fieldData instanceof IndexNumericFieldData == false) { + SortedNumericDVIndexFieldData numericFieldData = (SortedNumericDVIndexFieldData) fieldData; + NumericType resolvedType = resolveNumericType(numericType); + field = numericFieldData.sortField(resolvedType, missing, localSortMode, nested, reverse); + } else { + field = fieldData.sortField(missing, localSortMode, nested, reverse); + } + return new SortFieldAndFormat(field, fieldType.docValueFormat(null, null)); + } + + /** + * Throws an exception if the provided field requires a nested context. + */ + static void validateMissingNestedPath(QueryShardContext context, String field) { + ObjectMapper contextMapper = context.nestedScope().getObjectMapper(); + if (contextMapper != null && contextMapper.nested().isNested() == false) { + // already in nested context + return; + } + for (String parent = parentObject(field); parent != null; parent = parentObject(parent)) { + ObjectMapper parentMapper = context.getObjectMapper(parent); + if (parentMapper != null && parentMapper.nested().isNested()) { + if (contextMapper != null && contextMapper.fullPath().equals(parentMapper.fullPath())) { + // we are in a nested context that matches the path of the provided field so the nested path + // is not required + return ; + } + if (parentMapper.nested().isIncludeInRoot() == false) { throw new QueryShardException(context, - "[numeric_type] option cannot be set on a non-numeric field, got " + fieldType.typeName()); + "it is mandatory to set the [nested] context on the nested sort field: [" + field + "]."); } - SortedNumericDVIndexFieldData numericFieldData = (SortedNumericDVIndexFieldData) fieldData; - NumericType resolvedType = resolveNumericType(numericType); - field = numericFieldData.sortField(resolvedType, missing, localSortMode, nested, reverse); - } else { - field = fieldData.sortField(missing, localSortMode, nested, reverse); } - return new SortFieldAndFormat(field, fieldType.docValueFormat(null, null)); } } @@ -444,8 +398,7 @@ public boolean equals(Object other) { } FieldSortBuilder builder = (FieldSortBuilder) other; - return (Objects.equals(this.fieldName, builder.fieldName) && Objects.equals(this.nestedFilter, builder.nestedFilter) - && Objects.equals(this.nestedPath, builder.nestedPath) && Objects.equals(this.missing, builder.missing) + return (Objects.equals(this.fieldName, builder.fieldName) && Objects.equals(this.missing, builder.missing) && Objects.equals(this.order, builder.order) && Objects.equals(this.sortMode, builder.sortMode) && Objects.equals(this.unmappedType, builder.unmappedType) && Objects.equals(this.nestedSort, builder.nestedSort)) && Objects.equals(this.numericType, builder.numericType); @@ -453,7 +406,7 @@ public boolean equals(Object other) { @Override public int hashCode() { - return Objects.hash(this.fieldName, this.nestedFilter, this.nestedPath, this.nestedSort, this.missing, this.order, this.sortMode, + return Objects.hash(this.fieldName, this.nestedSort, this.missing, this.order, this.sortMode, this.unmappedType, this.numericType); } @@ -479,38 +432,22 @@ public static FieldSortBuilder fromXContent(XContentParser parser, String fieldN static { PARSER.declareField(FieldSortBuilder::missing, p -> p.objectText(), MISSING, ValueType.VALUE); - PARSER.declareString((fieldSortBuilder, nestedPath) -> { - deprecationLogger.deprecated("[nested_path] has been deprecated in favor of the [nested] parameter"); - fieldSortBuilder.setNestedPath(nestedPath); - }, NESTED_PATH_FIELD); PARSER.declareString(FieldSortBuilder::unmappedType , UNMAPPED_TYPE); PARSER.declareString((b, v) -> b.order(SortOrder.fromString(v)) , ORDER_FIELD); PARSER.declareString((b, v) -> b.sortMode(SortMode.fromString(v)), SORT_MODE); - PARSER.declareObject(FieldSortBuilder::setNestedFilter, (p, c) -> { - deprecationLogger.deprecated("[nested_filter] has been deprecated in favour for the [nested] parameter"); - return SortBuilder.parseNestedFilter(p); - }, NESTED_FILTER_FIELD); PARSER.declareObject(FieldSortBuilder::setNestedSort, (p, c) -> NestedSortBuilder.fromXContent(p), NESTED_FIELD); PARSER.declareString((b, v) -> b.setNumericType(v), NUMERIC_TYPE); } @Override public FieldSortBuilder rewrite(QueryRewriteContext ctx) throws IOException { - if (nestedFilter == null && nestedSort == null) { + if (nestedSort == null) { return this; } - if (nestedFilter != null) { - QueryBuilder rewrite = nestedFilter.rewrite(ctx); - if (nestedFilter == rewrite) { - return this; - } - return new FieldSortBuilder(this).setNestedFilter(rewrite); - } else { - NestedSortBuilder rewrite = nestedSort.rewrite(ctx); - if (nestedSort == rewrite) { - return this; - } - return new FieldSortBuilder(this).setNestedSort(rewrite); + NestedSortBuilder rewrite = nestedSort.rewrite(ctx); + if (nestedSort == rewrite) { + return this; } + return new FieldSortBuilder(this).setNestedSort(rewrite); } } diff --git a/server/src/main/java/org/elasticsearch/search/sort/GeoDistanceSortBuilder.java b/server/src/main/java/org/elasticsearch/search/sort/GeoDistanceSortBuilder.java index 116a54a71103b..630b93b4f34b5 100644 --- a/server/src/main/java/org/elasticsearch/search/sort/GeoDistanceSortBuilder.java +++ b/server/src/main/java/org/elasticsearch/search/sort/GeoDistanceSortBuilder.java @@ -19,7 +19,6 @@ package org.elasticsearch.search.sort; -import org.apache.logging.log4j.LogManager; import org.apache.lucene.document.LatLonDocValuesField; import org.apache.lucene.index.LeafReaderContext; import org.apache.lucene.index.NumericDocValues; @@ -36,7 +35,6 @@ import org.elasticsearch.common.geo.GeoUtils; import org.elasticsearch.common.io.stream.StreamInput; import org.elasticsearch.common.io.stream.StreamOutput; -import org.elasticsearch.common.logging.DeprecationLogger; import org.elasticsearch.common.unit.DistanceUnit; import org.elasticsearch.common.xcontent.XContentBuilder; import org.elasticsearch.common.xcontent.XContentParser; @@ -65,14 +63,13 @@ import java.util.Locale; import java.util.Objects; -import static org.elasticsearch.index.query.AbstractQueryBuilder.parseInnerQueryBuilder; +import static org.elasticsearch.search.sort.FieldSortBuilder.validateMissingNestedPath; import static org.elasticsearch.search.sort.NestedSortBuilder.NESTED_FIELD; /** * A geo distance based sorting on a geo point like field. */ public class GeoDistanceSortBuilder extends SortBuilder { - private static final DeprecationLogger deprecationLogger = new DeprecationLogger(LogManager.getLogger(GeoDistanceSortBuilder.class)); public static final String NAME = "_geo_distance"; public static final String ALTERNATIVE_NAME = "_geoDistance"; @@ -91,8 +88,6 @@ public class GeoDistanceSortBuilder extends SortBuilder private DistanceUnit unit = DistanceUnit.DEFAULT; private SortMode sortMode = null; - private QueryBuilder nestedFilter; - private String nestedPath; private NestedSortBuilder nestedSort; @@ -151,8 +146,6 @@ public GeoDistanceSortBuilder(String fieldName, String ... geohashes) { this.unit = original.unit; this.order = original.order; this.sortMode = original.sortMode; - this.nestedFilter = original.nestedFilter; - this.nestedPath = original.nestedPath; this.validation = original.validation; this.nestedSort = original.nestedSort; this.ignoreUnmapped = original.ignoreUnmapped; @@ -169,8 +162,13 @@ public GeoDistanceSortBuilder(StreamInput in) throws IOException { unit = DistanceUnit.readFromStream(in); order = SortOrder.readFromStream(in); sortMode = in.readOptionalWriteable(SortMode::readFromStream); - nestedFilter = in.readOptionalNamedWriteable(QueryBuilder.class); - nestedPath = in.readOptionalString(); + if (in.getVersion().before(Version.V_8_0_0)) { + if (in.readOptionalNamedWriteable(QueryBuilder.class) != null || in.readOptionalString() != null) { + throw new IOException("the [sort] options [nested_path] and [nested_filter] are removed in 8.x, " + + "please use [nested] instead"); + } + + } nestedSort = in.readOptionalWriteable(NestedSortBuilder::new); validation = GeoValidationMethod.readFromStream(in); ignoreUnmapped = in.readBoolean(); @@ -184,8 +182,10 @@ public void writeTo(StreamOutput out) throws IOException { unit.writeTo(out); order.writeTo(out); out.writeOptionalWriteable(sortMode); - out.writeOptionalNamedWriteable(nestedFilter); - out.writeOptionalString(nestedPath); + if (out.getVersion().before(Version.V_8_0_0)) { + out.writeOptionalNamedWriteable(null); + out.writeOptionalString(null); + } out.writeOptionalWriteable(nestedSort); validation.writeTo(out); out.writeBoolean(ignoreUnmapped); @@ -289,59 +289,6 @@ public SortMode sortMode() { return this.sortMode; } - /** - * Sets the nested filter that the nested objects should match with in order to - * be taken into account for sorting. - * - * @deprecated set nested sort with {@link #setNestedSort(NestedSortBuilder)} - * and retrieve with {@link #getNestedSort()} - **/ - @Deprecated - public GeoDistanceSortBuilder setNestedFilter(QueryBuilder nestedFilter) { - if (this.nestedSort != null) { - throw new IllegalArgumentException("Setting both nested_path/nested_filter and nested not allowed"); - } - this.nestedFilter = nestedFilter; - return this; - } - - /** - * Returns the nested filter that the nested objects should match with in order to be taken into account - * for sorting. - * @deprecated set nested sort with {@link #setNestedSort(NestedSortBuilder)} - * and retrieve with {@link #getNestedSort()} - **/ - @Deprecated - public QueryBuilder getNestedFilter() { - return this.nestedFilter; - } - - /** - * Sets the nested path if sorting occurs on a field that is inside a nested object. By default when sorting on a - * field inside a nested object, the nearest upper nested object is selected as nested path. - * @deprecated set nested sort with {@link #setNestedSort(NestedSortBuilder)} - * and retrieve with {@link #getNestedSort()} - **/ - @Deprecated - public GeoDistanceSortBuilder setNestedPath(String nestedPath) { - if (this.nestedSort != null) { - throw new IllegalArgumentException("Setting both nested_path/nested_filter and nested not allowed"); - } - this.nestedPath = nestedPath; - return this; - } - - /** - * Returns the nested path if sorting occurs on a field that is inside a nested object. By default when sorting on a - * field inside a nested object, the nearest upper nested object is selected as nested path. - * @deprecated set nested sort with {@link #setNestedSort(NestedSortBuilder)} - * and retrieve with {@link #getNestedSort()} - **/ - @Deprecated - public String getNestedPath() { - return this.nestedPath; - } - /** * Returns the {@link NestedSortBuilder} */ @@ -356,9 +303,6 @@ public NestedSortBuilder getNestedSort() { * order to be taken into account for sorting. */ public GeoDistanceSortBuilder setNestedSort(final NestedSortBuilder nestedSort) { - if (this.nestedFilter != null || this.nestedPath != null) { - throw new IllegalArgumentException("Setting both nested_path/nested_filter and nested not allowed"); - } this.nestedSort = nestedSort; return this; } @@ -394,12 +338,6 @@ public XContentBuilder toXContent(XContentBuilder builder, Params params) throws builder.field(SORTMODE_FIELD.getPreferredName(), sortMode); } - if (nestedPath != null) { - builder.field(NESTED_PATH_FIELD.getPreferredName(), nestedPath); - } - if (nestedFilter != null) { - builder.field(NESTED_FILTER_FIELD.getPreferredName(), nestedFilter, params); - } if (nestedSort != null) { builder.field(NESTED_FIELD.getPreferredName(), nestedSort); } @@ -433,8 +371,6 @@ public boolean equals(Object object) { Objects.equals(unit, other.unit) && Objects.equals(sortMode, other.sortMode) && Objects.equals(order, other.order) && - Objects.equals(nestedFilter, other.nestedFilter) && - Objects.equals(nestedPath, other.nestedPath) && Objects.equals(validation, other.validation) && Objects.equals(nestedSort, other.nestedSort) && ignoreUnmapped == other.ignoreUnmapped; @@ -443,8 +379,7 @@ public boolean equals(Object object) { @Override public int hashCode() { return Objects.hash(this.fieldName, this.points, this.geoDistance, - this.unit, this.sortMode, this.order, this.nestedFilter, - this.nestedPath, this.validation, this.nestedSort, this.ignoreUnmapped); + this.unit, this.sortMode, this.order, this.validation, this.nestedSort, this.ignoreUnmapped); } /** @@ -464,8 +399,6 @@ public static GeoDistanceSortBuilder fromXContent(XContentParser parser, String GeoDistance geoDistance = GeoDistance.ARC; SortOrder order = SortOrder.ASC; SortMode sortMode = null; - QueryBuilder nestedFilter = null; - String nestedPath = null; NestedSortBuilder nestedSort = null; GeoValidationMethod validation = null; boolean ignoreUnmapped = false; @@ -480,10 +413,7 @@ public static GeoDistanceSortBuilder fromXContent(XContentParser parser, String fieldName = currentName; } else if (token == XContentParser.Token.START_OBJECT) { - if (NESTED_FILTER_FIELD.match(currentName, parser.getDeprecationHandler())) { - deprecationLogger.deprecated("[nested_filter] has been deprecated in favour of the [nested] parameter"); - nestedFilter = parseInnerQueryBuilder(parser); - } else if (NESTED_FIELD.match(currentName, parser.getDeprecationHandler())) { + if (NESTED_FIELD.match(currentName, parser.getDeprecationHandler())) { nestedSort = NestedSortBuilder.fromXContent(parser); } else { // the json in the format of -> field : { lat : 30, lon : 12 } @@ -510,9 +440,6 @@ public static GeoDistanceSortBuilder fromXContent(XContentParser parser, String validation = GeoValidationMethod.fromString(parser.text()); } else if (SORTMODE_FIELD.match(currentName, parser.getDeprecationHandler())) { sortMode = SortMode.fromString(parser.text()); - } else if (NESTED_PATH_FIELD.match(currentName, parser.getDeprecationHandler())) { - deprecationLogger.deprecated("[nested_path] has been deprecated in favour of the [nested] parameter"); - nestedPath = parser.text(); } else if (IGNORE_UNMAPPED.match(currentName, parser.getDeprecationHandler())) { ignoreUnmapped = parser.booleanValue(); } else if (token == Token.VALUE_STRING){ @@ -550,10 +477,6 @@ public static GeoDistanceSortBuilder fromXContent(XContentParser parser, String if (sortMode != null) { result.sortMode(sortMode); } - if (nestedFilter != null) { - result.setNestedFilter(nestedFilter); - } - result.setNestedPath(nestedPath); if (nestedSort != null) { result.setNestedSort(nestedSort); } @@ -611,20 +534,15 @@ public SortFieldAndFormat build(QueryShardContext context) throws IOException { } final IndexGeoPointFieldData geoIndexFieldData = context.getForField(fieldType); - final Nested nested; + Nested nested = null; if (nestedSort != null) { - if (context.indexVersionCreated().before(Version.V_6_5_0) && nestedSort.getMaxChildren() != Integer.MAX_VALUE) { - throw new QueryShardException(context, - "max_children is only supported on v6.5.0 or higher"); - } if (nestedSort.getNestedSort() != null && nestedSort.getMaxChildren() != Integer.MAX_VALUE) { throw new QueryShardException(context, "max_children is only supported on last level of nested sort"); } - // new nested sorts takes priority nested = resolveNested(context, nestedSort); } else { - nested = resolveNested(context, nestedPath, nestedFilter); + validateMissingNestedPath(context, fieldName); } if (geoIndexFieldData.getClass() == LatLonPointDVIndexFieldData.class // only works with 5.x geo_point @@ -703,21 +621,13 @@ static void parseGeoPoints(XContentParser parser, List geoPoints) thro @Override public GeoDistanceSortBuilder rewrite(QueryRewriteContext ctx) throws IOException { - if (nestedFilter == null && nestedSort == null) { + if (nestedSort == null) { return this; } - if (nestedFilter != null) { - QueryBuilder rewrite = nestedFilter.rewrite(ctx); - if (nestedFilter == rewrite) { - return this; - } - return new GeoDistanceSortBuilder(this).setNestedFilter(rewrite); - } else { - NestedSortBuilder rewrite = nestedSort.rewrite(ctx); - if (nestedSort == rewrite) { - return this; - } - return new GeoDistanceSortBuilder(this).setNestedSort(rewrite); + NestedSortBuilder rewrite = nestedSort.rewrite(ctx); + if (nestedSort == rewrite) { + return this; } + return new GeoDistanceSortBuilder(this).setNestedSort(rewrite); } } diff --git a/server/src/main/java/org/elasticsearch/search/sort/ScriptSortBuilder.java b/server/src/main/java/org/elasticsearch/search/sort/ScriptSortBuilder.java index d858f0f7c6ac3..17fed4d9ac19c 100644 --- a/server/src/main/java/org/elasticsearch/search/sort/ScriptSortBuilder.java +++ b/server/src/main/java/org/elasticsearch/search/sort/ScriptSortBuilder.java @@ -19,7 +19,6 @@ package org.elasticsearch.search.sort; -import org.apache.logging.log4j.LogManager; import org.apache.lucene.index.BinaryDocValues; import org.apache.lucene.index.LeafReaderContext; import org.apache.lucene.search.Scorable; @@ -31,7 +30,6 @@ import org.elasticsearch.common.io.stream.StreamInput; import org.elasticsearch.common.io.stream.StreamOutput; import org.elasticsearch.common.io.stream.Writeable; -import org.elasticsearch.common.logging.DeprecationLogger; import org.elasticsearch.common.xcontent.ConstructingObjectParser; import org.elasticsearch.common.xcontent.ObjectParser.ValueType; import org.elasticsearch.common.xcontent.XContentBuilder; @@ -66,7 +64,6 @@ * Script sort builder allows to sort based on a custom script expression. */ public class ScriptSortBuilder extends SortBuilder { - private static final DeprecationLogger deprecationLogger = new DeprecationLogger(LogManager.getLogger(ScriptSortBuilder.class)); public static final String NAME = "_script"; public static final ParseField TYPE_FIELD = new ParseField("type"); @@ -79,10 +76,6 @@ public class ScriptSortBuilder extends SortBuilder { private SortMode sortMode; - private QueryBuilder nestedFilter; - - private String nestedPath; - private NestedSortBuilder nestedSort; /** @@ -106,8 +99,6 @@ public ScriptSortBuilder(Script script, ScriptSortType type) { this.type = original.type; this.order = original.order; this.sortMode = original.sortMode; - this.nestedFilter = original.nestedFilter; - this.nestedPath = original.nestedPath; this.nestedSort = original.nestedSort; } @@ -119,8 +110,12 @@ public ScriptSortBuilder(StreamInput in) throws IOException { type = ScriptSortType.readFromStream(in); order = SortOrder.readFromStream(in); sortMode = in.readOptionalWriteable(SortMode::readFromStream); - nestedPath = in.readOptionalString(); - nestedFilter = in.readOptionalNamedWriteable(QueryBuilder.class); + if (in.getVersion().before(Version.V_8_0_0)) { + if (in.readOptionalNamedWriteable(QueryBuilder.class) != null || in.readOptionalString() != null) { + throw new IOException("the [sort] options [nested_path] and [nested_filter] are removed in 8.x, " + + "please use [nested] instead"); + } + } nestedSort = in.readOptionalWriteable(NestedSortBuilder::new); } @@ -130,8 +125,10 @@ public void writeTo(StreamOutput out) throws IOException { type.writeTo(out); order.writeTo(out); out.writeOptionalWriteable(sortMode); - out.writeOptionalString(nestedPath); - out.writeOptionalNamedWriteable(nestedFilter); + if (out.getVersion().before(Version.V_8_0_0)) { + out.writeOptionalString(null); + out.writeOptionalNamedWriteable(null); + } out.writeOptionalWriteable(nestedSort); } @@ -170,56 +167,6 @@ public SortMode sortMode() { return this.sortMode; } - /** - * Sets the nested filter that the nested objects should match with in order to be taken into account - * for sorting. - * - * @deprecated set nested sort with {@link #setNestedSort(NestedSortBuilder)} and retrieve with {@link #getNestedSort()} - */ - @Deprecated - public ScriptSortBuilder setNestedFilter(QueryBuilder nestedFilter) { - if (this.nestedSort != null) { - throw new IllegalArgumentException("Setting both nested_path/nested_filter and nested not allowed"); - } - this.nestedFilter = nestedFilter; - return this; - } - - /** - * Gets the nested filter. - * - * @deprecated set nested sort with {@link #setNestedSort(NestedSortBuilder)} and retrieve with {@link #getNestedSort()} - */ - @Deprecated - public QueryBuilder getNestedFilter() { - return this.nestedFilter; - } - - /** - * Sets the nested path if sorting occurs on a field that is inside a nested object. For sorting by script this - * needs to be specified. - * - * @deprecated set nested sort with {@link #setNestedSort(NestedSortBuilder)} and retrieve with {@link #getNestedSort()} - */ - @Deprecated - public ScriptSortBuilder setNestedPath(String nestedPath) { - if (this.nestedSort != null) { - throw new IllegalArgumentException("Setting both nested_path/nested_filter and nested not allowed"); - } - this.nestedPath = nestedPath; - return this; - } - - /** - * Gets the nested path. - * - * @deprecated set nested sort with {@link #setNestedSort(NestedSortBuilder)} and retrieve with {@link #getNestedSort()} - */ - @Deprecated - public String getNestedPath() { - return this.nestedPath; - } - /** * Returns the {@link NestedSortBuilder} */ @@ -234,9 +181,6 @@ public NestedSortBuilder getNestedSort() { * order to be taken into account for sorting. */ public ScriptSortBuilder setNestedSort(final NestedSortBuilder nestedSort) { - if (this.nestedFilter != null || this.nestedPath != null) { - throw new IllegalArgumentException("Setting both nested_path/nested_filter and nested not allowed"); - } this.nestedSort = nestedSort; return this; } @@ -251,12 +195,6 @@ public XContentBuilder toXContent(XContentBuilder builder, Params builderParams) if (sortMode != null) { builder.field(SORTMODE_FIELD.getPreferredName(), sortMode); } - if (nestedPath != null) { - builder.field(NESTED_PATH_FIELD.getPreferredName(), nestedPath); - } - if (nestedFilter != null) { - builder.field(NESTED_FILTER_FIELD.getPreferredName(), nestedFilter, builderParams); - } if (nestedSort != null) { builder.field(NESTED_FIELD.getPreferredName(), nestedSort); } @@ -274,14 +212,6 @@ public XContentBuilder toXContent(XContentBuilder builder, Params builderParams) PARSER.declareField(constructorArg(), p -> ScriptSortType.fromString(p.text()), TYPE_FIELD, ValueType.STRING); PARSER.declareString((b, v) -> b.order(SortOrder.fromString(v)), ORDER_FIELD); PARSER.declareString((b, v) -> b.sortMode(SortMode.fromString(v)), SORTMODE_FIELD); - PARSER.declareString((fieldSortBuilder, nestedPath) -> { - deprecationLogger.deprecated("[nested_path] has been deprecated in favor of the [nested] parameter"); - fieldSortBuilder.setNestedPath(nestedPath); - }, NESTED_PATH_FIELD); - PARSER.declareObject(ScriptSortBuilder::setNestedFilter, (p, c) -> { - deprecationLogger.deprecated("[nested_filter] has been deprecated in favour for the [nested] parameter"); - return SortBuilder.parseNestedFilter(p); - }, NESTED_FILTER_FIELD); PARSER.declareObject(ScriptSortBuilder::setNestedSort, (p, c) -> NestedSortBuilder.fromXContent(p), NESTED_FIELD); } @@ -310,20 +240,13 @@ public SortFieldAndFormat build(QueryShardContext context) throws IOException { valueMode = reverse ? MultiValueMode.MAX : MultiValueMode.MIN; } - final Nested nested; + Nested nested = null; if (nestedSort != null) { - if (context.indexVersionCreated().before(Version.V_6_5_0) && nestedSort.getMaxChildren() != Integer.MAX_VALUE) { - throw new QueryShardException(context, - "max_children is only supported on v6.5.0 or higher"); - } if (nestedSort.getNestedSort() != null && nestedSort.getMaxChildren() != Integer.MAX_VALUE) { throw new QueryShardException(context, "max_children is only supported on last level of nested sort"); } - // new nested sorts takes priority nested = resolveNested(context, nestedSort); - } else { - nested = resolveNested(context, nestedPath, nestedFilter); } final IndexFieldData.XFieldComparatorSource fieldComparatorSource; @@ -404,14 +327,12 @@ public boolean equals(Object object) { Objects.equals(type, other.type) && Objects.equals(order, other.order) && Objects.equals(sortMode, other.sortMode) && - Objects.equals(nestedFilter, other.nestedFilter) && - Objects.equals(nestedPath, other.nestedPath) && Objects.equals(nestedSort, other.nestedSort); } @Override public int hashCode() { - return Objects.hash(script, type, order, sortMode, nestedFilter, nestedPath, nestedSort); + return Objects.hash(script, type, order, sortMode, nestedSort); } @Override @@ -457,21 +378,13 @@ public String toString() { @Override public ScriptSortBuilder rewrite(QueryRewriteContext ctx) throws IOException { - if (nestedFilter == null && nestedSort == null) { + if (nestedSort == null) { return this; } - if (nestedFilter != null) { - QueryBuilder rewrite = nestedFilter.rewrite(ctx); - if (nestedFilter == rewrite) { - return this; - } - return new ScriptSortBuilder(this).setNestedFilter(rewrite); - } else { - NestedSortBuilder rewrite = nestedSort.rewrite(ctx); - if (nestedSort == rewrite) { - return this; - } - return new ScriptSortBuilder(this).setNestedSort(rewrite); + NestedSortBuilder rewrite = nestedSort.rewrite(ctx); + if (nestedSort == rewrite) { + return this; } + return new ScriptSortBuilder(this).setNestedSort(rewrite); } } diff --git a/server/src/main/java/org/elasticsearch/search/sort/SortBuilder.java b/server/src/main/java/org/elasticsearch/search/sort/SortBuilder.java index 4d793de18443d..881df666802d7 100644 --- a/server/src/main/java/org/elasticsearch/search/sort/SortBuilder.java +++ b/server/src/main/java/org/elasticsearch/search/sort/SortBuilder.java @@ -46,6 +46,7 @@ import java.util.Optional; import static org.elasticsearch.index.query.AbstractQueryBuilder.parseInnerQueryBuilder; +import static org.elasticsearch.search.sort.NestedSortBuilder.FILTER_FIELD; public abstract class SortBuilder> implements NamedWriteable, ToXContentObject, Rewriteable> { @@ -53,8 +54,6 @@ public abstract class SortBuilder> implements NamedWrit // parse fields common to more than one SortBuilder public static final ParseField ORDER_FIELD = new ParseField("order"); - public static final ParseField NESTED_FILTER_FIELD = new ParseField("nested_filter"); - public static final ParseField NESTED_PATH_FIELD = new ParseField("nested_path"); private static final Map> PARSERS = Map.of( ScriptSortBuilder.NAME, ScriptSortBuilder::fromXContent, @@ -171,12 +170,6 @@ public static Optional buildSort(List> sortBuilde return Optional.empty(); } - protected static Nested resolveNested(QueryShardContext context, String nestedPath, QueryBuilder nestedFilter) throws IOException { - NestedSortBuilder nestedSortBuilder = new NestedSortBuilder(nestedPath); - nestedSortBuilder.setFilter(nestedFilter); - return resolveNested(context, nestedSortBuilder); - } - protected static Nested resolveNested(QueryShardContext context, NestedSortBuilder nestedSort) throws IOException { final Query childQuery = resolveNestedQuery(context, nestedSort, null); if (childQuery == null) { @@ -189,7 +182,7 @@ protected static Nested resolveNested(QueryShardContext context, NestedSortBuild } else { parentQuery = objectMapper.nestedTypeFilter(); } - return new Nested(context.bitsetFilter(parentQuery), childQuery, nestedSort); + return new Nested(context.bitsetFilter(parentQuery), childQuery, nestedSort, context::newCachedSearcher); } private static Query resolveNestedQuery(QueryShardContext context, NestedSortBuilder nestedSort, Query parentQuery) throws IOException { @@ -207,7 +200,7 @@ private static Query resolveNestedQuery(QueryShardContext context, NestedSortBui if (nestedObjectMapper == null) { throw new QueryShardException(context, "[nested] failed to find nested object under path [" + nestedPath + "]"); } - if (!nestedObjectMapper.nested().isNested()) { + if (nestedObjectMapper.nested().isNested() == false) { throw new QueryShardException(context, "[nested] nested object under path [" + nestedPath + "] is not of nested type"); } ObjectMapper objectMapper = context.nestedScope().getObjectMapper(); @@ -256,7 +249,7 @@ protected static QueryBuilder parseNestedFilter(XContentParser parser) { try { return parseInnerQueryBuilder(parser); } catch (Exception e) { - throw new ParsingException(parser.getTokenLocation(), "Expected " + NESTED_FILTER_FIELD.getPreferredName() + " element.", e); + throw new ParsingException(parser.getTokenLocation(), "Expected " + FILTER_FIELD.getPreferredName() + " element.", e); } } diff --git a/server/src/main/java/org/elasticsearch/search/suggest/Suggester.java b/server/src/main/java/org/elasticsearch/search/suggest/Suggester.java index 6c007bf4cb2b1..3146d7a0ffad9 100644 --- a/server/src/main/java/org/elasticsearch/search/suggest/Suggester.java +++ b/server/src/main/java/org/elasticsearch/search/suggest/Suggester.java @@ -29,12 +29,15 @@ public abstract class Suggester> innerExecute(String name, T suggestion, IndexSearcher searcher, CharsRefBuilder spare) throws IOException; + protected abstract Suggest.Suggestion> + emptySuggestion(String name, T suggestion, CharsRefBuilder spare) throws IOException; + public Suggest.Suggestion> execute(String name, T suggestion, IndexSearcher searcher, CharsRefBuilder spare) throws IOException { - // #3469 We want to ignore empty shards + // we only want to output an empty suggestion on empty shards if (searcher.getIndexReader().numDocs() == 0) { - return null; + return emptySuggestion(name, suggestion, spare); } return innerExecute(name, suggestion, searcher, spare); } diff --git a/server/src/main/java/org/elasticsearch/search/suggest/completion/CompletionSuggester.java b/server/src/main/java/org/elasticsearch/search/suggest/completion/CompletionSuggester.java index 60ad277deb2f7..3763fd49629b9 100644 --- a/server/src/main/java/org/elasticsearch/search/suggest/completion/CompletionSuggester.java +++ b/server/src/main/java/org/elasticsearch/search/suggest/completion/CompletionSuggester.java @@ -49,12 +49,7 @@ protected Suggest.Suggestion> emptySuggestion(String name, PhraseSuggestionContext suggestion, + CharsRefBuilder spare) throws IOException { + PhraseSuggestion phraseSuggestion = new PhraseSuggestion(name, suggestion.getSize()); + spare.copyUTF8Bytes(suggestion.getText()); + phraseSuggestion.addTerm(new PhraseSuggestion.Entry(new Text(spare.toString()), 0, spare.length())); + return phraseSuggestion; + } } diff --git a/server/src/main/java/org/elasticsearch/search/suggest/phrase/PhraseSuggestionBuilder.java b/server/src/main/java/org/elasticsearch/search/suggest/phrase/PhraseSuggestionBuilder.java index 74b9437d67821..5b66badc733e0 100644 --- a/server/src/main/java/org/elasticsearch/search/suggest/phrase/PhraseSuggestionBuilder.java +++ b/server/src/main/java/org/elasticsearch/search/suggest/phrase/PhraseSuggestionBuilder.java @@ -31,7 +31,7 @@ import org.elasticsearch.common.xcontent.XContentBuilder; import org.elasticsearch.common.xcontent.XContentParser; import org.elasticsearch.common.xcontent.XContentParser.Token; -import org.elasticsearch.index.analysis.CustomAnalyzer; +import org.elasticsearch.index.analysis.AnalyzerComponentsProvider; import org.elasticsearch.index.analysis.NamedAnalyzer; import org.elasticsearch.index.analysis.ShingleTokenFilterFactory; import org.elasticsearch.index.analysis.TokenFilterFactory; @@ -675,9 +675,8 @@ private static ShingleTokenFilterFactory.Factory getShingleFilterFactory(Analyze if (analyzer instanceof NamedAnalyzer) { analyzer = ((NamedAnalyzer)analyzer).analyzer(); } - if (analyzer instanceof CustomAnalyzer) { - final CustomAnalyzer a = (CustomAnalyzer) analyzer; - final TokenFilterFactory[] tokenFilters = a.tokenFilters(); + if (analyzer instanceof AnalyzerComponentsProvider) { + final TokenFilterFactory[] tokenFilters = ((AnalyzerComponentsProvider) analyzer).getComponents().getTokenFilters(); for (TokenFilterFactory tokenFilterFactory : tokenFilters) { if (tokenFilterFactory instanceof ShingleTokenFilterFactory) { return ((ShingleTokenFilterFactory)tokenFilterFactory).getInnerFactory(); diff --git a/server/src/main/java/org/elasticsearch/search/suggest/term/TermSuggester.java b/server/src/main/java/org/elasticsearch/search/suggest/term/TermSuggester.java index d6edf352d1911..34ce79af87cb5 100644 --- a/server/src/main/java/org/elasticsearch/search/suggest/term/TermSuggester.java +++ b/server/src/main/java/org/elasticsearch/search/suggest/term/TermSuggester.java @@ -94,4 +94,16 @@ private Token(Term term, int startOffset, int endOffset) { } } + + @Override + protected TermSuggestion emptySuggestion(String name, TermSuggestionContext suggestion, CharsRefBuilder spare) throws IOException { + TermSuggestion termSuggestion = new TermSuggestion(name, suggestion.getSize(), suggestion.getDirectSpellCheckerSettings().sort()); + List tokens = queryTerms(suggestion, spare); + for (Token token : tokens) { + Text key = new Text(new BytesArray(token.term.bytes())); + TermSuggestion.Entry resultEntry = new TermSuggestion.Entry(key, token.startOffset, token.endOffset - token.startOffset); + termSuggestion.addTerm(resultEntry); + } + return termSuggestion; + } } diff --git a/server/src/main/java/org/elasticsearch/snapshots/SnapshotInfo.java b/server/src/main/java/org/elasticsearch/snapshots/SnapshotInfo.java index 38aa945bcca47..3c7007bd27f17 100644 --- a/server/src/main/java/org/elasticsearch/snapshots/SnapshotInfo.java +++ b/server/src/main/java/org/elasticsearch/snapshots/SnapshotInfo.java @@ -42,6 +42,7 @@ import java.util.Collections; import java.util.Comparator; import java.util.List; +import java.util.Map; import java.util.Objects; /** @@ -51,6 +52,7 @@ public final class SnapshotInfo implements Comparable, ToXContent, public static final String CONTEXT_MODE_PARAM = "context_mode"; public static final String CONTEXT_MODE_SNAPSHOT = "SNAPSHOT"; + public static final Version METADATA_FIELD_INTRODUCED = Version.V_7_3_0; private static final DateFormatter DATE_TIME_FORMATTER = DateFormatter.forPattern("strictDateOptionalTime"); private static final String SNAPSHOT = "snapshot"; private static final String UUID = "uuid"; @@ -74,6 +76,7 @@ public final class SnapshotInfo implements Comparable, ToXContent, private static final String TOTAL_SHARDS = "total_shards"; private static final String SUCCESSFUL_SHARDS = "successful_shards"; private static final String INCLUDE_GLOBAL_STATE = "include_global_state"; + private static final String USER_METADATA = "metadata"; private static final Comparator COMPARATOR = Comparator.comparing(SnapshotInfo::startTime).thenComparing(SnapshotInfo::snapshotId); @@ -88,6 +91,7 @@ public static final class SnapshotInfoBuilder { private long endTime = 0L; private ShardStatsBuilder shardStatsBuilder = null; private Boolean includeGlobalState = null; + private Map userMetadata = null; private int version = -1; private List shardFailures = null; @@ -127,6 +131,10 @@ private void setIncludeGlobalState(Boolean includeGlobalState) { this.includeGlobalState = includeGlobalState; } + private void setUserMetadata(Map userMetadata) { + this.userMetadata = userMetadata; + } + private void setVersion(int version) { this.version = version; } @@ -153,7 +161,7 @@ public SnapshotInfo build() { } return new SnapshotInfo(snapshotId, indices, snapshotState, reason, version, startTime, endTime, - totalShards, successfulShards, shardFailures, includeGlobalState); + totalShards, successfulShards, shardFailures, includeGlobalState, userMetadata); } } @@ -194,6 +202,7 @@ int getSuccessfulShards() { SNAPSHOT_INFO_PARSER.declareLong(SnapshotInfoBuilder::setEndTime, new ParseField(END_TIME_IN_MILLIS)); SNAPSHOT_INFO_PARSER.declareObject(SnapshotInfoBuilder::setShardStatsBuilder, SHARD_STATS_PARSER, new ParseField(SHARDS)); SNAPSHOT_INFO_PARSER.declareBoolean(SnapshotInfoBuilder::setIncludeGlobalState, new ParseField(INCLUDE_GLOBAL_STATE)); + SNAPSHOT_INFO_PARSER.declareObject(SnapshotInfoBuilder::setUserMetadata, (p, c) -> p.map() , new ParseField(USER_METADATA)); SNAPSHOT_INFO_PARSER.declareInt(SnapshotInfoBuilder::setVersion, new ParseField(VERSION_ID)); SNAPSHOT_INFO_PARSER.declareObjectArray(SnapshotInfoBuilder::setShardFailures, SnapshotShardFailure.SNAPSHOT_SHARD_FAILURE_PARSER, new ParseField(FAILURES)); @@ -223,6 +232,9 @@ int getSuccessfulShards() { @Nullable private final Boolean includeGlobalState; + @Nullable + private final Map userMetadata; + @Nullable private final Version version; @@ -230,28 +242,30 @@ int getSuccessfulShards() { public SnapshotInfo(SnapshotId snapshotId, List indices, SnapshotState state) { this(snapshotId, indices, state, null, null, 0L, 0L, 0, 0, - Collections.emptyList(), null); + Collections.emptyList(), null, null); } public SnapshotInfo(SnapshotId snapshotId, List indices, SnapshotState state, Version version) { this(snapshotId, indices, state, null, version, 0L, 0L, 0, 0, - Collections.emptyList(), null); + Collections.emptyList(), null, null); } - public SnapshotInfo(SnapshotId snapshotId, List indices, long startTime, Boolean includeGlobalState) { + public SnapshotInfo(SnapshotId snapshotId, List indices, long startTime, Boolean includeGlobalState, + Map userMetadata) { this(snapshotId, indices, SnapshotState.IN_PROGRESS, null, Version.CURRENT, startTime, 0L, - 0, 0, Collections.emptyList(), includeGlobalState); + 0, 0, Collections.emptyList(), includeGlobalState, userMetadata); } public SnapshotInfo(SnapshotId snapshotId, List indices, long startTime, String reason, long endTime, - int totalShards, List shardFailures, Boolean includeGlobalState) { + int totalShards, List shardFailures, Boolean includeGlobalState, + Map userMetadata) { this(snapshotId, indices, snapshotState(reason, shardFailures), reason, Version.CURRENT, - startTime, endTime, totalShards, totalShards - shardFailures.size(), shardFailures, includeGlobalState); + startTime, endTime, totalShards, totalShards - shardFailures.size(), shardFailures, includeGlobalState, userMetadata); } private SnapshotInfo(SnapshotId snapshotId, List indices, SnapshotState state, String reason, Version version, long startTime, long endTime, int totalShards, int successfulShards, List shardFailures, - Boolean includeGlobalState) { + Boolean includeGlobalState, Map userMetadata) { this.snapshotId = Objects.requireNonNull(snapshotId); this.indices = Collections.unmodifiableList(Objects.requireNonNull(indices)); this.state = state; @@ -263,6 +277,7 @@ private SnapshotInfo(SnapshotId snapshotId, List indices, SnapshotState this.successfulShards = successfulShards; this.shardFailures = Objects.requireNonNull(shardFailures); this.includeGlobalState = includeGlobalState; + this.userMetadata = userMetadata; } /** @@ -294,6 +309,11 @@ public SnapshotInfo(final StreamInput in) throws IOException { } version = in.readBoolean() ? Version.readVersion(in) : null; includeGlobalState = in.readOptionalBoolean(); + if (in.getVersion().onOrAfter(METADATA_FIELD_INTRODUCED)) { + userMetadata = in.readMap(); + } else { + userMetadata = null; + } } /** @@ -304,7 +324,7 @@ public static SnapshotInfo incompatible(SnapshotId snapshotId) { return new SnapshotInfo(snapshotId, Collections.emptyList(), SnapshotState.INCOMPATIBLE, "the snapshot is incompatible with the current version of Elasticsearch and its exact version is unknown", null, 0L, 0L, 0, 0, - Collections.emptyList(), null); + Collections.emptyList(), null, null); } /** @@ -428,6 +448,15 @@ public Version version() { return version; } + /** + * Returns the custom metadata that was attached to this snapshot at creation time. + * @return custom metadata + */ + @Nullable + public Map userMetadata() { + return userMetadata; + } + /** * Compares two snapshots by their start time; if the start times are the same, then * compares the two snapshots by their snapshot ids. @@ -492,6 +521,9 @@ public XContentBuilder toXContent(final XContentBuilder builder, final Params pa if (includeGlobalState != null) { builder.field(INCLUDE_GLOBAL_STATE, includeGlobalState); } + if (userMetadata != null) { + builder.field(USER_METADATA, userMetadata); + } if (verbose || state != null) { builder.field(STATE, state); } @@ -543,6 +575,7 @@ private XContentBuilder toXContentInternal(final XContentBuilder builder, final if (includeGlobalState != null) { builder.field(INCLUDE_GLOBAL_STATE, includeGlobalState); } + builder.field(USER_METADATA, userMetadata); builder.field(START_TIME, startTime); builder.field(END_TIME, endTime); builder.field(TOTAL_SHARDS, totalShards); @@ -573,6 +606,7 @@ public static SnapshotInfo fromXContentInternal(final XContentParser parser) thr int totalShards = 0; int successfulShards = 0; Boolean includeGlobalState = null; + Map userMetadata = null; List shardFailures = Collections.emptyList(); if (parser.currentToken() == null) { // fresh parser? move to the first token parser.nextToken(); @@ -628,8 +662,12 @@ public static SnapshotInfo fromXContentInternal(final XContentParser parser) thr parser.skipChildren(); } } else if (token == XContentParser.Token.START_OBJECT) { - // It was probably created by newer version - ignoring - parser.skipChildren(); + if (USER_METADATA.equals(currentFieldName)) { + userMetadata = parser.map(); + } else { + // It was probably created by newer version - ignoring + parser.skipChildren(); + } } } } @@ -651,7 +689,8 @@ public static SnapshotInfo fromXContentInternal(final XContentParser parser) thr totalShards, successfulShards, shardFailures, - includeGlobalState); + includeGlobalState, + userMetadata); } @Override @@ -683,6 +722,9 @@ public void writeTo(final StreamOutput out) throws IOException { out.writeBoolean(false); } out.writeOptionalBoolean(includeGlobalState); + if (out.getVersion().onOrAfter(METADATA_FIELD_INTRODUCED)) { + out.writeMap(userMetadata); + } } private static SnapshotState snapshotState(final String reason, final List shardFailures) { @@ -712,13 +754,14 @@ public boolean equals(Object o) { Objects.equals(indices, that.indices) && Objects.equals(includeGlobalState, that.includeGlobalState) && Objects.equals(version, that.version) && - Objects.equals(shardFailures, that.shardFailures); + Objects.equals(shardFailures, that.shardFailures) && + Objects.equals(userMetadata, that.userMetadata); } @Override public int hashCode() { return Objects.hash(snapshotId, state, reason, indices, startTime, endTime, - totalShards, successfulShards, includeGlobalState, version, shardFailures); + totalShards, successfulShards, includeGlobalState, version, shardFailures, userMetadata); } } diff --git a/server/src/main/java/org/elasticsearch/snapshots/SnapshotShardsService.java b/server/src/main/java/org/elasticsearch/snapshots/SnapshotShardsService.java index 65e1191211ec2..40ddc9a320056 100644 --- a/server/src/main/java/org/elasticsearch/snapshots/SnapshotShardsService.java +++ b/server/src/main/java/org/elasticsearch/snapshots/SnapshotShardsService.java @@ -66,6 +66,7 @@ import org.elasticsearch.indices.IndicesService; import org.elasticsearch.repositories.IndexId; import org.elasticsearch.repositories.Repository; +import org.elasticsearch.tasks.Task; import org.elasticsearch.threadpool.ThreadPool; import org.elasticsearch.transport.TransportException; import org.elasticsearch.transport.TransportRequestDeduplicator; @@ -634,7 +635,7 @@ protected UpdateIndexShardSnapshotStatusResponse newResponse() { } @Override - protected void masterOperation(UpdateIndexShardSnapshotStatusRequest request, ClusterState state, + protected void masterOperation(Task task, UpdateIndexShardSnapshotStatusRequest request, ClusterState state, ActionListener listener) { innerUpdateSnapshotState(request, listener); } diff --git a/server/src/main/java/org/elasticsearch/snapshots/SnapshotsService.java b/server/src/main/java/org/elasticsearch/snapshots/SnapshotsService.java index a6138b8f6052b..2783d635c90a4 100644 --- a/server/src/main/java/org/elasticsearch/snapshots/SnapshotsService.java +++ b/server/src/main/java/org/elasticsearch/snapshots/SnapshotsService.java @@ -287,7 +287,8 @@ public ClusterState execute(ClusterState currentState) { snapshotIndices, System.currentTimeMillis(), repositoryData.getGenId(), - null); + null, + request.userMetadata()); initializingSnapshots.add(newSnapshot.snapshot()); snapshots = new SnapshotsInProgress(newSnapshot); } else { @@ -557,7 +558,8 @@ private void cleanupAfterError(Exception exception) { 0, Collections.emptyList(), snapshot.getRepositoryStateId(), - snapshot.includeGlobalState()); + snapshot.includeGlobalState(), + snapshot.userMetadata()); } catch (Exception inner) { inner.addSuppressed(exception); logger.warn(() -> new ParameterizedMessage("[{}] failed to close snapshot in repository", @@ -572,7 +574,7 @@ private void cleanupAfterError(Exception exception) { private static SnapshotInfo inProgressSnapshot(SnapshotsInProgress.Entry entry) { return new SnapshotInfo(entry.snapshot().getSnapshotId(), entry.indices().stream().map(IndexId::getName).collect(Collectors.toList()), - entry.startTime(), entry.includeGlobalState()); + entry.startTime(), entry.includeGlobalState(), entry.userMetadata()); } /** @@ -988,7 +990,8 @@ protected void doRun() { entry.shards().size(), unmodifiableList(shardFailures), entry.getRepositoryStateId(), - entry.includeGlobalState()); + entry.includeGlobalState(), + entry.userMetadata()); removeSnapshotFromClusterState(snapshot, snapshotInfo, null); logger.info("snapshot [{}] completed with state [{}]", snapshot, snapshotInfo.state()); } diff --git a/server/src/main/java/org/elasticsearch/snapshots/package-info.java b/server/src/main/java/org/elasticsearch/snapshots/package-info.java index d5149c9bd3592..010e63eae6b4f 100644 --- a/server/src/main/java/org/elasticsearch/snapshots/package-info.java +++ b/server/src/main/java/org/elasticsearch/snapshots/package-info.java @@ -20,7 +20,7 @@ /** *

    This package exposes the Elasticsearch Snapshot functionality.

    * - *

    Preliminaries

    + *

    Preliminaries

    * *

    There are two communication channels between all nodes and master in the snapshot functionality:

    *
      @@ -32,7 +32,7 @@ * snapshot's entry in the cluster state accordingly. *
    * - *

    Snapshot Creation

    + *

    Snapshot Creation

    *

    Snapshots are created by the following sequence of events:

    *
      *
    1. An invocation of {@link org.elasticsearch.snapshots.SnapshotsService#createSnapshot} enqueues a cluster state update to create @@ -68,7 +68,7 @@ *
    2. *
    * - *

    Deleting a Snapshot

    + *

    Deleting a Snapshot

    * *

    Deleting a snapshot can take the form of either simply deleting it from the repository or (if it has not completed yet) aborting it * and subsequently deleting it from the repository.

    diff --git a/server/src/main/java/org/elasticsearch/threadpool/Scheduler.java b/server/src/main/java/org/elasticsearch/threadpool/Scheduler.java index 588495dd27d32..04741cd87d04c 100644 --- a/server/src/main/java/org/elasticsearch/threadpool/Scheduler.java +++ b/server/src/main/java/org/elasticsearch/threadpool/Scheduler.java @@ -33,6 +33,7 @@ import java.util.concurrent.Delayed; import java.util.concurrent.Future; import java.util.concurrent.RejectedExecutionHandler; +import java.util.concurrent.RunnableFuture; import java.util.concurrent.ScheduledFuture; import java.util.concurrent.ScheduledThreadPoolExecutor; import java.util.concurrent.ThreadFactory; @@ -276,7 +277,11 @@ protected void afterExecute(Runnable r, Throwable t) { if (t != null) return; // Scheduler only allows Runnable's so we expect no checked exceptions here. If anyone uses submit directly on `this`, we // accept the wrapped exception in the output. - ExceptionsHelper.reThrowIfNotNull(EsExecutors.rethrowErrors(r)); + if (r instanceof RunnableFuture && ((RunnableFuture) r).isDone()) { + // only check this if task is done, which it always is except for periodic tasks. Periodic tasks will hang on + // RunnableFuture.get() + ExceptionsHelper.reThrowIfNotNull(EsExecutors.rethrowErrors(r)); + } } } } diff --git a/server/src/main/java/org/elasticsearch/transport/RemoteClusterAware.java b/server/src/main/java/org/elasticsearch/transport/RemoteClusterAware.java index 316fcd275a5a0..3126963f36930 100644 --- a/server/src/main/java/org/elasticsearch/transport/RemoteClusterAware.java +++ b/server/src/main/java/org/elasticsearch/transport/RemoteClusterAware.java @@ -22,6 +22,7 @@ import org.elasticsearch.Version; import org.elasticsearch.cluster.metadata.ClusterNameExpressionResolver; import org.elasticsearch.cluster.node.DiscoveryNode; +import org.elasticsearch.cluster.node.DiscoveryNodeRole; import org.elasticsearch.common.Strings; import org.elasticsearch.common.UUIDs; import org.elasticsearch.common.collect.Tuple; @@ -37,7 +38,6 @@ import java.util.ArrayList; import java.util.Arrays; import java.util.Collections; -import java.util.EnumSet; import java.util.HashMap; import java.util.List; import java.util.Map; @@ -136,7 +136,7 @@ static DiscoveryNode buildSeedNode(String clusterName, String address, boolean p TransportAddress transportAddress = new TransportAddress(TransportAddress.META_ADDRESS, 0); String hostName = address.substring(0, indexOfPortSeparator(address)); return new DiscoveryNode("", clusterName + "#" + address, UUIDs.randomBase64UUID(), hostName, address, - transportAddress, Collections.singletonMap("server_name", hostName), EnumSet.allOf(DiscoveryNode.Role.class), + transportAddress, Collections.singletonMap("server_name", hostName), DiscoveryNodeRole.BUILT_IN_ROLES, Version.CURRENT.minimumCompatibilityVersion()); } else { TransportAddress transportAddress = new TransportAddress(RemoteClusterAware.parseSeedAddress(address)); diff --git a/server/src/main/java/org/elasticsearch/transport/RemoteClusterAwareClient.java b/server/src/main/java/org/elasticsearch/transport/RemoteClusterAwareClient.java index 2ca42ff85abdf..69b8a1f68a682 100644 --- a/server/src/main/java/org/elasticsearch/transport/RemoteClusterAwareClient.java +++ b/server/src/main/java/org/elasticsearch/transport/RemoteClusterAwareClient.java @@ -18,7 +18,7 @@ */ package org.elasticsearch.transport; -import org.elasticsearch.action.Action; +import org.elasticsearch.action.ActionType; import org.elasticsearch.action.ActionListener; import org.elasticsearch.action.ActionListenerResponseHandler; import org.elasticsearch.action.ActionRequest; @@ -44,7 +44,7 @@ final class RemoteClusterAwareClient extends AbstractClient { @Override protected - void doExecute(Action action, Request request, ActionListener listener) { + void doExecute(ActionType action, Request request, ActionListener listener) { remoteClusterService.ensureConnected(clusterAlias, ActionListener.wrap(res -> { Transport.Connection connection; if (request instanceof RemoteClusterAwareRequest) { diff --git a/server/src/main/java/org/elasticsearch/transport/RemoteClusterConnection.java b/server/src/main/java/org/elasticsearch/transport/RemoteClusterConnection.java index 6b6ce18cfefbb..ca03e59a75858 100644 --- a/server/src/main/java/org/elasticsearch/transport/RemoteClusterConnection.java +++ b/server/src/main/java/org/elasticsearch/transport/RemoteClusterConnection.java @@ -176,6 +176,7 @@ boolean isSkipUnavailable() { @Override public void onNodeDisconnected(DiscoveryNode node) { boolean remove = connectedNodes.remove(node); + logger.trace("node disconnected: {}, removed: {}", node, remove); if (remove && connectedNodes.size() < maxNumRemoteConnections) { // try to reconnect and fill up the slot of the disconnected node connectHandler.forceConnect(); diff --git a/server/src/main/java/org/elasticsearch/transport/RemoteClusterService.java b/server/src/main/java/org/elasticsearch/transport/RemoteClusterService.java index 4f690d12acf1e..db4e0f021e05b 100644 --- a/server/src/main/java/org/elasticsearch/transport/RemoteClusterService.java +++ b/server/src/main/java/org/elasticsearch/transport/RemoteClusterService.java @@ -183,7 +183,8 @@ private synchronized void updateRemoteClusters(Map>> oldSeedNodes, + final List>> newSeedNodes) { + if (oldSeedNodes.size() != newSeedNodes.size()) { + return true; + } + Set oldSeeds = oldSeedNodes.stream().map(Tuple::v1).collect(Collectors.toSet()); + Set newSeeds = newSeedNodes.stream().map(Tuple::v1).collect(Collectors.toSet()); + return oldSeeds.equals(newSeeds) == false; + } + /** * Collects all nodes of the given clusters and returns / passes a (clusterAlias, nodeId) to {@link DiscoveryNode} * function on success. diff --git a/server/src/main/java/org/elasticsearch/transport/TcpTransport.java b/server/src/main/java/org/elasticsearch/transport/TcpTransport.java index eef9f4f42637c..ad9059dbc3757 100644 --- a/server/src/main/java/org/elasticsearch/transport/TcpTransport.java +++ b/server/src/main/java/org/elasticsearch/transport/TcpTransport.java @@ -280,7 +280,7 @@ private List initiateConnection(DiscoveryNode node, ConnectionProfil for (int i = 0; i < numConnections; ++i) { try { TcpChannel channel = initiateChannel(node); - logger.trace(() -> new ParameterizedMessage("Tcp transport client channel opened: {}", channel)); + logger.trace(() -> new ParameterizedMessage("Tcp transport channel opened: {}", channel)); channels.add(channel); } catch (ConnectTransportException e) { CloseableChannel.closeChannels(channels, false); @@ -933,34 +933,20 @@ public void onResponse(Void v) { if (countDown.countDown()) { final TcpChannel handshakeChannel = channels.get(0); try { - executeHandshake(node, handshakeChannel, connectionProfile, new ActionListener() { - @Override - public void onResponse(Version version) { - NodeChannels nodeChannels = new NodeChannels(node, channels, connectionProfile, version); - long relativeMillisTime = threadPool.relativeTimeInMillis(); - nodeChannels.channels.forEach(ch -> { - // Mark the channel init time - ch.getChannelStats().markAccessed(relativeMillisTime); - ch.addCloseListener(ActionListener.wrap(nodeChannels::close)); - }); - keepAlive.registerNodeConnection(nodeChannels.channels, connectionProfile); - listener.onResponse(nodeChannels); - } - - @Override - public void onFailure(Exception e) { - CloseableChannel.closeChannels(channels, false); - - if (e instanceof ConnectTransportException) { - listener.onFailure(e); - } else { - listener.onFailure(new ConnectTransportException(node, "general node connection failure", e)); - } - } - }); + executeHandshake(node, handshakeChannel, connectionProfile, ActionListener.wrap(version -> { + NodeChannels nodeChannels = new NodeChannels(node, channels, connectionProfile, version); + long relativeMillisTime = threadPool.relativeTimeInMillis(); + nodeChannels.channels.forEach(ch -> { + // Mark the channel init time + ch.getChannelStats().markAccessed(relativeMillisTime); + ch.addCloseListener(ActionListener.wrap(nodeChannels::close)); + }); + keepAlive.registerNodeConnection(nodeChannels.channels, connectionProfile); + listener.onResponse(nodeChannels); + }, e -> closeAndFail(e instanceof ConnectTransportException ? + e : new ConnectTransportException(node, "general node connection failure", e)))); } catch (Exception ex) { - CloseableChannel.closeChannels(channels, false); - listener.onFailure(ex); + closeAndFail(ex); } } } @@ -968,15 +954,23 @@ public void onFailure(Exception e) { @Override public void onFailure(Exception ex) { if (countDown.fastForward()) { - CloseableChannel.closeChannels(channels, false); - listener.onFailure(new ConnectTransportException(node, "connect_exception", ex)); + closeAndFail(new ConnectTransportException(node, "connect_exception", ex)); } } public void onTimeout() { if (countDown.fastForward()) { + closeAndFail(new ConnectTransportException(node, "connect_timeout[" + connectionProfile.getConnectTimeout() + "]")); + } + } + + private void closeAndFail(Exception e) { + try { CloseableChannel.closeChannels(channels, false); - listener.onFailure(new ConnectTransportException(node, "connect_timeout[" + connectionProfile.getConnectTimeout() + "]")); + } catch (Exception ex) { + e.addSuppressed(ex); + } finally { + listener.onFailure(e); } } } diff --git a/server/src/main/java/org/elasticsearch/transport/TransportService.java b/server/src/main/java/org/elasticsearch/transport/TransportService.java index a89784945db93..dca7f52e60474 100644 --- a/server/src/main/java/org/elasticsearch/transport/TransportService.java +++ b/server/src/main/java/org/elasticsearch/transport/TransportService.java @@ -24,8 +24,6 @@ import org.apache.logging.log4j.message.ParameterizedMessage; import org.elasticsearch.Version; import org.elasticsearch.action.ActionListener; -import org.elasticsearch.client.Client; -import org.elasticsearch.client.transport.TransportClient; import org.elasticsearch.cluster.ClusterName; import org.elasticsearch.cluster.node.DiscoveryNode; import org.elasticsearch.common.CheckedBiConsumer; @@ -109,8 +107,6 @@ protected boolean removeEldestEntry(Map.Entry eldest) { private final RemoteClusterService remoteClusterService; - private final boolean validateConnections; - /** if set will call requests sent to this id to shortcut and executed locally */ volatile DiscoveryNode localNode = null; private final Transport.Connection localNodeConnection = new Transport.Connection() { @@ -155,9 +151,6 @@ public TransportService(Settings settings, Transport transport, ThreadPool threa public TransportService(Settings settings, Transport transport, ThreadPool threadPool, TransportInterceptor transportInterceptor, Function localNodeFactory, @Nullable ClusterSettings clusterSettings, Set taskHeaders, ConnectionManager connectionManager) { - // The only time we do not want to validate node connections is when this is a transport client using the simple node sampler - this.validateConnections = TransportClient.CLIENT_TYPE.equals(settings.get(Client.CLIENT_TYPE_SETTING_S.getKey())) == false || - TransportClient.CLIENT_TRANSPORT_SNIFF.get(settings); this.transport = transport; this.threadPool = threadPool; this.localNodeFactory = localNodeFactory; @@ -348,7 +341,7 @@ public CheckedBiConsumer c return (newConnection, actualProfile) -> { // We don't validate cluster names to allow for CCS connections. final DiscoveryNode remote = handshake(newConnection, actualProfile.getHandshakeTimeout().millis(), cn -> true).discoveryNode; - if (validateConnections && node.equals(remote) == false) { + if (node.equals(remote) == false) { throw new ConnectTransportException(node, "handshake failed. unexpected remote node " + remote); } }; diff --git a/server/src/main/resources/org/elasticsearch/bootstrap/security.policy b/server/src/main/resources/org/elasticsearch/bootstrap/security.policy index 4df99ef6f8836..415828b8a533d 100644 --- a/server/src/main/resources/org/elasticsearch/bootstrap/security.policy +++ b/server/src/main/resources/org/elasticsearch/bootstrap/security.policy @@ -47,7 +47,7 @@ grant codeBase "${codebase.lucene-misc}" { permission java.nio.file.LinkPermission "hard"; }; -grant codeBase "${codebase.plugin-classloader}" { +grant codeBase "${codebase.elasticsearch-plugin-classloader}" { // needed to create the classloader which allows plugins to extend other plugins permission java.lang.RuntimePermission "createClassLoader"; }; @@ -55,6 +55,8 @@ grant codeBase "${codebase.plugin-classloader}" { //// Everything else: grant { + // needed by vendored Guice + permission java.lang.RuntimePermission "accessClassInPackage.jdk.internal.vm.annotation"; // checked by scripting engines, and before hacks and other issues in // third party code, to safeguard these against unprivileged code like scripts. diff --git a/server/src/test/java/org/apache/lucene/search/uhighlight/CustomUnifiedHighlighterTests.java b/server/src/test/java/org/apache/lucene/search/uhighlight/CustomUnifiedHighlighterTests.java index 4e4b04d1ff19c..3c24dc2d42b82 100644 --- a/server/src/test/java/org/apache/lucene/search/uhighlight/CustomUnifiedHighlighterTests.java +++ b/server/src/test/java/org/apache/lucene/search/uhighlight/CustomUnifiedHighlighterTests.java @@ -30,7 +30,6 @@ import org.apache.lucene.index.IndexWriterConfig; import org.apache.lucene.index.RandomIndexWriter; import org.apache.lucene.index.Term; -import org.apache.lucene.queries.CommonTermsQuery; import org.apache.lucene.search.BooleanClause; import org.apache.lucene.search.BooleanQuery; import org.apache.lucene.search.IndexSearcher; @@ -147,21 +146,6 @@ public void testMultiPhrasePrefixQuery() throws Exception { BreakIterator.getSentenceInstance(Locale.ROOT), 0, outputs); } - public void testCommonTermsQuery() throws Exception { - final String[] inputs = { - "The quick brown fox." - }; - final String[] outputs = { - "The quick brown fox." - }; - CommonTermsQuery query = new CommonTermsQuery(BooleanClause.Occur.SHOULD, BooleanClause.Occur.SHOULD, 128); - query.add(new Term("text", "quick")); - query.add(new Term("text", "brown")); - query.add(new Term("text", "fox")); - assertHighlightOneDoc("text", inputs, new StandardAnalyzer(), query, Locale.ROOT, - BreakIterator.getSentenceInstance(Locale.ROOT), 0, outputs); - } - public void testSentenceBoundedBreakIterator() throws Exception { final String[] inputs = { "The quick brown fox in a long sentence with another quick brown fox. " + diff --git a/server/src/test/java/org/apache/lucene/util/CombinedBitSetTests.java b/server/src/test/java/org/apache/lucene/util/CombinedBitSetTests.java new file mode 100644 index 0000000000000..65165272f5469 --- /dev/null +++ b/server/src/test/java/org/apache/lucene/util/CombinedBitSetTests.java @@ -0,0 +1,111 @@ +/* + * 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.apache.lucene.util; + +import org.apache.lucene.search.DocIdSetIterator; +import org.elasticsearch.test.ESTestCase; + +public class CombinedBitSetTests extends ESTestCase { + public void testEmpty() { + for (float percent : new float[] {0f, 0.1f, 0.5f, 0.9f, 1f}) { + testCase(randomIntBetween(1, 10000), 0f, percent); + testCase(randomIntBetween(1, 10000), percent, 0f); + } + } + + public void testSparse() { + for (float percent : new float[] {0f, 0.1f, 0.5f, 0.9f, 1f}) { + testCase(randomIntBetween(1, 10000), 0.1f, percent); + testCase(randomIntBetween(1, 10000), percent, 0.1f); + } + } + + public void testDense() { + for (float percent : new float[] {0f, 0.1f, 0.5f, 0.9f, 1f}) { + testCase(randomIntBetween(1, 10000), 0.9f, percent); + testCase(randomIntBetween(1, 10000), percent, 0.9f); + } + } + + public void testRandom() { + int iterations = atLeast(10); + for (int i = 0; i < iterations; i++) { + testCase(randomIntBetween(1, 10000), randomFloat(), randomFloat()); + } + } + + private void testCase(int numBits, float percent1, float percent2) { + BitSet first = randomSet(numBits, percent1); + BitSet second = randomSet(numBits, percent2); + CombinedBitSet actual = new CombinedBitSet(first, second); + FixedBitSet expected = new FixedBitSet(numBits); + or(expected, first); + and(expected, second); + assertEquals(expected.cardinality(), actual.cardinality()); + assertEquals(expected, actual, numBits); + for (int i = 0; i < numBits; ++i) { + assertEquals(expected.nextSetBit(i), actual.nextSetBit(i)); + assertEquals(Integer.toString(i), expected.prevSetBit(i), actual.prevSetBit(i)); + } + } + + private void or(BitSet set1, BitSet set2) { + int next = 0; + while (next < set2.length() && (next = set2.nextSetBit(next)) != DocIdSetIterator.NO_MORE_DOCS) { + set1.set(next); + next += 1; + } + } + + private void and(BitSet set1, BitSet set2) { + int next = 0; + while (next < set1.length() && (next = set1.nextSetBit(next)) != DocIdSetIterator.NO_MORE_DOCS) { + if (set2.get(next) == false) { + set1.clear(next); + } + next += 1; + } + } + + private void assertEquals(BitSet set1, BitSet set2, int maxDoc) { + for (int i = 0; i < maxDoc; ++i) { + assertEquals("Different at " + i, set1.get(i), set2.get(i)); + } + } + + private BitSet randomSet(int numBits, float percentSet) { + return randomSet(numBits, (int) (percentSet * numBits)); + } + + private BitSet randomSet(int numBits, int numBitsSet) { + assert numBitsSet <= numBits; + final BitSet set = randomBoolean() ? new SparseFixedBitSet(numBits) : new FixedBitSet(numBits); + for (int i = 0; i < numBitsSet; ++i) { + while (true) { + final int o = random().nextInt(numBits); + if (set.get(o) == false) { + set.set(o); + break; + } + } + } + return set; + } +} diff --git a/server/src/test/java/org/elasticsearch/BuildTests.java b/server/src/test/java/org/elasticsearch/BuildTests.java index 7a6f581bd7369..6e1c92a8a76bc 100644 --- a/server/src/test/java/org/elasticsearch/BuildTests.java +++ b/server/src/test/java/org/elasticsearch/BuildTests.java @@ -58,16 +58,6 @@ public void testIsProduction() { ); assertTrue(build.getQualifiedVersion(), build.isProductionRelease()); - assertFalse(new Build( - Build.CURRENT.flavor(), Build.CURRENT.type(), Build.CURRENT.shortHash(), Build.CURRENT.date(), - Build.CURRENT.isSnapshot(), "7.0.0-alpha1" - ).isProductionRelease()); - - assertFalse(new Build( - Build.CURRENT.flavor(), Build.CURRENT.type(), Build.CURRENT.shortHash(), Build.CURRENT.date(), - Build.CURRENT.isSnapshot(), "7.0.0-alpha1-SNAPSHOT" - ).isProductionRelease()); - assertFalse(new Build( Build.CURRENT.flavor(), Build.CURRENT.type(), Build.CURRENT.shortHash(), Build.CURRENT.date(), Build.CURRENT.isSnapshot(), "7.0.0-SNAPSHOT" diff --git a/server/src/test/java/org/elasticsearch/ExceptionsHelperTests.java b/server/src/test/java/org/elasticsearch/ExceptionsHelperTests.java index 2de2f259e6ff1..3b5d1ad43da97 100644 --- a/server/src/test/java/org/elasticsearch/ExceptionsHelperTests.java +++ b/server/src/test/java/org/elasticsearch/ExceptionsHelperTests.java @@ -35,9 +35,9 @@ import org.elasticsearch.test.ESTestCase; import org.elasticsearch.transport.RemoteClusterAware; +import java.io.IOException; import java.util.Optional; -import static org.elasticsearch.ExceptionsHelper.MAX_ITERATIONS; import static org.elasticsearch.ExceptionsHelper.maybeError; import static org.hamcrest.CoreMatchers.equalTo; import static org.hamcrest.CoreMatchers.instanceOf; @@ -81,20 +81,14 @@ public void testMaybeError() { if (fatal) { assertError(cause, error); } else { - assertFalse(maybeError(cause, logger).isPresent()); + assertFalse(maybeError(cause).isPresent()); } - assertFalse(maybeError(new Exception(new DecoderException()), logger).isPresent()); - - Throwable chain = outOfMemoryError; - for (int i = 0; i < MAX_ITERATIONS; i++) { - chain = new Exception(chain); - } - assertFalse(maybeError(chain, logger).isPresent()); + assertFalse(maybeError(new Exception(new DecoderException())).isPresent()); } private void assertError(final Throwable cause, final Error error) { - final Optional maybeError = maybeError(cause, logger); + final Optional maybeError = maybeError(cause); assertTrue(maybeError.isPresent()); assertThat(maybeError.get(), equalTo(error)); } @@ -211,4 +205,29 @@ public void testUnwrapCorruption() { withSuppressedException.addSuppressed(new RuntimeException()); assertThat(ExceptionsHelper.unwrapCorruption(withSuppressedException), nullValue()); } + + public void testSuppressedCycle() { + RuntimeException e1 = new RuntimeException(); + RuntimeException e2 = new RuntimeException(); + e1.addSuppressed(e2); + e2.addSuppressed(e1); + ExceptionsHelper.unwrapCorruption(e1); + + final CorruptIndexException corruptIndexException = new CorruptIndexException("corrupt", "resource"); + RuntimeException e3 = new RuntimeException(corruptIndexException); + e3.addSuppressed(e1); + assertThat(ExceptionsHelper.unwrapCorruption(e3), equalTo(corruptIndexException)); + + RuntimeException e4 = new RuntimeException(e1); + e4.addSuppressed(corruptIndexException); + assertThat(ExceptionsHelper.unwrapCorruption(e4), equalTo(corruptIndexException)); + } + + public void testCauseCycle() { + RuntimeException e1 = new RuntimeException(); + RuntimeException e2 = new RuntimeException(e1); + e1.initCause(e2); + ExceptionsHelper.unwrap(e1, IOException.class); + ExceptionsHelper.unwrapCorruption(e1); + } } diff --git a/server/src/test/java/org/elasticsearch/VersionTests.java b/server/src/test/java/org/elasticsearch/VersionTests.java index e5149b9bce515..7705b435fb1b7 100644 --- a/server/src/test/java/org/elasticsearch/VersionTests.java +++ b/server/src/test/java/org/elasticsearch/VersionTests.java @@ -66,11 +66,6 @@ public void testVersionComparison() throws Exception { assertThat(V_7_2_0.onOrAfter(V_7_2_0), is(true)); assertThat(V_8_0_0.onOrAfter(V_7_2_0), is(true)); - assertTrue(Version.fromString("5.0.0-alpha2").onOrAfter(Version.fromString("5.0.0-alpha1"))); - assertTrue(Version.fromString("5.0.0").onOrAfter(Version.fromString("5.0.0-beta2"))); - assertTrue(Version.fromString("5.0.0-rc1").onOrAfter(Version.fromString("5.0.0-beta24"))); - assertTrue(Version.fromString("5.0.0-alpha24").before(Version.fromString("5.0.0-beta0"))); - assertThat(V_7_2_0, is(lessThan(V_8_0_0))); assertThat(V_7_2_0.compareTo(V_7_2_0), is(0)); assertThat(V_8_0_0, is(greaterThan(V_7_2_0))); @@ -181,12 +176,12 @@ public void testMinCompatVersion() { assertThat(Version.fromString("5.3.0").minimumCompatibilityVersion(), equalTo(major5x)); Version major56x = Version.fromString("5.6.0"); - assertThat(Version.V_6_5_0.minimumCompatibilityVersion(), equalTo(major56x)); + assertThat(Version.fromString("6.4.0").minimumCompatibilityVersion(), equalTo(major56x)); assertThat(Version.fromString("6.3.1").minimumCompatibilityVersion(), equalTo(major56x)); // from 7.0 on we are supporting the latest minor of the previous major... this might fail once we add a new version ie. 5.x is // released since we need to bump the supported minor in Version#minimumCompatibilityVersion() - Version lastVersion = Version.V_6_8_0; // TODO: remove this once min compat version is a constant instead of method + Version lastVersion = Version.fromString("6.8.0"); // TODO: remove this once min compat version is a constant instead of method assertEquals(lastVersion.major, Version.V_7_0_0.minimumCompatibilityVersion().major); assertEquals("did you miss to bump the minor in Version#minimumCompatibilityVersion()", lastVersion.minor, Version.V_7_0_0.minimumCompatibilityVersion().minor); @@ -194,41 +189,12 @@ public void testMinCompatVersion() { } public void testToString() { - // with 2.0.beta we lowercase - assertEquals("2.0.0-beta1", Version.fromString("2.0.0-beta1").toString()); - assertEquals("5.0.0-alpha1", Version.fromId(5000001).toString()); + assertEquals("5.0.0", Version.fromId(5000099).toString()); assertEquals("2.3.0", Version.fromString("2.3.0").toString()); - assertEquals("0.90.0.Beta1", Version.fromString("0.90.0.Beta1").toString()); - assertEquals("1.0.0.Beta1", Version.fromString("1.0.0.Beta1").toString()); - assertEquals("2.0.0-beta1", Version.fromString("2.0.0-beta1").toString()); - assertEquals("5.0.0-beta1", Version.fromString("5.0.0-beta1").toString()); - assertEquals("5.0.0-alpha1", Version.fromString("5.0.0-alpha1").toString()); - } - - public void testIsBeta() { - assertTrue(Version.fromString("2.0.0-beta1").isBeta()); - assertTrue(Version.fromString("1.0.0.Beta1").isBeta()); - assertTrue(Version.fromString("0.90.0.Beta1").isBeta()); - } - - - public void testIsAlpha() { - assertTrue(new Version(5000001, org.apache.lucene.util.Version.LUCENE_7_0_0).isAlpha()); - assertFalse(new Version(4000002, org.apache.lucene.util.Version.LUCENE_7_0_0).isAlpha()); - assertTrue(new Version(4000002, org.apache.lucene.util.Version.LUCENE_7_0_0).isBeta()); - assertTrue(Version.fromString("5.0.0-alpha14").isAlpha()); - assertEquals(5000014, Version.fromString("5.0.0-alpha14").id); - assertTrue(Version.fromId(5000015).isAlpha()); - - for (int i = 0 ; i < 25; i++) { - assertEquals(Version.fromString("5.0.0-alpha" + i).id, Version.fromId(5000000 + i).id); - assertEquals("5.0.0-alpha" + i, Version.fromId(5000000 + i).toString()); - } - - for (int i = 0 ; i < 25; i++) { - assertEquals(Version.fromString("5.0.0-beta" + i).id, Version.fromId(5000000 + i + 25).id); - assertEquals("5.0.0-beta" + i, Version.fromId(5000000 + i + 25).toString()); - } + assertEquals("0.90.0", Version.fromString("0.90.0").toString()); + assertEquals("1.0.0", Version.fromString("1.0.0").toString()); + assertEquals("2.0.0", Version.fromString("2.0.0").toString()); + assertEquals("5.0.0", Version.fromString("5.0.0").toString()); } public void testParseVersion() { @@ -288,19 +254,8 @@ public void testAllVersionsMatchId() throws Exception { } assertEquals("Version id " + field.getName() + " does not point to " + constantName, v, Version.fromId(versionId)); assertEquals("Version " + constantName + " does not have correct id", versionId, v.id); - if (v.major >= 2) { - String number = v.toString(); - if (v.isBeta()) { - number = number.replace("-beta", "_beta"); - } else if (v.isRC()) { - number = number.replace("-rc", "_rc"); - } else if (v.isAlpha()) { - number = number.replace("-alpha", "_alpha"); - } - assertEquals("V_" + number.replace('.', '_'), constantName); - } else { - assertEquals("V_" + v.toString().replace('.', '_'), constantName); - } + String number = v.toString(); + assertEquals("V_" + number.replace('.', '_'), constantName); // only the latest version for a branch should be a snapshot (ie unreleased) String branchName = "" + v.major + "." + v.minor; @@ -328,8 +283,7 @@ public void testLuceneVersionIsSameOnMinorRelease() { assertTrue("lucene versions must be " + other + " >= " + version, other.luceneVersion.onOrAfter(version.luceneVersion)); } - if (other.isAlpha() == false && version.isAlpha() == false - && other.major == version.major && other.minor == version.minor) { + if (other.major == version.major && other.minor == version.minor) { assertEquals(version + " vs. " + other, other.luceneVersion.major, version.luceneVersion.major); assertEquals(version + " vs. " + other, other.luceneVersion.minor, version.luceneVersion.minor); // should we also assert the lucene bugfix version? @@ -345,10 +299,10 @@ public static void assertUnknownVersion(Version version) { public void testIsCompatible() { assertTrue(isCompatible(Version.CURRENT, Version.CURRENT.minimumCompatibilityVersion())); - assertFalse(isCompatible(Version.V_6_6_0, Version.V_7_0_0)); - assertTrue(isCompatible(Version.V_6_8_0, Version.V_7_0_0)); + assertFalse(isCompatible(Version.V_7_0_0, Version.V_8_0_0)); + assertTrue(isCompatible(Version.fromString("6.8.0"), Version.fromString("7.0.0"))); assertFalse(isCompatible(Version.fromId(2000099), Version.V_7_0_0)); - assertFalse(isCompatible(Version.fromId(2000099), Version.V_6_5_0)); + assertFalse(isCompatible(Version.fromId(2000099), Version.fromString("6.5.0"))); final Version currentMajorVersion = Version.fromId(Version.CURRENT.major * 1000000 + 99); final Version currentOrNextMajorVersion; diff --git a/server/src/test/java/org/elasticsearch/action/ActionModuleTests.java b/server/src/test/java/org/elasticsearch/action/ActionModuleTests.java index 1bde97d0b8ebd..979d0018d3c01 100644 --- a/server/src/test/java/org/elasticsearch/action/ActionModuleTests.java +++ b/server/src/test/java/org/elasticsearch/action/ActionModuleTests.java @@ -88,7 +88,7 @@ protected FakeTransportAction(String actionName, ActionFilters actionFilters, Ta protected void doExecute(Task task, FakeRequest request, ActionListener listener) { } } - class FakeAction extends Action { + class FakeAction extends StreamableResponseActionType { protected FakeAction() { super("fake"); } @@ -112,7 +112,7 @@ public ActionResponse newResponse() { public void testSetupRestHandlerContainsKnownBuiltin() { SettingsModule settings = new SettingsModule(Settings.EMPTY); UsageService usageService = new UsageService(); - ActionModule actionModule = new ActionModule(false, settings.getSettings(), new IndexNameExpressionResolver(), + ActionModule actionModule = new ActionModule(settings.getSettings(), new IndexNameExpressionResolver(), settings.getIndexScopedSettings(), settings.getClusterSettings(), settings.getSettingsFilter(), null, emptyList(), null, null, usageService); actionModule.initRestHandlers(null); @@ -135,7 +135,7 @@ public List getRestHandlers(Settings settings, RestController restC ThreadPool threadPool = new TestThreadPool(getTestName()); try { UsageService usageService = new UsageService(); - ActionModule actionModule = new ActionModule(false, settings.getSettings(), new IndexNameExpressionResolver(), + ActionModule actionModule = new ActionModule(settings.getSettings(), new IndexNameExpressionResolver(), settings.getIndexScopedSettings(), settings.getClusterSettings(), settings.getSettingsFilter(), threadPool, singletonList(dupsMainAction), null, null, usageService); Exception e = expectThrows(IllegalArgumentException.class, () -> actionModule.initRestHandlers(null)); @@ -167,7 +167,7 @@ public List getRestHandlers(Settings settings, RestController restC ThreadPool threadPool = new TestThreadPool(getTestName()); try { UsageService usageService = new UsageService(); - ActionModule actionModule = new ActionModule(false, settings.getSettings(), new IndexNameExpressionResolver(), + ActionModule actionModule = new ActionModule(settings.getSettings(), new IndexNameExpressionResolver(), settings.getIndexScopedSettings(), settings.getClusterSettings(), settings.getSettingsFilter(), threadPool, singletonList(registersFakeHandler), null, null, usageService); actionModule.initRestHandlers(null); diff --git a/server/src/test/java/org/elasticsearch/action/ActionTests.java b/server/src/test/java/org/elasticsearch/action/ActionTests.java index a7dca3f098d05..b9462ac68e749 100644 --- a/server/src/test/java/org/elasticsearch/action/ActionTests.java +++ b/server/src/test/java/org/elasticsearch/action/ActionTests.java @@ -24,7 +24,7 @@ public class ActionTests extends ESTestCase { public void testEquals() { - class FakeAction extends Action { + class FakeAction extends StreamableResponseActionType { protected FakeAction(String name) { super(name); } diff --git a/server/src/test/java/org/elasticsearch/action/IndicesRequestIT.java b/server/src/test/java/org/elasticsearch/action/IndicesRequestIT.java index 2dbb52d547f97..24e2e68ed4fc6 100644 --- a/server/src/test/java/org/elasticsearch/action/IndicesRequestIT.java +++ b/server/src/test/java/org/elasticsearch/action/IndicesRequestIT.java @@ -21,7 +21,6 @@ import org.elasticsearch.action.admin.indices.alias.Alias; import org.elasticsearch.action.admin.indices.analyze.AnalyzeAction; -import org.elasticsearch.action.admin.indices.analyze.AnalyzeRequest; import org.elasticsearch.action.admin.indices.cache.clear.ClearIndicesCacheAction; import org.elasticsearch.action.admin.indices.cache.clear.ClearIndicesCacheRequest; import org.elasticsearch.action.admin.indices.close.CloseIndexAction; @@ -207,7 +206,7 @@ public void testAnalyze() { String analyzeShardAction = AnalyzeAction.NAME + "[s]"; interceptTransportActions(analyzeShardAction); - AnalyzeRequest analyzeRequest = new AnalyzeRequest(randomIndexOrAlias()); + AnalyzeAction.Request analyzeRequest = new AnalyzeAction.Request(randomIndexOrAlias()); analyzeRequest.text("text"); internalCluster().coordOnlyNodeClient().admin().indices().analyze(analyzeRequest).actionGet(); @@ -341,7 +340,7 @@ public void testTermVector() { String termVectorShardAction = TermVectorsAction.NAME + "[s]"; interceptTransportActions(termVectorShardAction); - TermVectorsRequest termVectorsRequest = new TermVectorsRequest(randomIndexOrAlias(), "type", "id"); + TermVectorsRequest termVectorsRequest = new TermVectorsRequest(randomIndexOrAlias(), "id"); internalCluster().coordOnlyNodeClient().termVectors(termVectorsRequest).actionGet(); clearInterceptedActions(); @@ -357,7 +356,7 @@ public void testMultiTermVector() { int numDocs = iterations(1, 30); for (int i = 0; i < numDocs; i++) { String indexOrAlias = randomIndexOrAlias(); - multiTermVectorsRequest.add(indexOrAlias, "type", Integer.toString(i)); + multiTermVectorsRequest.add(indexOrAlias, Integer.toString(i)); indices.add(indexOrAlias); } internalCluster().coordOnlyNodeClient().multiTermVectors(multiTermVectorsRequest).actionGet(); diff --git a/server/src/test/java/org/elasticsearch/action/ListenerActionIT.java b/server/src/test/java/org/elasticsearch/action/ListenerActionIT.java index d0e8d997a1bb9..0dfc2fd58e86c 100644 --- a/server/src/test/java/org/elasticsearch/action/ListenerActionIT.java +++ b/server/src/test/java/org/elasticsearch/action/ListenerActionIT.java @@ -23,7 +23,6 @@ import org.elasticsearch.action.index.IndexResponse; import org.elasticsearch.client.Client; import org.elasticsearch.client.Requests; -import org.elasticsearch.client.transport.TransportClient; import org.elasticsearch.test.ESIntegTestCase; import java.util.concurrent.CountDownLatch; @@ -59,11 +58,6 @@ public void onFailure(Exception e) { latch.await(); - boolean shouldBeThreaded = TransportClient.CLIENT_TYPE.equals(Client.CLIENT_TYPE_SETTING_S.get(client.settings())); - if (shouldBeThreaded) { - assertTrue(threadName.get().contains("listener")); - } else { - assertFalse(threadName.get().contains("listener")); - } + assertFalse(threadName.get().contains("listener")); } } diff --git a/server/src/test/java/org/elasticsearch/action/RequestValidatorsTests.java b/server/src/test/java/org/elasticsearch/action/RequestValidatorsTests.java new file mode 100644 index 0000000000000..f339a0b3e072e --- /dev/null +++ b/server/src/test/java/org/elasticsearch/action/RequestValidatorsTests.java @@ -0,0 +1,94 @@ +/* + * 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; + +import org.elasticsearch.action.admin.indices.mapping.put.PutMappingRequest; +import org.elasticsearch.common.Randomness; +import org.elasticsearch.test.ESTestCase; +import org.elasticsearch.test.hamcrest.OptionalMatchers; +import org.hamcrest.Matchers; + +import java.util.ArrayList; +import java.util.List; +import java.util.Optional; + +public class RequestValidatorsTests extends ESTestCase { + + private final RequestValidators.RequestValidator EMPTY = (request, state, indices) -> Optional.empty(); + private final RequestValidators.RequestValidator FAIL = + (request, state, indices) -> Optional.of(new Exception("failure")); + + public void testValidates() { + final int numberOfValidations = randomIntBetween(0, 8); + final List> validators = new ArrayList<>(numberOfValidations); + for (int i = 0; i < numberOfValidations; i++) { + validators.add(EMPTY); + } + final RequestValidators requestValidators = new RequestValidators<>(validators); + assertThat(requestValidators.validateRequest(null, null, null), OptionalMatchers.isEmpty()); + } + + public void testFailure() { + final RequestValidators validators = new RequestValidators<>(List.of(FAIL)); + assertThat(validators.validateRequest(null, null, null), OptionalMatchers.isPresent()); + } + + public void testValidatesAfterFailure() { + final RequestValidators validators = new RequestValidators<>(List.of(FAIL, EMPTY)); + assertThat(validators.validateRequest(null, null, null), OptionalMatchers.isPresent()); + } + + public void testMultipleFailures() { + final int numberOfFailures = randomIntBetween(2, 8); + final List> validators = new ArrayList<>(numberOfFailures); + for (int i = 0; i < numberOfFailures; i++) { + validators.add(FAIL); + } + final RequestValidators requestValidators = new RequestValidators<>(validators); + final Optional e = requestValidators.validateRequest(null, null, null); + assertThat(e, OptionalMatchers.isPresent()); + // noinspection OptionalGetWithoutIsPresent + assertThat(e.get().getSuppressed(), Matchers.arrayWithSize(numberOfFailures - 1)); + } + + public void testRandom() { + final int numberOfValidations = randomIntBetween(0, 8); + final int numberOfFailures = randomIntBetween(0, 8); + final List> validators = + new ArrayList<>(numberOfValidations + numberOfFailures); + for (int i = 0; i < numberOfValidations; i++) { + validators.add(EMPTY); + } + for (int i = 0; i < numberOfFailures; i++) { + validators.add(FAIL); + } + Randomness.shuffle(validators); + final RequestValidators requestValidators = new RequestValidators<>(validators); + final Optional e = requestValidators.validateRequest(null, null, null); + if (numberOfFailures == 0) { + assertThat(e, OptionalMatchers.isEmpty()); + } else { + assertThat(e, OptionalMatchers.isPresent()); + // noinspection OptionalGetWithoutIsPresent + assertThat(e.get().getSuppressed(), Matchers.arrayWithSize(numberOfFailures - 1)); + } + } + +} diff --git a/server/src/test/java/org/elasticsearch/action/ShardValidateQueryRequestTests.java b/server/src/test/java/org/elasticsearch/action/ShardValidateQueryRequestTests.java index c884d68a27606..5d7f903a4a3cd 100644 --- a/server/src/test/java/org/elasticsearch/action/ShardValidateQueryRequestTests.java +++ b/server/src/test/java/org/elasticsearch/action/ShardValidateQueryRequestTests.java @@ -43,7 +43,7 @@ public class ShardValidateQueryRequestTests extends ESTestCase { public void setUp() throws Exception { super.setUp(); IndicesModule indicesModule = new IndicesModule(Collections.emptyList()); - SearchModule searchModule = new SearchModule(Settings.EMPTY, false, Collections.emptyList()); + SearchModule searchModule = new SearchModule(Settings.EMPTY, Collections.emptyList()); List entries = new ArrayList<>(); entries.addAll(indicesModule.getNamedWriteables()); entries.addAll(searchModule.getNamedWriteables()); diff --git a/server/src/test/java/org/elasticsearch/action/admin/cluster/configuration/AddVotingConfigExclusionsRequestTests.java b/server/src/test/java/org/elasticsearch/action/admin/cluster/configuration/AddVotingConfigExclusionsRequestTests.java index 52a5e7d3ce3dd..44880d5bdde59 100644 --- a/server/src/test/java/org/elasticsearch/action/admin/cluster/configuration/AddVotingConfigExclusionsRequestTests.java +++ b/server/src/test/java/org/elasticsearch/action/admin/cluster/configuration/AddVotingConfigExclusionsRequestTests.java @@ -25,16 +25,16 @@ import org.elasticsearch.cluster.coordination.CoordinationMetaData.VotingConfigExclusion; import org.elasticsearch.cluster.metadata.MetaData; import org.elasticsearch.cluster.node.DiscoveryNode; -import org.elasticsearch.cluster.node.DiscoveryNode.Role; +import org.elasticsearch.cluster.node.DiscoveryNodeRole; import org.elasticsearch.cluster.node.DiscoveryNodes.Builder; import org.elasticsearch.common.unit.TimeValue; import org.elasticsearch.test.ESTestCase; import java.io.IOException; +import java.util.Set; import static java.util.Collections.emptyMap; import static java.util.Collections.emptySet; -import static java.util.Collections.singleton; import static org.hamcrest.Matchers.contains; import static org.hamcrest.Matchers.containsInAnyOrder; import static org.hamcrest.Matchers.equalTo; @@ -55,14 +55,29 @@ public void testSerialization() throws IOException { } public void testResolve() { - final DiscoveryNode localNode - = new DiscoveryNode("local", "local", buildNewFakeTransportAddress(), emptyMap(), singleton(Role.MASTER), Version.CURRENT); + final DiscoveryNode localNode = new DiscoveryNode( + "local", + "local", + buildNewFakeTransportAddress(), + emptyMap(), + Set.of(DiscoveryNodeRole.MASTER_ROLE), + Version.CURRENT); final VotingConfigExclusion localNodeExclusion = new VotingConfigExclusion(localNode); - final DiscoveryNode otherNode1 - = new DiscoveryNode("other1", "other1", buildNewFakeTransportAddress(), emptyMap(), singleton(Role.MASTER), Version.CURRENT); + final DiscoveryNode otherNode1 = new DiscoveryNode( + "other1", + "other1", + buildNewFakeTransportAddress(), + emptyMap(), + Set.of(DiscoveryNodeRole.MASTER_ROLE), + Version.CURRENT); final VotingConfigExclusion otherNode1Exclusion = new VotingConfigExclusion(otherNode1); - final DiscoveryNode otherNode2 - = new DiscoveryNode("other2", "other2", buildNewFakeTransportAddress(), emptyMap(), singleton(Role.MASTER), Version.CURRENT); + final DiscoveryNode otherNode2 = new DiscoveryNode( + "other2", + "other2", + buildNewFakeTransportAddress(), + emptyMap(), + Set.of(DiscoveryNodeRole.MASTER_ROLE), + Version.CURRENT); final VotingConfigExclusion otherNode2Exclusion = new VotingConfigExclusion(otherNode2); final DiscoveryNode otherDataNode = new DiscoveryNode("data", "data", buildNewFakeTransportAddress(), emptyMap(), emptySet(), Version.CURRENT); @@ -85,14 +100,29 @@ public void testResolve() { } public void testResolveAndCheckMaximum() { - final DiscoveryNode localNode - = new DiscoveryNode("local", "local", buildNewFakeTransportAddress(), emptyMap(), singleton(Role.MASTER), Version.CURRENT); + final DiscoveryNode localNode = new DiscoveryNode( + "local", + "local", + buildNewFakeTransportAddress(), + emptyMap(), + Set.of(DiscoveryNodeRole.MASTER_ROLE), + Version.CURRENT); final VotingConfigExclusion localNodeExclusion = new VotingConfigExclusion(localNode); - final DiscoveryNode otherNode1 - = new DiscoveryNode("other1", "other1", buildNewFakeTransportAddress(), emptyMap(), singleton(Role.MASTER), Version.CURRENT); + final DiscoveryNode otherNode1 = new DiscoveryNode( + "other1", + "other1", + buildNewFakeTransportAddress(), + emptyMap(), + Set.of(DiscoveryNodeRole.MASTER_ROLE), + Version.CURRENT); final VotingConfigExclusion otherNode1Exclusion = new VotingConfigExclusion(otherNode1); - final DiscoveryNode otherNode2 - = new DiscoveryNode("other2", "other2", buildNewFakeTransportAddress(), emptyMap(), singleton(Role.MASTER), Version.CURRENT); + final DiscoveryNode otherNode2 = new DiscoveryNode( + "other2", + "other2", + buildNewFakeTransportAddress(), + emptyMap(), + Set.of(DiscoveryNodeRole.MASTER_ROLE), + Version.CURRENT); final VotingConfigExclusion otherNode2Exclusion = new VotingConfigExclusion(otherNode2); final ClusterState.Builder builder = ClusterState.builder(new ClusterName("cluster")).nodes(new Builder() diff --git a/server/src/test/java/org/elasticsearch/action/admin/cluster/configuration/TransportAddVotingConfigExclusionsActionTests.java b/server/src/test/java/org/elasticsearch/action/admin/cluster/configuration/TransportAddVotingConfigExclusionsActionTests.java index 06d21fa4ba15b..d4dda501f5395 100644 --- a/server/src/test/java/org/elasticsearch/action/admin/cluster/configuration/TransportAddVotingConfigExclusionsActionTests.java +++ b/server/src/test/java/org/elasticsearch/action/admin/cluster/configuration/TransportAddVotingConfigExclusionsActionTests.java @@ -28,12 +28,12 @@ import org.elasticsearch.cluster.ClusterStateObserver.Listener; import org.elasticsearch.cluster.ClusterStateUpdateTask; import org.elasticsearch.cluster.coordination.CoordinationMetaData; -import org.elasticsearch.cluster.coordination.CoordinationMetaData.VotingConfiguration; import org.elasticsearch.cluster.coordination.CoordinationMetaData.VotingConfigExclusion; +import org.elasticsearch.cluster.coordination.CoordinationMetaData.VotingConfiguration; import org.elasticsearch.cluster.metadata.IndexNameExpressionResolver; import org.elasticsearch.cluster.metadata.MetaData; import org.elasticsearch.cluster.node.DiscoveryNode; -import org.elasticsearch.cluster.node.DiscoveryNode.Role; +import org.elasticsearch.cluster.node.DiscoveryNodeRole; import org.elasticsearch.cluster.node.DiscoveryNodes.Builder; import org.elasticsearch.cluster.service.ClusterService; import org.elasticsearch.common.io.stream.StreamInput; @@ -60,7 +60,6 @@ import static java.util.Collections.emptyMap; import static java.util.Collections.emptySet; -import static java.util.Collections.singleton; import static org.elasticsearch.action.admin.cluster.configuration.TransportAddVotingConfigExclusionsAction.MAXIMUM_VOTING_CONFIG_EXCLUSIONS_SETTING; import static org.elasticsearch.cluster.ClusterState.builder; import static org.elasticsearch.test.ClusterServiceUtils.createClusterService; @@ -96,7 +95,13 @@ public static void createThreadPoolAndClusterService() { } private static DiscoveryNode makeDiscoveryNode(String name) { - return new DiscoveryNode(name, name, buildNewFakeTransportAddress(), emptyMap(), singleton(Role.MASTER), Version.CURRENT); + return new DiscoveryNode( + name, + name, + buildNewFakeTransportAddress(), + emptyMap(), + Set.of(DiscoveryNodeRole.MASTER_ROLE), + Version.CURRENT); } @AfterClass diff --git a/server/src/test/java/org/elasticsearch/action/admin/cluster/node/tasks/CancellableTasksTests.java b/server/src/test/java/org/elasticsearch/action/admin/cluster/node/tasks/CancellableTasksTests.java index c52fc58b90079..0a1e5c512df88 100644 --- a/server/src/test/java/org/elasticsearch/action/admin/cluster/node/tasks/CancellableTasksTests.java +++ b/server/src/test/java/org/elasticsearch/action/admin/cluster/node/tasks/CancellableTasksTests.java @@ -60,35 +60,30 @@ public class CancellableTasksTests extends TaskManagerTestCase { public static class CancellableNodeRequest extends BaseNodeRequest { protected String requestName; - protected String nodeId; public CancellableNodeRequest() { super(); } - public CancellableNodeRequest(CancellableNodesRequest request, String nodeId) { - super(nodeId); + public CancellableNodeRequest(CancellableNodesRequest request) { requestName = request.requestName; - this.nodeId = nodeId; } @Override public void readFrom(StreamInput in) throws IOException { super.readFrom(in); requestName = in.readString(); - nodeId = in.readString(); } @Override public void writeTo(StreamOutput out) throws IOException { super.writeTo(out); out.writeString(requestName); - out.writeString(nodeId); } @Override public String getDescription() { - return "CancellableNodeRequest[" + requestName + ", " + nodeId + "]"; + return "CancellableNodeRequest[" + requestName + "]"; } @Override @@ -161,19 +156,19 @@ class CancellableTestNodesAction extends AbstractTestNodesAction @@ -426,9 +416,9 @@ public void onFailure(Exception e) { } - private static void debugDelay(String nodeId, String name) { + private static void debugDelay(String name) { // Introduce an additional pseudo random repeatable race conditions - String delayName = RandomizedContext.current().getRunnerSeedAsString() + ":" + nodeId + ":" + name; + String delayName = RandomizedContext.current().getRunnerSeedAsString() + ":" + name; Random random = new Random(delayName.hashCode()); if (RandomNumbers.randomIntBetween(random, 0, 10) < 1) { try { diff --git a/server/src/test/java/org/elasticsearch/action/admin/cluster/node/tasks/TaskManagerTestCase.java b/server/src/test/java/org/elasticsearch/action/admin/cluster/node/tasks/TaskManagerTestCase.java index ecca51c7bbb83..bfaac83d51913 100644 --- a/server/src/test/java/org/elasticsearch/action/admin/cluster/node/tasks/TaskManagerTestCase.java +++ b/server/src/test/java/org/elasticsearch/action/admin/cluster/node/tasks/TaskManagerTestCase.java @@ -43,6 +43,7 @@ import org.elasticsearch.common.transport.BoundTransportAddress; import org.elasticsearch.common.util.PageCacheRecycler; import org.elasticsearch.indices.breaker.NoneCircuitBreakerService; +import org.elasticsearch.tasks.Task; import org.elasticsearch.tasks.TaskManager; import org.elasticsearch.test.ESTestCase; import org.elasticsearch.test.tasks.MockTaskManager; @@ -156,7 +157,7 @@ protected NodeResponse newNodeResponse() { } @Override - protected abstract NodeResponse nodeOperation(NodeRequest request); + protected abstract NodeResponse nodeOperation(NodeRequest request, Task task); } diff --git a/server/src/test/java/org/elasticsearch/action/admin/cluster/node/tasks/TasksIT.java b/server/src/test/java/org/elasticsearch/action/admin/cluster/node/tasks/TasksIT.java index fc758788e6197..40d5b028b4ba8 100644 --- a/server/src/test/java/org/elasticsearch/action/admin/cluster/node/tasks/TasksIT.java +++ b/server/src/test/java/org/elasticsearch/action/admin/cluster/node/tasks/TasksIT.java @@ -354,12 +354,12 @@ public void testSearchTaskDescriptions() { headers.put("Foo-Header", "bar"); headers.put("Custom-Task-Header", "my_value"); assertSearchResponse( - client().filterWithHeader(headers).prepareSearch("test").setTypes("doc").setQuery(QueryBuilders.matchAllQuery()).get()); + client().filterWithHeader(headers).prepareSearch("test").setQuery(QueryBuilders.matchAllQuery()).get()); // the search operation should produce one main task List mainTask = findEvents(SearchAction.NAME, Tuple::v1); assertEquals(1, mainTask.size()); - assertThat(mainTask.get(0).getDescription(), startsWith("indices[test], types[doc], search_type[")); + assertThat(mainTask.get(0).getDescription(), startsWith("indices[test], search_type[")); assertThat(mainTask.get(0).getDescription(), containsString("\"query\":{\"match_all\"")); assertTaskHeaders(mainTask.get(0)); @@ -747,13 +747,12 @@ public void testTaskStoringSuccesfulResult() throws Exception { assertNoFailures(client().admin().indices().prepareRefresh(TaskResultsService.TASK_INDEX).get()); SearchResponse searchResponse = client().prepareSearch(TaskResultsService.TASK_INDEX) - .setTypes(TaskResultsService.TASK_TYPE) .setSource(SearchSourceBuilder.searchSource().query(QueryBuilders.termQuery("task.action", taskInfo.getAction()))) .get(); assertEquals(1L, searchResponse.getHits().getTotalHits().value); - searchResponse = client().prepareSearch(TaskResultsService.TASK_INDEX).setTypes(TaskResultsService.TASK_TYPE) + searchResponse = client().prepareSearch(TaskResultsService.TASK_INDEX) .setSource(SearchSourceBuilder.searchSource().query(QueryBuilders.termQuery("task.node", taskInfo.getTaskId().getNodeId()))) .get(); diff --git a/server/src/test/java/org/elasticsearch/action/admin/cluster/node/tasks/TestTaskPlugin.java b/server/src/test/java/org/elasticsearch/action/admin/cluster/node/tasks/TestTaskPlugin.java index d39ab823dc490..d9c142498c917 100644 --- a/server/src/test/java/org/elasticsearch/action/admin/cluster/node/tasks/TestTaskPlugin.java +++ b/server/src/test/java/org/elasticsearch/action/admin/cluster/node/tasks/TestTaskPlugin.java @@ -18,13 +18,14 @@ */ package org.elasticsearch.action.admin.cluster.node.tasks; -import org.elasticsearch.action.Action; +import org.elasticsearch.action.ActionType; import org.elasticsearch.action.ActionListener; import org.elasticsearch.action.ActionRequest; import org.elasticsearch.action.ActionRequestBuilder; import org.elasticsearch.action.ActionResponse; import org.elasticsearch.action.FailedNodeException; import org.elasticsearch.action.IndicesRequest; +import org.elasticsearch.action.StreamableResponseActionType; import org.elasticsearch.action.TaskOperationFailure; import org.elasticsearch.action.support.ActionFilters; import org.elasticsearch.action.support.nodes.BaseNodeRequest; @@ -169,17 +170,14 @@ public XContentBuilder toXContent(XContentBuilder builder, Params params) throws public static class NodeRequest extends BaseNodeRequest { protected String requestName; - protected String nodeId; protected boolean shouldBlock; public NodeRequest() { super(); } - public NodeRequest(NodesRequest request, String nodeId, boolean shouldBlock) { - super(nodeId); + public NodeRequest(NodesRequest request, boolean shouldBlock) { requestName = request.requestName; - this.nodeId = nodeId; this.shouldBlock = shouldBlock; } @@ -187,7 +185,6 @@ public NodeRequest(NodesRequest request, String nodeId, boolean shouldBlock) { public void readFrom(StreamInput in) throws IOException { super.readFrom(in); requestName = in.readString(); - nodeId = in.readString(); shouldBlock = in.readBoolean(); } @@ -195,13 +192,12 @@ public void readFrom(StreamInput in) throws IOException { public void writeTo(StreamOutput out) throws IOException { super.writeTo(out); out.writeString(requestName); - out.writeString(nodeId); out.writeBoolean(shouldBlock); } @Override public String getDescription() { - return "NodeRequest[" + requestName + ", " + nodeId + "]"; + return "NodeRequest[" + requestName + "]"; } @Override @@ -301,8 +297,8 @@ protected NodesResponse newResponse(NodesRequest request, List res } @Override - protected NodeRequest newNodeRequest(String nodeId, NodesRequest request) { - return new NodeRequest(request, nodeId, request.getShouldBlock()); + protected NodeRequest newNodeRequest(NodesRequest request) { + return new NodeRequest(request, request.getShouldBlock()); } @Override @@ -333,15 +329,9 @@ protected NodeResponse nodeOperation(NodeRequest request, Task task) { logger.info("Test task finished on the node {}", clusterService.localNode()); return new NodeResponse(clusterService.localNode()); } - - @Override - protected NodeResponse nodeOperation(NodeRequest request) { - throw new UnsupportedOperationException("the task parameter is required"); - } - } - public static class TestTaskAction extends Action { + public static class TestTaskAction extends StreamableResponseActionType { public static final TestTaskAction INSTANCE = new TestTaskAction(); public static final String NAME = "cluster:admin/tasks/test"; @@ -358,7 +348,7 @@ public NodesResponse newResponse() { public static class NodesRequestBuilder extends NodesOperationRequestBuilder { - protected NodesRequestBuilder(ElasticsearchClient client, Action action) { + protected NodesRequestBuilder(ElasticsearchClient client, ActionType action) { super(client, action, new NodesRequest("test")); } @@ -466,7 +456,7 @@ protected void taskOperation(UnblockTestTasksRequest request, Task task, ActionL } - public static class UnblockTestTasksAction extends Action { + public static class UnblockTestTasksAction extends ActionType { public static final UnblockTestTasksAction INSTANCE = new UnblockTestTasksAction(); public static final String NAME = "cluster:admin/tasks/testunblock"; @@ -475,11 +465,6 @@ private UnblockTestTasksAction() { super(NAME); } - @Override - public UnblockTestTasksResponse newResponse() { - throw new UnsupportedOperationException("usage of Streamable is to be replaced by Writeable"); - } - @Override public Writeable.Reader getResponseReader() { return UnblockTestTasksResponse::new; @@ -489,7 +474,7 @@ public Writeable.Reader getResponseReader() { public static class UnblockTestTasksRequestBuilder extends ActionRequestBuilder { protected UnblockTestTasksRequestBuilder(ElasticsearchClient client, - Action action) { + ActionType action) { super(client, action, new UnblockTestTasksRequest()); } } diff --git a/server/src/test/java/org/elasticsearch/action/admin/cluster/node/tasks/TransportTasksActionTests.java b/server/src/test/java/org/elasticsearch/action/admin/cluster/node/tasks/TransportTasksActionTests.java index b883d593352c2..6d8749ad4f4c8 100644 --- a/server/src/test/java/org/elasticsearch/action/admin/cluster/node/tasks/TransportTasksActionTests.java +++ b/server/src/test/java/org/elasticsearch/action/admin/cluster/node/tasks/TransportTasksActionTests.java @@ -84,8 +84,7 @@ public NodeRequest() { super(); } - public NodeRequest(NodesRequest request, String nodeId) { - super(nodeId); + public NodeRequest(NodesRequest request) { requestName = request.requestName; } @@ -157,8 +156,8 @@ abstract class TestNodesAction extends AbstractTestNodesAction { +public class ClusterUpdateSettingsResponseTests extends AbstractSerializingTestCase { @Override protected ClusterUpdateSettingsResponse doParseInstance(XContentParser parser) { @@ -93,8 +94,7 @@ protected ClusterUpdateSettingsResponse createTestInstance() { } @Override - protected ClusterUpdateSettingsResponse createBlankInstance() { - return new ClusterUpdateSettingsResponse(); + protected Writeable.Reader instanceReader() { + return ClusterUpdateSettingsResponse::new; } - } diff --git a/server/src/test/java/org/elasticsearch/action/admin/cluster/shards/ClusterSearchShardsResponseTests.java b/server/src/test/java/org/elasticsearch/action/admin/cluster/shards/ClusterSearchShardsResponseTests.java index 534cea2ae4604..c83caae49a72f 100644 --- a/server/src/test/java/org/elasticsearch/action/admin/cluster/shards/ClusterSearchShardsResponseTests.java +++ b/server/src/test/java/org/elasticsearch/action/admin/cluster/shards/ClusterSearchShardsResponseTests.java @@ -73,7 +73,7 @@ public void testSerialization() throws Exception { ClusterSearchShardsResponse clusterSearchShardsResponse = new ClusterSearchShardsResponse(clusterSearchShardsGroups, nodes.toArray(new DiscoveryNode[nodes.size()]), indicesAndFilters); - SearchModule searchModule = new SearchModule(Settings.EMPTY, false, Collections.emptyList()); + SearchModule searchModule = new SearchModule(Settings.EMPTY, Collections.emptyList()); List entries = new ArrayList<>(); entries.addAll(searchModule.getNamedWriteables()); NamedWriteableRegistry namedWriteableRegistry = new NamedWriteableRegistry(entries); diff --git a/server/src/test/java/org/elasticsearch/action/admin/cluster/snapshots/SnapshotBlocksIT.java b/server/src/test/java/org/elasticsearch/action/admin/cluster/snapshots/SnapshotBlocksIT.java index 7dac3a38dddd7..9be5c23bb1bcd 100644 --- a/server/src/test/java/org/elasticsearch/action/admin/cluster/snapshots/SnapshotBlocksIT.java +++ b/server/src/test/java/org/elasticsearch/action/admin/cluster/snapshots/SnapshotBlocksIT.java @@ -132,7 +132,8 @@ public void testDeleteSnapshotWithBlocks() { public void testRestoreSnapshotWithBlocks() { assertAcked(client().admin().indices().prepareDelete(INDEX_NAME, OTHER_INDEX_NAME)); - assertFalse(client().admin().indices().prepareExists(INDEX_NAME, OTHER_INDEX_NAME).get().isExists()); + assertFalse(indexExists(INDEX_NAME)); + assertFalse(indexExists(OTHER_INDEX_NAME)); logger.info("--> restoring a snapshot is blocked when the cluster is read only"); try { @@ -148,8 +149,8 @@ public void testRestoreSnapshotWithBlocks() { .setWaitForCompletion(true) .execute().actionGet(); assertThat(response.status(), equalTo(RestStatus.OK)); - assertTrue(client().admin().indices().prepareExists(INDEX_NAME).get().isExists()); - assertTrue(client().admin().indices().prepareExists(OTHER_INDEX_NAME).get().isExists()); + assertTrue(indexExists(INDEX_NAME)); + assertTrue(indexExists(OTHER_INDEX_NAME)); } public void testGetSnapshotWithBlocks() { @@ -157,8 +158,8 @@ public void testGetSnapshotWithBlocks() { try { setClusterReadOnly(true); GetSnapshotsResponse response = client().admin().cluster().prepareGetSnapshots(REPOSITORY_NAME).execute().actionGet(); - assertThat(response.getSnapshots(), hasSize(1)); - assertThat(response.getSnapshots().get(0).snapshotId().getName(), equalTo(SNAPSHOT_NAME)); + assertThat(response.getSnapshots(REPOSITORY_NAME), hasSize(1)); + assertThat(response.getSnapshots(REPOSITORY_NAME).get(0).snapshotId().getName(), equalTo(SNAPSHOT_NAME)); } finally { setClusterReadOnly(false); } diff --git a/server/src/test/java/org/elasticsearch/action/admin/cluster/snapshots/create/CreateSnapshotRequestTests.java b/server/src/test/java/org/elasticsearch/action/admin/cluster/snapshots/create/CreateSnapshotRequestTests.java index 0b598be6849cb..9f7bd5f6a0149 100644 --- a/server/src/test/java/org/elasticsearch/action/admin/cluster/snapshots/create/CreateSnapshotRequestTests.java +++ b/server/src/test/java/org/elasticsearch/action/admin/cluster/snapshots/create/CreateSnapshotRequestTests.java @@ -19,10 +19,12 @@ package org.elasticsearch.action.admin.cluster.snapshots.create; +import org.elasticsearch.action.ActionRequestValidationException; import org.elasticsearch.action.support.IndicesOptions; import org.elasticsearch.action.support.IndicesOptions.Option; import org.elasticsearch.action.support.IndicesOptions.WildcardStates; import org.elasticsearch.common.bytes.BytesReference; +import org.elasticsearch.common.settings.Settings; import org.elasticsearch.common.xcontent.NamedXContentRegistry; import org.elasticsearch.common.xcontent.ToXContent.MapParams; import org.elasticsearch.common.xcontent.XContentBuilder; @@ -41,6 +43,10 @@ import java.util.List; import java.util.Map; +import static org.elasticsearch.snapshots.SnapshotInfoTests.randomUserMetadata; +import static org.hamcrest.Matchers.equalTo; +import static org.hamcrest.Matchers.hasSize; + public class CreateSnapshotRequestTests extends ESTestCase { // tests creating XContent and parsing with source(Map) equivalency @@ -80,6 +86,10 @@ public void testToXContent() throws IOException { original.includeGlobalState(randomBoolean()); } + if (randomBoolean()) { + original.userMetadata(randomUserMetadata()); + } + if (randomBoolean()) { Collection wildcardStates = randomSubsetOf(Arrays.asList(WildcardStates.values())); Collection