summaryrefslogtreecommitdiff
path: root/core/src
diff options
context:
space:
mode:
authorChristoph Büscher <christoph@elastic.co>2017-05-15 12:10:52 +0200
committerChristoph Büscher <christoph@elastic.co>2017-05-15 12:25:07 +0200
commit42e8d4b76109980d1974f1739263e6792981cc20 (patch)
treeb70abd5d75a2ae6b824709524d0ddfdce095efb4 /core/src
parentbb59ee51b07e61a19d8b359cf365fb386c8c4b3d (diff)
parentfdb6cd8088cb00ff09552c60f42d44c623490fd4 (diff)
Merge branch 'master' into feature/client_aggs_parsing
Conflicts: core/src/test/java/org/elasticsearch/search/aggregations/bucket/filter/InternalFilterTests.java core/src/test/java/org/elasticsearch/search/aggregations/bucket/global/InternalGlobalTests.java core/src/test/java/org/elasticsearch/search/aggregations/bucket/missing/InternalMissingTests.java core/src/test/java/org/elasticsearch/search/aggregations/bucket/nested/InternalNestedTests.java core/src/test/java/org/elasticsearch/search/aggregations/bucket/nested/InternalReverseNestedTests.java core/src/test/java/org/elasticsearch/search/aggregations/bucket/sampler/InternalSamplerTests.java modules/parent-join/src/test/java/org/elasticsearch/join/aggregations/InternalChildrenTests.java test/framework/src/main/java/org/elasticsearch/search/aggregations/InternalSingleBucketAggregationTestCase.java
Diffstat (limited to 'core/src')
-rw-r--r--core/src/main/java/org/elasticsearch/action/admin/cluster/repositories/get/GetRepositoriesResponse.java2
-rw-r--r--core/src/main/java/org/elasticsearch/action/admin/cluster/snapshots/get/GetSnapshotsResponse.java2
-rw-r--r--core/src/main/java/org/elasticsearch/action/admin/cluster/stats/ClusterStatsNodes.java4
-rw-r--r--core/src/main/java/org/elasticsearch/action/admin/indices/get/GetIndexResponse.java2
-rw-r--r--core/src/main/java/org/elasticsearch/action/support/tasks/BaseTasksResponse.java4
-rw-r--r--core/src/main/java/org/elasticsearch/bootstrap/Bootstrap.java9
-rw-r--r--core/src/main/java/org/elasticsearch/client/transport/TransportClientNodesService.java18
-rw-r--r--core/src/main/java/org/elasticsearch/cluster/block/ClusterBlock.java2
-rw-r--r--core/src/main/java/org/elasticsearch/cluster/metadata/IndexNameExpressionResolver.java10
-rw-r--r--core/src/main/java/org/elasticsearch/cluster/service/ClusterApplier.java7
-rw-r--r--core/src/main/java/org/elasticsearch/cluster/service/ClusterApplierService.java1
-rw-r--r--core/src/main/java/org/elasticsearch/common/inject/ModulesBuilder.java5
-rw-r--r--core/src/main/java/org/elasticsearch/common/inject/assistedinject/AssistedConstructor.java2
-rw-r--r--core/src/main/java/org/elasticsearch/common/lucene/search/MultiPhrasePrefixQuery.java4
-rw-r--r--core/src/main/java/org/elasticsearch/discovery/Discovery.java18
-rw-r--r--core/src/main/java/org/elasticsearch/discovery/DiscoveryModule.java4
-rw-r--r--core/src/main/java/org/elasticsearch/discovery/TribeDiscovery.java30
-rw-r--r--core/src/main/java/org/elasticsearch/discovery/single/SingleNodeDiscovery.java55
-rw-r--r--core/src/main/java/org/elasticsearch/discovery/zen/UnicastZenPing.java8
-rw-r--r--core/src/main/java/org/elasticsearch/discovery/zen/ZenDiscovery.java69
-rw-r--r--core/src/main/java/org/elasticsearch/discovery/zen/ZenPing.java2
-rw-r--r--core/src/main/java/org/elasticsearch/env/Environment.java13
-rw-r--r--core/src/main/java/org/elasticsearch/gateway/Gateway.java11
-rw-r--r--core/src/main/java/org/elasticsearch/gateway/GatewayService.java9
-rw-r--r--core/src/main/java/org/elasticsearch/index/IndexWarmer.java5
-rw-r--r--core/src/main/java/org/elasticsearch/index/analysis/CustomAnalyzerProvider.java4
-rw-r--r--core/src/main/java/org/elasticsearch/index/analysis/CustomNormalizerProvider.java4
-rw-r--r--core/src/main/java/org/elasticsearch/index/mapper/DynamicTemplate.java2
-rw-r--r--core/src/main/java/org/elasticsearch/index/query/AbstractQueryBuilder.java2
-rw-r--r--core/src/main/java/org/elasticsearch/index/query/GeoPolygonQueryBuilder.java2
-rw-r--r--core/src/main/java/org/elasticsearch/index/query/HasChildQueryBuilder.java490
-rw-r--r--core/src/main/java/org/elasticsearch/index/query/HasParentQueryBuilder.java321
-rw-r--r--core/src/main/java/org/elasticsearch/index/query/InnerHitBuilder.java6
-rw-r--r--core/src/main/java/org/elasticsearch/index/query/NestedQueryBuilder.java30
-rw-r--r--core/src/main/java/org/elasticsearch/index/query/QueryBuilders.java24
-rw-r--r--core/src/main/java/org/elasticsearch/index/snapshots/blobstore/BlobStoreIndexShardSnapshots.java2
-rw-r--r--core/src/main/java/org/elasticsearch/indices/breaker/HierarchyCircuitBreakerService.java2
-rw-r--r--core/src/main/java/org/elasticsearch/indices/fielddata/cache/IndicesFieldDataCache.java9
-rw-r--r--core/src/main/java/org/elasticsearch/node/InternalSettingsPreparer.java35
-rw-r--r--core/src/main/java/org/elasticsearch/node/Node.java4
-rw-r--r--core/src/main/java/org/elasticsearch/search/SearchModule.java12
-rw-r--r--core/src/main/java/org/elasticsearch/search/aggregations/AggregationBuilders.java17
-rw-r--r--core/src/main/java/org/elasticsearch/search/aggregations/AggregatorFactories.java2
-rw-r--r--core/src/main/java/org/elasticsearch/search/aggregations/BucketCollector.java2
-rw-r--r--core/src/main/java/org/elasticsearch/search/aggregations/bucket/adjacency/AdjacencyMatrixAggregationBuilder.java2
-rw-r--r--core/src/main/java/org/elasticsearch/search/aggregations/bucket/children/Children.java28
-rw-r--r--core/src/main/java/org/elasticsearch/search/aggregations/bucket/children/ChildrenAggregationBuilder.java167
-rw-r--r--core/src/main/java/org/elasticsearch/search/aggregations/bucket/children/ChildrenAggregatorFactory.java77
-rw-r--r--core/src/main/java/org/elasticsearch/search/aggregations/bucket/children/InternalChildren.java56
-rw-r--r--core/src/main/java/org/elasticsearch/search/aggregations/bucket/children/ParentToChildrenAggregator.java186
-rw-r--r--core/src/main/java/org/elasticsearch/search/aggregations/bucket/children/ParsedChildren.java36
-rw-r--r--core/src/main/java/org/elasticsearch/search/aggregations/bucket/filters/FiltersAggregationBuilder.java2
-rw-r--r--core/src/main/java/org/elasticsearch/search/aggregations/bucket/terms/DoubleTerms.java2
-rw-r--r--core/src/main/java/org/elasticsearch/search/aggregations/pipeline/cumulativesum/CumulativeSumPipelineAggregator.java2
-rw-r--r--core/src/main/java/org/elasticsearch/search/fetch/subphase/highlight/CustomQueryScorer.java11
-rw-r--r--core/src/main/java/org/elasticsearch/search/suggest/phrase/DirectCandidateGenerator.java5
-rw-r--r--core/src/main/java/org/elasticsearch/transport/TcpTransport.java16
-rw-r--r--core/src/main/java/org/elasticsearch/transport/Transport.java8
-rw-r--r--core/src/main/java/org/elasticsearch/transport/TransportConnectionListener.java8
-rw-r--r--core/src/main/java/org/elasticsearch/transport/TransportService.java39
-rw-r--r--core/src/test/java/org/elasticsearch/action/bulk/BulkWithUpdatesIT.java149
-rw-r--r--core/src/test/java/org/elasticsearch/aliases/IndexAliasesIT.java23
-rw-r--r--core/src/test/java/org/elasticsearch/bootstrap/BootstrapTests.java33
-rw-r--r--core/src/test/java/org/elasticsearch/client/transport/TransportClientNodesServiceTests.java186
-rw-r--r--core/src/test/java/org/elasticsearch/cluster/metadata/IndexNameExpressionResolverTests.java27
-rw-r--r--core/src/test/java/org/elasticsearch/cluster/metadata/WildcardExpressionResolverTests.java25
-rw-r--r--core/src/test/java/org/elasticsearch/discovery/single/SingleNodeDiscoveryIT.java18
-rw-r--r--core/src/test/java/org/elasticsearch/discovery/single/SingleNodeDiscoveryTests.java30
-rw-r--r--core/src/test/java/org/elasticsearch/discovery/zen/UnicastZenPingTests.java60
-rw-r--r--core/src/test/java/org/elasticsearch/discovery/zen/ZenDiscoveryUnitTests.java25
-rw-r--r--core/src/test/java/org/elasticsearch/gateway/GatewayServiceTests.java3
-rw-r--r--core/src/test/java/org/elasticsearch/index/query/HasChildQueryBuilderTests.java349
-rw-r--r--core/src/test/java/org/elasticsearch/index/query/HasParentQueryBuilderTests.java227
-rw-r--r--core/src/test/java/org/elasticsearch/index/query/InnerHitBuilderTests.java74
-rw-r--r--core/src/test/java/org/elasticsearch/index/query/NestedQueryBuilderTests.java44
-rw-r--r--core/src/test/java/org/elasticsearch/node/InternalSettingsPreparerTests.java30
-rw-r--r--core/src/test/java/org/elasticsearch/search/SearchModuleTests.java2
-rw-r--r--core/src/test/java/org/elasticsearch/search/aggregations/AggregationsTests.java9
-rw-r--r--core/src/test/java/org/elasticsearch/search/aggregations/AggregatorTestCase.java303
-rw-r--r--core/src/test/java/org/elasticsearch/search/aggregations/BaseAggregationTestCase.java198
-rw-r--r--core/src/test/java/org/elasticsearch/search/aggregations/bucket/ChildrenIT.java472
-rw-r--r--core/src/test/java/org/elasticsearch/search/aggregations/bucket/ChildrenTests.java35
-rw-r--r--core/src/test/java/org/elasticsearch/search/aggregations/bucket/InternalSingleBucketAggregationTestCase.java139
-rw-r--r--core/src/test/java/org/elasticsearch/search/aggregations/bucket/children/InternalChildrenTests.java52
-rw-r--r--core/src/test/java/org/elasticsearch/search/aggregations/bucket/children/ParentToChildrenAggregatorTests.java190
-rw-r--r--core/src/test/java/org/elasticsearch/search/aggregations/bucket/filter/InternalFilterTests.java2
-rw-r--r--core/src/test/java/org/elasticsearch/search/aggregations/bucket/global/InternalGlobalTests.java2
-rw-r--r--core/src/test/java/org/elasticsearch/search/aggregations/bucket/missing/InternalMissingTests.java2
-rw-r--r--core/src/test/java/org/elasticsearch/search/aggregations/bucket/nested/InternalNestedTests.java2
-rw-r--r--core/src/test/java/org/elasticsearch/search/aggregations/bucket/nested/InternalReverseNestedTests.java2
-rw-r--r--core/src/test/java/org/elasticsearch/search/aggregations/bucket/sampler/InternalSamplerTests.java4
-rw-r--r--core/src/test/java/org/elasticsearch/search/child/ChildQuerySearchIT.java1991
-rw-r--r--core/src/test/java/org/elasticsearch/search/fetch/subphase/InnerHitsIT.java458
-rw-r--r--core/src/test/java/org/elasticsearch/test/NoopDiscovery.java17
-rw-r--r--core/src/test/java/org/elasticsearch/transport/TCPTransportTests.java4
-rw-r--r--core/src/test/resources/config/elasticsearch.properties2
-rw-r--r--core/src/test/resources/config/elasticsearch.yml (renamed from core/src/test/resources/config/elasticsearch.yaml)0
97 files changed, 615 insertions, 6485 deletions
diff --git a/core/src/main/java/org/elasticsearch/action/admin/cluster/repositories/get/GetRepositoriesResponse.java b/core/src/main/java/org/elasticsearch/action/admin/cluster/repositories/get/GetRepositoriesResponse.java
index c933156fcb..6d4cb83934 100644
--- a/core/src/main/java/org/elasticsearch/action/admin/cluster/repositories/get/GetRepositoriesResponse.java
+++ b/core/src/main/java/org/elasticsearch/action/admin/cluster/repositories/get/GetRepositoriesResponse.java
@@ -60,7 +60,7 @@ public class GetRepositoriesResponse extends ActionResponse implements Iterable<
public void readFrom(StreamInput in) throws IOException {
super.readFrom(in);
int size = in.readVInt();
- List<RepositoryMetaData> repositoryListBuilder = new ArrayList<>();
+ List<RepositoryMetaData> repositoryListBuilder = new ArrayList<>(size);
for (int j = 0; j < size; j++) {
repositoryListBuilder.add(new RepositoryMetaData(
in.readString(),
diff --git a/core/src/main/java/org/elasticsearch/action/admin/cluster/snapshots/get/GetSnapshotsResponse.java b/core/src/main/java/org/elasticsearch/action/admin/cluster/snapshots/get/GetSnapshotsResponse.java
index 308a846c90..0d1e5eda7f 100644
--- a/core/src/main/java/org/elasticsearch/action/admin/cluster/snapshots/get/GetSnapshotsResponse.java
+++ b/core/src/main/java/org/elasticsearch/action/admin/cluster/snapshots/get/GetSnapshotsResponse.java
@@ -59,7 +59,7 @@ public class GetSnapshotsResponse extends ActionResponse implements ToXContentOb
public void readFrom(StreamInput in) throws IOException {
super.readFrom(in);
int size = in.readVInt();
- List<SnapshotInfo> builder = new ArrayList<>();
+ List<SnapshotInfo> builder = new ArrayList<>(size);
for (int i = 0; i < size; i++) {
builder.add(new SnapshotInfo(in));
}
diff --git a/core/src/main/java/org/elasticsearch/action/admin/cluster/stats/ClusterStatsNodes.java b/core/src/main/java/org/elasticsearch/action/admin/cluster/stats/ClusterStatsNodes.java
index 27276b27dd..41cacf2a85 100644
--- a/core/src/main/java/org/elasticsearch/action/admin/cluster/stats/ClusterStatsNodes.java
+++ b/core/src/main/java/org/elasticsearch/action/admin/cluster/stats/ClusterStatsNodes.java
@@ -65,8 +65,8 @@ public class ClusterStatsNodes implements ToXContent {
this.plugins = new HashSet<>();
Set<InetAddress> seenAddresses = new HashSet<>(nodeResponses.size());
- List<NodeInfo> nodeInfos = new ArrayList<>();
- List<NodeStats> nodeStats = new ArrayList<>();
+ List<NodeInfo> nodeInfos = new ArrayList<>(nodeResponses.size());
+ List<NodeStats> nodeStats = new ArrayList<>(nodeResponses.size());
for (ClusterStatsNodeResponse nodeResponse : nodeResponses) {
nodeInfos.add(nodeResponse.nodeInfo());
nodeStats.add(nodeResponse.nodeStats());
diff --git a/core/src/main/java/org/elasticsearch/action/admin/indices/get/GetIndexResponse.java b/core/src/main/java/org/elasticsearch/action/admin/indices/get/GetIndexResponse.java
index 6c2e462752..36bfa81a33 100644
--- a/core/src/main/java/org/elasticsearch/action/admin/indices/get/GetIndexResponse.java
+++ b/core/src/main/java/org/elasticsearch/action/admin/indices/get/GetIndexResponse.java
@@ -114,7 +114,7 @@ public class GetIndexResponse extends ActionResponse {
for (int i = 0; i < aliasesSize; i++) {
String key = in.readString();
int valueSize = in.readVInt();
- List<AliasMetaData> aliasEntryBuilder = new ArrayList<>();
+ List<AliasMetaData> aliasEntryBuilder = new ArrayList<>(valueSize);
for (int j = 0; j < valueSize; j++) {
aliasEntryBuilder.add(new AliasMetaData(in));
}
diff --git a/core/src/main/java/org/elasticsearch/action/support/tasks/BaseTasksResponse.java b/core/src/main/java/org/elasticsearch/action/support/tasks/BaseTasksResponse.java
index b62cfd714b..4ddbe54199 100644
--- a/core/src/main/java/org/elasticsearch/action/support/tasks/BaseTasksResponse.java
+++ b/core/src/main/java/org/elasticsearch/action/support/tasks/BaseTasksResponse.java
@@ -81,13 +81,13 @@ public class BaseTasksResponse extends ActionResponse {
public void readFrom(StreamInput in) throws IOException {
super.readFrom(in);
int size = in.readVInt();
- List<TaskOperationFailure> taskFailures = new ArrayList<>();
+ List<TaskOperationFailure> taskFailures = new ArrayList<>(size);
for (int i = 0; i < size; i++) {
taskFailures.add(new TaskOperationFailure(in));
}
size = in.readVInt();
this.taskFailures = Collections.unmodifiableList(taskFailures);
- List<FailedNodeException> nodeFailures = new ArrayList<>();
+ List<FailedNodeException> nodeFailures = new ArrayList<>(size);
for (int i = 0; i < size; i++) {
nodeFailures.add(new FailedNodeException(in));
}
diff --git a/core/src/main/java/org/elasticsearch/bootstrap/Bootstrap.java b/core/src/main/java/org/elasticsearch/bootstrap/Bootstrap.java
index d0462efc39..74fc600d62 100644
--- a/core/src/main/java/org/elasticsearch/bootstrap/Bootstrap.java
+++ b/core/src/main/java/org/elasticsearch/bootstrap/Bootstrap.java
@@ -298,7 +298,6 @@ final class Bootstrap {
throw new BootstrapException(e);
}
checkForCustomConfFile();
- checkConfigExtension(environment.configExtension());
if (environment.pidFile() != null) {
try {
@@ -414,14 +413,6 @@ final class Bootstrap {
}
}
- // pkg private for tests
- static void checkConfigExtension(String extension) {
- if (".yml".equals(extension) || ".json".equals(extension)) {
- final DeprecationLogger deprecationLogger = new DeprecationLogger(Loggers.getLogger(Bootstrap.class));
- deprecationLogger.deprecated("elasticsearch{} is deprecated; rename your configuration file to elasticsearch.yaml", extension);
- }
- }
-
@SuppressForbidden(reason = "Allowed to exit explicitly in bootstrap phase")
private static void exit(int status) {
System.exit(status);
diff --git a/core/src/main/java/org/elasticsearch/client/transport/TransportClientNodesService.java b/core/src/main/java/org/elasticsearch/client/transport/TransportClientNodesService.java
index 145208baf0..3e50b4d74c 100644
--- a/core/src/main/java/org/elasticsearch/client/transport/TransportClientNodesService.java
+++ b/core/src/main/java/org/elasticsearch/client/transport/TransportClientNodesService.java
@@ -469,14 +469,17 @@ final class TransportClientNodesService extends AbstractComponent implements Clo
*/
Transport.Connection connectionToClose = null;
- @Override
- public void onAfter() {
- IOUtils.closeWhileHandlingException(connectionToClose);
+ void onDone() {
+ try {
+ IOUtils.closeWhileHandlingException(connectionToClose);
+ } finally {
+ latch.countDown();
+ }
}
@Override
public void onFailure(Exception e) {
- latch.countDown();
+ onDone();
if (e instanceof ConnectTransportException) {
logger.debug((Supplier<?>)
() -> new ParameterizedMessage("failed to connect to node [{}], ignoring...", nodeToPing), e);
@@ -522,7 +525,7 @@ final class TransportClientNodesService extends AbstractComponent implements Clo
@Override
public void handleResponse(ClusterStateResponse response) {
clusterStateResponses.put(nodeToPing, response);
- latch.countDown();
+ onDone();
}
@Override
@@ -532,9 +535,8 @@ final class TransportClientNodesService extends AbstractComponent implements Clo
"failed to get local cluster state for {}, disconnecting...", nodeToPing), e);
try {
hostFailureListener.onNodeDisconnected(nodeToPing, e);
- }
- finally {
- latch.countDown();
+ } finally {
+ onDone();
}
}
});
diff --git a/core/src/main/java/org/elasticsearch/cluster/block/ClusterBlock.java b/core/src/main/java/org/elasticsearch/cluster/block/ClusterBlock.java
index 253206222b..f09e1dd9cd 100644
--- a/core/src/main/java/org/elasticsearch/cluster/block/ClusterBlock.java
+++ b/core/src/main/java/org/elasticsearch/cluster/block/ClusterBlock.java
@@ -125,7 +125,7 @@ public class ClusterBlock implements Streamable, ToXContent {
id = in.readVInt();
description = in.readString();
final int len = in.readVInt();
- ArrayList<ClusterBlockLevel> levels = new ArrayList<>();
+ ArrayList<ClusterBlockLevel> levels = new ArrayList<>(len);
for (int i = 0; i < len; i++) {
levels.add(ClusterBlockLevel.fromId(in.readVInt()));
}
diff --git a/core/src/main/java/org/elasticsearch/cluster/metadata/IndexNameExpressionResolver.java b/core/src/main/java/org/elasticsearch/cluster/metadata/IndexNameExpressionResolver.java
index d4c6ec587d..711d685c1d 100644
--- a/core/src/main/java/org/elasticsearch/cluster/metadata/IndexNameExpressionResolver.java
+++ b/core/src/main/java/org/elasticsearch/cluster/metadata/IndexNameExpressionResolver.java
@@ -29,6 +29,8 @@ import org.elasticsearch.common.collect.Tuple;
import org.elasticsearch.common.component.AbstractComponent;
import org.elasticsearch.common.joda.DateMathParser;
import org.elasticsearch.common.joda.FormatDateTimeFormatter;
+import org.elasticsearch.common.logging.DeprecationLogger;
+import org.elasticsearch.common.logging.Loggers;
import org.elasticsearch.common.regex.Regex;
import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.index.Index;
@@ -55,6 +57,8 @@ public class IndexNameExpressionResolver extends AbstractComponent {
private final List<ExpressionResolver> expressionResolvers;
private final DateMathExpressionResolver dateMathExpressionResolver;
+ private static final DeprecationLogger DEPRECATION_LOGGER =
+ new DeprecationLogger(Loggers.getLogger(IndexNameExpressionResolver.class));
public IndexNameExpressionResolver(Settings settings) {
super(settings);
@@ -159,7 +163,6 @@ public class IndexNameExpressionResolver extends AbstractComponent {
if (indexExpressions.length == 1) {
failNoIndices = options.allowNoIndices() == false;
}
-
List<String> expressions = Arrays.asList(indexExpressions);
for (ExpressionResolver expressionResolver : expressionResolvers) {
expressions = expressionResolver.resolve(context, expressions);
@@ -588,6 +591,7 @@ public class IndexNameExpressionResolver extends AbstractComponent {
private Set<String> innerResolve(Context context, List<String> expressions, IndicesOptions options, MetaData metaData) {
Set<String> result = null;
boolean wildcardSeen = false;
+ boolean plusSeen = false;
for (int i = 0; i < expressions.size(); i++) {
String expression = expressions.get(i);
if (aliasOrIndexExists(metaData, expression)) {
@@ -602,6 +606,7 @@ public class IndexNameExpressionResolver extends AbstractComponent {
boolean add = true;
if (expression.charAt(0) == '+') {
// if its the first, add empty result set
+ plusSeen = true;
if (i == 0) {
result = new HashSet<>();
}
@@ -649,6 +654,9 @@ public class IndexNameExpressionResolver extends AbstractComponent {
wildcardSeen = true;
}
}
+ if (plusSeen) {
+ DEPRECATION_LOGGER.deprecated("support for '+' as part of index expressions is deprecated");
+ }
return result;
}
diff --git a/core/src/main/java/org/elasticsearch/cluster/service/ClusterApplier.java b/core/src/main/java/org/elasticsearch/cluster/service/ClusterApplier.java
index ef3135af24..aa5c74e15e 100644
--- a/core/src/main/java/org/elasticsearch/cluster/service/ClusterApplier.java
+++ b/core/src/main/java/org/elasticsearch/cluster/service/ClusterApplier.java
@@ -24,9 +24,14 @@ import org.elasticsearch.cluster.ClusterStateTaskListener;
import java.util.function.Supplier;
-@FunctionalInterface
public interface ClusterApplier {
/**
+ * Sets the initial state for this applier. Should only be called once.
+ * @param initialState the initial state to set
+ */
+ void setInitialState(ClusterState initialState);
+
+ /**
* Method to invoke when a new cluster state is available to be applied
*
* @param source information where the cluster state came from
diff --git a/core/src/main/java/org/elasticsearch/cluster/service/ClusterApplierService.java b/core/src/main/java/org/elasticsearch/cluster/service/ClusterApplierService.java
index 540881718f..b029f10f5f 100644
--- a/core/src/main/java/org/elasticsearch/cluster/service/ClusterApplierService.java
+++ b/core/src/main/java/org/elasticsearch/cluster/service/ClusterApplierService.java
@@ -116,6 +116,7 @@ public class ClusterApplierService extends AbstractLifecycleComponent implements
this.nodeConnectionsService = nodeConnectionsService;
}
+ @Override
public void setInitialState(ClusterState initialState) {
if (lifecycle.started()) {
throw new IllegalStateException("can't set initial state when started");
diff --git a/core/src/main/java/org/elasticsearch/common/inject/ModulesBuilder.java b/core/src/main/java/org/elasticsearch/common/inject/ModulesBuilder.java
index 3321b75f4e..6928033c69 100644
--- a/core/src/main/java/org/elasticsearch/common/inject/ModulesBuilder.java
+++ b/core/src/main/java/org/elasticsearch/common/inject/ModulesBuilder.java
@@ -20,6 +20,7 @@
package org.elasticsearch.common.inject;
import java.util.ArrayList;
+import java.util.Collections;
import java.util.Iterator;
import java.util.List;
@@ -28,9 +29,7 @@ public class ModulesBuilder implements Iterable<Module> {
private final List<Module> modules = new ArrayList<>();
public ModulesBuilder add(Module... newModules) {
- for (Module module : newModules) {
- modules.add(module);
- }
+ Collections.addAll(modules, newModules);
return this;
}
diff --git a/core/src/main/java/org/elasticsearch/common/inject/assistedinject/AssistedConstructor.java b/core/src/main/java/org/elasticsearch/common/inject/assistedinject/AssistedConstructor.java
index edd45c290a..cb434a9036 100644
--- a/core/src/main/java/org/elasticsearch/common/inject/assistedinject/AssistedConstructor.java
+++ b/core/src/main/java/org/elasticsearch/common/inject/assistedinject/AssistedConstructor.java
@@ -49,7 +49,7 @@ class AssistedConstructor<T> {
Annotation[][] annotations = constructor.getParameterAnnotations();
List<Type> typeList = new ArrayList<>();
- allParameters = new ArrayList<>();
+ allParameters = new ArrayList<>(parameterTypes.size());
// categorize params as @Assisted or @Injected
for (int i = 0; i < parameterTypes.size(); i++) {
diff --git a/core/src/main/java/org/elasticsearch/common/lucene/search/MultiPhrasePrefixQuery.java b/core/src/main/java/org/elasticsearch/common/lucene/search/MultiPhrasePrefixQuery.java
index a76428e829..dbfc1f0af1 100644
--- a/core/src/main/java/org/elasticsearch/common/lucene/search/MultiPhrasePrefixQuery.java
+++ b/core/src/main/java/org/elasticsearch/common/lucene/search/MultiPhrasePrefixQuery.java
@@ -124,9 +124,7 @@ public class MultiPhrasePrefixQuery extends Query {
Term[][] terms = new Term[termArrays.size()][];
for (int i = 0; i < termArrays.size(); i++) {
terms[i] = new Term[termArrays.get(i).length];
- for (int j = 0; j < termArrays.get(i).length; j++) {
- terms[i][j] = termArrays.get(i)[j];
- }
+ System.arraycopy(termArrays.get(i), 0, terms[i], 0, termArrays.get(i).length);
}
return terms;
}
diff --git a/core/src/main/java/org/elasticsearch/discovery/Discovery.java b/core/src/main/java/org/elasticsearch/discovery/Discovery.java
index 7f68f417fc..3842e68d10 100644
--- a/core/src/main/java/org/elasticsearch/discovery/Discovery.java
+++ b/core/src/main/java/org/elasticsearch/discovery/Discovery.java
@@ -21,7 +21,6 @@ package org.elasticsearch.discovery;
import org.elasticsearch.ElasticsearchException;
import org.elasticsearch.cluster.ClusterChangedEvent;
-import org.elasticsearch.cluster.ClusterState;
import org.elasticsearch.cluster.node.DiscoveryNode;
import org.elasticsearch.common.Nullable;
import org.elasticsearch.common.component.LifecycleComponent;
@@ -48,18 +47,6 @@ public interface Discovery extends LifecycleComponent {
*/
void publish(ClusterChangedEvent clusterChangedEvent, AckListener ackListener);
- /**
- * Returns the initial cluster state provided by the discovery module. Used by
- * {@link org.elasticsearch.cluster.service.ClusterApplierService} as initial applied state.
- */
- ClusterState getInitialClusterState();
-
- /**
- * Returns latest cluster state used by the discovery module. Used by {@link org.elasticsearch.cluster.service.MasterService} to
- * calculate the next prospective state to publish.
- */
- ClusterState clusterState();
-
interface AckListener {
void onNodeAck(DiscoveryNode node, @Nullable Exception e);
void onTimeout();
@@ -90,9 +77,4 @@ public interface Discovery extends LifecycleComponent {
*/
void startInitialJoin();
- /***
- * @return the current value of minimum master nodes, or -1 for not set
- */
- int getMinimumMasterNodes();
-
}
diff --git a/core/src/main/java/org/elasticsearch/discovery/DiscoveryModule.java b/core/src/main/java/org/elasticsearch/discovery/DiscoveryModule.java
index b2367c6e95..c410cb88d6 100644
--- a/core/src/main/java/org/elasticsearch/discovery/DiscoveryModule.java
+++ b/core/src/main/java/org/elasticsearch/discovery/DiscoveryModule.java
@@ -86,8 +86,8 @@ public class DiscoveryModule {
discoveryTypes.put("zen",
() -> new ZenDiscovery(settings, threadPool, transportService, namedWriteableRegistry, masterService, clusterApplier,
clusterSettings, hostsProvider, allocationService));
- discoveryTypes.put("tribe", () -> new TribeDiscovery(settings, transportService, clusterApplier));
- discoveryTypes.put("single-node", () -> new SingleNodeDiscovery(settings, transportService, clusterApplier));
+ discoveryTypes.put("tribe", () -> new TribeDiscovery(settings, transportService, masterService, clusterApplier));
+ discoveryTypes.put("single-node", () -> new SingleNodeDiscovery(settings, transportService, masterService, clusterApplier));
for (DiscoveryPlugin plugin : plugins) {
plugin.getDiscoveryTypes(threadPool, transportService, namedWriteableRegistry,
masterService, clusterApplier, clusterSettings, hostsProvider, allocationService).entrySet().forEach(entry -> {
diff --git a/core/src/main/java/org/elasticsearch/discovery/TribeDiscovery.java b/core/src/main/java/org/elasticsearch/discovery/TribeDiscovery.java
index 9f802cc270..f3200be456 100644
--- a/core/src/main/java/org/elasticsearch/discovery/TribeDiscovery.java
+++ b/core/src/main/java/org/elasticsearch/discovery/TribeDiscovery.java
@@ -25,6 +25,7 @@ import org.elasticsearch.cluster.block.ClusterBlocks;
import org.elasticsearch.cluster.node.DiscoveryNode;
import org.elasticsearch.cluster.node.DiscoveryNodes;
import org.elasticsearch.cluster.service.ClusterApplier;
+import org.elasticsearch.cluster.service.MasterService;
import org.elasticsearch.common.inject.Inject;
import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.discovery.single.SingleNodeDiscovery;
@@ -44,26 +45,23 @@ import static org.elasticsearch.tribe.TribeService.TRIBE_WRITE_BLOCK;
public class TribeDiscovery extends SingleNodeDiscovery implements Discovery {
@Inject
- public TribeDiscovery(Settings settings, TransportService transportService, ClusterApplier clusterApplier) {
- super(settings, transportService, clusterApplier);
+ public TribeDiscovery(Settings settings, TransportService transportService,
+ MasterService masterService, ClusterApplier clusterApplier) {
+ super(settings, transportService, masterService, clusterApplier);
}
@Override
- public synchronized ClusterState getInitialClusterState() {
- if (initialState == null) {
- ClusterBlocks.Builder clusterBlocks = ClusterBlocks.builder(); // don't add no_master / state recovery block
- if (BLOCKS_WRITE_SETTING.get(settings)) {
- clusterBlocks.addGlobalBlock(TRIBE_WRITE_BLOCK);
- }
- if (BLOCKS_METADATA_SETTING.get(settings)) {
- clusterBlocks.addGlobalBlock(TRIBE_METADATA_BLOCK);
- }
- DiscoveryNode localNode = transportService.getLocalNode();
- initialState = ClusterState.builder(ClusterName.CLUSTER_NAME_SETTING.get(settings))
- .nodes(DiscoveryNodes.builder().add(localNode).localNodeId(localNode.getId()).build())
- .blocks(clusterBlocks).build();
+ protected ClusterState createInitialState(DiscoveryNode localNode) {
+ ClusterBlocks.Builder clusterBlocks = ClusterBlocks.builder(); // don't add no_master / state recovery block
+ if (BLOCKS_WRITE_SETTING.get(settings)) {
+ clusterBlocks.addGlobalBlock(TRIBE_WRITE_BLOCK);
}
- return initialState;
+ if (BLOCKS_METADATA_SETTING.get(settings)) {
+ clusterBlocks.addGlobalBlock(TRIBE_METADATA_BLOCK);
+ }
+ return ClusterState.builder(ClusterName.CLUSTER_NAME_SETTING.get(settings))
+ .nodes(DiscoveryNodes.builder().add(localNode).localNodeId(localNode.getId()).build())
+ .blocks(clusterBlocks).build();
}
@Override
diff --git a/core/src/main/java/org/elasticsearch/discovery/single/SingleNodeDiscovery.java b/core/src/main/java/org/elasticsearch/discovery/single/SingleNodeDiscovery.java
index 1152696179..a61253b7c2 100644
--- a/core/src/main/java/org/elasticsearch/discovery/single/SingleNodeDiscovery.java
+++ b/core/src/main/java/org/elasticsearch/discovery/single/SingleNodeDiscovery.java
@@ -28,6 +28,7 @@ import org.elasticsearch.cluster.block.ClusterBlocks;
import org.elasticsearch.cluster.node.DiscoveryNode;
import org.elasticsearch.cluster.node.DiscoveryNodes;
import org.elasticsearch.cluster.service.ClusterApplier;
+import org.elasticsearch.cluster.service.MasterService;
import org.elasticsearch.common.component.AbstractLifecycleComponent;
import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.discovery.Discovery;
@@ -48,13 +49,13 @@ public class SingleNodeDiscovery extends AbstractLifecycleComponent implements D
protected final TransportService transportService;
private final ClusterApplier clusterApplier;
- protected volatile ClusterState initialState;
private volatile ClusterState clusterState;
public SingleNodeDiscovery(final Settings settings, final TransportService transportService,
- ClusterApplier clusterApplier) {
+ final MasterService masterService, final ClusterApplier clusterApplier) {
super(Objects.requireNonNull(settings));
this.transportService = Objects.requireNonNull(transportService);
+ masterService.setClusterStateSupplier(() -> clusterState);
this.clusterApplier = clusterApplier;
}
@@ -82,7 +83,7 @@ public class SingleNodeDiscovery extends AbstractLifecycleComponent implements D
e);
}
};
- clusterApplier.onNewClusterState("apply-locally-on-node[" + event.source() + "]", this::clusterState, listener);
+ clusterApplier.onNewClusterState("apply-locally-on-node[" + event.source() + "]", () -> clusterState, listener);
try {
latch.await();
@@ -92,48 +93,38 @@ public class SingleNodeDiscovery extends AbstractLifecycleComponent implements D
}
@Override
- public synchronized ClusterState getInitialClusterState() {
- if (initialState == null) {
- DiscoveryNode localNode = transportService.getLocalNode();
- initialState = ClusterState.builder(ClusterName.CLUSTER_NAME_SETTING.get(settings))
- .nodes(DiscoveryNodes.builder().add(localNode)
- .localNodeId(localNode.getId())
- .masterNodeId(localNode.getId())
- .build())
- .blocks(ClusterBlocks.builder()
- .addGlobalBlock(STATE_NOT_RECOVERED_BLOCK))
- .build();
- }
- return initialState;
- }
-
- @Override
- public ClusterState clusterState() {
- return clusterState;
- }
-
- @Override
public DiscoveryStats stats() {
return new DiscoveryStats((PendingClusterStateStats) null);
}
@Override
public synchronized void startInitialJoin() {
+ if (lifecycle.started() == false) {
+ throw new IllegalStateException("can't start initial join when not started");
+ }
// apply a fresh cluster state just so that state recovery gets triggered by GatewayService
// TODO: give discovery module control over GatewayService
- clusterState = ClusterState.builder(getInitialClusterState()).build();
- clusterApplier.onNewClusterState("single-node-start-initial-join", this::clusterState, (source, e) -> {});
+ clusterState = ClusterState.builder(clusterState).build();
+ clusterApplier.onNewClusterState("single-node-start-initial-join", () -> clusterState, (source, e) -> {});
}
@Override
- public int getMinimumMasterNodes() {
- return 1;
+ protected synchronized void doStart() {
+ // set initial state
+ DiscoveryNode localNode = transportService.getLocalNode();
+ clusterState = createInitialState(localNode);
+ clusterApplier.setInitialState(clusterState);
}
- @Override
- protected synchronized void doStart() {
- initialState = getInitialClusterState();
- clusterState = initialState;
+ protected ClusterState createInitialState(DiscoveryNode localNode) {
+ return ClusterState.builder(ClusterName.CLUSTER_NAME_SETTING.get(settings))
+ .nodes(DiscoveryNodes.builder().add(localNode)
+ .localNodeId(localNode.getId())
+ .masterNodeId(localNode.getId())
+ .build())
+ .blocks(ClusterBlocks.builder()
+ .addGlobalBlock(STATE_NOT_RECOVERED_BLOCK))
+ .build();
}
@Override
diff --git a/core/src/main/java/org/elasticsearch/discovery/zen/UnicastZenPing.java b/core/src/main/java/org/elasticsearch/discovery/zen/UnicastZenPing.java
index 3a68b2b4cd..d0a9a212bd 100644
--- a/core/src/main/java/org/elasticsearch/discovery/zen/UnicastZenPing.java
+++ b/core/src/main/java/org/elasticsearch/discovery/zen/UnicastZenPing.java
@@ -116,7 +116,7 @@ public class UnicastZenPing extends AbstractComponent implements ZenPing {
private final int limitPortCounts;
- private volatile PingContextProvider contextProvider;
+ private final PingContextProvider contextProvider;
private final AtomicInteger pingingRoundIdGenerator = new AtomicInteger();
@@ -137,12 +137,13 @@ public class UnicastZenPing extends AbstractComponent implements ZenPing {
private volatile boolean closed = false;
public UnicastZenPing(Settings settings, ThreadPool threadPool, TransportService transportService,
- UnicastHostsProvider unicastHostsProvider) {
+ UnicastHostsProvider unicastHostsProvider, PingContextProvider contextProvider) {
super(settings);
this.threadPool = threadPool;
this.transportService = transportService;
this.clusterName = ClusterName.CLUSTER_NAME_SETTING.get(settings);
this.hostsProvider = unicastHostsProvider;
+ this.contextProvider = contextProvider;
final int concurrentConnects = DISCOVERY_ZEN_PING_UNICAST_CONCURRENT_CONNECTS_SETTING.get(settings);
if (DISCOVERY_ZEN_PING_UNICAST_HOSTS_SETTING.exists(settings)) {
@@ -260,8 +261,7 @@ public class UnicastZenPing extends AbstractComponent implements ZenPing {
}
@Override
- public void start(PingContextProvider contextProvider) {
- this.contextProvider = contextProvider;
+ public void start() {
}
/**
diff --git a/core/src/main/java/org/elasticsearch/discovery/zen/ZenDiscovery.java b/core/src/main/java/org/elasticsearch/discovery/zen/ZenDiscovery.java
index d08b148554..09e6357ba5 100644
--- a/core/src/main/java/org/elasticsearch/discovery/zen/ZenDiscovery.java
+++ b/core/src/main/java/org/elasticsearch/discovery/zen/ZenDiscovery.java
@@ -143,9 +143,8 @@ public class ZenDiscovery extends AbstractLifecycleComponent implements Discover
private final NodeRemovalClusterStateTaskExecutor nodeRemovalExecutor;
private final ClusterApplier clusterApplier;
- private final AtomicReference<ClusterState> state; // last committed cluster state
+ private final AtomicReference<ClusterState> committedState; // last committed cluster state
private final Object stateMutex = new Object();
- private volatile ClusterState initialState; // set lazily when discovery layer is started
public ZenDiscovery(Settings settings, ThreadPool threadPool, TransportService transportService,
NamedWriteableRegistry namedWriteableRegistry, MasterService masterService, ClusterApplier clusterApplier,
@@ -165,7 +164,7 @@ public class ZenDiscovery extends AbstractLifecycleComponent implements Discover
this.sendLeaveRequest = SEND_LEAVE_REQUEST_SETTING.get(settings);
this.threadPool = threadPool;
this.clusterName = ClusterName.CLUSTER_NAME_SETTING.get(settings);
- this.state = new AtomicReference<>();
+ this.committedState = new AtomicReference<>();
this.masterElectionIgnoreNonMasters = MASTER_ELECTION_IGNORE_NON_MASTER_PINGS_SETTING.get(settings);
this.masterElectionWaitForJoinsTimeout = MASTER_ELECTION_WAIT_FOR_JOINS_TIMEOUT_SETTING.get(settings);
@@ -214,6 +213,8 @@ public class ZenDiscovery extends AbstractLifecycleComponent implements Discover
this.nodeJoinController = new NodeJoinController(masterService, allocationService, electMaster, settings);
this.nodeRemovalExecutor = new NodeRemovalClusterStateTaskExecutor(allocationService, electMaster, this::submitRejoin, logger);
+ masterService.setClusterStateSupplier(this::clusterState);
+
transportService.registerRequestHandler(
DISCOVERY_REJOIN_ACTION_NAME, RejoinClusterRequest::new, ThreadPool.Names.SAME, new RejoinClusterRequestHandler());
}
@@ -221,7 +222,7 @@ public class ZenDiscovery extends AbstractLifecycleComponent implements Discover
// protected to allow overriding in tests
protected ZenPing newZenPing(Settings settings, ThreadPool threadPool, TransportService transportService,
UnicastHostsProvider hostsProvider) {
- return new UnicastZenPing(settings, threadPool, transportService, hostsProvider);
+ return new UnicastZenPing(settings, threadPool, transportService, hostsProvider, this);
}
@Override
@@ -229,12 +230,21 @@ public class ZenDiscovery extends AbstractLifecycleComponent implements Discover
DiscoveryNode localNode = transportService.getLocalNode();
assert localNode != null;
synchronized (stateMutex) {
- initialState = getInitialClusterState();
- state.set(initialState);
+ // set initial state
+ assert committedState.get() == null;
+ assert localNode != null;
+ ClusterState initialState = ClusterState.builder(clusterName)
+ .blocks(ClusterBlocks.builder()
+ .addGlobalBlock(STATE_NOT_RECOVERED_BLOCK)
+ .addGlobalBlock(discoverySettings.getNoMasterBlock()))
+ .nodes(DiscoveryNodes.builder().add(localNode).localNodeId(localNode.getId()))
+ .build();
+ committedState.set(initialState);
+ clusterApplier.setInitialState(initialState);
nodesFD.setLocalNode(localNode);
joinThreadControl.start();
}
- zenPing.start(this);
+ zenPing.start();
}
@Override
@@ -286,7 +296,7 @@ public class ZenDiscovery extends AbstractLifecycleComponent implements Discover
@Override
public ClusterState clusterState() {
- ClusterState clusterState = state.get();
+ ClusterState clusterState = committedState.get();
assert clusterState != null : "accessing cluster state before it is set";
return clusterState;
}
@@ -297,7 +307,7 @@ public class ZenDiscovery extends AbstractLifecycleComponent implements Discover
assert newState.getNodes().isLocalNodeElectedMaster() : "Shouldn't publish state when not master " + clusterChangedEvent.source();
// state got changed locally (maybe because another master published to us)
- if (clusterChangedEvent.previousState() != this.state.get()) {
+ if (clusterChangedEvent.previousState() != this.committedState.get()) {
throw new FailedToCommitClusterStateException("state was mutated while calculating new CS update");
}
@@ -345,7 +355,7 @@ public class ZenDiscovery extends AbstractLifecycleComponent implements Discover
});
synchronized (stateMutex) {
- if (clusterChangedEvent.previousState() != this.state.get()) {
+ if (clusterChangedEvent.previousState() != this.committedState.get()) {
throw new FailedToCommitClusterStateException("local state was mutated while CS update was published to other nodes");
}
@@ -371,22 +381,6 @@ public class ZenDiscovery extends AbstractLifecycleComponent implements Discover
}
}
- @Override
- public synchronized ClusterState getInitialClusterState() {
- if (initialState == null) {
- assert state.get() == null;
- DiscoveryNode localNode = transportService.getLocalNode();
- assert localNode != null;
- initialState = ClusterState.builder(clusterName)
- .blocks(ClusterBlocks.builder()
- .addGlobalBlock(STATE_NOT_RECOVERED_BLOCK)
- .addGlobalBlock(discoverySettings.getNoMasterBlock()))
- .nodes(DiscoveryNodes.builder().add(localNode).localNodeId(localNode.getId()))
- .build();
- }
- return initialState;
- }
-
/**
* Gets the current set of nodes involved in the node fault detection.
* NB: for testing purposes
@@ -405,11 +399,6 @@ public class ZenDiscovery extends AbstractLifecycleComponent implements Discover
return discoverySettings;
}
- @Override
- public int getMinimumMasterNodes() {
- return electMaster.minimumMasterNodes();
- }
-
/**
* returns true if zen discovery is started and there is a currently a background thread active for (re)joining
* the cluster used for testing.
@@ -548,9 +537,9 @@ public class ZenDiscovery extends AbstractLifecycleComponent implements Discover
}
// visible for testing
- void setState(ClusterState clusterState) {
+ void setCommittedState(ClusterState clusterState) {
synchronized (stateMutex) {
- state.set(clusterState);
+ committedState.set(clusterState);
}
}
@@ -693,7 +682,7 @@ public class ZenDiscovery extends AbstractLifecycleComponent implements Discover
}
synchronized (stateMutex) {
// check if we have enough master nodes, if not, we need to move into joining the cluster again
- if (!electMaster.hasEnoughMasterNodes(state.get().nodes())) {
+ if (!electMaster.hasEnoughMasterNodes(committedState.get().nodes())) {
rejoin("not enough master nodes on change of minimum_master_nodes from [" + prevMinimumMasterNode + "] to [" + minimumMasterNodes + "]");
}
}
@@ -712,7 +701,7 @@ public class ZenDiscovery extends AbstractLifecycleComponent implements Discover
logger.info((Supplier<?>) () -> new ParameterizedMessage("master_left [{}], reason [{}]", masterNode, reason), cause);
synchronized (stateMutex) {
- if (localNodeMaster() == false && masterNode.equals(state.get().nodes().getMasterNode())) {
+ if (localNodeMaster() == false && masterNode.equals(committedState.get().nodes().getMasterNode())) {
// flush any pending cluster states from old master, so it will not be set as master again
publishClusterState.pendingStatesQueue().failAllStatesAndClear(new ElasticsearchException("master left [{}]", reason));
rejoin("master left (reason = " + reason + ")");
@@ -725,7 +714,7 @@ public class ZenDiscovery extends AbstractLifecycleComponent implements Discover
assert Thread.holdsLock(stateMutex);
final ClusterState newClusterState = publishClusterState.pendingStatesQueue().getNextClusterStateToProcess();
- final ClusterState currentState = state.get();
+ final ClusterState currentState = committedState.get();
final ClusterState adaptedNewClusterState;
// all pending states have been processed
if (newClusterState == null) {
@@ -801,7 +790,7 @@ public class ZenDiscovery extends AbstractLifecycleComponent implements Discover
return false;
}
- state.set(adaptedNewClusterState);
+ committedState.set(adaptedNewClusterState);
// update failure detection only after the state has been updated to prevent race condition with handleLeaveRequest
// and handleNodeFailure as those check the current state to determine whether the failure is to be handled by this node
@@ -997,7 +986,7 @@ public class ZenDiscovery extends AbstractLifecycleComponent implements Discover
protected void rejoin(String reason) {
assert Thread.holdsLock(stateMutex);
- ClusterState clusterState = state.get();
+ ClusterState clusterState = committedState.get();
logger.warn("{}, current nodes: {}", reason, clusterState.nodes());
nodesFD.stop();
@@ -1021,7 +1010,7 @@ public class ZenDiscovery extends AbstractLifecycleComponent implements Discover
.nodes(discoveryNodes)
.build();
- state.set(clusterState);
+ committedState.set(clusterState);
clusterApplier.onNewClusterState(reason, this::clusterState, (source, e) -> {}); // don't wait for state to be applied
}
}
@@ -1123,7 +1112,7 @@ public class ZenDiscovery extends AbstractLifecycleComponent implements Discover
}
logger.debug("got a ping from another master {}. resolving who should rejoin. current ping count: [{}]", pingRequest.masterNode(), pingsWhileMaster.get());
synchronized (stateMutex) {
- ClusterState currentState = state.get();
+ ClusterState currentState = committedState.get();
if (currentState.nodes().isLocalNodeElectedMaster()) {
pingsWhileMaster.set(0);
handleAnotherMaster(currentState, pingRequest.masterNode(), pingRequest.clusterStateVersion(), "node fd ping");
diff --git a/core/src/main/java/org/elasticsearch/discovery/zen/ZenPing.java b/core/src/main/java/org/elasticsearch/discovery/zen/ZenPing.java
index 016d2a5423..d91dad8aee 100644
--- a/core/src/main/java/org/elasticsearch/discovery/zen/ZenPing.java
+++ b/core/src/main/java/org/elasticsearch/discovery/zen/ZenPing.java
@@ -40,7 +40,7 @@ import static org.elasticsearch.gateway.GatewayService.STATE_NOT_RECOVERED_BLOCK
public interface ZenPing extends Releasable {
- void start(PingContextProvider contextProvider);
+ void start();
void ping(Consumer<PingCollection> resultsConsumer, TimeValue timeout);
diff --git a/core/src/main/java/org/elasticsearch/env/Environment.java b/core/src/main/java/org/elasticsearch/env/Environment.java
index df859ab7de..7ac442d716 100644
--- a/core/src/main/java/org/elasticsearch/env/Environment.java
+++ b/core/src/main/java/org/elasticsearch/env/Environment.java
@@ -71,8 +71,6 @@ public class Environment {
private final Settings settings;
- private final String configExtension;
-
private final Path[] dataFiles;
private final Path[] dataWithClusterFiles;
@@ -104,12 +102,6 @@ public class Environment {
private final Path tmpFile = PathUtils.get(System.getProperty("java.io.tmpdir"));
public Environment(Settings settings) {
- this(settings, null);
- }
-
- // Note: Do not use this ctor, it is for correct deprecation logging in 5.5 and will be removed
- public Environment(Settings settings, String configExtension) {
- this.configExtension = configExtension;
final Path homeFile;
if (PATH_HOME_SETTING.exists(settings)) {
homeFile = PathUtils.get(cleanPath(PATH_HOME_SETTING.get(settings)));
@@ -281,11 +273,6 @@ public class Environment {
}
}
- /** Return then extension of the config file that was loaded, or*/
- public String configExtension() {
- return configExtension;
- }
-
// TODO: rename all these "file" methods to "dir"
/**
* The config directory.
diff --git a/core/src/main/java/org/elasticsearch/gateway/Gateway.java b/core/src/main/java/org/elasticsearch/gateway/Gateway.java
index 0d562fa4b2..4407e97d5a 100644
--- a/core/src/main/java/org/elasticsearch/gateway/Gateway.java
+++ b/core/src/main/java/org/elasticsearch/gateway/Gateway.java
@@ -32,13 +32,12 @@ import org.elasticsearch.cluster.service.ClusterService;
import org.elasticsearch.common.component.AbstractComponent;
import org.elasticsearch.common.settings.ClusterSettings;
import org.elasticsearch.common.settings.Settings;
-import org.elasticsearch.discovery.Discovery;
+import org.elasticsearch.discovery.zen.ElectMasterService;
import org.elasticsearch.index.Index;
import org.elasticsearch.indices.IndicesService;
import java.util.Arrays;
import java.util.Map;
-import java.util.function.Supplier;
public class Gateway extends AbstractComponent implements ClusterStateApplier {
@@ -48,18 +47,18 @@ public class Gateway extends AbstractComponent implements ClusterStateApplier {
private final TransportNodesListGatewayMetaState listGatewayMetaState;
- private final Supplier<Integer> minimumMasterNodesProvider;
+ private final int minimumMasterNodes;
private final IndicesService indicesService;
public Gateway(Settings settings, ClusterService clusterService, GatewayMetaState metaState,
- TransportNodesListGatewayMetaState listGatewayMetaState, Discovery discovery,
+ TransportNodesListGatewayMetaState listGatewayMetaState,
IndicesService indicesService) {
super(settings);
this.indicesService = indicesService;
this.clusterService = clusterService;
this.metaState = metaState;
this.listGatewayMetaState = listGatewayMetaState;
- this.minimumMasterNodesProvider = discovery::getMinimumMasterNodes;
+ this.minimumMasterNodes = ElectMasterService.DISCOVERY_ZEN_MINIMUM_MASTER_NODES_SETTING.get(settings);
clusterService.addLowPriorityApplier(this);
}
@@ -69,7 +68,7 @@ public class Gateway extends AbstractComponent implements ClusterStateApplier {
TransportNodesListGatewayMetaState.NodesGatewayMetaState nodesState = listGatewayMetaState.list(nodesIds, null).actionGet();
- int requiredAllocation = Math.max(1, minimumMasterNodesProvider.get());
+ int requiredAllocation = Math.max(1, minimumMasterNodes);
if (nodesState.hasFailures()) {
diff --git a/core/src/main/java/org/elasticsearch/gateway/GatewayService.java b/core/src/main/java/org/elasticsearch/gateway/GatewayService.java
index 0353deab6e..6e884af3b8 100644
--- a/core/src/main/java/org/elasticsearch/gateway/GatewayService.java
+++ b/core/src/main/java/org/elasticsearch/gateway/GatewayService.java
@@ -42,7 +42,6 @@ import org.elasticsearch.common.settings.Setting.Property;
import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.common.unit.TimeValue;
import org.elasticsearch.common.util.concurrent.AbstractRunnable;
-import org.elasticsearch.discovery.Discovery;
import org.elasticsearch.indices.IndicesService;
import org.elasticsearch.rest.RestStatus;
import org.elasticsearch.threadpool.ThreadPool;
@@ -93,10 +92,10 @@ public class GatewayService extends AbstractLifecycleComponent implements Cluste
@Inject
public GatewayService(Settings settings, AllocationService allocationService, ClusterService clusterService,
ThreadPool threadPool, GatewayMetaState metaState,
- TransportNodesListGatewayMetaState listGatewayMetaState, Discovery discovery,
+ TransportNodesListGatewayMetaState listGatewayMetaState,
IndicesService indicesService) {
super(settings);
- this.gateway = new Gateway(settings, clusterService, metaState, listGatewayMetaState, discovery,
+ this.gateway = new Gateway(settings, clusterService, metaState, listGatewayMetaState,
indicesService);
this.allocationService = allocationService;
this.clusterService = clusterService;
@@ -227,10 +226,6 @@ public class GatewayService extends AbstractLifecycleComponent implements Cluste
}
}
- public Gateway getGateway() {
- return gateway;
- }
-
class GatewayRecoveryListener implements Gateway.GatewayStateRecoveredListener {
@Override
diff --git a/core/src/main/java/org/elasticsearch/index/IndexWarmer.java b/core/src/main/java/org/elasticsearch/index/IndexWarmer.java
index fdaad19e52..e177ca668f 100644
--- a/core/src/main/java/org/elasticsearch/index/IndexWarmer.java
+++ b/core/src/main/java/org/elasticsearch/index/IndexWarmer.java
@@ -54,9 +54,8 @@ public final class IndexWarmer extends AbstractComponent {
ArrayList<Listener> list = new ArrayList<>();
final Executor executor = threadPool.executor(ThreadPool.Names.WARMER);
list.add(new FieldDataWarmer(executor));
- for (Listener listener : listeners) {
- list.add(listener);
- }
+
+ Collections.addAll(list, listeners);
this.listeners = Collections.unmodifiableList(list);
}
diff --git a/core/src/main/java/org/elasticsearch/index/analysis/CustomAnalyzerProvider.java b/core/src/main/java/org/elasticsearch/index/analysis/CustomAnalyzerProvider.java
index 282edaeaf7..f112304562 100644
--- a/core/src/main/java/org/elasticsearch/index/analysis/CustomAnalyzerProvider.java
+++ b/core/src/main/java/org/elasticsearch/index/analysis/CustomAnalyzerProvider.java
@@ -55,8 +55,8 @@ public class CustomAnalyzerProvider extends AbstractIndexAnalyzerProvider<Custom
throw new IllegalArgumentException("Custom Analyzer [" + name() + "] failed to find tokenizer under name [" + tokenizerName + "]");
}
- List<CharFilterFactory> charFiltersList = new ArrayList<>();
String[] charFilterNames = analyzerSettings.getAsArray("char_filter");
+ List<CharFilterFactory> charFiltersList = new ArrayList<>(charFilterNames.length);
for (String charFilterName : charFilterNames) {
CharFilterFactory charFilter = charFilters.get(charFilterName);
if (charFilter == null) {
@@ -65,8 +65,8 @@ public class CustomAnalyzerProvider extends AbstractIndexAnalyzerProvider<Custom
charFiltersList.add(charFilter);
}
- List<TokenFilterFactory> tokenFilterList = new ArrayList<>();
String[] tokenFilterNames = analyzerSettings.getAsArray("filter");
+ List<TokenFilterFactory> tokenFilterList = new ArrayList<>(tokenFilterNames.length);
for (String tokenFilterName : tokenFilterNames) {
TokenFilterFactory tokenFilter = tokenFilters.get(tokenFilterName);
if (tokenFilter == null) {
diff --git a/core/src/main/java/org/elasticsearch/index/analysis/CustomNormalizerProvider.java b/core/src/main/java/org/elasticsearch/index/analysis/CustomNormalizerProvider.java
index 4f50a34dd9..2fcc987df6 100644
--- a/core/src/main/java/org/elasticsearch/index/analysis/CustomNormalizerProvider.java
+++ b/core/src/main/java/org/elasticsearch/index/analysis/CustomNormalizerProvider.java
@@ -50,8 +50,8 @@ public final class CustomNormalizerProvider extends AbstractIndexAnalyzerProvide
throw new IllegalArgumentException("Custom normalizer [" + name() + "] cannot configure a tokenizer");
}
- List<CharFilterFactory> charFiltersList = new ArrayList<>();
String[] charFilterNames = analyzerSettings.getAsArray("char_filter");
+ List<CharFilterFactory> charFiltersList = new ArrayList<>(charFilterNames.length);
for (String charFilterName : charFilterNames) {
CharFilterFactory charFilter = charFilters.get(charFilterName);
if (charFilter == null) {
@@ -66,8 +66,8 @@ public final class CustomNormalizerProvider extends AbstractIndexAnalyzerProvide
charFiltersList.add(charFilter);
}
- List<TokenFilterFactory> tokenFilterList = new ArrayList<>();
String[] tokenFilterNames = analyzerSettings.getAsArray("filter");
+ List<TokenFilterFactory> tokenFilterList = new ArrayList<>(tokenFilterNames.length);
for (String tokenFilterName : tokenFilterNames) {
TokenFilterFactory tokenFilter = tokenFilters.get(tokenFilterName);
if (tokenFilter == null) {
diff --git a/core/src/main/java/org/elasticsearch/index/mapper/DynamicTemplate.java b/core/src/main/java/org/elasticsearch/index/mapper/DynamicTemplate.java
index b07492740a..ca8fcd1ffd 100644
--- a/core/src/main/java/org/elasticsearch/index/mapper/DynamicTemplate.java
+++ b/core/src/main/java/org/elasticsearch/index/mapper/DynamicTemplate.java
@@ -318,7 +318,7 @@ public class DynamicTemplate implements ToXContent {
}
private List processList(List list, String name, String dynamicType) {
- List processedList = new ArrayList();
+ List processedList = new ArrayList(list.size());
for (Object value : list) {
if (value instanceof Map) {
value = processMap((Map<String, Object>) value, name, dynamicType);
diff --git a/core/src/main/java/org/elasticsearch/index/query/AbstractQueryBuilder.java b/core/src/main/java/org/elasticsearch/index/query/AbstractQueryBuilder.java
index 6a3cd2f1ed..9d18e42138 100644
--- a/core/src/main/java/org/elasticsearch/index/query/AbstractQueryBuilder.java
+++ b/core/src/main/java/org/elasticsearch/index/query/AbstractQueryBuilder.java
@@ -250,8 +250,8 @@ public abstract class AbstractQueryBuilder<QB extends AbstractQueryBuilder<QB>>
}
protected static final List<QueryBuilder> readQueries(StreamInput in) throws IOException {
- List<QueryBuilder> queries = new ArrayList<>();
int size = in.readVInt();
+ List<QueryBuilder> queries = new ArrayList<>(size);
for (int i = 0; i < size; i++) {
queries.add(in.readNamedWriteable(QueryBuilder.class));
}
diff --git a/core/src/main/java/org/elasticsearch/index/query/GeoPolygonQueryBuilder.java b/core/src/main/java/org/elasticsearch/index/query/GeoPolygonQueryBuilder.java
index cc25ac4772..1f9e81cecc 100644
--- a/core/src/main/java/org/elasticsearch/index/query/GeoPolygonQueryBuilder.java
+++ b/core/src/main/java/org/elasticsearch/index/query/GeoPolygonQueryBuilder.java
@@ -163,7 +163,7 @@ public class GeoPolygonQueryBuilder extends AbstractQueryBuilder<GeoPolygonQuery
throw new QueryShardException(context, "field [" + fieldName + "] is not a geo_point field");
}
- List<GeoPoint> shell = new ArrayList<GeoPoint>();
+ List<GeoPoint> shell = new ArrayList<>(this.shell.size());
for (GeoPoint geoPoint : this.shell) {
shell.add(new GeoPoint(geoPoint));
}
diff --git a/core/src/main/java/org/elasticsearch/index/query/HasChildQueryBuilder.java b/core/src/main/java/org/elasticsearch/index/query/HasChildQueryBuilder.java
deleted file mode 100644
index 18ad7f9f31..0000000000
--- a/core/src/main/java/org/elasticsearch/index/query/HasChildQueryBuilder.java
+++ /dev/null
@@ -1,490 +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.index.DirectoryReader;
-import org.apache.lucene.index.IndexReader;
-import org.apache.lucene.index.MultiDocValues;
-import org.apache.lucene.search.IndexSearcher;
-import org.apache.lucene.search.MatchNoDocsQuery;
-import org.apache.lucene.search.Query;
-import org.apache.lucene.search.join.JoinUtil;
-import org.apache.lucene.search.join.ScoreMode;
-import org.apache.lucene.search.similarities.Similarity;
-import org.elasticsearch.common.ParseField;
-import org.elasticsearch.common.ParsingException;
-import org.elasticsearch.common.io.stream.StreamInput;
-import org.elasticsearch.common.io.stream.StreamOutput;
-import org.elasticsearch.common.lucene.search.Queries;
-import org.elasticsearch.common.xcontent.XContentBuilder;
-import org.elasticsearch.common.xcontent.XContentParser;
-import org.elasticsearch.index.fielddata.IndexParentChildFieldData;
-import org.elasticsearch.index.fielddata.plain.ParentChildIndexFieldData;
-import org.elasticsearch.index.mapper.DocumentMapper;
-import org.elasticsearch.index.mapper.ParentFieldMapper;
-
-import java.io.IOException;
-import java.util.Locale;
-import java.util.Map;
-import java.util.Objects;
-
-/**
- * A query builder for <tt>has_child</tt> query.
- */
-public class HasChildQueryBuilder extends AbstractQueryBuilder<HasChildQueryBuilder> {
- public static final String NAME = "has_child";
-
- /**
- * The default maximum number of children that are required to match for the parent to be considered a match.
- */
- public static final int DEFAULT_MAX_CHILDREN = Integer.MAX_VALUE;
- /**
- * The default minimum number of children that are required to match for the parent to be considered a match.
- */
- public static final int DEFAULT_MIN_CHILDREN = 0;
-
- /**
- * The default value for ignore_unmapped.
- */
- public static final boolean DEFAULT_IGNORE_UNMAPPED = false;
-
- private static final ParseField QUERY_FIELD = new ParseField("query", "filter");
- private static final ParseField TYPE_FIELD = new ParseField("type", "child_type");
- private static final ParseField MAX_CHILDREN_FIELD = new ParseField("max_children");
- private static final ParseField MIN_CHILDREN_FIELD = new ParseField("min_children");
- private static final ParseField SCORE_MODE_FIELD = new ParseField("score_mode");
- private static final ParseField INNER_HITS_FIELD = new ParseField("inner_hits");
- private static final ParseField IGNORE_UNMAPPED_FIELD = new ParseField("ignore_unmapped");
-
- private final QueryBuilder query;
- private final String type;
- private final ScoreMode scoreMode;
- private InnerHitBuilder innerHitBuilder;
- private int minChildren = DEFAULT_MIN_CHILDREN;
- private int maxChildren = DEFAULT_MAX_CHILDREN;
- private boolean ignoreUnmapped = false;
-
- public HasChildQueryBuilder(String type, QueryBuilder query, ScoreMode scoreMode) {
- this(type, query, DEFAULT_MIN_CHILDREN, DEFAULT_MAX_CHILDREN, scoreMode, null);
- }
-
- private HasChildQueryBuilder(String type, QueryBuilder query, int minChildren, int maxChildren, ScoreMode scoreMode,
- InnerHitBuilder innerHitBuilder) {
- this.type = requireValue(type, "[" + NAME + "] requires 'type' field");
- this.query = requireValue(query, "[" + NAME + "] requires 'query' field");
- this.scoreMode = requireValue(scoreMode, "[" + NAME + "] requires 'score_mode' field");
- this.innerHitBuilder = innerHitBuilder;
- this.minChildren = minChildren;
- this.maxChildren = maxChildren;
- }
-
- /**
- * Read from a stream.
- */
- public HasChildQueryBuilder(StreamInput in) throws IOException {
- super(in);
- type = in.readString();
- minChildren = in.readInt();
- maxChildren = in.readInt();
- scoreMode = ScoreMode.values()[in.readVInt()];
- query = in.readNamedWriteable(QueryBuilder.class);
- innerHitBuilder = in.readOptionalWriteable(InnerHitBuilder::new);
- ignoreUnmapped = in.readBoolean();
- }
-
- @Override
- protected void doWriteTo(StreamOutput out) throws IOException {
- out.writeString(type);
- out.writeInt(minChildren);
- out.writeInt(maxChildren);
- out.writeVInt(scoreMode.ordinal());
- out.writeNamedWriteable(query);
- out.writeOptionalWriteable(innerHitBuilder);
- out.writeBoolean(ignoreUnmapped);
- }
-
- /**
- * Defines the minimum number of children that are required to match for the parent to be considered a match and
- * the maximum number of children that are required to match for the parent to be considered a match.
- */
- public HasChildQueryBuilder minMaxChildren(int minChildren, int maxChildren) {
- if (minChildren < 0) {
- throw new IllegalArgumentException("[" + NAME + "] requires non-negative 'min_children' field");
- }
- if (maxChildren < 0) {
- throw new IllegalArgumentException("[" + NAME + "] requires non-negative 'max_children' field");
- }
- if (maxChildren < minChildren) {
- throw new IllegalArgumentException("[" + NAME + "] 'max_children' is less than 'min_children'");
- }
- this.minChildren = minChildren;
- this.maxChildren = maxChildren;
- return this;
- }
-
- /**
- * Returns inner hit definition in the scope of this query and reusing the defined type and query.
- */
- public InnerHitBuilder innerHit() {
- return innerHitBuilder;
- }
-
- public HasChildQueryBuilder innerHit(InnerHitBuilder innerHit, boolean ignoreUnmapped) {
- this.innerHitBuilder = new InnerHitBuilder(Objects.requireNonNull(innerHit), query, type, ignoreUnmapped);
- return this;
- }
-
- /**
- * Returns the children query to execute.
- */
- public QueryBuilder query() {
- return query;
- }
-
- /**
- * Returns the child type
- */
- public String childType() {
- return type;
- }
-
- /**
- * Returns how the scores from the matching child documents are mapped into the parent document.
- */
- public ScoreMode scoreMode() {
- return scoreMode;
- }
-
- /**
- * Returns the minimum number of children that are required to match for the parent to be considered a match.
- * The default is {@value #DEFAULT_MAX_CHILDREN}
- */
- public int minChildren() {
- return minChildren;
- }
-
- /**
- * Returns the maximum number of children that are required to match for the parent to be considered a match.
- * The default is {@value #DEFAULT_MIN_CHILDREN}
- */
- public int maxChildren() { return maxChildren; }
-
- /**
- * Sets whether the query builder should ignore unmapped types (and run a
- * {@link MatchNoDocsQuery} in place of this query) or throw an exception if
- * the type is unmapped.
- */
- public HasChildQueryBuilder ignoreUnmapped(boolean ignoreUnmapped) {
- this.ignoreUnmapped = ignoreUnmapped;
- return this;
- }
-
- /**
- * Gets whether the query builder will ignore unmapped types (and run a
- * {@link MatchNoDocsQuery} in place of this query) or throw an exception if
- * the type is unmapped.
- */
- public boolean ignoreUnmapped() {
- return ignoreUnmapped;
- }
-
- @Override
- protected void doXContent(XContentBuilder builder, Params params) throws IOException {
- builder.startObject(NAME);
- builder.field(QUERY_FIELD.getPreferredName());
- query.toXContent(builder, params);
- builder.field(TYPE_FIELD.getPreferredName(), type);
- builder.field(SCORE_MODE_FIELD.getPreferredName(), scoreModeAsString(scoreMode));
- builder.field(MIN_CHILDREN_FIELD.getPreferredName(), minChildren);
- builder.field(MAX_CHILDREN_FIELD.getPreferredName(), maxChildren);
- builder.field(IGNORE_UNMAPPED_FIELD.getPreferredName(), ignoreUnmapped);
- printBoostAndQueryName(builder);
- if (innerHitBuilder != null) {
- builder.field(INNER_HITS_FIELD.getPreferredName(), innerHitBuilder, params);
- }
- builder.endObject();
- }
-
- public static HasChildQueryBuilder fromXContent(QueryParseContext parseContext) throws IOException {
- XContentParser parser = parseContext.parser();
- float boost = AbstractQueryBuilder.DEFAULT_BOOST;
- String childType = null;
- ScoreMode scoreMode = ScoreMode.None;
- int minChildren = HasChildQueryBuilder.DEFAULT_MIN_CHILDREN;
- int maxChildren = HasChildQueryBuilder.DEFAULT_MAX_CHILDREN;
- boolean ignoreUnmapped = DEFAULT_IGNORE_UNMAPPED;
- String queryName = null;
- InnerHitBuilder innerHitBuilder = null;
- String currentFieldName = null;
- XContentParser.Token token;
- QueryBuilder iqb = null;
- while ((token = parser.nextToken()) != XContentParser.Token.END_OBJECT) {
- if (token == XContentParser.Token.FIELD_NAME) {
- currentFieldName = parser.currentName();
- } else if (parseContext.isDeprecatedSetting(currentFieldName)) {
- // skip
- } else if (token == XContentParser.Token.START_OBJECT) {
- if (QUERY_FIELD.match(currentFieldName)) {
- iqb = parseContext.parseInnerQueryBuilder();
- } else if (INNER_HITS_FIELD.match(currentFieldName)) {
- innerHitBuilder = InnerHitBuilder.fromXContent(parseContext);
- } else {
- throw new ParsingException(parser.getTokenLocation(), "[has_child] query does not support [" + currentFieldName + "]");
- }
- } else if (token.isValue()) {
- if (TYPE_FIELD.match(currentFieldName)) {
- childType = parser.text();
- } else if (SCORE_MODE_FIELD.match(currentFieldName)) {
- scoreMode = parseScoreMode(parser.text());
- } else if (AbstractQueryBuilder.BOOST_FIELD.match(currentFieldName)) {
- boost = parser.floatValue();
- } else if (MIN_CHILDREN_FIELD.match(currentFieldName)) {
- minChildren = parser.intValue(true);
- } else if (MAX_CHILDREN_FIELD.match(currentFieldName)) {
- maxChildren = parser.intValue(true);
- } else if (IGNORE_UNMAPPED_FIELD.match(currentFieldName)) {
- ignoreUnmapped = parser.booleanValue();
- } else if (AbstractQueryBuilder.NAME_FIELD.match(currentFieldName)) {
- queryName = parser.text();
- } else {
- throw new ParsingException(parser.getTokenLocation(), "[has_child] query does not support [" + currentFieldName + "]");
- }
- }
- }
- HasChildQueryBuilder hasChildQueryBuilder = new HasChildQueryBuilder(childType, iqb, scoreMode);
- hasChildQueryBuilder.minMaxChildren(minChildren, maxChildren);
- hasChildQueryBuilder.queryName(queryName);
- hasChildQueryBuilder.boost(boost);
- hasChildQueryBuilder.ignoreUnmapped(ignoreUnmapped);
- if (innerHitBuilder != null) {
- hasChildQueryBuilder.innerHit(innerHitBuilder, ignoreUnmapped);
- }
- return hasChildQueryBuilder;
- }
-
- public static ScoreMode parseScoreMode(String scoreModeString) {
- if ("none".equals(scoreModeString)) {
- return ScoreMode.None;
- } else if ("min".equals(scoreModeString)) {
- return ScoreMode.Min;
- } else if ("max".equals(scoreModeString)) {
- return ScoreMode.Max;
- } else if ("avg".equals(scoreModeString)) {
- return ScoreMode.Avg;
- } else if ("sum".equals(scoreModeString)) {
- return ScoreMode.Total;
- }
- throw new IllegalArgumentException("No score mode for child query [" + scoreModeString + "] found");
- }
-
- public static String scoreModeAsString(ScoreMode scoreMode) {
- if (scoreMode == ScoreMode.Total) {
- // Lucene uses 'total' but 'sum' is more consistent with other elasticsearch APIs
- return "sum";
- } else {
- return scoreMode.name().toLowerCase(Locale.ROOT);
- }
- }
-
- @Override
- public String getWriteableName() {
- return NAME;
- }
-
- @Override
- protected Query doToQuery(QueryShardContext context) throws IOException {
- Query innerQuery;
- final String[] previousTypes = context.getTypes();
- context.setTypes(type);
- try {
- innerQuery = query.toQuery(context);
- } finally {
- context.setTypes(previousTypes);
- }
-
- DocumentMapper childDocMapper = context.documentMapper(type);
- if (childDocMapper == null) {
- if (ignoreUnmapped) {
- return new MatchNoDocsQuery();
- } else {
- throw new QueryShardException(context, "[" + NAME + "] no mapping found for type [" + type + "]");
- }
- }
- ParentFieldMapper parentFieldMapper = childDocMapper.parentFieldMapper();
- if (parentFieldMapper.active() == false) {
- throw new QueryShardException(context, "[" + NAME + "] _parent field has no parent type configured");
- }
- String parentType = parentFieldMapper.type();
- DocumentMapper parentDocMapper = context.getMapperService().documentMapper(parentType);
- if (parentDocMapper == null) {
- throw new QueryShardException(context,
- "[" + NAME + "] Type [" + type + "] points to a non existent parent type [" + parentType + "]");
- }
-
- // wrap the query with type query
- innerQuery = Queries.filtered(innerQuery, childDocMapper.typeFilter(context));
-
- final ParentChildIndexFieldData parentChildIndexFieldData = context.getForField(parentFieldMapper.fieldType());
- return new LateParsingQuery(parentDocMapper.typeFilter(context), innerQuery, minChildren(), maxChildren(),
- parentType, scoreMode, parentChildIndexFieldData, context.getSearchSimilarity());
- }
-
- /**
- * A query that rewrites into another query using
- * {@link JoinUtil#createJoinQuery(String, Query, Query, IndexSearcher, ScoreMode, MultiDocValues.OrdinalMap, int, int)}
- * that executes the actual join.
- *
- * This query is exclusively used by the {@link HasChildQueryBuilder} and {@link HasParentQueryBuilder} to get access
- * to the {@link DirectoryReader} used by the current search in order to retrieve the {@link MultiDocValues.OrdinalMap}.
- * The {@link MultiDocValues.OrdinalMap} is required by {@link JoinUtil} to execute the join.
- */
- // TODO: Find a way to remove this query and let doToQuery(...) just return the query from JoinUtil.createJoinQuery(...)
- public static final class LateParsingQuery extends Query {
-
- private final Query toQuery;
- private final Query innerQuery;
- private final int minChildren;
- private final int maxChildren;
- private final String parentType;
- private final ScoreMode scoreMode;
- private final ParentChildIndexFieldData parentChildIndexFieldData;
- private final Similarity similarity;
-
- LateParsingQuery(Query toQuery, Query innerQuery, int minChildren, int maxChildren,
- String parentType, ScoreMode scoreMode, ParentChildIndexFieldData parentChildIndexFieldData,
- Similarity similarity) {
- this.toQuery = toQuery;
- this.innerQuery = innerQuery;
- this.minChildren = minChildren;
- this.maxChildren = maxChildren;
- this.parentType = parentType;
- this.scoreMode = scoreMode;
- this.parentChildIndexFieldData = parentChildIndexFieldData;
- this.similarity = similarity;
- }
-
- @Override
- public Query rewrite(IndexReader reader) throws IOException {
- Query rewritten = super.rewrite(reader);
- if (rewritten != this) {
- return rewritten;
- }
- if (reader instanceof DirectoryReader) {
- String joinField = ParentFieldMapper.joinField(parentType);
- IndexSearcher indexSearcher = new IndexSearcher(reader);
- indexSearcher.setQueryCache(null);
- indexSearcher.setSimilarity(similarity);
- IndexParentChildFieldData indexParentChildFieldData = parentChildIndexFieldData.loadGlobal((DirectoryReader) reader);
- MultiDocValues.OrdinalMap ordinalMap = ParentChildIndexFieldData.getOrdinalMap(indexParentChildFieldData, parentType);
- return JoinUtil.createJoinQuery(joinField, innerQuery, toQuery, indexSearcher, scoreMode,
- ordinalMap, minChildren, maxChildren);
- } else {
- if (reader.leaves().isEmpty() && reader.numDocs() == 0) {
- // asserting reader passes down a MultiReader during rewrite which makes this
- // blow up since for this query to work we have to have a DirectoryReader otherwise
- // we can't load global ordinals - for this to work we simply check if the reader has no leaves
- // and rewrite to match nothing
- return new MatchNoDocsQuery();
- }
- throw new IllegalStateException("can't load global ordinals for reader of type: " +
- reader.getClass() + " must be a DirectoryReader");
- }
- }
-
- @Override
- public boolean equals(Object o) {
- if (sameClassAs(o) == false) return false;
-
- LateParsingQuery that = (LateParsingQuery) o;
-
- if (minChildren != that.minChildren) return false;
- if (maxChildren != that.maxChildren) return false;
- if (!toQuery.equals(that.toQuery)) return false;
- if (!innerQuery.equals(that.innerQuery)) return false;
- if (!parentType.equals(that.parentType)) return false;
- return scoreMode == that.scoreMode;
- }
-
- @Override
- public int hashCode() {
- return Objects.hash(classHash(), toQuery, innerQuery, minChildren, maxChildren, parentType, scoreMode);
- }
-
- @Override
- public String toString(String s) {
- return "LateParsingQuery {parentType=" + parentType + "}";
- }
-
- public int getMinChildren() {
- return minChildren;
- }
-
- public int getMaxChildren() {
- return maxChildren;
- }
-
- public ScoreMode getScoreMode() {
- return scoreMode;
- }
-
- public Query getInnerQuery() {
- return innerQuery;
- }
-
- public Similarity getSimilarity() {
- return similarity;
- }
- }
-
- @Override
- protected boolean doEquals(HasChildQueryBuilder that) {
- return Objects.equals(query, that.query)
- && Objects.equals(type, that.type)
- && Objects.equals(scoreMode, that.scoreMode)
- && Objects.equals(minChildren, that.minChildren)
- && Objects.equals(maxChildren, that.maxChildren)
- && Objects.equals(innerHitBuilder, that.innerHitBuilder)
- && Objects.equals(ignoreUnmapped, that.ignoreUnmapped);
- }
-
- @Override
- protected int doHashCode() {
- return Objects.hash(query, type, scoreMode, minChildren, maxChildren, innerHitBuilder, ignoreUnmapped);
- }
-
- @Override
- protected QueryBuilder doRewrite(QueryRewriteContext queryRewriteContext) throws IOException {
- QueryBuilder rewrittenQuery = query.rewrite(queryRewriteContext);
- if (rewrittenQuery != query) {
- InnerHitBuilder rewrittenInnerHit = InnerHitBuilder.rewrite(innerHitBuilder, rewrittenQuery);
- HasChildQueryBuilder hasChildQueryBuilder =
- new HasChildQueryBuilder(type, rewrittenQuery, minChildren, maxChildren, scoreMode, rewrittenInnerHit);
- hasChildQueryBuilder.ignoreUnmapped(ignoreUnmapped);
- return hasChildQueryBuilder;
- }
- return this;
- }
-
- @Override
- protected void extractInnerHitBuilders(Map<String, InnerHitBuilder> innerHits) {
- if (innerHitBuilder != null) {
- innerHitBuilder.inlineInnerHits(innerHits);
- }
- }
-}
diff --git a/core/src/main/java/org/elasticsearch/index/query/HasParentQueryBuilder.java b/core/src/main/java/org/elasticsearch/index/query/HasParentQueryBuilder.java
deleted file mode 100644
index 63c9484691..0000000000
--- a/core/src/main/java/org/elasticsearch/index/query/HasParentQueryBuilder.java
+++ /dev/null
@@ -1,321 +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.search.BooleanClause;
-import org.apache.lucene.search.BooleanQuery;
-import org.apache.lucene.search.MatchNoDocsQuery;
-import org.apache.lucene.search.Query;
-import org.apache.lucene.search.join.ScoreMode;
-import org.elasticsearch.common.ParseField;
-import org.elasticsearch.common.ParsingException;
-import org.elasticsearch.common.io.stream.StreamInput;
-import org.elasticsearch.common.io.stream.StreamOutput;
-import org.elasticsearch.common.lucene.search.Queries;
-import org.elasticsearch.common.xcontent.XContentBuilder;
-import org.elasticsearch.common.xcontent.XContentParser;
-import org.elasticsearch.index.fielddata.plain.ParentChildIndexFieldData;
-import org.elasticsearch.index.mapper.DocumentMapper;
-import org.elasticsearch.index.mapper.ParentFieldMapper;
-
-import java.io.IOException;
-import java.util.HashSet;
-import java.util.Map;
-import java.util.Objects;
-import java.util.Set;
-
-/**
- * Builder for the 'has_parent' query.
- */
-public class HasParentQueryBuilder extends AbstractQueryBuilder<HasParentQueryBuilder> {
- public static final String NAME = "has_parent";
-
- /**
- * The default value for ignore_unmapped.
- */
- public static final boolean DEFAULT_IGNORE_UNMAPPED = false;
-
- private static final ParseField QUERY_FIELD = new ParseField("query", "filter");
- private static final ParseField SCORE_MODE_FIELD = new ParseField("score_mode").withAllDeprecated("score");
- private static final ParseField TYPE_FIELD = new ParseField("parent_type", "type");
- private static final ParseField SCORE_FIELD = new ParseField("score");
- private static final ParseField INNER_HITS_FIELD = new ParseField("inner_hits");
- private static final ParseField IGNORE_UNMAPPED_FIELD = new ParseField("ignore_unmapped");
-
- private final QueryBuilder query;
- private final String type;
- private final boolean score;
- private InnerHitBuilder innerHit;
- private boolean ignoreUnmapped = false;
-
- public HasParentQueryBuilder(String type, QueryBuilder query, boolean score) {
- this(type, query, score, null);
- }
-
- private HasParentQueryBuilder(String type, QueryBuilder query, boolean score, InnerHitBuilder innerHit) {
- this.type = requireValue(type, "[" + NAME + "] requires 'type' field");
- this.query = requireValue(query, "[" + NAME + "] requires 'query' field");
- this.score = score;
- this.innerHit = innerHit;
- }
-
- /**
- * Read from a stream.
- */
- public HasParentQueryBuilder(StreamInput in) throws IOException {
- super(in);
- type = in.readString();
- score = in.readBoolean();
- query = in.readNamedWriteable(QueryBuilder.class);
- innerHit = in.readOptionalWriteable(InnerHitBuilder::new);
- ignoreUnmapped = in.readBoolean();
- }
-
- @Override
- protected void doWriteTo(StreamOutput out) throws IOException {
- out.writeString(type);
- out.writeBoolean(score);
- out.writeNamedWriteable(query);
- out.writeOptionalWriteable(innerHit);
- out.writeBoolean(ignoreUnmapped);
- }
-
- /**
- * Returns the query to execute.
- */
- public QueryBuilder query() {
- return query;
- }
-
- /**
- * Returns <code>true</code> if the parent score is mapped into the child documents
- */
- public boolean score() {
- return score;
- }
-
- /**
- * Returns the parents type name
- */
- public String type() {
- return type;
- }
-
- /**
- * Returns inner hit definition in the scope of this query and reusing the defined type and query.
- */
- public InnerHitBuilder innerHit() {
- return innerHit;
- }
-
- public HasParentQueryBuilder innerHit(InnerHitBuilder innerHit, boolean ignoreUnmapped) {
- this.innerHit = new InnerHitBuilder(innerHit, query, type, ignoreUnmapped);
- return this;
- }
-
- /**
- * Sets whether the query builder should ignore unmapped types (and run a
- * {@link MatchNoDocsQuery} in place of this query) or throw an exception if
- * the type is unmapped.
- */
- public HasParentQueryBuilder ignoreUnmapped(boolean ignoreUnmapped) {
- this.ignoreUnmapped = ignoreUnmapped;
- return this;
- }
-
- /**
- * Gets whether the query builder will ignore unmapped types (and run a
- * {@link MatchNoDocsQuery} in place of this query) or throw an exception if
- * the type is unmapped.
- */
- public boolean ignoreUnmapped() {
- return ignoreUnmapped;
- }
-
- @Override
- protected Query doToQuery(QueryShardContext context) throws IOException {
- Query innerQuery;
- String[] previousTypes = context.getTypes();
- context.setTypes(type);
- try {
- innerQuery = query.toQuery(context);
- } finally {
- context.setTypes(previousTypes);
- }
-
- DocumentMapper parentDocMapper = context.documentMapper(type);
- if (parentDocMapper == null) {
- if (ignoreUnmapped) {
- return new MatchNoDocsQuery();
- } else {
- throw new QueryShardException(context, "[" + NAME + "] query configured 'parent_type' [" + type + "] is not a valid type");
- }
- }
-
- Set<String> childTypes = new HashSet<>();
- ParentChildIndexFieldData parentChildIndexFieldData = null;
- for (DocumentMapper documentMapper : context.getMapperService().docMappers(false)) {
- ParentFieldMapper parentFieldMapper = documentMapper.parentFieldMapper();
- if (parentFieldMapper.active() && type.equals(parentFieldMapper.type())) {
- childTypes.add(documentMapper.type());
- parentChildIndexFieldData = context.getForField(parentFieldMapper.fieldType());
- }
- }
-
- if (childTypes.isEmpty()) {
- throw new QueryShardException(context, "[" + NAME + "] no child types found for type [" + type + "]");
- }
-
- Query childrenQuery;
- if (childTypes.size() == 1) {
- DocumentMapper documentMapper = context.getMapperService().documentMapper(childTypes.iterator().next());
- childrenQuery = documentMapper.typeFilter(context);
- } else {
- BooleanQuery.Builder childrenFilter = new BooleanQuery.Builder();
- for (String childrenTypeStr : childTypes) {
- DocumentMapper documentMapper = context.getMapperService().documentMapper(childrenTypeStr);
- childrenFilter.add(documentMapper.typeFilter(context), BooleanClause.Occur.SHOULD);
- }
- childrenQuery = childrenFilter.build();
- }
-
- // wrap the query with type query
- innerQuery = Queries.filtered(innerQuery, parentDocMapper.typeFilter(context));
- return new HasChildQueryBuilder.LateParsingQuery(childrenQuery,
- innerQuery,
- HasChildQueryBuilder.DEFAULT_MIN_CHILDREN,
- HasChildQueryBuilder.DEFAULT_MAX_CHILDREN,
- type,
- score ? ScoreMode.Max : ScoreMode.None,
- parentChildIndexFieldData,
- context.getSearchSimilarity());
- }
-
- @Override
- protected void doXContent(XContentBuilder builder, Params params) throws IOException {
- builder.startObject(NAME);
- builder.field(QUERY_FIELD.getPreferredName());
- query.toXContent(builder, params);
- builder.field(TYPE_FIELD.getPreferredName(), type);
- builder.field(SCORE_FIELD.getPreferredName(), score);
- builder.field(IGNORE_UNMAPPED_FIELD.getPreferredName(), ignoreUnmapped);
- printBoostAndQueryName(builder);
- if (innerHit != null) {
- builder.field(INNER_HITS_FIELD.getPreferredName(), innerHit, params);
- }
- builder.endObject();
- }
-
- public static HasParentQueryBuilder fromXContent(QueryParseContext parseContext) throws IOException {
- XContentParser parser = parseContext.parser();
- float boost = AbstractQueryBuilder.DEFAULT_BOOST;
- String parentType = null;
- boolean score = false;
- String queryName = null;
- InnerHitBuilder innerHits = null;
- boolean ignoreUnmapped = DEFAULT_IGNORE_UNMAPPED;
-
- String currentFieldName = null;
- XContentParser.Token token;
- QueryBuilder iqb = 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) {
- if (QUERY_FIELD.match(currentFieldName)) {
- iqb = parseContext.parseInnerQueryBuilder();
- } else if (INNER_HITS_FIELD.match(currentFieldName)) {
- innerHits = InnerHitBuilder.fromXContent(parseContext);
- } else {
- throw new ParsingException(parser.getTokenLocation(), "[has_parent] query does not support [" + currentFieldName + "]");
- }
- } else if (token.isValue()) {
- if (TYPE_FIELD.match(currentFieldName)) {
- parentType = parser.text();
- } else if (SCORE_MODE_FIELD.match(currentFieldName)) {
- String scoreModeValue = parser.text();
- if ("score".equals(scoreModeValue)) {
- score = true;
- } else if ("none".equals(scoreModeValue)) {
- score = false;
- } else {
- throw new ParsingException(parser.getTokenLocation(), "[has_parent] query does not support [" +
- scoreModeValue + "] as an option for score_mode");
- }
- } else if (SCORE_FIELD.match(currentFieldName)) {
- score = parser.booleanValue();
- } else if (IGNORE_UNMAPPED_FIELD.match(currentFieldName)) {
- ignoreUnmapped = parser.booleanValue();
- } else if (AbstractQueryBuilder.BOOST_FIELD.match(currentFieldName)) {
- boost = parser.floatValue();
- } else if (AbstractQueryBuilder.NAME_FIELD.match(currentFieldName)) {
- queryName = parser.text();
- } else {
- throw new ParsingException(parser.getTokenLocation(), "[has_parent] query does not support [" + currentFieldName + "]");
- }
- }
- }
- HasParentQueryBuilder queryBuilder = new HasParentQueryBuilder(parentType, iqb, score)
- .ignoreUnmapped(ignoreUnmapped)
- .queryName(queryName)
- .boost(boost);
- if (innerHits != null) {
- queryBuilder.innerHit(innerHits, ignoreUnmapped);
- }
- return queryBuilder;
- }
-
- @Override
- public String getWriteableName() {
- return NAME;
- }
-
- @Override
- protected boolean doEquals(HasParentQueryBuilder that) {
- return Objects.equals(query, that.query)
- && Objects.equals(type, that.type)
- && Objects.equals(score, that.score)
- && Objects.equals(innerHit, that.innerHit)
- && Objects.equals(ignoreUnmapped, that.ignoreUnmapped);
- }
-
- @Override
- protected int doHashCode() {
- return Objects.hash(query, type, score, innerHit, ignoreUnmapped);
- }
-
- @Override
- protected QueryBuilder doRewrite(QueryRewriteContext queryShardContext) throws IOException {
- QueryBuilder rewrittenQuery = query.rewrite(queryShardContext);
- if (rewrittenQuery != query) {
- InnerHitBuilder rewrittenInnerHit = InnerHitBuilder.rewrite(innerHit, rewrittenQuery);
- HasParentQueryBuilder hasParentQueryBuilder = new HasParentQueryBuilder(type, rewrittenQuery, score, rewrittenInnerHit);
- hasParentQueryBuilder.ignoreUnmapped(ignoreUnmapped);
- return hasParentQueryBuilder;
- }
- return this;
- }
-
- @Override
- protected void extractInnerHitBuilders(Map<String, InnerHitBuilder> innerHits) {
- if (innerHit!= null) {
- innerHit.inlineInnerHits(innerHits);
- }
- }
-}
diff --git a/core/src/main/java/org/elasticsearch/index/query/InnerHitBuilder.java b/core/src/main/java/org/elasticsearch/index/query/InnerHitBuilder.java
index dd23429dbd..b6eb84b03b 100644
--- a/core/src/main/java/org/elasticsearch/index/query/InnerHitBuilder.java
+++ b/core/src/main/java/org/elasticsearch/index/query/InnerHitBuilder.java
@@ -195,7 +195,8 @@ public final class InnerHitBuilder extends ToXContentToBytes implements Writeabl
}
}
- InnerHitBuilder(InnerHitBuilder other, QueryBuilder query, String parentChildType, boolean ignoreUnmapped) {
+ // NORELEASE Do not use this ctr, it is public for hasChild and hasParent query but this is temporary
+ public InnerHitBuilder(InnerHitBuilder other, QueryBuilder query, String parentChildType, boolean ignoreUnmapped) {
this(other);
this.query = query;
this.parentChildType = parentChildType;
@@ -751,7 +752,8 @@ public final class InnerHitBuilder extends ToXContentToBytes implements Writeabl
}
}
- static InnerHitBuilder rewrite(InnerHitBuilder original, QueryBuilder rewrittenQuery) {
+ // TODO public for hasParent and hasChild query
+ public static InnerHitBuilder rewrite(InnerHitBuilder original, QueryBuilder rewrittenQuery) {
if (original == null) {
return null;
}
diff --git a/core/src/main/java/org/elasticsearch/index/query/NestedQueryBuilder.java b/core/src/main/java/org/elasticsearch/index/query/NestedQueryBuilder.java
index b0c49a3f4a..2d23f256f0 100644
--- a/core/src/main/java/org/elasticsearch/index/query/NestedQueryBuilder.java
+++ b/core/src/main/java/org/elasticsearch/index/query/NestedQueryBuilder.java
@@ -23,7 +23,6 @@ import org.apache.lucene.search.MatchNoDocsQuery;
import org.apache.lucene.search.Query;
import org.apache.lucene.search.join.BitSetProducer;
import org.apache.lucene.search.join.ScoreMode;
-import org.apache.lucene.search.join.ToParentBlockJoinQuery;
import org.elasticsearch.common.ParseField;
import org.elasticsearch.common.ParsingException;
import org.elasticsearch.common.io.stream.StreamInput;
@@ -36,6 +35,7 @@ import org.elasticsearch.index.search.ESToParentBlockJoinQuery;
import org.elasticsearch.index.search.NestedHelper;
import java.io.IOException;
+import java.util.Locale;
import java.util.Map;
import java.util.Objects;
@@ -144,7 +144,7 @@ public class NestedQueryBuilder extends AbstractQueryBuilder<NestedQueryBuilder>
builder.field(PATH_FIELD.getPreferredName(), path);
builder.field(IGNORE_UNMAPPED_FIELD.getPreferredName(), ignoreUnmapped);
if (scoreMode != null) {
- builder.field(SCORE_MODE_FIELD.getPreferredName(), HasChildQueryBuilder.scoreModeAsString(scoreMode));
+ builder.field(SCORE_MODE_FIELD.getPreferredName(), scoreModeAsString(scoreMode));
}
printBoostAndQueryName(builder);
if (innerHitBuilder != null) {
@@ -183,7 +183,7 @@ public class NestedQueryBuilder extends AbstractQueryBuilder<NestedQueryBuilder>
} else if (IGNORE_UNMAPPED_FIELD.match(currentFieldName)) {
ignoreUnmapped = parser.booleanValue();
} else if (SCORE_MODE_FIELD.match(currentFieldName)) {
- scoreMode = HasChildQueryBuilder.parseScoreMode(parser.text());
+ scoreMode = parseScoreMode(parser.text());
} else if (AbstractQueryBuilder.NAME_FIELD.match(currentFieldName)) {
queryName = parser.text();
} else {
@@ -201,6 +201,30 @@ public class NestedQueryBuilder extends AbstractQueryBuilder<NestedQueryBuilder>
return queryBuilder;
}
+ public static ScoreMode parseScoreMode(String scoreModeString) {
+ if ("none".equals(scoreModeString)) {
+ return ScoreMode.None;
+ } else if ("min".equals(scoreModeString)) {
+ return ScoreMode.Min;
+ } else if ("max".equals(scoreModeString)) {
+ return ScoreMode.Max;
+ } else if ("avg".equals(scoreModeString)) {
+ return ScoreMode.Avg;
+ } else if ("sum".equals(scoreModeString)) {
+ return ScoreMode.Total;
+ }
+ throw new IllegalArgumentException("No score mode for child query [" + scoreModeString + "] found");
+ }
+
+ public static String scoreModeAsString(ScoreMode scoreMode) {
+ if (scoreMode == ScoreMode.Total) {
+ // Lucene uses 'total' but 'sum' is more consistent with other elasticsearch APIs
+ return "sum";
+ } else {
+ return scoreMode.name().toLowerCase(Locale.ROOT);
+ }
+ }
+
@Override
public final String getWriteableName() {
return NAME;
diff --git a/core/src/main/java/org/elasticsearch/index/query/QueryBuilders.java b/core/src/main/java/org/elasticsearch/index/query/QueryBuilders.java
index 805ec5fa78..df0493d61c 100644
--- a/core/src/main/java/org/elasticsearch/index/query/QueryBuilders.java
+++ b/core/src/main/java/org/elasticsearch/index/query/QueryBuilders.java
@@ -472,30 +472,6 @@ public abstract class QueryBuilders {
}
/**
- * Constructs a new has_child query, with the child type and the query to run on the child documents. The
- * results of this query are the parent docs that those child docs matched.
- *
- * @param type The child type.
- * @param query The query.
- * @param scoreMode How the scores from the children hits should be aggregated into the parent hit.
- */
- public static HasChildQueryBuilder hasChildQuery(String type, QueryBuilder query, ScoreMode scoreMode) {
- return new HasChildQueryBuilder(type, query, scoreMode);
- }
-
- /**
- * Constructs a new parent query, with the parent type and the query to run on the parent documents. The
- * results of this query are the children docs that those parent docs matched.
- *
- * @param type The parent type.
- * @param query The query.
- * @param score Whether the score from the parent hit should propagate to the child hit
- */
- public static HasParentQueryBuilder hasParentQuery(String type, QueryBuilder query, boolean score) {
- return new HasParentQueryBuilder(type, query, score);
- }
-
- /**
* Constructs a new parent id query that returns all child documents of the specified type that
* point to the specified id.
*/
diff --git a/core/src/main/java/org/elasticsearch/index/snapshots/blobstore/BlobStoreIndexShardSnapshots.java b/core/src/main/java/org/elasticsearch/index/snapshots/blobstore/BlobStoreIndexShardSnapshots.java
index 359c3165f5..56c343a5ae 100644
--- a/core/src/main/java/org/elasticsearch/index/snapshots/blobstore/BlobStoreIndexShardSnapshots.java
+++ b/core/src/main/java/org/elasticsearch/index/snapshots/blobstore/BlobStoreIndexShardSnapshots.java
@@ -284,7 +284,7 @@ public class BlobStoreIndexShardSnapshots implements Iterable<SnapshotFiles>, To
}
}
- List<SnapshotFiles> snapshots = new ArrayList<>();
+ List<SnapshotFiles> snapshots = new ArrayList<>(snapshotsMap.size());
for (Map.Entry<String, List<String>> entry : snapshotsMap.entrySet()) {
List<FileInfo> fileInfosBuilder = new ArrayList<>();
for (String file : entry.getValue()) {
diff --git a/core/src/main/java/org/elasticsearch/indices/breaker/HierarchyCircuitBreakerService.java b/core/src/main/java/org/elasticsearch/indices/breaker/HierarchyCircuitBreakerService.java
index 88cff7f559..b8ec92ba15 100644
--- a/core/src/main/java/org/elasticsearch/indices/breaker/HierarchyCircuitBreakerService.java
+++ b/core/src/main/java/org/elasticsearch/indices/breaker/HierarchyCircuitBreakerService.java
@@ -177,7 +177,7 @@ public class HierarchyCircuitBreakerService extends CircuitBreakerService {
@Override
public AllCircuitBreakerStats stats() {
long parentEstimated = 0;
- List<CircuitBreakerStats> allStats = new ArrayList<>();
+ List<CircuitBreakerStats> allStats = new ArrayList<>(this.breakers.size());
// Gather the "estimated" count for the parent breaker by adding the
// estimations for each individual breaker
for (CircuitBreaker breaker : this.breakers.values()) {
diff --git a/core/src/main/java/org/elasticsearch/indices/fielddata/cache/IndicesFieldDataCache.java b/core/src/main/java/org/elasticsearch/indices/fielddata/cache/IndicesFieldDataCache.java
index fcd925c258..ff7c3009dc 100644
--- a/core/src/main/java/org/elasticsearch/indices/fielddata/cache/IndicesFieldDataCache.java
+++ b/core/src/main/java/org/elasticsearch/indices/fielddata/cache/IndicesFieldDataCache.java
@@ -19,6 +19,7 @@
package org.elasticsearch.indices.fielddata.cache;
+import java.util.Collections;
import org.apache.logging.log4j.Logger;
import org.apache.lucene.index.DirectoryReader;
import org.apache.lucene.index.IndexReader;
@@ -134,9 +135,7 @@ public class IndicesFieldDataCache extends AbstractComponent implements RemovalL
//noinspection unchecked
final Accountable accountable = cache.computeIfAbsent(key, k -> {
cacheHelper.addClosedListener(IndexFieldCache.this);
- for (Listener listener : this.listeners) {
- k.listeners.add(listener);
- }
+ Collections.addAll(k.listeners, this.listeners);
final AtomicFieldData fieldData = indexFieldData.loadDirect(context);
for (Listener listener : k.listeners) {
try {
@@ -162,9 +161,7 @@ public class IndicesFieldDataCache extends AbstractComponent implements RemovalL
//noinspection unchecked
final Accountable accountable = cache.computeIfAbsent(key, k -> {
ElasticsearchDirectoryReader.addReaderCloseListener(indexReader, IndexFieldCache.this);
- for (Listener listener : this.listeners) {
- k.listeners.add(listener);
- }
+ Collections.addAll(k.listeners, this.listeners);
final Accountable ifd = (Accountable) indexFieldData.localGlobalDirect(indexReader);
for (Listener listener : k.listeners) {
try {
diff --git a/core/src/main/java/org/elasticsearch/node/InternalSettingsPreparer.java b/core/src/main/java/org/elasticsearch/node/InternalSettingsPreparer.java
index b8862fb20e..32ad3e9332 100644
--- a/core/src/main/java/org/elasticsearch/node/InternalSettingsPreparer.java
+++ b/core/src/main/java/org/elasticsearch/node/InternalSettingsPreparer.java
@@ -86,27 +86,23 @@ public class InternalSettingsPreparer {
initializeSettings(output, input, properties);
Environment environment = new Environment(output.build());
+ if (Files.exists(environment.configFile().resolve("elasticsearch.yaml"))) {
+ throw new SettingsException("elasticsearch.yaml was deprecated in 5.5.0 and must be renamed to elasticsearch.yml");
+ }
+
+ if (Files.exists(environment.configFile().resolve("elasticsearch.json"))) {
+ throw new SettingsException("elasticsearch.json was deprecated in 5.5.0 and must be converted to elasticsearch.yml");
+ }
+
output = Settings.builder(); // start with a fresh output
- boolean settingsFileFound = false;
- Set<String> foundSuffixes = new HashSet<>();
- for (String allowedSuffix : ALLOWED_SUFFIXES) {
- Path path = environment.configFile().resolve("elasticsearch" + allowedSuffix);
- if (Files.exists(path)) {
- if (!settingsFileFound) {
- try {
- output.loadFromPath(path);
- } catch (IOException e) {
- throw new SettingsException("Failed to load settings from " + path.toString(), e);
- }
- }
- settingsFileFound = true;
- foundSuffixes.add(allowedSuffix);
+ Path path = environment.configFile().resolve("elasticsearch.yml");
+ if (Files.exists(path)) {
+ try {
+ output.loadFromPath(path);
+ } catch (IOException e) {
+ throw new SettingsException("Failed to load settings from " + path.toString(), e);
}
}
- if (foundSuffixes.size() > 1) {
- throw new SettingsException("multiple settings files found with suffixes: "
- + Strings.collectionToDelimitedString(foundSuffixes, ","));
- }
// re-initialize settings now that the config file has been loaded
initializeSettings(output, input, properties);
@@ -116,8 +112,7 @@ public class InternalSettingsPreparer {
// we put back the path.logs so we can use it in the logging configuration file
output.put(Environment.PATH_LOGS_SETTING.getKey(), cleanPath(environment.logsFile().toAbsolutePath().toString()));
- String configExtension = foundSuffixes.isEmpty() ? null : foundSuffixes.iterator().next();
- return new Environment(output.build(), configExtension);
+ return new Environment(output.build());
}
/**
diff --git a/core/src/main/java/org/elasticsearch/node/Node.java b/core/src/main/java/org/elasticsearch/node/Node.java
index 0643c6b96a..08f9856c76 100644
--- a/core/src/main/java/org/elasticsearch/node/Node.java
+++ b/core/src/main/java/org/elasticsearch/node/Node.java
@@ -684,11 +684,9 @@ public class Node implements Closeable {
.flatMap(p -> p.getBootstrapChecks().stream()).collect(Collectors.toList()));
clusterService.addStateApplier(transportService.getTaskManager());
- clusterService.getMasterService().setClusterStateSupplier(discovery::clusterState);
- clusterService.getClusterApplierService().setInitialState(discovery.getInitialClusterState());
// start after transport service so the local disco is known
+ discovery.start(); // start before cluster service so that it can set initial state on ClusterApplierService
clusterService.start();
- discovery.start();
assert clusterService.localNode().equals(localNodeFactory.getNode())
: "clusterService has a different local node than the factory provided";
transportService.acceptIncomingRequests();
diff --git a/core/src/main/java/org/elasticsearch/search/SearchModule.java b/core/src/main/java/org/elasticsearch/search/SearchModule.java
index 8707d851d3..85308f0c24 100644
--- a/core/src/main/java/org/elasticsearch/search/SearchModule.java
+++ b/core/src/main/java/org/elasticsearch/search/SearchModule.java
@@ -20,8 +20,6 @@
package org.elasticsearch.search;
import org.apache.lucene.search.BooleanQuery;
-import org.elasticsearch.action.search.SearchRequest;
-import org.elasticsearch.action.search.SearchResponse;
import org.elasticsearch.common.NamedRegistry;
import org.elasticsearch.common.geo.ShapesAvailability;
import org.elasticsearch.common.geo.builders.ShapeBuilders;
@@ -45,8 +43,6 @@ import org.elasticsearch.index.query.GeoBoundingBoxQueryBuilder;
import org.elasticsearch.index.query.GeoDistanceQueryBuilder;
import org.elasticsearch.index.query.GeoPolygonQueryBuilder;
import org.elasticsearch.index.query.GeoShapeQueryBuilder;
-import org.elasticsearch.index.query.HasChildQueryBuilder;
-import org.elasticsearch.index.query.HasParentQueryBuilder;
import org.elasticsearch.index.query.IdsQueryBuilder;
import org.elasticsearch.index.query.MatchAllQueryBuilder;
import org.elasticsearch.index.query.MatchNoneQueryBuilder;
@@ -103,8 +99,6 @@ import org.elasticsearch.search.aggregations.InternalAggregation;
import org.elasticsearch.search.aggregations.PipelineAggregationBuilder;
import org.elasticsearch.search.aggregations.bucket.adjacency.AdjacencyMatrixAggregationBuilder;
import org.elasticsearch.search.aggregations.bucket.adjacency.InternalAdjacencyMatrix;
-import org.elasticsearch.search.aggregations.bucket.children.ChildrenAggregationBuilder;
-import org.elasticsearch.search.aggregations.bucket.children.InternalChildren;
import org.elasticsearch.search.aggregations.bucket.filter.FilterAggregationBuilder;
import org.elasticsearch.search.aggregations.bucket.filter.InternalFilter;
import org.elasticsearch.search.aggregations.bucket.filters.FiltersAggregationBuilder;
@@ -256,7 +250,6 @@ import org.elasticsearch.search.suggest.term.TermSuggestionBuilder;
import java.util.ArrayList;
import java.util.List;
import java.util.Map;
-import java.util.function.BiConsumer;
import java.util.function.Consumer;
import java.util.function.Function;
@@ -410,9 +403,6 @@ public class SearchModule {
GeoCentroidAggregationBuilder::parse).addResultReader(InternalGeoCentroid::new));
registerAggregation(new AggregationSpec(ScriptedMetricAggregationBuilder.NAME, ScriptedMetricAggregationBuilder::new,
ScriptedMetricAggregationBuilder::parse).addResultReader(InternalScriptedMetric::new));
- registerAggregation(new AggregationSpec(ChildrenAggregationBuilder.NAME, ChildrenAggregationBuilder::new,
- ChildrenAggregationBuilder::parse).addResultReader(InternalChildren::new));
-
registerFromPlugin(plugins, SearchPlugin::getAggregations, this::registerAggregation);
}
@@ -706,8 +696,6 @@ public class SearchModule {
MatchPhrasePrefixQueryBuilder::fromXContent));
registerQuery(new QuerySpec<>(MultiMatchQueryBuilder.NAME, MultiMatchQueryBuilder::new, MultiMatchQueryBuilder::fromXContent));
registerQuery(new QuerySpec<>(NestedQueryBuilder.NAME, NestedQueryBuilder::new, NestedQueryBuilder::fromXContent));
- registerQuery(new QuerySpec<>(HasChildQueryBuilder.NAME, HasChildQueryBuilder::new, HasChildQueryBuilder::fromXContent));
- registerQuery(new QuerySpec<>(HasParentQueryBuilder.NAME, HasParentQueryBuilder::new, HasParentQueryBuilder::fromXContent));
registerQuery(new QuerySpec<>(DisMaxQueryBuilder.NAME, DisMaxQueryBuilder::new, DisMaxQueryBuilder::fromXContent));
registerQuery(new QuerySpec<>(IdsQueryBuilder.NAME, IdsQueryBuilder::new, IdsQueryBuilder::fromXContent));
registerQuery(new QuerySpec<>(MatchAllQueryBuilder.NAME, MatchAllQueryBuilder::new, MatchAllQueryBuilder::fromXContent));
diff --git a/core/src/main/java/org/elasticsearch/search/aggregations/AggregationBuilders.java b/core/src/main/java/org/elasticsearch/search/aggregations/AggregationBuilders.java
index f43c2670ab..8b704ee8a6 100644
--- a/core/src/main/java/org/elasticsearch/search/aggregations/AggregationBuilders.java
+++ b/core/src/main/java/org/elasticsearch/search/aggregations/AggregationBuilders.java
@@ -23,8 +23,6 @@ import org.elasticsearch.common.geo.GeoPoint;
import org.elasticsearch.index.query.QueryBuilder;
import org.elasticsearch.search.aggregations.bucket.adjacency.AdjacencyMatrix;
import org.elasticsearch.search.aggregations.bucket.adjacency.AdjacencyMatrixAggregationBuilder;
-import org.elasticsearch.search.aggregations.bucket.children.Children;
-import org.elasticsearch.search.aggregations.bucket.children.ChildrenAggregationBuilder;
import org.elasticsearch.search.aggregations.bucket.filter.Filter;
import org.elasticsearch.search.aggregations.bucket.filter.FilterAggregationBuilder;
import org.elasticsearch.search.aggregations.bucket.filters.Filters;
@@ -163,20 +161,20 @@ public class AggregationBuilders {
public static FiltersAggregationBuilder filters(String name, QueryBuilder... filters) {
return new FiltersAggregationBuilder(name, filters);
}
-
+
/**
* Create a new {@link AdjacencyMatrix} aggregation with the given name.
*/
public static AdjacencyMatrixAggregationBuilder adjacencyMatrix(String name, Map<String, QueryBuilder> filters) {
return new AdjacencyMatrixAggregationBuilder(name, filters);
- }
-
+ }
+
/**
* Create a new {@link AdjacencyMatrix} aggregation with the given name and separator
*/
public static AdjacencyMatrixAggregationBuilder adjacencyMatrix(String name, String separator, Map<String, QueryBuilder> filters) {
return new AdjacencyMatrixAggregationBuilder(name, separator, filters);
- }
+ }
/**
* Create a new {@link Sampler} aggregation with the given name.
@@ -221,13 +219,6 @@ public class AggregationBuilders {
}
/**
- * Create a new {@link Children} aggregation with the given name.
- */
- public static ChildrenAggregationBuilder children(String name, String childType) {
- return new ChildrenAggregationBuilder(name, childType);
- }
-
- /**
* Create a new {@link GeoDistance} aggregation with the given name.
*/
public static GeoDistanceAggregationBuilder geoDistance(String name, GeoPoint origin) {
diff --git a/core/src/main/java/org/elasticsearch/search/aggregations/AggregatorFactories.java b/core/src/main/java/org/elasticsearch/search/aggregations/AggregatorFactories.java
index 106335380c..d635952ec1 100644
--- a/core/src/main/java/org/elasticsearch/search/aggregations/AggregatorFactories.java
+++ b/core/src/main/java/org/elasticsearch/search/aggregations/AggregatorFactories.java
@@ -183,7 +183,7 @@ public class AggregatorFactories {
}
public List<PipelineAggregator> createPipelineAggregators() throws IOException {
- List<PipelineAggregator> pipelineAggregators = new ArrayList<>();
+ List<PipelineAggregator> pipelineAggregators = new ArrayList<>(this.pipelineAggregatorFactories.size());
for (PipelineAggregationBuilder factory : this.pipelineAggregatorFactories) {
pipelineAggregators.add(factory.create());
}
diff --git a/core/src/main/java/org/elasticsearch/search/aggregations/BucketCollector.java b/core/src/main/java/org/elasticsearch/search/aggregations/BucketCollector.java
index 2de6ae0cf9..40e66bd964 100644
--- a/core/src/main/java/org/elasticsearch/search/aggregations/BucketCollector.java
+++ b/core/src/main/java/org/elasticsearch/search/aggregations/BucketCollector.java
@@ -70,7 +70,7 @@ public abstract class BucketCollector implements Collector {
@Override
public LeafBucketCollector getLeafCollector(LeafReaderContext ctx) throws IOException {
- List<LeafBucketCollector> leafCollectors = new ArrayList<>();
+ List<LeafBucketCollector> leafCollectors = new ArrayList<>(collectors.length);
for (BucketCollector c : collectors) {
leafCollectors.add(c.getLeafCollector(ctx));
}
diff --git a/core/src/main/java/org/elasticsearch/search/aggregations/bucket/adjacency/AdjacencyMatrixAggregationBuilder.java b/core/src/main/java/org/elasticsearch/search/aggregations/bucket/adjacency/AdjacencyMatrixAggregationBuilder.java
index 526db6a79b..eeb60d393e 100644
--- a/core/src/main/java/org/elasticsearch/search/aggregations/bucket/adjacency/AdjacencyMatrixAggregationBuilder.java
+++ b/core/src/main/java/org/elasticsearch/search/aggregations/bucket/adjacency/AdjacencyMatrixAggregationBuilder.java
@@ -197,7 +197,7 @@ public class AdjacencyMatrixAggregationBuilder extends AbstractAggregationBuilde
+ "] index level setting.");
}
- List<KeyedFilter> rewrittenFilters = new ArrayList<>();
+ List<KeyedFilter> rewrittenFilters = new ArrayList<>(filters.size());
for (KeyedFilter kf : filters) {
rewrittenFilters.add(new KeyedFilter(kf.key(), QueryBuilder.rewriteQuery(kf.filter(), context.getQueryShardContext())));
}
diff --git a/core/src/main/java/org/elasticsearch/search/aggregations/bucket/children/Children.java b/core/src/main/java/org/elasticsearch/search/aggregations/bucket/children/Children.java
deleted file mode 100644
index b1e4b2877a..0000000000
--- a/core/src/main/java/org/elasticsearch/search/aggregations/bucket/children/Children.java
+++ /dev/null
@@ -1,28 +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.search.aggregations.bucket.children;
-
-import org.elasticsearch.search.aggregations.bucket.SingleBucketAggregation;
-
-/**
- * An single bucket aggregation that translates parent documents to their children documents.
- */
-public interface Children extends SingleBucketAggregation {
-}
diff --git a/core/src/main/java/org/elasticsearch/search/aggregations/bucket/children/ChildrenAggregationBuilder.java b/core/src/main/java/org/elasticsearch/search/aggregations/bucket/children/ChildrenAggregationBuilder.java
deleted file mode 100644
index 3a0d2fff98..0000000000
--- a/core/src/main/java/org/elasticsearch/search/aggregations/bucket/children/ChildrenAggregationBuilder.java
+++ /dev/null
@@ -1,167 +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.search.aggregations.bucket.children;
-
-import org.apache.lucene.search.Query;
-import org.elasticsearch.common.ParsingException;
-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.fielddata.plain.ParentChildIndexFieldData;
-import org.elasticsearch.index.mapper.DocumentMapper;
-import org.elasticsearch.index.mapper.ParentFieldMapper;
-import org.elasticsearch.index.query.QueryParseContext;
-import org.elasticsearch.search.aggregations.AggregatorFactories.Builder;
-import org.elasticsearch.search.aggregations.AggregatorFactory;
-import org.elasticsearch.search.aggregations.support.FieldContext;
-import org.elasticsearch.search.aggregations.support.ValueType;
-import org.elasticsearch.search.aggregations.support.ValuesSource.Bytes.ParentChild;
-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.ValuesSourceType;
-import org.elasticsearch.search.internal.SearchContext;
-
-import java.io.IOException;
-import java.util.Objects;
-
-public class ChildrenAggregationBuilder extends ValuesSourceAggregationBuilder<ParentChild, ChildrenAggregationBuilder> {
- public static final String NAME = "children";
-
- private String parentType;
- private final String childType;
- private Query parentFilter;
- private Query childFilter;
-
- /**
- * @param name
- * the name of this aggregation
- * @param childType
- * the type of children documents
- */
- public ChildrenAggregationBuilder(String name, String childType) {
- super(name, ValuesSourceType.BYTES, ValueType.STRING);
- if (childType == null) {
- throw new IllegalArgumentException("[childType] must not be null: [" + name + "]");
- }
- this.childType = childType;
- }
-
- /**
- * Read from a stream.
- */
- public ChildrenAggregationBuilder(StreamInput in) throws IOException {
- super(in, ValuesSourceType.BYTES, ValueType.STRING);
- childType = in.readString();
- }
-
- @Override
- protected void innerWriteTo(StreamOutput out) throws IOException {
- out.writeString(childType);
- }
-
- @Override
- protected ValuesSourceAggregatorFactory<ParentChild, ?> innerBuild(SearchContext context,
- ValuesSourceConfig<ParentChild> config, AggregatorFactory<?> parent, Builder subFactoriesBuilder) throws IOException {
- return new ChildrenAggregatorFactory(name, config, parentType, childFilter, parentFilter, context, parent,
- subFactoriesBuilder, metaData);
- }
-
- @Override
- protected ValuesSourceConfig<ParentChild> resolveConfig(SearchContext context) {
- ValuesSourceConfig<ParentChild> config = new ValuesSourceConfig<>(ValuesSourceType.BYTES);
- DocumentMapper childDocMapper = context.mapperService().documentMapper(childType);
-
- if (childDocMapper != null) {
- ParentFieldMapper parentFieldMapper = childDocMapper.parentFieldMapper();
- if (!parentFieldMapper.active()) {
- throw new IllegalArgumentException("[children] no [_parent] field not configured that points to a parent type");
- }
- parentType = parentFieldMapper.type();
- DocumentMapper parentDocMapper = context.mapperService().documentMapper(parentType);
- if (parentDocMapper != null) {
- parentFilter = parentDocMapper.typeFilter(context.getQueryShardContext());
- childFilter = childDocMapper.typeFilter(context.getQueryShardContext());
- ParentChildIndexFieldData parentChildIndexFieldData = context.fieldData()
- .getForField(parentFieldMapper.fieldType());
- config.fieldContext(new FieldContext(parentFieldMapper.fieldType().name(), parentChildIndexFieldData,
- parentFieldMapper.fieldType()));
- } else {
- config.unmapped(true);
- }
- } else {
- config.unmapped(true);
- }
- return config;
- }
-
- @Override
- protected XContentBuilder doXContentBody(XContentBuilder builder, Params params) throws IOException {
- builder.field(ParentToChildrenAggregator.TYPE_FIELD.getPreferredName(), childType);
- return builder;
- }
-
- public static ChildrenAggregationBuilder parse(String aggregationName, QueryParseContext context) throws IOException {
- String childType = null;
-
- XContentParser.Token token;
- String currentFieldName = null;
- XContentParser parser = context.parser();
- while ((token = parser.nextToken()) != XContentParser.Token.END_OBJECT) {
- if (token == XContentParser.Token.FIELD_NAME) {
- currentFieldName = parser.currentName();
- } else if (token == XContentParser.Token.VALUE_STRING) {
- if ("type".equals(currentFieldName)) {
- childType = parser.text();
- } else {
- throw new ParsingException(parser.getTokenLocation(),
- "Unknown key for a " + token + " in [" + aggregationName + "]: [" + currentFieldName + "].");
- }
- } else {
- throw new ParsingException(parser.getTokenLocation(), "Unexpected token " + token + " in [" + aggregationName + "].");
- }
- }
-
- if (childType == null) {
- throw new ParsingException(parser.getTokenLocation(),
- "Missing [child_type] field for children aggregation [" + aggregationName + "]");
- }
-
-
- return new ChildrenAggregationBuilder(aggregationName, childType);
- }
-
- @Override
- protected int innerHashCode() {
- return Objects.hash(childType);
- }
-
- @Override
- protected boolean innerEquals(Object obj) {
- ChildrenAggregationBuilder other = (ChildrenAggregationBuilder) obj;
- return Objects.equals(childType, other.childType);
- }
-
- @Override
- public String getType() {
- return NAME;
- }
-}
diff --git a/core/src/main/java/org/elasticsearch/search/aggregations/bucket/children/ChildrenAggregatorFactory.java b/core/src/main/java/org/elasticsearch/search/aggregations/bucket/children/ChildrenAggregatorFactory.java
deleted file mode 100644
index b0a4c64305..0000000000
--- a/core/src/main/java/org/elasticsearch/search/aggregations/bucket/children/ChildrenAggregatorFactory.java
+++ /dev/null
@@ -1,77 +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.search.aggregations.bucket.children;
-
-import org.apache.lucene.search.Query;
-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.ValuesSource.Bytes.ParentChild;
-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 ChildrenAggregatorFactory
- extends ValuesSourceAggregatorFactory<ValuesSource.Bytes.WithOrdinals.ParentChild, ChildrenAggregatorFactory> {
-
- private final String parentType;
- private final Query parentFilter;
- private final Query childFilter;
-
- public ChildrenAggregatorFactory(String name, ValuesSourceConfig<ParentChild> config, String parentType, Query childFilter,
- Query parentFilter, SearchContext context, AggregatorFactory<?> parent, AggregatorFactories.Builder subFactoriesBuilder,
- Map<String, Object> metaData) throws IOException {
- super(name, config, context, parent, subFactoriesBuilder, metaData);
- this.parentType = parentType;
- this.childFilter = childFilter;
- this.parentFilter = parentFilter;
- }
-
- @Override
- protected Aggregator createUnmapped(Aggregator parent, List<PipelineAggregator> pipelineAggregators, Map<String, Object> metaData)
- throws IOException {
- return new NonCollectingAggregator(name, context, parent, pipelineAggregators, metaData) {
-
- @Override
- public InternalAggregation buildEmptyAggregation() {
- return new InternalChildren(name, 0, buildEmptySubAggregations(), pipelineAggregators(), metaData());
- }
-
- };
- }
-
- @Override
- protected Aggregator doCreateInternal(ValuesSource.Bytes.WithOrdinals.ParentChild valuesSource, Aggregator parent,
- boolean collectsFromSingleBucket, List<PipelineAggregator> pipelineAggregators, Map<String, Object> metaData)
- throws IOException {
- long maxOrd = valuesSource.globalMaxOrd(context.searcher(), parentType);
- return new ParentToChildrenAggregator(name, factories, context, parent, parentType, childFilter, parentFilter, valuesSource, maxOrd,
- pipelineAggregators, metaData);
- }
-
-}
diff --git a/core/src/main/java/org/elasticsearch/search/aggregations/bucket/children/InternalChildren.java b/core/src/main/java/org/elasticsearch/search/aggregations/bucket/children/InternalChildren.java
deleted file mode 100644
index 05a38c8cd5..0000000000
--- a/core/src/main/java/org/elasticsearch/search/aggregations/bucket/children/InternalChildren.java
+++ /dev/null
@@ -1,56 +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.search.aggregations.bucket.children;
-
-import org.elasticsearch.common.io.stream.StreamInput;
-import org.elasticsearch.search.aggregations.InternalAggregations;
-import org.elasticsearch.search.aggregations.bucket.InternalSingleBucketAggregation;
-import org.elasticsearch.search.aggregations.pipeline.PipelineAggregator;
-
-import java.io.IOException;
-import java.util.List;
-import java.util.Map;
-
-/**
- * Results of the {@link ParentToChildrenAggregator}.
- */
-public class InternalChildren extends InternalSingleBucketAggregation implements Children {
- public InternalChildren(String name, long docCount, InternalAggregations aggregations, List<PipelineAggregator> pipelineAggregators,
- Map<String, Object> metaData) {
- super(name, docCount, aggregations, pipelineAggregators, metaData);
- }
-
- /**
- * Read from a stream.
- */
- public InternalChildren(StreamInput in) throws IOException {
- super(in);
- }
-
- @Override
- public String getWriteableName() {
- return ChildrenAggregationBuilder.NAME;
- }
-
- @Override
- protected InternalSingleBucketAggregation newAggregation(String name, long docCount, InternalAggregations subAggregations) {
- return new InternalChildren(name, docCount, subAggregations, pipelineAggregators(), getMetaData());
- }
-}
diff --git a/core/src/main/java/org/elasticsearch/search/aggregations/bucket/children/ParentToChildrenAggregator.java b/core/src/main/java/org/elasticsearch/search/aggregations/bucket/children/ParentToChildrenAggregator.java
deleted file mode 100644
index 37a443e9ba..0000000000
--- a/core/src/main/java/org/elasticsearch/search/aggregations/bucket/children/ParentToChildrenAggregator.java
+++ /dev/null
@@ -1,186 +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.search.aggregations.bucket.children;
-
-import org.apache.lucene.index.IndexReader;
-import org.apache.lucene.index.LeafReaderContext;
-import org.apache.lucene.index.SortedDocValues;
-import org.apache.lucene.search.ConstantScoreScorer;
-import org.apache.lucene.search.DocIdSetIterator;
-import org.apache.lucene.search.Query;
-import org.apache.lucene.search.Scorer;
-import org.apache.lucene.search.Weight;
-import org.apache.lucene.util.Bits;
-import org.elasticsearch.common.ParseField;
-import org.elasticsearch.common.lease.Releasables;
-import org.elasticsearch.common.lucene.Lucene;
-import org.elasticsearch.common.util.LongArray;
-import org.elasticsearch.common.util.LongObjectPagedHashMap;
-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.bucket.SingleBucketAggregator;
-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.Arrays;
-import java.util.List;
-import java.util.Map;
-
-// The RecordingPerReaderBucketCollector assumes per segment recording which isn't the case for this
-// aggregation, for this reason that collector can't be used
-public class ParentToChildrenAggregator extends SingleBucketAggregator {
-
- static final ParseField TYPE_FIELD = new ParseField("type");
-
- private final String parentType;
- private final Weight childFilter;
- private final Weight parentFilter;
- private final ValuesSource.Bytes.WithOrdinals.ParentChild valuesSource;
-
- // Maybe use PagedGrowableWriter? This will be less wasteful than LongArray,
- // but then we don't have the reuse feature of BigArrays.
- // Also if we know the highest possible value that a parent agg will create
- // then we store multiple values into one slot
- private final LongArray parentOrdToBuckets;
-
- // Only pay the extra storage price if the a parentOrd has multiple buckets
- // Most of the times a parent doesn't have multiple buckets, since there is
- // only one document per parent ord,
- // only in the case of terms agg if a parent doc has multiple terms per
- // field this is needed:
- private final LongObjectPagedHashMap<long[]> parentOrdToOtherBuckets;
- private boolean multipleBucketsPerParentOrd = false;
-
- public ParentToChildrenAggregator(String name, AggregatorFactories factories,
- SearchContext context, Aggregator parent, String parentType, Query childFilter,
- Query parentFilter, ValuesSource.Bytes.WithOrdinals.ParentChild valuesSource,
- long maxOrd, List<PipelineAggregator> pipelineAggregators, Map<String, Object> metaData)
- throws IOException {
- super(name, factories, context, parent, pipelineAggregators, metaData);
- this.parentType = parentType;
- // these two filters are cached in the parser
- this.childFilter = context.searcher().createNormalizedWeight(childFilter, false);
- this.parentFilter = context.searcher().createNormalizedWeight(parentFilter, false);
- this.parentOrdToBuckets = context.bigArrays().newLongArray(maxOrd, false);
- this.parentOrdToBuckets.fill(0, maxOrd, -1);
- this.parentOrdToOtherBuckets = new LongObjectPagedHashMap<>(context.bigArrays());
- this.valuesSource = valuesSource;
- }
-
- @Override
- public InternalAggregation buildAggregation(long owningBucketOrdinal) throws IOException {
- return new InternalChildren(name, bucketDocCount(owningBucketOrdinal),
- bucketAggregations(owningBucketOrdinal), pipelineAggregators(), metaData());
- }
-
- @Override
- public InternalAggregation buildEmptyAggregation() {
- return new InternalChildren(name, 0, buildEmptySubAggregations(), pipelineAggregators(),
- metaData());
- }
-
- @Override
- public LeafBucketCollector getLeafCollector(LeafReaderContext ctx,
- final LeafBucketCollector sub) throws IOException {
- if (valuesSource == null) {
- return LeafBucketCollector.NO_OP_COLLECTOR;
- }
-
- final SortedDocValues globalOrdinals = valuesSource.globalOrdinalsValues(parentType, ctx);
- assert globalOrdinals != null;
- Scorer parentScorer = parentFilter.scorer(ctx);
- final Bits parentDocs = Lucene.asSequentialAccessBits(ctx.reader().maxDoc(), parentScorer);
- return new LeafBucketCollector() {
-
- @Override
- public void collect(int docId, long bucket) throws IOException {
- if (parentDocs.get(docId) && globalOrdinals.advanceExact(docId)) {
- long globalOrdinal = globalOrdinals.ordValue();
- if (globalOrdinal != -1) {
- if (parentOrdToBuckets.get(globalOrdinal) == -1) {
- parentOrdToBuckets.set(globalOrdinal, bucket);
- } else {
- long[] bucketOrds = parentOrdToOtherBuckets.get(globalOrdinal);
- if (bucketOrds != null) {
- bucketOrds = Arrays.copyOf(bucketOrds, bucketOrds.length + 1);
- bucketOrds[bucketOrds.length - 1] = bucket;
- parentOrdToOtherBuckets.put(globalOrdinal, bucketOrds);
- } else {
- parentOrdToOtherBuckets.put(globalOrdinal, new long[] { bucket });
- }
- multipleBucketsPerParentOrd = true;
- }
- }
- }
- }
- };
- }
-
- @Override
- protected void doPostCollection() throws IOException {
- IndexReader indexReader = context().searcher().getIndexReader();
- for (LeafReaderContext ctx : indexReader.leaves()) {
- Scorer childDocsScorer = childFilter.scorer(ctx);
- if (childDocsScorer == null) {
- continue;
- }
- DocIdSetIterator childDocsIter = childDocsScorer.iterator();
-
- final LeafBucketCollector sub = collectableSubAggregators.getLeafCollector(ctx);
- final SortedDocValues globalOrdinals = valuesSource.globalOrdinalsValues(parentType,
- ctx);
-
- // Set the scorer, since we now replay only the child docIds
- sub.setScorer(new ConstantScoreScorer(null, 1f, childDocsIter));
-
- final Bits liveDocs = ctx.reader().getLiveDocs();
- for (int docId = childDocsIter
- .nextDoc(); docId != DocIdSetIterator.NO_MORE_DOCS; docId = childDocsIter
- .nextDoc()) {
- if (liveDocs != null && liveDocs.get(docId) == false) {
- continue;
- }
- if (globalOrdinals.advanceExact(docId)) {
- long globalOrdinal = globalOrdinals.ordValue();
- long bucketOrd = parentOrdToBuckets.get(globalOrdinal);
- if (bucketOrd != -1) {
- collectBucket(sub, docId, bucketOrd);
- if (multipleBucketsPerParentOrd) {
- long[] otherBucketOrds = parentOrdToOtherBuckets.get(globalOrdinal);
- if (otherBucketOrds != null) {
- for (long otherBucketOrd : otherBucketOrds) {
- collectBucket(sub, docId, otherBucketOrd);
- }
- }
- }
- }
- }
- }
- }
- }
-
- @Override
- protected void doClose() {
- Releasables.close(parentOrdToBuckets, parentOrdToOtherBuckets);
- }
-}
diff --git a/core/src/main/java/org/elasticsearch/search/aggregations/bucket/children/ParsedChildren.java b/core/src/main/java/org/elasticsearch/search/aggregations/bucket/children/ParsedChildren.java
deleted file mode 100644
index 9ce6661923..0000000000
--- a/core/src/main/java/org/elasticsearch/search/aggregations/bucket/children/ParsedChildren.java
+++ /dev/null
@@ -1,36 +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.search.aggregations.bucket.children;
-
-import org.elasticsearch.common.xcontent.XContentParser;
-import org.elasticsearch.search.aggregations.bucket.ParsedSingleBucketAggregation;
-
-import java.io.IOException;
-
-public class ParsedChildren extends ParsedSingleBucketAggregation implements Children {
-
- @Override
- public String getType() {
- return ChildrenAggregationBuilder.NAME;
- }
-
- public static ParsedChildren fromXContent(XContentParser parser, final String name) throws IOException {
- return parseXContent(parser, new ParsedChildren(), name);
- }
-}
diff --git a/core/src/main/java/org/elasticsearch/search/aggregations/bucket/filters/FiltersAggregationBuilder.java b/core/src/main/java/org/elasticsearch/search/aggregations/bucket/filters/FiltersAggregationBuilder.java
index f34df7368a..b80ce74c3d 100644
--- a/core/src/main/java/org/elasticsearch/search/aggregations/bucket/filters/FiltersAggregationBuilder.java
+++ b/core/src/main/java/org/elasticsearch/search/aggregations/bucket/filters/FiltersAggregationBuilder.java
@@ -169,7 +169,7 @@ public class FiltersAggregationBuilder extends AbstractAggregationBuilder<Filter
@Override
protected AggregatorFactory<?> doBuild(SearchContext context, AggregatorFactory<?> parent, Builder subFactoriesBuilder)
throws IOException {
- List<KeyedFilter> rewrittenFilters = new ArrayList<>();
+ List<KeyedFilter> rewrittenFilters = new ArrayList<>(filters.size());
for(KeyedFilter kf : filters) {
rewrittenFilters.add(new KeyedFilter(kf.key(), QueryBuilder.rewriteQuery(kf.filter(),
context.getQueryShardContext())));
diff --git a/core/src/main/java/org/elasticsearch/search/aggregations/bucket/terms/DoubleTerms.java b/core/src/main/java/org/elasticsearch/search/aggregations/bucket/terms/DoubleTerms.java
index e4885bc053..b87ce51c4b 100644
--- a/core/src/main/java/org/elasticsearch/search/aggregations/bucket/terms/DoubleTerms.java
+++ b/core/src/main/java/org/elasticsearch/search/aggregations/bucket/terms/DoubleTerms.java
@@ -164,7 +164,7 @@ public class DoubleTerms extends InternalMappedTerms<DoubleTerms, DoubleTerms.Bu
if (promoteToDouble == false) {
return super.doReduce(aggregations, reduceContext);
}
- List<InternalAggregation> newAggs = new ArrayList<>();
+ List<InternalAggregation> newAggs = new ArrayList<>(aggregations.size());
for (InternalAggregation agg : aggregations) {
if (agg instanceof LongTerms) {
DoubleTerms dTerms = LongTerms.convertLongTermsToDouble((LongTerms) agg, format);
diff --git a/core/src/main/java/org/elasticsearch/search/aggregations/pipeline/cumulativesum/CumulativeSumPipelineAggregator.java b/core/src/main/java/org/elasticsearch/search/aggregations/pipeline/cumulativesum/CumulativeSumPipelineAggregator.java
index e79ba1047e..8a1b70fdd1 100644
--- a/core/src/main/java/org/elasticsearch/search/aggregations/pipeline/cumulativesum/CumulativeSumPipelineAggregator.java
+++ b/core/src/main/java/org/elasticsearch/search/aggregations/pipeline/cumulativesum/CumulativeSumPipelineAggregator.java
@@ -75,7 +75,7 @@ public class CumulativeSumPipelineAggregator extends PipelineAggregator {
InternalMultiBucketAggregation.InternalBucket>) aggregation;
List<? extends InternalMultiBucketAggregation.InternalBucket> buckets = histo.getBuckets();
HistogramFactory factory = (HistogramFactory) histo;
- List<Bucket> newBuckets = new ArrayList<>();
+ List<Bucket> newBuckets = new ArrayList<>(buckets.size());
double sum = 0;
for (InternalMultiBucketAggregation.InternalBucket bucket : buckets) {
Double thisBucketValue = resolveBucketValue(histo, bucket, bucketsPaths()[0], GapPolicy.INSERT_ZEROS);
diff --git a/core/src/main/java/org/elasticsearch/search/fetch/subphase/highlight/CustomQueryScorer.java b/core/src/main/java/org/elasticsearch/search/fetch/subphase/highlight/CustomQueryScorer.java
index e25e7b7494..2f728e21b6 100644
--- a/core/src/main/java/org/elasticsearch/search/fetch/subphase/highlight/CustomQueryScorer.java
+++ b/core/src/main/java/org/elasticsearch/search/fetch/subphase/highlight/CustomQueryScorer.java
@@ -26,7 +26,6 @@ import org.apache.lucene.search.highlight.WeightedSpanTerm;
import org.apache.lucene.search.highlight.WeightedSpanTermExtractor;
import org.elasticsearch.common.lucene.search.function.FiltersFunctionScoreQuery;
import org.elasticsearch.common.lucene.search.function.FunctionScoreQuery;
-import org.elasticsearch.index.query.HasChildQueryBuilder;
import java.io.IOException;
import java.util.Map;
@@ -83,7 +82,7 @@ public final class CustomQueryScorer extends QueryScorer {
}
protected void extract(Query query, float boost, Map<String, WeightedSpanTerm> terms) throws IOException {
- if (query instanceof HasChildQueryBuilder.LateParsingQuery) {
+ if (isChildOrParentQuery(query.getClass())) {
// skip has_child or has_parent queries, see: https://github.com/elastic/elasticsearch/issues/14999
return;
} else if (query instanceof FunctionScoreQuery) {
@@ -94,5 +93,13 @@ public final class CustomQueryScorer extends QueryScorer {
super.extract(query, boost, terms);
}
}
+
+ /**
+ * Workaround to detect parent/child query
+ */
+ private static final String PARENT_CHILD_QUERY_NAME = "HasChildQueryBuilder$LateParsingQuery";
+ private static boolean isChildOrParentQuery(Class<?> clazz) {
+ return clazz.getName().endsWith(PARENT_CHILD_QUERY_NAME);
+ }
}
}
diff --git a/core/src/main/java/org/elasticsearch/search/suggest/phrase/DirectCandidateGenerator.java b/core/src/main/java/org/elasticsearch/search/suggest/phrase/DirectCandidateGenerator.java
index 24f2647167..b874c3aeca 100644
--- a/core/src/main/java/org/elasticsearch/search/suggest/phrase/DirectCandidateGenerator.java
+++ b/core/src/main/java/org/elasticsearch/search/suggest/phrase/DirectCandidateGenerator.java
@@ -229,9 +229,8 @@ public final class DirectCandidateGenerator extends CandidateGenerator {
// Merge new candidates into existing ones,
// deduping:
final Set<Candidate> set = new HashSet<>(candidates);
- for (int i = 0; i < this.candidates.length; i++) {
- set.add(this.candidates[i]);
- }
+ Collections.addAll(set, this.candidates);
+
this.candidates = set.toArray(new Candidate[set.size()]);
// Sort strongest to weakest:
Arrays.sort(this.candidates, Collections.reverseOrder());
diff --git a/core/src/main/java/org/elasticsearch/transport/TcpTransport.java b/core/src/main/java/org/elasticsearch/transport/TcpTransport.java
index 30a18e0561..ba355e4147 100644
--- a/core/src/main/java/org/elasticsearch/transport/TcpTransport.java
+++ b/core/src/main/java/org/elasticsearch/transport/TcpTransport.java
@@ -101,6 +101,7 @@ import java.util.concurrent.atomic.AtomicLong;
import java.util.concurrent.atomic.AtomicReference;
import java.util.concurrent.locks.ReadWriteLock;
import java.util.concurrent.locks.ReentrantReadWriteLock;
+import java.util.function.Consumer;
import java.util.regex.Matcher;
import java.util.regex.Pattern;
import java.util.stream.Collectors;
@@ -357,8 +358,9 @@ public abstract class TcpTransport<Channel> extends AbstractLifecycleComponent i
private final DiscoveryNode node;
private final AtomicBoolean closed = new AtomicBoolean(false);
private final Version version;
+ private final Consumer<Connection> onClose;
- public NodeChannels(DiscoveryNode node, Channel[] channels, ConnectionProfile connectionProfile) {
+ public NodeChannels(DiscoveryNode node, Channel[] channels, ConnectionProfile connectionProfile, Consumer<Connection> onClose) {
this.node = node;
this.channels = channels;
assert channels.length == connectionProfile.getNumConnections() : "expected channels size to be == "
@@ -369,6 +371,7 @@ public abstract class TcpTransport<Channel> extends AbstractLifecycleComponent i
typeMapping.put(type, handle);
}
version = node.getVersion();
+ this.onClose = onClose;
}
NodeChannels(NodeChannels channels, Version handshakeVersion) {
@@ -376,6 +379,7 @@ public abstract class TcpTransport<Channel> extends AbstractLifecycleComponent i
this.channels = channels.channels;
this.typeMapping = channels.typeMapping;
this.version = handshakeVersion;
+ this.onClose = channels.onClose;
}
@Override
@@ -407,7 +411,11 @@ public abstract class TcpTransport<Channel> extends AbstractLifecycleComponent i
@Override
public synchronized void close() throws IOException {
if (closed.compareAndSet(false, true)) {
- closeChannels(Arrays.stream(channels).filter(Objects::nonNull).collect(Collectors.toList()));
+ try {
+ closeChannels(Arrays.stream(channels).filter(Objects::nonNull).collect(Collectors.toList()));
+ } finally {
+ onClose.accept(this);
+ }
}
}
@@ -519,8 +527,8 @@ public abstract class TcpTransport<Channel> extends AbstractLifecycleComponent i
final TimeValue handshakeTimeout = connectionProfile.getHandshakeTimeout() == null ?
connectTimeout : connectionProfile.getHandshakeTimeout();
final Version version = executeHandshake(node, channel, handshakeTimeout);
- transportServiceAdapter.onConnectionOpened(node);
- nodeChannels = new NodeChannels(nodeChannels, version);// clone the channels - we now have the correct version
+ transportServiceAdapter.onConnectionOpened(nodeChannels);
+ nodeChannels = new NodeChannels(nodeChannels, version); // clone the channels - we now have the correct version
success = true;
return nodeChannels;
} catch (ConnectTransportException e) {
diff --git a/core/src/main/java/org/elasticsearch/transport/Transport.java b/core/src/main/java/org/elasticsearch/transport/Transport.java
index d3dcd8bb5c..a32289332e 100644
--- a/core/src/main/java/org/elasticsearch/transport/Transport.java
+++ b/core/src/main/java/org/elasticsearch/transport/Transport.java
@@ -132,5 +132,13 @@ public interface Transport extends LifecycleComponent {
default Version getVersion() {
return getNode().getVersion();
}
+
+ /**
+ * Returns a key that this connection can be cached on. Delegating subclasses must delegate method call to
+ * the original connection.
+ */
+ default Object getCacheKey() {
+ return this;
+ }
}
}
diff --git a/core/src/main/java/org/elasticsearch/transport/TransportConnectionListener.java b/core/src/main/java/org/elasticsearch/transport/TransportConnectionListener.java
index 3f277a0ee1..de767986b9 100644
--- a/core/src/main/java/org/elasticsearch/transport/TransportConnectionListener.java
+++ b/core/src/main/java/org/elasticsearch/transport/TransportConnectionListener.java
@@ -34,7 +34,13 @@ public interface TransportConnectionListener {
default void onNodeDisconnected(DiscoveryNode node) {}
/**
+ * Called once a node connection is closed. The connection might not have been registered in the
+ * transport as a shared connection to a specific node
+ */
+ default void onConnectionClosed(Transport.Connection connection) {}
+
+ /**
* Called once a node connection is opened.
*/
- default void onConnectionOpened(DiscoveryNode node) {}
+ default void onConnectionOpened(Transport.Connection connection) {}
}
diff --git a/core/src/main/java/org/elasticsearch/transport/TransportService.java b/core/src/main/java/org/elasticsearch/transport/TransportService.java
index 7de9606361..e5382e4e26 100644
--- a/core/src/main/java/org/elasticsearch/transport/TransportService.java
+++ b/core/src/main/java/org/elasticsearch/transport/TransportService.java
@@ -569,7 +569,7 @@ public class TransportService extends AbstractLifecycleComponent {
}
Supplier<ThreadContext.StoredContext> storedContextSupplier = threadPool.getThreadContext().newRestorableContext(true);
TransportResponseHandler<T> responseHandler = new ContextRestoreResponseHandler<>(storedContextSupplier, handler);
- clientHandlers.put(requestId, new RequestHolder<>(responseHandler, connection.getNode(), action, timeoutHandler));
+ clientHandlers.put(requestId, new RequestHolder<>(responseHandler, connection, action, timeoutHandler));
if (lifecycle.stoppedOrClosed()) {
// if we are not started the exception handling will remove the RequestHolder again and calls the handler to notify
// the caller. It will only notify if the toStop code hasn't done the work yet.
@@ -810,7 +810,7 @@ public class TransportService extends AbstractLifecycleComponent {
}
holder.cancelTimeout();
if (traceEnabled() && shouldTraceAction(holder.action())) {
- traceReceivedResponse(requestId, holder.node(), holder.action());
+ traceReceivedResponse(requestId, holder.connection().getNode(), holder.action());
}
return holder.handler();
}
@@ -855,12 +855,12 @@ public class TransportService extends AbstractLifecycleComponent {
}
@Override
- public void onConnectionOpened(DiscoveryNode node) {
+ public void onConnectionOpened(Transport.Connection connection) {
// capture listeners before spawning the background callback so the following pattern won't trigger a call
// connectToNode(); connection is completed successfully
// addConnectionListener(); this listener shouldn't be called
final Stream<TransportConnectionListener> listenersToNotify = TransportService.this.connectionListeners.stream();
- threadPool.generic().execute(() -> listenersToNotify.forEach(listener -> listener.onConnectionOpened(node)));
+ threadPool.generic().execute(() -> listenersToNotify.forEach(listener -> listener.onConnectionOpened(connection)));
}
@Override
@@ -871,20 +871,28 @@ public class TransportService extends AbstractLifecycleComponent {
connectionListener.onNodeDisconnected(node);
}
});
+ } catch (EsRejectedExecutionException ex) {
+ logger.debug("Rejected execution on NodeDisconnected", ex);
+ }
+ }
+
+ @Override
+ public void onConnectionClosed(Transport.Connection connection) {
+ try {
for (Map.Entry<Long, RequestHolder> entry : clientHandlers.entrySet()) {
RequestHolder holder = entry.getValue();
- if (holder.node().equals(node)) {
+ if (holder.connection().getCacheKey().equals(connection.getCacheKey())) {
final RequestHolder holderToNotify = clientHandlers.remove(entry.getKey());
if (holderToNotify != null) {
// callback that an exception happened, but on a different thread since we don't
// want handlers to worry about stack overflows
- threadPool.generic().execute(() -> holderToNotify.handler().handleException(new NodeDisconnectedException(node,
- holderToNotify.action())));
+ threadPool.generic().execute(() -> holderToNotify.handler().handleException(new NodeDisconnectedException(
+ connection.getNode(), holderToNotify.action())));
}
}
}
} catch (EsRejectedExecutionException ex) {
- logger.debug("Rejected execution on NodeDisconnected", ex);
+ logger.debug("Rejected execution on onConnectionClosed", ex);
}
}
@@ -929,13 +937,14 @@ public class TransportService extends AbstractLifecycleComponent {
if (holder != null) {
// add it to the timeout information holder, in case we are going to get a response later
long timeoutTime = System.currentTimeMillis();
- timeoutInfoHandlers.put(requestId, new TimeoutInfoHolder(holder.node(), holder.action(), sentTime, timeoutTime));
+ timeoutInfoHandlers.put(requestId, new TimeoutInfoHolder(holder.connection().getNode(), holder.action(), sentTime,
+ timeoutTime));
// now that we have the information visible via timeoutInfoHandlers, we try to remove the request id
final RequestHolder removedHolder = clientHandlers.remove(requestId);
if (removedHolder != null) {
assert removedHolder == holder : "two different holder instances for request [" + requestId + "]";
removedHolder.handler().handleException(
- new ReceiveTimeoutTransportException(holder.node(), holder.action(),
+ new ReceiveTimeoutTransportException(holder.connection().getNode(), holder.action(),
"request_id [" + requestId + "] timed out after [" + (timeoutTime - sentTime) + "ms]"));
} else {
// response was processed, remove timeout info.
@@ -990,15 +999,15 @@ public class TransportService extends AbstractLifecycleComponent {
private final TransportResponseHandler<T> handler;
- private final DiscoveryNode node;
+ private final Transport.Connection connection;
private final String action;
private final TimeoutHandler timeoutHandler;
- RequestHolder(TransportResponseHandler<T> handler, DiscoveryNode node, String action, TimeoutHandler timeoutHandler) {
+ RequestHolder(TransportResponseHandler<T> handler, Transport.Connection connection, String action, TimeoutHandler timeoutHandler) {
this.handler = handler;
- this.node = node;
+ this.connection = connection;
this.action = action;
this.timeoutHandler = timeoutHandler;
}
@@ -1007,8 +1016,8 @@ public class TransportService extends AbstractLifecycleComponent {
return handler;
}
- public DiscoveryNode node() {
- return this.node;
+ public Transport.Connection connection() {
+ return this.connection;
}
public String action() {
diff --git a/core/src/test/java/org/elasticsearch/action/bulk/BulkWithUpdatesIT.java b/core/src/test/java/org/elasticsearch/action/bulk/BulkWithUpdatesIT.java
index 21930b3763..cf41042ab8 100644
--- a/core/src/test/java/org/elasticsearch/action/bulk/BulkWithUpdatesIT.java
+++ b/core/src/test/java/org/elasticsearch/action/bulk/BulkWithUpdatesIT.java
@@ -450,156 +450,7 @@ public class BulkWithUpdatesIT extends ESIntegTestCase {
assertHitCount(countResponse, numDocs);
}
- /*
- Test for https://github.com/elastic/elasticsearch/issues/3444
- */
- public void testBulkUpdateDocAsUpsertWithParent() throws Exception {
- client().admin().indices().prepareCreate("test")
- .setSettings("index.mapping.single_type", false)
- .addMapping("parent", "{\"parent\":{}}", XContentType.JSON)
- .addMapping("child", "{\"child\": {\"_parent\": {\"type\": \"parent\"}}}", XContentType.JSON)
- .execute().actionGet();
- ensureGreen();
-
- BulkRequestBuilder builder = client().prepareBulk();
-
- // It's important to use JSON parsing here and request objects: issue 3444 is related to incomplete option parsing
- byte[] addParent = new BytesArray(
- "{" +
- " \"index\" : {" +
- " \"_index\" : \"test\"," +
- " \"_type\" : \"parent\"," +
- " \"_id\" : \"parent1\"" +
- " }" +
- "}" +
- "\n" +
- "{" +
- " \"field1\" : \"value1\"" +
- "}" +
- "\n").array();
-
- byte[] addChild = new BytesArray(
- "{" +
- " \"update\" : {" +
- " \"_index\" : \"test\"," +
- " \"_type\" : \"child\"," +
- " \"_id\" : \"child1\"," +
- " \"parent\" : \"parent1\"" +
- " }" +
- "}" +
- "\n" +
- "{" +
- " \"doc\" : {" +
- " \"field1\" : \"value1\"" +
- " }," +
- " \"doc_as_upsert\" : \"true\"" +
- "}" +
- "\n").array();
-
- builder.add(addParent, 0, addParent.length, XContentType.JSON);
- builder.add(addChild, 0, addChild.length, XContentType.JSON);
-
- BulkResponse bulkResponse = builder.get();
- assertThat(bulkResponse.getItems().length, equalTo(2));
- assertThat(bulkResponse.getItems()[0].isFailed(), equalTo(false));
- assertThat(bulkResponse.getItems()[1].isFailed(), equalTo(false));
-
- client().admin().indices().prepareRefresh("test").get();
- //we check that the _parent field was set on the child document by using the has parent query
- SearchResponse searchResponse = client().prepareSearch("test")
- .setQuery(QueryBuilders.hasParentQuery("parent", QueryBuilders.matchAllQuery(), false))
- .get();
-
- assertNoFailures(searchResponse);
- assertSearchHits(searchResponse, "child1");
- }
-
- /*
- Test for https://github.com/elastic/elasticsearch/issues/3444
- */
- public void testBulkUpdateUpsertWithParent() throws Exception {
- assertAcked(prepareCreate("test")
- .setSettings("index.mapping.single_type", false)
- .addMapping("parent", "{\"parent\":{}}", XContentType.JSON)
- .addMapping("child", "{\"child\": {\"_parent\": {\"type\": \"parent\"}}}", XContentType.JSON));
- ensureGreen();
-
- BulkRequestBuilder builder = client().prepareBulk();
-
- byte[] addParent = new BytesArray(
- "{" +
- " \"index\" : {" +
- " \"_index\" : \"test\"," +
- " \"_type\" : \"parent\"," +
- " \"_id\" : \"parent1\"" +
- " }" +
- "}" +
- "\n" +
- "{" +
- " \"field1\" : \"value1\"" +
- "}" +
- "\n").array();
-
- byte[] addChild1 = new BytesArray(
- "{" +
- " \"update\" : {" +
- " \"_index\" : \"test\"," +
- " \"_type\" : \"child\"," +
- " \"_id\" : \"child1\"," +
- " \"parent\" : \"parent1\"" +
- " }" +
- "}" +
- "\n" +
- "{" +
- " \"script\" : {" +
- " \"inline\" : \"ctx._source.field2 = 'value2'\"" +
- " }," +
- " \"lang\" : \"" + CustomScriptPlugin.NAME + "\"," +
- " \"upsert\" : {" +
- " \"field1\" : \"value1'\"" +
- " }" +
- "}" +
- "\n").array();
-
- byte[] addChild2 = new BytesArray(
- "{" +
- " \"update\" : {" +
- " \"_index\" : \"test\"," +
- " \"_type\" : \"child\"," +
- " \"_id\" : \"child1\"," +
- " \"parent\" : \"parent1\"" +
- " }" +
- "}" +
- "\n" +
- "{" +
- " \"script\" : \"ctx._source.field2 = 'value2'\"," +
- " \"upsert\" : {" +
- " \"field1\" : \"value1'\"" +
- " }" +
- "}" +
- "\n").array();
-
- builder.add(addParent, 0, addParent.length, XContentType.JSON);
- builder.add(addChild1, 0, addChild1.length, XContentType.JSON);
- builder.add(addChild2, 0, addChild2.length, XContentType.JSON);
-
- BulkResponse bulkResponse = builder.get();
- assertThat(bulkResponse.getItems().length, equalTo(3));
- assertThat(bulkResponse.getItems()[0].isFailed(), equalTo(false));
- assertThat(bulkResponse.getItems()[1].isFailed(), equalTo(false));
- assertThat(bulkResponse.getItems()[2].isFailed(), equalTo(true));
- assertThat(bulkResponse.getItems()[2].getFailure().getCause().getCause().getMessage(),
- equalTo("script_lang not supported [painless]"));
-
- client().admin().indices().prepareRefresh("test").get();
-
- SearchResponse searchResponse = client().prepareSearch("test")
- .setQuery(QueryBuilders.hasParentQuery("parent", QueryBuilders.matchAllQuery(), false))
- .get();
-
- assertSearchHits(searchResponse, "child1");
- }
/*
* Test for https://github.com/elastic/elasticsearch/issues/8365
diff --git a/core/src/test/java/org/elasticsearch/aliases/IndexAliasesIT.java b/core/src/test/java/org/elasticsearch/aliases/IndexAliasesIT.java
index 711804153c..e711117fb6 100644
--- a/core/src/test/java/org/elasticsearch/aliases/IndexAliasesIT.java
+++ b/core/src/test/java/org/elasticsearch/aliases/IndexAliasesIT.java
@@ -63,8 +63,6 @@ import static org.elasticsearch.cluster.metadata.IndexMetaData.SETTING_BLOCKS_ME
import static org.elasticsearch.cluster.metadata.IndexMetaData.SETTING_BLOCKS_READ;
import static org.elasticsearch.cluster.metadata.IndexMetaData.SETTING_BLOCKS_WRITE;
import static org.elasticsearch.cluster.metadata.IndexMetaData.SETTING_READ_ONLY;
-import static org.elasticsearch.index.query.QueryBuilders.hasChildQuery;
-import static org.elasticsearch.index.query.QueryBuilders.hasParentQuery;
import static org.elasticsearch.index.query.QueryBuilders.matchAllQuery;
import static org.elasticsearch.index.query.QueryBuilders.rangeQuery;
import static org.elasticsearch.index.query.QueryBuilders.termQuery;
@@ -824,27 +822,6 @@ public class IndexAliasesIT extends ESIntegTestCase {
}
}
- public void testAliasesFilterWithHasChildQuery() throws Exception {
- assertAcked(prepareCreate("my-index")
- .setSettings("index.mapping.single_type", false)
- .addMapping("parent")
- .addMapping("child", "_parent", "type=parent")
- );
- client().prepareIndex("my-index", "parent", "1").setSource("{}", XContentType.JSON).get();
- client().prepareIndex("my-index", "child", "2").setSource("{}", XContentType.JSON).setParent("1").get();
- refresh();
-
- assertAcked(admin().indices().prepareAliases().addAlias("my-index", "filter1", hasChildQuery("child", matchAllQuery(), ScoreMode.None)));
- assertAcked(admin().indices().prepareAliases().addAlias("my-index", "filter2", hasParentQuery("parent", matchAllQuery(), false)));
-
- SearchResponse response = client().prepareSearch("filter1").get();
- assertHitCount(response, 1);
- assertThat(response.getHits().getAt(0).getId(), equalTo("1"));
- response = client().prepareSearch("filter2").get();
- assertHitCount(response, 1);
- assertThat(response.getHits().getAt(0).getId(), equalTo("2"));
- }
-
public void testAliasesWithBlocks() {
createIndex("test");
ensureGreen();
diff --git a/core/src/test/java/org/elasticsearch/bootstrap/BootstrapTests.java b/core/src/test/java/org/elasticsearch/bootstrap/BootstrapTests.java
deleted file mode 100644
index b2fab7746f..0000000000
--- a/core/src/test/java/org/elasticsearch/bootstrap/BootstrapTests.java
+++ /dev/null
@@ -1,33 +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.bootstrap;
-
-import org.elasticsearch.test.ESTestCase;
-
-public class BootstrapTests extends ESTestCase {
-
- public void testConfigDeprecation() {
- Bootstrap.checkConfigExtension(".json");
- assertWarnings("elasticsearch.json is deprecated; rename your configuration file to elasticsearch.yaml");
- Bootstrap.checkConfigExtension(".yml");
- assertWarnings("elasticsearch.yml is deprecated; rename your configuration file to elasticsearch.yaml");
- Bootstrap.checkConfigExtension(".yaml"); // no warnings, will be checked in @After
- }
-}
diff --git a/core/src/test/java/org/elasticsearch/client/transport/TransportClientNodesServiceTests.java b/core/src/test/java/org/elasticsearch/client/transport/TransportClientNodesServiceTests.java
index a6e2431a15..ed15660c85 100644
--- a/core/src/test/java/org/elasticsearch/client/transport/TransportClientNodesServiceTests.java
+++ b/core/src/test/java/org/elasticsearch/client/transport/TransportClientNodesServiceTests.java
@@ -19,21 +19,13 @@
package org.elasticsearch.client.transport;
-import java.io.Closeable;
-import java.util.ArrayList;
-import java.util.Collections;
-import java.util.HashMap;
-import java.util.List;
-import java.util.Map;
-import java.util.concurrent.CountDownLatch;
-import java.util.concurrent.TimeUnit;
-import java.util.concurrent.atomic.AtomicInteger;
-import java.util.concurrent.atomic.AtomicReference;
-
import org.elasticsearch.Version;
import org.elasticsearch.action.ActionListener;
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.ClusterStateRequest;
+import org.elasticsearch.action.admin.cluster.state.ClusterStateResponse;
import org.elasticsearch.cluster.ClusterName;
import org.elasticsearch.cluster.ClusterState;
import org.elasticsearch.cluster.node.DiscoveryNode;
@@ -41,19 +33,40 @@ import org.elasticsearch.cluster.node.DiscoveryNodes;
import org.elasticsearch.common.UUIDs;
import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.common.transport.TransportAddress;
+import org.elasticsearch.common.unit.TimeValue;
+import org.elasticsearch.node.Node;
import org.elasticsearch.test.ESTestCase;
+import org.elasticsearch.test.transport.MockTransportService;
import org.elasticsearch.threadpool.TestThreadPool;
import org.elasticsearch.threadpool.ThreadPool;
+import org.elasticsearch.transport.ConnectionProfile;
import org.elasticsearch.transport.Transport;
+import org.elasticsearch.transport.TransportChannel;
import org.elasticsearch.transport.TransportException;
import org.elasticsearch.transport.TransportInterceptor;
import org.elasticsearch.transport.TransportRequest;
+import org.elasticsearch.transport.TransportRequestHandler;
import org.elasticsearch.transport.TransportRequestOptions;
import org.elasticsearch.transport.TransportResponse;
import org.elasticsearch.transport.TransportResponseHandler;
import org.elasticsearch.transport.TransportService;
import org.hamcrest.CustomMatcher;
+import java.io.Closeable;
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.CopyOnWriteArrayList;
+import java.util.concurrent.CountDownLatch;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.concurrent.atomic.AtomicReference;
+
+import static org.elasticsearch.test.transport.MockTransportService.createNewService;
import static org.hamcrest.CoreMatchers.equalTo;
import static org.hamcrest.CoreMatchers.everyItem;
import static org.hamcrest.CoreMatchers.hasItem;
@@ -322,6 +335,157 @@ public class TransportClientNodesServiceTests extends ESTestCase {
}
}
+ public void testSniffNodesSamplerClosesConnections() throws Exception {
+ final TestThreadPool threadPool = new TestThreadPool("testSniffNodesSamplerClosesConnections");
+
+ Settings remoteSettings = Settings.builder().put(Node.NODE_NAME_SETTING.getKey(), "remote").build();
+ try (MockTransportService remoteService = createNewService(remoteSettings, Version.CURRENT, threadPool, null)) {
+ final MockHandler handler = new MockHandler(remoteService);
+ remoteService.registerRequestHandler(ClusterStateAction.NAME, ClusterStateRequest::new, ThreadPool.Names.SAME, handler);
+ remoteService.start();
+ remoteService.acceptIncomingRequests();
+
+ Settings clientSettings = Settings.builder()
+ .put(TransportClient.CLIENT_TRANSPORT_SNIFF.getKey(), true)
+ .put(TransportClient.CLIENT_TRANSPORT_PING_TIMEOUT.getKey(), TimeValue.timeValueSeconds(1))
+ .put(TransportClient.CLIENT_TRANSPORT_NODES_SAMPLER_INTERVAL.getKey(), TimeValue.timeValueSeconds(30))
+ .build();
+
+ try (MockTransportService clientService = createNewService(clientSettings, Version.CURRENT, threadPool, null)) {
+ final List<MockConnection> establishedConnections = new CopyOnWriteArrayList<>();
+ final List<MockConnection> reusedConnections = new CopyOnWriteArrayList<>();
+
+ clientService.addDelegate(remoteService, new MockTransportService.DelegateTransport(clientService.original()) {
+ @Override
+ public Connection openConnection(DiscoveryNode node, ConnectionProfile profile) throws IOException {
+ MockConnection connection = new MockConnection(super.openConnection(node, profile));
+ establishedConnections.add(connection);
+ return connection;
+ }
+
+ @Override
+ public Connection getConnection(DiscoveryNode node) {
+ MockConnection connection = new MockConnection(super.getConnection(node));
+ reusedConnections.add(connection);
+ return connection;
+ }
+ });
+
+ clientService.start();
+ clientService.acceptIncomingRequests();
+
+ try (TransportClientNodesService transportClientNodesService =
+ new TransportClientNodesService(clientSettings, clientService, threadPool, (a, b) -> {})) {
+ assertEquals(0, transportClientNodesService.connectedNodes().size());
+ assertEquals(0, establishedConnections.size());
+ assertEquals(0, reusedConnections.size());
+
+ transportClientNodesService.addTransportAddresses(remoteService.getLocalDiscoNode().getAddress());
+ assertEquals(1, transportClientNodesService.connectedNodes().size());
+ assertClosedConnections(establishedConnections, 1);
+
+ transportClientNodesService.doSample();
+ assertClosedConnections(establishedConnections, 2);
+ assertOpenConnections(reusedConnections, 1);
+
+ handler.blockRequest();
+ Thread thread = new Thread(transportClientNodesService::doSample);
+ thread.start();
+
+ assertBusy(() -> assertEquals(3, establishedConnections.size()));
+ assertFalse("Temporary ping connection must be opened", establishedConnections.get(2).isClosed());
+
+ handler.releaseRequest();
+ thread.join();
+
+ assertClosedConnections(establishedConnections, 3);
+ }
+ }
+ } finally {
+ terminate(threadPool);
+ }
+ }
+
+ private void assertClosedConnections(final List<MockConnection> connections, final int size) {
+ assertEquals("Expecting " + size + " closed connections but got " + connections.size(), size, connections.size());
+ connections.forEach(c -> assertConnection(c, true));
+ }
+
+ private void assertOpenConnections(final List<MockConnection> connections, final int size) {
+ assertEquals("Expecting " + size + " open connections but got " + connections.size(), size, connections.size());
+ connections.forEach(c -> assertConnection(c, false));
+ }
+
+ private static void assertConnection(final MockConnection connection, final boolean closed) {
+ assertEquals("Connection [" + connection + "] must be " + (closed ? "closed" : "open"), closed, connection.isClosed());
+ }
+
+ class MockConnection implements Transport.Connection {
+ private final AtomicBoolean closed = new AtomicBoolean(false);
+ private final Transport.Connection connection;
+
+ private MockConnection(Transport.Connection connection) {
+ this.connection = connection;
+ }
+
+ @Override
+ public DiscoveryNode getNode() {
+ return connection.getNode();
+ }
+
+ @Override
+ public Version getVersion() {
+ return connection.getVersion();
+ }
+
+ @Override
+ public void sendRequest(long requestId, String action, TransportRequest request, TransportRequestOptions options)
+ throws IOException, TransportException {
+ connection.sendRequest(requestId, action, request, options);
+ }
+
+ @Override
+ public void close() throws IOException {
+ if (closed.compareAndSet(false, true)) {
+ connection.close();
+ }
+ }
+
+ boolean isClosed() {
+ return closed.get();
+ }
+ }
+
+ class MockHandler implements TransportRequestHandler<ClusterStateRequest> {
+ private final AtomicBoolean block = new AtomicBoolean(false);
+ private final CountDownLatch release = new CountDownLatch(1);
+ private final MockTransportService transportService;
+
+ MockHandler(MockTransportService transportService) {
+ this.transportService = transportService;
+ }
+
+ @Override
+ public void messageReceived(ClusterStateRequest request, TransportChannel channel) throws Exception {
+ if (block.get()) {
+ release.await();
+ return;
+ }
+ DiscoveryNodes discoveryNodes = DiscoveryNodes.builder().add(transportService.getLocalDiscoNode()).build();
+ ClusterState build = ClusterState.builder(ClusterName.DEFAULT).nodes(discoveryNodes).build();
+ channel.sendResponse(new ClusterStateResponse(ClusterName.DEFAULT, build, 0L));
+ }
+
+ void blockRequest() {
+ if (block.compareAndSet(false, true) == false) {
+ throw new AssertionError("Request handler is already marked as blocking");
+ }
+ }
+ void releaseRequest() {
+ release.countDown();
+ }
+ }
+
public static class TestRequest extends TransportRequest {
}
diff --git a/core/src/test/java/org/elasticsearch/cluster/metadata/IndexNameExpressionResolverTests.java b/core/src/test/java/org/elasticsearch/cluster/metadata/IndexNameExpressionResolverTests.java
index 7d3ca04e5a..2143e5e67d 100644
--- a/core/src/test/java/org/elasticsearch/cluster/metadata/IndexNameExpressionResolverTests.java
+++ b/core/src/test/java/org/elasticsearch/cluster/metadata/IndexNameExpressionResolverTests.java
@@ -33,7 +33,6 @@ import org.elasticsearch.test.ESTestCase;
import java.util.Arrays;
import java.util.Collections;
import java.util.HashSet;
-import java.util.function.Predicate;
import static org.elasticsearch.common.util.set.Sets.newHashSet;
import static org.hamcrest.Matchers.arrayContaining;
@@ -621,7 +620,7 @@ public class IndexNameExpressionResolverTests extends ESTestCase {
assertThat(newHashSet(indexNameExpressionResolver.concreteIndexNames(context, "test*", "-testXXX")),
equalTo(newHashSet("testYYX", "testXYY", "testYYY", "testXXY")));
- assertThat(newHashSet(indexNameExpressionResolver.concreteIndexNames(context, "+testXXX", "+testXXY", "+testYYY", "-testYYY")),
+ assertThat(newHashSet(indexNameExpressionResolver.concreteIndexNames(context, "testXXX", "testXXY", "testYYY", "-testYYY")),
equalTo(newHashSet("testXXX", "testXXY", "testYYY", "-testYYY")));
assertThat(newHashSet(indexNameExpressionResolver.concreteIndexNames(context, "testYYY", "testYYX", "testX*", "-testXXX")),
@@ -637,7 +636,7 @@ public class IndexNameExpressionResolverTests extends ESTestCase {
equalTo(newHashSet("-testXYZ", "-testXZZ", "-testYYY")));
assertThat(newHashSet(indexNameExpressionResolver.concreteIndexNames(state, IndicesOptions.lenientExpandOpen(),
- "+testXXX", "+testXXY", "+testXYY", "-testXXY")),
+ "testXXX", "testXXY", "testXYY", "-testXXY")),
equalTo(newHashSet("testXXX", "testXYY", "testXXY")));
indexNames = indexNameExpressionResolver.concreteIndexNames(state, IndicesOptions.lenientExpandOpen(), "*", "-*");
@@ -817,7 +816,7 @@ public class IndexNameExpressionResolverTests extends ESTestCase {
}
public void testIsPatternMatchingAllIndicesMatchingSingleExclusion() throws Exception {
- String[] indicesOrAliases = new String[]{"-index1", "+index1"};
+ String[] indicesOrAliases = new String[]{"-index1", "index1"};
String[] concreteIndices = new String[]{"index1", "index2", "index3"};
MetaData metaData = metaDataBuilder(concreteIndices);
assertThat(indexNameExpressionResolver.isPatternMatchingAllIndices(metaData, indicesOrAliases, concreteIndices), equalTo(true));
@@ -832,7 +831,7 @@ public class IndexNameExpressionResolverTests extends ESTestCase {
}
public void testIsPatternMatchingAllIndicesMatchingTrailingWildcardAndExclusion() throws Exception {
- String[] indicesOrAliases = new String[]{"index*", "-index1", "+index1"};
+ String[] indicesOrAliases = new String[]{"index*", "-index1", "index1"};
String[] concreteIndices = new String[]{"index1", "index2", "index3"};
MetaData metaData = metaDataBuilder(concreteIndices);
assertThat(indexNameExpressionResolver.isPatternMatchingAllIndices(metaData, indicesOrAliases, concreteIndices), equalTo(true));
@@ -970,4 +969,22 @@ public class IndexNameExpressionResolverTests extends ESTestCase {
Arrays.sort(strings);
assertArrayEquals(new String[] {"test-alias-0", "test-alias-1", "test-alias-non-filtering"}, strings);
}
+
+ public void testConcreteIndicesForDeprecatedPattern() {
+ MetaData.Builder mdBuilder = MetaData.builder()
+ .put(indexBuilder("testXXX").state(State.OPEN))
+ .put(indexBuilder("testXXY").state(State.OPEN))
+ .put(indexBuilder("testYYY").state(State.OPEN));
+ ClusterState state = ClusterState.builder(new ClusterName("_name")).metaData(mdBuilder).build();
+
+ IndexNameExpressionResolver.Context context = new IndexNameExpressionResolver.Context(state,
+ IndicesOptions.fromOptions(true, true, true, true));
+ assertThat(newHashSet(indexNameExpressionResolver.concreteIndexNames(context, "+testX*")),
+ equalTo(newHashSet("testXXX", "testXXY")));
+ assertThat(newHashSet(indexNameExpressionResolver.concreteIndexNames(context, "+testXXX", "+testXXY", "+testYYY", "-testYYY")),
+ equalTo(newHashSet("testXXX", "testXXY", "testYYY")));
+ assertThat(newHashSet(indexNameExpressionResolver.concreteIndexNames(context, "+testXX*", "+testY*")),
+ equalTo(newHashSet("testXXX", "testXXY", "testYYY")));
+ assertWarnings("support for '+' as part of index expressions is deprecated");
+ }
}
diff --git a/core/src/test/java/org/elasticsearch/cluster/metadata/WildcardExpressionResolverTests.java b/core/src/test/java/org/elasticsearch/cluster/metadata/WildcardExpressionResolverTests.java
index bac9a68134..2778525f7d 100644
--- a/core/src/test/java/org/elasticsearch/cluster/metadata/WildcardExpressionResolverTests.java
+++ b/core/src/test/java/org/elasticsearch/cluster/metadata/WildcardExpressionResolverTests.java
@@ -49,9 +49,9 @@ public class WildcardExpressionResolverTests extends ESTestCase {
assertThat(newHashSet(resolver.resolve(context, Arrays.asList("testX*", "kuku"))), equalTo(newHashSet("testXXX", "testXYY", "kuku")));
assertThat(newHashSet(resolver.resolve(context, Arrays.asList("*"))), equalTo(newHashSet("testXXX", "testXYY", "testYYY", "kuku")));
assertThat(newHashSet(resolver.resolve(context, Arrays.asList("*", "-kuku"))), equalTo(newHashSet("testXXX", "testXYY", "testYYY")));
- assertThat(newHashSet(resolver.resolve(context, Arrays.asList("testXXX", "+testYYY"))), equalTo(newHashSet("testXXX", "testYYY")));
+ assertThat(newHashSet(resolver.resolve(context, Arrays.asList("testXXX", "testYYY"))), equalTo(newHashSet("testXXX", "testYYY")));
assertThat(newHashSet(resolver.resolve(context, Arrays.asList("testXXX", "-testXXX"))), equalTo(newHashSet("testXXX", "-testXXX")));
- assertThat(newHashSet(resolver.resolve(context, Arrays.asList("testXXX", "+testY*"))), equalTo(newHashSet("testXXX", "testYYY")));
+ assertThat(newHashSet(resolver.resolve(context, Arrays.asList("testXXX", "testY*"))), equalTo(newHashSet("testXXX", "testYYY")));
assertThat(newHashSet(resolver.resolve(context, Arrays.asList("testXXX", "-testX*"))), equalTo(newHashSet("testXXX")));
}
@@ -67,9 +67,9 @@ public class WildcardExpressionResolverTests extends ESTestCase {
IndexNameExpressionResolver.Context context = new IndexNameExpressionResolver.Context(state, IndicesOptions.lenientExpandOpen());
assertThat(newHashSet(resolver.resolve(context, Arrays.asList("testYY*", "alias*"))), equalTo(newHashSet("testXXX", "testXYY", "testYYY")));
assertThat(newHashSet(resolver.resolve(context, Arrays.asList("-kuku"))), equalTo(newHashSet("-kuku")));
- assertThat(newHashSet(resolver.resolve(context, Arrays.asList("+test*", "-testYYY"))), equalTo(newHashSet("testXXX", "testXYY")));
- assertThat(newHashSet(resolver.resolve(context, Arrays.asList("+testX*", "+testYYY"))), equalTo(newHashSet("testXXX", "testXYY", "testYYY")));
- assertThat(newHashSet(resolver.resolve(context, Arrays.asList("+testYYY", "+testX*"))), equalTo(newHashSet("testXXX", "testXYY", "testYYY")));
+ assertThat(newHashSet(resolver.resolve(context, Arrays.asList("test*", "-testYYY"))), equalTo(newHashSet("testXXX", "testXYY")));
+ assertThat(newHashSet(resolver.resolve(context, Arrays.asList("testX*", "testYYY"))), equalTo(newHashSet("testXXX", "testXYY", "testYYY")));
+ assertThat(newHashSet(resolver.resolve(context, Arrays.asList("testYYY", "testX*"))), equalTo(newHashSet("testXXX", "testXYY", "testYYY")));
}
public void testConvertWildcardsOpenClosedIndicesTests() {
@@ -129,4 +129,19 @@ public class WildcardExpressionResolverTests extends ESTestCase {
return IndexMetaData.builder(index).settings(settings(Version.CURRENT).put(IndexMetaData.SETTING_NUMBER_OF_SHARDS, 1).put(IndexMetaData.SETTING_NUMBER_OF_REPLICAS, 0));
}
+ public void testForDeprecatedPlusPattern() {
+ MetaData.Builder mdBuilder = MetaData.builder()
+ .put(indexBuilder("testXXX").state(IndexMetaData.State.OPEN))
+ .put(indexBuilder("testXYY").state(IndexMetaData.State.OPEN))
+ .put(indexBuilder("testYYY").state(IndexMetaData.State.OPEN));
+ ClusterState state = ClusterState.builder(new ClusterName("_name")).metaData(mdBuilder).build();
+ IndexNameExpressionResolver.WildcardExpressionResolver resolver = new IndexNameExpressionResolver.WildcardExpressionResolver();
+
+ IndexNameExpressionResolver.Context context = new IndexNameExpressionResolver.Context(state, IndicesOptions.fromOptions(true, true, true, true));
+ assertThat(newHashSet(resolver.resolve(context, Arrays.asList("+testX*", "-testYYY"))), equalTo(newHashSet("testXXX", "testXYY")));
+ assertThat(newHashSet(resolver.resolve(context, Arrays.asList("+testYYY", "+testXY*"))), equalTo(newHashSet("testYYY", "testXYY")));
+ assertThat(newHashSet(resolver.resolve(context, Arrays.asList("testYYY", "+testXX*"))), equalTo(newHashSet("testXXX", "testYYY")));
+ assertWarnings("support for '+' as part of index expressions is deprecated");
+ }
+
}
diff --git a/core/src/test/java/org/elasticsearch/discovery/single/SingleNodeDiscoveryIT.java b/core/src/test/java/org/elasticsearch/discovery/single/SingleNodeDiscoveryIT.java
index d2a520c328..d1b86aeaa1 100644
--- a/core/src/test/java/org/elasticsearch/discovery/single/SingleNodeDiscoveryIT.java
+++ b/core/src/test/java/org/elasticsearch/discovery/single/SingleNodeDiscoveryIT.java
@@ -85,14 +85,6 @@ public class SingleNodeDiscoveryIT extends ESIntegTestCase {
final UnicastHostsProvider provider =
() -> Collections.singletonList(nodeTransport.getLocalNode());
final CountDownLatch latch = new CountDownLatch(1);
- final UnicastZenPing unicastZenPing =
- new UnicastZenPing(settings, threadPool, pingTransport, provider) {
- @Override
- protected void finishPingingRound(PingingRound pingingRound) {
- latch.countDown();
- super.finishPingingRound(pingingRound);
- }
- };
final DiscoveryNodes nodes = DiscoveryNodes.builder()
.add(nodeTransport.getLocalNode())
.add(pingTransport.getLocalNode())
@@ -100,7 +92,15 @@ public class SingleNodeDiscoveryIT extends ESIntegTestCase {
.build();
final ClusterName clusterName = new ClusterName(internalCluster().getClusterName());
final ClusterState state = ClusterState.builder(clusterName).nodes(nodes).build();
- unicastZenPing.start(() -> state);
+ final UnicastZenPing unicastZenPing =
+ new UnicastZenPing(settings, threadPool, pingTransport, provider, () -> state) {
+ @Override
+ protected void finishPingingRound(PingingRound pingingRound) {
+ latch.countDown();
+ super.finishPingingRound(pingingRound);
+ }
+ };
+ unicastZenPing.start();
closeables.push(unicastZenPing);
final CompletableFuture<ZenPing.PingCollection> responses = new CompletableFuture<>();
unicastZenPing.ping(responses::complete, TimeValue.timeValueSeconds(3));
diff --git a/core/src/test/java/org/elasticsearch/discovery/single/SingleNodeDiscoveryTests.java b/core/src/test/java/org/elasticsearch/discovery/single/SingleNodeDiscoveryTests.java
index 917894b60d..135692fd64 100644
--- a/core/src/test/java/org/elasticsearch/discovery/single/SingleNodeDiscoveryTests.java
+++ b/core/src/test/java/org/elasticsearch/discovery/single/SingleNodeDiscoveryTests.java
@@ -21,9 +21,12 @@ package org.elasticsearch.discovery.single;
import org.apache.lucene.util.IOUtils;
import org.elasticsearch.Version;
+import org.elasticsearch.cluster.ClusterState;
+import org.elasticsearch.cluster.ClusterStateTaskListener;
import org.elasticsearch.cluster.node.DiscoveryNode;
import org.elasticsearch.cluster.node.DiscoveryNodes;
-import org.elasticsearch.cluster.service.ClusterService;
+import org.elasticsearch.cluster.service.ClusterApplier;
+import org.elasticsearch.cluster.service.MasterService;
import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.test.ESTestCase;
import org.elasticsearch.test.transport.MockTransportService;
@@ -32,8 +35,10 @@ import org.elasticsearch.threadpool.ThreadPool;
import java.io.Closeable;
import java.util.Stack;
+import java.util.concurrent.atomic.AtomicReference;
+import java.util.function.Supplier;
-import static org.elasticsearch.test.ClusterServiceUtils.createClusterService;
+import static org.elasticsearch.test.ClusterServiceUtils.createMasterService;
import static org.hamcrest.Matchers.equalTo;
public class SingleNodeDiscoveryTests extends ESTestCase {
@@ -49,13 +54,26 @@ public class SingleNodeDiscoveryTests extends ESTestCase {
stack.push(transportService);
transportService.start();
final DiscoveryNode node = transportService.getLocalNode();
- final ClusterService clusterService = createClusterService(threadPool, node);
- stack.push(clusterService);
+ final MasterService masterService = createMasterService(threadPool, node);
+ AtomicReference<ClusterState> clusterState = new AtomicReference<>();
final SingleNodeDiscovery discovery =
new SingleNodeDiscovery(Settings.EMPTY, transportService,
- clusterService.getClusterApplierService());
+ masterService, new ClusterApplier() {
+ @Override
+ public void setInitialState(ClusterState initialState) {
+ clusterState.set(initialState);
+ }
+
+ @Override
+ public void onNewClusterState(String source, Supplier<ClusterState> clusterStateSupplier,
+ ClusterStateTaskListener listener) {
+ clusterState.set(clusterStateSupplier.get());
+ listener.clusterStateProcessed(source, clusterState.get(), clusterState.get());
+ }
+ });
+ discovery.start();
discovery.startInitialJoin();
- final DiscoveryNodes nodes = discovery.getInitialClusterState().nodes();
+ final DiscoveryNodes nodes = clusterState.get().nodes();
assertThat(nodes.getSize(), equalTo(1));
assertThat(nodes.getMasterNode().getId(), equalTo(node.getId()));
} finally {
diff --git a/core/src/test/java/org/elasticsearch/discovery/zen/UnicastZenPingTests.java b/core/src/test/java/org/elasticsearch/discovery/zen/UnicastZenPingTests.java
index fa53f94f42..6aa47d27bb 100644
--- a/core/src/test/java/org/elasticsearch/discovery/zen/UnicastZenPingTests.java
+++ b/core/src/test/java/org/elasticsearch/discovery/zen/UnicastZenPingTests.java
@@ -188,40 +188,40 @@ public class UnicastZenPingTests extends ESTestCase {
.build();
Settings hostsSettingsMismatch = Settings.builder().put(hostsSettings).put(settingsMismatch).build();
- TestUnicastZenPing zenPingA = new TestUnicastZenPing(hostsSettings, threadPool, handleA, EMPTY_HOSTS_PROVIDER);
ClusterState stateA = ClusterState.builder(state)
.blocks(ClusterBlocks.builder().addGlobalBlock(STATE_NOT_RECOVERED_BLOCK))
.nodes(DiscoveryNodes.builder().add(handleA.node).localNodeId("UZP_A"))
.build();
- zenPingA.start(() -> stateA);
+ TestUnicastZenPing zenPingA = new TestUnicastZenPing(hostsSettings, threadPool, handleA, EMPTY_HOSTS_PROVIDER, () -> stateA);
+ zenPingA.start();
closeables.push(zenPingA);
- TestUnicastZenPing zenPingB = new TestUnicastZenPing(hostsSettings, threadPool, handleB, EMPTY_HOSTS_PROVIDER);
ClusterState stateB = ClusterState.builder(state)
.nodes(DiscoveryNodes.builder().add(handleB.node).localNodeId("UZP_B"))
.build();
- zenPingB.start(() -> stateB);
+ TestUnicastZenPing zenPingB = new TestUnicastZenPing(hostsSettings, threadPool, handleB, EMPTY_HOSTS_PROVIDER, () -> stateB);
+ zenPingB.start();
closeables.push(zenPingB);
+ ClusterState stateC = ClusterState.builder(stateMismatch)
+ .nodes(DiscoveryNodes.builder().add(handleC.node).localNodeId("UZP_C"))
+ .build();
TestUnicastZenPing zenPingC = new TestUnicastZenPing(hostsSettingsMismatch, threadPool, handleC,
- EMPTY_HOSTS_PROVIDER) {
+ EMPTY_HOSTS_PROVIDER, () -> stateC) {
@Override
protected Version getVersion() {
return versionD;
}
};
- ClusterState stateC = ClusterState.builder(stateMismatch)
- .nodes(DiscoveryNodes.builder().add(handleC.node).localNodeId("UZP_C"))
- .build();
- zenPingC.start(() -> stateC);
+ zenPingC.start();
closeables.push(zenPingC);
- TestUnicastZenPing zenPingD = new TestUnicastZenPing(hostsSettingsMismatch, threadPool, handleD,
- EMPTY_HOSTS_PROVIDER);
ClusterState stateD = ClusterState.builder(stateMismatch)
.nodes(DiscoveryNodes.builder().add(handleD.node).localNodeId("UZP_D"))
.build();
- zenPingD.start(() -> stateD);
+ TestUnicastZenPing zenPingD = new TestUnicastZenPing(hostsSettingsMismatch, threadPool, handleD,
+ EMPTY_HOSTS_PROVIDER, () -> stateD);
+ zenPingD.start();
closeables.push(zenPingD);
logger.info("ping from UZP_A");
@@ -311,26 +311,26 @@ public class UnicastZenPingTests extends ESTestCase {
final ClusterState state = ClusterState.builder(new ClusterName("test")).version(randomNonNegativeLong()).build();
- final TestUnicastZenPing zenPingA = new TestUnicastZenPing(hostsSettings, threadPool, handleA, EMPTY_HOSTS_PROVIDER);
ClusterState stateA = ClusterState.builder(state)
.blocks(ClusterBlocks.builder().addGlobalBlock(STATE_NOT_RECOVERED_BLOCK))
.nodes(DiscoveryNodes.builder().add(handleA.node).localNodeId("UZP_A"))
.build();
- zenPingA.start(() -> stateA);
+ final TestUnicastZenPing zenPingA = new TestUnicastZenPing(hostsSettings, threadPool, handleA, EMPTY_HOSTS_PROVIDER, () -> stateA);
+ zenPingA.start();
closeables.push(zenPingA);
- TestUnicastZenPing zenPingB = new TestUnicastZenPing(hostsSettings, threadPool, handleB, EMPTY_HOSTS_PROVIDER);
ClusterState stateB = ClusterState.builder(state)
.nodes(DiscoveryNodes.builder().add(handleB.node).localNodeId("UZP_B"))
.build();
- zenPingB.start(() -> stateB);
+ TestUnicastZenPing zenPingB = new TestUnicastZenPing(hostsSettings, threadPool, handleB, EMPTY_HOSTS_PROVIDER, () -> stateB);
+ zenPingB.start();
closeables.push(zenPingB);
- TestUnicastZenPing zenPingC = new TestUnicastZenPing(hostsSettings, threadPool, handleC, EMPTY_HOSTS_PROVIDER);
ClusterState stateC = ClusterState.builder(state)
.nodes(DiscoveryNodes.builder().add(handleC.node).localNodeId("UZP_C"))
.build();
- zenPingC.start(() -> stateC);
+ TestUnicastZenPing zenPingC = new TestUnicastZenPing(hostsSettings, threadPool, handleC, EMPTY_HOSTS_PROVIDER, () -> stateC);
+ zenPingC.start();
closeables.push(zenPingC);
// the presence of an unresolvable host should not prevent resolvable hosts from being pinged
@@ -604,24 +604,24 @@ public class UnicastZenPingTests extends ESTestCase {
// install a listener to check that no new connections are made
handleA.transportService.addConnectionListener(new TransportConnectionListener() {
@Override
- public void onConnectionOpened(DiscoveryNode node) {
- fail("should not open any connections. got [" + node + "]");
+ public void onConnectionOpened(Transport.Connection connection) {
+ fail("should not open any connections. got [" + connection.getNode() + "]");
}
});
- final TestUnicastZenPing zenPingA = new TestUnicastZenPing(hostsSettings, threadPool, handleA, EMPTY_HOSTS_PROVIDER);
final ClusterState stateA = ClusterState.builder(state)
.blocks(ClusterBlocks.builder().addGlobalBlock(STATE_NOT_RECOVERED_BLOCK))
.nodes(DiscoveryNodes.builder().add(handleA.node).add(handleB.node).localNodeId("UZP_A"))
.build();
- zenPingA.start(() -> stateA);
+ final TestUnicastZenPing zenPingA = new TestUnicastZenPing(hostsSettings, threadPool, handleA, EMPTY_HOSTS_PROVIDER, () -> stateA);
+ zenPingA.start();
closeables.push(zenPingA);
- TestUnicastZenPing zenPingB = new TestUnicastZenPing(hostsSettings, threadPool, handleB, EMPTY_HOSTS_PROVIDER);
final ClusterState stateB = ClusterState.builder(state)
.nodes(DiscoveryNodes.builder().add(handleB.node).localNodeId("UZP_B"))
.build();
- zenPingB.start(() -> stateB);
+ TestUnicastZenPing zenPingB = new TestUnicastZenPing(hostsSettings, threadPool, handleB, EMPTY_HOSTS_PROVIDER, () -> stateB);
+ zenPingB.start();
closeables.push(zenPingB);
Collection<ZenPing.PingResponse> pingResponses = zenPingA.pingAndWait().toList();
@@ -660,15 +660,15 @@ public class UnicastZenPingTests extends ESTestCase {
.blocks(ClusterBlocks.builder().addGlobalBlock(STATE_NOT_RECOVERED_BLOCK))
.nodes(DiscoveryNodes.builder().add(handleA.node).add(handleB.node).localNodeId("UZP_A")).build();
- final TestUnicastZenPing zenPingA = new TestUnicastZenPing(hostsSettings, threadPool, handleA, EMPTY_HOSTS_PROVIDER);
- zenPingA.start(() -> stateA);
+ final TestUnicastZenPing zenPingA = new TestUnicastZenPing(hostsSettings, threadPool, handleA, EMPTY_HOSTS_PROVIDER, () -> stateA);
+ zenPingA.start();
closeables.push(zenPingA);
// Node B doesn't know about A!
final ClusterState stateB = ClusterState.builder(state).nodes(
DiscoveryNodes.builder().add(handleB.node).localNodeId("UZP_B")).build();
- TestUnicastZenPing zenPingB = new TestUnicastZenPing(hostsSettings, threadPool, handleB, EMPTY_HOSTS_PROVIDER);
- zenPingB.start(() -> stateB);
+ TestUnicastZenPing zenPingB = new TestUnicastZenPing(hostsSettings, threadPool, handleB, EMPTY_HOSTS_PROVIDER, () -> stateB);
+ zenPingB.start();
closeables.push(zenPingB);
{
@@ -796,9 +796,9 @@ public class UnicastZenPingTests extends ESTestCase {
private static class TestUnicastZenPing extends UnicastZenPing {
TestUnicastZenPing(Settings settings, ThreadPool threadPool, NetworkHandle networkHandle,
- UnicastHostsProvider unicastHostsProvider) {
+ UnicastHostsProvider unicastHostsProvider, PingContextProvider contextProvider) {
super(Settings.builder().put("node.name", networkHandle.node.getName()).put(settings).build(),
- threadPool, networkHandle.transportService, unicastHostsProvider);
+ threadPool, networkHandle.transportService, unicastHostsProvider, contextProvider);
}
volatile CountDownLatch allTasksCompleted;
diff --git a/core/src/test/java/org/elasticsearch/discovery/zen/ZenDiscoveryUnitTests.java b/core/src/test/java/org/elasticsearch/discovery/zen/ZenDiscoveryUnitTests.java
index d0c138954a..b961b6d6fb 100644
--- a/core/src/test/java/org/elasticsearch/discovery/zen/ZenDiscoveryUnitTests.java
+++ b/core/src/test/java/org/elasticsearch/discovery/zen/ZenDiscoveryUnitTests.java
@@ -26,6 +26,8 @@ import org.elasticsearch.cluster.ClusterChangedEvent;
import org.elasticsearch.cluster.ClusterModule;
import org.elasticsearch.cluster.ClusterName;
import org.elasticsearch.cluster.ClusterState;
+import org.elasticsearch.cluster.ClusterStateTaskListener;
+import org.elasticsearch.cluster.ESAllocationTestCase;
import org.elasticsearch.cluster.metadata.IndexMetaData;
import org.elasticsearch.cluster.metadata.MetaData;
import org.elasticsearch.cluster.node.DiscoveryNode;
@@ -37,6 +39,7 @@ import org.elasticsearch.cluster.routing.RoutingTable;
import org.elasticsearch.cluster.routing.ShardRoutingState;
import org.elasticsearch.cluster.routing.TestShardRouting;
import org.elasticsearch.cluster.routing.UnassignedInfo;
+import org.elasticsearch.cluster.service.ClusterApplier;
import org.elasticsearch.cluster.service.MasterService;
import org.elasticsearch.common.io.stream.NamedWriteableRegistry;
import org.elasticsearch.common.settings.ClusterSettings;
@@ -67,7 +70,7 @@ import java.util.List;
import java.util.Set;
import java.util.concurrent.TimeUnit;
import java.util.concurrent.atomic.AtomicBoolean;
-import java.util.concurrent.atomic.AtomicReference;
+import java.util.function.Supplier;
import java.util.stream.Collectors;
import static java.util.Collections.emptyMap;
@@ -195,7 +198,7 @@ public class ZenDiscoveryUnitTests extends ESTestCase {
ZenDiscovery masterZen = buildZenDiscovery(
settingsWithClusterName,
masterTransport, masterMasterService, threadPool);
- masterZen.setState(state);
+ masterZen.setCommittedState(state);
toClose.addFirst(masterZen);
masterTransport.acceptIncomingRequests();
@@ -209,7 +212,7 @@ public class ZenDiscoveryUnitTests extends ESTestCase {
MasterService otherMasterService = ClusterServiceUtils.createMasterService(threadPool, otherNode);
toClose.addFirst(otherMasterService);
ZenDiscovery otherZen = buildZenDiscovery(settingsWithClusterName, otherTransport, otherMasterService, threadPool);
- otherZen.setState(otherState);
+ otherZen.setCommittedState(otherState);
toClose.addFirst(otherZen);
otherTransport.acceptIncomingRequests();
@@ -262,7 +265,7 @@ public class ZenDiscoveryUnitTests extends ESTestCase {
toClose.addFirst(masterMasterService);
state = ClusterState.builder(discoveryState(masterMasterService).getClusterName()).nodes(state.nodes()).build();
ZenDiscovery masterZen = buildZenDiscovery(settings, masterTransport, masterMasterService, threadPool);
- masterZen.setState(state);
+ masterZen.setCommittedState(state);
toClose.addFirst(masterZen);
masterTransport.acceptIncomingRequests();
@@ -297,9 +300,19 @@ public class ZenDiscoveryUnitTests extends ESTestCase {
private ZenDiscovery buildZenDiscovery(Settings settings, TransportService service, MasterService masterService,
ThreadPool threadPool) {
ClusterSettings clusterSettings = new ClusterSettings(Settings.EMPTY, ClusterSettings.BUILT_IN_CLUSTER_SETTINGS);
+ ClusterApplier clusterApplier = new ClusterApplier() {
+ @Override
+ public void setInitialState(ClusterState initialState) {
+
+ }
+
+ @Override
+ public void onNewClusterState(String source, Supplier<ClusterState> clusterStateSupplier, ClusterStateTaskListener listener) {
+ listener.clusterStateProcessed(source, clusterStateSupplier.get(), clusterStateSupplier.get());
+ }
+ };
ZenDiscovery zenDiscovery = new ZenDiscovery(settings, threadPool, service, new NamedWriteableRegistry(ClusterModule.getNamedWriteables()),
- masterService, (source, clusterStateSupplier, listener) -> listener.clusterStateProcessed(source, clusterStateSupplier.get(), clusterStateSupplier.get()),
- clusterSettings, Collections::emptyList, null);
+ masterService, clusterApplier, clusterSettings, Collections::emptyList, ESAllocationTestCase.createAllocationService());
zenDiscovery.start();
return zenDiscovery;
}
diff --git a/core/src/test/java/org/elasticsearch/gateway/GatewayServiceTests.java b/core/src/test/java/org/elasticsearch/gateway/GatewayServiceTests.java
index 2bec3d5ede..3e4a3dce09 100644
--- a/core/src/test/java/org/elasticsearch/gateway/GatewayServiceTests.java
+++ b/core/src/test/java/org/elasticsearch/gateway/GatewayServiceTests.java
@@ -24,7 +24,6 @@ import org.elasticsearch.common.settings.ClusterSettings;
import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.common.unit.TimeValue;
import org.elasticsearch.test.ESTestCase;
-import org.elasticsearch.test.NoopDiscovery;
import org.hamcrest.Matchers;
import java.io.IOException;
@@ -36,7 +35,7 @@ public class GatewayServiceTests extends ESTestCase {
new ClusterSettings(Settings.EMPTY, ClusterSettings.BUILT_IN_CLUSTER_SETTINGS),
null);
return new GatewayService(settings.build(),
- null, clusterService, null, null, null, new NoopDiscovery(), null);
+ null, clusterService, null, null, null, null);
}
public void testDefaultRecoverAfterTime() throws IOException {
diff --git a/core/src/test/java/org/elasticsearch/index/query/HasChildQueryBuilderTests.java b/core/src/test/java/org/elasticsearch/index/query/HasChildQueryBuilderTests.java
deleted file mode 100644
index 49523fe923..0000000000
--- a/core/src/test/java/org/elasticsearch/index/query/HasChildQueryBuilderTests.java
+++ /dev/null
@@ -1,349 +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 com.carrotsearch.randomizedtesting.generators.RandomPicks;
-import org.apache.lucene.search.TermInSetQuery;
-import org.apache.lucene.search.BooleanClause;
-import org.apache.lucene.search.BooleanQuery;
-import org.apache.lucene.search.ConstantScoreQuery;
-import org.apache.lucene.search.MatchNoDocsQuery;
-import org.apache.lucene.search.Query;
-import org.apache.lucene.search.TermQuery;
-import org.apache.lucene.search.join.ScoreMode;
-import org.apache.lucene.search.similarities.PerFieldSimilarityWrapper;
-import org.apache.lucene.search.similarities.Similarity;
-import org.apache.lucene.util.BytesRef;
-import org.elasticsearch.Version;
-import org.elasticsearch.action.admin.indices.mapping.put.PutMappingRequest;
-import org.elasticsearch.cluster.metadata.IndexMetaData;
-import org.elasticsearch.common.compress.CompressedXContent;
-import org.elasticsearch.common.settings.Settings;
-import org.elasticsearch.index.mapper.MapperService;
-import org.elasticsearch.index.mapper.TypeFieldMapper;
-import org.elasticsearch.index.mapper.Uid;
-import org.elasticsearch.index.mapper.UidFieldMapper;
-import org.elasticsearch.index.similarity.SimilarityService;
-import org.elasticsearch.search.fetch.subphase.InnerHitsContext;
-import org.elasticsearch.search.internal.SearchContext;
-import org.elasticsearch.search.sort.FieldSortBuilder;
-import org.elasticsearch.search.sort.SortOrder;
-import org.elasticsearch.test.AbstractQueryTestCase;
-
-import java.io.IOException;
-import java.util.Collections;
-import java.util.HashMap;
-import java.util.Map;
-
-import static org.hamcrest.CoreMatchers.containsString;
-import static org.hamcrest.CoreMatchers.equalTo;
-import static org.hamcrest.CoreMatchers.instanceOf;
-import static org.hamcrest.CoreMatchers.notNullValue;
-
-public class HasChildQueryBuilderTests extends AbstractQueryTestCase<HasChildQueryBuilder> {
- protected static final String PARENT_TYPE = "parent";
- protected static final String CHILD_TYPE = "child";
-
- private static String similarity;
-
- boolean requiresRewrite = false;
-
- @Override
- protected void initializeAdditionalMappings(MapperService mapperService) throws IOException {
- similarity = randomFrom("classic", "BM25");
- mapperService.merge(PARENT_TYPE, new CompressedXContent(PutMappingRequest.buildFromSimplifiedDef(PARENT_TYPE,
- STRING_FIELD_NAME, "type=text",
- STRING_FIELD_NAME_2, "type=keyword",
- INT_FIELD_NAME, "type=integer",
- DOUBLE_FIELD_NAME, "type=double",
- BOOLEAN_FIELD_NAME, "type=boolean",
- DATE_FIELD_NAME, "type=date",
- OBJECT_FIELD_NAME, "type=object"
- ).string()), MapperService.MergeReason.MAPPING_UPDATE, false);
- mapperService.merge(CHILD_TYPE, new CompressedXContent(PutMappingRequest.buildFromSimplifiedDef(CHILD_TYPE,
- "_parent", "type=" + PARENT_TYPE,
- STRING_FIELD_NAME, "type=text",
- "custom_string", "type=text,similarity=" + similarity,
- INT_FIELD_NAME, "type=integer",
- DOUBLE_FIELD_NAME, "type=double",
- BOOLEAN_FIELD_NAME, "type=boolean",
- DATE_FIELD_NAME, "type=date",
- OBJECT_FIELD_NAME, "type=object"
- ).string()), MapperService.MergeReason.MAPPING_UPDATE, false);
- }
-
- /**
- * @return a {@link HasChildQueryBuilder} with random values all over the place
- */
- @Override
- protected HasChildQueryBuilder doCreateTestQueryBuilder() {
- int min = randomIntBetween(0, Integer.MAX_VALUE / 2);
- int max = randomIntBetween(min, Integer.MAX_VALUE);
-
- QueryBuilder innerQueryBuilder = RandomQueryBuilder.createQuery(random());
- if (randomBoolean()) {
- requiresRewrite = true;
- innerQueryBuilder = new WrapperQueryBuilder(innerQueryBuilder.toString());
- }
-
- HasChildQueryBuilder hqb = new HasChildQueryBuilder(CHILD_TYPE, innerQueryBuilder,
- RandomPicks.randomFrom(random(), ScoreMode.values()));
- hqb.minMaxChildren(min, max);
- hqb.ignoreUnmapped(randomBoolean());
- if (randomBoolean()) {
- hqb.innerHit(new InnerHitBuilder()
- .setName(randomAlphaOfLengthBetween(1, 10))
- .setSize(randomIntBetween(0, 100))
- .addSort(new FieldSortBuilder(STRING_FIELD_NAME_2).order(SortOrder.ASC)), hqb.ignoreUnmapped());
- }
- return hqb;
- }
-
- @Override
- protected void doAssertLuceneQuery(HasChildQueryBuilder queryBuilder, Query query, SearchContext searchContext) throws IOException {
- assertThat(query, instanceOf(HasChildQueryBuilder.LateParsingQuery.class));
- HasChildQueryBuilder.LateParsingQuery lpq = (HasChildQueryBuilder.LateParsingQuery) query;
- assertEquals(queryBuilder.minChildren(), lpq.getMinChildren());
- assertEquals(queryBuilder.maxChildren(), lpq.getMaxChildren());
- assertEquals(queryBuilder.scoreMode(), lpq.getScoreMode()); // WTF is this why do we have two?
- if (queryBuilder.innerHit() != null) {
- // have to rewrite again because the provided queryBuilder hasn't been rewritten (directly returned from
- // doCreateTestQueryBuilder)
- queryBuilder = (HasChildQueryBuilder) queryBuilder.rewrite(searchContext.getQueryShardContext());
- Map<String, InnerHitBuilder> innerHitBuilders = new HashMap<>();
- InnerHitBuilder.extractInnerHits(queryBuilder, innerHitBuilders);
- for (InnerHitBuilder builder : innerHitBuilders.values()) {
- builder.build(searchContext, searchContext.innerHits());
- }
- assertNotNull(searchContext.innerHits());
- assertEquals(1, searchContext.innerHits().getInnerHits().size());
- assertTrue(searchContext.innerHits().getInnerHits().containsKey(queryBuilder.innerHit().getName()));
- InnerHitsContext.BaseInnerHits innerHits =
- searchContext.innerHits().getInnerHits().get(queryBuilder.innerHit().getName());
- assertEquals(innerHits.size(), queryBuilder.innerHit().getSize());
- assertEquals(innerHits.sort().sort.getSort().length, 1);
- assertEquals(innerHits.sort().sort.getSort()[0].getField(), STRING_FIELD_NAME_2);
- }
- }
-
- public void testIllegalValues() {
- QueryBuilder query = RandomQueryBuilder.createQuery(random());
- IllegalArgumentException e = expectThrows(IllegalArgumentException.class,
- () -> QueryBuilders.hasChildQuery(null, query, ScoreMode.None));
- assertEquals("[has_child] requires 'type' field", e.getMessage());
-
- e = expectThrows(IllegalArgumentException.class, () -> QueryBuilders.hasChildQuery("foo", null, ScoreMode.None));
- assertEquals("[has_child] requires 'query' field", e.getMessage());
-
- e = expectThrows(IllegalArgumentException.class, () -> QueryBuilders.hasChildQuery("foo", query, null));
- assertEquals("[has_child] requires 'score_mode' field", e.getMessage());
-
- int positiveValue = randomIntBetween(0, Integer.MAX_VALUE);
- HasChildQueryBuilder foo = QueryBuilders.hasChildQuery("foo", query, ScoreMode.None); // all good
- e = expectThrows(IllegalArgumentException.class, () -> foo.minMaxChildren(randomIntBetween(Integer.MIN_VALUE, -1), positiveValue));
- assertEquals("[has_child] requires non-negative 'min_children' field", e.getMessage());
-
- e = expectThrows(IllegalArgumentException.class, () -> foo.minMaxChildren(positiveValue, randomIntBetween(Integer.MIN_VALUE, -1)));
- assertEquals("[has_child] requires non-negative 'max_children' field", e.getMessage());
-
- e = expectThrows(IllegalArgumentException.class, () -> foo.minMaxChildren(positiveValue, positiveValue - 10));
- assertEquals("[has_child] 'max_children' is less than 'min_children'", e.getMessage());
- }
-
- public void testFromJson() throws IOException {
- String query =
- "{\n" +
- " \"has_child\" : {\n" +
- " \"query\" : {\n" +
- " \"range\" : {\n" +
- " \"mapped_string\" : {\n" +
- " \"from\" : \"agJhRET\",\n" +
- " \"to\" : \"zvqIq\",\n" +
- " \"include_lower\" : true,\n" +
- " \"include_upper\" : true,\n" +
- " \"boost\" : 1.0\n" +
- " }\n" +
- " }\n" +
- " },\n" +
- " \"type\" : \"child\",\n" +
- " \"score_mode\" : \"avg\",\n" +
- " \"min_children\" : 883170873,\n" +
- " \"max_children\" : 1217235442,\n" +
- " \"ignore_unmapped\" : false,\n" +
- " \"boost\" : 2.0,\n" +
- " \"_name\" : \"WNzYMJKRwePuRBh\",\n" +
- " \"inner_hits\" : {\n" +
- " \"name\" : \"inner_hits_name\",\n" +
- " \"ignore_unmapped\" : false,\n" +
- " \"from\" : 0,\n" +
- " \"size\" : 100,\n" +
- " \"version\" : false,\n" +
- " \"explain\" : false,\n" +
- " \"track_scores\" : false,\n" +
- " \"sort\" : [ {\n" +
- " \"mapped_string\" : {\n" +
- " \"order\" : \"asc\"\n" +
- " }\n" +
- " } ]\n" +
- " }\n" +
- " }\n" +
- "}";
- HasChildQueryBuilder queryBuilder = (HasChildQueryBuilder) parseQuery(query);
- checkGeneratedJson(query, queryBuilder);
- assertEquals(query, queryBuilder.maxChildren(), 1217235442);
- assertEquals(query, queryBuilder.minChildren(), 883170873);
- assertEquals(query, queryBuilder.boost(), 2.0f, 0.0f);
- assertEquals(query, queryBuilder.queryName(), "WNzYMJKRwePuRBh");
- assertEquals(query, queryBuilder.childType(), "child");
- assertEquals(query, queryBuilder.scoreMode(), ScoreMode.Avg);
- assertNotNull(query, queryBuilder.innerHit());
- InnerHitBuilder expected = new InnerHitBuilder(new InnerHitBuilder(), queryBuilder.query(), "child", false)
- .setName("inner_hits_name")
- .setSize(100)
- .addSort(new FieldSortBuilder("mapped_string").order(SortOrder.ASC));
- assertEquals(query, queryBuilder.innerHit(), expected);
- }
-
- public void testToQueryInnerQueryType() throws IOException {
- String[] searchTypes = new String[]{PARENT_TYPE};
- QueryShardContext shardContext = createShardContext();
- shardContext.setTypes(searchTypes);
- HasChildQueryBuilder hasChildQueryBuilder = QueryBuilders.hasChildQuery(CHILD_TYPE, 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_TYPE, "id");
- }
-
- static void assertLateParsingQuery(Query query, String type, String id) throws IOException {
- assertThat(query, instanceOf(HasChildQueryBuilder.LateParsingQuery.class));
- HasChildQueryBuilder.LateParsingQuery lateParsingQuery = (HasChildQueryBuilder.LateParsingQuery) query;
- assertThat(lateParsingQuery.getInnerQuery(), instanceOf(BooleanQuery.class));
- BooleanQuery booleanQuery = (BooleanQuery) lateParsingQuery.getInnerQuery();
- assertThat(booleanQuery.clauses().size(), equalTo(2));
- //check the inner ids query, we have to call rewrite to get to check the type it's executed against
- assertThat(booleanQuery.clauses().get(0).getOccur(), equalTo(BooleanClause.Occur.MUST));
- assertThat(booleanQuery.clauses().get(0).getQuery(), instanceOf(TermInSetQuery.class));
- TermInSetQuery termsQuery = (TermInSetQuery) booleanQuery.clauses().get(0).getQuery();
- Query rewrittenTermsQuery = termsQuery.rewrite(null);
- assertThat(rewrittenTermsQuery, instanceOf(ConstantScoreQuery.class));
- ConstantScoreQuery constantScoreQuery = (ConstantScoreQuery) rewrittenTermsQuery;
- assertThat(constantScoreQuery.getQuery(), instanceOf(BooleanQuery.class));
- BooleanQuery booleanTermsQuery = (BooleanQuery) constantScoreQuery.getQuery();
- assertThat(booleanTermsQuery.clauses().toString(), booleanTermsQuery.clauses().size(), equalTo(1));
- assertThat(booleanTermsQuery.clauses().get(0).getOccur(), equalTo(BooleanClause.Occur.SHOULD));
- assertThat(booleanTermsQuery.clauses().get(0).getQuery(), instanceOf(TermQuery.class));
- TermQuery termQuery = (TermQuery) booleanTermsQuery.clauses().get(0).getQuery();
- assertThat(termQuery.getTerm().field(), equalTo(UidFieldMapper.NAME));
- //we want to make sure that the inner ids query gets executed against the child type rather than the main type we initially set to the context
- BytesRef[] ids = Uid.createUidsForTypesAndIds(Collections.singletonList(type), Collections.singletonList(id));
- assertThat(termQuery.getTerm().bytes(), equalTo(ids[0]));
- //check the type filter
- assertThat(booleanQuery.clauses().get(1).getOccur(), equalTo(BooleanClause.Occur.FILTER));
- assertEquals(new TypeFieldMapper.TypesQuery(new BytesRef(type)), booleanQuery.clauses().get(1).getQuery());
- }
-
- @Override
- public void testMustRewrite() throws IOException {
- try {
- super.testMustRewrite();
- } catch (UnsupportedOperationException e) {
- if (requiresRewrite == false) {
- throw e;
- }
- }
- }
-
- public void testNonDefaultSimilarity() throws Exception {
- QueryShardContext shardContext = createShardContext();
- HasChildQueryBuilder hasChildQueryBuilder = QueryBuilders.hasChildQuery(CHILD_TYPE, new TermQueryBuilder("custom_string", "value"), ScoreMode.None);
- HasChildQueryBuilder.LateParsingQuery query = (HasChildQueryBuilder.LateParsingQuery) hasChildQueryBuilder.toQuery(shardContext);
- Similarity expected = SimilarityService.BUILT_IN.get(similarity)
- .apply(similarity, Settings.EMPTY, Settings.builder().put(IndexMetaData.SETTING_VERSION_CREATED, Version.CURRENT).build())
- .get();
- assertThat(((PerFieldSimilarityWrapper) query.getSimilarity()).get("custom_string"), instanceOf(expected.getClass()));
- }
-
- public void testMinFromString() {
- assertThat("fromString(min) != MIN", ScoreMode.Min, equalTo(HasChildQueryBuilder.parseScoreMode("min")));
- assertThat("min", equalTo(HasChildQueryBuilder.scoreModeAsString(ScoreMode.Min)));
- }
-
- public void testMaxFromString() {
- assertThat("fromString(max) != MAX", ScoreMode.Max, equalTo(HasChildQueryBuilder.parseScoreMode("max")));
- assertThat("max", equalTo(HasChildQueryBuilder.scoreModeAsString(ScoreMode.Max)));
- }
-
- public void testAvgFromString() {
- assertThat("fromString(avg) != AVG", ScoreMode.Avg, equalTo(HasChildQueryBuilder.parseScoreMode("avg")));
- assertThat("avg", equalTo(HasChildQueryBuilder.scoreModeAsString(ScoreMode.Avg)));
- }
-
- public void testSumFromString() {
- assertThat("fromString(total) != SUM", ScoreMode.Total, equalTo(HasChildQueryBuilder.parseScoreMode("sum")));
- assertThat("sum", equalTo(HasChildQueryBuilder.scoreModeAsString(ScoreMode.Total)));
- }
-
- public void testNoneFromString() {
- assertThat("fromString(none) != NONE", ScoreMode.None, equalTo(HasChildQueryBuilder.parseScoreMode("none")));
- assertThat("none", equalTo(HasChildQueryBuilder.scoreModeAsString(ScoreMode.None)));
- }
-
- /**
- * Should throw {@link IllegalArgumentException} instead of NPE.
- */
- public void testThatNullFromStringThrowsException() {
- IllegalArgumentException e = expectThrows(IllegalArgumentException.class, () -> HasChildQueryBuilder.parseScoreMode(null));
- assertEquals("No score mode for child query [null] found", e.getMessage());
- }
-
- /**
- * Failure should not change (and the value should never match anything...).
- */
- public void testThatUnrecognizedFromStringThrowsException() {
- IllegalArgumentException e = expectThrows(IllegalArgumentException.class,
- () -> HasChildQueryBuilder.parseScoreMode("unrecognized value"));
- assertEquals("No score mode for child query [unrecognized value] found", e.getMessage());
- }
-
- public void testIgnoreUnmapped() throws IOException {
- final HasChildQueryBuilder queryBuilder = new HasChildQueryBuilder("unmapped", new MatchAllQueryBuilder(), ScoreMode.None);
- queryBuilder.ignoreUnmapped(true);
- Query query = queryBuilder.toQuery(createShardContext());
- assertThat(query, notNullValue());
- assertThat(query, instanceOf(MatchNoDocsQuery.class));
-
- final HasChildQueryBuilder failingQueryBuilder = new HasChildQueryBuilder("unmapped", new MatchAllQueryBuilder(), ScoreMode.None);
- failingQueryBuilder.ignoreUnmapped(false);
- QueryShardException e = expectThrows(QueryShardException.class, () -> failingQueryBuilder.toQuery(createShardContext()));
- assertThat(e.getMessage(), containsString("[" + HasChildQueryBuilder.NAME + "] no mapping found for type [unmapped]"));
- }
-
- public void testIgnoreUnmappedWithRewrite() throws IOException {
- // WrapperQueryBuilder makes sure we always rewrite
- final HasChildQueryBuilder queryBuilder
- = new HasChildQueryBuilder("unmapped", new WrapperQueryBuilder(new MatchAllQueryBuilder().toString()), ScoreMode.None);
- queryBuilder.ignoreUnmapped(true);
- QueryShardContext queryShardContext = createShardContext();
- Query query = queryBuilder.rewrite(queryShardContext).toQuery(queryShardContext);
- assertThat(query, notNullValue());
- assertThat(query, instanceOf(MatchNoDocsQuery.class));
- }
-}
diff --git a/core/src/test/java/org/elasticsearch/index/query/HasParentQueryBuilderTests.java b/core/src/test/java/org/elasticsearch/index/query/HasParentQueryBuilderTests.java
deleted file mode 100644
index 22ea2cc9c4..0000000000
--- a/core/src/test/java/org/elasticsearch/index/query/HasParentQueryBuilderTests.java
+++ /dev/null
@@ -1,227 +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.search.MatchNoDocsQuery;
-import org.apache.lucene.search.Query;
-import org.apache.lucene.search.join.ScoreMode;
-import org.elasticsearch.action.admin.indices.mapping.put.PutMappingRequest;
-import org.elasticsearch.common.compress.CompressedXContent;
-import org.elasticsearch.common.xcontent.ToXContent;
-import org.elasticsearch.common.xcontent.XContentBuilder;
-import org.elasticsearch.common.xcontent.XContentFactory;
-import org.elasticsearch.index.mapper.MapperService;
-import org.elasticsearch.search.fetch.subphase.InnerHitsContext;
-import org.elasticsearch.search.internal.SearchContext;
-import org.elasticsearch.search.sort.FieldSortBuilder;
-import org.elasticsearch.search.sort.SortOrder;
-import org.elasticsearch.test.AbstractQueryTestCase;
-
-import java.io.IOException;
-import java.util.HashMap;
-import java.util.Map;
-
-import static org.hamcrest.CoreMatchers.containsString;
-import static org.hamcrest.CoreMatchers.equalTo;
-import static org.hamcrest.CoreMatchers.instanceOf;
-import static org.hamcrest.CoreMatchers.notNullValue;
-
-public class HasParentQueryBuilderTests extends AbstractQueryTestCase<HasParentQueryBuilder> {
- protected static final String PARENT_TYPE = "parent";
- protected static final String CHILD_TYPE = "child";
-
- boolean requiresRewrite = false;
-
- @Override
- protected void initializeAdditionalMappings(MapperService mapperService) throws IOException {
- mapperService.merge(PARENT_TYPE, new CompressedXContent(PutMappingRequest.buildFromSimplifiedDef(PARENT_TYPE,
- STRING_FIELD_NAME, "type=text",
- STRING_FIELD_NAME_2, "type=keyword",
- INT_FIELD_NAME, "type=integer",
- DOUBLE_FIELD_NAME, "type=double",
- BOOLEAN_FIELD_NAME, "type=boolean",
- DATE_FIELD_NAME, "type=date",
- OBJECT_FIELD_NAME, "type=object"
- ).string()), MapperService.MergeReason.MAPPING_UPDATE, false);
- mapperService.merge(CHILD_TYPE, new CompressedXContent(PutMappingRequest.buildFromSimplifiedDef(CHILD_TYPE,
- "_parent", "type=" + PARENT_TYPE,
- STRING_FIELD_NAME, "type=text",
- STRING_FIELD_NAME_2, "type=keyword",
- INT_FIELD_NAME, "type=integer",
- DOUBLE_FIELD_NAME, "type=double",
- BOOLEAN_FIELD_NAME, "type=boolean",
- DATE_FIELD_NAME, "type=date",
- OBJECT_FIELD_NAME, "type=object"
- ).string()), MapperService.MergeReason.MAPPING_UPDATE, false);
- mapperService.merge("just_a_type", new CompressedXContent(PutMappingRequest.buildFromSimplifiedDef("just_a_type"
- ).string()), MapperService.MergeReason.MAPPING_UPDATE, false);
- }
-
- /**
- * @return a {@link HasChildQueryBuilder} with random values all over the place
- */
- @Override
- protected HasParentQueryBuilder doCreateTestQueryBuilder() {
- QueryBuilder innerQueryBuilder = RandomQueryBuilder.createQuery(random());
- if (randomBoolean()) {
- requiresRewrite = true;
- innerQueryBuilder = new WrapperQueryBuilder(innerQueryBuilder.toString());
- }
- HasParentQueryBuilder hqb = new HasParentQueryBuilder(PARENT_TYPE, innerQueryBuilder, randomBoolean());
- hqb.ignoreUnmapped(randomBoolean());
- if (randomBoolean()) {
- hqb.innerHit(new InnerHitBuilder()
- .setName(randomAlphaOfLengthBetween(1, 10))
- .setSize(randomIntBetween(0, 100))
- .addSort(new FieldSortBuilder(STRING_FIELD_NAME_2).order(SortOrder.ASC)), hqb.ignoreUnmapped());
- }
- return hqb;
- }
-
- @Override
- protected void doAssertLuceneQuery(HasParentQueryBuilder queryBuilder, Query query, SearchContext searchContext) throws IOException {
- assertThat(query, instanceOf(HasChildQueryBuilder.LateParsingQuery.class));
- HasChildQueryBuilder.LateParsingQuery lpq = (HasChildQueryBuilder.LateParsingQuery) query;
- assertEquals(queryBuilder.score() ? ScoreMode.Max : ScoreMode.None, lpq.getScoreMode());
-
- if (queryBuilder.innerHit() != null) {
- // have to rewrite again because the provided queryBuilder hasn't been rewritten (directly returned from
- // doCreateTestQueryBuilder)
- queryBuilder = (HasParentQueryBuilder) queryBuilder.rewrite(searchContext.getQueryShardContext());
-
- assertNotNull(searchContext);
- Map<String, InnerHitBuilder> innerHitBuilders = new HashMap<>();
- InnerHitBuilder.extractInnerHits(queryBuilder, innerHitBuilders);
- for (InnerHitBuilder builder : innerHitBuilders.values()) {
- builder.build(searchContext, searchContext.innerHits());
- }
- assertNotNull(searchContext.innerHits());
- assertEquals(1, searchContext.innerHits().getInnerHits().size());
- assertTrue(searchContext.innerHits().getInnerHits().containsKey(queryBuilder.innerHit().getName()));
- InnerHitsContext.BaseInnerHits innerHits = searchContext.innerHits()
- .getInnerHits().get(queryBuilder.innerHit().getName());
- assertEquals(innerHits.size(), queryBuilder.innerHit().getSize());
- assertEquals(innerHits.sort().sort.getSort().length, 1);
- assertEquals(innerHits.sort().sort.getSort()[0].getField(), STRING_FIELD_NAME_2);
- }
- }
-
- public void testIllegalValues() throws IOException {
- QueryBuilder query = RandomQueryBuilder.createQuery(random());
- IllegalArgumentException e = expectThrows(IllegalArgumentException.class,
- () -> QueryBuilders.hasParentQuery(null, query, false));
- assertThat(e.getMessage(), equalTo("[has_parent] requires 'type' field"));
-
- e = expectThrows(IllegalArgumentException.class,
- () -> QueryBuilders.hasParentQuery("foo", null, false));
- assertThat(e.getMessage(), equalTo("[has_parent] requires 'query' field"));
-
- QueryShardContext context = createShardContext();
- HasParentQueryBuilder qb = QueryBuilders.hasParentQuery("just_a_type", new MatchAllQueryBuilder(), false);
- QueryShardException qse = expectThrows(QueryShardException.class, () -> qb.doToQuery(context));
- assertThat(qse.getMessage(), equalTo("[has_parent] no child types found for type [just_a_type]"));
- }
-
- public void testDeprecatedXContent() throws IOException {
- XContentBuilder builder = XContentFactory.jsonBuilder().prettyPrint();
- builder.startObject();
- builder.startObject("has_parent");
- builder.field("query");
- new TermQueryBuilder("a", "a").toXContent(builder, ToXContent.EMPTY_PARAMS);
- builder.field("type", "foo"); // deprecated
- builder.endObject();
- builder.endObject();
- HasParentQueryBuilder queryBuilder = (HasParentQueryBuilder) parseQuery(builder.string());
- assertEquals("foo", queryBuilder.type());
- assertWarnings("Deprecated field [type] used, expected [parent_type] instead");
- }
-
- public void testToQueryInnerQueryType() throws IOException {
- String[] searchTypes = new String[]{CHILD_TYPE};
- QueryShardContext shardContext = createShardContext();
- shardContext.setTypes(searchTypes);
- HasParentQueryBuilder hasParentQueryBuilder = new HasParentQueryBuilder(PARENT_TYPE, 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_TYPE, "id");
- }
-
- @Override
- public void testMustRewrite() throws IOException {
- try {
- super.testMustRewrite();
- } catch (UnsupportedOperationException e) {
- if (requiresRewrite == false) {
- throw e;
- }
- }
- }
-
- public void testFromJson() throws IOException {
- String json =
- "{\n" +
- " \"has_parent\" : {\n" +
- " \"query\" : {\n" +
- " \"term\" : {\n" +
- " \"tag\" : {\n" +
- " \"value\" : \"something\",\n" +
- " \"boost\" : 1.0\n" +
- " }\n" +
- " }\n" +
- " },\n" +
- " \"parent_type\" : \"blog\",\n" +
- " \"score\" : true,\n" +
- " \"ignore_unmapped\" : false,\n" +
- " \"boost\" : 1.0\n" +
- " }\n" +
- "}";
- HasParentQueryBuilder parsed = (HasParentQueryBuilder) parseQuery(json);
- checkGeneratedJson(json, parsed);
- assertEquals(json, "blog", parsed.type());
- assertEquals(json, "something", ((TermQueryBuilder) parsed.query()).value());
- }
-
- public void testIgnoreUnmapped() throws IOException {
- final HasParentQueryBuilder queryBuilder = new HasParentQueryBuilder("unmapped", new MatchAllQueryBuilder(), false);
- queryBuilder.ignoreUnmapped(true);
- Query query = queryBuilder.toQuery(createShardContext());
- assertThat(query, notNullValue());
- assertThat(query, instanceOf(MatchNoDocsQuery.class));
-
- final HasParentQueryBuilder failingQueryBuilder = new HasParentQueryBuilder("unmapped", new MatchAllQueryBuilder(), false);
- failingQueryBuilder.ignoreUnmapped(false);
- QueryShardException e = expectThrows(QueryShardException.class, () -> failingQueryBuilder.toQuery(createShardContext()));
- assertThat(e.getMessage(),
- containsString("[" + HasParentQueryBuilder.NAME + "] query configured 'parent_type' [unmapped] is not a valid type"));
- }
-
- public void testIgnoreUnmappedWithRewrite() throws IOException {
- // WrapperQueryBuilder makes sure we always rewrite
- final HasParentQueryBuilder queryBuilder =
- new HasParentQueryBuilder("unmapped", new WrapperQueryBuilder(new MatchAllQueryBuilder().toString()), false);
- queryBuilder.ignoreUnmapped(true);
- QueryShardContext queryShardContext = createShardContext();
- Query query = queryBuilder.rewrite(queryShardContext).toQuery(queryShardContext);
- assertThat(query, notNullValue());
- assertThat(query, instanceOf(MatchNoDocsQuery.class));
- }
-}
diff --git a/core/src/test/java/org/elasticsearch/index/query/InnerHitBuilderTests.java b/core/src/test/java/org/elasticsearch/index/query/InnerHitBuilderTests.java
index cd6622368c..9e199f71ce 100644
--- a/core/src/test/java/org/elasticsearch/index/query/InnerHitBuilderTests.java
+++ b/core/src/test/java/org/elasticsearch/index/query/InnerHitBuilderTests.java
@@ -153,24 +153,6 @@ public class InnerHitBuilderTests extends ESTestCase {
assertThat(innerHitBuilders.get(leafInnerHits.getName()), notNullValue());
}
- public void testInlineLeafInnerHitsHasChildQuery() throws Exception {
- InnerHitBuilder leafInnerHits = randomInnerHits();
- HasChildQueryBuilder hasChildQueryBuilder = new HasChildQueryBuilder("type", new MatchAllQueryBuilder(), ScoreMode.None)
- .innerHit(leafInnerHits, false);
- Map<String, InnerHitBuilder> innerHitBuilders = new HashMap<>();
- hasChildQueryBuilder.extractInnerHitBuilders(innerHitBuilders);
- assertThat(innerHitBuilders.get(leafInnerHits.getName()), notNullValue());
- }
-
- public void testInlineLeafInnerHitsHasParentQuery() throws Exception {
- InnerHitBuilder leafInnerHits = randomInnerHits();
- HasParentQueryBuilder hasParentQueryBuilder = new HasParentQueryBuilder("type", new MatchAllQueryBuilder(), false)
- .innerHit(leafInnerHits, false);
- Map<String, InnerHitBuilder> innerHitBuilders = new HashMap<>();
- hasParentQueryBuilder.extractInnerHitBuilders(innerHitBuilders);
- assertThat(innerHitBuilders.get(leafInnerHits.getName()), notNullValue());
- }
-
public void testInlineLeafInnerHitsNestedQueryViaBoolQuery() {
InnerHitBuilder leafInnerHits = randomInnerHits();
NestedQueryBuilder nestedQueryBuilder = new NestedQueryBuilder("path", new MatchAllQueryBuilder(), ScoreMode.None)
@@ -181,25 +163,6 @@ public class InnerHitBuilderTests extends ESTestCase {
assertThat(innerHitBuilders.get(leafInnerHits.getName()), notNullValue());
}
- public void testInlineLeafInnerHitsNestedQueryViaDisMaxQuery() {
- InnerHitBuilder leafInnerHits1 = randomInnerHits();
- NestedQueryBuilder nestedQueryBuilder = new NestedQueryBuilder("path", new MatchAllQueryBuilder(), ScoreMode.None)
- .innerHit(leafInnerHits1, false);
-
- InnerHitBuilder leafInnerHits2 = randomInnerHits();
- HasChildQueryBuilder hasChildQueryBuilder = new HasChildQueryBuilder("type", new MatchAllQueryBuilder(), ScoreMode.None)
- .innerHit(leafInnerHits2, false);
-
- DisMaxQueryBuilder disMaxQueryBuilder = new DisMaxQueryBuilder();
- disMaxQueryBuilder.add(nestedQueryBuilder);
- disMaxQueryBuilder.add(hasChildQueryBuilder);
- Map<String, InnerHitBuilder> innerHitBuilders = new HashMap<>();
- disMaxQueryBuilder.extractInnerHitBuilders(innerHitBuilders);
- assertThat(innerHitBuilders.size(), equalTo(2));
- assertThat(innerHitBuilders.get(leafInnerHits1.getName()), notNullValue());
- assertThat(innerHitBuilders.get(leafInnerHits2.getName()), notNullValue());
- }
-
public void testInlineLeafInnerHitsNestedQueryViaConstantScoreQuery() {
InnerHitBuilder leafInnerHits = randomInnerHits();
NestedQueryBuilder nestedQueryBuilder = new NestedQueryBuilder("path", new MatchAllQueryBuilder(), ScoreMode.None)
@@ -252,43 +215,6 @@ public class InnerHitBuilderTests extends ESTestCase {
assertThat(innerHitsContext.getInnerHits().size(), equalTo(0));
}
- public void testBuild_ignoreUnmappedHasChildQuery() throws Exception {
- QueryShardContext queryShardContext = mock(QueryShardContext.class);
- when(queryShardContext.documentMapper("type")).thenReturn(null);
- SearchContext searchContext = mock(SearchContext.class);
- when(searchContext.getQueryShardContext()).thenReturn(queryShardContext);
-
- InnerHitBuilder leafInnerHits = randomInnerHits();
- HasChildQueryBuilder query1 = new HasChildQueryBuilder("type", new MatchAllQueryBuilder(), ScoreMode.None)
- .innerHit(leafInnerHits, false);
- expectThrows(IllegalStateException.class, () -> query1.innerHit().build(searchContext, new InnerHitsContext()));
-
- HasChildQueryBuilder query2 = new HasChildQueryBuilder("type", new MatchAllQueryBuilder(), ScoreMode.None)
- .innerHit(leafInnerHits, true);
- InnerHitsContext innerHitsContext = new InnerHitsContext();
- query2.innerHit().build(searchContext, innerHitsContext);
- assertThat(innerHitsContext.getInnerHits().size(), equalTo(0));
- }
-
- public void testBuild_ingoreUnmappedHasParentQuery() throws Exception {
- QueryShardContext queryShardContext = mock(QueryShardContext.class);
- when(queryShardContext.documentMapper("type")).thenReturn(null);
- SearchContext searchContext = mock(SearchContext.class);
- when(searchContext.getQueryShardContext()).thenReturn(queryShardContext);
-
- InnerHitBuilder leafInnerHits = randomInnerHits();
- HasParentQueryBuilder query1 = new HasParentQueryBuilder("type", new MatchAllQueryBuilder(), false)
- .innerHit(leafInnerHits, false);
- expectThrows(IllegalStateException.class, () -> query1.innerHit().build(searchContext, new InnerHitsContext()));
-
- HasParentQueryBuilder query2 = new HasParentQueryBuilder("type", new MatchAllQueryBuilder(), false)
- .innerHit(leafInnerHits, true);
- InnerHitsContext innerHitsContext = new InnerHitsContext();
- query2.innerHit().build(searchContext, innerHitsContext);
- assertThat(innerHitsContext.getInnerHits().size(), equalTo(0));
- }
-
-
public static InnerHitBuilder randomInnerHits() {
return randomInnerHits(true, true);
}
diff --git a/core/src/test/java/org/elasticsearch/index/query/NestedQueryBuilderTests.java b/core/src/test/java/org/elasticsearch/index/query/NestedQueryBuilderTests.java
index 267963878e..ed4fbcd53c 100644
--- a/core/src/test/java/org/elasticsearch/index/query/NestedQueryBuilderTests.java
+++ b/core/src/test/java/org/elasticsearch/index/query/NestedQueryBuilderTests.java
@@ -61,7 +61,7 @@ public class NestedQueryBuilderTests extends AbstractQueryTestCase<NestedQueryBu
}
/**
- * @return a {@link HasChildQueryBuilder} with random values all over the place
+ * @return a {@link NestedQueryBuilder} with random values all over the place
*/
@Override
protected NestedQueryBuilder doCreateTestQueryBuilder() {
@@ -203,4 +203,46 @@ public class NestedQueryBuilderTests extends AbstractQueryTestCase<NestedQueryBu
assertThat(query, notNullValue());
assertThat(query, instanceOf(MatchNoDocsQuery.class));
}
+
+ public void testMinFromString() {
+ assertThat("fromString(min) != MIN", ScoreMode.Min, equalTo(NestedQueryBuilder.parseScoreMode("min")));
+ assertThat("min", equalTo(NestedQueryBuilder.scoreModeAsString(ScoreMode.Min)));
+ }
+
+ public void testMaxFromString() {
+ assertThat("fromString(max) != MAX", ScoreMode.Max, equalTo(NestedQueryBuilder.parseScoreMode("max")));
+ assertThat("max", equalTo(NestedQueryBuilder.scoreModeAsString(ScoreMode.Max)));
+ }
+
+ public void testAvgFromString() {
+ assertThat("fromString(avg) != AVG", ScoreMode.Avg, equalTo(NestedQueryBuilder.parseScoreMode("avg")));
+ assertThat("avg", equalTo(NestedQueryBuilder.scoreModeAsString(ScoreMode.Avg)));
+ }
+
+ public void testSumFromString() {
+ assertThat("fromString(total) != SUM", ScoreMode.Total, equalTo(NestedQueryBuilder.parseScoreMode("sum")));
+ assertThat("sum", equalTo(NestedQueryBuilder.scoreModeAsString(ScoreMode.Total)));
+ }
+
+ public void testNoneFromString() {
+ assertThat("fromString(none) != NONE", ScoreMode.None, equalTo(NestedQueryBuilder.parseScoreMode("none")));
+ assertThat("none", equalTo(NestedQueryBuilder.scoreModeAsString(ScoreMode.None)));
+ }
+
+ /**
+ * Should throw {@link IllegalArgumentException} instead of NPE.
+ */
+ public void testThatNullFromStringThrowsException() {
+ IllegalArgumentException e = expectThrows(IllegalArgumentException.class, () -> NestedQueryBuilder.parseScoreMode(null));
+ assertEquals("No score mode for child query [null] found", e.getMessage());
+ }
+
+ /**
+ * Failure should not change (and the value should never match anything...).
+ */
+ public void testThatUnrecognizedFromStringThrowsException() {
+ IllegalArgumentException e = expectThrows(IllegalArgumentException.class,
+ () -> NestedQueryBuilder.parseScoreMode("unrecognized value"));
+ assertEquals("No score mode for child query [unrecognized value] found", e.getMessage());
+ }
}
diff --git a/core/src/test/java/org/elasticsearch/node/InternalSettingsPreparerTests.java b/core/src/test/java/org/elasticsearch/node/InternalSettingsPreparerTests.java
index 5db397ab16..bf23da1868 100644
--- a/core/src/test/java/org/elasticsearch/node/InternalSettingsPreparerTests.java
+++ b/core/src/test/java/org/elasticsearch/node/InternalSettingsPreparerTests.java
@@ -150,38 +150,24 @@ public class InternalSettingsPreparerTests extends ESTestCase {
}
}
- public void testMultipleSettingsFileNotAllowed() throws IOException {
- InputStream yaml = getClass().getResourceAsStream("/config/elasticsearch.yaml");
- InputStream json = getClass().getResourceAsStream("/config/elasticsearch.json");
+ public void testYamlNotAllowed() throws IOException {
+ InputStream yaml = getClass().getResourceAsStream("/config/elasticsearch.yml");
Path config = homeDir.resolve("config");
Files.createDirectory(config);
Files.copy(yaml, config.resolve("elasticsearch.yaml"));
- Files.copy(json, config.resolve("elasticsearch.json"));
-
SettingsException e = expectThrows(SettingsException.class, () ->
- InternalSettingsPreparer.prepareEnvironment(Settings.builder().put(baseEnvSettings).build(), null)
- );
- assertTrue(e.getMessage(), e.getMessage().contains("multiple settings files found with suffixes"));
- assertTrue(e.getMessage(), e.getMessage().contains(".yaml"));
- assertTrue(e.getMessage(), e.getMessage().contains(".json"));
- }
-
- public void testYmlExtension() throws IOException {
- InputStream yaml = getClass().getResourceAsStream("/config/elasticsearch.yaml");
- Path config = homeDir.resolve("config");
- Files.createDirectory(config);
- Files.copy(yaml, config.resolve("elasticsearch.yml"));
- Environment env = InternalSettingsPreparer.prepareEnvironment(Settings.builder().put(baseEnvSettings).build(), null);
- assertEquals(".yml", env.configExtension());
+ InternalSettingsPreparer.prepareEnvironment(Settings.builder().put(baseEnvSettings).build(), null));
+ assertEquals("elasticsearch.yaml was deprecated in 5.5.0 and must be renamed to elasticsearch.yml", e.getMessage());
}
- public void testJsonExtension() throws IOException {
+ public void testJsonNotAllowed() throws IOException {
InputStream yaml = getClass().getResourceAsStream("/config/elasticsearch.json");
Path config = homeDir.resolve("config");
Files.createDirectory(config);
Files.copy(yaml, config.resolve("elasticsearch.json"));
- Environment env = InternalSettingsPreparer.prepareEnvironment(Settings.builder().put(baseEnvSettings).build(), null);
- assertEquals(".json", env.configExtension());
+ SettingsException e = expectThrows(SettingsException.class, () ->
+ InternalSettingsPreparer.prepareEnvironment(Settings.builder().put(baseEnvSettings).build(), null));
+ assertEquals("elasticsearch.json was deprecated in 5.5.0 and must be converted to elasticsearch.yml", e.getMessage());
}
public void testSecureSettings() {
diff --git a/core/src/test/java/org/elasticsearch/search/SearchModuleTests.java b/core/src/test/java/org/elasticsearch/search/SearchModuleTests.java
index 8514096b83..96767c99b9 100644
--- a/core/src/test/java/org/elasticsearch/search/SearchModuleTests.java
+++ b/core/src/test/java/org/elasticsearch/search/SearchModuleTests.java
@@ -270,8 +270,6 @@ public class SearchModuleTests extends ModuleTestCase {
"geo_distance",
"geo_polygon",
"geo_shape",
- "has_child",
- "has_parent",
"ids",
"match",
"match_all",
diff --git a/core/src/test/java/org/elasticsearch/search/aggregations/AggregationsTests.java b/core/src/test/java/org/elasticsearch/search/aggregations/AggregationsTests.java
index 4cfc87c171..a70c3c2d1b 100644
--- a/core/src/test/java/org/elasticsearch/search/aggregations/AggregationsTests.java
+++ b/core/src/test/java/org/elasticsearch/search/aggregations/AggregationsTests.java
@@ -26,20 +26,18 @@ import org.elasticsearch.common.xcontent.XContentHelper;
import org.elasticsearch.common.xcontent.XContentParser;
import org.elasticsearch.common.xcontent.XContentType;
import org.elasticsearch.rest.action.search.RestSearchAction;
-import org.elasticsearch.search.aggregations.bucket.InternalSingleBucketAggregationTestCase;
-import org.elasticsearch.search.aggregations.bucket.children.InternalChildrenTests;
import org.elasticsearch.search.aggregations.bucket.filter.InternalFilterTests;
-import org.elasticsearch.search.aggregations.bucket.global.InternalGlobalTests;
import org.elasticsearch.search.aggregations.bucket.geogrid.InternalGeoHashGridTests;
+import org.elasticsearch.search.aggregations.bucket.global.InternalGlobalTests;
import org.elasticsearch.search.aggregations.bucket.histogram.InternalDateHistogramTests;
import org.elasticsearch.search.aggregations.bucket.histogram.InternalHistogramTests;
import org.elasticsearch.search.aggregations.bucket.missing.InternalMissingTests;
import org.elasticsearch.search.aggregations.bucket.nested.InternalNestedTests;
import org.elasticsearch.search.aggregations.bucket.nested.InternalReverseNestedTests;
-import org.elasticsearch.search.aggregations.bucket.sampler.InternalSamplerTests;
import org.elasticsearch.search.aggregations.bucket.range.InternalRangeTests;
import org.elasticsearch.search.aggregations.bucket.range.date.InternalDateRangeTests;
import org.elasticsearch.search.aggregations.bucket.range.geodistance.InternalGeoDistanceTests;
+import org.elasticsearch.search.aggregations.bucket.sampler.InternalSamplerTests;
import org.elasticsearch.search.aggregations.bucket.terms.DoubleTermsTests;
import org.elasticsearch.search.aggregations.bucket.terms.LongTermsTests;
import org.elasticsearch.search.aggregations.bucket.terms.StringTermsTests;
@@ -117,7 +115,8 @@ public class AggregationsTests extends ESTestCase {
aggsTests.add(new InternalMissingTests());
aggsTests.add(new InternalNestedTests());
aggsTests.add(new InternalReverseNestedTests());
- aggsTests.add(new InternalChildrenTests());
+ // TODO can we find a way to include the children aggregation in this test?
+ //aggsTests.add(new InternalChildrenTests());
aggsTests.add(new InternalGlobalTests());
aggsTests.add(new InternalFilterTests());
aggsTests.add(new InternalSamplerTests());
diff --git a/core/src/test/java/org/elasticsearch/search/aggregations/AggregatorTestCase.java b/core/src/test/java/org/elasticsearch/search/aggregations/AggregatorTestCase.java
deleted file mode 100644
index 99cd626a7d..0000000000
--- a/core/src/test/java/org/elasticsearch/search/aggregations/AggregatorTestCase.java
+++ /dev/null
@@ -1,303 +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.search.aggregations;
-
-import org.apache.lucene.index.CompositeReaderContext;
-import org.apache.lucene.index.DirectoryReader;
-import org.apache.lucene.index.IndexReaderContext;
-import org.apache.lucene.index.LeafReaderContext;
-import org.apache.lucene.search.Collector;
-import org.apache.lucene.search.IndexSearcher;
-import org.apache.lucene.search.Query;
-import org.apache.lucene.search.QueryCache;
-import org.apache.lucene.search.QueryCachingPolicy;
-import org.apache.lucene.search.Weight;
-import org.elasticsearch.Version;
-import org.elasticsearch.cluster.metadata.IndexMetaData;
-import org.elasticsearch.common.lease.Releasable;
-import org.elasticsearch.common.lease.Releasables;
-import org.elasticsearch.common.lucene.index.ElasticsearchDirectoryReader;
-import org.elasticsearch.common.settings.Settings;
-import org.elasticsearch.common.util.MockBigArrays;
-import org.elasticsearch.index.Index;
-import org.elasticsearch.index.IndexSettings;
-import org.elasticsearch.index.cache.bitset.BitsetFilterCache;
-import org.elasticsearch.index.cache.bitset.BitsetFilterCache.Listener;
-import org.elasticsearch.index.cache.query.DisabledQueryCache;
-import org.elasticsearch.index.engine.Engine;
-import org.elasticsearch.index.fielddata.IndexFieldDataCache;
-import org.elasticsearch.index.fielddata.IndexFieldDataService;
-import org.elasticsearch.index.mapper.ContentPath;
-import org.elasticsearch.index.mapper.MappedFieldType;
-import org.elasticsearch.index.mapper.Mapper.BuilderContext;
-import org.elasticsearch.index.mapper.MapperService;
-import org.elasticsearch.index.mapper.ObjectMapper;
-import org.elasticsearch.index.mapper.ObjectMapper.Nested;
-import org.elasticsearch.index.query.QueryShardContext;
-import org.elasticsearch.index.query.support.NestedScope;
-import org.elasticsearch.index.shard.ShardId;
-import org.elasticsearch.indices.breaker.CircuitBreakerService;
-import org.elasticsearch.indices.breaker.NoneCircuitBreakerService;
-import org.elasticsearch.indices.fielddata.cache.IndicesFieldDataCache;
-import org.elasticsearch.mock.orig.Mockito;
-import org.elasticsearch.search.fetch.FetchPhase;
-import org.elasticsearch.search.fetch.subphase.DocValueFieldsFetchSubPhase;
-import org.elasticsearch.search.fetch.subphase.FetchSourceSubPhase;
-import org.elasticsearch.search.internal.ContextIndexSearcher;
-import org.elasticsearch.search.internal.SearchContext;
-import org.elasticsearch.search.lookup.SearchLookup;
-import org.elasticsearch.test.ESTestCase;
-import org.mockito.Matchers;
-
-import java.io.IOException;
-import java.util.ArrayList;
-import java.util.Arrays;
-import java.util.Collections;
-import java.util.List;
-
-import static org.mockito.Matchers.anyObject;
-import static org.mockito.Matchers.anyString;
-import static org.mockito.Mockito.doAnswer;
-import static org.mockito.Mockito.mock;
-import static org.mockito.Mockito.when;
-
-/**
- * Base class for testing {@link Aggregator} implementations.
- * Provides helpers for constructing and searching an {@link Aggregator} implementation based on a provided
- * {@link AggregationBuilder} instance.
- */
-public abstract class AggregatorTestCase extends ESTestCase {
- private static final String NESTEDFIELD_PREFIX = "nested_";
- private List<Releasable> releasables = new ArrayList<>();
-
- /** Create a factory for the given aggregation builder. */
- protected AggregatorFactory<?> createAggregatorFactory(AggregationBuilder aggregationBuilder,
- IndexSearcher indexSearcher,
- MappedFieldType... fieldTypes) throws IOException {
- IndexSettings indexSettings = createIndexSettings();
- SearchContext searchContext = createSearchContext(indexSearcher, indexSettings);
- CircuitBreakerService circuitBreakerService = new NoneCircuitBreakerService();
- when(searchContext.bigArrays()).thenReturn(new MockBigArrays(Settings.EMPTY, circuitBreakerService));
- // TODO: now just needed for top_hits, this will need to be revised for other agg unit tests:
- MapperService mapperService = mapperServiceMock();
- when(mapperService.getIndexSettings()).thenReturn(indexSettings);
- when(mapperService.hasNested()).thenReturn(false);
- when(searchContext.mapperService()).thenReturn(mapperService);
- IndexFieldDataService ifds = new IndexFieldDataService(indexSettings,
- new IndicesFieldDataCache(Settings.EMPTY, new IndexFieldDataCache.Listener() {
- }), circuitBreakerService, mapperService);
- when(searchContext.fieldData()).thenReturn(ifds);
-
- SearchLookup searchLookup = new SearchLookup(mapperService, ifds, new String[]{"type"});
- when(searchContext.lookup()).thenReturn(searchLookup);
-
- QueryShardContext queryShardContext = queryShardContextMock(mapperService, fieldTypes, circuitBreakerService);
- when(searchContext.getQueryShardContext()).thenReturn(queryShardContext);
-
- return aggregationBuilder.build(searchContext, null);
- }
-
- protected <A extends Aggregator> A createAggregator(AggregationBuilder aggregationBuilder,
- IndexSearcher indexSearcher,
- MappedFieldType... fieldTypes) throws IOException {
- @SuppressWarnings("unchecked")
- A aggregator = (A) createAggregatorFactory(aggregationBuilder, indexSearcher, fieldTypes).create(null, true);
- return aggregator;
- }
-
- protected SearchContext createSearchContext(IndexSearcher indexSearcher, IndexSettings indexSettings) {
- Engine.Searcher searcher = new Engine.Searcher("aggregator_test", indexSearcher);
- QueryCache queryCache = new DisabledQueryCache(indexSettings);
- QueryCachingPolicy queryCachingPolicy = new QueryCachingPolicy() {
- @Override
- public void onUse(Query query) {
- }
-
- @Override
- public boolean shouldCache(Query query) throws IOException {
- // never cache a query
- return false;
- }
- };
- ContextIndexSearcher contextIndexSearcher = new ContextIndexSearcher(searcher, queryCache, queryCachingPolicy);
-
- SearchContext searchContext = mock(SearchContext.class);
- when(searchContext.numberOfShards()).thenReturn(1);
- when(searchContext.searcher()).thenReturn(contextIndexSearcher);
- when(searchContext.fetchPhase())
- .thenReturn(new FetchPhase(Arrays.asList(new FetchSourceSubPhase(), new DocValueFieldsFetchSubPhase())));
- when(searchContext.getObjectMapper(anyString())).thenAnswer(invocation -> {
- String fieldName = (String) invocation.getArguments()[0];
- if (fieldName.startsWith(NESTEDFIELD_PREFIX)) {
- BuilderContext context = new BuilderContext(indexSettings.getSettings(), new ContentPath());
- return new ObjectMapper.Builder<>(fieldName).nested(Nested.newNested(false, false)).build(context);
- }
- return null;
- });
- when(searchContext.bitsetFilterCache()).thenReturn(new BitsetFilterCache(indexSettings, mock(Listener.class)));
- doAnswer(invocation -> {
- /* Store the releasables so we can release them at the end of the test case. This is important because aggregations don't
- * close their sub-aggregations. This is fairly similar to what the production code does. */
- releasables.add((Releasable) invocation.getArguments()[0]);
- return null;
- }).when(searchContext).addReleasable(anyObject(), anyObject());
- return searchContext;
- }
-
- protected IndexSettings createIndexSettings() {
- return new IndexSettings(
- IndexMetaData.builder("_index").settings(Settings.builder().put(IndexMetaData.SETTING_VERSION_CREATED, Version.CURRENT))
- .numberOfShards(1)
- .numberOfReplicas(0)
- .creationDate(System.currentTimeMillis())
- .build(),
- Settings.EMPTY
- );
- }
-
- /**
- * sub-tests that need a more complex mock can overwrite this
- */
- protected MapperService mapperServiceMock() {
- return mock(MapperService.class);
- }
-
- /**
- * sub-tests that need a more complex mock can overwrite this
- */
- protected QueryShardContext queryShardContextMock(MapperService mapperService, MappedFieldType[] fieldTypes,
- CircuitBreakerService circuitBreakerService) {
- QueryShardContext queryShardContext = mock(QueryShardContext.class);
- when(queryShardContext.getMapperService()).thenReturn(mapperService);
- for (MappedFieldType fieldType : fieldTypes) {
- when(queryShardContext.fieldMapper(fieldType.name())).thenReturn(fieldType);
- when(queryShardContext.getForField(fieldType)).then(invocation -> fieldType.fielddataBuilder().build(
- mapperService.getIndexSettings(), fieldType, new IndexFieldDataCache.None(), circuitBreakerService,
- mapperService));
- }
- NestedScope nestedScope = new NestedScope();
- when(queryShardContext.isFilter()).thenCallRealMethod();
- Mockito.doCallRealMethod().when(queryShardContext).setIsFilter(Matchers.anyBoolean());
- when(queryShardContext.nestedScope()).thenReturn(nestedScope);
- return queryShardContext;
- }
-
- protected <A extends InternalAggregation, C extends Aggregator> A search(IndexSearcher searcher,
- Query query,
- AggregationBuilder builder,
- MappedFieldType... fieldTypes) throws IOException {
- C a = createAggregator(builder, searcher, fieldTypes);
- try {
- a.preCollection();
- searcher.search(query, a);
- a.postCollection();
- @SuppressWarnings("unchecked")
- A internalAgg = (A) a.buildAggregation(0L);
- return internalAgg;
- } finally {
- Releasables.close(releasables);
- releasables.clear();
- }
- }
-
- /**
- * Divides the provided {@link IndexSearcher} in sub-searcher, one for each segment,
- * builds an aggregator for each sub-searcher filtered by the provided {@link Query} and
- * returns the reduced {@link InternalAggregation}.
- */
- protected <A extends InternalAggregation, C extends Aggregator> A searchAndReduce(IndexSearcher searcher,
- Query query,
- AggregationBuilder builder,
- MappedFieldType... fieldTypes) throws IOException {
- final IndexReaderContext ctx = searcher.getTopReaderContext();
-
- final ShardSearcher[] subSearchers;
- if (ctx instanceof LeafReaderContext) {
- subSearchers = new ShardSearcher[1];
- subSearchers[0] = new ShardSearcher((LeafReaderContext) ctx, ctx);
- } else {
- final CompositeReaderContext compCTX = (CompositeReaderContext) ctx;
- final int size = compCTX.leaves().size();
- subSearchers = new ShardSearcher[size];
- for(int searcherIDX=0;searcherIDX<subSearchers.length;searcherIDX++) {
- final LeafReaderContext leave = compCTX.leaves().get(searcherIDX);
- subSearchers[searcherIDX] = new ShardSearcher(leave, compCTX);
- }
- }
-
- List<InternalAggregation> aggs = new ArrayList<> ();
- Query rewritten = searcher.rewrite(query);
- Weight weight = searcher.createWeight(rewritten, true, 1f);
- C root = createAggregator(builder, searcher, fieldTypes);
- try {
- for (ShardSearcher subSearcher : subSearchers) {
- C a = createAggregator(builder, subSearcher, fieldTypes);
- a.preCollection();
- subSearcher.search(weight, a);
- a.postCollection();
- aggs.add(a.buildAggregation(0L));
- }
- if (aggs.isEmpty()) {
- return null;
- } else {
- if (randomBoolean() && aggs.size() > 1) {
- // sometimes do an incremental reduce
- int toReduceSize = aggs.size();
- Collections.shuffle(aggs, random());
- int r = randomIntBetween(1, toReduceSize);
- List<InternalAggregation> toReduce = aggs.subList(0, r);
- A reduced = (A) aggs.get(0).doReduce(toReduce,
- new InternalAggregation.ReduceContext(root.context().bigArrays(), null, false));
- aggs = new ArrayList<>(aggs.subList(r, toReduceSize));
- aggs.add(reduced);
- }
- // now do the final reduce
- @SuppressWarnings("unchecked")
- A internalAgg = (A) aggs.get(0).doReduce(aggs, new InternalAggregation.ReduceContext(root.context().bigArrays(), null,
- true));
- return internalAgg;
- }
- } finally {
- Releasables.close(releasables);
- releasables.clear();
- }
- }
-
- private static class ShardSearcher extends IndexSearcher {
- private final List<LeafReaderContext> ctx;
-
- ShardSearcher(LeafReaderContext ctx, IndexReaderContext parent) {
- super(parent);
- this.ctx = Collections.singletonList(ctx);
- }
-
- public void search(Weight weight, Collector collector) throws IOException {
- search(ctx, weight, collector);
- }
-
- @Override
- public String toString() {
- return "ShardSearcher(" + ctx.get(0) + ")";
- }
- }
-
- protected static DirectoryReader wrap(DirectoryReader directoryReader) throws IOException {
- return ElasticsearchDirectoryReader.wrap(directoryReader, new ShardId(new Index("_index", "_na_"), 0));
- }
-}
diff --git a/core/src/test/java/org/elasticsearch/search/aggregations/BaseAggregationTestCase.java b/core/src/test/java/org/elasticsearch/search/aggregations/BaseAggregationTestCase.java
deleted file mode 100644
index c76d1a5f0d..0000000000
--- a/core/src/test/java/org/elasticsearch/search/aggregations/BaseAggregationTestCase.java
+++ /dev/null
@@ -1,198 +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.search.aggregations;
-
-import org.elasticsearch.common.io.stream.BytesStreamOutput;
-import org.elasticsearch.common.io.stream.NamedWriteableAwareStreamInput;
-import org.elasticsearch.common.io.stream.NamedWriteableRegistry;
-import org.elasticsearch.common.io.stream.StreamInput;
-import org.elasticsearch.common.settings.Settings;
-import org.elasticsearch.common.xcontent.NamedXContentRegistry;
-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.XContentType;
-import org.elasticsearch.env.Environment;
-import org.elasticsearch.index.query.QueryParseContext;
-import org.elasticsearch.indices.IndicesModule;
-import org.elasticsearch.search.SearchModule;
-import org.elasticsearch.search.aggregations.support.ValuesSourceAggregationBuilder;
-import org.elasticsearch.test.AbstractQueryTestCase;
-import org.elasticsearch.test.ESTestCase;
-
-import java.io.IOException;
-import java.util.ArrayList;
-import java.util.Collections;
-import java.util.List;
-
-import static java.util.Collections.emptyList;
-import static org.elasticsearch.test.EqualsHashCodeTestUtils.checkEqualsAndHashCode;
-import static org.hamcrest.Matchers.hasSize;
-
-public abstract class BaseAggregationTestCase<AB extends AbstractAggregationBuilder<AB>> extends ESTestCase {
-
- protected static final String STRING_FIELD_NAME = "mapped_string";
- protected static final String INT_FIELD_NAME = "mapped_int";
- protected static final String DOUBLE_FIELD_NAME = "mapped_double";
- protected static final String BOOLEAN_FIELD_NAME = "mapped_boolean";
- protected static final String DATE_FIELD_NAME = "mapped_date";
- protected static final String IP_FIELD_NAME = "mapped_ip";
-
- private String[] currentTypes;
-
- protected String[] getCurrentTypes() {
- return currentTypes;
- }
-
- private NamedWriteableRegistry namedWriteableRegistry;
- private NamedXContentRegistry xContentRegistry;
- protected abstract AB createTestAggregatorBuilder();
-
- /**
- * Setup for the whole base test class.
- */
- @Override
- public void setUp() throws Exception {
- super.setUp();
- Settings settings = Settings.builder()
- .put("node.name", AbstractQueryTestCase.class.toString())
- .put(Environment.PATH_HOME_SETTING.getKey(), createTempDir())
- .build();
- IndicesModule indicesModule = new IndicesModule(Collections.emptyList());
- SearchModule searchModule = new SearchModule(settings, false, emptyList());
- List<NamedWriteableRegistry.Entry> entries = new ArrayList<>();
- entries.addAll(indicesModule.getNamedWriteables());
- entries.addAll(searchModule.getNamedWriteables());
- namedWriteableRegistry = new NamedWriteableRegistry(entries);
- xContentRegistry = new NamedXContentRegistry(searchModule.getNamedXContents());
- //create some random type with some default field, those types will stick around for all of the subclasses
- currentTypes = new String[randomIntBetween(0, 5)];
- for (int i = 0; i < currentTypes.length; i++) {
- String type = randomAlphaOfLengthBetween(1, 10);
- currentTypes[i] = type;
- }
- }
-
- @Override
- protected NamedXContentRegistry xContentRegistry() {
- return xContentRegistry;
- }
-
- /**
- * Generic test that creates new AggregatorFactory from the test
- * AggregatorFactory and checks both for equality and asserts equality on
- * the two queries.
- */
- public void testFromXContent() throws IOException {
- AB testAgg = createTestAggregatorBuilder();
- AggregatorFactories.Builder factoriesBuilder = AggregatorFactories.builder().addAggregator(testAgg);
- XContentBuilder builder = XContentFactory.contentBuilder(randomFrom(XContentType.values()));
- if (randomBoolean()) {
- builder.prettyPrint();
- }
- factoriesBuilder.toXContent(builder, ToXContent.EMPTY_PARAMS);
- XContentBuilder shuffled = shuffleXContent(builder);
- XContentParser parser = createParser(shuffled);
- AggregationBuilder newAgg = parse(parser);
- assertNotSame(newAgg, testAgg);
- assertEquals(testAgg, newAgg);
- assertEquals(testAgg.hashCode(), newAgg.hashCode());
- }
-
- protected AggregationBuilder parse(XContentParser parser) throws IOException {
- QueryParseContext parseContext = new QueryParseContext(parser);
- assertSame(XContentParser.Token.START_OBJECT, parser.nextToken());
- AggregatorFactories.Builder parsed = AggregatorFactories.parseAggregators(parseContext);
- assertThat(parsed.getAggregatorFactories(), hasSize(1));
- assertThat(parsed.getPipelineAggregatorFactories(), hasSize(0));
- AggregationBuilder newAgg = parsed.getAggregatorFactories().get(0);
- assertNull(parser.nextToken());
- assertNotNull(newAgg);
- return newAgg;
- }
-
- /**
- * Test serialization and deserialization of the test AggregatorFactory.
- */
- public void testSerialization() throws IOException {
- AB testAgg = createTestAggregatorBuilder();
- try (BytesStreamOutput output = new BytesStreamOutput()) {
- output.writeNamedWriteable(testAgg);
- try (StreamInput in = new NamedWriteableAwareStreamInput(output.bytes().streamInput(), namedWriteableRegistry)) {
- AggregationBuilder deserialized = in.readNamedWriteable(AggregationBuilder.class);
- assertEquals(testAgg, deserialized);
- assertEquals(testAgg.hashCode(), deserialized.hashCode());
- assertNotSame(testAgg, deserialized);
- }
- }
- }
-
-
- public void testEqualsAndHashcode() throws IOException {
- // TODO we only change name and boost, we should extend by any sub-test supplying a "mutate" method that randomly changes one
- // aspect of the object under test
- checkEqualsAndHashCode(createTestAggregatorBuilder(), this::copyAggregation);
- }
-
- // we use the streaming infra to create a copy of the query provided as
- // argument
- private AB copyAggregation(AB agg) throws IOException {
- try (BytesStreamOutput output = new BytesStreamOutput()) {
- agg.writeTo(output);
- try (StreamInput in = new NamedWriteableAwareStreamInput(output.bytes().streamInput(), namedWriteableRegistry)) {
- @SuppressWarnings("unchecked")
- AB secondAgg = (AB) namedWriteableRegistry.getReader(AggregationBuilder.class, agg.getWriteableName()).read(in);
- return secondAgg;
- }
- }
- }
-
- public String randomNumericField() {
- int randomInt = randomInt(3);
- switch (randomInt) {
- case 0:
- return DATE_FIELD_NAME;
- case 1:
- return DOUBLE_FIELD_NAME;
- case 2:
- default:
- return INT_FIELD_NAME;
- }
- }
-
- protected void randomFieldOrScript(ValuesSourceAggregationBuilder<?, ?> factory, String field) {
- int choice = randomInt(2);
- switch (choice) {
- case 0:
- factory.field(field);
- break;
- case 1:
- factory.field(field);
- factory.script(mockScript("_value + 1"));
- break;
- case 2:
- factory.script(mockScript("doc[" + field + "] + 1"));
- break;
- default:
- throw new AssertionError("Unknow random operation [" + choice + "]");
- }
- }
-}
diff --git a/core/src/test/java/org/elasticsearch/search/aggregations/bucket/ChildrenIT.java b/core/src/test/java/org/elasticsearch/search/aggregations/bucket/ChildrenIT.java
deleted file mode 100644
index bfe483ca89..0000000000
--- a/core/src/test/java/org/elasticsearch/search/aggregations/bucket/ChildrenIT.java
+++ /dev/null
@@ -1,472 +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.search.aggregations.bucket;
-
-import org.apache.lucene.search.join.ScoreMode;
-import org.elasticsearch.action.index.IndexRequestBuilder;
-import org.elasticsearch.action.search.SearchResponse;
-import org.elasticsearch.action.update.UpdateResponse;
-import org.elasticsearch.client.Requests;
-import org.elasticsearch.cluster.metadata.IndexMetaData;
-import org.elasticsearch.common.settings.Settings;
-import org.elasticsearch.common.xcontent.XContentType;
-import org.elasticsearch.search.SearchHit;
-import org.elasticsearch.search.aggregations.AggregationBuilders;
-import org.elasticsearch.search.aggregations.InternalAggregation;
-import org.elasticsearch.search.aggregations.bucket.children.Children;
-import org.elasticsearch.search.aggregations.bucket.terms.Terms;
-import org.elasticsearch.search.aggregations.metrics.sum.Sum;
-import org.elasticsearch.search.aggregations.metrics.tophits.TopHits;
-import org.elasticsearch.search.sort.SortOrder;
-import org.elasticsearch.test.ESIntegTestCase;
-
-import java.util.ArrayList;
-import java.util.HashMap;
-import java.util.HashSet;
-import java.util.List;
-import java.util.Map;
-import java.util.Set;
-
-import static org.elasticsearch.index.query.QueryBuilders.hasChildQuery;
-import static org.elasticsearch.index.query.QueryBuilders.matchQuery;
-import static org.elasticsearch.index.query.QueryBuilders.termQuery;
-import static org.elasticsearch.search.aggregations.AggregationBuilders.children;
-import static org.elasticsearch.search.aggregations.AggregationBuilders.sum;
-import static org.elasticsearch.search.aggregations.AggregationBuilders.terms;
-import static org.elasticsearch.search.aggregations.AggregationBuilders.topHits;
-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.assertSearchResponse;
-import static org.hamcrest.Matchers.equalTo;
-import static org.hamcrest.Matchers.greaterThan;
-import static org.hamcrest.Matchers.is;
-import static org.hamcrest.Matchers.sameInstance;
-
-@ESIntegTestCase.SuiteScopeTestCase
-public class ChildrenIT extends ESIntegTestCase {
-
- private static final Map<String, Control> categoryToControl = new HashMap<>();
-
- @Override
- public void setupSuiteScopeCluster() throws Exception {
- assertAcked(
- prepareCreate("test")
- .setSettings("index.mapping.single_type", false)
- .addMapping("article", "category", "type=keyword")
- .addMapping("comment", "_parent", "type=article", "commenter", "type=keyword")
- );
-
- List<IndexRequestBuilder> requests = new ArrayList<>();
- String[] uniqueCategories = new String[randomIntBetween(1, 25)];
- for (int i = 0; i < uniqueCategories.length; i++) {
- uniqueCategories[i] = Integer.toString(i);
- }
- int catIndex = 0;
-
- int numParentDocs = randomIntBetween(uniqueCategories.length, uniqueCategories.length * 5);
- for (int i = 0; i < numParentDocs; i++) {
- String id = Integer.toString(i);
-
- // TODO: this array is always of length 1, and testChildrenAggs fails if this is changed
- String[] categories = new String[randomIntBetween(1,1)];
- for (int j = 0; j < categories.length; j++) {
- String category = categories[j] = uniqueCategories[catIndex++ % uniqueCategories.length];
- Control control = categoryToControl.get(category);
- if (control == null) {
- categoryToControl.put(category, control = new Control(category));
- }
- control.articleIds.add(id);
- }
-
- requests.add(client().prepareIndex("test", "article", id).setCreate(true).setSource("category", categories, "randomized", true));
- }
-
- String[] commenters = new String[randomIntBetween(5, 50)];
- for (int i = 0; i < commenters.length; i++) {
- commenters[i] = Integer.toString(i);
- }
-
- int id = 0;
- for (Control control : categoryToControl.values()) {
- for (String articleId : control.articleIds) {
- int numChildDocsPerParent = randomIntBetween(0, 5);
- for (int i = 0; i < numChildDocsPerParent; i++) {
- String commenter = commenters[id % commenters.length];
- String idValue = Integer.toString(id++);
- control.commentIds.add(idValue);
- Set<String> ids = control.commenterToCommentId.get(commenter);
- if (ids == null) {
- control.commenterToCommentId.put(commenter, ids = new HashSet<>());
- }
- ids.add(idValue);
- requests.add(client().prepareIndex("test", "comment", idValue).setCreate(true).setParent(articleId).setSource("commenter", commenter));
- }
- }
- }
-
- requests.add(client().prepareIndex("test", "article", "a").setSource("category", new String[]{"a"}, "randomized", false));
- requests.add(client().prepareIndex("test", "article", "b").setSource("category", new String[]{"a", "b"}, "randomized", false));
- requests.add(client().prepareIndex("test", "article", "c").setSource("category", new String[]{"a", "b", "c"}, "randomized", false));
- requests.add(client().prepareIndex("test", "article", "d").setSource("category", new String[]{"c"}, "randomized", false));
- requests.add(client().prepareIndex("test", "comment", "a").setParent("a").setSource("{}", XContentType.JSON));
- requests.add(client().prepareIndex("test", "comment", "c").setParent("c").setSource("{}", XContentType.JSON));
-
- indexRandom(true, requests);
- ensureSearchable("test");
- }
-
- public void testChildrenAggs() throws Exception {
- SearchResponse searchResponse = client().prepareSearch("test")
- .setQuery(matchQuery("randomized", true))
- .addAggregation(
- terms("category").field("category").size(10000).subAggregation(children("to_comment", "comment")
- .subAggregation(
- terms("commenters").field("commenter").size(10000).subAggregation(
- topHits("top_comments")
- ))
- )
- ).get();
- assertSearchResponse(searchResponse);
-
- Terms categoryTerms = searchResponse.getAggregations().get("category");
- assertThat(categoryTerms.getBuckets().size(), equalTo(categoryToControl.size()));
- for (Map.Entry<String, Control> entry1 : categoryToControl.entrySet()) {
- Terms.Bucket categoryBucket = categoryTerms.getBucketByKey(entry1.getKey());
- assertThat(categoryBucket.getKeyAsString(), equalTo(entry1.getKey()));
- assertThat(categoryBucket.getDocCount(), equalTo((long) entry1.getValue().articleIds.size()));
-
- Children childrenBucket = categoryBucket.getAggregations().get("to_comment");
- assertThat(childrenBucket.getName(), equalTo("to_comment"));
- assertThat(childrenBucket.getDocCount(), equalTo((long) entry1.getValue().commentIds.size()));
- assertThat((long) ((InternalAggregation)childrenBucket).getProperty("_count"), equalTo((long) entry1.getValue().commentIds.size()));
-
- Terms commentersTerms = childrenBucket.getAggregations().get("commenters");
- assertThat((Terms) ((InternalAggregation)childrenBucket).getProperty("commenters"), sameInstance(commentersTerms));
- assertThat(commentersTerms.getBuckets().size(), equalTo(entry1.getValue().commenterToCommentId.size()));
- for (Map.Entry<String, Set<String>> entry2 : entry1.getValue().commenterToCommentId.entrySet()) {
- Terms.Bucket commentBucket = commentersTerms.getBucketByKey(entry2.getKey());
- assertThat(commentBucket.getKeyAsString(), equalTo(entry2.getKey()));
- assertThat(commentBucket.getDocCount(), equalTo((long) entry2.getValue().size()));
-
- TopHits topHits = commentBucket.getAggregations().get("top_comments");
- for (SearchHit searchHit : topHits.getHits().getHits()) {
- assertThat(entry2.getValue().contains(searchHit.getId()), is(true));
- }
- }
- }
- }
-
- public void testParentWithMultipleBuckets() throws Exception {
- SearchResponse searchResponse = client().prepareSearch("test")
- .setQuery(matchQuery("randomized", false))
- .addAggregation(
- terms("category").field("category").size(10000).subAggregation(
- children("to_comment", "comment").subAggregation(topHits("top_comments").sort("_uid", SortOrder.ASC))
- )
- ).get();
- assertSearchResponse(searchResponse);
-
- Terms categoryTerms = searchResponse.getAggregations().get("category");
- assertThat(categoryTerms.getBuckets().size(), equalTo(3));
-
- for (Terms.Bucket bucket : categoryTerms.getBuckets()) {
- logger.info("bucket={}", bucket.getKey());
- Children childrenBucket = bucket.getAggregations().get("to_comment");
- TopHits topHits = childrenBucket.getAggregations().get("top_comments");
- logger.info("total_hits={}", topHits.getHits().getTotalHits());
- for (SearchHit searchHit : topHits.getHits()) {
- logger.info("hit= {} {} {}", searchHit.getSortValues()[0], searchHit.getType(), searchHit.getId());
- }
- }
-
- Terms.Bucket categoryBucket = categoryTerms.getBucketByKey("a");
- assertThat(categoryBucket.getKeyAsString(), equalTo("a"));
- assertThat(categoryBucket.getDocCount(), equalTo(3L));
-
- Children childrenBucket = categoryBucket.getAggregations().get("to_comment");
- assertThat(childrenBucket.getName(), equalTo("to_comment"));
- assertThat(childrenBucket.getDocCount(), equalTo(2L));
- TopHits topHits = childrenBucket.getAggregations().get("top_comments");
- assertThat(topHits.getHits().getTotalHits(), equalTo(2L));
- assertThat(topHits.getHits().getAt(0).getId(), equalTo("a"));
- assertThat(topHits.getHits().getAt(0).getType(), equalTo("comment"));
- assertThat(topHits.getHits().getAt(1).getId(), equalTo("c"));
- assertThat(topHits.getHits().getAt(1).getType(), equalTo("comment"));
-
- categoryBucket = categoryTerms.getBucketByKey("b");
- assertThat(categoryBucket.getKeyAsString(), equalTo("b"));
- assertThat(categoryBucket.getDocCount(), equalTo(2L));
-
- childrenBucket = categoryBucket.getAggregations().get("to_comment");
- assertThat(childrenBucket.getName(), equalTo("to_comment"));
- assertThat(childrenBucket.getDocCount(), equalTo(1L));
- topHits = childrenBucket.getAggregations().get("top_comments");
- assertThat(topHits.getHits().getTotalHits(), equalTo(1L));
- assertThat(topHits.getHits().getAt(0).getId(), equalTo("c"));
- assertThat(topHits.getHits().getAt(0).getType(), equalTo("comment"));
-
- categoryBucket = categoryTerms.getBucketByKey("c");
- assertThat(categoryBucket.getKeyAsString(), equalTo("c"));
- assertThat(categoryBucket.getDocCount(), equalTo(2L));
-
- childrenBucket = categoryBucket.getAggregations().get("to_comment");
- assertThat(childrenBucket.getName(), equalTo("to_comment"));
- assertThat(childrenBucket.getDocCount(), equalTo(1L));
- topHits = childrenBucket.getAggregations().get("top_comments");
- assertThat(topHits.getHits().getTotalHits(), equalTo(1L));
- assertThat(topHits.getHits().getAt(0).getId(), equalTo("c"));
- assertThat(topHits.getHits().getAt(0).getType(), equalTo("comment"));
- }
-
- public void testWithDeletes() throws Exception {
- String indexName = "xyz";
- assertAcked(
- prepareCreate(indexName)
- .setSettings("index.mapping.single_type", false)
- .addMapping("parent")
- .addMapping("child", "_parent", "type=parent", "count", "type=long")
- );
-
- List<IndexRequestBuilder> requests = new ArrayList<>();
- requests.add(client().prepareIndex(indexName, "parent", "1").setSource("{}", XContentType.JSON));
- requests.add(client().prepareIndex(indexName, "child", "0").setParent("1").setSource("count", 1));
- requests.add(client().prepareIndex(indexName, "child", "1").setParent("1").setSource("count", 1));
- requests.add(client().prepareIndex(indexName, "child", "2").setParent("1").setSource("count", 1));
- requests.add(client().prepareIndex(indexName, "child", "3").setParent("1").setSource("count", 1));
- indexRandom(true, requests);
-
- for (int i = 0; i < 10; i++) {
- SearchResponse searchResponse = client().prepareSearch(indexName)
- .addAggregation(children("children", "child").subAggregation(sum("counts").field("count")))
- .get();
-
- assertNoFailures(searchResponse);
- Children children = searchResponse.getAggregations().get("children");
- assertThat(children.getDocCount(), equalTo(4L));
-
- Sum count = children.getAggregations().get("counts");
- assertThat(count.getValue(), equalTo(4.));
-
- String idToUpdate = Integer.toString(randomInt(3));
- /*
- * The whole point of this test is to test these things with deleted
- * docs in the index so we turn off detect_noop to make sure that
- * the updates cause that.
- */
- UpdateResponse updateResponse = client().prepareUpdate(indexName, "child", idToUpdate)
- .setParent("1")
- .setDoc(Requests.INDEX_CONTENT_TYPE, "count", 1)
- .setDetectNoop(false)
- .get();
- assertThat(updateResponse.getVersion(), greaterThan(1L));
- refresh();
- }
- }
-
- public void testNonExistingChildType() throws Exception {
- SearchResponse searchResponse = client().prepareSearch("test")
- .addAggregation(
-children("non-existing", "xyz")
- ).get();
- assertSearchResponse(searchResponse);
-
- Children children = searchResponse.getAggregations().get("non-existing");
- assertThat(children.getName(), equalTo("non-existing"));
- assertThat(children.getDocCount(), equalTo(0L));
- }
-
- public void testPostCollection() throws Exception {
- String indexName = "prodcatalog";
- String masterType = "masterprod";
- String childType = "variantsku";
- assertAcked(
- prepareCreate(indexName)
- .setSettings("index.mapping.single_type", false)
- .addMapping(masterType, "brand", "type=text", "name", "type=keyword", "material", "type=text")
- .addMapping(childType, "_parent", "type=masterprod", "color", "type=keyword", "size", "type=keyword")
- );
-
- List<IndexRequestBuilder> requests = new ArrayList<>();
- requests.add(client().prepareIndex(indexName, masterType, "1").setSource("brand", "Levis", "name", "Style 501", "material", "Denim"));
- requests.add(client().prepareIndex(indexName, childType, "0").setParent("1").setSource("color", "blue", "size", "32"));
- requests.add(client().prepareIndex(indexName, childType, "1").setParent("1").setSource("color", "blue", "size", "34"));
- requests.add(client().prepareIndex(indexName, childType, "2").setParent("1").setSource("color", "blue", "size", "36"));
- requests.add(client().prepareIndex(indexName, childType, "3").setParent("1").setSource("color", "black", "size", "38"));
- requests.add(client().prepareIndex(indexName, childType, "4").setParent("1").setSource("color", "black", "size", "40"));
- requests.add(client().prepareIndex(indexName, childType, "5").setParent("1").setSource("color", "gray", "size", "36"));
-
- requests.add(client().prepareIndex(indexName, masterType, "2").setSource("brand", "Wrangler", "name", "Regular Cut", "material", "Leather"));
- requests.add(client().prepareIndex(indexName, childType, "6").setParent("2").setSource("color", "blue", "size", "32"));
- requests.add(client().prepareIndex(indexName, childType, "7").setParent("2").setSource("color", "blue", "size", "34"));
- requests.add(client().prepareIndex(indexName, childType, "8").setParent("2").setSource("color", "black", "size", "36"));
- requests.add(client().prepareIndex(indexName, childType, "9").setParent("2").setSource("color", "black", "size", "38"));
- requests.add(client().prepareIndex(indexName, childType, "10").setParent("2").setSource("color", "black", "size", "40"));
- requests.add(client().prepareIndex(indexName, childType, "11").setParent("2").setSource("color", "orange", "size", "36"));
- requests.add(client().prepareIndex(indexName, childType, "12").setParent("2").setSource("color", "green", "size", "44"));
- indexRandom(true, requests);
-
- SearchResponse response = client().prepareSearch(indexName).setTypes(masterType)
- .setQuery(hasChildQuery(childType, termQuery("color", "orange"), ScoreMode.None))
-.addAggregation(children("my-refinements", childType)
- .subAggregation(terms("my-colors").field("color"))
- .subAggregation(terms("my-sizes").field("size"))
- ).get();
- assertNoFailures(response);
- assertHitCount(response, 1);
-
- Children childrenAgg = response.getAggregations().get("my-refinements");
- assertThat(childrenAgg.getDocCount(), equalTo(7L));
-
- Terms termsAgg = childrenAgg.getAggregations().get("my-colors");
- assertThat(termsAgg.getBuckets().size(), equalTo(4));
- assertThat(termsAgg.getBucketByKey("black").getDocCount(), equalTo(3L));
- assertThat(termsAgg.getBucketByKey("blue").getDocCount(), equalTo(2L));
- assertThat(termsAgg.getBucketByKey("green").getDocCount(), equalTo(1L));
- assertThat(termsAgg.getBucketByKey("orange").getDocCount(), equalTo(1L));
-
- termsAgg = childrenAgg.getAggregations().get("my-sizes");
- assertThat(termsAgg.getBuckets().size(), equalTo(6));
- assertThat(termsAgg.getBucketByKey("36").getDocCount(), equalTo(2L));
- assertThat(termsAgg.getBucketByKey("32").getDocCount(), equalTo(1L));
- assertThat(termsAgg.getBucketByKey("34").getDocCount(), equalTo(1L));
- assertThat(termsAgg.getBucketByKey("38").getDocCount(), equalTo(1L));
- assertThat(termsAgg.getBucketByKey("40").getDocCount(), equalTo(1L));
- assertThat(termsAgg.getBucketByKey("44").getDocCount(), equalTo(1L));
- }
-
- public void testHierarchicalChildrenAggs() {
- String indexName = "geo";
- String grandParentType = "continent";
- String parentType = "country";
- String childType = "city";
- assertAcked(
- prepareCreate(indexName)
- .setSettings(Settings.builder()
- .put(IndexMetaData.SETTING_NUMBER_OF_SHARDS, 1)
- .put(IndexMetaData.SETTING_NUMBER_OF_REPLICAS, 0)
- )
- .setSettings("index.mapping.single_type", false)
- .addMapping(grandParentType, "name", "type=keyword")
- .addMapping(parentType, "_parent", "type=" + grandParentType)
- .addMapping(childType, "_parent", "type=" + parentType)
- );
-
- client().prepareIndex(indexName, grandParentType, "1").setSource("name", "europe").get();
- client().prepareIndex(indexName, parentType, "2").setParent("1").setSource("name", "belgium").get();
- client().prepareIndex(indexName, childType, "3").setParent("2").setRouting("1").setSource("name", "brussels").get();
- refresh();
-
- SearchResponse response = client().prepareSearch(indexName)
- .setQuery(matchQuery("name", "europe"))
- .addAggregation(
- children(parentType, parentType).subAggregation(children(childType, childType).subAggregation(
- terms("name").field("name")
- )
- )
- )
- .get();
- assertNoFailures(response);
- assertHitCount(response, 1);
-
- Children children = response.getAggregations().get(parentType);
- assertThat(children.getName(), equalTo(parentType));
- assertThat(children.getDocCount(), equalTo(1L));
- children = children.getAggregations().get(childType);
- assertThat(children.getName(), equalTo(childType));
- assertThat(children.getDocCount(), equalTo(1L));
- Terms terms = children.getAggregations().get("name");
- assertThat(terms.getBuckets().size(), equalTo(1));
- assertThat(terms.getBuckets().get(0).getKey().toString(), equalTo("brussels"));
- assertThat(terms.getBuckets().get(0).getDocCount(), equalTo(1L));
- }
-
- public void testPostCollectAllLeafReaders() throws Exception {
- // The 'towns' and 'parent_names' aggs operate on parent docs and if child docs are in different segments we need
- // to ensure those segments which child docs are also evaluated to in the post collect phase.
-
- // Before we only evaluated segments that yielded matches in 'towns' and 'parent_names' aggs, which caused
- // us to miss to evaluate child docs in segments we didn't have parent matches for.
-
- assertAcked(
- prepareCreate("index")
- .setSettings("index.mapping.single_type", false)
- .addMapping("parentType", "name", "type=keyword", "town", "type=keyword")
- .addMapping("childType", "_parent", "type=parentType", "name", "type=keyword", "age", "type=integer")
- );
- List<IndexRequestBuilder> requests = new ArrayList<>();
- requests.add(client().prepareIndex("index", "parentType", "1").setSource("name", "Bob", "town", "Memphis"));
- requests.add(client().prepareIndex("index", "parentType", "2").setSource("name", "Alice", "town", "Chicago"));
- requests.add(client().prepareIndex("index", "parentType", "3").setSource("name", "Bill", "town", "Chicago"));
- requests.add(client().prepareIndex("index", "childType", "1").setSource("name", "Jill", "age", 5).setParent("1"));
- requests.add(client().prepareIndex("index", "childType", "2").setSource("name", "Joey", "age", 3).setParent("1"));
- requests.add(client().prepareIndex("index", "childType", "3").setSource("name", "John", "age", 2).setParent("2"));
- requests.add(client().prepareIndex("index", "childType", "4").setSource("name", "Betty", "age", 6).setParent("3"));
- requests.add(client().prepareIndex("index", "childType", "5").setSource("name", "Dan", "age", 1).setParent("3"));
- indexRandom(true, requests);
-
- SearchResponse response = client().prepareSearch("index")
- .setSize(0)
- .addAggregation(AggregationBuilders.terms("towns").field("town")
- .subAggregation(AggregationBuilders.terms("parent_names").field("name")
-.subAggregation(AggregationBuilders.children("child_docs", "childType"))
- )
- )
- .get();
-
- Terms towns = response.getAggregations().get("towns");
- assertThat(towns.getBuckets().size(), equalTo(2));
- assertThat(towns.getBuckets().get(0).getKeyAsString(), equalTo("Chicago"));
- assertThat(towns.getBuckets().get(0).getDocCount(), equalTo(2L));
-
- Terms parents = towns.getBuckets().get(0).getAggregations().get("parent_names");
- assertThat(parents.getBuckets().size(), equalTo(2));
- assertThat(parents.getBuckets().get(0).getKeyAsString(), equalTo("Alice"));
- assertThat(parents.getBuckets().get(0).getDocCount(), equalTo(1L));
- Children children = parents.getBuckets().get(0).getAggregations().get("child_docs");
- assertThat(children.getDocCount(), equalTo(1L));
-
- assertThat(parents.getBuckets().get(1).getKeyAsString(), equalTo("Bill"));
- assertThat(parents.getBuckets().get(1).getDocCount(), equalTo(1L));
- children = parents.getBuckets().get(1).getAggregations().get("child_docs");
- assertThat(children.getDocCount(), equalTo(2L));
-
- assertThat(towns.getBuckets().get(1).getKeyAsString(), equalTo("Memphis"));
- assertThat(towns.getBuckets().get(1).getDocCount(), equalTo(1L));
- parents = towns.getBuckets().get(1).getAggregations().get("parent_names");
- assertThat(parents.getBuckets().size(), equalTo(1));
- assertThat(parents.getBuckets().get(0).getKeyAsString(), equalTo("Bob"));
- assertThat(parents.getBuckets().get(0).getDocCount(), equalTo(1L));
- children = parents.getBuckets().get(0).getAggregations().get("child_docs");
- assertThat(children.getDocCount(), equalTo(2L));
- }
-
- private static final class Control {
-
- final String category;
- final Set<String> articleIds = new HashSet<>();
- final Set<String> commentIds = new HashSet<>();
- final Map<String, Set<String>> commenterToCommentId = new HashMap<>();
-
- private Control(String category) {
- this.category = category;
- }
- }
-
-}
diff --git a/core/src/test/java/org/elasticsearch/search/aggregations/bucket/ChildrenTests.java b/core/src/test/java/org/elasticsearch/search/aggregations/bucket/ChildrenTests.java
deleted file mode 100644
index 4098e85c62..0000000000
--- a/core/src/test/java/org/elasticsearch/search/aggregations/bucket/ChildrenTests.java
+++ /dev/null
@@ -1,35 +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.search.aggregations.bucket;
-
-import org.elasticsearch.search.aggregations.BaseAggregationTestCase;
-import org.elasticsearch.search.aggregations.bucket.children.ChildrenAggregationBuilder;
-
-public class ChildrenTests extends BaseAggregationTestCase<ChildrenAggregationBuilder> {
-
- @Override
- protected ChildrenAggregationBuilder createTestAggregatorBuilder() {
- String name = randomAlphaOfLengthBetween(3, 20);
- String childType = randomAlphaOfLengthBetween(5, 40);
- ChildrenAggregationBuilder factory = new ChildrenAggregationBuilder(name, childType);
- return factory;
- }
-
-}
diff --git a/core/src/test/java/org/elasticsearch/search/aggregations/bucket/InternalSingleBucketAggregationTestCase.java b/core/src/test/java/org/elasticsearch/search/aggregations/bucket/InternalSingleBucketAggregationTestCase.java
deleted file mode 100644
index cae34768ec..0000000000
--- a/core/src/test/java/org/elasticsearch/search/aggregations/bucket/InternalSingleBucketAggregationTestCase.java
+++ /dev/null
@@ -1,139 +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.search.aggregations.bucket;
-
-import org.elasticsearch.common.bytes.BytesReference;
-import org.elasticsearch.common.xcontent.ToXContent;
-import org.elasticsearch.common.xcontent.XContentType;
-import org.elasticsearch.rest.action.search.RestSearchAction;
-import org.elasticsearch.search.aggregations.Aggregation;
-import org.elasticsearch.search.aggregations.InternalAggregation;
-import org.elasticsearch.search.aggregations.InternalAggregations;
-import org.elasticsearch.search.aggregations.ParsedAggregation;
-import org.elasticsearch.search.aggregations.metrics.max.InternalMax;
-import org.elasticsearch.search.aggregations.metrics.min.InternalMin;
-import org.elasticsearch.search.aggregations.pipeline.PipelineAggregator;
-import org.elasticsearch.test.InternalAggregationTestCase;
-
-import java.io.IOException;
-import java.util.ArrayList;
-import java.util.HashMap;
-import java.util.List;
-import java.util.Map;
-import java.util.function.Supplier;
-
-import static java.util.Collections.emptyList;
-import static java.util.Collections.emptyMap;
-import static java.util.Collections.singletonMap;
-import static org.elasticsearch.common.xcontent.XContentHelper.toXContent;
-import static org.elasticsearch.test.hamcrest.ElasticsearchAssertions.assertToXContentEquivalent;
-
-public abstract class InternalSingleBucketAggregationTestCase<T extends InternalSingleBucketAggregation>
- extends InternalAggregationTestCase<T> {
-
- private boolean hasInternalMax;
- private boolean hasInternalMin;
-
- public Supplier<InternalAggregations> subAggregationsSupplier;
-
- @Override
- public void setUp() throws Exception {
- super.setUp();
- hasInternalMax = randomBoolean();
- hasInternalMin = randomBoolean();
- subAggregationsSupplier = () -> {
- List<InternalAggregation> aggs = new ArrayList<>();
- if (hasInternalMax) {
- aggs.add(new InternalMax("max", randomDouble(), randomNumericDocValueFormat(), emptyList(), emptyMap()));
- }
- if (hasInternalMin) {
- aggs.add(new InternalMin("min", randomDouble(), randomNumericDocValueFormat(), emptyList(), emptyMap()));
- }
- return new InternalAggregations(aggs);
- };
- }
-
- protected abstract T createTestInstance(String name, long docCount, InternalAggregations aggregations,
- List<PipelineAggregator> pipelineAggregators, Map<String, Object> metaData);
- protected abstract void extraAssertReduced(T reduced, List<T> inputs);
-
- @Override
- protected final T createTestInstance(String name, List<PipelineAggregator> pipelineAggregators, Map<String, Object> metaData) {
- // we shouldn't use the full long range here since we sum doc count on reduce, and don't want to overflow the long range there
- long docCount = between(0, Integer.MAX_VALUE);
- return createTestInstance(name, docCount, subAggregationsSupplier.get(), pipelineAggregators, metaData);
- }
-
- @Override
- protected final void assertReduced(T reduced, List<T> inputs) {
- assertEquals(inputs.stream().mapToLong(InternalSingleBucketAggregation::getDocCount).sum(), reduced.getDocCount());
- if (hasInternalMax) {
- double expected = inputs.stream().mapToDouble(i -> {
- InternalMax max = i.getAggregations().get("max");
- return max.getValue();
- }).max().getAsDouble();
- InternalMax reducedMax = reduced.getAggregations().get("max");
- assertEquals(expected, reducedMax.getValue(), 0);
- }
- if (hasInternalMin) {
- double expected = inputs.stream().mapToDouble(i -> {
- InternalMin min = i.getAggregations().get("min");
- return min.getValue();
- }).min().getAsDouble();
- InternalMin reducedMin = reduced.getAggregations().get("min");
- assertEquals(expected, reducedMin.getValue(), 0);
- }
- extraAssertReduced(reduced, inputs);
- }
-
- @Override
- protected void assertFromXContent(T aggregation, ParsedAggregation parsedAggregation) throws IOException {
- assertTrue(parsedAggregation instanceof ParsedSingleBucketAggregation);
- ParsedSingleBucketAggregation parsed = (ParsedSingleBucketAggregation) parsedAggregation;
-
- assertEquals(aggregation.getDocCount(), parsed.getDocCount());
- InternalAggregations aggregations = aggregation.getAggregations();
- Map<String, Aggregation> expectedAggregations = new HashMap<>();
- int expectedNumberOfAggregations = 0;
- for (Aggregation expectedAggregation : aggregations) {
- // since we shuffle xContent, we cannot rely on the order of the original inner aggregations for comparison
- assertTrue(expectedAggregation instanceof InternalAggregation);
- expectedAggregations.put(expectedAggregation.getName(), expectedAggregation);
- expectedNumberOfAggregations++;
- }
- int parsedNumberOfAggregations = 0;
- for (Aggregation parsedAgg : parsed.getAggregations()) {
- assertTrue(parsedAgg instanceof ParsedAggregation);
- assertTrue(expectedAggregations.keySet().contains(parsedAgg.getName()));
- Aggregation expectedInternalAggregation = expectedAggregations.get(parsedAgg.getName());
- final XContentType xContentType = randomFrom(XContentType.values());
- final ToXContent.Params params = new ToXContent.MapParams(singletonMap(RestSearchAction.TYPED_KEYS_PARAM, "true"));
- BytesReference expectedBytes = toXContent(expectedInternalAggregation, xContentType, params, false);
- BytesReference actualBytes = toXContent(parsedAgg, xContentType, params, false);
- assertToXContentEquivalent(expectedBytes, actualBytes, xContentType);
- parsedNumberOfAggregations++;
- }
- assertEquals(expectedNumberOfAggregations, parsedNumberOfAggregations);
- Class<? extends ParsedSingleBucketAggregation> parsedClass = implementationClass();
- assertTrue(parsedClass != null && parsedClass.isInstance(parsedAggregation));
- }
-
- protected abstract Class<? extends ParsedSingleBucketAggregation> implementationClass();
-}
diff --git a/core/src/test/java/org/elasticsearch/search/aggregations/bucket/children/InternalChildrenTests.java b/core/src/test/java/org/elasticsearch/search/aggregations/bucket/children/InternalChildrenTests.java
deleted file mode 100644
index 285837c6e4..0000000000
--- a/core/src/test/java/org/elasticsearch/search/aggregations/bucket/children/InternalChildrenTests.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.search.aggregations.bucket.children;
-
-import org.elasticsearch.common.io.stream.Writeable.Reader;
-import org.elasticsearch.search.aggregations.InternalAggregations;
-import org.elasticsearch.search.aggregations.bucket.InternalSingleBucketAggregationTestCase;
-import org.elasticsearch.search.aggregations.bucket.ParsedSingleBucketAggregation;
-import org.elasticsearch.search.aggregations.pipeline.PipelineAggregator;
-
-import java.util.List;
-import java.util.Map;
-
-public class InternalChildrenTests extends InternalSingleBucketAggregationTestCase<InternalChildren> {
- @Override
- protected InternalChildren createTestInstance(String name, long docCount, InternalAggregations aggregations,
- List<PipelineAggregator> pipelineAggregators, Map<String, Object> metaData) {
- return new InternalChildren(name, docCount, aggregations, pipelineAggregators, metaData);
- }
-
- @Override
- protected void extraAssertReduced(InternalChildren reduced, List<InternalChildren> inputs) {
- // Nothing extra to assert
- }
-
- @Override
- protected Reader<InternalChildren> instanceReader() {
- return InternalChildren::new;
- }
-
- @Override
- protected Class<? extends ParsedSingleBucketAggregation> implementationClass() {
- return ParsedChildren.class;
- }
-}
diff --git a/core/src/test/java/org/elasticsearch/search/aggregations/bucket/children/ParentToChildrenAggregatorTests.java b/core/src/test/java/org/elasticsearch/search/aggregations/bucket/children/ParentToChildrenAggregatorTests.java
deleted file mode 100644
index 17152bc450..0000000000
--- a/core/src/test/java/org/elasticsearch/search/aggregations/bucket/children/ParentToChildrenAggregatorTests.java
+++ /dev/null
@@ -1,190 +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.search.aggregations.bucket.children;
-
-import org.apache.lucene.document.Field;
-import org.apache.lucene.document.SortedDocValuesField;
-import org.apache.lucene.document.SortedNumericDocValuesField;
-import org.apache.lucene.document.StringField;
-import org.apache.lucene.index.DirectoryReader;
-import org.apache.lucene.index.IndexReader;
-import org.apache.lucene.index.RandomIndexWriter;
-import org.apache.lucene.search.IndexSearcher;
-import org.apache.lucene.search.MatchAllDocsQuery;
-import org.apache.lucene.search.Query;
-import org.apache.lucene.search.TermInSetQuery;
-import org.apache.lucene.store.Directory;
-import org.apache.lucene.util.BytesRef;
-import org.elasticsearch.Version;
-import org.elasticsearch.cluster.metadata.IndexMetaData;
-import org.elasticsearch.common.collect.Tuple;
-import org.elasticsearch.common.lucene.index.ElasticsearchDirectoryReader;
-import org.elasticsearch.common.settings.Settings;
-import org.elasticsearch.index.Index;
-import org.elasticsearch.index.mapper.ContentPath;
-import org.elasticsearch.index.mapper.DocumentMapper;
-import org.elasticsearch.index.mapper.MappedFieldType;
-import org.elasticsearch.index.mapper.Mapper;
-import org.elasticsearch.index.mapper.MapperService;
-import org.elasticsearch.index.mapper.NumberFieldMapper;
-import org.elasticsearch.index.mapper.ParentFieldMapper;
-import org.elasticsearch.index.mapper.TypeFieldMapper;
-import org.elasticsearch.index.mapper.Uid;
-import org.elasticsearch.index.mapper.UidFieldMapper;
-import org.elasticsearch.index.shard.ShardId;
-import org.elasticsearch.search.aggregations.AggregatorTestCase;
-import org.elasticsearch.search.aggregations.metrics.min.InternalMin;
-import org.elasticsearch.search.aggregations.metrics.min.MinAggregationBuilder;
-import org.mockito.Mockito;
-
-import java.io.IOException;
-import java.util.Arrays;
-import java.util.HashMap;
-import java.util.List;
-import java.util.Map;
-import java.util.function.Consumer;
-
-import static org.mockito.Mockito.mock;
-import static org.mockito.Mockito.when;
-
-public class ParentToChildrenAggregatorTests extends AggregatorTestCase {
-
- private static final String CHILD_TYPE = "child_type";
- private static final String PARENT_TYPE = "parent_type";
-
- public void testNoDocs() throws IOException {
- Directory directory = newDirectory();
-
- RandomIndexWriter indexWriter = new RandomIndexWriter(random(), directory);
- // intentionally not writing any docs
- indexWriter.close();
- IndexReader indexReader = DirectoryReader.open(directory);
-
- testCase(new MatchAllDocsQuery(), newSearcher(indexReader, false, true), parentToChild -> {
- assertEquals(0, parentToChild.getDocCount());
- assertEquals(Double.POSITIVE_INFINITY, ((InternalMin) parentToChild.getAggregations().get("in_child")).getValue(),
- Double.MIN_VALUE);
- });
- indexReader.close();
- directory.close();
- }
-
- public void testParentChild() throws IOException {
- Directory directory = newDirectory();
- RandomIndexWriter indexWriter = new RandomIndexWriter(random(), directory);
-
- final Map<String, Tuple<Integer, Integer>> expectedParentChildRelations = setupIndex(indexWriter);
- indexWriter.close();
-
- IndexReader indexReader = ElasticsearchDirectoryReader.wrap(DirectoryReader.open(directory),
- new ShardId(new Index("foo", "_na_"), 1));
- // TODO set "maybeWrap" to true for IndexSearcher once #23338 is resolved
- IndexSearcher indexSearcher = newSearcher(indexReader, false, true);
-
- testCase(new MatchAllDocsQuery(), indexSearcher, child -> {
- int expectedTotalChildren = 0;
- int expectedMinValue = Integer.MAX_VALUE;
- for (Tuple<Integer, Integer> expectedValues : expectedParentChildRelations.values()) {
- expectedTotalChildren += expectedValues.v1();
- expectedMinValue = Math.min(expectedMinValue, expectedValues.v2());
- }
- assertEquals(expectedTotalChildren, child.getDocCount());
- assertEquals(expectedMinValue, ((InternalMin) child.getAggregations().get("in_child")).getValue(), Double.MIN_VALUE);
- });
-
- for (String parent : expectedParentChildRelations.keySet()) {
- testCase(new TermInSetQuery(UidFieldMapper.NAME, new BytesRef(Uid.createUid(PARENT_TYPE, parent))), indexSearcher, child -> {
- assertEquals((long) expectedParentChildRelations.get(parent).v1(), child.getDocCount());
- assertEquals(expectedParentChildRelations.get(parent).v2(),
- ((InternalMin) child.getAggregations().get("in_child")).getValue(), Double.MIN_VALUE);
- });
- }
- indexReader.close();
- directory.close();
- }
-
- private static Map<String, Tuple<Integer, Integer>> setupIndex(RandomIndexWriter iw) throws IOException {
- Map<String, Tuple<Integer, Integer>> expectedValues = new HashMap<>();
- int numParents = randomIntBetween(1, 10);
- for (int i = 0; i < numParents; i++) {
- String parent = "parent" + i;
- iw.addDocument(createParentDocument(parent));
- int numChildren = randomIntBetween(1, 10);
- int minValue = Integer.MAX_VALUE;
- for (int c = 0; c < numChildren; c++) {
- int randomValue = randomIntBetween(0, 100);
- minValue = Math.min(minValue, randomValue);
- iw.addDocument(createChildDocument("child" + c + "_" + parent, parent, randomValue));
- }
- expectedValues.put(parent, new Tuple<>(numChildren, minValue));
- }
- return expectedValues;
- }
-
- private static List<Field> createParentDocument(String id) {
- return Arrays.asList(new StringField(TypeFieldMapper.NAME, PARENT_TYPE, Field.Store.NO),
- new StringField(UidFieldMapper.NAME, Uid.createUid(PARENT_TYPE, id), Field.Store.NO),
- createJoinField(PARENT_TYPE, id));
- }
-
- private static List<Field> createChildDocument(String childId, String parentId, int value) {
- return Arrays.asList(new StringField(TypeFieldMapper.NAME, CHILD_TYPE, Field.Store.NO),
- new StringField(UidFieldMapper.NAME, Uid.createUid(CHILD_TYPE, childId), Field.Store.NO),
- new SortedNumericDocValuesField("number", value),
- createJoinField(PARENT_TYPE, parentId));
- }
-
- private static SortedDocValuesField createJoinField(String parentType, String id) {
- return new SortedDocValuesField(ParentFieldMapper.joinField(parentType), new BytesRef(id));
- }
-
- @Override
- protected MapperService mapperServiceMock() {
- MapperService mapperService = mock(MapperService.class);
- DocumentMapper childDocMapper = mock(DocumentMapper.class);
- DocumentMapper parentDocMapper = mock(DocumentMapper.class);
- ParentFieldMapper parentFieldMapper = createParentFieldMapper();
- when(childDocMapper.parentFieldMapper()).thenReturn(parentFieldMapper);
- when(parentDocMapper.parentFieldMapper()).thenReturn(parentFieldMapper);
- when(mapperService.documentMapper(CHILD_TYPE)).thenReturn(childDocMapper);
- when(mapperService.documentMapper(PARENT_TYPE)).thenReturn(parentDocMapper);
- when(mapperService.docMappers(false)).thenReturn(Arrays.asList(new DocumentMapper[] { childDocMapper, parentDocMapper }));
- when(parentDocMapper.typeFilter(Mockito.any())).thenReturn(new TypeFieldMapper.TypesQuery(new BytesRef(PARENT_TYPE)));
- when(childDocMapper.typeFilter(Mockito.any())).thenReturn(new TypeFieldMapper.TypesQuery(new BytesRef(CHILD_TYPE)));
- return mapperService;
- }
-
- private static ParentFieldMapper createParentFieldMapper() {
- Settings settings = Settings.builder().put(IndexMetaData.SETTING_VERSION_CREATED, Version.CURRENT).build();
- return new ParentFieldMapper.Builder("parent").type(PARENT_TYPE).build(new Mapper.BuilderContext(settings, new ContentPath(0)));
- }
-
- private void testCase(Query query, IndexSearcher indexSearcher, Consumer<InternalChildren> verify)
- throws IOException {
-
- ChildrenAggregationBuilder aggregationBuilder = new ChildrenAggregationBuilder("_name", CHILD_TYPE);
- aggregationBuilder.subAggregation(new MinAggregationBuilder("in_child").field("number"));
-
- MappedFieldType fieldType = new NumberFieldMapper.NumberFieldType(NumberFieldMapper.NumberType.LONG);
- fieldType.setName("number");
- InternalChildren result = search(indexSearcher, query, aggregationBuilder, fieldType);
- verify.accept(result);
- }
-}
diff --git a/core/src/test/java/org/elasticsearch/search/aggregations/bucket/filter/InternalFilterTests.java b/core/src/test/java/org/elasticsearch/search/aggregations/bucket/filter/InternalFilterTests.java
index 8f888e13af..d3c092ab65 100644
--- a/core/src/test/java/org/elasticsearch/search/aggregations/bucket/filter/InternalFilterTests.java
+++ b/core/src/test/java/org/elasticsearch/search/aggregations/bucket/filter/InternalFilterTests.java
@@ -21,7 +21,7 @@ package org.elasticsearch.search.aggregations.bucket.filter;
import org.elasticsearch.common.io.stream.Writeable.Reader;
import org.elasticsearch.search.aggregations.InternalAggregations;
-import org.elasticsearch.search.aggregations.bucket.InternalSingleBucketAggregationTestCase;
+import org.elasticsearch.search.aggregations.InternalSingleBucketAggregationTestCase;
import org.elasticsearch.search.aggregations.bucket.ParsedSingleBucketAggregation;
import org.elasticsearch.search.aggregations.pipeline.PipelineAggregator;
diff --git a/core/src/test/java/org/elasticsearch/search/aggregations/bucket/global/InternalGlobalTests.java b/core/src/test/java/org/elasticsearch/search/aggregations/bucket/global/InternalGlobalTests.java
index 392f88b4d4..cb11574f86 100644
--- a/core/src/test/java/org/elasticsearch/search/aggregations/bucket/global/InternalGlobalTests.java
+++ b/core/src/test/java/org/elasticsearch/search/aggregations/bucket/global/InternalGlobalTests.java
@@ -21,7 +21,7 @@ package org.elasticsearch.search.aggregations.bucket.global;
import org.elasticsearch.common.io.stream.Writeable.Reader;
import org.elasticsearch.search.aggregations.InternalAggregations;
-import org.elasticsearch.search.aggregations.bucket.InternalSingleBucketAggregationTestCase;
+import org.elasticsearch.search.aggregations.InternalSingleBucketAggregationTestCase;
import org.elasticsearch.search.aggregations.bucket.ParsedSingleBucketAggregation;
import org.elasticsearch.search.aggregations.pipeline.PipelineAggregator;
diff --git a/core/src/test/java/org/elasticsearch/search/aggregations/bucket/missing/InternalMissingTests.java b/core/src/test/java/org/elasticsearch/search/aggregations/bucket/missing/InternalMissingTests.java
index 75a28e87ce..71df2af079 100644
--- a/core/src/test/java/org/elasticsearch/search/aggregations/bucket/missing/InternalMissingTests.java
+++ b/core/src/test/java/org/elasticsearch/search/aggregations/bucket/missing/InternalMissingTests.java
@@ -21,7 +21,7 @@ package org.elasticsearch.search.aggregations.bucket.missing;
import org.elasticsearch.common.io.stream.Writeable.Reader;
import org.elasticsearch.search.aggregations.InternalAggregations;
-import org.elasticsearch.search.aggregations.bucket.InternalSingleBucketAggregationTestCase;
+import org.elasticsearch.search.aggregations.InternalSingleBucketAggregationTestCase;
import org.elasticsearch.search.aggregations.bucket.ParsedSingleBucketAggregation;
import org.elasticsearch.search.aggregations.pipeline.PipelineAggregator;
diff --git a/core/src/test/java/org/elasticsearch/search/aggregations/bucket/nested/InternalNestedTests.java b/core/src/test/java/org/elasticsearch/search/aggregations/bucket/nested/InternalNestedTests.java
index f6299ebf7b..a43e0dd519 100644
--- a/core/src/test/java/org/elasticsearch/search/aggregations/bucket/nested/InternalNestedTests.java
+++ b/core/src/test/java/org/elasticsearch/search/aggregations/bucket/nested/InternalNestedTests.java
@@ -21,7 +21,7 @@ package org.elasticsearch.search.aggregations.bucket.nested;
import org.elasticsearch.common.io.stream.Writeable.Reader;
import org.elasticsearch.search.aggregations.InternalAggregations;
-import org.elasticsearch.search.aggregations.bucket.InternalSingleBucketAggregationTestCase;
+import org.elasticsearch.search.aggregations.InternalSingleBucketAggregationTestCase;
import org.elasticsearch.search.aggregations.bucket.ParsedSingleBucketAggregation;
import org.elasticsearch.search.aggregations.pipeline.PipelineAggregator;
diff --git a/core/src/test/java/org/elasticsearch/search/aggregations/bucket/nested/InternalReverseNestedTests.java b/core/src/test/java/org/elasticsearch/search/aggregations/bucket/nested/InternalReverseNestedTests.java
index 08940fcd3a..ddfe10b27e 100644
--- a/core/src/test/java/org/elasticsearch/search/aggregations/bucket/nested/InternalReverseNestedTests.java
+++ b/core/src/test/java/org/elasticsearch/search/aggregations/bucket/nested/InternalReverseNestedTests.java
@@ -21,7 +21,7 @@ package org.elasticsearch.search.aggregations.bucket.nested;
import org.elasticsearch.common.io.stream.Writeable.Reader;
import org.elasticsearch.search.aggregations.InternalAggregations;
-import org.elasticsearch.search.aggregations.bucket.InternalSingleBucketAggregationTestCase;
+import org.elasticsearch.search.aggregations.InternalSingleBucketAggregationTestCase;
import org.elasticsearch.search.aggregations.bucket.ParsedSingleBucketAggregation;
import org.elasticsearch.search.aggregations.pipeline.PipelineAggregator;
diff --git a/core/src/test/java/org/elasticsearch/search/aggregations/bucket/sampler/InternalSamplerTests.java b/core/src/test/java/org/elasticsearch/search/aggregations/bucket/sampler/InternalSamplerTests.java
index 0631908092..f6288a9712 100644
--- a/core/src/test/java/org/elasticsearch/search/aggregations/bucket/sampler/InternalSamplerTests.java
+++ b/core/src/test/java/org/elasticsearch/search/aggregations/bucket/sampler/InternalSamplerTests.java
@@ -20,7 +20,7 @@ package org.elasticsearch.search.aggregations.bucket.sampler;
import org.elasticsearch.common.io.stream.Writeable;
import org.elasticsearch.search.aggregations.InternalAggregations;
-import org.elasticsearch.search.aggregations.bucket.InternalSingleBucketAggregationTestCase;
+import org.elasticsearch.search.aggregations.InternalSingleBucketAggregationTestCase;
import org.elasticsearch.search.aggregations.bucket.ParsedSingleBucketAggregation;
import org.elasticsearch.search.aggregations.pipeline.PipelineAggregator;
@@ -48,4 +48,4 @@ public class InternalSamplerTests extends InternalSingleBucketAggregationTestCas
protected Class<? extends ParsedSingleBucketAggregation> implementationClass() {
return ParsedSampler.class;
}
-} \ No newline at end of file
+}
diff --git a/core/src/test/java/org/elasticsearch/search/child/ChildQuerySearchIT.java b/core/src/test/java/org/elasticsearch/search/child/ChildQuerySearchIT.java
deleted file mode 100644
index 697352c5ed..0000000000
--- a/core/src/test/java/org/elasticsearch/search/child/ChildQuerySearchIT.java
+++ /dev/null
@@ -1,1991 +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.search.child;
-
-import org.apache.lucene.search.join.ScoreMode;
-import org.elasticsearch.action.admin.indices.mapping.get.GetMappingsResponse;
-import org.elasticsearch.action.admin.indices.mapping.put.PutMappingResponse;
-import org.elasticsearch.action.explain.ExplainResponse;
-import org.elasticsearch.action.index.IndexRequestBuilder;
-import org.elasticsearch.action.search.SearchPhaseExecutionException;
-import org.elasticsearch.action.search.SearchResponse;
-import org.elasticsearch.action.search.SearchType;
-import org.elasticsearch.action.support.WriteRequest.RefreshPolicy;
-import org.elasticsearch.common.lucene.search.function.CombineFunction;
-import org.elasticsearch.common.lucene.search.function.FiltersFunctionScoreQuery;
-import org.elasticsearch.common.settings.Settings;
-import org.elasticsearch.common.unit.TimeValue;
-import org.elasticsearch.common.xcontent.XContentType;
-import org.elasticsearch.index.IndexModule;
-import org.elasticsearch.index.query.BoolQueryBuilder;
-import org.elasticsearch.index.query.HasChildQueryBuilder;
-import org.elasticsearch.index.query.HasParentQueryBuilder;
-import org.elasticsearch.index.query.IdsQueryBuilder;
-import org.elasticsearch.index.query.InnerHitBuilder;
-import org.elasticsearch.index.query.MatchAllQueryBuilder;
-import org.elasticsearch.index.query.MatchQueryBuilder;
-import org.elasticsearch.index.query.QueryBuilder;
-import org.elasticsearch.index.query.QueryBuilders;
-import org.elasticsearch.index.query.functionscore.FunctionScoreQueryBuilder;
-import org.elasticsearch.rest.RestStatus;
-import org.elasticsearch.search.SearchHit;
-import org.elasticsearch.search.aggregations.AggregationBuilders;
-import org.elasticsearch.search.aggregations.bucket.filter.Filter;
-import org.elasticsearch.search.aggregations.bucket.global.Global;
-import org.elasticsearch.search.aggregations.bucket.terms.Terms;
-import org.elasticsearch.search.builder.SearchSourceBuilder;
-import org.elasticsearch.search.fetch.subphase.highlight.HighlightBuilder;
-import org.elasticsearch.search.fetch.subphase.highlight.HighlightBuilder.Field;
-import org.elasticsearch.search.fetch.subphase.highlight.HighlightField;
-import org.elasticsearch.search.sort.SortBuilders;
-import org.elasticsearch.search.sort.SortOrder;
-import org.elasticsearch.test.ESIntegTestCase;
-import org.elasticsearch.test.ESIntegTestCase.ClusterScope;
-import org.elasticsearch.test.ESIntegTestCase.Scope;
-import org.hamcrest.Matchers;
-
-import java.io.IOException;
-import java.util.ArrayList;
-import java.util.HashMap;
-import java.util.HashSet;
-import java.util.List;
-import java.util.Locale;
-import java.util.Map;
-import java.util.Set;
-
-import static org.elasticsearch.common.xcontent.XContentFactory.jsonBuilder;
-import static org.elasticsearch.index.query.QueryBuilders.boolQuery;
-import static org.elasticsearch.index.query.QueryBuilders.constantScoreQuery;
-import static org.elasticsearch.index.query.QueryBuilders.hasChildQuery;
-import static org.elasticsearch.index.query.QueryBuilders.hasParentQuery;
-import static org.elasticsearch.index.query.QueryBuilders.idsQuery;
-import static org.elasticsearch.index.query.QueryBuilders.matchAllQuery;
-import static org.elasticsearch.index.query.QueryBuilders.matchQuery;
-import static org.elasticsearch.index.query.QueryBuilders.multiMatchQuery;
-import static org.elasticsearch.index.query.QueryBuilders.parentId;
-import static org.elasticsearch.index.query.QueryBuilders.prefixQuery;
-import static org.elasticsearch.index.query.QueryBuilders.queryStringQuery;
-import static org.elasticsearch.index.query.QueryBuilders.termQuery;
-import static org.elasticsearch.index.query.QueryBuilders.termsQuery;
-import static org.elasticsearch.index.query.functionscore.ScoreFunctionBuilders.fieldValueFactorFunction;
-import static org.elasticsearch.index.query.functionscore.ScoreFunctionBuilders.weightFactorFunction;
-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.assertSearchHit;
-import static org.elasticsearch.test.hamcrest.ElasticsearchAssertions.assertSearchHits;
-import static org.elasticsearch.test.hamcrest.ElasticsearchAssertions.hasId;
-import static org.hamcrest.Matchers.anyOf;
-import static org.hamcrest.Matchers.containsString;
-import static org.hamcrest.Matchers.equalTo;
-import static org.hamcrest.Matchers.greaterThanOrEqualTo;
-import static org.hamcrest.Matchers.is;
-import static org.hamcrest.Matchers.notNullValue;
-
-@ClusterScope(scope = Scope.SUITE)
-public class ChildQuerySearchIT extends ESIntegTestCase {
-
- @Override
- public Settings indexSettings() {
- return Settings.builder().put(super.indexSettings())
- // aggressive filter caching so that we can assert on the filter cache size
- .put(IndexModule.INDEX_QUERY_CACHE_ENABLED_SETTING.getKey(), true)
- .put(IndexModule.INDEX_QUERY_CACHE_EVERYTHING_SETTING.getKey(), true)
- .build();
- }
-
- public void testSelfReferentialIsForbidden() {
- IllegalArgumentException e = expectThrows(IllegalArgumentException.class, () ->
- prepareCreate("test").addMapping("type", "_parent", "type=type").get());
- assertThat(e.getMessage(), equalTo("The [_parent.type] option can't point to the same type"));
- }
-
- public void testMultiLevelChild() throws Exception {
- assertAcked(prepareCreate("test")
- .setSettings("index.mapping.single_type", false)
- .addMapping("parent")
- .addMapping("child", "_parent", "type=parent")
- .addMapping("grandchild", "_parent", "type=child"));
- ensureGreen();
-
- client().prepareIndex("test", "parent", "p1").setSource("p_field", "p_value1").get();
- client().prepareIndex("test", "child", "c1").setSource("c_field", "c_value1").setParent("p1").get();
- client().prepareIndex("test", "grandchild", "gc1").setSource("gc_field", "gc_value1")
- .setParent("c1").setRouting("p1").get();
- refresh();
-
- SearchResponse searchResponse = client()
- .prepareSearch("test")
- .setQuery(
- boolQuery()
- .must(matchAllQuery())
- .filter(hasChildQuery(
- "child",
- boolQuery().must(termQuery("c_field", "c_value1"))
- .filter(hasChildQuery("grandchild", termQuery("gc_field", "gc_value1"), ScoreMode.None))
- , ScoreMode.None))).get();
- assertNoFailures(searchResponse);
- assertThat(searchResponse.getHits().getTotalHits(), equalTo(1L));
- assertThat(searchResponse.getHits().getAt(0).getId(), equalTo("p1"));
-
- searchResponse = client().prepareSearch("test")
- .setQuery(boolQuery().must(matchAllQuery()).filter(hasParentQuery("parent", termQuery("p_field", "p_value1"), false))).execute()
- .actionGet();
- assertNoFailures(searchResponse);
- assertThat(searchResponse.getHits().getTotalHits(), equalTo(1L));
- assertThat(searchResponse.getHits().getAt(0).getId(), equalTo("c1"));
-
- searchResponse = client().prepareSearch("test")
- .setQuery(boolQuery().must(matchAllQuery()).filter(hasParentQuery("child", termQuery("c_field", "c_value1"), false))).execute()
- .actionGet();
- assertNoFailures(searchResponse);
- assertThat(searchResponse.getHits().getTotalHits(), equalTo(1L));
- assertThat(searchResponse.getHits().getAt(0).getId(), equalTo("gc1"));
-
- searchResponse = client().prepareSearch("test").setQuery(hasParentQuery("parent", termQuery("p_field", "p_value1"), false)).execute()
- .actionGet();
- assertNoFailures(searchResponse);
- assertThat(searchResponse.getHits().getTotalHits(), equalTo(1L));
- assertThat(searchResponse.getHits().getAt(0).getId(), equalTo("c1"));
-
- searchResponse = client().prepareSearch("test").setQuery(hasParentQuery("child", termQuery("c_field", "c_value1"), false)).execute()
- .actionGet();
- assertNoFailures(searchResponse);
- assertThat(searchResponse.getHits().getTotalHits(), equalTo(1L));
- assertThat(searchResponse.getHits().getAt(0).getId(), equalTo("gc1"));
- }
-
- // see #2744
- public void test2744() throws IOException {
- assertAcked(prepareCreate("test")
- .setSettings("index.mapping.single_type", false)
- .addMapping("foo")
- .addMapping("test", "_parent", "type=foo"));
- ensureGreen();
-
- // index simple data
- client().prepareIndex("test", "foo", "1").setSource("foo", 1).get();
- client().prepareIndex("test", "test").setSource("foo", 1).setParent("1").get();
- refresh();
- SearchResponse searchResponse = client().prepareSearch("test").
- setQuery(hasChildQuery("test", matchQuery("foo", 1), ScoreMode.None))
- .get();
- assertNoFailures(searchResponse);
- assertThat(searchResponse.getHits().getTotalHits(), equalTo(1L));
- assertThat(searchResponse.getHits().getAt(0).getId(), equalTo("1"));
-
- }
-
- public void testSimpleChildQuery() throws Exception {
- assertAcked(prepareCreate("test")
- .setSettings("index.mapping.single_type", false)
- .addMapping("parent")
- .addMapping("child", "_parent", "type=parent"));
- ensureGreen();
-
- // index simple data
- client().prepareIndex("test", "parent", "p1").setSource("p_field", "p_value1").get();
- client().prepareIndex("test", "child", "c1").setSource("c_field", "red").setParent("p1").get();
- client().prepareIndex("test", "child", "c2").setSource("c_field", "yellow").setParent("p1").get();
- client().prepareIndex("test", "parent", "p2").setSource("p_field", "p_value2").get();
- client().prepareIndex("test", "child", "c3").setSource("c_field", "blue").setParent("p2").get();
- client().prepareIndex("test", "child", "c4").setSource("c_field", "red").setParent("p2").get();
- refresh();
-
- // TEST FETCHING _parent from child
- SearchResponse searchResponse = client().prepareSearch("test").setQuery(idsQuery("child").addIds("c1")).storedFields("_parent").execute()
- .actionGet();
- assertNoFailures(searchResponse);
- assertThat(searchResponse.getHits().getTotalHits(), equalTo(1L));
- assertThat(searchResponse.getHits().getAt(0).getId(), equalTo("c1"));
- assertThat(searchResponse.getHits().getAt(0).field("_parent").getValue().toString(), equalTo("p1"));
-
- // TEST matching on parent
- searchResponse = client().prepareSearch("test").setQuery(termQuery("_parent#parent", "p1")).storedFields("_parent").get();
- assertNoFailures(searchResponse);
- assertThat(searchResponse.getHits().getTotalHits(), equalTo(2L));
- assertThat(searchResponse.getHits().getAt(0).getId(), anyOf(equalTo("c1"), equalTo("c2")));
- assertThat(searchResponse.getHits().getAt(0).field("_parent").getValue().toString(), equalTo("p1"));
- assertThat(searchResponse.getHits().getAt(1).getId(), anyOf(equalTo("c1"), equalTo("c2")));
- assertThat(searchResponse.getHits().getAt(1).field("_parent").getValue().toString(), equalTo("p1"));
-
- searchResponse = client().prepareSearch("test").setQuery(queryStringQuery("_parent#parent:p1")).storedFields("_parent").get();
- assertNoFailures(searchResponse);
- assertThat(searchResponse.getHits().getTotalHits(), equalTo(2L));
- assertThat(searchResponse.getHits().getAt(0).getId(), anyOf(equalTo("c1"), equalTo("c2")));
- assertThat(searchResponse.getHits().getAt(0).field("_parent").getValue().toString(), equalTo("p1"));
- assertThat(searchResponse.getHits().getAt(1).getId(), anyOf(equalTo("c1"), equalTo("c2")));
- assertThat(searchResponse.getHits().getAt(1).field("_parent").getValue().toString(), equalTo("p1"));
-
- // HAS CHILD
- searchResponse = client().prepareSearch("test").setQuery(randomHasChild("child", "c_field", "yellow"))
- .get();
- assertHitCount(searchResponse, 1L);
- assertThat(searchResponse.getHits().getTotalHits(), equalTo(1L));
- assertThat(searchResponse.getHits().getAt(0).getId(), equalTo("p1"));
-
- searchResponse = client().prepareSearch("test").setQuery(randomHasChild("child", "c_field", "blue")).execute()
- .actionGet();
- assertHitCount(searchResponse, 1L);
- assertThat(searchResponse.getHits().getAt(0).getId(), equalTo("p2"));
-
- searchResponse = client().prepareSearch("test").setQuery(randomHasChild("child", "c_field", "red")).get();
- assertHitCount(searchResponse, 2L);
- assertThat(searchResponse.getHits().getAt(0).getId(), anyOf(equalTo("p2"), equalTo("p1")));
- assertThat(searchResponse.getHits().getAt(1).getId(), anyOf(equalTo("p2"), equalTo("p1")));
-
- // HAS PARENT
- searchResponse = client().prepareSearch("test")
- .setQuery(randomHasParent("parent", "p_field", "p_value2")).get();
- assertNoFailures(searchResponse);
- assertHitCount(searchResponse, 2L);
- assertThat(searchResponse.getHits().getAt(0).getId(), equalTo("c3"));
- assertThat(searchResponse.getHits().getAt(1).getId(), equalTo("c4"));
-
- searchResponse = client().prepareSearch("test")
- .setQuery(randomHasParent("parent", "p_field", "p_value1")).get();
- assertHitCount(searchResponse, 2L);
- assertThat(searchResponse.getHits().getAt(0).getId(), equalTo("c1"));
- assertThat(searchResponse.getHits().getAt(1).getId(), equalTo("c2"));
- }
-
- // Issue #3290
- public void testCachingBugWithFqueryFilter() throws Exception {
- assertAcked(prepareCreate("test")
- .setSettings("index.mapping.single_type", false)
- .addMapping("parent")
- .addMapping("child", "_parent", "type=parent"));
- ensureGreen();
- List<IndexRequestBuilder> builders = new ArrayList<>();
- // index simple data
- for (int i = 0; i < 10; i++) {
- builders.add(client().prepareIndex("test", "parent", Integer.toString(i)).setSource("p_field", i));
- }
- indexRandom(randomBoolean(), builders);
- builders.clear();
- for (int j = 0; j < 2; j++) {
- for (int i = 0; i < 10; i++) {
- builders.add(client().prepareIndex("test", "child", Integer.toString(i)).setSource("c_field", i).setParent("" + 0));
- }
- for (int i = 0; i < 10; i++) {
- builders.add(client().prepareIndex("test", "child", Integer.toString(i + 10)).setSource("c_field", i + 10).setParent(Integer.toString(i)));
- }
-
- if (randomBoolean()) {
- break; // randomly break out and dont' have deletes / updates
- }
- }
- indexRandom(true, builders);
-
- for (int i = 1; i <= 10; i++) {
- logger.info("Round {}", i);
- SearchResponse searchResponse = client().prepareSearch("test")
- .setQuery(constantScoreQuery(hasChildQuery("child", matchAllQuery(), ScoreMode.Max)))
- .get();
- assertNoFailures(searchResponse);
- searchResponse = client().prepareSearch("test")
- .setQuery(constantScoreQuery(hasParentQuery("parent", matchAllQuery(), true)))
- .get();
- assertNoFailures(searchResponse);
- }
- }
-
- public void testHasParentFilter() throws Exception {
- assertAcked(prepareCreate("test")
- .setSettings("index.mapping.single_type", false)
- .addMapping("parent")
- .addMapping("child", "_parent", "type=parent"));
- ensureGreen();
- Map<String, Set<String>> parentToChildren = new HashMap<>();
- // Childless parent
- client().prepareIndex("test", "parent", "p0").setSource("p_field", "p0").get();
- parentToChildren.put("p0", new HashSet<>());
-
- String previousParentId = null;
- int numChildDocs = 32;
- int numChildDocsPerParent = 0;
- List<IndexRequestBuilder> builders = new ArrayList<>();
- for (int i = 1; i <= numChildDocs; i++) {
-
- if (previousParentId == null || i % numChildDocsPerParent == 0) {
- previousParentId = "p" + i;
- builders.add(client().prepareIndex("test", "parent", previousParentId).setSource("p_field", previousParentId));
- numChildDocsPerParent++;
- }
-
- String childId = "c" + i;
- builders.add(client().prepareIndex("test", "child", childId).setSource("c_field", childId).setParent(previousParentId));
-
- if (!parentToChildren.containsKey(previousParentId)) {
- parentToChildren.put(previousParentId, new HashSet<>());
- }
- assertThat(parentToChildren.get(previousParentId).add(childId), is(true));
- }
- indexRandom(true, builders.toArray(new IndexRequestBuilder[builders.size()]));
-
- assertThat(parentToChildren.isEmpty(), equalTo(false));
- for (Map.Entry<String, Set<String>> parentToChildrenEntry : parentToChildren.entrySet()) {
- SearchResponse searchResponse = client().prepareSearch("test")
- .setQuery(constantScoreQuery(hasParentQuery("parent", termQuery("p_field", parentToChildrenEntry.getKey()), false)))
- .setSize(numChildDocsPerParent).get();
-
- assertNoFailures(searchResponse);
- Set<String> childIds = parentToChildrenEntry.getValue();
- assertThat(searchResponse.getHits().getTotalHits(), equalTo((long) childIds.size()));
- for (int i = 0; i < searchResponse.getHits().getTotalHits(); i++) {
- assertThat(childIds.remove(searchResponse.getHits().getAt(i).getId()), is(true));
- assertThat(searchResponse.getHits().getAt(i).getScore(), is(1.0f));
- }
- assertThat(childIds.size(), is(0));
- }
- }
-
- public void testSimpleChildQueryWithFlush() throws Exception {
- assertAcked(prepareCreate("test")
- .setSettings("index.mapping.single_type", false)
- .addMapping("parent")
- .addMapping("child", "_parent", "type=parent"));
- ensureGreen();
-
- // index simple data with flushes, so we have many segments
- client().prepareIndex("test", "parent", "p1").setSource("p_field", "p_value1").get();
- client().admin().indices().prepareFlush().get();
- client().prepareIndex("test", "child", "c1").setSource("c_field", "red").setParent("p1").get();
- client().admin().indices().prepareFlush().get();
- client().prepareIndex("test", "child", "c2").setSource("c_field", "yellow").setParent("p1").get();
- client().admin().indices().prepareFlush().get();
- client().prepareIndex("test", "parent", "p2").setSource("p_field", "p_value2").get();
- client().admin().indices().prepareFlush().get();
- client().prepareIndex("test", "child", "c3").setSource("c_field", "blue").setParent("p2").get();
- client().admin().indices().prepareFlush().get();
- client().prepareIndex("test", "child", "c4").setSource("c_field", "red").setParent("p2").get();
- client().admin().indices().prepareFlush().get();
- refresh();
-
- // HAS CHILD QUERY
-
- SearchResponse searchResponse = client().prepareSearch("test")
- .setQuery(hasChildQuery("child", termQuery("c_field", "yellow"), ScoreMode.None))
- .get();
- assertNoFailures(searchResponse);
- assertThat(searchResponse.getHits().getTotalHits(), equalTo(1L));
- assertThat(searchResponse.getHits().getAt(0).getId(), equalTo("p1"));
-
- searchResponse = client().prepareSearch("test")
- .setQuery(hasChildQuery("child", termQuery("c_field", "blue"), ScoreMode.None))
- .get();
- assertNoFailures(searchResponse);
- assertThat(searchResponse.getHits().getTotalHits(), equalTo(1L));
- assertThat(searchResponse.getHits().getAt(0).getId(), equalTo("p2"));
-
- searchResponse = client().prepareSearch("test")
- .setQuery(hasChildQuery("child", termQuery("c_field", "red"), ScoreMode.None))
- .get();
- assertNoFailures(searchResponse);
- assertThat(searchResponse.getHits().getTotalHits(), equalTo(2L));
- assertThat(searchResponse.getHits().getAt(0).getId(), anyOf(equalTo("p2"), equalTo("p1")));
- assertThat(searchResponse.getHits().getAt(1).getId(), anyOf(equalTo("p2"), equalTo("p1")));
-
- // HAS CHILD FILTER
- searchResponse = client().prepareSearch("test")
- .setQuery(constantScoreQuery(hasChildQuery("child", termQuery("c_field", "yellow"), ScoreMode.None)))
- .get();
- assertNoFailures(searchResponse);
- assertThat(searchResponse.getHits().getTotalHits(), equalTo(1L));
- assertThat(searchResponse.getHits().getAt(0).getId(), equalTo("p1"));
-
- searchResponse = client().prepareSearch("test")
- .setQuery(constantScoreQuery(hasChildQuery("child", termQuery("c_field", "blue"), ScoreMode.None)))
- .get();
- assertNoFailures(searchResponse);
- assertThat(searchResponse.getHits().getTotalHits(), equalTo(1L));
- assertThat(searchResponse.getHits().getAt(0).getId(), equalTo("p2"));
-
- searchResponse = client().prepareSearch("test")
- .setQuery(constantScoreQuery(hasChildQuery("child", termQuery("c_field", "red"), ScoreMode.None)))
- .get();
- assertNoFailures(searchResponse);
- assertThat(searchResponse.getHits().getTotalHits(), equalTo(2L));
- assertThat(searchResponse.getHits().getAt(0).getId(), anyOf(equalTo("p2"), equalTo("p1")));
- assertThat(searchResponse.getHits().getAt(1).getId(), anyOf(equalTo("p2"), equalTo("p1")));
- }
-
- public void testScopedFacet() throws Exception {
- assertAcked(prepareCreate("test")
- .setSettings("index.mapping.single_type", false)
- .addMapping("parent")
- .addMapping("child", "_parent", "type=parent", "c_field", "type=keyword"));
- ensureGreen();
-
- // index simple data
- client().prepareIndex("test", "parent", "p1").setSource("p_field", "p_value1").get();
- client().prepareIndex("test", "child", "c1").setSource("c_field", "red").setParent("p1").get();
- client().prepareIndex("test", "child", "c2").setSource("c_field", "yellow").setParent("p1").get();
- client().prepareIndex("test", "parent", "p2").setSource("p_field", "p_value2").get();
- client().prepareIndex("test", "child", "c3").setSource("c_field", "blue").setParent("p2").get();
- client().prepareIndex("test", "child", "c4").setSource("c_field", "red").setParent("p2").get();
-
- refresh();
-
- SearchResponse searchResponse = client()
- .prepareSearch("test")
- .setQuery(hasChildQuery("child", boolQuery().should(termQuery("c_field", "red")).should(termQuery("c_field", "yellow")), ScoreMode.None))
- .addAggregation(AggregationBuilders.global("global").subAggregation(
- AggregationBuilders.filter("filter", boolQuery().should(termQuery("c_field", "red")).should(termQuery("c_field", "yellow"))).subAggregation(
- AggregationBuilders.terms("facet1").field("c_field")))).get();
- assertNoFailures(searchResponse);
- assertThat(searchResponse.getHits().getTotalHits(), equalTo(2L));
- assertThat(searchResponse.getHits().getAt(0).getId(), anyOf(equalTo("p2"), equalTo("p1")));
- assertThat(searchResponse.getHits().getAt(1).getId(), anyOf(equalTo("p2"), equalTo("p1")));
-
- Global global = searchResponse.getAggregations().get("global");
- Filter filter = global.getAggregations().get("filter");
- Terms termsFacet = filter.getAggregations().get("facet1");
- assertThat(termsFacet.getBuckets().size(), equalTo(2));
- assertThat(termsFacet.getBuckets().get(0).getKeyAsString(), equalTo("red"));
- assertThat(termsFacet.getBuckets().get(0).getDocCount(), equalTo(2L));
- assertThat(termsFacet.getBuckets().get(1).getKeyAsString(), equalTo("yellow"));
- assertThat(termsFacet.getBuckets().get(1).getDocCount(), equalTo(1L));
- }
-
- public void testDeletedParent() throws Exception {
- assertAcked(prepareCreate("test")
- .setSettings("index.mapping.single_type", false)
- .addMapping("parent")
- .addMapping("child", "_parent", "type=parent"));
- ensureGreen();
- // index simple data
- client().prepareIndex("test", "parent", "p1").setSource("p_field", "p_value1").get();
- client().prepareIndex("test", "child", "c1").setSource("c_field", "red").setParent("p1").get();
- client().prepareIndex("test", "child", "c2").setSource("c_field", "yellow").setParent("p1").get();
- client().prepareIndex("test", "parent", "p2").setSource("p_field", "p_value2").get();
- client().prepareIndex("test", "child", "c3").setSource("c_field", "blue").setParent("p2").get();
- client().prepareIndex("test", "child", "c4").setSource("c_field", "red").setParent("p2").get();
-
- refresh();
-
- SearchResponse searchResponse = client().prepareSearch("test")
- .setQuery(constantScoreQuery(hasChildQuery("child", termQuery("c_field", "yellow"), ScoreMode.None))).get();
- assertNoFailures(searchResponse);
- assertThat(searchResponse.getHits().getTotalHits(), equalTo(1L));
- assertThat(searchResponse.getHits().getAt(0).getId(), equalTo("p1"));
- assertThat(searchResponse.getHits().getAt(0).getSourceAsString(), containsString("\"p_value1\""));
-
- // update p1 and see what that we get updated values...
-
- client().prepareIndex("test", "parent", "p1").setSource("p_field", "p_value1_updated").get();
- client().admin().indices().prepareRefresh().get();
-
- searchResponse = client().prepareSearch("test")
- .setQuery(constantScoreQuery(hasChildQuery("child", termQuery("c_field", "yellow"), ScoreMode.None))).get();
- assertNoFailures(searchResponse);
- assertThat(searchResponse.getHits().getTotalHits(), equalTo(1L));
- assertThat(searchResponse.getHits().getAt(0).getId(), equalTo("p1"));
- assertThat(searchResponse.getHits().getAt(0).getSourceAsString(), containsString("\"p_value1_updated\""));
- }
-
- public void testDfsSearchType() throws Exception {
- assertAcked(prepareCreate("test")
- .setSettings("index.mapping.single_type", false)
- .addMapping("parent")
- .addMapping("child", "_parent", "type=parent"));
- ensureGreen();
-
- // index simple data
- client().prepareIndex("test", "parent", "p1").setSource("p_field", "p_value1").get();
- client().prepareIndex("test", "child", "c1").setSource("c_field", "red").setParent("p1").get();
- client().prepareIndex("test", "child", "c2").setSource("c_field", "yellow").setParent("p1").get();
- client().prepareIndex("test", "parent", "p2").setSource("p_field", "p_value2").get();
- client().prepareIndex("test", "child", "c3").setSource("c_field", "blue").setParent("p2").get();
- client().prepareIndex("test", "child", "c4").setSource("c_field", "red").setParent("p2").get();
-
- refresh();
-
- SearchResponse searchResponse = client().prepareSearch("test").setSearchType(SearchType.DFS_QUERY_THEN_FETCH)
- .setQuery(boolQuery().mustNot(hasChildQuery("child", boolQuery().should(queryStringQuery("c_field:*")), ScoreMode.None)))
- .get();
- assertNoFailures(searchResponse);
-
- searchResponse = client().prepareSearch("test").setSearchType(SearchType.DFS_QUERY_THEN_FETCH)
- .setQuery(boolQuery().mustNot(hasParentQuery("parent", boolQuery().should(queryStringQuery("p_field:*")), false))).execute()
- .actionGet();
- assertNoFailures(searchResponse);
- }
-
- public void testHasChildAndHasParentFailWhenSomeSegmentsDontContainAnyParentOrChildDocs() throws Exception {
- assertAcked(prepareCreate("test")
- .setSettings("index.mapping.single_type", false)
- .addMapping("parent")
- .addMapping("child", "_parent", "type=parent"));
- ensureGreen();
-
- client().prepareIndex("test", "parent", "1").setSource("p_field", 1).get();
- client().prepareIndex("test", "child", "1").setParent("1").setSource("c_field", 1).get();
- client().admin().indices().prepareFlush("test").get();
-
- client().prepareIndex("test", "type1", "1").setSource("p_field", 1).get();
- client().admin().indices().prepareFlush("test").get();
-
- SearchResponse searchResponse = client().prepareSearch("test")
- .setQuery(boolQuery().must(matchAllQuery()).filter(hasChildQuery("child", matchAllQuery(), ScoreMode.None))).get();
- assertNoFailures(searchResponse);
- assertThat(searchResponse.getHits().getTotalHits(), equalTo(1L));
-
- searchResponse = client().prepareSearch("test")
- .setQuery(boolQuery().must(matchAllQuery()).filter(hasParentQuery("parent", matchAllQuery(), false))).get();
- assertNoFailures(searchResponse);
- assertThat(searchResponse.getHits().getTotalHits(), equalTo(1L));
- }
-
- public void testCountApiUsage() throws Exception {
- assertAcked(prepareCreate("test")
- .setSettings("index.mapping.single_type", false)
- .addMapping("parent")
- .addMapping("child", "_parent", "type=parent"));
- ensureGreen();
-
- String parentId = "p1";
- client().prepareIndex("test", "parent", parentId).setSource("p_field", "1").get();
- client().prepareIndex("test", "child", "c1").setSource("c_field", "1").setParent(parentId).get();
- refresh();
-
- SearchResponse countResponse = client().prepareSearch("test").setSize(0)
- .setQuery(hasChildQuery("child", termQuery("c_field", "1"), ScoreMode.Max))
- .get();
- assertHitCount(countResponse, 1L);
-
- countResponse = client().prepareSearch("test").setSize(0).setQuery(hasParentQuery("parent", termQuery("p_field", "1"), true))
- .get();
- assertHitCount(countResponse, 1L);
-
- countResponse = client().prepareSearch("test").setSize(0)
- .setQuery(constantScoreQuery(hasChildQuery("child", termQuery("c_field", "1"), ScoreMode.None)))
- .get();
- assertHitCount(countResponse, 1L);
-
- countResponse = client().prepareSearch("test").setSize(0).setQuery(constantScoreQuery(hasParentQuery("parent", termQuery("p_field", "1"), false)))
- .get();
- assertHitCount(countResponse, 1L);
- }
-
- public void testExplainUsage() throws Exception {
- assertAcked(prepareCreate("test")
- .setSettings("index.mapping.single_type", false)
- .addMapping("parent")
- .addMapping("child", "_parent", "type=parent"));
- ensureGreen();
-
- String parentId = "p1";
- client().prepareIndex("test", "parent", parentId).setSource("p_field", "1").get();
- client().prepareIndex("test", "child", "c1").setSource("c_field", "1").setParent(parentId).get();
- refresh();
-
- SearchResponse searchResponse = client().prepareSearch("test")
- .setExplain(true)
- .setQuery(hasChildQuery("child", termQuery("c_field", "1"), ScoreMode.Max))
- .get();
- assertHitCount(searchResponse, 1L);
- assertThat(searchResponse.getHits().getAt(0).getExplanation().getDescription(), containsString("join value p1"));
-
- searchResponse = client().prepareSearch("test")
- .setExplain(true)
- .setQuery(hasParentQuery("parent", termQuery("p_field", "1"), true))
- .get();
- assertHitCount(searchResponse, 1L);
- assertThat(searchResponse.getHits().getAt(0).getExplanation().getDescription(), containsString("join value p1"));
-
- ExplainResponse explainResponse = client().prepareExplain("test", "parent", parentId)
- .setQuery(hasChildQuery("child", termQuery("c_field", "1"), ScoreMode.Max))
- .get();
- assertThat(explainResponse.isExists(), equalTo(true));
- assertThat(explainResponse.getExplanation().getDetails()[0].getDescription(), containsString("join value p1"));
- }
-
- List<IndexRequestBuilder> createDocBuilders() {
- List<IndexRequestBuilder> indexBuilders = new ArrayList<>();
- // Parent 1 and its children
- indexBuilders.add(client().prepareIndex().setType("parent").setId("1").setIndex("test").setSource("p_field", "p_value1"));
- indexBuilders.add(client().prepareIndex().setType("child").setId("1").setIndex("test")
- .setSource("c_field1", 1, "c_field2", 0).setParent("1"));
- indexBuilders.add(client().prepareIndex().setType("child").setId("2").setIndex("test")
- .setSource("c_field1", 1, "c_field2", 0).setParent("1"));
- indexBuilders.add(client().prepareIndex().setType("child").setId("3").setIndex("test")
- .setSource("c_field1", 2, "c_field2", 0).setParent("1"));
- indexBuilders.add(client().prepareIndex().setType("child").setId("4").setIndex("test")
- .setSource("c_field1", 2, "c_field2", 0).setParent("1"));
- indexBuilders.add(client().prepareIndex().setType("child").setId("5").setIndex("test")
- .setSource("c_field1", 1, "c_field2", 1).setParent("1"));
- indexBuilders.add(client().prepareIndex().setType("child").setId("6").setIndex("test")
- .setSource("c_field1", 1, "c_field2", 2).setParent("1"));
-
- // Parent 2 and its children
- indexBuilders.add(client().prepareIndex().setType("parent").setId("2").setIndex("test").setSource("p_field", "p_value2"));
- indexBuilders.add(client().prepareIndex().setType("child").setId("7").setIndex("test")
- .setSource("c_field1", 3, "c_field2", 0).setParent("2"));
- indexBuilders.add(client().prepareIndex().setType("child").setId("8").setIndex("test")
- .setSource("c_field1", 1, "c_field2", 1).setParent("2"));
- indexBuilders.add(client().prepareIndex().setType("child").setId("9").setIndex("test")
- .setSource("c_field1", 1, "c_field2", 1).setParent("p")); // why
- // "p"????
- indexBuilders.add(client().prepareIndex().setType("child").setId("10").setIndex("test")
- .setSource("c_field1", 1, "c_field2", 1).setParent("2"));
- indexBuilders.add(client().prepareIndex().setType("child").setId("11").setIndex("test")
- .setSource("c_field1", 1, "c_field2", 1).setParent("2"));
- indexBuilders.add(client().prepareIndex().setType("child").setId("12").setIndex("test")
- .setSource("c_field1", 1, "c_field2", 2).setParent("2"));
-
- // Parent 3 and its children
-
- indexBuilders.add(client().prepareIndex().setType("parent").setId("3").setIndex("test")
- .setSource("p_field1", "p_value3", "p_field2", 5));
- indexBuilders.add(client().prepareIndex().setType("child").setId("13").setIndex("test")
- .setSource("c_field1", 4, "c_field2", 0, "c_field3", 0).setParent("3"));
- indexBuilders.add(client().prepareIndex().setType("child").setId("14").setIndex("test")
- .setSource("c_field1", 1, "c_field2", 1, "c_field3", 1).setParent("3"));
- indexBuilders.add(client().prepareIndex().setType("child").setId("15").setIndex("test")
- .setSource("c_field1", 1, "c_field2", 2, "c_field3", 2).setParent("3")); // why
- // "p"????
- indexBuilders.add(client().prepareIndex().setType("child").setId("16").setIndex("test")
- .setSource("c_field1", 1, "c_field2", 2, "c_field3", 3).setParent("3"));
- indexBuilders.add(client().prepareIndex().setType("child").setId("17").setIndex("test")
- .setSource("c_field1", 1, "c_field2", 2, "c_field3", 4).setParent("3"));
- indexBuilders.add(client().prepareIndex().setType("child").setId("18").setIndex("test")
- .setSource("c_field1", 1, "c_field2", 2, "c_field3", 5).setParent("3"));
- indexBuilders.add(client().prepareIndex().setType("child1").setId("1").setIndex("test")
- .setSource("c_field1", 1, "c_field2", 2, "c_field3", 6).setParent("3"));
-
- return indexBuilders;
- }
-
- public void testScoreForParentChildQueriesWithFunctionScore() throws Exception {
- assertAcked(prepareCreate("test")
- .setSettings("index.mapping.single_type", false)
- .addMapping("parent")
- .addMapping("child", "_parent", "type=parent")
- .addMapping("child1", "_parent", "type=parent"));
- ensureGreen();
-
- indexRandom(true, createDocBuilders().toArray(new IndexRequestBuilder[0]));
- SearchResponse response = client()
- .prepareSearch("test")
- .setQuery(
- QueryBuilders.hasChildQuery(
- "child",
- QueryBuilders.functionScoreQuery(matchQuery("c_field2", 0),
- fieldValueFactorFunction("c_field1"))
- .boostMode(CombineFunction.REPLACE), ScoreMode.Total)).get();
-
- assertThat(response.getHits().getTotalHits(), equalTo(3L));
- assertThat(response.getHits().getHits()[0].getId(), equalTo("1"));
- assertThat(response.getHits().getHits()[0].getScore(), equalTo(6f));
- assertThat(response.getHits().getHits()[1].getId(), equalTo("3"));
- assertThat(response.getHits().getHits()[1].getScore(), equalTo(4f));
- assertThat(response.getHits().getHits()[2].getId(), equalTo("2"));
- assertThat(response.getHits().getHits()[2].getScore(), equalTo(3f));
-
- response = client()
- .prepareSearch("test")
- .setQuery(
- QueryBuilders.hasChildQuery(
- "child",
- QueryBuilders.functionScoreQuery(matchQuery("c_field2", 0),
- fieldValueFactorFunction("c_field1"))
- .boostMode(CombineFunction.REPLACE), ScoreMode.Max)).get();
-
- assertThat(response.getHits().getTotalHits(), equalTo(3L));
- assertThat(response.getHits().getHits()[0].getId(), equalTo("3"));
- assertThat(response.getHits().getHits()[0].getScore(), equalTo(4f));
- assertThat(response.getHits().getHits()[1].getId(), equalTo("2"));
- assertThat(response.getHits().getHits()[1].getScore(), equalTo(3f));
- assertThat(response.getHits().getHits()[2].getId(), equalTo("1"));
- assertThat(response.getHits().getHits()[2].getScore(), equalTo(2f));
-
- response = client()
- .prepareSearch("test")
- .setQuery(
- QueryBuilders.hasChildQuery(
- "child",
- QueryBuilders.functionScoreQuery(matchQuery("c_field2", 0),
- fieldValueFactorFunction("c_field1"))
- .boostMode(CombineFunction.REPLACE), ScoreMode.Avg)).get();
-
- assertThat(response.getHits().getTotalHits(), equalTo(3L));
- assertThat(response.getHits().getHits()[0].getId(), equalTo("3"));
- assertThat(response.getHits().getHits()[0].getScore(), equalTo(4f));
- assertThat(response.getHits().getHits()[1].getId(), equalTo("2"));
- assertThat(response.getHits().getHits()[1].getScore(), equalTo(3f));
- assertThat(response.getHits().getHits()[2].getId(), equalTo("1"));
- assertThat(response.getHits().getHits()[2].getScore(), equalTo(1.5f));
-
- response = client()
- .prepareSearch("test")
- .setQuery(
- QueryBuilders.hasParentQuery(
- "parent",
- QueryBuilders.functionScoreQuery(matchQuery("p_field1", "p_value3"),
- fieldValueFactorFunction("p_field2"))
- .boostMode(CombineFunction.REPLACE), true))
- .addSort(SortBuilders.fieldSort("c_field3")).addSort(SortBuilders.scoreSort()).get();
-
- assertThat(response.getHits().getTotalHits(), equalTo(7L));
- assertThat(response.getHits().getHits()[0].getId(), equalTo("13"));
- assertThat(response.getHits().getHits()[0].getScore(), equalTo(5f));
- assertThat(response.getHits().getHits()[1].getId(), equalTo("14"));
- assertThat(response.getHits().getHits()[1].getScore(), equalTo(5f));
- assertThat(response.getHits().getHits()[2].getId(), equalTo("15"));
- assertThat(response.getHits().getHits()[2].getScore(), equalTo(5f));
- assertThat(response.getHits().getHits()[3].getId(), equalTo("16"));
- assertThat(response.getHits().getHits()[3].getScore(), equalTo(5f));
- assertThat(response.getHits().getHits()[4].getId(), equalTo("17"));
- assertThat(response.getHits().getHits()[4].getScore(), equalTo(5f));
- assertThat(response.getHits().getHits()[5].getId(), equalTo("18"));
- assertThat(response.getHits().getHits()[5].getScore(), equalTo(5f));
- assertThat(response.getHits().getHits()[6].getId(), equalTo("1"));
- assertThat(response.getHits().getHits()[6].getScore(), equalTo(5f));
- }
-
- // Issue #2536
- public void testParentChildQueriesCanHandleNoRelevantTypesInIndex() throws Exception {
- assertAcked(prepareCreate("test")
- .setSettings("index.mapping.single_type", false)
- .addMapping("parent")
- .addMapping("child", "_parent", "type=parent"));
- ensureGreen();
-
- SearchResponse response = client().prepareSearch("test")
- .setQuery(QueryBuilders.hasChildQuery("child", matchQuery("text", "value"), ScoreMode.None)).get();
- assertNoFailures(response);
- assertThat(response.getHits().getTotalHits(), equalTo(0L));
-
- client().prepareIndex("test", "child1").setSource(jsonBuilder().startObject().field("text", "value").endObject())
- .setRefreshPolicy(RefreshPolicy.IMMEDIATE).get();
-
- response = client().prepareSearch("test")
- .setQuery(QueryBuilders.hasChildQuery("child", matchQuery("text", "value"), ScoreMode.None)).get();
- assertNoFailures(response);
- assertThat(response.getHits().getTotalHits(), equalTo(0L));
-
- response = client().prepareSearch("test").setQuery(QueryBuilders.hasChildQuery("child", matchQuery("text", "value"), ScoreMode.Max))
- .get();
- assertNoFailures(response);
- assertThat(response.getHits().getTotalHits(), equalTo(0L));
-
- response = client().prepareSearch("test").setQuery(QueryBuilders.hasParentQuery("parent", matchQuery("text", "value"), false)).get();
- assertNoFailures(response);
- assertThat(response.getHits().getTotalHits(), equalTo(0L));
-
- response = client().prepareSearch("test").setQuery(QueryBuilders.hasParentQuery("parent", matchQuery("text", "value"), true))
- .get();
- assertNoFailures(response);
- assertThat(response.getHits().getTotalHits(), equalTo(0L));
- }
-
- public void testHasChildAndHasParentFilter_withFilter() throws Exception {
- assertAcked(prepareCreate("test")
- .setSettings("index.mapping.single_type", false)
- .addMapping("parent")
- .addMapping("child", "_parent", "type=parent"));
- ensureGreen();
-
- client().prepareIndex("test", "parent", "1").setSource("p_field", 1).get();
- client().prepareIndex("test", "child", "2").setParent("1").setSource("c_field", 1).get();
- client().admin().indices().prepareFlush("test").get();
-
- client().prepareIndex("test", "type1", "3").setSource("p_field", 2).get();
- client().admin().indices().prepareFlush("test").get();
-
- SearchResponse searchResponse = client().prepareSearch("test")
- .setQuery(boolQuery().must(matchAllQuery()).filter(hasChildQuery("child", termQuery("c_field", 1), ScoreMode.None)))
- .get();
- assertNoFailures(searchResponse);
- assertThat(searchResponse.getHits().getTotalHits(), equalTo(1L));
- assertThat(searchResponse.getHits().getHits()[0].getId(), equalTo("1"));
-
- searchResponse = client().prepareSearch("test")
- .setQuery(boolQuery().must(matchAllQuery()).filter(hasParentQuery("parent", termQuery("p_field", 1), false))).get();
- assertNoFailures(searchResponse);
- assertThat(searchResponse.getHits().getTotalHits(), equalTo(1L));
- assertThat(searchResponse.getHits().getHits()[0].getId(), equalTo("2"));
- }
-
- public void testHasChildInnerHitsHighlighting() throws Exception {
- assertAcked(prepareCreate("test")
- .setSettings("index.mapping.single_type", false)
- .addMapping("parent")
- .addMapping("child", "_parent", "type=parent"));
- ensureGreen();
-
- client().prepareIndex("test", "parent", "1").setSource("p_field", 1).get();
- client().prepareIndex("test", "child", "2").setParent("1").setSource("c_field", "foo bar").get();
- client().admin().indices().prepareFlush("test").get();
-
- SearchResponse searchResponse = client().prepareSearch("test").setQuery(
- hasChildQuery("child", matchQuery("c_field", "foo"), ScoreMode.None)
- .innerHit(new InnerHitBuilder().setHighlightBuilder(
- new HighlightBuilder().field(new Field("c_field")
- .highlightQuery(QueryBuilders.matchQuery("c_field", "bar")))), false))
- .get();
- assertNoFailures(searchResponse);
- assertThat(searchResponse.getHits().getTotalHits(), equalTo(1L));
- assertThat(searchResponse.getHits().getHits()[0].getId(), equalTo("1"));
- SearchHit[] searchHits = searchResponse.getHits().getHits()[0].getInnerHits().get("child").getHits();
- assertThat(searchHits.length, equalTo(1));
- assertThat(searchHits[0].getHighlightFields().get("c_field").getFragments().length, equalTo(1));
- assertThat(searchHits[0].getHighlightFields().get("c_field").getFragments()[0].string(), equalTo("foo <em>bar</em>"));
- }
-
- public void testHasChildAndHasParentWrappedInAQueryFilter() throws Exception {
- assertAcked(prepareCreate("test")
- .setSettings("index.mapping.single_type", false)
- .addMapping("parent")
- .addMapping("child", "_parent", "type=parent"));
- ensureGreen();
-
- // query filter in case for p/c shouldn't execute per segment, but rather
- client().prepareIndex("test", "parent", "1").setSource("p_field", 1).get();
- client().admin().indices().prepareFlush("test").setForce(true).get();
- client().prepareIndex("test", "child", "2").setParent("1").setSource("c_field", 1).get();
- refresh();
-
- SearchResponse searchResponse = client().prepareSearch("test")
- .setQuery(boolQuery().must(matchAllQuery()).filter(hasChildQuery("child", matchQuery("c_field", 1), ScoreMode.None)))
- .get();
- assertSearchHit(searchResponse, 1, hasId("1"));
-
- searchResponse = client().prepareSearch("test")
- .setQuery(boolQuery().must(matchAllQuery()).filter(hasParentQuery("parent", matchQuery("p_field", 1), false))).get();
- assertSearchHit(searchResponse, 1, hasId("2"));
-
- searchResponse = client().prepareSearch("test")
- .setQuery(boolQuery().must(matchAllQuery()).filter(boolQuery().must(hasChildQuery("child", matchQuery("c_field", 1), ScoreMode.None))))
- .get();
- assertSearchHit(searchResponse, 1, hasId("1"));
-
- searchResponse = client().prepareSearch("test")
- .setQuery(boolQuery().must(matchAllQuery()).filter(boolQuery().must(hasParentQuery("parent", matchQuery("p_field", 1), false)))).get();
- assertSearchHit(searchResponse, 1, hasId("2"));
- }
-
- public void testSimpleQueryRewrite() throws Exception {
- assertAcked(prepareCreate("test")
- .setSettings("index.mapping.single_type", false)
- .addMapping("parent", "p_field", "type=keyword")
- .addMapping("child", "_parent", "type=parent", "c_field", "type=keyword"));
- ensureGreen();
-
- // index simple data
- int childId = 0;
- for (int i = 0; i < 10; i++) {
- String parentId = String.format(Locale.ROOT, "p%03d", i);
- client().prepareIndex("test", "parent", parentId).setSource("p_field", parentId).get();
- int j = childId;
- for (; j < childId + 50; j++) {
- String childUid = String.format(Locale.ROOT, "c%03d", j);
- client().prepareIndex("test", "child", childUid).setSource("c_field", childUid).setParent(parentId).get();
- }
- childId = j;
- }
- refresh();
-
- SearchType[] searchTypes = new SearchType[]{SearchType.QUERY_THEN_FETCH, SearchType.DFS_QUERY_THEN_FETCH};
- for (SearchType searchType : searchTypes) {
- SearchResponse searchResponse = client().prepareSearch("test").setSearchType(searchType)
- .setQuery(hasChildQuery("child", prefixQuery("c_field", "c"), ScoreMode.Max))
- .addSort("p_field", SortOrder.ASC)
- .setSize(5).get();
- assertNoFailures(searchResponse);
- assertThat(searchResponse.getHits().getTotalHits(), equalTo(10L));
- assertThat(searchResponse.getHits().getHits()[0].getId(), equalTo("p000"));
- assertThat(searchResponse.getHits().getHits()[1].getId(), equalTo("p001"));
- assertThat(searchResponse.getHits().getHits()[2].getId(), equalTo("p002"));
- assertThat(searchResponse.getHits().getHits()[3].getId(), equalTo("p003"));
- assertThat(searchResponse.getHits().getHits()[4].getId(), equalTo("p004"));
-
- searchResponse = client().prepareSearch("test").setSearchType(searchType)
- .setQuery(hasParentQuery("parent", prefixQuery("p_field", "p"), true)).addSort("c_field", SortOrder.ASC)
- .setSize(5).get();
- assertNoFailures(searchResponse);
- assertThat(searchResponse.getHits().getTotalHits(), equalTo(500L));
- assertThat(searchResponse.getHits().getHits()[0].getId(), equalTo("c000"));
- assertThat(searchResponse.getHits().getHits()[1].getId(), equalTo("c001"));
- assertThat(searchResponse.getHits().getHits()[2].getId(), equalTo("c002"));
- assertThat(searchResponse.getHits().getHits()[3].getId(), equalTo("c003"));
- assertThat(searchResponse.getHits().getHits()[4].getId(), equalTo("c004"));
- }
- }
-
- // Issue #3144
- public void testReIndexingParentAndChildDocuments() throws Exception {
- assertAcked(prepareCreate("test")
- .setSettings("index.mapping.single_type", false)
- .addMapping("parent")
- .addMapping("child", "_parent", "type=parent"));
- ensureGreen();
-
- // index simple data
- client().prepareIndex("test", "parent", "p1").setSource("p_field", "p_value1").get();
- client().prepareIndex("test", "child", "c1").setSource("c_field", "red").setParent("p1").get();
- client().prepareIndex("test", "child", "c2").setSource("c_field", "yellow").setParent("p1").get();
- client().prepareIndex("test", "parent", "p2").setSource("p_field", "p_value2").get();
- client().prepareIndex("test", "child", "c3").setSource("c_field", "x").setParent("p2").get();
- client().prepareIndex("test", "child", "c4").setSource("c_field", "x").setParent("p2").get();
-
- refresh();
-
- SearchResponse searchResponse = client().prepareSearch("test")
- .setQuery(hasChildQuery("child", termQuery("c_field", "yellow"), ScoreMode.Total)).get();
- assertNoFailures(searchResponse);
- assertThat(searchResponse.getHits().getTotalHits(), equalTo(1L));
- assertThat(searchResponse.getHits().getAt(0).getId(), equalTo("p1"));
- assertThat(searchResponse.getHits().getAt(0).getSourceAsString(), containsString("\"p_value1\""));
-
- searchResponse = client()
- .prepareSearch("test")
- .setQuery(
- boolQuery().must(matchQuery("c_field", "x")).must(
- hasParentQuery("parent", termQuery("p_field", "p_value2"), true))).get();
- assertNoFailures(searchResponse);
- assertThat(searchResponse.getHits().getTotalHits(), equalTo(2L));
- assertThat(searchResponse.getHits().getAt(0).getId(), equalTo("c3"));
- assertThat(searchResponse.getHits().getAt(1).getId(), equalTo("c4"));
-
- // re-index
- for (int i = 0; i < 10; i++) {
- client().prepareIndex("test", "parent", "p1").setSource("p_field", "p_value1").get();
- client().prepareIndex("test", "child", "d" + i).setSource("c_field", "red").setParent("p1").get();
- client().prepareIndex("test", "parent", "p2").setSource("p_field", "p_value2").get();
- client().prepareIndex("test", "child", "c3").setSource("c_field", "x").setParent("p2").get();
- client().admin().indices().prepareRefresh("test").get();
- }
-
- searchResponse = client().prepareSearch("test").setQuery(hasChildQuery("child", termQuery("c_field", "yellow"), ScoreMode.Total))
- .get();
- assertNoFailures(searchResponse);
- assertThat(searchResponse.getHits().getTotalHits(), equalTo(1L));
- assertThat(searchResponse.getHits().getAt(0).getId(), equalTo("p1"));
- assertThat(searchResponse.getHits().getAt(0).getSourceAsString(), containsString("\"p_value1\""));
-
- searchResponse = client()
- .prepareSearch("test")
- .setQuery(
- boolQuery().must(matchQuery("c_field", "x")).must(
- hasParentQuery("parent", termQuery("p_field", "p_value2"), true))).get();
- assertNoFailures(searchResponse);
- assertThat(searchResponse.getHits().getTotalHits(), equalTo(2L));
- assertThat(searchResponse.getHits().getAt(0).getId(), Matchers.anyOf(equalTo("c3"), equalTo("c4")));
- assertThat(searchResponse.getHits().getAt(1).getId(), Matchers.anyOf(equalTo("c3"), equalTo("c4")));
- }
-
- // Issue #3203
- public void testHasChildQueryWithMinimumScore() throws Exception {
- assertAcked(prepareCreate("test")
- .setSettings("index.mapping.single_type", false)
- .addMapping("parent")
- .addMapping("child", "_parent", "type=parent"));
- ensureGreen();
-
- // index simple data
- client().prepareIndex("test", "parent", "p1").setSource("p_field", "p_value1").get();
- client().prepareIndex("test", "child", "c1").setSource("c_field", "x").setParent("p1").get();
- client().prepareIndex("test", "parent", "p2").setSource("p_field", "p_value2").get();
- client().prepareIndex("test", "child", "c3").setSource("c_field", "x").setParent("p2").get();
- client().prepareIndex("test", "child", "c4").setSource("c_field", "x").setParent("p2").get();
- client().prepareIndex("test", "child", "c5").setSource("c_field", "x").setParent("p2").get();
- refresh();
-
- SearchResponse searchResponse = client().prepareSearch("test").setQuery(hasChildQuery("child", matchAllQuery(), ScoreMode.Total))
- .setMinScore(3) // Score needs to be 3 or above!
- .get();
- assertNoFailures(searchResponse);
- assertThat(searchResponse.getHits().getTotalHits(), equalTo(1L));
- assertThat(searchResponse.getHits().getAt(0).getId(), equalTo("p2"));
- assertThat(searchResponse.getHits().getAt(0).getScore(), equalTo(3.0f));
- }
-
- public void testParentFieldQuery() throws Exception {
- assertAcked(prepareCreate("test")
- .setSettings("index.refresh_interval", -1, "index.mapping.single_type", false)
- .addMapping("parent")
- .addMapping("child", "_parent", "type=parent"));
- ensureGreen();
-
- SearchResponse response = client().prepareSearch("test").setQuery(termQuery("_parent", "p1"))
- .get();
- assertHitCount(response, 0L);
-
- client().prepareIndex("test", "child", "c1").setSource("{}", XContentType.JSON).setParent("p1").get();
- refresh();
-
- response = client().prepareSearch("test").setQuery(termQuery("_parent#parent", "p1")).get();
- assertHitCount(response, 1L);
-
- response = client().prepareSearch("test").setQuery(queryStringQuery("_parent#parent:p1")).get();
- assertHitCount(response, 1L);
-
- client().prepareIndex("test", "child", "c2").setSource("{}", XContentType.JSON).setParent("p2").get();
- refresh();
- response = client().prepareSearch("test").setQuery(termsQuery("_parent#parent", "p1", "p2")).get();
- assertHitCount(response, 2L);
-
- response = client().prepareSearch("test")
- .setQuery(boolQuery()
- .should(termQuery("_parent#parent", "p1"))
- .should(termQuery("_parent#parent", "p2"))
- ).get();
- assertHitCount(response, 2L);
- }
-
- public void testParentIdQuery() throws Exception {
- assertAcked(prepareCreate("test")
- .setSettings("index.refresh_interval", -1, "index.mapping.single_type", false)
- .addMapping("parent")
- .addMapping("child", "_parent", "type=parent"));
- ensureGreen();
-
- client().prepareIndex("test", "child", "c1").setSource("{}", XContentType.JSON).setParent("p1").get();
- refresh();
-
- SearchResponse response = client().prepareSearch("test").setQuery(parentId("child", "p1")).get();
- assertHitCount(response, 1L);
-
- client().prepareIndex("test", "child", "c2").setSource("{}", XContentType.JSON).setParent("p2").get();
- refresh();
-
- response = client().prepareSearch("test")
- .setQuery(boolQuery()
- .should(parentId("child", "p1"))
- .should(parentId("child", "p2"))
- ).get();
- assertHitCount(response, 2L);
- }
-
- public void testHasChildNotBeingCached() throws IOException {
- assertAcked(prepareCreate("test")
- .setSettings("index.mapping.single_type", false)
- .addMapping("parent")
- .addMapping("child", "_parent", "type=parent"));
- ensureGreen();
-
- // index simple data
- client().prepareIndex("test", "parent", "p1").setSource("p_field", "p_value1").get();
- client().prepareIndex("test", "parent", "p2").setSource("p_field", "p_value2").get();
- client().prepareIndex("test", "parent", "p3").setSource("p_field", "p_value3").get();
- client().prepareIndex("test", "parent", "p4").setSource("p_field", "p_value4").get();
- client().prepareIndex("test", "parent", "p5").setSource("p_field", "p_value5").get();
- client().prepareIndex("test", "parent", "p6").setSource("p_field", "p_value6").get();
- client().prepareIndex("test", "parent", "p7").setSource("p_field", "p_value7").get();
- client().prepareIndex("test", "parent", "p8").setSource("p_field", "p_value8").get();
- client().prepareIndex("test", "parent", "p9").setSource("p_field", "p_value9").get();
- client().prepareIndex("test", "parent", "p10").setSource("p_field", "p_value10").get();
- client().prepareIndex("test", "child", "c1").setParent("p1").setSource("c_field", "blue").get();
- client().admin().indices().prepareFlush("test").get();
- client().admin().indices().prepareRefresh("test").get();
-
- SearchResponse searchResponse = client().prepareSearch("test")
- .setQuery(constantScoreQuery(hasChildQuery("child", termQuery("c_field", "blue"), ScoreMode.None)))
- .get();
- assertNoFailures(searchResponse);
- assertThat(searchResponse.getHits().getTotalHits(), equalTo(1L));
-
- client().prepareIndex("test", "child", "c2").setParent("p2").setSource("c_field", "blue").get();
- client().admin().indices().prepareRefresh("test").get();
-
- searchResponse = client().prepareSearch("test")
- .setQuery(constantScoreQuery(hasChildQuery("child", termQuery("c_field", "blue"), ScoreMode.None)))
- .get();
- assertNoFailures(searchResponse);
- assertThat(searchResponse.getHits().getTotalHits(), equalTo(2L));
- }
-
- private QueryBuilder randomHasChild(String type, String field, String value) {
- if (randomBoolean()) {
- if (randomBoolean()) {
- return constantScoreQuery(hasChildQuery(type, termQuery(field, value), ScoreMode.None));
- } else {
- return boolQuery().must(matchAllQuery()).filter(hasChildQuery(type, termQuery(field, value), ScoreMode.None));
- }
- } else {
- return hasChildQuery(type, termQuery(field, value), ScoreMode.None);
- }
- }
-
- private QueryBuilder randomHasParent(String type, String field, String value) {
- if (randomBoolean()) {
- if (randomBoolean()) {
- return constantScoreQuery(hasParentQuery(type, termQuery(field, value), false));
- } else {
- return boolQuery().must(matchAllQuery()).filter(hasParentQuery(type, termQuery(field, value), false));
- }
- } else {
- return hasParentQuery(type, termQuery(field, value), false);
- }
- }
-
- // Issue #3818
- public void testHasChildQueryOnlyReturnsSingleChildType() {
- assertAcked(prepareCreate("grandissue")
- .setSettings("index.mapping.single_type", false)
- .addMapping("grandparent", "name", "type=text")
- .addMapping("parent", "_parent", "type=grandparent")
- .addMapping("child_type_one", "_parent", "type=parent")
- .addMapping("child_type_two", "_parent", "type=parent"));
-
- client().prepareIndex("grandissue", "grandparent", "1").setSource("name", "Grandpa").get();
- client().prepareIndex("grandissue", "parent", "2").setParent("1").setSource("name", "Dana").get();
- client().prepareIndex("grandissue", "child_type_one", "3").setParent("2").setRouting("1")
- .setSource("name", "William")
- .get();
- client().prepareIndex("grandissue", "child_type_two", "4").setParent("2").setRouting("1")
- .setSource("name", "Kate")
- .get();
- refresh();
-
- SearchResponse searchResponse = client().prepareSearch("grandissue").setQuery(
- boolQuery().must(
- hasChildQuery(
- "parent",
- boolQuery().must(
- hasChildQuery(
- "child_type_one",
- boolQuery().must(
- queryStringQuery("name:William*")
- ),
- ScoreMode.None)
- ),
- ScoreMode.None)
- )
- ).get();
- assertHitCount(searchResponse, 1L);
-
- searchResponse = client().prepareSearch("grandissue").setQuery(
- boolQuery().must(
- hasChildQuery(
- "parent",
- boolQuery().must(
- hasChildQuery(
- "child_type_two",
- boolQuery().must(
- queryStringQuery("name:William*")
- ),
- ScoreMode.None)
- ),
- ScoreMode.None)
- )
- ).get();
- assertHitCount(searchResponse, 0L);
- }
-
- public void testIndexChildDocWithNoParentMapping() throws IOException {
- assertAcked(prepareCreate("test")
- .setSettings("index.mapping.single_type", false)
- .addMapping("parent")
- .addMapping("child1"));
- ensureGreen();
-
- client().prepareIndex("test", "parent", "p1").setSource("p_field", "p_value1").get();
- try {
- client().prepareIndex("test", "child1", "c1").setParent("p1").setSource("c_field", "blue").get();
- fail();
- } catch (IllegalArgumentException e) {
- assertThat(e.toString(), containsString("can't specify parent if no parent field has been configured"));
- }
- try {
- client().prepareIndex("test", "child2", "c2").setParent("p1").setSource("c_field", "blue").get();
- fail();
- } catch (IllegalArgumentException e) {
- assertThat(e.toString(), containsString("can't specify parent if no parent field has been configured"));
- }
-
- refresh();
- }
-
- public void testAddingParentToExistingMapping() throws IOException {
- createIndex("test");
- ensureGreen();
-
- PutMappingResponse putMappingResponse = client().admin().indices().preparePutMapping("test").setType("child").setSource("number", "type=integer")
- .get();
- assertThat(putMappingResponse.isAcknowledged(), equalTo(true));
-
- GetMappingsResponse getMappingsResponse = client().admin().indices().prepareGetMappings("test").get();
- Map<String, Object> mapping = getMappingsResponse.getMappings().get("test").get("child").getSourceAsMap();
- assertThat(mapping.size(), greaterThanOrEqualTo(1)); // there are potentially some meta fields configured randomly
- assertThat(mapping.get("properties"), notNullValue());
-
- try {
- // Adding _parent metadata field to existing mapping is prohibited:
- client().admin().indices().preparePutMapping("test").setType("child").setSource(jsonBuilder().startObject().startObject("child")
- .startObject("_parent").field("type", "parent").endObject()
- .endObject().endObject()).get();
- fail();
- } catch (IllegalArgumentException e) {
- assertThat(e.toString(), containsString("The _parent field's type option can't be changed: [null]->[parent]"));
- }
- }
-
- public void testHasChildQueryWithNestedInnerObjects() throws Exception {
- assertAcked(prepareCreate("test")
- .setSettings("index.mapping.single_type", false)
- .addMapping("parent", "objects", "type=nested")
- .addMapping("child", "_parent", "type=parent"));
- ensureGreen();
-
- client().prepareIndex("test", "parent", "p1")
- .setSource(jsonBuilder().startObject().field("p_field", "1").startArray("objects")
- .startObject().field("i_field", "1").endObject()
- .startObject().field("i_field", "2").endObject()
- .startObject().field("i_field", "3").endObject()
- .startObject().field("i_field", "4").endObject()
- .startObject().field("i_field", "5").endObject()
- .startObject().field("i_field", "6").endObject()
- .endArray().endObject())
- .get();
- client().prepareIndex("test", "parent", "p2")
- .setSource(jsonBuilder().startObject().field("p_field", "2").startArray("objects")
- .startObject().field("i_field", "1").endObject()
- .startObject().field("i_field", "2").endObject()
- .endArray().endObject())
- .get();
- client().prepareIndex("test", "child", "c1").setParent("p1").setSource("c_field", "blue").get();
- client().prepareIndex("test", "child", "c2").setParent("p1").setSource("c_field", "red").get();
- client().prepareIndex("test", "child", "c3").setParent("p2").setSource("c_field", "red").get();
- refresh();
-
- ScoreMode scoreMode = randomFrom(ScoreMode.values());
- SearchResponse searchResponse = client().prepareSearch("test")
- .setQuery(boolQuery().must(QueryBuilders.hasChildQuery("child", termQuery("c_field", "blue"), scoreMode)).filter(boolQuery().mustNot(termQuery("p_field", "3"))))
- .get();
- assertNoFailures(searchResponse);
- assertThat(searchResponse.getHits().getTotalHits(), equalTo(1L));
-
- searchResponse = client().prepareSearch("test")
- .setQuery(boolQuery().must(QueryBuilders.hasChildQuery("child", termQuery("c_field", "red"), scoreMode)).filter(boolQuery().mustNot(termQuery("p_field", "3"))))
- .get();
- assertNoFailures(searchResponse);
- assertThat(searchResponse.getHits().getTotalHits(), equalTo(2L));
- }
-
- public void testNamedFilters() throws Exception {
- assertAcked(prepareCreate("test")
- .setSettings("index.mapping.single_type", false)
- .addMapping("parent")
- .addMapping("child", "_parent", "type=parent"));
- ensureGreen();
-
- String parentId = "p1";
- client().prepareIndex("test", "parent", parentId).setSource("p_field", "1").get();
- client().prepareIndex("test", "child", "c1").setSource("c_field", "1").setParent(parentId).get();
- refresh();
-
- SearchResponse searchResponse = client().prepareSearch("test").setQuery(hasChildQuery("child", termQuery("c_field", "1"), ScoreMode.Max).queryName("test"))
- .get();
- assertHitCount(searchResponse, 1L);
- assertThat(searchResponse.getHits().getAt(0).getMatchedQueries().length, equalTo(1));
- assertThat(searchResponse.getHits().getAt(0).getMatchedQueries()[0], equalTo("test"));
-
- searchResponse = client().prepareSearch("test").setQuery(hasParentQuery("parent", termQuery("p_field", "1"), true).queryName("test"))
- .get();
- assertHitCount(searchResponse, 1L);
- assertThat(searchResponse.getHits().getAt(0).getMatchedQueries().length, equalTo(1));
- assertThat(searchResponse.getHits().getAt(0).getMatchedQueries()[0], equalTo("test"));
-
- searchResponse = client().prepareSearch("test").setQuery(constantScoreQuery(hasChildQuery("child", termQuery("c_field", "1"), ScoreMode.None).queryName("test")))
- .get();
- assertHitCount(searchResponse, 1L);
- assertThat(searchResponse.getHits().getAt(0).getMatchedQueries().length, equalTo(1));
- assertThat(searchResponse.getHits().getAt(0).getMatchedQueries()[0], equalTo("test"));
-
- searchResponse = client().prepareSearch("test").setQuery(constantScoreQuery(hasParentQuery("parent", termQuery("p_field", "1"), false).queryName("test")))
- .get();
- assertHitCount(searchResponse, 1L);
- assertThat(searchResponse.getHits().getAt(0).getMatchedQueries().length, equalTo(1));
- assertThat(searchResponse.getHits().getAt(0).getMatchedQueries()[0], equalTo("test"));
- }
-
- public void testParentChildQueriesNoParentType() throws Exception {
- assertAcked(prepareCreate("test")
- .setSettings(Settings.builder()
- .put(indexSettings())
- .put("index.refresh_interval", -1)));
- ensureGreen();
-
- String parentId = "p1";
- client().prepareIndex("test", "parent", parentId).setSource("p_field", "1").get();
- refresh();
-
- try {
- client().prepareSearch("test")
- .setQuery(hasChildQuery("child", termQuery("c_field", "1"), ScoreMode.None))
- .get();
- fail();
- } catch (SearchPhaseExecutionException e) {
- assertThat(e.status(), equalTo(RestStatus.BAD_REQUEST));
- }
-
- try {
- client().prepareSearch("test")
- .setQuery(hasChildQuery("child", termQuery("c_field", "1"), ScoreMode.Max))
- .get();
- fail();
- } catch (SearchPhaseExecutionException e) {
- assertThat(e.status(), equalTo(RestStatus.BAD_REQUEST));
- }
-
- try {
- client().prepareSearch("test")
- .setPostFilter(hasChildQuery("child", termQuery("c_field", "1"), ScoreMode.None))
- .get();
- fail();
- } catch (SearchPhaseExecutionException e) {
- assertThat(e.status(), equalTo(RestStatus.BAD_REQUEST));
- }
-
- try {
- client().prepareSearch("test")
- .setQuery(hasParentQuery("parent", termQuery("p_field", "1"), true))
- .get();
- fail();
- } catch (SearchPhaseExecutionException e) {
- assertThat(e.status(), equalTo(RestStatus.BAD_REQUEST));
- }
-
- try {
- client().prepareSearch("test")
- .setPostFilter(hasParentQuery("parent", termQuery("p_field", "1"), false))
- .get();
- fail();
- } catch (SearchPhaseExecutionException e) {
- assertThat(e.status(), equalTo(RestStatus.BAD_REQUEST));
- }
- }
-
- public void testParentChildCaching() throws Exception {
- assertAcked(prepareCreate("test")
- .setSettings("index.refresh_interval", -1, "index.mapping.single_type", false)
- .addMapping("parent")
- .addMapping("child", "_parent", "type=parent"));
- ensureGreen();
-
- // index simple data
- client().prepareIndex("test", "parent", "p1").setSource("p_field", "p_value1").get();
- client().prepareIndex("test", "parent", "p2").setSource("p_field", "p_value2").get();
- client().prepareIndex("test", "child", "c1").setParent("p1").setSource("c_field", "blue").get();
- client().prepareIndex("test", "child", "c2").setParent("p1").setSource("c_field", "red").get();
- client().prepareIndex("test", "child", "c3").setParent("p2").setSource("c_field", "red").get();
- client().admin().indices().prepareForceMerge("test").setMaxNumSegments(1).setFlush(true).get();
- client().prepareIndex("test", "parent", "p3").setSource("p_field", "p_value3").get();
- client().prepareIndex("test", "parent", "p4").setSource("p_field", "p_value4").get();
- client().prepareIndex("test", "child", "c4").setParent("p3").setSource("c_field", "green").get();
- client().prepareIndex("test", "child", "c5").setParent("p3").setSource("c_field", "blue").get();
- client().prepareIndex("test", "child", "c6").setParent("p4").setSource("c_field", "blue").get();
- client().admin().indices().prepareFlush("test").get();
- client().admin().indices().prepareRefresh("test").get();
-
- for (int i = 0; i < 2; i++) {
- SearchResponse searchResponse = client().prepareSearch()
- .setQuery(boolQuery().must(matchAllQuery()).filter(boolQuery()
- .must(QueryBuilders.hasChildQuery("child", matchQuery("c_field", "red"), ScoreMode.None))
- .must(matchAllQuery())))
- .get();
- assertThat(searchResponse.getHits().getTotalHits(), equalTo(2L));
- }
-
-
- client().prepareIndex("test", "child", "c3").setParent("p2").setSource("c_field", "blue").get();
- client().admin().indices().prepareRefresh("test").get();
-
- SearchResponse searchResponse = client().prepareSearch()
- .setQuery(boolQuery().must(matchAllQuery()).filter(boolQuery()
- .must(QueryBuilders.hasChildQuery("child", matchQuery("c_field", "red"), ScoreMode.None))
- .must(matchAllQuery())))
- .get();
-
- assertThat(searchResponse.getHits().getTotalHits(), equalTo(1L));
- }
-
- public void testParentChildQueriesViaScrollApi() throws Exception {
- assertAcked(prepareCreate("test")
- .setSettings("index.mapping.single_type", false)
- .addMapping("parent")
- .addMapping("child", "_parent", "type=parent"));
- ensureGreen();
- for (int i = 0; i < 10; i++) {
- client().prepareIndex("test", "parent", "p" + i).setSource("{}", XContentType.JSON).get();
- client().prepareIndex("test", "child", "c" + i).setSource("{}", XContentType.JSON).setParent("p" + i).get();
- }
-
- refresh();
-
- QueryBuilder[] queries = new QueryBuilder[]{
- hasChildQuery("child", matchAllQuery(), ScoreMode.None),
- boolQuery().must(matchAllQuery()).filter(hasChildQuery("child", matchAllQuery(), ScoreMode.None)),
- hasParentQuery("parent", matchAllQuery(), false),
- boolQuery().must(matchAllQuery()).filter(hasParentQuery("parent", matchAllQuery(), false))
- };
-
- for (QueryBuilder query : queries) {
- SearchResponse scrollResponse = client().prepareSearch("test")
- .setScroll(TimeValue.timeValueSeconds(30))
- .setSize(1)
- .addStoredField("_id")
- .setQuery(query)
- .execute()
- .actionGet();
-
- assertNoFailures(scrollResponse);
- assertThat(scrollResponse.getHits().getTotalHits(), equalTo(10L));
- int scannedDocs = 0;
- do {
- assertThat(scrollResponse.getHits().getTotalHits(), equalTo(10L));
- scannedDocs += scrollResponse.getHits().getHits().length;
- scrollResponse = client()
- .prepareSearchScroll(scrollResponse.getScrollId())
- .setScroll(TimeValue.timeValueSeconds(30)).get();
- } while (scrollResponse.getHits().getHits().length > 0);
- clearScroll(scrollResponse.getScrollId());
- assertThat(scannedDocs, equalTo(10));
- }
- }
-
- // Issue #5783
- public void testQueryBeforeChildType() throws Exception {
- assertAcked(prepareCreate("test")
- .setSettings("index.mapping.single_type", false)
- .addMapping("features")
- .addMapping("posts", "_parent", "type=features")
- .addMapping("specials"));
- ensureGreen();
-
- client().prepareIndex("test", "features", "1").setSource("field", "foo").get();
- client().prepareIndex("test", "posts", "1").setParent("1").setSource("field", "bar").get();
- refresh();
-
- SearchResponse resp;
- resp = client().prepareSearch("test")
- .setSource(new SearchSourceBuilder().query(QueryBuilders.hasChildQuery("posts", QueryBuilders.matchQuery("field", "bar"), ScoreMode.None)))
- .get();
- assertHitCount(resp, 1L);
- }
-
- // Issue #6256
- public void testParentFieldInMultiMatchField() throws Exception {
- assertAcked(prepareCreate("test")
- .setSettings("index.mapping.single_type", false)
- .addMapping("type1")
- .addMapping("type2", "_parent", "type=type1")
- );
- ensureGreen();
-
- client().prepareIndex("test", "type2", "1").setParent("1").setSource("field", "value").get();
- refresh();
-
- SearchResponse response = client().prepareSearch("test")
- .setQuery(multiMatchQuery("1", "_parent#type1"))
- .get();
-
- assertThat(response.getHits().getTotalHits(), equalTo(1L));
- assertThat(response.getHits().getAt(0).getId(), equalTo("1"));
- }
-
- public void testTypeIsAppliedInHasParentInnerQuery() throws Exception {
- assertAcked(prepareCreate("test")
- .setSettings("index.mapping.single_type", false)
- .addMapping("parent")
- .addMapping("child", "_parent", "type=parent"));
- ensureGreen();
-
- List<IndexRequestBuilder> indexRequests = new ArrayList<>();
- indexRequests.add(client().prepareIndex("test", "parent", "1").setSource("field1", "a"));
- indexRequests.add(client().prepareIndex("test", "child", "1").setParent("1").setSource("{}", XContentType.JSON));
- indexRequests.add(client().prepareIndex("test", "child", "2").setParent("1").setSource("{}", XContentType.JSON));
- indexRandom(true, indexRequests);
-
- SearchResponse searchResponse = client().prepareSearch("test")
- .setQuery(constantScoreQuery(hasParentQuery("parent", boolQuery().mustNot(termQuery("field1", "a")), false)))
- .get();
- assertHitCount(searchResponse, 0L);
-
- searchResponse = client().prepareSearch("test")
- .setQuery(hasParentQuery("parent", constantScoreQuery(boolQuery().mustNot(termQuery("field1", "a"))), false))
- .get();
- assertHitCount(searchResponse, 0L);
-
- searchResponse = client().prepareSearch("test")
- .setQuery(constantScoreQuery(hasParentQuery("parent", termQuery("field1", "a"), false)))
- .get();
- assertHitCount(searchResponse, 2L);
-
- searchResponse = client().prepareSearch("test")
- .setQuery(hasParentQuery("parent", constantScoreQuery(termQuery("field1", "a")), false))
- .get();
- assertHitCount(searchResponse, 2L);
- }
-
- private List<IndexRequestBuilder> createMinMaxDocBuilders() {
- List<IndexRequestBuilder> indexBuilders = new ArrayList<>();
- // Parent 1 and its children
- indexBuilders.add(client().prepareIndex().setType("parent").setId("1").setIndex("test").setSource("id",1));
- indexBuilders.add(client().prepareIndex().setType("child").setId("10").setIndex("test")
- .setSource("foo", "one").setParent("1"));
-
- // Parent 2 and its children
- indexBuilders.add(client().prepareIndex().setType("parent").setId("2").setIndex("test").setSource("id",2));
- indexBuilders.add(client().prepareIndex().setType("child").setId("11").setIndex("test")
- .setSource("foo", "one").setParent("2"));
- indexBuilders.add(client().prepareIndex().setType("child").setId("12").setIndex("test")
- .setSource("foo", "one two").setParent("2"));
-
- // Parent 3 and its children
- indexBuilders.add(client().prepareIndex().setType("parent").setId("3").setIndex("test").setSource("id",3));
- indexBuilders.add(client().prepareIndex().setType("child").setId("13").setIndex("test")
- .setSource("foo", "one").setParent("3"));
- indexBuilders.add(client().prepareIndex().setType("child").setId("14").setIndex("test")
- .setSource("foo", "one two").setParent("3"));
- indexBuilders.add(client().prepareIndex().setType("child").setId("15").setIndex("test")
- .setSource("foo", "one two three").setParent("3"));
-
- // Parent 4 and its children
- indexBuilders.add(client().prepareIndex().setType("parent").setId("4").setIndex("test").setSource("id",4));
- indexBuilders.add(client().prepareIndex().setType("child").setId("16").setIndex("test")
- .setSource("foo", "one").setParent("4"));
- indexBuilders.add(client().prepareIndex().setType("child").setId("17").setIndex("test")
- .setSource("foo", "one two").setParent("4"));
- indexBuilders.add(client().prepareIndex().setType("child").setId("18").setIndex("test")
- .setSource("foo", "one two three").setParent("4"));
- indexBuilders.add(client().prepareIndex().setType("child").setId("19").setIndex("test")
- .setSource("foo", "one two three four").setParent("4"));
-
- return indexBuilders;
- }
-
- private SearchResponse minMaxQuery(ScoreMode scoreMode, int minChildren, Integer maxChildren) throws SearchPhaseExecutionException {
- HasChildQueryBuilder hasChildQuery = hasChildQuery(
- "child",
- QueryBuilders.functionScoreQuery(constantScoreQuery(QueryBuilders.termQuery("foo", "two")),
- new FunctionScoreQueryBuilder.FilterFunctionBuilder[]{
- new FunctionScoreQueryBuilder.FilterFunctionBuilder(weightFactorFunction(1)),
- new FunctionScoreQueryBuilder.FilterFunctionBuilder(QueryBuilders.termQuery("foo", "three"), weightFactorFunction(1)),
- new FunctionScoreQueryBuilder.FilterFunctionBuilder(QueryBuilders.termQuery("foo", "four"), weightFactorFunction(1))
- }).boostMode(CombineFunction.REPLACE).scoreMode(FiltersFunctionScoreQuery.ScoreMode.SUM), scoreMode)
- .minMaxChildren(minChildren, maxChildren != null ? maxChildren : HasChildQueryBuilder.DEFAULT_MAX_CHILDREN);
-
- return client()
- .prepareSearch("test")
- .setQuery(hasChildQuery)
- .addSort("_score", SortOrder.DESC).addSort("id", SortOrder.ASC).get();
- }
-
- public void testMinMaxChildren() throws Exception {
- assertAcked(prepareCreate("test")
- .setSettings("index.mapping.single_type", false)
- .addMapping("parent", "id", "type=long")
- .addMapping("child", "_parent", "type=parent"));
- ensureGreen();
-
- indexRandom(true, createMinMaxDocBuilders().toArray(new IndexRequestBuilder[0]));
- SearchResponse response;
-
- // Score mode = NONE
- response = minMaxQuery(ScoreMode.None, 0, null);
-
- assertThat(response.getHits().getTotalHits(), equalTo(3L));
- assertThat(response.getHits().getHits()[0].getId(), equalTo("2"));
- assertThat(response.getHits().getHits()[0].getScore(), equalTo(1f));
- assertThat(response.getHits().getHits()[1].getId(), equalTo("3"));
- assertThat(response.getHits().getHits()[1].getScore(), equalTo(1f));
- assertThat(response.getHits().getHits()[2].getId(), equalTo("4"));
- assertThat(response.getHits().getHits()[2].getScore(), equalTo(1f));
-
- response = minMaxQuery(ScoreMode.None, 1, null);
-
- assertThat(response.getHits().getTotalHits(), equalTo(3L));
- assertThat(response.getHits().getHits()[0].getId(), equalTo("2"));
- assertThat(response.getHits().getHits()[0].getScore(), equalTo(1f));
- assertThat(response.getHits().getHits()[1].getId(), equalTo("3"));
- assertThat(response.getHits().getHits()[1].getScore(), equalTo(1f));
- assertThat(response.getHits().getHits()[2].getId(), equalTo("4"));
- assertThat(response.getHits().getHits()[2].getScore(), equalTo(1f));
-
- response = minMaxQuery(ScoreMode.None, 2, null);
-
- assertThat(response.getHits().getTotalHits(), equalTo(2L));
- assertThat(response.getHits().getHits()[0].getId(), equalTo("3"));
- assertThat(response.getHits().getHits()[0].getScore(), equalTo(1f));
- assertThat(response.getHits().getHits()[1].getId(), equalTo("4"));
- assertThat(response.getHits().getHits()[1].getScore(), equalTo(1f));
-
- response = minMaxQuery(ScoreMode.None, 3, null);
-
- assertThat(response.getHits().getTotalHits(), equalTo(1L));
- assertThat(response.getHits().getHits()[0].getId(), equalTo("4"));
- assertThat(response.getHits().getHits()[0].getScore(), equalTo(1f));
-
- response = minMaxQuery(ScoreMode.None, 4, null);
-
- assertThat(response.getHits().getTotalHits(), equalTo(0L));
-
- response = minMaxQuery(ScoreMode.None, 0, 4);
-
- assertThat(response.getHits().getTotalHits(), equalTo(3L));
- assertThat(response.getHits().getHits()[0].getId(), equalTo("2"));
- assertThat(response.getHits().getHits()[0].getScore(), equalTo(1f));
- assertThat(response.getHits().getHits()[1].getId(), equalTo("3"));
- assertThat(response.getHits().getHits()[1].getScore(), equalTo(1f));
- assertThat(response.getHits().getHits()[2].getId(), equalTo("4"));
- assertThat(response.getHits().getHits()[2].getScore(), equalTo(1f));
-
- response = minMaxQuery(ScoreMode.None, 0, 3);
-
- assertThat(response.getHits().getTotalHits(), equalTo(3L));
- assertThat(response.getHits().getHits()[0].getId(), equalTo("2"));
- assertThat(response.getHits().getHits()[0].getScore(), equalTo(1f));
- assertThat(response.getHits().getHits()[1].getId(), equalTo("3"));
- assertThat(response.getHits().getHits()[1].getScore(), equalTo(1f));
- assertThat(response.getHits().getHits()[2].getId(), equalTo("4"));
- assertThat(response.getHits().getHits()[2].getScore(), equalTo(1f));
-
- response = minMaxQuery(ScoreMode.None, 0, 2);
-
- assertThat(response.getHits().getTotalHits(), equalTo(2L));
- assertThat(response.getHits().getHits()[0].getId(), equalTo("2"));
- assertThat(response.getHits().getHits()[0].getScore(), equalTo(1f));
- assertThat(response.getHits().getHits()[1].getId(), equalTo("3"));
- assertThat(response.getHits().getHits()[1].getScore(), equalTo(1f));
-
- response = minMaxQuery(ScoreMode.None, 2, 2);
-
- assertThat(response.getHits().getTotalHits(), equalTo(1L));
- assertThat(response.getHits().getHits()[0].getId(), equalTo("3"));
- assertThat(response.getHits().getHits()[0].getScore(), equalTo(1f));
-
- IllegalArgumentException e = expectThrows(IllegalArgumentException.class, () -> minMaxQuery(ScoreMode.None, 3, 2));
- assertThat(e.getMessage(), equalTo("[has_child] 'max_children' is less than 'min_children'"));
-
- // Score mode = SUM
- response = minMaxQuery(ScoreMode.Total, 0, null);
-
- assertThat(response.getHits().getTotalHits(), equalTo(3L));
- assertThat(response.getHits().getHits()[0].getId(), equalTo("4"));
- assertThat(response.getHits().getHits()[0].getScore(), equalTo(6f));
- assertThat(response.getHits().getHits()[1].getId(), equalTo("3"));
- assertThat(response.getHits().getHits()[1].getScore(), equalTo(3f));
- assertThat(response.getHits().getHits()[2].getId(), equalTo("2"));
- assertThat(response.getHits().getHits()[2].getScore(), equalTo(1f));
-
- response = minMaxQuery(ScoreMode.Total, 1, null);
-
- assertThat(response.getHits().getTotalHits(), equalTo(3L));
- assertThat(response.getHits().getHits()[0].getId(), equalTo("4"));
- assertThat(response.getHits().getHits()[0].getScore(), equalTo(6f));
- assertThat(response.getHits().getHits()[1].getId(), equalTo("3"));
- assertThat(response.getHits().getHits()[1].getScore(), equalTo(3f));
- assertThat(response.getHits().getHits()[2].getId(), equalTo("2"));
- assertThat(response.getHits().getHits()[2].getScore(), equalTo(1f));
-
- response = minMaxQuery(ScoreMode.Total, 2, null);
-
- assertThat(response.getHits().getTotalHits(), equalTo(2L));
- assertThat(response.getHits().getHits()[0].getId(), equalTo("4"));
- assertThat(response.getHits().getHits()[0].getScore(), equalTo(6f));
- assertThat(response.getHits().getHits()[1].getId(), equalTo("3"));
- assertThat(response.getHits().getHits()[1].getScore(), equalTo(3f));
-
- response = minMaxQuery(ScoreMode.Total, 3, null);
-
- assertThat(response.getHits().getTotalHits(), equalTo(1L));
- assertThat(response.getHits().getHits()[0].getId(), equalTo("4"));
- assertThat(response.getHits().getHits()[0].getScore(), equalTo(6f));
-
- response = minMaxQuery(ScoreMode.Total, 4, null);
-
- assertThat(response.getHits().getTotalHits(), equalTo(0L));
-
- response = minMaxQuery(ScoreMode.Total, 0, 4);
-
- assertThat(response.getHits().getTotalHits(), equalTo(3L));
- assertThat(response.getHits().getHits()[0].getId(), equalTo("4"));
- assertThat(response.getHits().getHits()[0].getScore(), equalTo(6f));
- assertThat(response.getHits().getHits()[1].getId(), equalTo("3"));
- assertThat(response.getHits().getHits()[1].getScore(), equalTo(3f));
- assertThat(response.getHits().getHits()[2].getId(), equalTo("2"));
- assertThat(response.getHits().getHits()[2].getScore(), equalTo(1f));
-
- response = minMaxQuery(ScoreMode.Total, 0, 3);
-
- assertThat(response.getHits().getTotalHits(), equalTo(3L));
- assertThat(response.getHits().getHits()[0].getId(), equalTo("4"));
- assertThat(response.getHits().getHits()[0].getScore(), equalTo(6f));
- assertThat(response.getHits().getHits()[1].getId(), equalTo("3"));
- assertThat(response.getHits().getHits()[1].getScore(), equalTo(3f));
- assertThat(response.getHits().getHits()[2].getId(), equalTo("2"));
- assertThat(response.getHits().getHits()[2].getScore(), equalTo(1f));
-
- response = minMaxQuery(ScoreMode.Total, 0, 2);
-
- assertThat(response.getHits().getTotalHits(), equalTo(2L));
- assertThat(response.getHits().getHits()[0].getId(), equalTo("3"));
- assertThat(response.getHits().getHits()[0].getScore(), equalTo(3f));
- assertThat(response.getHits().getHits()[1].getId(), equalTo("2"));
- assertThat(response.getHits().getHits()[1].getScore(), equalTo(1f));
-
- response = minMaxQuery(ScoreMode.Total, 2, 2);
-
- assertThat(response.getHits().getTotalHits(), equalTo(1L));
- assertThat(response.getHits().getHits()[0].getId(), equalTo("3"));
- assertThat(response.getHits().getHits()[0].getScore(), equalTo(3f));
-
- e = expectThrows(IllegalArgumentException.class, () -> minMaxQuery(ScoreMode.Total, 3, 2));
- assertThat(e.getMessage(), equalTo("[has_child] 'max_children' is less than 'min_children'"));
-
- // Score mode = MAX
- response = minMaxQuery(ScoreMode.Max, 0, null);
-
- assertThat(response.getHits().getTotalHits(), equalTo(3L));
- assertThat(response.getHits().getHits()[0].getId(), equalTo("4"));
- assertThat(response.getHits().getHits()[0].getScore(), equalTo(3f));
- assertThat(response.getHits().getHits()[1].getId(), equalTo("3"));
- assertThat(response.getHits().getHits()[1].getScore(), equalTo(2f));
- assertThat(response.getHits().getHits()[2].getId(), equalTo("2"));
- assertThat(response.getHits().getHits()[2].getScore(), equalTo(1f));
-
- response = minMaxQuery(ScoreMode.Max, 1, null);
-
- assertThat(response.getHits().getTotalHits(), equalTo(3L));
- assertThat(response.getHits().getHits()[0].getId(), equalTo("4"));
- assertThat(response.getHits().getHits()[0].getScore(), equalTo(3f));
- assertThat(response.getHits().getHits()[1].getId(), equalTo("3"));
- assertThat(response.getHits().getHits()[1].getScore(), equalTo(2f));
- assertThat(response.getHits().getHits()[2].getId(), equalTo("2"));
- assertThat(response.getHits().getHits()[2].getScore(), equalTo(1f));
-
- response = minMaxQuery(ScoreMode.Max, 2, null);
-
- assertThat(response.getHits().getTotalHits(), equalTo(2L));
- assertThat(response.getHits().getHits()[0].getId(), equalTo("4"));
- assertThat(response.getHits().getHits()[0].getScore(), equalTo(3f));
- assertThat(response.getHits().getHits()[1].getId(), equalTo("3"));
- assertThat(response.getHits().getHits()[1].getScore(), equalTo(2f));
-
- response = minMaxQuery(ScoreMode.Max, 3, null);
-
- assertThat(response.getHits().getTotalHits(), equalTo(1L));
- assertThat(response.getHits().getHits()[0].getId(), equalTo("4"));
- assertThat(response.getHits().getHits()[0].getScore(), equalTo(3f));
-
- response = minMaxQuery(ScoreMode.Max, 4, null);
-
- assertThat(response.getHits().getTotalHits(), equalTo(0L));
-
- response = minMaxQuery(ScoreMode.Max, 0, 4);
-
- assertThat(response.getHits().getTotalHits(), equalTo(3L));
- assertThat(response.getHits().getHits()[0].getId(), equalTo("4"));
- assertThat(response.getHits().getHits()[0].getScore(), equalTo(3f));
- assertThat(response.getHits().getHits()[1].getId(), equalTo("3"));
- assertThat(response.getHits().getHits()[1].getScore(), equalTo(2f));
- assertThat(response.getHits().getHits()[2].getId(), equalTo("2"));
- assertThat(response.getHits().getHits()[2].getScore(), equalTo(1f));
-
- response = minMaxQuery(ScoreMode.Max, 0, 3);
-
- assertThat(response.getHits().getTotalHits(), equalTo(3L));
- assertThat(response.getHits().getHits()[0].getId(), equalTo("4"));
- assertThat(response.getHits().getHits()[0].getScore(), equalTo(3f));
- assertThat(response.getHits().getHits()[1].getId(), equalTo("3"));
- assertThat(response.getHits().getHits()[1].getScore(), equalTo(2f));
- assertThat(response.getHits().getHits()[2].getId(), equalTo("2"));
- assertThat(response.getHits().getHits()[2].getScore(), equalTo(1f));
-
- response = minMaxQuery(ScoreMode.Max, 0, 2);
-
- assertThat(response.getHits().getTotalHits(), equalTo(2L));
- assertThat(response.getHits().getHits()[0].getId(), equalTo("3"));
- assertThat(response.getHits().getHits()[0].getScore(), equalTo(2f));
- assertThat(response.getHits().getHits()[1].getId(), equalTo("2"));
- assertThat(response.getHits().getHits()[1].getScore(), equalTo(1f));
-
- response = minMaxQuery(ScoreMode.Max, 2, 2);
-
- assertThat(response.getHits().getTotalHits(), equalTo(1L));
- assertThat(response.getHits().getHits()[0].getId(), equalTo("3"));
- assertThat(response.getHits().getHits()[0].getScore(), equalTo(2f));
-
- e = expectThrows(IllegalArgumentException.class, () -> minMaxQuery(ScoreMode.Max, 3, 2));
- assertThat(e.getMessage(), equalTo("[has_child] 'max_children' is less than 'min_children'"));
-
- // Score mode = AVG
- response = minMaxQuery(ScoreMode.Avg, 0, null);
-
- assertThat(response.getHits().getTotalHits(), equalTo(3L));
- assertThat(response.getHits().getHits()[0].getId(), equalTo("4"));
- assertThat(response.getHits().getHits()[0].getScore(), equalTo(2f));
- assertThat(response.getHits().getHits()[1].getId(), equalTo("3"));
- assertThat(response.getHits().getHits()[1].getScore(), equalTo(1.5f));
- assertThat(response.getHits().getHits()[2].getId(), equalTo("2"));
- assertThat(response.getHits().getHits()[2].getScore(), equalTo(1f));
-
- response = minMaxQuery(ScoreMode.Avg, 1, null);
-
- assertThat(response.getHits().getTotalHits(), equalTo(3L));
- assertThat(response.getHits().getHits()[0].getId(), equalTo("4"));
- assertThat(response.getHits().getHits()[0].getScore(), equalTo(2f));
- assertThat(response.getHits().getHits()[1].getId(), equalTo("3"));
- assertThat(response.getHits().getHits()[1].getScore(), equalTo(1.5f));
- assertThat(response.getHits().getHits()[2].getId(), equalTo("2"));
- assertThat(response.getHits().getHits()[2].getScore(), equalTo(1f));
-
- response = minMaxQuery(ScoreMode.Avg, 2, null);
-
- assertThat(response.getHits().getTotalHits(), equalTo(2L));
- assertThat(response.getHits().getHits()[0].getId(), equalTo("4"));
- assertThat(response.getHits().getHits()[0].getScore(), equalTo(2f));
- assertThat(response.getHits().getHits()[1].getId(), equalTo("3"));
- assertThat(response.getHits().getHits()[1].getScore(), equalTo(1.5f));
-
- response = minMaxQuery(ScoreMode.Avg, 3, null);
-
- assertThat(response.getHits().getTotalHits(), equalTo(1L));
- assertThat(response.getHits().getHits()[0].getId(), equalTo("4"));
- assertThat(response.getHits().getHits()[0].getScore(), equalTo(2f));
-
- response = minMaxQuery(ScoreMode.Avg, 4, null);
-
- assertThat(response.getHits().getTotalHits(), equalTo(0L));
-
- response = minMaxQuery(ScoreMode.Avg, 0, 4);
-
- assertThat(response.getHits().getTotalHits(), equalTo(3L));
- assertThat(response.getHits().getHits()[0].getId(), equalTo("4"));
- assertThat(response.getHits().getHits()[0].getScore(), equalTo(2f));
- assertThat(response.getHits().getHits()[1].getId(), equalTo("3"));
- assertThat(response.getHits().getHits()[1].getScore(), equalTo(1.5f));
- assertThat(response.getHits().getHits()[2].getId(), equalTo("2"));
- assertThat(response.getHits().getHits()[2].getScore(), equalTo(1f));
-
- response = minMaxQuery(ScoreMode.Avg, 0, 3);
-
- assertThat(response.getHits().getTotalHits(), equalTo(3L));
- assertThat(response.getHits().getHits()[0].getId(), equalTo("4"));
- assertThat(response.getHits().getHits()[0].getScore(), equalTo(2f));
- assertThat(response.getHits().getHits()[1].getId(), equalTo("3"));
- assertThat(response.getHits().getHits()[1].getScore(), equalTo(1.5f));
- assertThat(response.getHits().getHits()[2].getId(), equalTo("2"));
- assertThat(response.getHits().getHits()[2].getScore(), equalTo(1f));
-
- response = minMaxQuery(ScoreMode.Avg, 0, 2);
-
- assertThat(response.getHits().getTotalHits(), equalTo(2L));
- assertThat(response.getHits().getHits()[0].getId(), equalTo("3"));
- assertThat(response.getHits().getHits()[0].getScore(), equalTo(1.5f));
- assertThat(response.getHits().getHits()[1].getId(), equalTo("2"));
- assertThat(response.getHits().getHits()[1].getScore(), equalTo(1f));
-
- response = minMaxQuery(ScoreMode.Avg, 2, 2);
-
- assertThat(response.getHits().getTotalHits(), equalTo(1L));
- assertThat(response.getHits().getHits()[0].getId(), equalTo("3"));
- assertThat(response.getHits().getHits()[0].getScore(), equalTo(1.5f));
-
- e = expectThrows(IllegalArgumentException.class, () -> minMaxQuery(ScoreMode.Avg, 3, 2));
- assertThat(e.getMessage(), equalTo("[has_child] 'max_children' is less than 'min_children'"));
- }
-
- public void testParentFieldToNonExistingType() {
- assertAcked(prepareCreate("test")
- .setSettings("index.mapping.single_type", false)
- .addMapping("parent").addMapping("child", "_parent", "type=parent2"));
- client().prepareIndex("test", "parent", "1").setSource("{}", XContentType.JSON).get();
- client().prepareIndex("test", "child", "1").setParent("1").setSource("{}", XContentType.JSON).get();
- refresh();
-
- try {
- client().prepareSearch("test")
- .setQuery(QueryBuilders.hasChildQuery("child", matchAllQuery(), ScoreMode.None))
- .get();
- fail();
- } catch (SearchPhaseExecutionException e) {
- }
- }
-
- public void testHasParentInnerQueryType() {
- assertAcked(prepareCreate("test")
- .setSettings("index.mapping.single_type", false)
- .addMapping("parent-type").addMapping("child-type", "_parent", "type=parent-type"));
- client().prepareIndex("test", "child-type", "child-id").setParent("parent-id").setSource("{}", XContentType.JSON).get();
- client().prepareIndex("test", "parent-type", "parent-id").setSource("{}", XContentType.JSON).get();
- refresh();
- //make sure that when we explicitly set a type, the inner query is executed in the context of the parent type instead
- SearchResponse searchResponse = client().prepareSearch("test").setTypes("child-type").setQuery(
- QueryBuilders.hasParentQuery("parent-type", new IdsQueryBuilder().addIds("parent-id"), false)).get();
- assertSearchHits(searchResponse, "child-id");
- }
-
- public void testHasChildInnerQueryType() {
- assertAcked(prepareCreate("test")
- .setSettings("index.mapping.single_type", false)
- .addMapping("parent-type").addMapping("child-type", "_parent", "type=parent-type"));
- client().prepareIndex("test", "child-type", "child-id").setParent("parent-id").setSource("{}", XContentType.JSON).get();
- client().prepareIndex("test", "parent-type", "parent-id").setSource("{}", XContentType.JSON).get();
- refresh();
- //make sure that when we explicitly set a type, the inner query is executed in the context of the child type instead
- SearchResponse searchResponse = client().prepareSearch("test").setTypes("parent-type").setQuery(
- QueryBuilders.hasChildQuery("child-type", new IdsQueryBuilder().addIds("child-id"), ScoreMode.None)).get();
- assertSearchHits(searchResponse, "parent-id");
- }
-
- public void testHighlightersIgnoreParentChild() {
- assertAcked(prepareCreate("test")
- .setSettings("index.mapping.single_type", false)
- .addMapping("parent-type", "searchText", "type=text,term_vector=with_positions_offsets,index_options=offsets")
- .addMapping("child-type", "_parent", "type=parent-type", "searchText",
- "type=text,term_vector=with_positions_offsets,index_options=offsets"));
- client().prepareIndex("test", "parent-type", "parent-id").setSource("searchText", "quick brown fox").get();
- client().prepareIndex("test", "child-type", "child-id").setParent("parent-id").setSource("searchText", "quick brown fox").get();
- refresh();
-
- String[] highlightTypes = new String[] {"plain", "fvh", "postings"};
- for (String highlightType : highlightTypes) {
- logger.info("Testing with highlight type [{}]", highlightType);
- SearchResponse searchResponse = client().prepareSearch("test")
- .setQuery(new BoolQueryBuilder()
- .must(new MatchQueryBuilder("searchText", "fox"))
- .must(new HasChildQueryBuilder("child-type", new MatchAllQueryBuilder(), ScoreMode.None))
- )
- .highlighter(new HighlightBuilder().field(new HighlightBuilder.Field("searchText").highlighterType(highlightType)))
- .get();
- assertHitCount(searchResponse, 1);
- assertThat(searchResponse.getHits().getAt(0).getId(), equalTo("parent-id"));
- HighlightField highlightField = searchResponse.getHits().getAt(0).getHighlightFields().get("searchText");
- assertThat(highlightField.getFragments()[0].string(), equalTo("quick brown <em>fox</em>"));
-
- searchResponse = client().prepareSearch("test")
- .setQuery(new BoolQueryBuilder()
- .must(new MatchQueryBuilder("searchText", "fox"))
- .must(new HasParentQueryBuilder("parent-type", new MatchAllQueryBuilder(), false))
- )
- .highlighter(new HighlightBuilder().field(new HighlightBuilder.Field("searchText").highlighterType(highlightType)))
- .get();
- assertHitCount(searchResponse, 1);
- assertThat(searchResponse.getHits().getAt(0).getId(), equalTo("child-id"));
- highlightField = searchResponse.getHits().getAt(0).getHighlightFields().get("searchText");
- assertThat(highlightField.getFragments()[0].string(), equalTo("quick brown <em>fox</em>"));
- }
- }
-
-}
diff --git a/core/src/test/java/org/elasticsearch/search/fetch/subphase/InnerHitsIT.java b/core/src/test/java/org/elasticsearch/search/fetch/subphase/InnerHitsIT.java
index 6fbda92ba2..8eca505564 100644
--- a/core/src/test/java/org/elasticsearch/search/fetch/subphase/InnerHitsIT.java
+++ b/core/src/test/java/org/elasticsearch/search/fetch/subphase/InnerHitsIT.java
@@ -55,8 +55,6 @@ import static org.elasticsearch.common.xcontent.XContentFactory.jsonBuilder;
import static org.elasticsearch.common.xcontent.support.XContentMapValues.extractValue;
import static org.elasticsearch.index.query.QueryBuilders.boolQuery;
import static org.elasticsearch.index.query.QueryBuilders.constantScoreQuery;
-import static org.elasticsearch.index.query.QueryBuilders.hasChildQuery;
-import static org.elasticsearch.index.query.QueryBuilders.hasParentQuery;
import static org.elasticsearch.index.query.QueryBuilders.matchAllQuery;
import static org.elasticsearch.index.query.QueryBuilders.matchQuery;
import static org.elasticsearch.index.query.QueryBuilders.nestedQuery;
@@ -246,268 +244,6 @@ public class InnerHitsIT extends ESIntegTestCase {
}
}
- public void testSimpleParentChild() throws Exception {
- assertAcked(prepareCreate("articles")
- .setSettings("index.mapping.single_type", false)
- .addMapping("article", "title", "type=text")
- .addMapping("comment", "_parent", "type=article", "message", "type=text,fielddata=true")
- );
-
- List<IndexRequestBuilder> requests = new ArrayList<>();
- requests.add(client().prepareIndex("articles", "article", "1").setSource("title", "quick brown fox"));
- requests.add(client().prepareIndex("articles", "comment", "1").setParent("1").setSource("message", "fox eat quick"));
- requests.add(client().prepareIndex("articles", "comment", "2").setParent("1").setSource("message", "fox ate rabbit x y z"));
- requests.add(client().prepareIndex("articles", "comment", "3").setParent("1").setSource("message", "rabbit got away"));
- requests.add(client().prepareIndex("articles", "article", "2").setSource("title", "big gray elephant"));
- requests.add(client().prepareIndex("articles", "comment", "4").setParent("2").setSource("message", "elephant captured"));
- requests.add(client().prepareIndex("articles", "comment", "5").setParent("2").setSource("message", "mice squashed by elephant x"));
- requests.add(client().prepareIndex("articles", "comment", "6").setParent("2").setSource("message", "elephant scared by mice x y"));
- indexRandom(true, requests);
-
- SearchResponse response = client().prepareSearch("articles")
- .setQuery(hasChildQuery("comment", matchQuery("message", "fox"), ScoreMode.None)
- .innerHit(new InnerHitBuilder(), false))
- .get();
- assertNoFailures(response);
- assertHitCount(response, 1);
- assertSearchHit(response, 1, hasId("1"));
- assertThat(response.getHits().getAt(0).getShard(), notNullValue());
-
- assertThat(response.getHits().getAt(0).getInnerHits().size(), equalTo(1));
- SearchHits innerHits = response.getHits().getAt(0).getInnerHits().get("comment");
- assertThat(innerHits.getTotalHits(), equalTo(2L));
-
- assertThat(innerHits.getAt(0).getId(), equalTo("1"));
- assertThat(innerHits.getAt(0).getType(), equalTo("comment"));
- assertThat(innerHits.getAt(1).getId(), equalTo("2"));
- assertThat(innerHits.getAt(1).getType(), equalTo("comment"));
-
- response = client().prepareSearch("articles")
- .setQuery(hasChildQuery("comment", matchQuery("message", "elephant"), ScoreMode.None)
- .innerHit(new InnerHitBuilder(), false))
- .get();
- assertNoFailures(response);
- assertHitCount(response, 1);
- assertSearchHit(response, 1, hasId("2"));
-
- assertThat(response.getHits().getAt(0).getInnerHits().size(), equalTo(1));
- innerHits = response.getHits().getAt(0).getInnerHits().get("comment");
- assertThat(innerHits.getTotalHits(), equalTo(3L));
-
- assertThat(innerHits.getAt(0).getId(), equalTo("4"));
- assertThat(innerHits.getAt(0).getType(), equalTo("comment"));
- assertThat(innerHits.getAt(1).getId(), equalTo("5"));
- assertThat(innerHits.getAt(1).getType(), equalTo("comment"));
- assertThat(innerHits.getAt(2).getId(), equalTo("6"));
- assertThat(innerHits.getAt(2).getType(), equalTo("comment"));
-
- response = client().prepareSearch("articles")
- .setQuery(
- hasChildQuery("comment", matchQuery("message", "fox"), ScoreMode.None).innerHit(
- new InnerHitBuilder()
- .addDocValueField("message")
- .setHighlightBuilder(new HighlightBuilder().field("message"))
- .setExplain(true).setSize(1)
- .addScriptField("script", new Script(ScriptType.INLINE, MockScriptEngine.NAME, "5",
- Collections.emptyMap())),
- false)
- ).get();
- assertNoFailures(response);
- innerHits = response.getHits().getAt(0).getInnerHits().get("comment");
- assertThat(innerHits.getHits().length, equalTo(1));
- assertThat(innerHits.getAt(0).getHighlightFields().get("message").getFragments()[0].string(), equalTo("<em>fox</em> eat quick"));
- assertThat(innerHits.getAt(0).getExplanation().toString(), containsString("weight(message:fox"));
- assertThat(innerHits.getAt(0).getFields().get("message").getValue().toString(), equalTo("eat"));
- assertThat(innerHits.getAt(0).getFields().get("script").getValue().toString(), equalTo("5"));
- }
-
- public void testRandomParentChild() throws Exception {
- assertAcked(prepareCreate("idx")
- .setSettings("index.mapping.single_type", false)
- .addMapping("parent")
- .addMapping("child1", "_parent", "type=parent")
- .addMapping("child2", "_parent", "type=parent")
- );
- int numDocs = scaledRandomIntBetween(5, 50);
- List<IndexRequestBuilder> requestBuilders = new ArrayList<>();
-
- int child1 = 0;
- int child2 = 0;
- int[] child1InnerObjects = new int[numDocs];
- int[] child2InnerObjects = new int[numDocs];
- for (int parent = 0; parent < numDocs; parent++) {
- String parentId = String.format(Locale.ENGLISH, "%03d", parent);
- requestBuilders.add(client().prepareIndex("idx", "parent", parentId).setSource("{}", XContentType.JSON));
-
- int numChildDocs = child1InnerObjects[parent] = scaledRandomIntBetween(1, numDocs);
- int limit = child1 + numChildDocs;
- for (; child1 < limit; child1++) {
- requestBuilders.add(client().prepareIndex("idx", "child1",
- String.format(Locale.ENGLISH, "%04d", child1)).setParent(parentId).setSource("{}", XContentType.JSON));
- }
- numChildDocs = child2InnerObjects[parent] = scaledRandomIntBetween(1, numDocs);
- limit = child2 + numChildDocs;
- for (; child2 < limit; child2++) {
- requestBuilders.add(client().prepareIndex("idx", "child2",
- String.format(Locale.ENGLISH, "%04d", child2)).setParent(parentId).setSource("{}", XContentType.JSON));
- }
- }
- indexRandom(true, requestBuilders);
-
- int size = randomIntBetween(0, numDocs);
- BoolQueryBuilder boolQuery = new BoolQueryBuilder();
- boolQuery.should(constantScoreQuery(hasChildQuery("child1", matchAllQuery(), ScoreMode.None)
- .innerHit(new InnerHitBuilder().setName("a")
- .addSort(new FieldSortBuilder("_uid").order(SortOrder.ASC)).setSize(size), false)));
- boolQuery.should(constantScoreQuery(hasChildQuery("child2", matchAllQuery(), ScoreMode.None)
- .innerHit(new InnerHitBuilder().setName("b")
- .addSort(new FieldSortBuilder("_uid").order(SortOrder.ASC)).setSize(size), false)));
- SearchResponse searchResponse = client().prepareSearch("idx")
- .setSize(numDocs)
- .setTypes("parent")
- .addSort("_uid", SortOrder.ASC)
- .setQuery(boolQuery)
- .get();
-
- assertNoFailures(searchResponse);
- assertHitCount(searchResponse, numDocs);
- assertThat(searchResponse.getHits().getHits().length, equalTo(numDocs));
-
- int offset1 = 0;
- int offset2 = 0;
- for (int parent = 0; parent < numDocs; parent++) {
- SearchHit searchHit = searchResponse.getHits().getAt(parent);
- assertThat(searchHit.getType(), equalTo("parent"));
- assertThat(searchHit.getId(), equalTo(String.format(Locale.ENGLISH, "%03d", parent)));
- assertThat(searchHit.getShard(), notNullValue());
-
- SearchHits inner = searchHit.getInnerHits().get("a");
- assertThat(inner.getTotalHits(), equalTo((long) child1InnerObjects[parent]));
- for (int child = 0; child < child1InnerObjects[parent] && child < size; child++) {
- SearchHit innerHit = inner.getAt(child);
- assertThat(innerHit.getType(), equalTo("child1"));
- String childId = String.format(Locale.ENGLISH, "%04d", offset1 + child);
- assertThat(innerHit.getId(), equalTo(childId));
- assertThat(innerHit.getNestedIdentity(), nullValue());
- }
- offset1 += child1InnerObjects[parent];
-
- inner = searchHit.getInnerHits().get("b");
- assertThat(inner.getTotalHits(), equalTo((long) child2InnerObjects[parent]));
- for (int child = 0; child < child2InnerObjects[parent] && child < size; child++) {
- SearchHit innerHit = inner.getAt(child);
- assertThat(innerHit.getType(), equalTo("child2"));
- String childId = String.format(Locale.ENGLISH, "%04d", offset2 + child);
- assertThat(innerHit.getId(), equalTo(childId));
- assertThat(innerHit.getNestedIdentity(), nullValue());
- }
- offset2 += child2InnerObjects[parent];
- }
- }
-
- public void testInnerHitsOnHasParent() throws Exception {
- assertAcked(prepareCreate("stack")
- .setSettings("index.mapping.single_type", false)
- .addMapping("question", "body", "type=text")
- .addMapping("answer", "_parent", "type=question", "body", "type=text")
- );
- List<IndexRequestBuilder> requests = new ArrayList<>();
- requests.add(client().prepareIndex("stack", "question", "1").setSource("body", "I'm using HTTPS + Basic authentication "
- + "to protect a resource. How can I throttle authentication attempts to protect against brute force attacks?"));
- requests.add(client().prepareIndex("stack", "answer", "1").setParent("1").setSource("body",
- "install fail2ban and enable rules for apache"));
- requests.add(client().prepareIndex("stack", "question", "2").setSource("body",
- "I have firewall rules set up and also denyhosts installed.\\ndo I also need to install fail2ban?"));
- requests.add(client().prepareIndex("stack", "answer", "2").setParent("2").setSource("body",
- "Denyhosts protects only ssh; Fail2Ban protects all daemons."));
- indexRandom(true, requests);
-
- SearchResponse response = client().prepareSearch("stack")
- .setTypes("answer")
- .addSort("_uid", SortOrder.ASC)
- .setQuery(
- boolQuery()
- .must(matchQuery("body", "fail2ban"))
- .must(hasParentQuery("question", matchAllQuery(), false).innerHit(new InnerHitBuilder(), false))
- ).get();
- assertNoFailures(response);
- assertHitCount(response, 2);
-
- SearchHit searchHit = response.getHits().getAt(0);
- assertThat(searchHit.getId(), equalTo("1"));
- assertThat(searchHit.getType(), equalTo("answer"));
- assertThat(searchHit.getInnerHits().get("question").getTotalHits(), equalTo(1L));
- assertThat(searchHit.getInnerHits().get("question").getAt(0).getType(), equalTo("question"));
- assertThat(searchHit.getInnerHits().get("question").getAt(0).getId(), equalTo("1"));
-
- searchHit = response.getHits().getAt(1);
- assertThat(searchHit.getId(), equalTo("2"));
- assertThat(searchHit.getType(), equalTo("answer"));
- assertThat(searchHit.getInnerHits().get("question").getTotalHits(), equalTo(1L));
- assertThat(searchHit.getInnerHits().get("question").getAt(0).getType(), equalTo("question"));
- assertThat(searchHit.getInnerHits().get("question").getAt(0).getId(), equalTo("2"));
- }
-
- public void testParentChildMultipleLayers() throws Exception {
- assertAcked(prepareCreate("articles")
- .setSettings("index.mapping.single_type", false)
- .addMapping("article", "title", "type=text")
- .addMapping("comment", "_parent", "type=article", "message", "type=text")
- .addMapping("remark", "_parent", "type=comment", "message", "type=text")
- );
-
- List<IndexRequestBuilder> requests = new ArrayList<>();
- requests.add(client().prepareIndex("articles", "article", "1").setSource("title", "quick brown fox"));
- requests.add(client().prepareIndex("articles", "comment", "1").setParent("1").setSource("message", "fox eat quick"));
- requests.add(client().prepareIndex("articles", "remark", "1").setParent("1").setRouting("1").setSource("message", "good"));
- requests.add(client().prepareIndex("articles", "article", "2").setSource("title", "big gray elephant"));
- requests.add(client().prepareIndex("articles", "comment", "2").setParent("2").setSource("message", "elephant captured"));
- requests.add(client().prepareIndex("articles", "remark", "2").setParent("2").setRouting("2").setSource("message", "bad"));
- indexRandom(true, requests);
-
- SearchResponse response = client().prepareSearch("articles")
- .setQuery(hasChildQuery("comment",
- hasChildQuery("remark", matchQuery("message", "good"), ScoreMode.None).innerHit(new InnerHitBuilder(), false),
- ScoreMode.None).innerHit(new InnerHitBuilder(), false))
- .get();
-
- assertNoFailures(response);
- assertHitCount(response, 1);
- assertSearchHit(response, 1, hasId("1"));
-
- assertThat(response.getHits().getAt(0).getInnerHits().size(), equalTo(1));
- SearchHits innerHits = response.getHits().getAt(0).getInnerHits().get("comment");
- assertThat(innerHits.getTotalHits(), equalTo(1L));
- assertThat(innerHits.getAt(0).getId(), equalTo("1"));
- assertThat(innerHits.getAt(0).getType(), equalTo("comment"));
-
- innerHits = innerHits.getAt(0).getInnerHits().get("remark");
- assertThat(innerHits.getTotalHits(), equalTo(1L));
- assertThat(innerHits.getAt(0).getId(), equalTo("1"));
- assertThat(innerHits.getAt(0).getType(), equalTo("remark"));
-
- response = client().prepareSearch("articles")
- .setQuery(hasChildQuery("comment",
- hasChildQuery("remark", matchQuery("message", "bad"), ScoreMode.None).innerHit(new InnerHitBuilder(), false),
- ScoreMode.None).innerHit(new InnerHitBuilder(), false))
- .get();
-
- assertNoFailures(response);
- assertHitCount(response, 1);
- assertSearchHit(response, 1, hasId("2"));
-
- assertThat(response.getHits().getAt(0).getInnerHits().size(), equalTo(1));
- innerHits = response.getHits().getAt(0).getInnerHits().get("comment");
- assertThat(innerHits.getTotalHits(), equalTo(1L));
- assertThat(innerHits.getAt(0).getId(), equalTo("2"));
- assertThat(innerHits.getAt(0).getType(), equalTo("comment"));
-
- innerHits = innerHits.getAt(0).getInnerHits().get("remark");
- assertThat(innerHits.getTotalHits(), equalTo(1L));
- assertThat(innerHits.getAt(0).getId(), equalTo("2"));
- assertThat(innerHits.getAt(0).getType(), equalTo("remark"));
- }
-
public void testNestedMultipleLayers() throws Exception {
assertAcked(prepareCreate("articles").addMapping("article", jsonBuilder().startObject()
.startObject("article").startObject("properties")
@@ -724,92 +460,6 @@ public class InnerHitsIT extends ESIntegTestCase {
assertThat(messages.getAt(0).getNestedIdentity().getChild(), nullValue());
}
- public void testRoyals() throws Exception {
- assertAcked(
- prepareCreate("royals")
- .setSettings("index.mapping.single_type", false)
- .addMapping("king")
- .addMapping("prince", "_parent", "type=king")
- .addMapping("duke", "_parent", "type=prince")
- .addMapping("earl", "_parent", "type=duke")
- .addMapping("baron", "_parent", "type=earl")
- );
-
- List<IndexRequestBuilder> requests = new ArrayList<>();
- requests.add(client().prepareIndex("royals", "king", "king").setSource("{}", XContentType.JSON));
- requests.add(client().prepareIndex("royals", "prince", "prince").setParent("king").setSource("{}", XContentType.JSON));
- requests.add(client().prepareIndex("royals", "duke", "duke").setParent("prince").setRouting("king")
- .setSource("{}", XContentType.JSON));
- requests.add(client().prepareIndex("royals", "earl", "earl1").setParent("duke").setRouting("king")
- .setSource("{}", XContentType.JSON));
- requests.add(client().prepareIndex("royals", "earl", "earl2").setParent("duke").setRouting("king")
- .setSource("{}", XContentType.JSON));
- requests.add(client().prepareIndex("royals", "earl", "earl3").setParent("duke").setRouting("king")
- .setSource("{}", XContentType.JSON));
- requests.add(client().prepareIndex("royals", "earl", "earl4").setParent("duke").setRouting("king")
- .setSource("{}", XContentType.JSON));
- requests.add(client().prepareIndex("royals", "baron", "baron1").setParent("earl1").setRouting("king")
- .setSource("{}", XContentType.JSON));
- requests.add(client().prepareIndex("royals", "baron", "baron2").setParent("earl2").setRouting("king")
- .setSource("{}", XContentType.JSON));
- requests.add(client().prepareIndex("royals", "baron", "baron3").setParent("earl3").setRouting("king")
- .setSource("{}", XContentType.JSON));
- requests.add(client().prepareIndex("royals", "baron", "baron4").setParent("earl4").setRouting("king")
- .setSource("{}", XContentType.JSON));
- indexRandom(true, requests);
-
- SearchResponse response = client().prepareSearch("royals")
- .setTypes("duke")
- .setQuery(boolQuery()
- .filter(hasParentQuery("prince",
- hasParentQuery("king", matchAllQuery(), false).innerHit(new InnerHitBuilder().setName("kings"), false),
- false).innerHit(new InnerHitBuilder().setName("princes"), false)
- )
- .filter(hasChildQuery("earl",
- hasChildQuery("baron", matchAllQuery(), ScoreMode.None)
- .innerHit(new InnerHitBuilder().setName("barons"), false),
- ScoreMode.None).innerHit(new InnerHitBuilder()
- .addSort(SortBuilders.fieldSort("_uid").order(SortOrder.ASC))
- .setName("earls")
- .setSize(4), false)
- )
- )
- .get();
- assertHitCount(response, 1);
- assertThat(response.getHits().getAt(0).getId(), equalTo("duke"));
-
- SearchHits innerHits = response.getHits().getAt(0).getInnerHits().get("earls");
- assertThat(innerHits.getTotalHits(), equalTo(4L));
- assertThat(innerHits.getAt(0).getId(), equalTo("earl1"));
- assertThat(innerHits.getAt(1).getId(), equalTo("earl2"));
- assertThat(innerHits.getAt(2).getId(), equalTo("earl3"));
- assertThat(innerHits.getAt(3).getId(), equalTo("earl4"));
-
- SearchHits innerInnerHits = innerHits.getAt(0).getInnerHits().get("barons");
- assertThat(innerInnerHits.getTotalHits(), equalTo(1L));
- assertThat(innerInnerHits.getAt(0).getId(), equalTo("baron1"));
-
- innerInnerHits = innerHits.getAt(1).getInnerHits().get("barons");
- assertThat(innerInnerHits.getTotalHits(), equalTo(1L));
- assertThat(innerInnerHits.getAt(0).getId(), equalTo("baron2"));
-
- innerInnerHits = innerHits.getAt(2).getInnerHits().get("barons");
- assertThat(innerInnerHits.getTotalHits(), equalTo(1L));
- assertThat(innerInnerHits.getAt(0).getId(), equalTo("baron3"));
-
- innerInnerHits = innerHits.getAt(3).getInnerHits().get("barons");
- assertThat(innerInnerHits.getTotalHits(), equalTo(1L));
- assertThat(innerInnerHits.getAt(0).getId(), equalTo("baron4"));
-
- innerHits = response.getHits().getAt(0).getInnerHits().get("princes");
- assertThat(innerHits.getTotalHits(), equalTo(1L));
- assertThat(innerHits.getAt(0).getId(), equalTo("prince"));
-
- innerInnerHits = innerHits.getAt(0).getInnerHits().get("kings");
- assertThat(innerInnerHits.getTotalHits(), equalTo(1L));
- assertThat(innerInnerHits.getAt(0).getId(), equalTo("king"));
- }
-
public void testMatchesQueriesNestedInnerHits() throws Exception {
XContentBuilder builder = jsonBuilder().startObject()
.startObject("type1")
@@ -914,84 +564,6 @@ public class InnerHitsIT extends ESIntegTestCase {
}
}
- public void testMatchesQueriesParentChildInnerHits() throws Exception {
- assertAcked(prepareCreate("index")
- .setSettings("index.mapping.single_type", false)
- .addMapping("child", "_parent", "type=parent"));
- List<IndexRequestBuilder> requests = new ArrayList<>();
- requests.add(client().prepareIndex("index", "parent", "1").setSource("{}", XContentType.JSON));
- requests.add(client().prepareIndex("index", "child", "1").setParent("1").setSource("field", "value1"));
- requests.add(client().prepareIndex("index", "child", "2").setParent("1").setSource("field", "value2"));
- requests.add(client().prepareIndex("index", "parent", "2").setSource("{}", XContentType.JSON));
- requests.add(client().prepareIndex("index", "child", "3").setParent("2").setSource("field", "value1"));
- indexRandom(true, requests);
-
- SearchResponse response = client().prepareSearch("index")
- .setQuery(hasChildQuery("child", matchQuery("field", "value1").queryName("_name1"), ScoreMode.None)
- .innerHit(new InnerHitBuilder(), false))
- .addSort("_uid", SortOrder.ASC)
- .get();
- assertHitCount(response, 2);
- assertThat(response.getHits().getAt(0).getId(), equalTo("1"));
- assertThat(response.getHits().getAt(0).getInnerHits().get("child").getTotalHits(), equalTo(1L));
- assertThat(response.getHits().getAt(0).getInnerHits().get("child").getAt(0).getMatchedQueries().length, equalTo(1));
- assertThat(response.getHits().getAt(0).getInnerHits().get("child").getAt(0).getMatchedQueries()[0], equalTo("_name1"));
-
- assertThat(response.getHits().getAt(1).getId(), equalTo("2"));
- assertThat(response.getHits().getAt(1).getInnerHits().get("child").getTotalHits(), equalTo(1L));
- assertThat(response.getHits().getAt(1).getInnerHits().get("child").getAt(0).getMatchedQueries().length, equalTo(1));
- assertThat(response.getHits().getAt(1).getInnerHits().get("child").getAt(0).getMatchedQueries()[0], equalTo("_name1"));
-
- QueryBuilder query = hasChildQuery("child", matchQuery("field", "value2").queryName("_name2"), ScoreMode.None)
- .innerHit(new InnerHitBuilder(), false);
- response = client().prepareSearch("index")
- .setQuery(query)
- .addSort("_uid", SortOrder.ASC)
- .get();
- assertHitCount(response, 1);
- assertThat(response.getHits().getAt(0).getId(), equalTo("1"));
- assertThat(response.getHits().getAt(0).getInnerHits().get("child").getTotalHits(), equalTo(1L));
- assertThat(response.getHits().getAt(0).getInnerHits().get("child").getAt(0).getMatchedQueries().length, equalTo(1));
- assertThat(response.getHits().getAt(0).getInnerHits().get("child").getAt(0).getMatchedQueries()[0], equalTo("_name2"));
- }
-
- public void testDontExplode() throws Exception {
- assertAcked(prepareCreate("index1")
- .setSettings("index.mapping.single_type", false)
- .addMapping("child", "_parent", "type=parent"));
- List<IndexRequestBuilder> requests = new ArrayList<>();
- requests.add(client().prepareIndex("index1", "parent", "1").setSource("{}", XContentType.JSON));
- requests.add(client().prepareIndex("index1", "child", "1").setParent("1").setSource("field", "value1"));
- indexRandom(true, requests);
-
- QueryBuilder query = hasChildQuery("child", matchQuery("field", "value1"), ScoreMode.None)
- .innerHit(new InnerHitBuilder().setSize(ArrayUtil.MAX_ARRAY_LENGTH - 1), false);
- SearchResponse response = client().prepareSearch("index1")
- .setQuery(query)
- .get();
- assertNoFailures(response);
- assertHitCount(response, 1);
-
- assertAcked(prepareCreate("index2").addMapping("type", "nested", "type=nested"));
- client().prepareIndex("index2", "type", "1").setSource(jsonBuilder().startObject()
- .startArray("nested")
- .startObject()
- .field("field", "value1")
- .endObject()
- .endArray()
- .endObject())
- .setRefreshPolicy(IMMEDIATE)
- .get();
-
- query = nestedQuery("nested", matchQuery("nested.field", "value1"), ScoreMode.Avg)
- .innerHit(new InnerHitBuilder().setSize(ArrayUtil.MAX_ARRAY_LENGTH - 1), false);
- response = client().prepareSearch("index2")
- .setQuery(query)
- .get();
- assertNoFailures(response);
- assertHitCount(response, 1);
- }
-
public void testNestedSourceFiltering() throws Exception {
assertAcked(prepareCreate("index1").addMapping("message", "comments", "type=nested"));
client().prepareIndex("index1", "message", "1").setSource(jsonBuilder().startObject()
@@ -1021,25 +593,6 @@ public class InnerHitsIT extends ESIntegTestCase {
equalTo("fox ate rabbit x y z"));
}
- public void testNestedInnerHitWrappedInParentChildInnerhit() throws Exception {
- assertAcked(prepareCreate("test")
- .setSettings("index.mapping.single_type", false)
- .addMapping("child_type", "_parent", "type=parent_type", "nested_type", "type=nested"));
- client().prepareIndex("test", "parent_type", "1").setSource("key", "value").get();
- client().prepareIndex("test", "child_type", "2").setParent("1").setSource("nested_type", Collections.singletonMap("key", "value"))
- .get();
- refresh();
- SearchResponse response = client().prepareSearch("test")
- .setQuery(boolQuery().must(matchQuery("key", "value"))
- .should(hasChildQuery("child_type", nestedQuery("nested_type", matchAllQuery(), ScoreMode.None)
- .innerHit(new InnerHitBuilder(), false), ScoreMode.None).innerHit(new InnerHitBuilder(), false)))
- .get();
- assertHitCount(response, 1);
- SearchHit hit = response.getHits().getAt(0);
- assertThat(hit.getInnerHits().get("child_type").getAt(0).field("_parent").getValue(), equalTo("1"));
- assertThat(hit.getInnerHits().get("child_type").getAt(0).getInnerHits().get("nested_type").getAt(0).field("_parent"), nullValue());
- }
-
public void testInnerHitsWithIgnoreUnmapped() throws Exception {
assertAcked(prepareCreate("index1")
.setSettings("index.mapping.single_type", false)
@@ -1062,17 +615,6 @@ public class InnerHitsIT extends ESIntegTestCase {
assertNoFailures(response);
assertHitCount(response, 2);
assertSearchHits(response, "1", "3");
-
- response = client().prepareSearch("index1", "index2")
- .setQuery(boolQuery()
- .should(hasChildQuery("child_type", matchAllQuery(), ScoreMode.None).ignoreUnmapped(true)
- .innerHit(new InnerHitBuilder(), true))
- .should(termQuery("key", "value"))
- )
- .get();
- assertNoFailures(response);
- assertHitCount(response, 2);
- assertSearchHits(response, "1", "3");
}
}
diff --git a/core/src/test/java/org/elasticsearch/test/NoopDiscovery.java b/core/src/test/java/org/elasticsearch/test/NoopDiscovery.java
index 5b33fb1c8d..bcb9efd5b5 100644
--- a/core/src/test/java/org/elasticsearch/test/NoopDiscovery.java
+++ b/core/src/test/java/org/elasticsearch/test/NoopDiscovery.java
@@ -19,8 +19,6 @@
package org.elasticsearch.test;
import org.elasticsearch.cluster.ClusterChangedEvent;
-import org.elasticsearch.cluster.ClusterState;
-import org.elasticsearch.cluster.routing.allocation.AllocationService;
import org.elasticsearch.common.component.Lifecycle;
import org.elasticsearch.common.component.LifecycleListener;
import org.elasticsearch.discovery.Discovery;
@@ -34,16 +32,6 @@ public class NoopDiscovery implements Discovery {
}
@Override
- public ClusterState getInitialClusterState() {
- return null;
- }
-
- @Override
- public ClusterState clusterState() {
- return null;
- }
-
- @Override
public DiscoveryStats stats() {
return null;
}
@@ -54,11 +42,6 @@ public class NoopDiscovery implements Discovery {
}
@Override
- public int getMinimumMasterNodes() {
- return -1;
- }
-
- @Override
public Lifecycle.State lifecycleState() {
return null;
}
diff --git a/core/src/test/java/org/elasticsearch/transport/TCPTransportTests.java b/core/src/test/java/org/elasticsearch/transport/TCPTransportTests.java
index c14d6ec9e0..eb9e649652 100644
--- a/core/src/test/java/org/elasticsearch/transport/TCPTransportTests.java
+++ b/core/src/test/java/org/elasticsearch/transport/TCPTransportTests.java
@@ -204,7 +204,7 @@ public class TCPTransportTests extends ESTestCase {
@Override
protected NodeChannels connectToChannels(DiscoveryNode node, ConnectionProfile profile) throws IOException {
- return new NodeChannels(node, new Object[profile.getNumConnections()], profile);
+ return new NodeChannels(node, new Object[profile.getNumConnections()], profile, c -> {});
}
@Override
@@ -220,7 +220,7 @@ public class TCPTransportTests extends ESTestCase {
@Override
public NodeChannels getConnection(DiscoveryNode node) {
return new NodeChannels(node, new Object[MockTcpTransport.LIGHT_PROFILE.getNumConnections()],
- MockTcpTransport.LIGHT_PROFILE);
+ MockTcpTransport.LIGHT_PROFILE, c -> {});
}
};
DiscoveryNode node = new DiscoveryNode("foo", buildNewFakeTransportAddress(), Version.CURRENT);
diff --git a/core/src/test/resources/config/elasticsearch.properties b/core/src/test/resources/config/elasticsearch.properties
deleted file mode 100644
index d3f822cafb..0000000000
--- a/core/src/test/resources/config/elasticsearch.properties
+++ /dev/null
@@ -1,2 +0,0 @@
-
-properties.config.exists: true
diff --git a/core/src/test/resources/config/elasticsearch.yaml b/core/src/test/resources/config/elasticsearch.yml
index b6ebc6bd10..b6ebc6bd10 100644
--- a/core/src/test/resources/config/elasticsearch.yaml
+++ b/core/src/test/resources/config/elasticsearch.yml