diff --git a/buildSrc/src/main/groovy/org/elasticsearch/gradle/test/ClusterFormationTasks.groovy b/buildSrc/src/main/groovy/org/elasticsearch/gradle/test/ClusterFormationTasks.groovy index 4b7c05ec2eba..0d116a32712c 100644 --- a/buildSrc/src/main/groovy/org/elasticsearch/gradle/test/ClusterFormationTasks.groovy +++ b/buildSrc/src/main/groovy/org/elasticsearch/gradle/test/ClusterFormationTasks.groovy @@ -46,9 +46,9 @@ class ClusterFormationTasks { /** * Adds dependent tasks to the given task to start and stop a cluster with the given configuration. * - * Returns an object that will resolve at execution time of the given task to a uri for the cluster. + * Returns a NodeInfo object for the first node in the cluster. */ - static Object setup(Project project, Task task, ClusterConfiguration config) { + static NodeInfo setup(Project project, Task task, ClusterConfiguration config) { if (task.getEnabled() == false) { // no need to add cluster formation tasks if the task won't run! return @@ -66,7 +66,7 @@ class ClusterFormationTasks { task.dependsOn(wait) // delay the resolution of the uri by wrapping in a closure, so it is not used until read for tests - return "${-> nodes[0].transportUri()}" + return nodes[0] } /** Adds a dependency on the given distribution */ diff --git a/buildSrc/src/main/groovy/org/elasticsearch/gradle/test/RestIntegTestTask.groovy b/buildSrc/src/main/groovy/org/elasticsearch/gradle/test/RestIntegTestTask.groovy index 5656be57b8f5..3bfe9d610188 100644 --- a/buildSrc/src/main/groovy/org/elasticsearch/gradle/test/RestIntegTestTask.groovy +++ b/buildSrc/src/main/groovy/org/elasticsearch/gradle/test/RestIntegTestTask.groovy @@ -20,7 +20,6 @@ package org.elasticsearch.gradle.test import com.carrotsearch.gradle.junit4.RandomizedTestingTask import org.elasticsearch.gradle.BuildPlugin -import org.gradle.api.GradleException import org.gradle.api.Task import org.gradle.api.internal.tasks.options.Option import org.gradle.api.plugins.JavaBasePlugin @@ -61,8 +60,12 @@ public class RestIntegTestTask extends RandomizedTestingTask { // this must run after all projects have been configured, so we know any project // references can be accessed as a fully configured project.gradle.projectsEvaluated { - Object clusterUri = ClusterFormationTasks.setup(project, this, clusterConfig) - systemProperty('tests.cluster', clusterUri) + NodeInfo node = ClusterFormationTasks.setup(project, this, clusterConfig) + systemProperty('tests.rest.cluster', "${-> node.httpUri()}") + // TODO: our "client" qa tests currently use the rest-test plugin. instead they should have their own plugin + // that sets up the test cluster and passes this transport uri instead of http uri. Until then, we pass + // both as separate sysprops + systemProperty('tests.cluster', "${-> node.transportUri()}") } } diff --git a/core/src/main/java/org/elasticsearch/action/ActionModule.java b/core/src/main/java/org/elasticsearch/action/ActionModule.java index 5f1a181fabb0..67f256c6bd1e 100644 --- a/core/src/main/java/org/elasticsearch/action/ActionModule.java +++ b/core/src/main/java/org/elasticsearch/action/ActionModule.java @@ -200,7 +200,7 @@ public class ActionModule extends AbstractModule { private final Map actions = new HashMap<>(); private final List> actionFilters = new ArrayList<>(); - static class ActionEntry { + static class ActionEntry, Response extends ActionResponse> { public final GenericAction action; public final Class> transportAction; public final Class[] supportTransportActions; @@ -229,7 +229,7 @@ public class ActionModule extends AbstractModule { * @param The request type. * @param The response type. */ - public void registerAction(GenericAction action, Class> transportAction, Class... supportTransportActions) { + public , Response extends ActionResponse> void registerAction(GenericAction action, Class> transportAction, Class... supportTransportActions) { actions.put(action.name(), new ActionEntry<>(action, transportAction, supportTransportActions)); } diff --git a/core/src/main/java/org/elasticsearch/action/admin/indices/refresh/TransportRefreshAction.java b/core/src/main/java/org/elasticsearch/action/admin/indices/refresh/TransportRefreshAction.java index a76b714b31de..aaaf11e45349 100644 --- a/core/src/main/java/org/elasticsearch/action/admin/indices/refresh/TransportRefreshAction.java +++ b/core/src/main/java/org/elasticsearch/action/admin/indices/refresh/TransportRefreshAction.java @@ -22,7 +22,7 @@ package org.elasticsearch.action.admin.indices.refresh; import org.elasticsearch.action.ReplicationResponse; import org.elasticsearch.action.ShardOperationFailedException; import org.elasticsearch.action.support.ActionFilters; -import org.elasticsearch.action.support.replication.ReplicationRequest; +import org.elasticsearch.action.support.replication.BasicReplicationRequest; import org.elasticsearch.action.support.replication.TransportBroadcastReplicationAction; import org.elasticsearch.cluster.ClusterService; import org.elasticsearch.cluster.metadata.IndexNameExpressionResolver; @@ -37,7 +37,7 @@ import java.util.List; /** * Refresh action. */ -public class TransportRefreshAction extends TransportBroadcastReplicationAction { +public class TransportRefreshAction extends TransportBroadcastReplicationAction { @Inject public TransportRefreshAction(Settings settings, ThreadPool threadPool, ClusterService clusterService, @@ -53,8 +53,8 @@ public class TransportRefreshAction extends TransportBroadcastReplicationAction< } @Override - protected ReplicationRequest newShardRequest(RefreshRequest request, ShardId shardId) { - return new ReplicationRequest(request, shardId); + protected BasicReplicationRequest newShardRequest(RefreshRequest request, ShardId shardId) { + return new BasicReplicationRequest(request, shardId); } @Override diff --git a/core/src/main/java/org/elasticsearch/action/admin/indices/refresh/TransportShardRefreshAction.java b/core/src/main/java/org/elasticsearch/action/admin/indices/refresh/TransportShardRefreshAction.java index c78977fb3627..7c9979e7374f 100644 --- a/core/src/main/java/org/elasticsearch/action/admin/indices/refresh/TransportShardRefreshAction.java +++ b/core/src/main/java/org/elasticsearch/action/admin/indices/refresh/TransportShardRefreshAction.java @@ -21,7 +21,7 @@ package org.elasticsearch.action.admin.indices.refresh; import org.elasticsearch.action.ReplicationResponse; import org.elasticsearch.action.support.ActionFilters; -import org.elasticsearch.action.support.replication.ReplicationRequest; +import org.elasticsearch.action.support.replication.BasicReplicationRequest; import org.elasticsearch.action.support.replication.TransportReplicationAction; import org.elasticsearch.cluster.ClusterService; import org.elasticsearch.cluster.action.index.MappingUpdatedAction; @@ -41,7 +41,7 @@ import org.elasticsearch.transport.TransportService; /** * */ -public class TransportShardRefreshAction extends TransportReplicationAction { +public class TransportShardRefreshAction extends TransportReplicationAction { public static final String NAME = RefreshAction.NAME + "[s]"; @@ -51,7 +51,7 @@ public class TransportShardRefreshAction extends TransportReplicationAction shardOperationOnPrimary(MetaData metaData, ReplicationRequest shardRequest) throws Throwable { + protected Tuple shardOperationOnPrimary(MetaData metaData, BasicReplicationRequest shardRequest) throws Throwable { IndexShard indexShard = indicesService.indexServiceSafe(shardRequest.shardId().getIndex()).getShard(shardRequest.shardId().id()); indexShard.refresh("api"); logger.trace("{} refresh request executed on primary", indexShard.shardId()); @@ -68,7 +68,7 @@ public class TransportShardRefreshAction extends TransportReplicationAction scheduledFuture; private final AtomicLong executionIdGen = new AtomicLong(); @@ -250,24 +250,24 @@ public class BulkProcessor implements Closeable { * (for example, if no id is provided, one will be generated, or usage of the create flag). */ public BulkProcessor add(IndexRequest request) { - return add((ActionRequest) request); + return add((ActionRequest) request); } /** * Adds an {@link DeleteRequest} to the list of actions to execute. */ public BulkProcessor add(DeleteRequest request) { - return add((ActionRequest) request); + return add((ActionRequest) request); } /** * Adds either a delete or an index request. */ - public BulkProcessor add(ActionRequest request) { + public BulkProcessor add(ActionRequest request) { return add(request, null); } - public BulkProcessor add(ActionRequest request, @Nullable Object payload) { + public BulkProcessor add(ActionRequest request, @Nullable Object payload) { internalAdd(request, payload); return this; } @@ -282,7 +282,7 @@ public class BulkProcessor implements Closeable { } } - private synchronized void internalAdd(ActionRequest request, @Nullable Object payload) { + private synchronized void internalAdd(ActionRequest request, @Nullable Object payload) { ensureOpen(); bulkRequest.add(request, payload); executeIfNeeded(); diff --git a/core/src/main/java/org/elasticsearch/action/bulk/BulkRequest.java b/core/src/main/java/org/elasticsearch/action/bulk/BulkRequest.java index 02e0ea40d65a..002606448925 100644 --- a/core/src/main/java/org/elasticsearch/action/bulk/BulkRequest.java +++ b/core/src/main/java/org/elasticsearch/action/bulk/BulkRequest.java @@ -56,7 +56,7 @@ public class BulkRequest extends ActionRequest implements Composite private static final int REQUEST_OVERHEAD = 50; - final List requests = new ArrayList<>(); + final List> requests = new ArrayList<>(); List payloads = null; protected TimeValue timeout = BulkShardRequest.DEFAULT_TIMEOUT; @@ -72,21 +72,21 @@ public class BulkRequest extends ActionRequest implements Composite * Creates a bulk request caused by some other request, which is provided as an * argument so that its headers and context can be copied to the new request */ - public BulkRequest(ActionRequest request) { + public BulkRequest(ActionRequest request) { super(request); } /** * Adds a list of requests to be executed. Either index or delete requests. */ - public BulkRequest add(ActionRequest... requests) { - for (ActionRequest request : requests) { + public BulkRequest add(ActionRequest... requests) { + for (ActionRequest request : requests) { add(request, null); } return this; } - public BulkRequest add(ActionRequest request) { + public BulkRequest add(ActionRequest request) { return add(request, null); } @@ -96,7 +96,7 @@ public class BulkRequest extends ActionRequest implements Composite * @param payload Optional payload * @return the current bulk request */ - public BulkRequest add(ActionRequest request, @Nullable Object payload) { + public BulkRequest add(ActionRequest request, @Nullable Object payload) { if (request instanceof IndexRequest) { add((IndexRequest) request, payload); } else if (request instanceof DeleteRequest) { @@ -112,8 +112,8 @@ public class BulkRequest extends ActionRequest implements Composite /** * Adds a list of requests to be executed. Either index or delete requests. */ - public BulkRequest add(Iterable requests) { - for (ActionRequest request : requests) { + public BulkRequest add(Iterable> requests) { + for (ActionRequest request : requests) { add(request); } return this; @@ -196,15 +196,14 @@ public class BulkRequest extends ActionRequest implements Composite /** * The list of requests in this bulk request. */ - public List requests() { + public List> requests() { return this.requests; } @Override - @SuppressWarnings("unchecked") public List subRequests() { List indicesRequests = new ArrayList<>(); - for (ActionRequest request : requests) { + for (ActionRequest request : requests) { assert request instanceof IndicesRequest; indicesRequests.add((IndicesRequest) request); } @@ -486,7 +485,7 @@ public class BulkRequest extends ActionRequest implements Composite if (requests.isEmpty()) { validationException = addValidationError("no requests added", validationException); } - for (ActionRequest request : requests) { + for (ActionRequest request : requests) { // We first check if refresh has been set if ((request instanceof DeleteRequest && ((DeleteRequest)request).refresh()) || (request instanceof UpdateRequest && ((UpdateRequest)request).refresh()) || @@ -535,7 +534,7 @@ public class BulkRequest extends ActionRequest implements Composite super.writeTo(out); out.writeByte(consistencyLevel.id()); out.writeVInt(requests.size()); - for (ActionRequest request : requests) { + for (ActionRequest request : requests) { if (request instanceof IndexRequest) { out.writeByte((byte) 0); } else if (request instanceof DeleteRequest) { diff --git a/core/src/main/java/org/elasticsearch/action/percolate/TransportShardMultiPercolateAction.java b/core/src/main/java/org/elasticsearch/action/percolate/TransportShardMultiPercolateAction.java index 7140af93ed0f..c2ae538085b1 100644 --- a/core/src/main/java/org/elasticsearch/action/percolate/TransportShardMultiPercolateAction.java +++ b/core/src/main/java/org/elasticsearch/action/percolate/TransportShardMultiPercolateAction.java @@ -23,7 +23,6 @@ import org.elasticsearch.ElasticsearchException; import org.elasticsearch.action.ActionRequestValidationException; import org.elasticsearch.action.ActionResponse; import org.elasticsearch.action.IndicesRequest; -import org.elasticsearch.action.OriginalIndices; import org.elasticsearch.action.support.ActionFilters; import org.elasticsearch.action.support.TransportActions; import org.elasticsearch.action.support.single.shard.SingleShardRequest; @@ -109,7 +108,7 @@ public class TransportShardMultiPercolateAction extends TransportSingleShardActi } - public static class Request extends SingleShardRequest implements IndicesRequest { + public static class Request extends SingleShardRequest implements IndicesRequest { private int shardId; private String preference; @@ -237,7 +236,7 @@ public class TransportShardMultiPercolateAction extends TransportSingleShardActi shardResponse.readFrom(in); items.add(new Item(slot, shardResponse)); } else { - items.add(new Item(slot, (Throwable)in.readThrowable())); + items.add(new Item(slot, in.readThrowable())); } } } diff --git a/core/src/main/java/org/elasticsearch/action/support/ActionFilter.java b/core/src/main/java/org/elasticsearch/action/support/ActionFilter.java index b80cf9ddbd4c..6c08eec323f1 100644 --- a/core/src/main/java/org/elasticsearch/action/support/ActionFilter.java +++ b/core/src/main/java/org/elasticsearch/action/support/ActionFilter.java @@ -40,13 +40,13 @@ public interface ActionFilter { * Enables filtering the execution of an action on the request side, either by sending a response through the * {@link ActionListener} or by continuing the execution through the given {@link ActionFilterChain chain} */ - void apply(Task task, String action, ActionRequest request, ActionListener listener, ActionFilterChain chain); + void apply(Task task, String action, ActionRequest request, ActionListener listener, ActionFilterChain chain); /** * Enables filtering the execution of an action on the response side, either by sending a response through the * {@link ActionListener} or by continuing the execution through the given {@link ActionFilterChain chain} */ - void apply(String action, ActionResponse response, ActionListener listener, ActionFilterChain chain); + void apply(String action, ActionResponse response, ActionListener listener, ActionFilterChain chain); /** * A simple base class for injectable action filters that spares the implementation from handling the @@ -60,7 +60,7 @@ public interface ActionFilter { } @Override - public final void apply(Task task, String action, ActionRequest request, ActionListener listener, ActionFilterChain chain) { + public final void apply(Task task, String action, ActionRequest request, ActionListener listener, ActionFilterChain chain) { if (apply(action, request, listener)) { chain.proceed(task, action, request, listener); } @@ -70,10 +70,10 @@ public interface ActionFilter { * Applies this filter and returns {@code true} if the execution chain should proceed, or {@code false} * if it should be aborted since the filter already handled the request and called the given listener. */ - protected abstract boolean apply(String action, ActionRequest request, ActionListener listener); + protected abstract boolean apply(String action, ActionRequest request, ActionListener listener); @Override - public final void apply(String action, ActionResponse response, ActionListener listener, ActionFilterChain chain) { + public final void apply(String action, ActionResponse response, ActionListener listener, ActionFilterChain chain) { if (apply(action, response, listener)) { chain.proceed(action, response, listener); } @@ -83,6 +83,6 @@ public interface ActionFilter { * Applies this filter and returns {@code true} if the execution chain should proceed, or {@code false} * if it should be aborted since the filter already handled the response by calling the given listener. */ - protected abstract boolean apply(String action, ActionResponse response, ActionListener listener); + protected abstract boolean apply(String action, ActionResponse response, ActionListener listener); } } diff --git a/core/src/main/java/org/elasticsearch/action/support/HandledTransportAction.java b/core/src/main/java/org/elasticsearch/action/support/HandledTransportAction.java index bd9556f05003..a439117fef6e 100644 --- a/core/src/main/java/org/elasticsearch/action/support/HandledTransportAction.java +++ b/core/src/main/java/org/elasticsearch/action/support/HandledTransportAction.java @@ -34,8 +34,8 @@ import java.util.function.Supplier; /** * A TransportAction that self registers a handler into the transport service */ -public abstract class HandledTransportAction extends TransportAction{ - +public abstract class HandledTransportAction, Response extends ActionResponse> + extends TransportAction { protected HandledTransportAction(Settings settings, String actionName, ThreadPool threadPool, TransportService transportService, ActionFilters actionFilters, IndexNameExpressionResolver indexNameExpressionResolver, Supplier request) { super(settings, actionName, threadPool, actionFilters, indexNameExpressionResolver, transportService.getTaskManager()); transportService.registerRequestHandler(actionName, request, ThreadPool.Names.SAME, new TransportHandler()); diff --git a/core/src/main/java/org/elasticsearch/action/support/TransportAction.java b/core/src/main/java/org/elasticsearch/action/support/TransportAction.java index 3e0454550bad..eb62903bf349 100644 --- a/core/src/main/java/org/elasticsearch/action/support/TransportAction.java +++ b/core/src/main/java/org/elasticsearch/action/support/TransportAction.java @@ -40,7 +40,7 @@ import static org.elasticsearch.action.support.PlainActionFuture.newFuture; /** * */ -public abstract class TransportAction extends AbstractComponent { +public abstract class TransportAction, Response extends ActionResponse> extends AbstractComponent { protected final ThreadPool threadPool; protected final String actionName; @@ -66,7 +66,7 @@ public abstract class TransportAction listener) { + public final Task execute(Request request, ActionListener listener) { Task task = taskManager.register("transport", actionName, request); if (task == null) { execute(null, request, listener); @@ -85,6 +85,7 @@ public abstract class TransportAction listener) { @@ -114,7 +115,7 @@ public abstract class TransportAction listener); - private static class RequestFilterChain implements ActionFilterChain { + private static class RequestFilterChain, Response extends ActionResponse> implements ActionFilterChain { private final TransportAction action; private final AtomicInteger index = new AtomicInteger(); diff --git a/core/src/main/java/org/elasticsearch/action/support/broadcast/TransportBroadcastAction.java b/core/src/main/java/org/elasticsearch/action/support/broadcast/TransportBroadcastAction.java index 3159c3ab2b6d..be851cfa7e24 100644 --- a/core/src/main/java/org/elasticsearch/action/support/broadcast/TransportBroadcastAction.java +++ b/core/src/main/java/org/elasticsearch/action/support/broadcast/TransportBroadcastAction.java @@ -49,7 +49,7 @@ import java.util.function.Supplier; /** * */ -public abstract class TransportBroadcastAction +public abstract class TransportBroadcastAction, Response extends BroadcastResponse, ShardRequest extends BroadcastShardRequest, ShardResponse extends BroadcastShardResponse> extends HandledTransportAction { protected final ClusterService clusterService; diff --git a/core/src/main/java/org/elasticsearch/action/support/broadcast/node/TransportBroadcastByNodeAction.java b/core/src/main/java/org/elasticsearch/action/support/broadcast/node/TransportBroadcastByNodeAction.java index e8f4a0d83cd9..613de1aa9233 100644 --- a/core/src/main/java/org/elasticsearch/action/support/broadcast/node/TransportBroadcastByNodeAction.java +++ b/core/src/main/java/org/elasticsearch/action/support/broadcast/node/TransportBroadcastByNodeAction.java @@ -74,7 +74,7 @@ import java.util.function.Supplier; * @param the response to the client request * @param per-shard operation results */ -public abstract class TransportBroadcastByNodeAction, Response extends BroadcastResponse, ShardOperationResult extends Streamable> extends HandledTransportAction { @@ -447,10 +447,12 @@ public abstract class TransportBroadcastByNodeAction extends HandledTransportAction { +public abstract class TransportMasterNodeAction, Response extends ActionResponse> extends HandledTransportAction { protected final TransportService transportService; protected final ClusterService clusterService; diff --git a/core/src/main/java/org/elasticsearch/action/support/master/TransportMasterNodeReadAction.java b/core/src/main/java/org/elasticsearch/action/support/master/TransportMasterNodeReadAction.java index d0f64cbb9be3..681b9f0a648e 100644 --- a/core/src/main/java/org/elasticsearch/action/support/master/TransportMasterNodeReadAction.java +++ b/core/src/main/java/org/elasticsearch/action/support/master/TransportMasterNodeReadAction.java @@ -33,7 +33,8 @@ import java.util.function.Supplier; * A base class for read operations that needs to be performed on the master node. * Can also be executed on the local node if needed. */ -public abstract class TransportMasterNodeReadAction extends TransportMasterNodeAction { +public abstract class TransportMasterNodeReadAction, Response extends ActionResponse> + extends TransportMasterNodeAction { public static final String FORCE_LOCAL_SETTING = "action.master.force_local"; diff --git a/core/src/main/java/org/elasticsearch/action/support/master/info/TransportClusterInfoAction.java b/core/src/main/java/org/elasticsearch/action/support/master/info/TransportClusterInfoAction.java index ec9c3eb46c3c..7e42036c1d13 100644 --- a/core/src/main/java/org/elasticsearch/action/support/master/info/TransportClusterInfoAction.java +++ b/core/src/main/java/org/elasticsearch/action/support/master/info/TransportClusterInfoAction.java @@ -33,7 +33,8 @@ import java.util.function.Supplier; /** */ -public abstract class TransportClusterInfoAction extends TransportMasterNodeReadAction { +public abstract class TransportClusterInfoAction, Response extends ActionResponse> + extends TransportMasterNodeReadAction { public TransportClusterInfoAction(Settings settings, String actionName, TransportService transportService, ClusterService clusterService, ThreadPool threadPool, ActionFilters actionFilters, diff --git a/core/src/main/java/org/elasticsearch/action/support/nodes/TransportNodesAction.java b/core/src/main/java/org/elasticsearch/action/support/nodes/TransportNodesAction.java index b83081b86bdb..f90d194287bc 100644 --- a/core/src/main/java/org/elasticsearch/action/support/nodes/TransportNodesAction.java +++ b/core/src/main/java/org/elasticsearch/action/support/nodes/TransportNodesAction.java @@ -50,7 +50,7 @@ import java.util.function.Supplier; /** * */ -public abstract class TransportNodesAction extends HandledTransportAction { +public abstract class TransportNodesAction, NodesResponse extends BaseNodesResponse, NodeRequest extends BaseNodeRequest, NodeResponse extends BaseNodeResponse> extends HandledTransportAction { protected final ClusterName clusterName; protected final ClusterService clusterService; diff --git a/core/src/main/java/org/elasticsearch/action/support/replication/BasicReplicationRequest.java b/core/src/main/java/org/elasticsearch/action/support/replication/BasicReplicationRequest.java new file mode 100644 index 000000000000..3778275d4006 --- /dev/null +++ b/core/src/main/java/org/elasticsearch/action/support/replication/BasicReplicationRequest.java @@ -0,0 +1,59 @@ +/* + * Licensed to Elasticsearch under one or more contributor + * license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright + * ownership. Elasticsearch licenses this file to you under + * the Apache License, Version 2.0 (the "License"); you may + * not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.elasticsearch.action.support.replication; + +import org.elasticsearch.action.ActionRequest; +import org.elasticsearch.index.shard.ShardId; + +/** + * A replication request that has no more information than ReplicationRequest. + * Unfortunately ReplicationRequest can't be declared as a type parameter + * because it has a self referential type parameter of its own. So use this + * instead. + */ +public class BasicReplicationRequest extends ReplicationRequest { + public BasicReplicationRequest() { + + } + + /** + * Creates a new request that inherits headers and context from the request + * provided as argument. + */ + public BasicReplicationRequest(ActionRequest request) { + super(request); + } + + /** + * Creates a new request with resolved shard id + */ + public BasicReplicationRequest(ActionRequest request, ShardId shardId) { + super(request, shardId); + } + + /** + * Copy constructor that creates a new request that is a copy of the one + * provided as an argument. + */ + protected BasicReplicationRequest(BasicReplicationRequest request) { + super(request); + } + +} diff --git a/core/src/main/java/org/elasticsearch/action/support/replication/ReplicationRequest.java b/core/src/main/java/org/elasticsearch/action/support/replication/ReplicationRequest.java index 9b9569183064..a6c9b8f65a38 100644 --- a/core/src/main/java/org/elasticsearch/action/support/replication/ReplicationRequest.java +++ b/core/src/main/java/org/elasticsearch/action/support/replication/ReplicationRequest.java @@ -38,7 +38,7 @@ import static org.elasticsearch.action.ValidateActions.addValidationError; /** * */ -public class ReplicationRequest> extends ActionRequest implements IndicesRequest { +public abstract class ReplicationRequest> extends ActionRequest implements IndicesRequest { public static final TimeValue DEFAULT_TIMEOUT = new TimeValue(1, TimeUnit.MINUTES); diff --git a/core/src/main/java/org/elasticsearch/action/support/replication/TransportBroadcastReplicationAction.java b/core/src/main/java/org/elasticsearch/action/support/replication/TransportBroadcastReplicationAction.java index 33a9d349e806..ab88d73d3b08 100644 --- a/core/src/main/java/org/elasticsearch/action/support/replication/TransportBroadcastReplicationAction.java +++ b/core/src/main/java/org/elasticsearch/action/support/replication/TransportBroadcastReplicationAction.java @@ -20,6 +20,7 @@ package org.elasticsearch.action.support.replication; import com.carrotsearch.hppc.cursors.IntObjectCursor; + import org.elasticsearch.ExceptionsHelper; import org.elasticsearch.action.ActionListener; import org.elasticsearch.action.ReplicationResponse; @@ -52,7 +53,8 @@ import java.util.function.Supplier; * Base class for requests that should be executed on all shards of an index or several indices. * This action sends shard requests to all primary shards of the indices and they are then replicated like write requests */ -public abstract class TransportBroadcastReplicationAction extends HandledTransportAction { +public abstract class TransportBroadcastReplicationAction, Response extends BroadcastResponse, ShardRequest extends ReplicationRequest, ShardResponse extends ReplicationResponse> + extends HandledTransportAction { private final TransportReplicationAction replicatedBroadcastShardAction; private final ClusterService clusterService; diff --git a/core/src/main/java/org/elasticsearch/action/support/replication/TransportReplicationAction.java b/core/src/main/java/org/elasticsearch/action/support/replication/TransportReplicationAction.java index a02564359960..930a43fd8408 100644 --- a/core/src/main/java/org/elasticsearch/action/support/replication/TransportReplicationAction.java +++ b/core/src/main/java/org/elasticsearch/action/support/replication/TransportReplicationAction.java @@ -90,7 +90,7 @@ import java.util.function.Supplier; * primary node to validate request before primary operation followed by sampling state again for resolving * nodes with replica copies to perform replication. */ -public abstract class TransportReplicationAction extends TransportAction { +public abstract class TransportReplicationAction, ReplicaRequest extends ReplicationRequest, Response extends ReplicationResponse> extends TransportAction { protected final TransportService transportService; protected final ClusterService clusterService; diff --git a/core/src/main/java/org/elasticsearch/action/support/single/instance/TransportInstanceSingleOperationAction.java b/core/src/main/java/org/elasticsearch/action/support/single/instance/TransportInstanceSingleOperationAction.java index 50cdd0ebeb27..74d9f3c7017b 100644 --- a/core/src/main/java/org/elasticsearch/action/support/single/instance/TransportInstanceSingleOperationAction.java +++ b/core/src/main/java/org/elasticsearch/action/support/single/instance/TransportInstanceSingleOperationAction.java @@ -54,8 +54,8 @@ import java.util.function.Supplier; /** * */ -public abstract class TransportInstanceSingleOperationAction extends HandledTransportAction { - +public abstract class TransportInstanceSingleOperationAction, Response extends ActionResponse> + extends HandledTransportAction { protected final ClusterService clusterService; protected final TransportService transportService; diff --git a/core/src/main/java/org/elasticsearch/action/support/single/shard/TransportSingleShardAction.java b/core/src/main/java/org/elasticsearch/action/support/single/shard/TransportSingleShardAction.java index 47eebc9cfcdf..1a77a411a943 100644 --- a/core/src/main/java/org/elasticsearch/action/support/single/shard/TransportSingleShardAction.java +++ b/core/src/main/java/org/elasticsearch/action/support/single/shard/TransportSingleShardAction.java @@ -54,7 +54,7 @@ import static org.elasticsearch.action.support.TransportActions.isShardNotAvaila * the read operation can be performed on other shard copies. Concrete implementations can provide their own list * of candidate shards to try the read operation on. */ -public abstract class TransportSingleShardAction extends TransportAction { +public abstract class TransportSingleShardAction, Response extends ActionResponse> extends TransportAction { protected final ClusterService clusterService; diff --git a/core/src/main/java/org/elasticsearch/client/ElasticsearchClient.java b/core/src/main/java/org/elasticsearch/client/ElasticsearchClient.java index 4cf5a5a961dc..d9ddc56d48a0 100644 --- a/core/src/main/java/org/elasticsearch/client/ElasticsearchClient.java +++ b/core/src/main/java/org/elasticsearch/client/ElasticsearchClient.java @@ -40,7 +40,8 @@ public interface ElasticsearchClient { * @param The request builder type. * @return A future allowing to get back the response. */ - > ActionFuture execute(final Action action, final Request request); + , Response extends ActionResponse, RequestBuilder extends ActionRequestBuilder> ActionFuture execute( + final Action action, final Request request); /** * Executes a generic action, denoted by an {@link Action}. @@ -52,7 +53,8 @@ public interface ElasticsearchClient { * @param The response type. * @param The request builder type. */ - > void execute(final Action action, final Request request, ActionListener listener); + , Response extends ActionResponse, RequestBuilder extends ActionRequestBuilder> void execute( + final Action action, final Request request, ActionListener listener); /** * Prepares a request builder to execute, specified by {@link Action}. @@ -63,7 +65,8 @@ public interface ElasticsearchClient { * @param The request builder. * @return The request builder, that can, at a later stage, execute the request. */ - > RequestBuilder prepareExecute(final Action action); + , Response extends ActionResponse, RequestBuilder extends ActionRequestBuilder> RequestBuilder prepareExecute( + final Action action); /** * Returns the threadpool used to execute requests on this client diff --git a/core/src/main/java/org/elasticsearch/client/FilterClient.java b/core/src/main/java/org/elasticsearch/client/FilterClient.java index 06d81f0c9d54..77abceef17af 100644 --- a/core/src/main/java/org/elasticsearch/client/FilterClient.java +++ b/core/src/main/java/org/elasticsearch/client/FilterClient.java @@ -52,7 +52,8 @@ public abstract class FilterClient extends AbstractClient { } @Override - protected > void doExecute(Action action, Request request, ActionListener listener) { + protected , Response extends ActionResponse, RequestBuilder extends ActionRequestBuilder> void doExecute( + Action action, Request request, ActionListener listener) { in().execute(action, request, listener); } diff --git a/core/src/main/java/org/elasticsearch/client/node/NodeClient.java b/core/src/main/java/org/elasticsearch/client/node/NodeClient.java index 65adfad64dc0..4f64f63f8d72 100644 --- a/core/src/main/java/org/elasticsearch/client/node/NodeClient.java +++ b/core/src/main/java/org/elasticsearch/client/node/NodeClient.java @@ -56,7 +56,8 @@ public class NodeClient extends AbstractClient { @SuppressWarnings("unchecked") @Override - public > void doExecute(Action action, Request request, ActionListener listener) { + public , Response extends ActionResponse, RequestBuilder extends ActionRequestBuilder> void doExecute( + Action action, Request request, ActionListener listener) { TransportAction transportAction = actions.get(action); if (transportAction == null) { throw new IllegalStateException("failed to find action [" + action + "] to execute"); diff --git a/core/src/main/java/org/elasticsearch/client/support/AbstractClient.java b/core/src/main/java/org/elasticsearch/client/support/AbstractClient.java index e5a465442bb0..e5e1bea6cb51 100644 --- a/core/src/main/java/org/elasticsearch/client/support/AbstractClient.java +++ b/core/src/main/java/org/elasticsearch/client/support/AbstractClient.java @@ -363,12 +363,14 @@ public abstract class AbstractClient extends AbstractComponent implements Client } @Override - public final > RequestBuilder prepareExecute(final Action action) { + public final , Response extends ActionResponse, RequestBuilder extends ActionRequestBuilder> RequestBuilder prepareExecute( + final Action action) { return action.newRequestBuilder(this); } @Override - public final > ActionFuture execute(Action action, Request request) { + public final , Response extends ActionResponse, RequestBuilder extends ActionRequestBuilder> ActionFuture execute( + Action action, Request request) { PlainActionFuture actionFuture = PlainActionFuture.newFuture(); execute(action, request, actionFuture); return actionFuture; @@ -378,13 +380,14 @@ public abstract class AbstractClient extends AbstractComponent implements Client * This is the single execution point of *all* clients. */ @Override - public final > void execute(Action action, Request request, ActionListener listener) { + public final , Response extends ActionResponse, RequestBuilder extends ActionRequestBuilder> void execute( + Action action, Request request, ActionListener listener) { headers.applyTo(request); listener = threadedWrapper.wrap(listener); doExecute(action, request, listener); } - protected abstract > void doExecute(final Action action, final Request request, ActionListener listener); + protected abstract , Response extends ActionResponse, RequestBuilder extends ActionRequestBuilder> void doExecute(final Action action, final Request request, ActionListener listener); @Override public ActionFuture index(final IndexRequest request) { @@ -821,17 +824,20 @@ public abstract class AbstractClient extends AbstractComponent implements Client } @Override - public > ActionFuture execute(Action action, Request request) { + public , Response extends ActionResponse, RequestBuilder extends ActionRequestBuilder> ActionFuture execute( + Action action, Request request) { return client.execute(action, request); } @Override - public > void execute(Action action, Request request, ActionListener listener) { + public , Response extends ActionResponse, RequestBuilder extends ActionRequestBuilder> void execute( + Action action, Request request, ActionListener listener) { client.execute(action, request, listener); } @Override - public > RequestBuilder prepareExecute(Action action) { + public , Response extends ActionResponse, RequestBuilder extends ActionRequestBuilder> RequestBuilder prepareExecute( + Action action) { return client.prepareExecute(action); } @@ -1178,17 +1184,20 @@ public abstract class AbstractClient extends AbstractComponent implements Client } @Override - public > ActionFuture execute(Action action, Request request) { + public , Response extends ActionResponse, RequestBuilder extends ActionRequestBuilder> ActionFuture execute( + Action action, Request request) { return client.execute(action, request); } @Override - public > void execute(Action action, Request request, ActionListener listener) { + public , Response extends ActionResponse, RequestBuilder extends ActionRequestBuilder> void execute( + Action action, Request request, ActionListener listener) { client.execute(action, request, listener); } @Override - public > RequestBuilder prepareExecute(Action action) { + public , Response extends ActionResponse, RequestBuilder extends ActionRequestBuilder> RequestBuilder prepareExecute( + Action action) { return client.prepareExecute(action); } diff --git a/core/src/main/java/org/elasticsearch/client/transport/TransportClient.java b/core/src/main/java/org/elasticsearch/client/transport/TransportClient.java index 3b8be668f43c..b870c68c7920 100644 --- a/core/src/main/java/org/elasticsearch/client/transport/TransportClient.java +++ b/core/src/main/java/org/elasticsearch/client/transport/TransportClient.java @@ -19,6 +19,10 @@ package org.elasticsearch.client.transport; +import java.util.ArrayList; +import java.util.List; +import java.util.concurrent.TimeUnit; + import org.elasticsearch.Version; import org.elasticsearch.action.Action; import org.elasticsearch.action.ActionListener; @@ -36,6 +40,7 @@ import org.elasticsearch.common.component.LifecycleComponent; import org.elasticsearch.common.inject.Injector; import org.elasticsearch.common.inject.Module; import org.elasticsearch.common.inject.ModulesBuilder; +import org.elasticsearch.common.io.stream.NamedWriteableRegistry; import org.elasticsearch.common.network.NetworkModule; import org.elasticsearch.common.network.NetworkService; import org.elasticsearch.common.settings.Settings; @@ -54,10 +59,6 @@ import org.elasticsearch.threadpool.ThreadPoolModule; import org.elasticsearch.transport.TransportService; import org.elasticsearch.transport.netty.NettyTransport; -import java.util.ArrayList; -import java.util.List; -import java.util.concurrent.TimeUnit; - import static org.elasticsearch.common.settings.Settings.settingsBuilder; /** @@ -128,6 +129,7 @@ public class TransportClient extends AbstractClient { final ThreadPool threadPool = new ThreadPool(settings); final NetworkService networkService = new NetworkService(settings); final SettingsFilter settingsFilter = new SettingsFilter(settings); + NamedWriteableRegistry namedWriteableRegistry = new NamedWriteableRegistry(); boolean success = false; try { ModulesBuilder modules = new ModulesBuilder(); @@ -138,10 +140,10 @@ public class TransportClient extends AbstractClient { } modules.add(new PluginsModule(pluginsService)); modules.add(new SettingsModule(this.settings, settingsFilter )); - modules.add(new NetworkModule(networkService, this.settings, true)); + modules.add(new NetworkModule(networkService, this.settings, true, namedWriteableRegistry)); modules.add(new ClusterNameModule(this.settings)); modules.add(new ThreadPoolModule(threadPool)); - modules.add(new SearchModule() { + modules.add(new SearchModule(settings, namedWriteableRegistry) { @Override protected void configure() { // noop @@ -276,7 +278,7 @@ public class TransportClient extends AbstractClient { } @Override - protected > void doExecute(Action action, Request request, ActionListener listener) { + protected , Response extends ActionResponse, RequestBuilder extends ActionRequestBuilder> void doExecute(Action action, Request request, ActionListener listener) { proxy.execute(action, request, listener); } } diff --git a/core/src/main/java/org/elasticsearch/cluster/metadata/MetaDataCreateIndexService.java b/core/src/main/java/org/elasticsearch/cluster/metadata/MetaDataCreateIndexService.java index 3d9894699a5a..2344e9af77cb 100644 --- a/core/src/main/java/org/elasticsearch/cluster/metadata/MetaDataCreateIndexService.java +++ b/core/src/main/java/org/elasticsearch/cluster/metadata/MetaDataCreateIndexService.java @@ -317,7 +317,7 @@ public class MetaDataCreateIndexService extends AbstractComponent { // first, add the default mapping if (mappings.containsKey(MapperService.DEFAULT_MAPPING)) { try { - mapperService.merge(MapperService.DEFAULT_MAPPING, new CompressedXContent(XContentFactory.jsonBuilder().map(mappings.get(MapperService.DEFAULT_MAPPING)).string()), false, request.updateAllTypes()); + mapperService.merge(MapperService.DEFAULT_MAPPING, new CompressedXContent(XContentFactory.jsonBuilder().map(mappings.get(MapperService.DEFAULT_MAPPING)).string()), MapperService.MergeReason.MAPPING_UPDATE, request.updateAllTypes()); } catch (Exception e) { removalReason = "failed on parsing default mapping on index creation"; throw new MapperParsingException("Failed to parse mapping [{}]: {}", e, MapperService.DEFAULT_MAPPING, e.getMessage()); @@ -329,7 +329,7 @@ public class MetaDataCreateIndexService extends AbstractComponent { } try { // apply the default here, its the first time we parse it - mapperService.merge(entry.getKey(), new CompressedXContent(XContentFactory.jsonBuilder().map(entry.getValue()).string()), true, request.updateAllTypes()); + mapperService.merge(entry.getKey(), new CompressedXContent(XContentFactory.jsonBuilder().map(entry.getValue()).string()), MapperService.MergeReason.MAPPING_UPDATE, request.updateAllTypes()); } catch (Exception e) { removalReason = "failed on parsing mappings on index creation"; throw new MapperParsingException("Failed to parse mapping [{}]: {}", e, entry.getKey(), e.getMessage()); diff --git a/core/src/main/java/org/elasticsearch/cluster/metadata/MetaDataIndexAliasesService.java b/core/src/main/java/org/elasticsearch/cluster/metadata/MetaDataIndexAliasesService.java index 32a66bfb764e..1e9f968f7a6c 100644 --- a/core/src/main/java/org/elasticsearch/cluster/metadata/MetaDataIndexAliasesService.java +++ b/core/src/main/java/org/elasticsearch/cluster/metadata/MetaDataIndexAliasesService.java @@ -104,12 +104,9 @@ public class MetaDataIndexAliasesService extends AbstractComponent { // temporarily create the index and add mappings so we can parse the filter try { indexService = indicesService.createIndex(nodeServicesProvider, indexMetaData, Collections.emptyList()); - if (indexMetaData.getMappings().containsKey(MapperService.DEFAULT_MAPPING)) { - indexService.mapperService().merge(MapperService.DEFAULT_MAPPING, indexMetaData.getMappings().get(MapperService.DEFAULT_MAPPING).source(), false, false); - } for (ObjectCursor cursor : indexMetaData.getMappings().values()) { MappingMetaData mappingMetaData = cursor.value; - indexService.mapperService().merge(mappingMetaData.type(), mappingMetaData.source(), false, false); + indexService.mapperService().merge(mappingMetaData.type(), mappingMetaData.source(), MapperService.MergeReason.MAPPING_RECOVERY, false); } } catch (Exception e) { logger.warn("[{}] failed to temporary create in order to apply alias action", e, indexMetaData.getIndex()); diff --git a/core/src/main/java/org/elasticsearch/cluster/metadata/MetaDataIndexUpgradeService.java b/core/src/main/java/org/elasticsearch/cluster/metadata/MetaDataIndexUpgradeService.java index 43944541cc14..c6ec2a4376d2 100644 --- a/core/src/main/java/org/elasticsearch/cluster/metadata/MetaDataIndexUpgradeService.java +++ b/core/src/main/java/org/elasticsearch/cluster/metadata/MetaDataIndexUpgradeService.java @@ -125,7 +125,7 @@ public class MetaDataIndexUpgradeService extends AbstractComponent { try (MapperService mapperService = new MapperService(indexSettings, analysisService, similarityService, mapperRegistry, () -> null)) { for (ObjectCursor cursor : indexMetaData.getMappings().values()) { MappingMetaData mappingMetaData = cursor.value; - mapperService.merge(mappingMetaData.type(), mappingMetaData.source(), false, false); + mapperService.merge(mappingMetaData.type(), mappingMetaData.source(), MapperService.MergeReason.MAPPING_RECOVERY, false); } } } diff --git a/core/src/main/java/org/elasticsearch/cluster/metadata/MetaDataMappingService.java b/core/src/main/java/org/elasticsearch/cluster/metadata/MetaDataMappingService.java index 14f9f500c455..1d13fc2079e9 100644 --- a/core/src/main/java/org/elasticsearch/cluster/metadata/MetaDataMappingService.java +++ b/core/src/main/java/org/elasticsearch/cluster/metadata/MetaDataMappingService.java @@ -143,7 +143,7 @@ public class MetaDataMappingService extends AbstractComponent { removeIndex = true; for (ObjectCursor metaData : indexMetaData.getMappings().values()) { // don't apply the default mapping, it has been applied when the mapping was created - indexService.mapperService().merge(metaData.value.type(), metaData.value.source(), false, true); + indexService.mapperService().merge(metaData.value.type(), metaData.value.source(), MapperService.MergeReason.MAPPING_RECOVERY, true); } } @@ -223,7 +223,7 @@ public class MetaDataMappingService extends AbstractComponent { IndexService indexService = indicesService.createIndex(nodeServicesProvider, indexMetaData, Collections.emptyList()); // add mappings for all types, we need them for cross-type validation for (ObjectCursor mapping : indexMetaData.getMappings().values()) { - indexService.mapperService().merge(mapping.value.type(), mapping.value.source(), false, request.updateAllTypes()); + indexService.mapperService().merge(mapping.value.type(), mapping.value.source(), MapperService.MergeReason.MAPPING_RECOVERY, request.updateAllTypes()); } } } @@ -303,7 +303,7 @@ public class MetaDataMappingService extends AbstractComponent { if (existingMapper != null) { existingSource = existingMapper.mappingSource(); } - DocumentMapper mergedMapper = indexService.mapperService().merge(mappingType, mappingUpdateSource, true, request.updateAllTypes()); + DocumentMapper mergedMapper = indexService.mapperService().merge(mappingType, mappingUpdateSource, MapperService.MergeReason.MAPPING_UPDATE, request.updateAllTypes()); CompressedXContent updatedSource = mergedMapper.mappingSource(); if (existingSource != null) { diff --git a/core/src/main/java/org/elasticsearch/common/geo/builders/CircleBuilder.java b/core/src/main/java/org/elasticsearch/common/geo/builders/CircleBuilder.java index c15e0d9130ea..bda0106f2b6c 100644 --- a/core/src/main/java/org/elasticsearch/common/geo/builders/CircleBuilder.java +++ b/core/src/main/java/org/elasticsearch/common/geo/builders/CircleBuilder.java @@ -36,7 +36,7 @@ public class CircleBuilder extends ShapeBuilder { public static final String FIELD_RADIUS = "radius"; public static final GeoShapeType TYPE = GeoShapeType.CIRCLE; - static final CircleBuilder PROTOTYPE = new CircleBuilder(); + public static final CircleBuilder PROTOTYPE = new CircleBuilder(); private DistanceUnit unit = DistanceUnit.DEFAULT; private double radius; diff --git a/core/src/main/java/org/elasticsearch/common/geo/builders/EnvelopeBuilder.java b/core/src/main/java/org/elasticsearch/common/geo/builders/EnvelopeBuilder.java index 9ad51292977f..426cbbf78004 100644 --- a/core/src/main/java/org/elasticsearch/common/geo/builders/EnvelopeBuilder.java +++ b/core/src/main/java/org/elasticsearch/common/geo/builders/EnvelopeBuilder.java @@ -33,7 +33,7 @@ public class EnvelopeBuilder extends ShapeBuilder { public static final GeoShapeType TYPE = GeoShapeType.ENVELOPE; - static final EnvelopeBuilder PROTOTYPE = new EnvelopeBuilder(new Coordinate(-1.0, 1.0), new Coordinate(1.0, -1.0)); + public static final EnvelopeBuilder PROTOTYPE = new EnvelopeBuilder(new Coordinate(-1.0, 1.0), new Coordinate(1.0, -1.0)); private Coordinate topLeft; private Coordinate bottomRight; diff --git a/core/src/main/java/org/elasticsearch/common/geo/builders/GeometryCollectionBuilder.java b/core/src/main/java/org/elasticsearch/common/geo/builders/GeometryCollectionBuilder.java index 6e0094f71659..420f61a67992 100644 --- a/core/src/main/java/org/elasticsearch/common/geo/builders/GeometryCollectionBuilder.java +++ b/core/src/main/java/org/elasticsearch/common/geo/builders/GeometryCollectionBuilder.java @@ -36,7 +36,7 @@ public class GeometryCollectionBuilder extends ShapeBuilder { public static final GeoShapeType TYPE = GeoShapeType.GEOMETRYCOLLECTION; - static final GeometryCollectionBuilder PROTOTYPE = new GeometryCollectionBuilder(); + public static final GeometryCollectionBuilder PROTOTYPE = new GeometryCollectionBuilder(); protected final ArrayList shapes = new ArrayList<>(); diff --git a/core/src/main/java/org/elasticsearch/common/geo/builders/LineStringBuilder.java b/core/src/main/java/org/elasticsearch/common/geo/builders/LineStringBuilder.java index 4542da436bdf..8c2870e1e094 100644 --- a/core/src/main/java/org/elasticsearch/common/geo/builders/LineStringBuilder.java +++ b/core/src/main/java/org/elasticsearch/common/geo/builders/LineStringBuilder.java @@ -57,7 +57,7 @@ public class LineStringBuilder extends CoordinateCollection { public static final GeoShapeType TYPE = GeoShapeType.LINESTRING; - static final LineStringBuilder PROTOTYPE = new LineStringBuilder(new CoordinatesBuilder().coordinate(0.0, 0.0).coordinate(1.0, 1.0)); + public static final LineStringBuilder PROTOTYPE = new LineStringBuilder(new CoordinatesBuilder().coordinate(0.0, 0.0).coordinate(1.0, 1.0)); @Override public XContentBuilder toXContent(XContentBuilder builder, Params params) throws IOException { diff --git a/core/src/main/java/org/elasticsearch/common/geo/builders/MultiLineStringBuilder.java b/core/src/main/java/org/elasticsearch/common/geo/builders/MultiLineStringBuilder.java index 0de79ac8fe80..e69c0abe4f8a 100644 --- a/core/src/main/java/org/elasticsearch/common/geo/builders/MultiLineStringBuilder.java +++ b/core/src/main/java/org/elasticsearch/common/geo/builders/MultiLineStringBuilder.java @@ -37,7 +37,7 @@ public class MultiLineStringBuilder extends ShapeBuilder { public static final GeoShapeType TYPE = GeoShapeType.MULTILINESTRING; - static final MultiLineStringBuilder PROTOTYPE = new MultiLineStringBuilder(); + public static final MultiLineStringBuilder PROTOTYPE = new MultiLineStringBuilder(); private final ArrayList lines = new ArrayList<>(); diff --git a/core/src/main/java/org/elasticsearch/common/geo/builders/MultiPointBuilder.java b/core/src/main/java/org/elasticsearch/common/geo/builders/MultiPointBuilder.java index 1ed976f43eec..12b16254957e 100644 --- a/core/src/main/java/org/elasticsearch/common/geo/builders/MultiPointBuilder.java +++ b/core/src/main/java/org/elasticsearch/common/geo/builders/MultiPointBuilder.java @@ -37,7 +37,7 @@ public class MultiPointBuilder extends CoordinateCollection { public static final GeoShapeType TYPE = GeoShapeType.MULTIPOINT; - final static MultiPointBuilder PROTOTYPE = new MultiPointBuilder(new CoordinatesBuilder().coordinate(0.0, 0.0).build()); + public static final MultiPointBuilder PROTOTYPE = new MultiPointBuilder(new CoordinatesBuilder().coordinate(0.0, 0.0).build()); /** * Create a new {@link MultiPointBuilder}. diff --git a/core/src/main/java/org/elasticsearch/common/geo/builders/MultiPolygonBuilder.java b/core/src/main/java/org/elasticsearch/common/geo/builders/MultiPolygonBuilder.java index 8d77d3951550..394892d909d6 100644 --- a/core/src/main/java/org/elasticsearch/common/geo/builders/MultiPolygonBuilder.java +++ b/core/src/main/java/org/elasticsearch/common/geo/builders/MultiPolygonBuilder.java @@ -36,7 +36,7 @@ import java.util.Objects; public class MultiPolygonBuilder extends ShapeBuilder { public static final GeoShapeType TYPE = GeoShapeType.MULTIPOLYGON; - static final MultiPolygonBuilder PROTOTYPE = new MultiPolygonBuilder(); + public static final MultiPolygonBuilder PROTOTYPE = new MultiPolygonBuilder(); private final ArrayList polygons = new ArrayList<>(); diff --git a/core/src/main/java/org/elasticsearch/common/geo/builders/PointBuilder.java b/core/src/main/java/org/elasticsearch/common/geo/builders/PointBuilder.java index 40e57566d48b..1cee6525e7a7 100644 --- a/core/src/main/java/org/elasticsearch/common/geo/builders/PointBuilder.java +++ b/core/src/main/java/org/elasticsearch/common/geo/builders/PointBuilder.java @@ -32,7 +32,7 @@ import java.util.Objects; public class PointBuilder extends ShapeBuilder { public static final GeoShapeType TYPE = GeoShapeType.POINT; - static final PointBuilder PROTOTYPE = new PointBuilder(); + public static final PointBuilder PROTOTYPE = new PointBuilder(); private Coordinate coordinate; diff --git a/core/src/main/java/org/elasticsearch/common/geo/builders/PolygonBuilder.java b/core/src/main/java/org/elasticsearch/common/geo/builders/PolygonBuilder.java index 36589a4a4a79..ab480cfbc24c 100644 --- a/core/src/main/java/org/elasticsearch/common/geo/builders/PolygonBuilder.java +++ b/core/src/main/java/org/elasticsearch/common/geo/builders/PolygonBuilder.java @@ -53,7 +53,7 @@ import java.util.concurrent.atomic.AtomicBoolean; public class PolygonBuilder extends ShapeBuilder { public static final GeoShapeType TYPE = GeoShapeType.POLYGON; - static final PolygonBuilder PROTOTYPE = new PolygonBuilder(new CoordinatesBuilder().coordinate(0.0, 0.0).coordinate(0.0, 1.0) + public static final PolygonBuilder PROTOTYPE = new PolygonBuilder(new CoordinatesBuilder().coordinate(0.0, 0.0).coordinate(0.0, 1.0) .coordinate(1.0, 0.0).coordinate(0.0, 0.0)); private static final Coordinate[][] EMPTY = new Coordinate[0][]; diff --git a/core/src/main/java/org/elasticsearch/common/geo/builders/ShapeBuilderRegistry.java b/core/src/main/java/org/elasticsearch/common/geo/builders/ShapeBuilderRegistry.java deleted file mode 100644 index c66e969aa3a1..000000000000 --- a/core/src/main/java/org/elasticsearch/common/geo/builders/ShapeBuilderRegistry.java +++ /dev/null @@ -1,45 +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.common.geo.builders; - -import org.elasticsearch.common.geo.ShapesAvailability; -import org.elasticsearch.common.inject.Inject; -import org.elasticsearch.common.io.stream.NamedWriteableRegistry; - -/** - * Register the shape builder prototypes with the {@link NamedWriteableRegistry} - */ -public class ShapeBuilderRegistry { - - @Inject - public ShapeBuilderRegistry(NamedWriteableRegistry namedWriteableRegistry) { - if (ShapesAvailability.JTS_AVAILABLE && ShapesAvailability.SPATIAL4J_AVAILABLE) { - namedWriteableRegistry.registerPrototype(ShapeBuilder.class, PointBuilder.PROTOTYPE); - namedWriteableRegistry.registerPrototype(ShapeBuilder.class, CircleBuilder.PROTOTYPE); - namedWriteableRegistry.registerPrototype(ShapeBuilder.class, EnvelopeBuilder.PROTOTYPE); - namedWriteableRegistry.registerPrototype(ShapeBuilder.class, MultiPointBuilder.PROTOTYPE); - namedWriteableRegistry.registerPrototype(ShapeBuilder.class, LineStringBuilder.PROTOTYPE); - namedWriteableRegistry.registerPrototype(ShapeBuilder.class, MultiLineStringBuilder.PROTOTYPE); - namedWriteableRegistry.registerPrototype(ShapeBuilder.class, PolygonBuilder.PROTOTYPE); - namedWriteableRegistry.registerPrototype(ShapeBuilder.class, MultiPolygonBuilder.PROTOTYPE); - namedWriteableRegistry.registerPrototype(ShapeBuilder.class, GeometryCollectionBuilder.PROTOTYPE); - } - } -} diff --git a/core/src/main/java/org/elasticsearch/common/inject/matcher/AbstractMatcher.java b/core/src/main/java/org/elasticsearch/common/inject/matcher/AbstractMatcher.java index 9ad7bc407d05..931d290fc19c 100644 --- a/core/src/main/java/org/elasticsearch/common/inject/matcher/AbstractMatcher.java +++ b/core/src/main/java/org/elasticsearch/common/inject/matcher/AbstractMatcher.java @@ -49,8 +49,8 @@ public abstract class AbstractMatcher implements Matcher { @Override public boolean equals(Object other) { return other instanceof AndMatcher - && ((AndMatcher) other).a.equals(a) - && ((AndMatcher) other).b.equals(b); + && ((AndMatcher) other).a.equals(a) + && ((AndMatcher) other).b.equals(b); } @Override @@ -80,8 +80,8 @@ public abstract class AbstractMatcher implements Matcher { @Override public boolean equals(Object other) { return other instanceof OrMatcher - && ((OrMatcher) other).a.equals(a) - && ((OrMatcher) other).b.equals(b); + && ((OrMatcher) other).a.equals(a) + && ((OrMatcher) other).b.equals(b); } @Override diff --git a/core/src/main/java/org/elasticsearch/common/lucene/ShardCoreKeyMap.java b/core/src/main/java/org/elasticsearch/common/lucene/ShardCoreKeyMap.java index b1271e7338d5..92aa02ba0020 100644 --- a/core/src/main/java/org/elasticsearch/common/lucene/ShardCoreKeyMap.java +++ b/core/src/main/java/org/elasticsearch/common/lucene/ShardCoreKeyMap.java @@ -20,9 +20,11 @@ package org.elasticsearch.common.lucene; import org.apache.lucene.index.LeafReader; +import org.apache.lucene.index.LeafReader.CoreClosedListener; import org.elasticsearch.index.shard.ShardId; import org.elasticsearch.index.shard.ShardUtils; +import java.io.IOException; import java.util.Collection; import java.util.Collections; import java.util.HashMap; @@ -72,7 +74,7 @@ public final class ShardCoreKeyMap { } final boolean added = objects.add(coreKey); assert added; - reader.addCoreClosedListener(ownerCoreCacheKey -> { + CoreClosedListener listener = ownerCoreCacheKey -> { assert coreKey == ownerCoreCacheKey; synchronized (ShardCoreKeyMap.this) { coreKeyToShard.remove(ownerCoreCacheKey); @@ -83,7 +85,20 @@ public final class ShardCoreKeyMap { indexToCoreKey.remove(index); } } - }); + }; + boolean addedListener = false; + try { + reader.addCoreClosedListener(listener); + addedListener = true; + } finally { + if (false == addedListener) { + try { + listener.onClose(coreKey); + } catch (IOException e) { + throw new RuntimeException("Blow up trying to recover from failure to add listener", e); + } + } + } } } } diff --git a/core/src/main/java/org/elasticsearch/common/network/NetworkModule.java b/core/src/main/java/org/elasticsearch/common/network/NetworkModule.java index b3abed6e230b..a1e8261e63cb 100644 --- a/core/src/main/java/org/elasticsearch/common/network/NetworkModule.java +++ b/core/src/main/java/org/elasticsearch/common/network/NetworkModule.java @@ -19,6 +19,9 @@ package org.elasticsearch.common.network; +import java.util.Arrays; +import java.util.List; + import org.elasticsearch.client.support.Headers; import org.elasticsearch.client.transport.TransportClientNodesService; import org.elasticsearch.client.transport.support.TransportProxyClient; @@ -135,9 +138,6 @@ import org.elasticsearch.transport.TransportService; import org.elasticsearch.transport.local.LocalTransport; import org.elasticsearch.transport.netty.NettyTransport; -import java.util.Arrays; -import java.util.List; - /** * A module to handle registering and binding all network related classes. */ @@ -291,6 +291,7 @@ public class NetworkModule extends AbstractModule { private final ExtensionPoint.ClassSet restHandlers = new ExtensionPoint.ClassSet<>("rest_handler", RestHandler.class); // we must separate the cat rest handlers so RestCatAction can collect them... private final ExtensionPoint.ClassSet catHandlers = new ExtensionPoint.ClassSet<>("cat_handler", AbstractCatAction.class); + private final NamedWriteableRegistry namedWriteableRegistry; /** * Creates a network module that custom networking classes can be plugged into. @@ -298,11 +299,13 @@ public class NetworkModule extends AbstractModule { * @param networkService A constructed network service object to bind. * @param settings The settings for the node * @param transportClient True if only transport classes should be allowed to be registered, false otherwise. + * @param namedWriteableRegistry registry for named writeables for use during streaming */ - public NetworkModule(NetworkService networkService, Settings settings, boolean transportClient) { + public NetworkModule(NetworkService networkService, Settings settings, boolean transportClient, NamedWriteableRegistry namedWriteableRegistry) { this.networkService = networkService; this.settings = settings; this.transportClient = transportClient; + this.namedWriteableRegistry = namedWriteableRegistry; registerTransportService(NETTY_TRANSPORT, TransportService.class); registerTransport(LOCAL_TRANSPORT, LocalTransport.class); registerTransport(NETTY_TRANSPORT, NettyTransport.class); @@ -354,7 +357,7 @@ public class NetworkModule extends AbstractModule { @Override protected void configure() { bind(NetworkService.class).toInstance(networkService); - bind(NamedWriteableRegistry.class).asEagerSingleton(); + bind(NamedWriteableRegistry.class).toInstance(namedWriteableRegistry); transportServiceTypes.bindType(binder(), settings, TRANSPORT_SERVICE_TYPE_KEY, NETTY_TRANSPORT); String defaultTransport = DiscoveryNode.localNode(settings) ? LOCAL_TRANSPORT : NETTY_TRANSPORT; diff --git a/core/src/main/java/org/elasticsearch/common/settings/AbstractScopedSettings.java b/core/src/main/java/org/elasticsearch/common/settings/AbstractScopedSettings.java index f38856079842..73d13b2de208 100644 --- a/core/src/main/java/org/elasticsearch/common/settings/AbstractScopedSettings.java +++ b/core/src/main/java/org/elasticsearch/common/settings/AbstractScopedSettings.java @@ -40,7 +40,7 @@ import java.util.function.Consumer; */ public abstract class AbstractScopedSettings extends AbstractComponent { private Settings lastSettingsApplied = Settings.EMPTY; - private final List settingUpdaters = new ArrayList<>(); + private final List> settingUpdaters = new ArrayList<>(); private final Map> complexMatchers = new HashMap<>(); private final Map> keySettings = new HashMap<>(); private final Setting.Scope scope; @@ -86,7 +86,7 @@ public abstract class AbstractScopedSettings extends AbstractComponent { final Settings current = Settings.builder().put(this.settings).put(settings).build(); final Settings previous = Settings.builder().put(this.settings).put(this.lastSettingsApplied).build(); List exceptions = new ArrayList<>(); - for (SettingUpdater settingUpdater : settingUpdaters) { + for (SettingUpdater settingUpdater : settingUpdaters) { try { if (settingUpdater.hasChanged(current, previous)) { settingUpdater.getValue(current, previous); @@ -117,7 +117,7 @@ public abstract class AbstractScopedSettings extends AbstractComponent { final Settings previous = Settings.builder().put(this.settings).put(this.lastSettingsApplied).build(); try { List applyRunnables = new ArrayList<>(); - for (SettingUpdater settingUpdater : settingUpdaters) { + for (SettingUpdater settingUpdater : settingUpdaters) { try { applyRunnables.add(settingUpdater.updater(current, previous)); } catch (Exception ex) { @@ -210,6 +210,7 @@ public abstract class AbstractScopedSettings extends AbstractComponent { /** * Transactional interface to update settings. * @see Setting + * @param the type of the value of the setting */ public interface SettingUpdater { @@ -262,17 +263,16 @@ public abstract class AbstractScopedSettings extends AbstractComponent { /** * Returns the {@link Setting} for the given key or null if the setting can not be found. */ - public Setting get(String key) { + public Setting get(String key) { Setting setting = keySettings.get(key); - if (setting == null) { - for (Map.Entry> entry : complexMatchers.entrySet()) { - if (entry.getValue().match(key)) { - return entry.getValue(); - } - } - } else { + if (setting != null) { return setting; } + for (Map.Entry> entry : complexMatchers.entrySet()) { + if (entry.getValue().match(key)) { + return entry.getValue(); + } + } return null; } @@ -280,7 +280,7 @@ public abstract class AbstractScopedSettings extends AbstractComponent { * Returns true if the setting for the given key is dynamically updateable. Otherwise false. */ public boolean hasDynamicSetting(String key) { - final Setting setting = get(key); + final Setting setting = get(key); return setting != null && setting.isDynamic(); } diff --git a/core/src/main/java/org/elasticsearch/common/settings/Setting.java b/core/src/main/java/org/elasticsearch/common/settings/Setting.java index 2258b295ab91..0fc5b062e69c 100644 --- a/core/src/main/java/org/elasticsearch/common/settings/Setting.java +++ b/core/src/main/java/org/elasticsearch/common/settings/Setting.java @@ -43,6 +43,8 @@ import java.util.function.Function; import java.util.regex.Pattern; /** + * A setting. Encapsulates typical stuff like default value, parsing, and scope. + * Some (dynamic=true) can by modified at run time using the API. */ public class Setting extends ToXContentToBytes { private final String key; @@ -184,11 +186,18 @@ public class Setting extends ToXContentToBytes { INDEX; } - final AbstractScopedSettings.SettingUpdater newUpdater(Consumer consumer, ESLogger logger) { + /** + * Build a new updater with a noop validator. + */ + final AbstractScopedSettings.SettingUpdater newUpdater(Consumer consumer, ESLogger logger) { return newUpdater(consumer, logger, (s) -> {}); } - AbstractScopedSettings.SettingUpdater newUpdater(Consumer consumer, ESLogger logger, Consumer validator) { + /** + * Build the updater responsible for validating new values, logging the new + * value, and eventually setting the value where it belongs. + */ + AbstractScopedSettings.SettingUpdater newUpdater(Consumer consumer, ESLogger logger, Consumer validator) { if (isDynamic()) { return new Updater(consumer, logger, validator); } else { @@ -366,6 +375,7 @@ public class Setting extends ToXContentToBytes { return array == null ? defaultValue.apply(settings) : arrayToParsableString(array); } + @Override public boolean match(String toTest) { return pattern.matcher(toTest).matches(); } diff --git a/core/src/main/java/org/elasticsearch/index/engine/Engine.java b/core/src/main/java/org/elasticsearch/index/engine/Engine.java index 0e112118da83..79610906b14e 100644 --- a/core/src/main/java/org/elasticsearch/index/engine/Engine.java +++ b/core/src/main/java/org/elasticsearch/index/engine/Engine.java @@ -1065,7 +1065,7 @@ public abstract class Engine implements Closeable { } } - public static class CommitId implements Writeable { + public static class CommitId implements Writeable { private final byte[] id; diff --git a/core/src/main/java/org/elasticsearch/index/engine/InternalEngine.java b/core/src/main/java/org/elasticsearch/index/engine/InternalEngine.java index 74cac49b76d6..d9ee2f4177a1 100644 --- a/core/src/main/java/org/elasticsearch/index/engine/InternalEngine.java +++ b/core/src/main/java/org/elasticsearch/index/engine/InternalEngine.java @@ -231,7 +231,8 @@ public class InternalEngine extends Engine { protected void recoverFromTranslog(EngineConfig engineConfig, Translog.TranslogGeneration translogGeneration) throws IOException { int opsRecovered = 0; final TranslogRecoveryPerformer handler = engineConfig.getTranslogRecoveryPerformer(); - try (Translog.Snapshot snapshot = translog.newSnapshot()) { + try { + Translog.Snapshot snapshot = translog.newSnapshot(); Translog.Operation operation; while ((operation = snapshot.next()) != null) { try { diff --git a/core/src/main/java/org/elasticsearch/index/mapper/DocumentMapper.java b/core/src/main/java/org/elasticsearch/index/mapper/DocumentMapper.java index c2d644d393d1..0528541238ac 100644 --- a/core/src/main/java/org/elasticsearch/index/mapper/DocumentMapper.java +++ b/core/src/main/java/org/elasticsearch/index/mapper/DocumentMapper.java @@ -79,10 +79,20 @@ public class DocumentMapper implements ToXContent { this.builderContext = new Mapper.BuilderContext(indexSettings, new ContentPath(1)); this.rootObjectMapper = builder.build(builderContext); + final String type = rootObjectMapper.name(); + DocumentMapper existingMapper = mapperService.documentMapper(type); for (Map.Entry entry : mapperService.mapperRegistry.getMetadataMapperParsers().entrySet()) { final String name = entry.getKey(); - final TypeParser parser = entry.getValue(); - final MetadataFieldMapper metadataMapper = parser.getDefault(indexSettings, mapperService.fullName(name), builder.name()); + final MetadataFieldMapper existingMetadataMapper = existingMapper == null + ? null + : (MetadataFieldMapper) existingMapper.mappers().getMapper(name); + final MetadataFieldMapper metadataMapper; + if (existingMetadataMapper == null) { + final TypeParser parser = entry.getValue(); + metadataMapper = parser.getDefault(indexSettings, mapperService.fullName(name), builder.name()); + } else { + metadataMapper = existingMetadataMapper; + } metadataMappers.put(metadataMapper.getClass(), metadataMapper); } } diff --git a/core/src/main/java/org/elasticsearch/index/mapper/MapperService.java b/core/src/main/java/org/elasticsearch/index/mapper/MapperService.java index 2e517ea2bc8e..46977a72d9d4 100755 --- a/core/src/main/java/org/elasticsearch/index/mapper/MapperService.java +++ b/core/src/main/java/org/elasticsearch/index/mapper/MapperService.java @@ -20,6 +20,7 @@ package org.elasticsearch.index.mapper; import com.carrotsearch.hppc.ObjectHashSet; + import org.apache.lucene.analysis.Analyzer; import org.apache.lucene.analysis.DelegatingAnalyzerWrapper; import org.apache.lucene.index.IndexOptions; @@ -33,7 +34,6 @@ import org.apache.lucene.search.Query; import org.apache.lucene.search.TermQuery; import org.apache.lucene.util.BytesRef; import org.elasticsearch.ElasticsearchGenerationException; -import org.elasticsearch.Version; import org.elasticsearch.common.Nullable; import org.elasticsearch.common.compress.CompressedXContent; import org.elasticsearch.common.lucene.search.Queries; @@ -60,7 +60,6 @@ import java.util.Collection; import java.util.Collections; import java.util.HashMap; import java.util.HashSet; -import java.util.IdentityHashMap; import java.util.List; import java.util.Map; import java.util.Set; @@ -79,6 +78,22 @@ import static org.elasticsearch.common.collect.MapBuilder.newMapBuilder; */ public class MapperService extends AbstractIndexComponent implements Closeable { + /** + * The reason why a mapping is being merged. + */ + public enum MergeReason { + /** + * Create or update a mapping. + */ + MAPPING_UPDATE, + /** + * Recovery of an existing mapping, for instance because of a restart, + * if a shard was moved to a different node or for administrative + * purposes. + */ + MAPPING_RECOVERY; + } + public static final String DEFAULT_MAPPING = "_default_"; public static final boolean INDEX_MAPPER_DYNAMIC_DEFAULT = true; public static final Setting INDEX_MAPPER_DYNAMIC_SETTING = Setting.boolSetting("index.mapper.dynamic", INDEX_MAPPER_DYNAMIC_DEFAULT, false, Setting.Scope.INDEX); @@ -204,7 +219,7 @@ public class MapperService extends AbstractIndexComponent implements Closeable { typeListeners.remove(listener); } - public DocumentMapper merge(String type, CompressedXContent mappingSource, boolean applyDefault, boolean updateAllTypes) { + public DocumentMapper merge(String type, CompressedXContent mappingSource, MergeReason reason, boolean updateAllTypes) { if (DEFAULT_MAPPING.equals(type)) { // verify we can parse it // NOTE: never apply the default here @@ -222,9 +237,13 @@ public class MapperService extends AbstractIndexComponent implements Closeable { return mapper; } else { synchronized (this) { - // only apply the default mapping if we don't have the type yet - applyDefault &= mappers.containsKey(type) == false; - return merge(parse(type, mappingSource, applyDefault), updateAllTypes); + final boolean applyDefault = + // the default was already applied if we are recovering + reason != MergeReason.MAPPING_RECOVERY + // only apply the default mapping if we don't have the type yet + && mappers.containsKey(type) == false; + DocumentMapper mergeWith = parse(type, mappingSource, applyDefault); + return merge(mergeWith, updateAllTypes); } } } diff --git a/core/src/main/java/org/elasticsearch/index/mapper/ip/IpFieldMapper.java b/core/src/main/java/org/elasticsearch/index/mapper/ip/IpFieldMapper.java index 9984463ffc0c..c83428d2239d 100644 --- a/core/src/main/java/org/elasticsearch/index/mapper/ip/IpFieldMapper.java +++ b/core/src/main/java/org/elasticsearch/index/mapper/ip/IpFieldMapper.java @@ -213,11 +213,24 @@ public class IpFieldMapper extends NumberFieldMapper { @Override public Query termQuery(Object value, @Nullable QueryShardContext context) { if (value != null) { - long[] fromTo; + String term; if (value instanceof BytesRef) { - fromTo = Cidrs.cidrMaskToMinMax(((BytesRef) value).utf8ToString()); + term = ((BytesRef) value).utf8ToString(); } else { - fromTo = Cidrs.cidrMaskToMinMax(value.toString()); + term = value.toString(); + } + long[] fromTo; + // assume that the term is either a CIDR range or the + // term is a single IPv4 address; if either of these + // assumptions is wrong, the CIDR parsing will fail + // anyway, and that is okay + if (term.contains("/")) { + // treat the term as if it is in CIDR notation + fromTo = Cidrs.cidrMaskToMinMax(term); + } else { + // treat the term as if it is a single IPv4, and + // apply a CIDR mask equivalent to the host route + fromTo = Cidrs.cidrMaskToMinMax(term + "/32"); } if (fromTo != null) { return rangeQuery(fromTo[0] == 0 ? null : fromTo[0], 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 79ba3804ecdd..d2116ae3c05f 100644 --- a/core/src/main/java/org/elasticsearch/index/query/AbstractQueryBuilder.java +++ b/core/src/main/java/org/elasticsearch/index/query/AbstractQueryBuilder.java @@ -42,7 +42,7 @@ import java.util.Objects; * Base class for all classes producing lucene queries. * Supports conversion to BytesReference and creation of lucene Query objects. */ -public abstract class AbstractQueryBuilder extends ToXContentToBytes implements QueryBuilder { +public abstract class AbstractQueryBuilder> extends ToXContentToBytes implements QueryBuilder { /** Default for boost to apply to resulting Lucene query. Defaults to 1.0*/ public static final float DEFAULT_BOOST = 1.0f; @@ -225,10 +225,10 @@ public abstract class AbstractQueryBuilder exte * their {@link QueryBuilder#toQuery(QueryShardContext)} method are not added to the * resulting collection. */ - protected static Collection toQueries(Collection queryBuilders, QueryShardContext context) throws QueryShardException, + protected static Collection toQueries(Collection> queryBuilders, QueryShardContext context) throws QueryShardException, IOException { List queries = new ArrayList<>(queryBuilders.size()); - for (QueryBuilder queryBuilder : queryBuilders) { + for (QueryBuilder queryBuilder : queryBuilders) { Query query = queryBuilder.toQuery(context); if (query != null) { queries.add(query); @@ -243,15 +243,15 @@ public abstract class AbstractQueryBuilder exte return getWriteableName(); } - protected final void writeQueries(StreamOutput out, List queries) throws IOException { + protected final void writeQueries(StreamOutput out, List> queries) throws IOException { out.writeVInt(queries.size()); - for (QueryBuilder query : queries) { + for (QueryBuilder query : queries) { out.writeQuery(query); } } - protected final List readQueries(StreamInput in) throws IOException { - List queries = new ArrayList<>(); + protected final List> readQueries(StreamInput in) throws IOException { + List> queries = new ArrayList<>(); int size = in.readVInt(); for (int i = 0; i < size; i++) { queries.add(in.readQuery()); diff --git a/core/src/main/java/org/elasticsearch/index/query/BoolQueryBuilder.java b/core/src/main/java/org/elasticsearch/index/query/BoolQueryBuilder.java index 69ee2a810619..f7f4926d950d 100644 --- a/core/src/main/java/org/elasticsearch/index/query/BoolQueryBuilder.java +++ b/core/src/main/java/org/elasticsearch/index/query/BoolQueryBuilder.java @@ -49,13 +49,13 @@ public class BoolQueryBuilder extends AbstractQueryBuilder { static final BoolQueryBuilder PROTOTYPE = new BoolQueryBuilder(); - private final List mustClauses = new ArrayList<>(); + private final List> mustClauses = new ArrayList<>(); - private final List mustNotClauses = new ArrayList<>(); + private final List> mustNotClauses = new ArrayList<>(); - private final List filterClauses = new ArrayList<>(); + private final List> filterClauses = new ArrayList<>(); - private final List shouldClauses = new ArrayList<>(); + private final List> shouldClauses = new ArrayList<>(); private boolean disableCoord = DISABLE_COORD_DEFAULT; @@ -67,7 +67,7 @@ public class BoolQueryBuilder extends AbstractQueryBuilder { * Adds a query that must appear in the matching documents and will * contribute to scoring. No null value allowed. */ - public BoolQueryBuilder must(QueryBuilder queryBuilder) { + public BoolQueryBuilder must(QueryBuilder queryBuilder) { if (queryBuilder == null) { throw new IllegalArgumentException("inner bool query clause cannot be null"); } @@ -78,7 +78,7 @@ public class BoolQueryBuilder extends AbstractQueryBuilder { /** * Gets the queries that must appear in the matching documents. */ - public List must() { + public List> must() { return this.mustClauses; } @@ -86,7 +86,7 @@ public class BoolQueryBuilder extends AbstractQueryBuilder { * Adds a query that must appear in the matching documents but will * not contribute to scoring. No null value allowed. */ - public BoolQueryBuilder filter(QueryBuilder queryBuilder) { + public BoolQueryBuilder filter(QueryBuilder queryBuilder) { if (queryBuilder == null) { throw new IllegalArgumentException("inner bool query clause cannot be null"); } @@ -95,9 +95,9 @@ public class BoolQueryBuilder extends AbstractQueryBuilder { } /** - * Gets the queries that must appear in the matching documents but don't conntribute to scoring + * Gets the queries that must appear in the matching documents but don't contribute to scoring */ - public List filter() { + public List> filter() { return this.filterClauses; } @@ -105,7 +105,7 @@ public class BoolQueryBuilder extends AbstractQueryBuilder { * Adds a query that must not appear in the matching documents. * No null value allowed. */ - public BoolQueryBuilder mustNot(QueryBuilder queryBuilder) { + public BoolQueryBuilder mustNot(QueryBuilder queryBuilder) { if (queryBuilder == null) { throw new IllegalArgumentException("inner bool query clause cannot be null"); } @@ -116,7 +116,7 @@ public class BoolQueryBuilder extends AbstractQueryBuilder { /** * Gets the queries that must not appear in the matching documents. */ - public List mustNot() { + public List> mustNot() { return this.mustNotClauses; } @@ -127,7 +127,7 @@ public class BoolQueryBuilder extends AbstractQueryBuilder { * * @see #minimumNumberShouldMatch(int) */ - public BoolQueryBuilder should(QueryBuilder queryBuilder) { + public BoolQueryBuilder should(QueryBuilder queryBuilder) { if (queryBuilder == null) { throw new IllegalArgumentException("inner bool query clause cannot be null"); } @@ -141,7 +141,7 @@ public class BoolQueryBuilder extends AbstractQueryBuilder { * @see #should(QueryBuilder) * @see #minimumNumberShouldMatch(int) */ - public List should() { + public List> should() { return this.shouldClauses; } @@ -244,12 +244,12 @@ public class BoolQueryBuilder extends AbstractQueryBuilder { builder.endObject(); } - private static void doXArrayContent(String field, List clauses, XContentBuilder builder, Params params) throws IOException { + private static void doXArrayContent(String field, List> clauses, XContentBuilder builder, Params params) throws IOException { if (clauses.isEmpty()) { return; } builder.startArray(field); - for (QueryBuilder clause : clauses) { + for (QueryBuilder clause : clauses) { clause.toXContent(builder, params); } builder.endArray(); @@ -282,8 +282,8 @@ public class BoolQueryBuilder extends AbstractQueryBuilder { return adjustPureNegative ? fixNegativeQueryIfNeeded(query) : query; } - private static void addBooleanClauses(QueryShardContext context, BooleanQuery.Builder booleanQueryBuilder, List clauses, Occur occurs) throws IOException { - for (QueryBuilder query : clauses) { + private static void addBooleanClauses(QueryShardContext context, BooleanQuery.Builder booleanQueryBuilder, List> clauses, Occur occurs) throws IOException { + for (QueryBuilder query : clauses) { Query luceneQuery = null; switch (occurs) { case MUST: @@ -321,7 +321,7 @@ public class BoolQueryBuilder extends AbstractQueryBuilder { @Override protected BoolQueryBuilder doReadFrom(StreamInput in) throws IOException { BoolQueryBuilder boolQueryBuilder = new BoolQueryBuilder(); - List queryBuilders = readQueries(in); + List> queryBuilders = readQueries(in); boolQueryBuilder.mustClauses.addAll(queryBuilders); queryBuilders = readQueries(in); boolQueryBuilder.mustNotClauses.addAll(queryBuilders); diff --git a/core/src/main/java/org/elasticsearch/index/query/BoolQueryParser.java b/core/src/main/java/org/elasticsearch/index/query/BoolQueryParser.java index d0d130f9774e..53c1b81989f8 100644 --- a/core/src/main/java/org/elasticsearch/index/query/BoolQueryParser.java +++ b/core/src/main/java/org/elasticsearch/index/query/BoolQueryParser.java @@ -19,17 +19,14 @@ package org.elasticsearch.index.query; -import org.apache.lucene.search.BooleanQuery; -import org.elasticsearch.common.ParseField; -import org.elasticsearch.common.ParsingException; -import org.elasticsearch.common.inject.Inject; -import org.elasticsearch.common.settings.Settings; -import org.elasticsearch.common.xcontent.XContentParser; - import java.io.IOException; import java.util.ArrayList; import java.util.List; +import org.elasticsearch.common.ParseField; +import org.elasticsearch.common.ParsingException; +import org.elasticsearch.common.xcontent.XContentParser; + /** * Parser for bool query */ @@ -45,11 +42,6 @@ public class BoolQueryParser implements QueryParser { public static final ParseField MINIMUM_NUMBER_SHOULD_MATCH = new ParseField("minimum_number_should_match"); public static final ParseField ADJUST_PURE_NEGATIVE = new ParseField("adjust_pure_negative"); - @Inject - public BoolQueryParser(Settings settings) { - BooleanQuery.setMaxClauseCount(settings.getAsInt("index.query.bool.max_clause_count", settings.getAsInt("indices.query.bool.max_clause_count", BooleanQuery.getMaxClauseCount()))); - } - @Override public String[] names() { return new String[]{BoolQueryBuilder.NAME}; diff --git a/core/src/main/java/org/elasticsearch/index/query/DisMaxQueryBuilder.java b/core/src/main/java/org/elasticsearch/index/query/DisMaxQueryBuilder.java index 43b1917d3181..e682ae2c1c08 100644 --- a/core/src/main/java/org/elasticsearch/index/query/DisMaxQueryBuilder.java +++ b/core/src/main/java/org/elasticsearch/index/query/DisMaxQueryBuilder.java @@ -40,7 +40,7 @@ public class DisMaxQueryBuilder extends AbstractQueryBuilder public static final String NAME = "dis_max"; - private final ArrayList queries = new ArrayList<>(); + private final List> queries = new ArrayList<>(); /** Default multiplication factor for breaking ties in document scores.*/ public static float DEFAULT_TIE_BREAKER = 0.0f; @@ -51,7 +51,7 @@ public class DisMaxQueryBuilder extends AbstractQueryBuilder /** * Add a sub-query to this disjunction. */ - public DisMaxQueryBuilder add(QueryBuilder queryBuilder) { + public DisMaxQueryBuilder add(QueryBuilder queryBuilder) { if (queryBuilder == null) { throw new IllegalArgumentException("inner dismax query clause cannot be null"); } @@ -62,7 +62,7 @@ public class DisMaxQueryBuilder extends AbstractQueryBuilder /** * @return an immutable list copy of the current sub-queries of this disjunction */ - public List innerQueries() { + public List> innerQueries() { return this.queries; } @@ -90,7 +90,7 @@ public class DisMaxQueryBuilder extends AbstractQueryBuilder builder.startObject(NAME); builder.field(DisMaxQueryParser.TIE_BREAKER_FIELD.getPreferredName(), tieBreaker); builder.startArray(DisMaxQueryParser.QUERIES_FIELD.getPreferredName()); - for (QueryBuilder queryBuilder : queries) { + for (QueryBuilder queryBuilder : queries) { queryBuilder.toXContent(builder, params); } builder.endArray(); @@ -112,7 +112,7 @@ public class DisMaxQueryBuilder extends AbstractQueryBuilder @Override protected DisMaxQueryBuilder doReadFrom(StreamInput in) throws IOException { DisMaxQueryBuilder disMax = new DisMaxQueryBuilder(); - List queryBuilders = readQueries(in); + List> queryBuilders = readQueries(in); disMax.queries.addAll(queryBuilders); disMax.tieBreaker = in.readFloat(); return disMax; diff --git a/core/src/main/java/org/elasticsearch/index/query/NestedQueryParser.java b/core/src/main/java/org/elasticsearch/index/query/NestedQueryParser.java index 7cdb66bd126b..ba5d7c2447e2 100644 --- a/core/src/main/java/org/elasticsearch/index/query/NestedQueryParser.java +++ b/core/src/main/java/org/elasticsearch/index/query/NestedQueryParser.java @@ -19,15 +19,14 @@ package org.elasticsearch.index.query; +import java.io.IOException; + import org.apache.lucene.search.join.ScoreMode; import org.elasticsearch.common.ParseField; import org.elasticsearch.common.ParsingException; -import org.elasticsearch.common.Strings; import org.elasticsearch.common.xcontent.XContentParser; import org.elasticsearch.index.query.support.QueryInnerHits; -import java.io.IOException; - public class NestedQueryParser implements QueryParser { private static final NestedQueryBuilder PROTOTYPE = new NestedQueryBuilder("", EmptyQueryBuilder.PROTOTYPE); @@ -38,7 +37,7 @@ public class NestedQueryParser implements QueryParser { @Override public String[] names() { - return new String[]{NestedQueryBuilder.NAME, Strings.toCamelCase(NestedQueryBuilder.NAME)}; + return new String[]{NestedQueryBuilder.NAME}; } @Override diff --git a/core/src/main/java/org/elasticsearch/index/query/QueryBuilder.java b/core/src/main/java/org/elasticsearch/index/query/QueryBuilder.java index 2fde316a561b..b75406c8641b 100644 --- a/core/src/main/java/org/elasticsearch/index/query/QueryBuilder.java +++ b/core/src/main/java/org/elasticsearch/index/query/QueryBuilder.java @@ -25,7 +25,7 @@ import org.elasticsearch.common.xcontent.ToXContent; import java.io.IOException; -public interface QueryBuilder extends NamedWriteable, ToXContent { +public interface QueryBuilder> extends NamedWriteable, ToXContent { /** * Converts this QueryBuilder to a lucene {@link Query}. diff --git a/core/src/main/java/org/elasticsearch/index/query/SpanNearQueryBuilder.java b/core/src/main/java/org/elasticsearch/index/query/SpanNearQueryBuilder.java index 80eb94f42769..5f30ac28e7fd 100644 --- a/core/src/main/java/org/elasticsearch/index/query/SpanNearQueryBuilder.java +++ b/core/src/main/java/org/elasticsearch/index/query/SpanNearQueryBuilder.java @@ -43,7 +43,7 @@ public class SpanNearQueryBuilder extends AbstractQueryBuilder clauses = new ArrayList<>(); + private final List> clauses = new ArrayList<>(); private final int slop; @@ -55,7 +55,7 @@ public class SpanNearQueryBuilder extends AbstractQueryBuilder initialClause, int slop) { if (initialClause == null) { throw new IllegalArgumentException("query must include at least one clause"); } @@ -70,7 +70,7 @@ public class SpanNearQueryBuilder extends AbstractQueryBuilder clause) { if (clause == null) { throw new IllegalArgumentException("query clauses cannot be null"); } @@ -81,7 +81,7 @@ public class SpanNearQueryBuilder extends AbstractQueryBuilder clauses() { + public List> clauses() { return this.clauses; } @@ -106,7 +106,7 @@ public class SpanNearQueryBuilder extends AbstractQueryBuilder clause : clauses) { clause.toXContent(builder, params); } builder.endArray(); @@ -129,10 +129,10 @@ public class SpanNearQueryBuilder extends AbstractQueryBuilder clauses = readQueries(in); - SpanNearQueryBuilder queryBuilder = new SpanNearQueryBuilder((SpanQueryBuilder)clauses.get(0), in.readVInt()); + List> clauses = readQueries(in); + SpanNearQueryBuilder queryBuilder = new SpanNearQueryBuilder((SpanQueryBuilder)clauses.get(0), in.readVInt()); for (int i = 1; i < clauses.size(); i++) { - queryBuilder.clauses.add((SpanQueryBuilder)clauses.get(i)); + queryBuilder.clauses.add((SpanQueryBuilder)clauses.get(i)); } queryBuilder.inOrder = in.readBoolean(); return queryBuilder; diff --git a/core/src/main/java/org/elasticsearch/index/query/SpanOrQueryBuilder.java b/core/src/main/java/org/elasticsearch/index/query/SpanOrQueryBuilder.java index 3b8681c685b7..8d1fb4f7ff32 100644 --- a/core/src/main/java/org/elasticsearch/index/query/SpanOrQueryBuilder.java +++ b/core/src/main/java/org/elasticsearch/index/query/SpanOrQueryBuilder.java @@ -38,18 +38,18 @@ public class SpanOrQueryBuilder extends AbstractQueryBuilder public static final String NAME = "span_or"; - private final List clauses = new ArrayList<>(); + private final List> clauses = new ArrayList<>(); static final SpanOrQueryBuilder PROTOTYPE = new SpanOrQueryBuilder(SpanTermQueryBuilder.PROTOTYPE); - public SpanOrQueryBuilder(SpanQueryBuilder initialClause) { + public SpanOrQueryBuilder(SpanQueryBuilder initialClause) { if (initialClause == null) { throw new IllegalArgumentException("query must include at least one clause"); } clauses.add(initialClause); } - public SpanOrQueryBuilder clause(SpanQueryBuilder clause) { + public SpanOrQueryBuilder clause(SpanQueryBuilder clause) { if (clause == null) { throw new IllegalArgumentException("inner bool query clause cannot be null"); } @@ -60,7 +60,7 @@ public class SpanOrQueryBuilder extends AbstractQueryBuilder /** * @return the {@link SpanQueryBuilder} clauses that were set for this query */ - public List clauses() { + public List> clauses() { return this.clauses; } @@ -68,7 +68,7 @@ public class SpanOrQueryBuilder extends AbstractQueryBuilder protected void doXContent(XContentBuilder builder, Params params) throws IOException { builder.startObject(NAME); builder.startArray(SpanOrQueryParser.CLAUSES_FIELD.getPreferredName()); - for (SpanQueryBuilder clause : clauses) { + for (SpanQueryBuilder clause : clauses) { clause.toXContent(builder, params); } builder.endArray(); @@ -89,10 +89,10 @@ public class SpanOrQueryBuilder extends AbstractQueryBuilder @Override protected SpanOrQueryBuilder doReadFrom(StreamInput in) throws IOException { - List clauses = readQueries(in); - SpanOrQueryBuilder queryBuilder = new SpanOrQueryBuilder((SpanQueryBuilder)clauses.get(0)); + List> clauses = readQueries(in); + SpanOrQueryBuilder queryBuilder = new SpanOrQueryBuilder((SpanQueryBuilder)clauses.get(0)); for (int i = 1; i < clauses.size(); i++) { - queryBuilder.clauses.add((SpanQueryBuilder)clauses.get(i)); + queryBuilder.clauses.add((SpanQueryBuilder)clauses.get(i)); } return queryBuilder; diff --git a/core/src/main/java/org/elasticsearch/index/query/SpanQueryBuilder.java b/core/src/main/java/org/elasticsearch/index/query/SpanQueryBuilder.java index d35dcbc536a5..90a75a5af1b7 100644 --- a/core/src/main/java/org/elasticsearch/index/query/SpanQueryBuilder.java +++ b/core/src/main/java/org/elasticsearch/index/query/SpanQueryBuilder.java @@ -22,6 +22,6 @@ package org.elasticsearch.index.query; /** * Marker interface for a specific type of {@link QueryBuilder} that allows to build span queries */ -public interface SpanQueryBuilder extends QueryBuilder { +public interface SpanQueryBuilder> extends QueryBuilder { } diff --git a/core/src/main/java/org/elasticsearch/index/query/functionscore/FunctionScoreQueryParser.java b/core/src/main/java/org/elasticsearch/index/query/functionscore/FunctionScoreQueryParser.java index 6822ab3e240c..8e503e24d3c6 100644 --- a/core/src/main/java/org/elasticsearch/index/query/functionscore/FunctionScoreQueryParser.java +++ b/core/src/main/java/org/elasticsearch/index/query/functionscore/FunctionScoreQueryParser.java @@ -19,10 +19,13 @@ package org.elasticsearch.index.query.functionscore; +import java.io.IOException; +import java.util.ArrayList; +import java.util.List; + import org.elasticsearch.common.ParseField; import org.elasticsearch.common.ParsingException; import org.elasticsearch.common.Strings; -import org.elasticsearch.common.inject.Inject; import org.elasticsearch.common.lucene.search.function.CombineFunction; import org.elasticsearch.common.lucene.search.function.FiltersFunctionScoreQuery; import org.elasticsearch.common.lucene.search.function.FunctionScoreQuery; @@ -36,10 +39,6 @@ import org.elasticsearch.index.query.QueryParseContext; import org.elasticsearch.index.query.QueryParser; import org.elasticsearch.index.query.functionscore.weight.WeightBuilder; -import java.io.IOException; -import java.util.ArrayList; -import java.util.List; - /** * Parser for function_score query */ @@ -54,7 +53,6 @@ public class FunctionScoreQueryParser implements QueryParser> functionParsers; - @Inject - public ScoreFunctionParserMapper(Set parsers, NamedWriteableRegistry namedWriteableRegistry) { - Map> map = new HashMap<>(); - // built-in parsers - addParser(new ScriptScoreFunctionParser(), map, namedWriteableRegistry); - addParser(new GaussDecayFunctionParser(), map, namedWriteableRegistry); - addParser(new LinearDecayFunctionParser(), map, namedWriteableRegistry); - addParser(new ExponentialDecayFunctionParser(), map, namedWriteableRegistry); - addParser(new RandomScoreFunctionParser(), map, namedWriteableRegistry); - addParser(new FieldValueFactorFunctionParser(), map, namedWriteableRegistry); - for (ScoreFunctionParser scoreFunctionParser : parsers) { - addParser(scoreFunctionParser, map, namedWriteableRegistry); - } - this.functionParsers = Collections.unmodifiableMap(map); - //weight doesn't have its own parser, so every function supports it out of the box. - //Can be a single function too when not associated to any other function, which is why it needs to be registered manually here. - namedWriteableRegistry.registerPrototype(ScoreFunctionBuilder.class, new WeightBuilder()); + public ScoreFunctionParserMapper(Map> functionParsers) { + this.functionParsers = unmodifiableMap(functionParsers); } - public ScoreFunctionParser get(XContentLocation contentLocation, String parserName) { - ScoreFunctionParser functionParser = get(parserName); + public ScoreFunctionParser get(XContentLocation contentLocation, String parserName) { + ScoreFunctionParser functionParser = get(parserName); if (functionParser == null) { throw new ParsingException(contentLocation, "No function with the name [" + parserName + "] is registered."); } return functionParser; } - private ScoreFunctionParser get(String parserName) { + private ScoreFunctionParser get(String parserName) { return functionParsers.get(parserName); } - - private static void addParser(ScoreFunctionParser scoreFunctionParser, Map> map, NamedWriteableRegistry namedWriteableRegistry) { - for (String name : scoreFunctionParser.getNames()) { - map.put(name, scoreFunctionParser); - - } - @SuppressWarnings("unchecked") NamedWriteable sfb = scoreFunctionParser.getBuilderPrototype(); - namedWriteableRegistry.registerPrototype(ScoreFunctionBuilder.class, sfb); - } } diff --git a/core/src/main/java/org/elasticsearch/index/translog/BaseTranslogReader.java b/core/src/main/java/org/elasticsearch/index/translog/BaseTranslogReader.java new file mode 100644 index 000000000000..c98ea69f87f6 --- /dev/null +++ b/core/src/main/java/org/elasticsearch/index/translog/BaseTranslogReader.java @@ -0,0 +1,136 @@ +/* + * Licensed to Elasticsearch under one or more contributor + * license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright + * ownership. Elasticsearch licenses this file to you under + * the Apache License, Version 2.0 (the "License"); you may + * not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.elasticsearch.index.translog; + +import org.elasticsearch.ElasticsearchException; +import org.elasticsearch.common.io.stream.ByteBufferStreamInput; + +import java.io.IOException; +import java.nio.ByteBuffer; +import java.nio.channels.FileChannel; +import java.nio.file.Path; + +/** + * A base class for all classes that allows reading ops from translog files + */ +public abstract class BaseTranslogReader implements Comparable { + + protected final long generation; + protected final FileChannel channel; + protected final Path path; + protected final long firstOperationOffset; + + public BaseTranslogReader(long generation, FileChannel channel, Path path, long firstOperationOffset) { + assert Translog.parseIdFromFileName(path) == generation : "generation missmatch. Path: " + Translog.parseIdFromFileName(path) + " but generation: " + generation; + + this.generation = generation; + this.path = path; + this.channel = channel; + this.firstOperationOffset = firstOperationOffset; + } + + public long getGeneration() { + return this.generation; + } + + public abstract long sizeInBytes(); + + abstract public int totalOperations(); + + public final long getFirstOperationOffset() { + return firstOperationOffset; + } + + public Translog.Operation read(Translog.Location location) throws IOException { + assert location.generation == generation : "read location's translog generation [" + location.generation + "] is not [" + generation + "]"; + ByteBuffer buffer = ByteBuffer.allocate(location.size); + try (BufferedChecksumStreamInput checksumStreamInput = checksummedStream(buffer, location.translogLocation, location.size, null)) { + return read(checksumStreamInput); + } + } + + /** read the size of the op (i.e., number of bytes, including the op size) written at the given position */ + protected final int readSize(ByteBuffer reusableBuffer, long position) { + // read op size from disk + assert reusableBuffer.capacity() >= 4 : "reusable buffer must have capacity >=4 when reading opSize. got [" + reusableBuffer.capacity() + "]"; + try { + reusableBuffer.clear(); + reusableBuffer.limit(4); + readBytes(reusableBuffer, position); + reusableBuffer.flip(); + // Add an extra 4 to account for the operation size integer itself + final int size = reusableBuffer.getInt() + 4; + final long maxSize = sizeInBytes() - position; + if (size < 0 || size > maxSize) { + throw new TranslogCorruptedException("operation size is corrupted must be [0.." + maxSize + "] but was: " + size); + } + + return size; + } catch (IOException e) { + throw new ElasticsearchException("unexpected exception reading from translog snapshot of " + this.path, e); + } + } + + public Translog.Snapshot newSnapshot() { + return new TranslogSnapshot(generation, channel, path, firstOperationOffset, sizeInBytes(), totalOperations()); + } + + /** + * reads an operation at the given position and returns it. The buffer length is equal to the number + * of bytes reads. + */ + protected final BufferedChecksumStreamInput checksummedStream(ByteBuffer reusableBuffer, long position, int opSize, BufferedChecksumStreamInput reuse) throws IOException { + final ByteBuffer buffer; + if (reusableBuffer.capacity() >= opSize) { + buffer = reusableBuffer; + } else { + buffer = ByteBuffer.allocate(opSize); + } + buffer.clear(); + buffer.limit(opSize); + readBytes(buffer, position); + buffer.flip(); + return new BufferedChecksumStreamInput(new ByteBufferStreamInput(buffer), reuse); + } + + protected Translog.Operation read(BufferedChecksumStreamInput inStream) throws IOException { + return Translog.readOperation(inStream); + } + + /** + * reads bytes at position into the given buffer, filling it. + */ + abstract protected void readBytes(ByteBuffer buffer, long position) throws IOException; + + @Override + public String toString() { + return "translog [" + generation + "][" + path + "]"; + } + + @Override + public int compareTo(BaseTranslogReader o) { + return Long.compare(getGeneration(), o.getGeneration()); + } + + + public Path path() { + return path; + } +} diff --git a/core/src/main/java/org/elasticsearch/index/translog/ChannelReference.java b/core/src/main/java/org/elasticsearch/index/translog/ChannelReference.java deleted file mode 100644 index b3f60a4c89f8..000000000000 --- a/core/src/main/java/org/elasticsearch/index/translog/ChannelReference.java +++ /dev/null @@ -1,71 +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.translog; - -import org.apache.lucene.util.IOUtils; -import org.elasticsearch.common.util.Callback; -import org.elasticsearch.common.util.concurrent.AbstractRefCounted; - -import java.io.IOException; -import java.nio.channels.FileChannel; -import java.nio.file.Path; - -final class ChannelReference extends AbstractRefCounted { - private final Path file; - private final FileChannel channel; - protected final long generation; - private final Callback onClose; - - ChannelReference(Path file, long generation, FileChannel channel, Callback onClose) throws IOException { - super(file.toString()); - this.generation = generation; - this.file = file; - this.channel = channel; - this.onClose = onClose; - } - - public long getGeneration() { - return generation; - } - - public Path getPath() { - return this.file; - } - - public FileChannel getChannel() { - return this.channel; - } - - @Override - public String toString() { - return "channel: file [" + file + "], ref count [" + refCount() + "]"; - } - - @Override - protected void closeInternal() { - try { - IOUtils.closeWhileHandlingException(channel); - } finally { - if (onClose != null) { - onClose.handle(this); - } - } - } -} diff --git a/core/src/main/java/org/elasticsearch/index/translog/LegacyTranslogReader.java b/core/src/main/java/org/elasticsearch/index/translog/LegacyTranslogReader.java deleted file mode 100644 index 463c5998f1d2..000000000000 --- a/core/src/main/java/org/elasticsearch/index/translog/LegacyTranslogReader.java +++ /dev/null @@ -1,58 +0,0 @@ -/* - * Licensed to Elasticsearch under one or more contributor - * license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright - * ownership. Elasticsearch licenses this file to you under - * the Apache License, Version 2.0 (the "License"); you may - * not use this file except in compliance with the License. - * You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, - * software distributed under the License is distributed on an - * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY - * KIND, either express or implied. See the License for the - * specific language governing permissions and limitations - * under the License. - */ - -package org.elasticsearch.index.translog; - -import java.io.IOException; - -/** - * Version 0 of the translog format, there is no header in this file - */ -@Deprecated -public final class LegacyTranslogReader extends LegacyTranslogReaderBase { - - /** - * Create a snapshot of translog file channel. The length parameter should be consistent with totalOperations and point - * at the end of the last operation in this snapshot. - */ - LegacyTranslogReader(long generation, ChannelReference channelReference, long fileLength) { - super(generation, channelReference, 0, fileLength); - } - - @Override - protected Translog.Operation read(BufferedChecksumStreamInput in) throws IOException { - // read the opsize before an operation. - // Note that this was written & read out side of the stream when this class was used, but it makes things more consistent - // to read this here - in.readInt(); - Translog.Operation.Type type = Translog.Operation.Type.fromId(in.readByte()); - Translog.Operation operation = Translog.newOperationFromType(type); - operation.readFrom(in); - return operation; - } - - - - @Override - protected ImmutableTranslogReader newReader(long generation, ChannelReference channelReference, long firstOperationOffset, long length, int totalOperations) { - assert totalOperations == -1 : "expected unknown but was: " + totalOperations; - assert firstOperationOffset == 0; - return new LegacyTranslogReader(generation, channelReference, length); - } -} diff --git a/core/src/main/java/org/elasticsearch/index/translog/LegacyTranslogReaderBase.java b/core/src/main/java/org/elasticsearch/index/translog/LegacyTranslogReaderBase.java deleted file mode 100644 index d9e9e17f7929..000000000000 --- a/core/src/main/java/org/elasticsearch/index/translog/LegacyTranslogReaderBase.java +++ /dev/null @@ -1,64 +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.translog; - -import java.io.IOException; -import java.nio.ByteBuffer; - -/** - * Version 1 of the translog format, there is checkpoint and therefore no notion of op count - */ -@Deprecated -class LegacyTranslogReaderBase extends ImmutableTranslogReader { - - /** - * Create a snapshot of translog file channel. The length parameter should be consistent with totalOperations and point - * at the end of the last operation in this snapshot. - * - */ - LegacyTranslogReaderBase(long generation, ChannelReference channelReference, long firstOperationOffset, long fileLength) { - super(generation, channelReference, firstOperationOffset, fileLength, TranslogReader.UNKNOWN_OP_COUNT); - } - - - @Override - protected Translog.Snapshot newReaderSnapshot(final int totalOperations, ByteBuffer reusableBuffer) { - assert totalOperations == -1 : "legacy we had no idea how many ops: " + totalOperations; - return new ReaderSnapshot(totalOperations, reusableBuffer) { - @Override - public Translog.Operation next() throws IOException { - if (position >= sizeInBytes()) { // this is the legacy case.... - return null; - } - try { - return readOperation(); - } catch (TruncatedTranslogException ex) { - return null; // legacy case - } - } - }; - } - - @Override - protected ImmutableTranslogReader newReader(long generation, ChannelReference channelReference, long firstOperationOffset, long length, int totalOperations) { - assert totalOperations == -1 : "expected unknown but was: " + totalOperations; - return new LegacyTranslogReaderBase(generation, channelReference, firstOperationOffset, length); - } -} diff --git a/core/src/main/java/org/elasticsearch/index/translog/MultiSnapshot.java b/core/src/main/java/org/elasticsearch/index/translog/MultiSnapshot.java index b76214dc2e7c..7b1a05e1ac1e 100644 --- a/core/src/main/java/org/elasticsearch/index/translog/MultiSnapshot.java +++ b/core/src/main/java/org/elasticsearch/index/translog/MultiSnapshot.java @@ -19,12 +19,8 @@ package org.elasticsearch.index.translog; -import org.apache.lucene.store.AlreadyClosedException; -import org.elasticsearch.ElasticsearchException; -import org.elasticsearch.common.lease.Releasables; - import java.io.IOException; -import java.util.concurrent.atomic.AtomicBoolean; +import java.util.Arrays; /** * A snapshot composed out of multiple snapshots @@ -32,8 +28,7 @@ import java.util.concurrent.atomic.AtomicBoolean; final class MultiSnapshot implements Translog.Snapshot { private final Translog.Snapshot[] translogs; - private AtomicBoolean closed = new AtomicBoolean(false); - private final int estimatedTotalOperations; + private final int totalOperations; private int index; /** @@ -41,30 +36,18 @@ final class MultiSnapshot implements Translog.Snapshot { */ MultiSnapshot(Translog.Snapshot[] translogs) { this.translogs = translogs; - int ops = 0; - for (Translog.Snapshot translog : translogs) { - - final int tops = translog.estimatedTotalOperations(); - if (tops == TranslogReader.UNKNOWN_OP_COUNT) { - ops = TranslogReader.UNKNOWN_OP_COUNT; - break; - } - assert tops >= 0 : "tops must be positive but was: " + tops; - ops += tops; - } - estimatedTotalOperations = ops; + totalOperations = Arrays.stream(translogs).mapToInt(Translog.Snapshot::totalOperations).sum(); index = 0; } @Override - public int estimatedTotalOperations() { - return estimatedTotalOperations; + public int totalOperations() { + return totalOperations; } @Override public Translog.Operation next() throws IOException { - ensureOpen(); for (; index < translogs.length; index++) { final Translog.Snapshot current = translogs[index]; Translog.Operation op = current.next(); @@ -74,17 +57,4 @@ final class MultiSnapshot implements Translog.Snapshot { } return null; } - - protected void ensureOpen() { - if (closed.get()) { - throw new AlreadyClosedException("snapshot already closed"); - } - } - - @Override - public void close() throws ElasticsearchException { - if (closed.compareAndSet(false, true)) { - Releasables.close(translogs); - } - } } diff --git a/core/src/main/java/org/elasticsearch/index/translog/Translog.java b/core/src/main/java/org/elasticsearch/index/translog/Translog.java index 0001fda07527..b2e81de044b0 100644 --- a/core/src/main/java/org/elasticsearch/index/translog/Translog.java +++ b/core/src/main/java/org/elasticsearch/index/translog/Translog.java @@ -34,12 +34,9 @@ import org.elasticsearch.common.io.stream.ReleasableBytesStreamOutput; import org.elasticsearch.common.io.stream.StreamInput; import org.elasticsearch.common.io.stream.StreamOutput; import org.elasticsearch.common.io.stream.Streamable; -import org.elasticsearch.common.lease.Releasable; import org.elasticsearch.common.lease.Releasables; import org.elasticsearch.common.lucene.uid.Versions; -import org.elasticsearch.common.unit.ByteSizeValue; import org.elasticsearch.common.util.BigArrays; -import org.elasticsearch.common.util.Callback; import org.elasticsearch.common.util.concurrent.ConcurrentCollections; import org.elasticsearch.common.util.concurrent.FutureUtils; import org.elasticsearch.common.util.concurrent.ReleasableLock; @@ -53,7 +50,6 @@ import java.io.EOFException; import java.io.IOException; import java.nio.ByteBuffer; import java.nio.channels.FileChannel; -import java.nio.file.DirectoryStream; import java.nio.file.Files; import java.nio.file.Path; import java.nio.file.StandardCopyOption; @@ -69,6 +65,8 @@ import java.util.concurrent.locks.ReadWriteLock; import java.util.concurrent.locks.ReentrantReadWriteLock; import java.util.regex.Matcher; import java.util.regex.Pattern; +import java.util.stream.Collectors; +import java.util.stream.Stream; /** * A Translog is a per index shard component that records all non-committed index operations in a durable manner. @@ -112,29 +110,25 @@ public class Translog extends AbstractIndexShardComponent implements IndexShardC static final Pattern PARSE_STRICT_ID_PATTERN = Pattern.compile("^" + TRANSLOG_FILE_PREFIX + "(\\d+)(\\.tlog)$"); - private final List recoveredTranslogs; + // the list of translog readers is guaranteed to be in order of translog generation + private final List readers = new ArrayList<>(); private volatile ScheduledFuture syncScheduler; // this is a concurrent set and is not protected by any of the locks. The main reason - // is that is being accessed by two separate classes (additions & reading are done by FsTranslog, remove by FsView when closed) + // is that is being accessed by two separate classes (additions & reading are done by Translog, remove by View when closed) private final Set outstandingViews = ConcurrentCollections.newConcurrentSet(); private BigArrays bigArrays; protected final ReleasableLock readLock; protected final ReleasableLock writeLock; private final Path location; private TranslogWriter current; - private volatile ImmutableTranslogReader currentCommittingTranslog; - private volatile long lastCommittedTranslogFileGeneration = -1; // -1 is safe as it will not cause an translog deletion. + + private final static long NOT_SET_GENERATION = -1; // -1 is safe as it will not cause a translog deletion. + + private volatile long currentCommittingGeneration = NOT_SET_GENERATION; + private volatile long lastCommittedTranslogFileGeneration = NOT_SET_GENERATION; private final AtomicBoolean closed = new AtomicBoolean(); private final TranslogConfig config; private final String translogUUID; - private Callback onViewClose = new Callback() { - @Override - public void handle(View view) { - logger.trace("closing view starting at translog [{}]", view.minTranslogGeneration()); - boolean removed = outstandingViews.remove(view); - assert removed : "View was never set but was supposed to be removed"; - } - }; /** @@ -176,11 +170,11 @@ public class Translog extends AbstractIndexShardComponent implements IndexShardC // if not we don't even try to clean it up and wait until we fail creating it assert Files.exists(nextTranslogFile) == false || Files.size(nextTranslogFile) <= TranslogWriter.getHeaderLength(translogUUID) : "unexpected translog file: [" + nextTranslogFile + "]"; if (Files.exists(currentCheckpointFile) // current checkpoint is already copied - && Files.deleteIfExists(nextTranslogFile)) { // delete it and log a warning + && Files.deleteIfExists(nextTranslogFile)) { // delete it and log a warning logger.warn("deleted previously created, but not yet committed, next generation [{}]. This can happen due to a tragic exception when creating a new generation", nextTranslogFile.getFileName()); } - this.recoveredTranslogs = recoverFromFiles(translogGeneration, checkpoint); - if (recoveredTranslogs.isEmpty()) { + this.readers.addAll(recoverFromFiles(translogGeneration, checkpoint)); + if (readers.isEmpty()) { throw new IllegalStateException("at least one reader must be recovered"); } boolean success = false; @@ -193,11 +187,10 @@ public class Translog extends AbstractIndexShardComponent implements IndexShardC // for instance if we have a lot of tlog and we can't create the writer we keep on holding // on to all the uncommitted tlog files if we don't close if (success == false) { - IOUtils.closeWhileHandlingException(recoveredTranslogs); + IOUtils.closeWhileHandlingException(readers); } } } else { - this.recoveredTranslogs = Collections.emptyList(); IOUtils.rm(location); logger.debug("wipe translog location - creating new translog"); Files.createDirectories(location); @@ -205,21 +198,22 @@ public class Translog extends AbstractIndexShardComponent implements IndexShardC Checkpoint checkpoint = new Checkpoint(0, 0, generation); Checkpoint.write(location.resolve(CHECKPOINT_FILE_NAME), checkpoint, StandardOpenOption.WRITE, StandardOpenOption.CREATE_NEW); current = createWriter(generation); - this.lastCommittedTranslogFileGeneration = -1; // playing safe + this.lastCommittedTranslogFileGeneration = NOT_SET_GENERATION; } // now that we know which files are there, create a new current one. } catch (Throwable t) { // close the opened translog files if we fail to create a new translog... - IOUtils.closeWhileHandlingException(currentCommittingTranslog, current); + IOUtils.closeWhileHandlingException(current); + IOUtils.closeWhileHandlingException(readers); throw t; } } /** recover all translog files found on disk */ - private final ArrayList recoverFromFiles(TranslogGeneration translogGeneration, Checkpoint checkpoint) throws IOException { + private final ArrayList recoverFromFiles(TranslogGeneration translogGeneration, Checkpoint checkpoint) throws IOException { boolean success = false; - ArrayList foundTranslogs = new ArrayList<>(); + ArrayList foundTranslogs = new ArrayList<>(); final Path tempFile = Files.createTempFile(location, TRANSLOG_FILE_PREFIX, TRANSLOG_FILE_SUFFIX); // a temp file to copy checkpoint to - note it must be in on the same FS otherwise atomic move won't work boolean tempFileRenamed = false; try (ReleasableLock lock = writeLock.acquire()) { @@ -230,7 +224,7 @@ public class Translog extends AbstractIndexShardComponent implements IndexShardC if (Files.exists(committedTranslogFile) == false) { throw new IllegalStateException("translog file doesn't exist with generation: " + i + " lastCommitted: " + lastCommittedTranslogFileGeneration + " checkpoint: " + checkpoint.generation + " - translog ids must be consecutive"); } - final ImmutableTranslogReader reader = openReader(committedTranslogFile, Checkpoint.read(location.resolve(getCommitCheckpointFileName(i)))); + final TranslogReader reader = openReader(committedTranslogFile, Checkpoint.read(location.resolve(getCommitCheckpointFileName(i)))); foundTranslogs.add(reader); logger.debug("recovered local translog from checkpoint {}", checkpoint); } @@ -267,17 +261,11 @@ public class Translog extends AbstractIndexShardComponent implements IndexShardC return foundTranslogs; } - ImmutableTranslogReader openReader(Path path, Checkpoint checkpoint) throws IOException { - final long generation; - try { - generation = parseIdFromFileName(path); - } catch (IllegalArgumentException ex) { - throw new TranslogException(shardId, "failed to parse generation from file name matching pattern " + path, ex); - } + TranslogReader openReader(Path path, Checkpoint checkpoint) throws IOException { FileChannel channel = FileChannel.open(path, StandardOpenOption.READ); try { - final ChannelReference raf = new ChannelReference(path, generation, channel, new OnCloseRunnable()); - ImmutableTranslogReader reader = ImmutableTranslogReader.open(raf, checkpoint, translogUUID); + assert Translog.parseIdFromFileName(path) == checkpoint.generation : "expected generation: " + Translog.parseIdFromFileName(path) + " but got: " + checkpoint.generation; + TranslogReader reader = TranslogReader.open(channel, path, checkpoint, translogUUID); channel = null; return reader; } finally { @@ -315,12 +303,7 @@ public class Translog extends AbstractIndexShardComponent implements IndexShardC try { current.sync(); } finally { - try { - IOUtils.close(current, currentCommittingTranslog); - } finally { - IOUtils.close(recoveredTranslogs); - recoveredTranslogs.clear(); - } + closeFilesIfNoPendingViews(); } } finally { FutureUtils.cancel(syncScheduler); @@ -349,41 +332,49 @@ public class Translog extends AbstractIndexShardComponent implements IndexShardC /** * Returns the number of operations in the transaction files that aren't committed to lucene.. - * Note: may return -1 if unknown */ public int totalOperations() { - int ops = 0; - try (ReleasableLock lock = readLock.acquire()) { - ops += current.totalOperations(); - if (currentCommittingTranslog != null) { - int tops = currentCommittingTranslog.totalOperations(); - assert tops != TranslogReader.UNKNOWN_OP_COUNT; - assert tops >= 0; - ops += tops; - } - } - return ops; + return totalOperations(lastCommittedTranslogFileGeneration); } /** * Returns the size in bytes of the translog files that aren't committed to lucene. */ public long sizeInBytes() { - long size = 0; - try (ReleasableLock lock = readLock.acquire()) { - size += current.sizeInBytes(); - if (currentCommittingTranslog != null) { - size += currentCommittingTranslog.sizeInBytes(); - } + return sizeInBytes(lastCommittedTranslogFileGeneration); + } + + /** + * Returns the number of operations in the transaction files that aren't committed to lucene.. + */ + private int totalOperations(long minGeneration) { + try (ReleasableLock ignored = readLock.acquire()) { + ensureOpen(); + return Stream.concat(readers.stream(), Stream.of(current)) + .filter(r -> r.getGeneration() >= minGeneration) + .mapToInt(BaseTranslogReader::totalOperations) + .sum(); + } + } + + /** + * Returns the size in bytes of the translog files that aren't committed to lucene. + */ + private long sizeInBytes(long minGeneration) { + try (ReleasableLock ignored = readLock.acquire()) { + ensureOpen(); + return Stream.concat(readers.stream(), Stream.of(current)) + .filter(r -> r.getGeneration() >= minGeneration) + .mapToLong(BaseTranslogReader::sizeInBytes) + .sum(); } - return size; } TranslogWriter createWriter(long fileGeneration) throws IOException { TranslogWriter newFile; try { - newFile = TranslogWriter.create(shardId, translogUUID, fileGeneration, location.resolve(getFilename(fileGeneration)), new OnCloseRunnable(), getChannelFactory(), config.getBufferSize()); + newFile = TranslogWriter.create(shardId, translogUUID, fileGeneration, location.resolve(getFilename(fileGeneration)), getChannelFactory(), config.getBufferSize()); } catch (IOException e) { throw new TranslogException(shardId, "failed to create new translog file", e); } @@ -398,12 +389,12 @@ public class Translog extends AbstractIndexShardComponent implements IndexShardC */ public Translog.Operation read(Location location) { try (ReleasableLock lock = readLock.acquire()) { - final TranslogReader reader; + final BaseTranslogReader reader; final long currentGeneration = current.getGeneration(); if (currentGeneration == location.generation) { reader = current; - } else if (currentCommittingTranslog != null && currentCommittingTranslog.getGeneration() == location.generation) { - reader = currentCommittingTranslog; + } else if (readers.isEmpty() == false && readers.get(readers.size() - 1).getGeneration() == location.generation) { + reader = readers.get(readers.size() - 1); } else if (currentGeneration < location.generation) { throw new IllegalStateException("location generation [" + location.generation + "] is greater than the current generation [" + currentGeneration + "]"); } else { @@ -467,33 +458,16 @@ public class Translog extends AbstractIndexShardComponent implements IndexShardC * Snapshots are fixed in time and will not be updated with future operations. */ public Snapshot newSnapshot() { - ensureOpen(); - try (ReleasableLock lock = readLock.acquire()) { - ArrayList toOpen = new ArrayList<>(); - toOpen.addAll(recoveredTranslogs); - if (currentCommittingTranslog != null) { - toOpen.add(currentCommittingTranslog); - } - toOpen.add(current); - return createSnapshot(toOpen.toArray(new TranslogReader[toOpen.size()])); - } + return createSnapshot(Long.MIN_VALUE); } - private static Snapshot createSnapshot(TranslogReader... translogs) { - Snapshot[] snapshots = new Snapshot[translogs.length]; - boolean success = false; - try { - for (int i = 0; i < translogs.length; i++) { - snapshots[i] = translogs[i].newSnapshot(); - } - - Snapshot snapshot = new MultiSnapshot(snapshots); - success = true; - return snapshot; - } finally { - if (success == false) { - Releasables.close(snapshots); - } + private Snapshot createSnapshot(long minGeneration) { + try (ReleasableLock ignored = readLock.acquire()) { + ensureOpen(); + Snapshot[] snapshots = Stream.concat(readers.stream(), Stream.of(current)) + .filter(reader -> reader.getGeneration() >= minGeneration) + .map(BaseTranslogReader::newSnapshot).toArray(Snapshot[]::new); + return new MultiSnapshot(snapshots); } } @@ -502,25 +476,11 @@ public class Translog extends AbstractIndexShardComponent implements IndexShardC * while receiving future ones as well */ public Translog.View newView() { - // we need to acquire the read lock to make sure no new translog is created - // and will be missed by the view we're making try (ReleasableLock lock = readLock.acquire()) { - ArrayList translogs = new ArrayList<>(); - try { - if (currentCommittingTranslog != null) { - translogs.add(currentCommittingTranslog.clone()); - } - translogs.add(current.newReaderFromWriter()); - View view = new View(translogs, onViewClose); - // this is safe as we know that no new translog is being made at the moment - // (we hold a read lock) and the view will be notified of any future one - outstandingViews.add(view); - translogs.clear(); - return view; - } finally { - // close if anything happend and we didn't reach the clear - IOUtils.closeWhileHandlingException(translogs); - } + ensureOpen(); + View view = new View(lastCommittedTranslogFileGeneration); + outstandingViews.add(view); + return view; } } @@ -561,7 +521,7 @@ public class Translog extends AbstractIndexShardComponent implements IndexShardC */ public boolean ensureSynced(Location location) throws IOException { try (ReleasableLock lock = readLock.acquire()) { - if (location.generation == current.generation) { // if we have a new one it's already synced + if (location.generation == current.getGeneration()) { // if we have a new one it's already synced ensureOpen(); return current.syncUpTo(location.translogLocation + location.size); } @@ -604,151 +564,67 @@ public class Translog extends AbstractIndexShardComponent implements IndexShardC return config; } - - private final class OnCloseRunnable implements Callback { - @Override - public void handle(ChannelReference channelReference) { - if (isReferencedGeneration(channelReference.getGeneration()) == false) { - Path translogPath = channelReference.getPath(); - assert channelReference.getPath().getParent().equals(location) : "translog files must be in the location folder: " + location + " but was: " + translogPath; - // if the given translogPath is not the current we can safely delete the file since all references are released - logger.trace("delete translog file - not referenced and not current anymore {}", translogPath); - IOUtils.deleteFilesIgnoringExceptions(translogPath); - IOUtils.deleteFilesIgnoringExceptions(translogPath.resolveSibling(getCommitCheckpointFileName(channelReference.getGeneration()))); - - } - try (DirectoryStream stream = Files.newDirectoryStream(location)) { - for (Path path : stream) { - Matcher matcher = PARSE_STRICT_ID_PATTERN.matcher(path.getFileName().toString()); - if (matcher.matches()) { - long generation = Long.parseLong(matcher.group(1)); - if (isReferencedGeneration(generation) == false) { - logger.trace("delete translog file - not referenced and not current anymore {}", path); - IOUtils.deleteFilesIgnoringExceptions(path); - IOUtils.deleteFilesIgnoringExceptions(path.resolveSibling(getCommitCheckpointFileName(channelReference.getGeneration()))); - } - } - } - } catch (IOException e) { - logger.warn("failed to delete unreferenced translog files", e); - } - } - } - /** * a view into the translog, capturing all translog file at the moment of creation * and updated with any future translog. */ - public static final class View implements Closeable { - public static final Translog.View EMPTY_VIEW = new View(Collections.emptyList(), null); + /** + * a view into the translog, capturing all translog file at the moment of creation + * and updated with any future translog. + */ + public class View implements Closeable { - boolean closed; - // last in this list is always FsTranslog.current - final List orderedTranslogs; - private final Callback onClose; + AtomicBoolean closed = new AtomicBoolean(); + final long minGeneration; - View(List orderedTranslogs, Callback onClose) { - // clone so we can safely mutate.. - this.orderedTranslogs = new ArrayList<>(orderedTranslogs); - this.onClose = onClose; - } - - /** - * Called by the parent class when ever the current translog changes - * - * @param oldCurrent a new read only reader for the old current (should replace the previous reference) - * @param newCurrent a reader into the new current. - */ - synchronized void onNewTranslog(TranslogReader oldCurrent, TranslogReader newCurrent) throws IOException { - // even though the close method removes this view from outstandingViews, there is no synchronisation in place - // between that operation and an ongoing addition of a new translog, already having an iterator. - // As such, this method can be called despite of the fact that we are closed. We need to check and ignore. - if (closed) { - // we have to close the new references created for as as we will not hold them - IOUtils.close(oldCurrent, newCurrent); - return; - } - orderedTranslogs.remove(orderedTranslogs.size() - 1).close(); - orderedTranslogs.add(oldCurrent); - orderedTranslogs.add(newCurrent); + View(long minGeneration) { + this.minGeneration = minGeneration; } /** this smallest translog generation in this view */ - public synchronized long minTranslogGeneration() { - ensureOpen(); - return orderedTranslogs.get(0).getGeneration(); + public long minTranslogGeneration() { + return minGeneration; } /** * The total number of operations in the view. */ - public synchronized int totalOperations() { - int ops = 0; - for (TranslogReader translog : orderedTranslogs) { - int tops = translog.totalOperations(); - if (tops == TranslogReader.UNKNOWN_OP_COUNT) { - return -1; - } - assert tops >= 0; - ops += tops; - } - return ops; + public int totalOperations() { + return Translog.this.totalOperations(minGeneration); } /** * Returns the size in bytes of the files behind the view. */ - public synchronized long sizeInBytes() { - long size = 0; - for (TranslogReader translog : orderedTranslogs) { - size += translog.sizeInBytes(); - } - return size; + public long sizeInBytes() { + return Translog.this.sizeInBytes(minGeneration); } /** create a snapshot from this view */ - public synchronized Snapshot snapshot() { + public Snapshot snapshot() { ensureOpen(); - return createSnapshot(orderedTranslogs.toArray(new TranslogReader[orderedTranslogs.size()])); + return Translog.this.createSnapshot(minGeneration); } - void ensureOpen() { - if (closed) { - throw new ElasticsearchException("View is already closed"); + if (closed.get()) { + throw new AlreadyClosedException("View is already closed"); } } @Override - public void close() { - final List toClose = new ArrayList<>(); - try { - synchronized (this) { - if (closed == false) { - try { - if (onClose != null) { - onClose.handle(this); - } - } finally { - closed = true; - toClose.addAll(orderedTranslogs); - orderedTranslogs.clear(); - } - } - } - } finally { - try { - // Close out of lock to prevent deadlocks between channel close which checks for - // references in InternalChannelReference.closeInternal (waiting on a read lock) - // and other FsTranslog#newTranslog calling FsView.onNewTranslog (while having a write lock) - IOUtils.close(toClose); - } catch (Exception e) { - throw new ElasticsearchException("failed to close view", e); - } + public void close() throws IOException { + if (closed.getAndSet(true) == false) { + logger.trace("closing view starting at translog [{}]", minTranslogGeneration()); + boolean removed = outstandingViews.remove(this); + assert removed : "View was never set but was supposed to be removed"; + trimUnreferencedReaders(); + closeFilesIfNoPendingViews(); } } } + public static class Location implements Accountable, Comparable { public final long generation; @@ -817,12 +693,12 @@ public class Translog extends AbstractIndexShardComponent implements IndexShardC /** * A snapshot of the transaction log, allows to iterate over all the transaction log operations. */ - public interface Snapshot extends Releasable { + public interface Snapshot { /** * The total number of operations in the translog. */ - int estimatedTotalOperations(); + int totalOperations(); /** * Returns the next operation in the snapshot or null if we reached the end. @@ -1320,13 +1196,12 @@ public class Translog extends AbstractIndexShardComponent implements IndexShardC public void prepareCommit() throws IOException { try (ReleasableLock lock = writeLock.acquire()) { ensureOpen(); - if (currentCommittingTranslog != null) { - throw new IllegalStateException("already committing a translog with generation: " + currentCommittingTranslog.getGeneration()); + if (currentCommittingGeneration != NOT_SET_GENERATION) { + throw new IllegalStateException("already committing a translog with generation: " + currentCommittingGeneration); } - final TranslogWriter oldCurrent = current; - oldCurrent.ensureOpen(); - oldCurrent.sync(); - currentCommittingTranslog = current.immutableReader(); + currentCommittingGeneration = current.getGeneration(); + TranslogReader currentCommittingTranslog = current.closeIntoReader(); + readers.add(currentCommittingTranslog); Path checkpoint = location.resolve(CHECKPOINT_FILE_NAME); assert Checkpoint.read(checkpoint).generation == currentCommittingTranslog.getGeneration(); Path commitCheckpoint = location.resolve(getCommitCheckpointFileName(currentCommittingTranslog.getGeneration())); @@ -1335,14 +1210,7 @@ public class Translog extends AbstractIndexShardComponent implements IndexShardC IOUtils.fsync(commitCheckpoint.getParent(), true); // create a new translog file - this will sync it and update the checkpoint data; current = createWriter(current.getGeneration() + 1); - // notify all outstanding views of the new translog (no views are created now as - // we hold a write lock). - for (View view : outstandingViews) { - view.onNewTranslog(currentCommittingTranslog.clone(), current.newReaderFromWriter()); - } - IOUtils.close(oldCurrent); logger.trace("current translog set to [{}]", current.getGeneration()); - assert oldCurrent.syncNeeded() == false : "old translog oldCurrent must not need a sync"; } catch (Throwable t) { IOUtils.closeWhileHandlingException(this); // tragic event @@ -1352,24 +1220,53 @@ public class Translog extends AbstractIndexShardComponent implements IndexShardC @Override public void commit() throws IOException { - ImmutableTranslogReader toClose = null; try (ReleasableLock lock = writeLock.acquire()) { ensureOpen(); - if (currentCommittingTranslog == null) { + if (currentCommittingGeneration == NOT_SET_GENERATION) { prepareCommit(); } + assert currentCommittingGeneration != NOT_SET_GENERATION; + assert readers.stream().filter(r -> r.getGeneration() == currentCommittingGeneration).findFirst().isPresent() + : "reader list doesn't contain committing generation [" + currentCommittingGeneration + "]"; lastCommittedTranslogFileGeneration = current.getGeneration(); // this is important - otherwise old files will not be cleaned up - if (recoveredTranslogs.isEmpty() == false) { - IOUtils.close(recoveredTranslogs); - recoveredTranslogs.clear(); - } - toClose = this.currentCommittingTranslog; - this.currentCommittingTranslog = null; - } finally { - IOUtils.close(toClose); + currentCommittingGeneration = NOT_SET_GENERATION; + trimUnreferencedReaders(); } } + void trimUnreferencedReaders() { + try (ReleasableLock ignored = writeLock.acquire()) { + if (closed.get()) { + // we're shutdown potentially on some tragic event - don't delete anything + return; + } + long minReferencedGen = outstandingViews.stream().mapToLong(View::minTranslogGeneration).min().orElse(Long.MAX_VALUE); + minReferencedGen = Math.min(lastCommittedTranslogFileGeneration, minReferencedGen); + final long finalMinReferencedGen = minReferencedGen; + List unreferenced = readers.stream().filter(r -> r.getGeneration() < finalMinReferencedGen).collect(Collectors.toList()); + for (final TranslogReader unreferencedReader : unreferenced) { + Path translogPath = unreferencedReader.path(); + logger.trace("delete translog file - not referenced and not current anymore {}", translogPath); + IOUtils.closeWhileHandlingException(unreferencedReader); + IOUtils.deleteFilesIgnoringExceptions(translogPath, + translogPath.resolveSibling(getCommitCheckpointFileName(unreferencedReader.getGeneration()))); + } + readers.removeAll(unreferenced); + } + } + + void closeFilesIfNoPendingViews() throws IOException { + try (ReleasableLock ignored = writeLock.acquire()) { + if (closed.get() && outstandingViews.isEmpty()) { + logger.trace("closing files. translog is closed and there are no pending views"); + ArrayList toClose = new ArrayList<>(readers); + toClose.add(current); + IOUtils.close(toClose); + } + } + } + + @Override public void rollback() throws IOException { ensureOpen(); @@ -1435,9 +1332,11 @@ public class Translog extends AbstractIndexShardComponent implements IndexShardC return TranslogWriter.ChannelFactory.DEFAULT; } - /** If this {@code Translog} was closed as a side-effect of a tragic exception, - * e.g. disk full while flushing a new segment, this returns the root cause exception. - * Otherwise (no tragic exception has occurred) it returns null. */ + /** + * If this {@code Translog} was closed as a side-effect of a tragic exception, + * e.g. disk full while flushing a new segment, this returns the root cause exception. + * Otherwise (no tragic exception has occurred) it returns null. + */ public Throwable getTragicException() { return current.getTragicException(); } diff --git a/core/src/main/java/org/elasticsearch/index/translog/TranslogReader.java b/core/src/main/java/org/elasticsearch/index/translog/TranslogReader.java index 71dff6ec36e8..ecc3822361c2 100644 --- a/core/src/main/java/org/elasticsearch/index/translog/TranslogReader.java +++ b/core/src/main/java/org/elasticsearch/index/translog/TranslogReader.java @@ -27,161 +27,46 @@ import org.apache.lucene.store.AlreadyClosedException; import org.apache.lucene.store.InputStreamDataInput; import org.apache.lucene.util.BytesRef; import org.apache.lucene.util.RamUsageEstimator; -import org.elasticsearch.ElasticsearchException; -import org.elasticsearch.common.io.stream.ByteBufferStreamInput; +import org.elasticsearch.common.io.Channels; import org.elasticsearch.common.io.stream.InputStreamStreamInput; import java.io.Closeable; +import java.io.EOFException; import java.io.IOException; import java.nio.ByteBuffer; -import java.nio.channels.Channels; import java.nio.channels.FileChannel; -import java.nio.file.Files; import java.nio.file.Path; import java.util.concurrent.atomic.AtomicBoolean; /** - * A base class for all classes that allows reading ops from translog files + * an immutable translog filereader */ -public abstract class TranslogReader implements Closeable, Comparable { - public static final int UNKNOWN_OP_COUNT = -1; +public class TranslogReader extends BaseTranslogReader implements Closeable { private static final byte LUCENE_CODEC_HEADER_BYTE = 0x3f; private static final byte UNVERSIONED_TRANSLOG_HEADER_BYTE = 0x00; - protected final long generation; - protected final ChannelReference channelReference; - protected final FileChannel channel; + + private final int totalOperations; + protected final long length; protected final AtomicBoolean closed = new AtomicBoolean(false); - protected final long firstOperationOffset; - public TranslogReader(long generation, ChannelReference channelReference, long firstOperationOffset) { - this.generation = generation; - this.channelReference = channelReference; - this.channel = channelReference.getChannel(); - this.firstOperationOffset = firstOperationOffset; - } - - public long getGeneration() { - return this.generation; - } - - public abstract long sizeInBytes(); - - abstract public int totalOperations(); - - public final long getFirstOperationOffset() { - return firstOperationOffset; - } - - public Translog.Operation read(Translog.Location location) throws IOException { - assert location.generation == generation : "read location's translog generation [" + location.generation + "] is not [" + generation + "]"; - ByteBuffer buffer = ByteBuffer.allocate(location.size); - try (BufferedChecksumStreamInput checksumStreamInput = checksummedStream(buffer, location.translogLocation, location.size, null)) { - return read(checksumStreamInput); - } - } - - /** read the size of the op (i.e., number of bytes, including the op size) written at the given position */ - private final int readSize(ByteBuffer reusableBuffer, long position) { - // read op size from disk - assert reusableBuffer.capacity() >= 4 : "reusable buffer must have capacity >=4 when reading opSize. got [" + reusableBuffer.capacity() + "]"; - try { - reusableBuffer.clear(); - reusableBuffer.limit(4); - readBytes(reusableBuffer, position); - reusableBuffer.flip(); - // Add an extra 4 to account for the operation size integer itself - final int size = reusableBuffer.getInt() + 4; - final long maxSize = sizeInBytes() - position; - if (size < 0 || size > maxSize) { - throw new TranslogCorruptedException("operation size is corrupted must be [0.." + maxSize + "] but was: " + size); - } - - return size; - } catch (IOException e) { - throw new ElasticsearchException("unexpected exception reading from translog snapshot of " + this.channelReference.getPath(), e); - } - } - - public Translog.Snapshot newSnapshot() { - final ByteBuffer reusableBuffer = ByteBuffer.allocate(1024); - final int totalOperations = totalOperations(); - channelReference.incRef(); - return newReaderSnapshot(totalOperations, reusableBuffer); + /** + * Create a reader of translog file channel. The length parameter should be consistent with totalOperations and point + * at the end of the last operation in this snapshot. + */ + public TranslogReader(long generation, FileChannel channel, Path path, long firstOperationOffset, long length, int totalOperations) { + super(generation, channel, path, firstOperationOffset); + this.length = length; + this.totalOperations = totalOperations; } /** - * reads an operation at the given position and returns it. The buffer length is equal to the number - * of bytes reads. + * Given a file, opens an {@link TranslogReader}, taking of checking and validating the file header. */ - private final BufferedChecksumStreamInput checksummedStream(ByteBuffer reusableBuffer, long position, int opSize, BufferedChecksumStreamInput reuse) throws IOException { - final ByteBuffer buffer; - if (reusableBuffer.capacity() >= opSize) { - buffer = reusableBuffer; - } else { - buffer = ByteBuffer.allocate(opSize); - } - buffer.clear(); - buffer.limit(opSize); - readBytes(buffer, position); - buffer.flip(); - return new BufferedChecksumStreamInput(new ByteBufferStreamInput(buffer), reuse); - } - - protected Translog.Operation read(BufferedChecksumStreamInput inStream) throws IOException { - return Translog.readOperation(inStream); - } - - /** - * reads bytes at position into the given buffer, filling it. - */ - abstract protected void readBytes(ByteBuffer buffer, long position) throws IOException; - - @Override - public final void close() throws IOException { - if (closed.compareAndSet(false, true)) { - channelReference.decRef(); - } - } - - protected final boolean isClosed() { - return closed.get(); - } - - protected void ensureOpen() { - if (isClosed()) { - throw new AlreadyClosedException("translog [" + getGeneration() + "] is already closed"); - } - } - - @Override - public String toString() { - return "translog [" + generation + "][" + channelReference.getPath() + "]"; - } - - @Override - public int compareTo(TranslogReader o) { - return Long.compare(getGeneration(), o.getGeneration()); - } - - - /** - * Given a file, return a VersionedTranslogStream based on an - * optionally-existing header in the file. If the file does not exist, or - * has zero length, returns the latest version. If the header does not - * exist, assumes Version 0 of the translog file format. - */ - public static ImmutableTranslogReader open(ChannelReference channelReference, Checkpoint checkpoint, String translogUUID) throws IOException { - final FileChannel channel = channelReference.getChannel(); - final Path path = channelReference.getPath(); - assert channelReference.getGeneration() == checkpoint.generation : "expected generation: " + channelReference.getGeneration() + " but got: " + checkpoint.generation; + public static TranslogReader open(FileChannel channel, Path path, Checkpoint checkpoint, String translogUUID) throws IOException { try { - if (checkpoint.offset == 0 && checkpoint.numOps == TranslogReader.UNKNOWN_OP_COUNT) { // only old files can be empty - return new LegacyTranslogReader(channelReference.getGeneration(), channelReference, 0); - } - - InputStreamStreamInput headerStream = new InputStreamStreamInput(Channels.newInputStream(channel)); // don't close + InputStreamStreamInput headerStream = new InputStreamStreamInput(java.nio.channels.Channels.newInputStream(channel)); // don't close // Lucene's CodecUtil writes a magic number of 0x3FD76C17 with the // header, in binary this looks like: // @@ -208,20 +93,17 @@ public abstract class TranslogReader implements Closeable, Comparable TranslogReader.UNKNOWN_OP_COUNT: "expected at least 0 operatin but got: " + checkpoint.numOps; + assert checkpoint.numOps >= 0 : "expected at least 0 operatin but got: " + checkpoint.numOps; assert checkpoint.offset <= channel.size() : "checkpoint is inconsistent with channel length: " + channel.size() + " " + checkpoint; int len = headerStream.readInt(); if (len > channel.size()) { @@ -232,78 +114,61 @@ public abstract class TranslogReader implements Closeable, Comparable= length) { + throw new EOFException("read requested past EOF. pos [" + position + "] end: [" + length + "]"); } - - @Override - public final int estimatedTotalOperations() { - return totalOperations; + if (position < firstOperationOffset) { + throw new IOException("read requested before position of first ops. pos [" + position + "] first op on: [" + firstOperationOffset + "]"); } + Channels.readFromFileChannelWithEofException(channel, position, buffer); + } - @Override - public Translog.Operation next() throws IOException { - if (readOperations < totalOperations) { - assert readOperations < totalOperations : "readOpeartions must be less than totalOperations"; - return readOperation(); - } else { - return null; - } + public Checkpoint getInfo() { + return new Checkpoint(length, totalOperations, getGeneration()); + } + + @Override + public final void close() throws IOException { + if (closed.compareAndSet(false, true)) { + channel.close(); } + } - protected final Translog.Operation readOperation() throws IOException { - final int opSize = readSize(reusableBuffer, position); - reuse = checksummedStream(reusableBuffer, position, opSize, reuse); - Translog.Operation op = read(reuse); - position += opSize; - readOperations++; - return op; - } + protected final boolean isClosed() { + return closed.get(); + } - @Override - public void close() { - if (closed.compareAndSet(false, true)) { - channelReference.decRef(); - } + protected void ensureOpen() { + if (isClosed()) { + throw new AlreadyClosedException(toString() + " is already closed"); } } } diff --git a/core/src/main/java/org/elasticsearch/index/translog/ImmutableTranslogReader.java b/core/src/main/java/org/elasticsearch/index/translog/TranslogSnapshot.java similarity index 50% rename from core/src/main/java/org/elasticsearch/index/translog/ImmutableTranslogReader.java rename to core/src/main/java/org/elasticsearch/index/translog/TranslogSnapshot.java index 1d6d3b45a634..10f381f8eba9 100644 --- a/core/src/main/java/org/elasticsearch/index/translog/ImmutableTranslogReader.java +++ b/core/src/main/java/org/elasticsearch/index/translog/TranslogSnapshot.java @@ -16,7 +16,6 @@ * specific language governing permissions and limitations * under the License. */ - package org.elasticsearch.index.translog; import org.elasticsearch.common.io.Channels; @@ -24,68 +23,82 @@ import org.elasticsearch.common.io.Channels; import java.io.EOFException; import java.io.IOException; import java.nio.ByteBuffer; +import java.nio.channels.FileChannel; +import java.nio.file.Path; -/** - * a translog reader which is fixed in length - */ -public class ImmutableTranslogReader extends TranslogReader { - +public class TranslogSnapshot extends BaseTranslogReader implements Translog.Snapshot { private final int totalOperations; protected final long length; + private final ByteBuffer reusableBuffer; + private long position; + private int readOperations; + private BufferedChecksumStreamInput reuse; + + /** * Create a snapshot of translog file channel. The length parameter should be consistent with totalOperations and point * at the end of the last operation in this snapshot. */ - public ImmutableTranslogReader(long generation, ChannelReference channelReference, long firstOperationOffset, long length, int totalOperations) { - super(generation, channelReference, firstOperationOffset); + public TranslogSnapshot(long generation, FileChannel channel, Path path, long firstOperationOffset, long length, int totalOperations) { + super(generation, channel, path, firstOperationOffset); this.length = length; this.totalOperations = totalOperations; + this.reusableBuffer = ByteBuffer.allocate(1024); + readOperations = 0; + position = firstOperationOffset; + reuse = null; } @Override - public final TranslogReader clone() { - if (channelReference.tryIncRef()) { - try { - ImmutableTranslogReader reader = newReader(generation, channelReference, firstOperationOffset, length, totalOperations); - channelReference.incRef(); // for the new object - return reader; - } finally { - channelReference.decRef(); - } + public final int totalOperations() { + return totalOperations; + } + + @Override + public Translog.Operation next() throws IOException { + if (readOperations < totalOperations) { + return readOperation(); } else { - throw new IllegalStateException("can't increment translog [" + generation + "] channel ref count"); + return null; } } - - protected ImmutableTranslogReader newReader(long generation, ChannelReference channelReference, long offset, long length, int totalOperations) { - return new ImmutableTranslogReader(generation, channelReference, offset, length, totalOperations); + protected final Translog.Operation readOperation() throws IOException { + final int opSize = readSize(reusableBuffer, position); + reuse = checksummedStream(reusableBuffer, position, opSize, reuse); + Translog.Operation op = read(reuse); + position += opSize; + readOperations++; + return op; } + public long sizeInBytes() { return length; } - public int totalOperations() { - return totalOperations; - } - /** * reads an operation at the given position into the given buffer. */ protected void readBytes(ByteBuffer buffer, long position) throws IOException { if (position >= length) { - throw new EOFException("read requested past EOF. pos [" + position + "] end: [" + length + "]"); + throw new EOFException("read requested past EOF. pos [" + position + "] end: [" + length + "], generation: [" + getGeneration() + "], path: [" + path + "]"); } - if (position < firstOperationOffset) { - throw new IOException("read requested before position of first ops. pos [" + position + "] first op on: [" + firstOperationOffset + "]"); + if (position < getFirstOperationOffset()) { + throw new IOException("read requested before position of first ops. pos [" + position + "] first op on: [" + getFirstOperationOffset() + "], generation: [" + getGeneration() + "], path: [" + path + "]"); } Channels.readFromFileChannelWithEofException(channel, position, buffer); } - public Checkpoint getInfo() { - return new Checkpoint(length, totalOperations, getGeneration()); + @Override + public String toString() { + return "TranslogSnapshot{" + + "readOperations=" + readOperations + + ", position=" + position + + ", totalOperations=" + totalOperations + + ", length=" + length + + ", reusableBuffer=" + reusableBuffer + + '}'; } - -} +} \ No newline at end of file diff --git a/core/src/main/java/org/elasticsearch/index/translog/TranslogWriter.java b/core/src/main/java/org/elasticsearch/index/translog/TranslogWriter.java index 517e4a5b30ed..f7d0cd571e8d 100644 --- a/core/src/main/java/org/elasticsearch/index/translog/TranslogWriter.java +++ b/core/src/main/java/org/elasticsearch/index/translog/TranslogWriter.java @@ -25,24 +25,23 @@ import org.apache.lucene.store.OutputStreamDataOutput; import org.apache.lucene.util.BytesRef; import org.apache.lucene.util.IOUtils; import org.apache.lucene.util.RamUsageEstimator; -import org.elasticsearch.common.bytes.BytesArray; import org.elasticsearch.common.bytes.BytesReference; import org.elasticsearch.common.io.Channels; import org.elasticsearch.common.unit.ByteSizeValue; -import org.elasticsearch.common.util.Callback; import org.elasticsearch.index.shard.ShardId; import java.io.BufferedOutputStream; +import java.io.Closeable; import java.io.IOException; import java.io.OutputStream; import java.nio.ByteBuffer; import java.nio.channels.FileChannel; -import java.nio.file.Files; import java.nio.file.OpenOption; import java.nio.file.Path; import java.nio.file.StandardOpenOption; +import java.util.concurrent.atomic.AtomicBoolean; -public class TranslogWriter extends TranslogReader { +public class TranslogWriter extends BaseTranslogReader implements Closeable { public static final String TRANSLOG_CODEC = "translog"; public static final int VERSION_CHECKSUMS = 1; @@ -61,11 +60,14 @@ public class TranslogWriter extends TranslogReader { /* the total offset of this file including the bytes written to the file as well as into the buffer */ private volatile long totalOffset; - public TranslogWriter(ShardId shardId, long generation, ChannelReference channelReference, ByteSizeValue bufferSize) throws IOException { - super(generation, channelReference, channelReference.getChannel().position()); + protected final AtomicBoolean closed = new AtomicBoolean(false); + + + public TranslogWriter(ShardId shardId, long generation, FileChannel channel, Path path, ByteSizeValue bufferSize) throws IOException { + super(generation, channel, path, channel.position()); this.shardId = shardId; - this.outputStream = new BufferedChannelOutputStream(java.nio.channels.Channels.newOutputStream(channelReference.getChannel()), bufferSize.bytesAsInt()); - this.lastSyncedOffset = channelReference.getChannel().position(); + this.outputStream = new BufferedChannelOutputStream(java.nio.channels.Channels.newOutputStream(channel), bufferSize.bytesAsInt()); + this.lastSyncedOffset = channel.position(); totalOffset = lastSyncedOffset; } @@ -74,10 +76,10 @@ public class TranslogWriter extends TranslogReader { } private static int getHeaderLength(int uuidLength) { - return CodecUtil.headerLength(TRANSLOG_CODEC) + uuidLength + RamUsageEstimator.NUM_BYTES_INT; + return CodecUtil.headerLength(TRANSLOG_CODEC) + uuidLength + RamUsageEstimator.NUM_BYTES_INT; } - public static TranslogWriter create(ShardId shardId, String translogUUID, long fileGeneration, Path file, Callback onClose, ChannelFactory channelFactory, ByteSizeValue bufferSize) throws IOException { + public static TranslogWriter create(ShardId shardId, String translogUUID, long fileGeneration, Path file, ChannelFactory channelFactory, ByteSizeValue bufferSize) throws IOException { final BytesRef ref = new BytesRef(translogUUID); final int headerLength = getHeaderLength(ref.length); final FileChannel channel = channelFactory.open(file); @@ -90,7 +92,7 @@ public class TranslogWriter extends TranslogReader { out.writeBytes(ref.bytes, ref.offset, ref.length); channel.force(true); writeCheckpoint(headerLength, 0, file.getParent(), fileGeneration, StandardOpenOption.WRITE); - final TranslogWriter writer = new TranslogWriter(shardId, fileGeneration, new ChannelReference(file, fileGeneration, channel, onClose), bufferSize); + final TranslogWriter writer = new TranslogWriter(shardId, fileGeneration, channel, file, bufferSize); return writer; } catch (Throwable throwable) { // if we fail to bake the file-generation into the checkpoint we stick with the file and once we recover and that @@ -99,9 +101,12 @@ public class TranslogWriter extends TranslogReader { throw throwable; } } - /** If this {@code TranslogWriter} was closed as a side-effect of a tragic exception, - * e.g. disk full while flushing a new segment, this returns the root cause exception. - * Otherwise (no tragic exception has occurred) it returns null. */ + + /** + * If this {@code TranslogWriter} was closed as a side-effect of a tragic exception, + * e.g. disk full while flushing a new segment, this returns the root cause exception. + * Otherwise (no tragic exception has occurred) it returns null. + */ public Throwable getTragicException() { return tragedy; } @@ -110,7 +115,9 @@ public class TranslogWriter extends TranslogReader { assert throwable != null : "throwable must not be null in a tragic event"; if (tragedy == null) { tragedy = throwable; - } else { + } else if (tragedy != throwable) { + // it should be safe to call closeWithTragicEvents on multiple layers without + // worrying about self suppression. tragedy.addSuppressed(throwable); } close(); @@ -134,29 +141,27 @@ public class TranslogWriter extends TranslogReader { } /** - * write all buffered ops to disk and fsync file + * write all buffered ops to disk and fsync file. + * + * Note: any exception during the sync process will be interpreted as a tragic exception and the writer will be closed before + * raising the exception. */ public void sync() throws IOException { if (syncNeeded()) { synchronized (this) { - ensureOpen(); // this call gives a better exception that the incRef if we are closed by a tragic event - channelReference.incRef(); + ensureOpen(); + final long offsetToSync; + final int opsCounter; try { - final long offsetToSync; - final int opsCounter; outputStream.flush(); offsetToSync = totalOffset; opsCounter = operationCounter; - try { - checkpoint(offsetToSync, opsCounter, channelReference); - } catch (Throwable ex) { - closeWithTragicEvent(ex); - throw ex; - } - lastSyncedOffset = offsetToSync; - } finally { - channelReference.decRef(); + checkpoint(offsetToSync, opsCounter, generation, channel, path); + } catch (Throwable ex) { + closeWithTragicEvent(ex); + throw ex; } + lastSyncedOffset = offsetToSync; } } } @@ -177,76 +182,36 @@ public class TranslogWriter extends TranslogReader { } /** - * returns a new reader that follows the current writes (most importantly allows making - * repeated snapshots that includes new content) + * closes this writer and transfers it's underlying file channel to a new immutable reader */ - public TranslogReader newReaderFromWriter() { - ensureOpen(); - channelReference.incRef(); - boolean success = false; + public synchronized TranslogReader closeIntoReader() throws IOException { try { - final TranslogReader reader = new InnerReader(this.generation, firstOperationOffset, channelReference); - success = true; - return reader; - } finally { - if (!success) { - channelReference.decRef(); - } + sync(); // sync before we close.. + } catch (IOException e) { + closeWithTragicEvent(e); + throw e; } - } - - /** - * returns a new immutable reader which only exposes the current written operation * - */ - public ImmutableTranslogReader immutableReader() throws TranslogException { - if (channelReference.tryIncRef()) { - synchronized (this) { - try { - ensureOpen(); - outputStream.flush(); - ImmutableTranslogReader reader = new ImmutableTranslogReader(this.generation, channelReference, firstOperationOffset, getWrittenOffset(), operationCounter); - channelReference.incRef(); // for new reader - return reader; - } catch (Exception e) { - throw new TranslogException(shardId, "exception while creating an immutable reader", e); - } finally { - channelReference.decRef(); - } - } + if (closed.compareAndSet(false, true)) { + return new TranslogReader(generation, channel, path, firstOperationOffset, getWrittenOffset(), operationCounter); } else { - throw new TranslogException(shardId, "can't increment channel [" + channelReference + "] ref count"); + throw new AlreadyClosedException("translog [" + getGeneration() + "] is already closed (path [" + path + "]", tragedy); } } + @Override + public synchronized Translog.Snapshot newSnapshot() { + ensureOpen(); + try { + sync(); + } catch (IOException e) { + throw new TranslogException(shardId, "exception while syncing before creating a snapshot", e); + } + return super.newSnapshot(); + } + private long getWrittenOffset() throws IOException { - return channelReference.getChannel().position(); - } - - /** - * this class is used when one wants a reference to this file which exposes all recently written operation. - * as such it needs access to the internals of the current reader - */ - final class InnerReader extends TranslogReader { - - public InnerReader(long generation, long fistOperationOffset, ChannelReference channelReference) { - super(generation, channelReference, fistOperationOffset); - } - - @Override - public long sizeInBytes() { - return TranslogWriter.this.sizeInBytes(); - } - - @Override - public int totalOperations() { - return TranslogWriter.this.totalOperations(); - } - - @Override - protected void readBytes(ByteBuffer buffer, long position) throws IOException { - TranslogWriter.this.readBytes(buffer, position); - } + return channel.position(); } /** @@ -264,13 +229,13 @@ public class TranslogWriter extends TranslogReader { @Override protected void readBytes(ByteBuffer targetBuffer, long position) throws IOException { - if (position+targetBuffer.remaining() > getWrittenOffset()) { + if (position + targetBuffer.remaining() > getWrittenOffset()) { synchronized (this) { // we only flush here if it's really really needed - try to minimize the impact of the read operation // in some cases ie. a tragic event we might still be able to read the relevant value // which is not really important in production but some test can make most strict assumptions // if we don't fail in this call unless absolutely necessary. - if (position+targetBuffer.remaining() > getWrittenOffset()) { + if (position + targetBuffer.remaining() > getWrittenOffset()) { outputStream.flush(); } } @@ -280,9 +245,9 @@ public class TranslogWriter extends TranslogReader { Channels.readFromFileChannelWithEofException(channel, position, targetBuffer); } - private synchronized void checkpoint(long lastSyncPosition, int operationCounter, ChannelReference channelReference) throws IOException { - channelReference.getChannel().force(false); - writeCheckpoint(lastSyncPosition, operationCounter, channelReference.getPath().getParent(), channelReference.getGeneration(), StandardOpenOption.WRITE); + private synchronized void checkpoint(long lastSyncPosition, int operationCounter, long generation, FileChannel translogFileChannel, Path translogFilePath) throws IOException { + translogFileChannel.force(false); + writeCheckpoint(lastSyncPosition, operationCounter, translogFilePath.getParent(), generation, StandardOpenOption.WRITE); } private static void writeCheckpoint(long syncPosition, int numOperations, Path translogFile, long generation, OpenOption... options) throws IOException { @@ -307,6 +272,17 @@ public class TranslogWriter extends TranslogReader { } } + @Override + public final void close() throws IOException { + if (closed.compareAndSet(false, true)) { + channel.close(); + } + } + + protected final boolean isClosed() { + return closed.get(); + } + private final class BufferedChannelOutputStream extends BufferedOutputStream { diff --git a/core/src/main/java/org/elasticsearch/indices/IndicesModule.java b/core/src/main/java/org/elasticsearch/indices/IndicesModule.java index 3f4c4acb7ebc..907a64f81750 100644 --- a/core/src/main/java/org/elasticsearch/indices/IndicesModule.java +++ b/core/src/main/java/org/elasticsearch/indices/IndicesModule.java @@ -22,9 +22,7 @@ package org.elasticsearch.indices; import org.elasticsearch.action.update.UpdateHelper; import org.elasticsearch.cluster.metadata.MetaDataIndexUpgradeService; import org.elasticsearch.common.geo.ShapesAvailability; -import org.elasticsearch.common.geo.builders.ShapeBuilderRegistry; import org.elasticsearch.common.inject.AbstractModule; -import org.elasticsearch.common.util.ExtensionPoint; import org.elasticsearch.index.NodeServicesProvider; import org.elasticsearch.index.mapper.Mapper; import org.elasticsearch.index.mapper.MetadataFieldMapper; @@ -57,52 +55,6 @@ import org.elasticsearch.index.mapper.internal.VersionFieldMapper; import org.elasticsearch.index.mapper.ip.IpFieldMapper; import org.elasticsearch.index.mapper.object.ObjectMapper; import org.elasticsearch.index.percolator.PercolatorFieldMapper; -import org.elasticsearch.index.query.BoolQueryParser; -import org.elasticsearch.index.query.BoostingQueryParser; -import org.elasticsearch.index.query.CommonTermsQueryParser; -import org.elasticsearch.index.query.ConstantScoreQueryParser; -import org.elasticsearch.index.query.DisMaxQueryParser; -import org.elasticsearch.index.query.ExistsQueryParser; -import org.elasticsearch.index.query.FieldMaskingSpanQueryParser; -import org.elasticsearch.index.query.FuzzyQueryParser; -import org.elasticsearch.index.query.GeoBoundingBoxQueryParser; -import org.elasticsearch.index.query.GeoDistanceQueryParser; -import org.elasticsearch.index.query.GeoDistanceRangeQueryParser; -import org.elasticsearch.index.query.GeoPolygonQueryParser; -import org.elasticsearch.index.query.GeoShapeQueryParser; -import org.elasticsearch.index.query.GeohashCellQuery; -import org.elasticsearch.index.query.HasChildQueryParser; -import org.elasticsearch.index.query.HasParentQueryParser; -import org.elasticsearch.index.query.IdsQueryParser; -import org.elasticsearch.index.query.IndicesQueryParser; -import org.elasticsearch.index.query.MatchAllQueryParser; -import org.elasticsearch.index.query.MatchNoneQueryParser; -import org.elasticsearch.index.query.MatchQueryParser; -import org.elasticsearch.index.query.MoreLikeThisQueryParser; -import org.elasticsearch.index.query.MultiMatchQueryParser; -import org.elasticsearch.index.query.NestedQueryParser; -import org.elasticsearch.index.query.PrefixQueryParser; -import org.elasticsearch.index.query.QueryParser; -import org.elasticsearch.index.query.QueryStringQueryParser; -import org.elasticsearch.index.query.RangeQueryParser; -import org.elasticsearch.index.query.RegexpQueryParser; -import org.elasticsearch.index.query.ScriptQueryParser; -import org.elasticsearch.index.query.SimpleQueryStringParser; -import org.elasticsearch.index.query.SpanContainingQueryParser; -import org.elasticsearch.index.query.SpanFirstQueryParser; -import org.elasticsearch.index.query.SpanMultiTermQueryParser; -import org.elasticsearch.index.query.SpanNearQueryParser; -import org.elasticsearch.index.query.SpanNotQueryParser; -import org.elasticsearch.index.query.SpanOrQueryParser; -import org.elasticsearch.index.query.SpanTermQueryParser; -import org.elasticsearch.index.query.SpanWithinQueryParser; -import org.elasticsearch.index.query.TemplateQueryParser; -import org.elasticsearch.index.query.TermQueryParser; -import org.elasticsearch.index.query.TermsQueryParser; -import org.elasticsearch.index.query.TypeQueryParser; -import org.elasticsearch.index.query.WildcardQueryParser; -import org.elasticsearch.index.query.WrapperQueryParser; -import org.elasticsearch.index.query.functionscore.FunctionScoreQueryParser; import org.elasticsearch.index.termvectors.TermVectorsService; import org.elasticsearch.indices.cache.query.IndicesQueryCache; import org.elasticsearch.indices.cache.request.IndicesRequestCache; @@ -111,7 +63,6 @@ import org.elasticsearch.indices.fielddata.cache.IndicesFieldDataCache; import org.elasticsearch.indices.fielddata.cache.IndicesFieldDataCacheListener; import org.elasticsearch.indices.flush.SyncedFlushService; import org.elasticsearch.indices.mapper.MapperRegistry; -import org.elasticsearch.indices.query.IndicesQueriesRegistry; import org.elasticsearch.indices.recovery.RecoverySettings; import org.elasticsearch.indices.recovery.RecoverySource; import org.elasticsearch.indices.recovery.RecoveryTarget; @@ -127,9 +78,6 @@ import java.util.Map; */ public class IndicesModule extends AbstractModule { - private final ExtensionPoint.ClassSet queryParsers - = new ExtensionPoint.ClassSet<>("query_parser", QueryParser.class); - private final Map mapperParsers = new LinkedHashMap<>(); // Use a LinkedHashMap for metadataMappers because iteration order matters @@ -137,62 +85,10 @@ public class IndicesModule extends AbstractModule { = new LinkedHashMap<>(); public IndicesModule() { - registerBuiltinQueryParsers(); registerBuiltInMappers(); registerBuiltInMetadataMappers(); } - private void registerBuiltinQueryParsers() { - registerQueryParser(MatchQueryParser.class); - registerQueryParser(MultiMatchQueryParser.class); - registerQueryParser(NestedQueryParser.class); - registerQueryParser(HasChildQueryParser.class); - registerQueryParser(HasParentQueryParser.class); - registerQueryParser(DisMaxQueryParser.class); - registerQueryParser(IdsQueryParser.class); - registerQueryParser(MatchAllQueryParser.class); - registerQueryParser(QueryStringQueryParser.class); - registerQueryParser(BoostingQueryParser.class); - registerQueryParser(BoolQueryParser.class); - registerQueryParser(TermQueryParser.class); - registerQueryParser(TermsQueryParser.class); - registerQueryParser(FuzzyQueryParser.class); - registerQueryParser(RegexpQueryParser.class); - registerQueryParser(RangeQueryParser.class); - registerQueryParser(PrefixQueryParser.class); - registerQueryParser(WildcardQueryParser.class); - registerQueryParser(ConstantScoreQueryParser.class); - registerQueryParser(SpanTermQueryParser.class); - registerQueryParser(SpanNotQueryParser.class); - registerQueryParser(SpanWithinQueryParser.class); - registerQueryParser(SpanContainingQueryParser.class); - registerQueryParser(FieldMaskingSpanQueryParser.class); - registerQueryParser(SpanFirstQueryParser.class); - registerQueryParser(SpanNearQueryParser.class); - registerQueryParser(SpanOrQueryParser.class); - registerQueryParser(MoreLikeThisQueryParser.class); - registerQueryParser(WrapperQueryParser.class); - registerQueryParser(IndicesQueryParser.class); - registerQueryParser(CommonTermsQueryParser.class); - registerQueryParser(SpanMultiTermQueryParser.class); - registerQueryParser(FunctionScoreQueryParser.class); - registerQueryParser(SimpleQueryStringParser.class); - registerQueryParser(TemplateQueryParser.class); - registerQueryParser(TypeQueryParser.class); - registerQueryParser(ScriptQueryParser.class); - registerQueryParser(GeoDistanceQueryParser.class); - registerQueryParser(GeoDistanceRangeQueryParser.class); - registerQueryParser(GeoBoundingBoxQueryParser.class); - registerQueryParser(GeohashCellQuery.Parser.class); - registerQueryParser(GeoPolygonQueryParser.class); - registerQueryParser(ExistsQueryParser.class); - registerQueryParser(MatchNoneQueryParser.class); - - if (ShapesAvailability.JTS_AVAILABLE && ShapesAvailability.SPATIAL4J_AVAILABLE) { - registerQueryParser(GeoShapeQueryParser.class); - } - } - private void registerBuiltInMappers() { registerMapper(ByteFieldMapper.CONTENT_TYPE, new ByteFieldMapper.TypeParser()); registerMapper(ShortFieldMapper.CONTENT_TYPE, new ShortFieldMapper.TypeParser()); @@ -236,10 +132,6 @@ public class IndicesModule extends AbstractModule { // last so that it can see all other mappers, including those coming from plugins } - public void registerQueryParser(Class queryParser) { - queryParsers.registerExtension(queryParser); - } - /** * Register a mapper for the given type. */ @@ -262,7 +154,6 @@ public class IndicesModule extends AbstractModule { @Override protected void configure() { - bindQueryParsersExtension(); bindMapperExtension(); bind(IndicesService.class).asEagerSingleton(); @@ -283,7 +174,6 @@ public class IndicesModule extends AbstractModule { bind(IndicesFieldDataCacheListener.class).asEagerSingleton(); bind(TermVectorsService.class).asEagerSingleton(); bind(NodeServicesProvider.class).asEagerSingleton(); - bind(ShapeBuilderRegistry.class).asEagerSingleton(); } // public for testing @@ -302,9 +192,4 @@ public class IndicesModule extends AbstractModule { protected void bindMapperExtension() { bind(MapperRegistry.class).toInstance(getMapperRegistry()); } - - protected void bindQueryParsersExtension() { - queryParsers.bind(binder()); - bind(IndicesQueriesRegistry.class).asEagerSingleton(); - } } diff --git a/core/src/main/java/org/elasticsearch/indices/cluster/IndicesClusterStateService.java b/core/src/main/java/org/elasticsearch/indices/cluster/IndicesClusterStateService.java index 28c6ce8f372d..82c97a48d889 100644 --- a/core/src/main/java/org/elasticsearch/indices/cluster/IndicesClusterStateService.java +++ b/core/src/main/java/org/elasticsearch/indices/cluster/IndicesClusterStateService.java @@ -412,8 +412,7 @@ public class IndicesClusterStateService extends AbstractLifecycleComponent> queryParsers; - @Inject - public IndicesQueriesRegistry(Settings settings, Set injectedQueryParsers, NamedWriteableRegistry namedWriteableRegistry) { + public IndicesQueriesRegistry(Settings settings, Map> queryParsers) { super(settings); - Map> queryParsers = new HashMap<>(); - for (@SuppressWarnings("unchecked") QueryParser queryParser : injectedQueryParsers) { - for (String name : queryParser.names()) { - queryParsers.put(name, queryParser); - } - @SuppressWarnings("unchecked") NamedWriteable qb = queryParser.getBuilderPrototype(); - namedWriteableRegistry.registerPrototype(QueryBuilder.class, qb); - } - // EmptyQueryBuilder is not registered as query parser but used internally. - // We need to register it with the NamedWriteableRegistry in order to serialize it - namedWriteableRegistry.registerPrototype(QueryBuilder.class, EmptyQueryBuilder.PROTOTYPE); - this.queryParsers = unmodifiableMap(queryParsers); + this.queryParsers = queryParsers; } /** diff --git a/core/src/main/java/org/elasticsearch/indices/recovery/RecoverySource.java b/core/src/main/java/org/elasticsearch/indices/recovery/RecoverySource.java index c0270e71721e..1ef9215b7b4d 100644 --- a/core/src/main/java/org/elasticsearch/indices/recovery/RecoverySource.java +++ b/core/src/main/java/org/elasticsearch/indices/recovery/RecoverySource.java @@ -37,6 +37,7 @@ import org.elasticsearch.transport.TransportChannel; import org.elasticsearch.transport.TransportRequestHandler; import org.elasticsearch.transport.TransportService; +import java.io.IOException; import java.util.ArrayList; import java.util.HashMap; import java.util.HashSet; @@ -82,7 +83,7 @@ public class RecoverySource extends AbstractComponent implements IndexEventListe } } - private RecoveryResponse recover(final StartRecoveryRequest request) { + private RecoveryResponse recover(final StartRecoveryRequest request) throws IOException { final IndexService indexService = indicesService.indexServiceSafe(request.shardId().index().name()); final IndexShard shard = indexService.getShard(request.shardId().id()); diff --git a/core/src/main/java/org/elasticsearch/indices/recovery/RecoverySourceHandler.java b/core/src/main/java/org/elasticsearch/indices/recovery/RecoverySourceHandler.java index 94c78efccd87..4699e8d5ace1 100644 --- a/core/src/main/java/org/elasticsearch/indices/recovery/RecoverySourceHandler.java +++ b/core/src/main/java/org/elasticsearch/indices/recovery/RecoverySourceHandler.java @@ -120,7 +120,7 @@ public class RecoverySourceHandler { /** * performs the recovery from the local engine to the target */ - public RecoveryResponse recoverToTarget() { + public RecoveryResponse recoverToTarget() throws IOException { try (Translog.View translogView = shard.acquireTranslogView()) { logger.trace("captured translog id [{}] for recovery", translogView.minTranslogGeneration()); final IndexCommit phase1Snapshot; @@ -144,8 +144,8 @@ public class RecoverySourceHandler { } logger.trace("snapshot translog for recovery. current size is [{}]", translogView.totalOperations()); - try (Translog.Snapshot phase2Snapshot = translogView.snapshot()) { - phase2(phase2Snapshot); + try { + phase2(translogView.snapshot()); } catch (Throwable e) { throw new RecoveryEngineException(shard.shardId(), 2, "phase2 failed", e); } @@ -308,7 +308,7 @@ public class RecoverySourceHandler { }); } - prepareTargetForTranslog(translogView); + prepareTargetForTranslog(translogView.totalOperations()); logger.trace("[{}][{}] recovery [phase1] to {}: took [{}]", indexName, shardId, request.targetNode(), stopWatch.totalTime()); response.phase1Time = stopWatch.totalTime().millis(); @@ -320,8 +320,7 @@ public class RecoverySourceHandler { } - - protected void prepareTargetForTranslog(final Translog.View translogView) { + protected void prepareTargetForTranslog(final int totalTranslogOps) { StopWatch stopWatch = new StopWatch().start(); logger.trace("{} recovery [phase1] to {}: prepare remote engine for translog", request.shardId(), request.targetNode()); final long startEngineStart = stopWatch.totalTime().millis(); @@ -332,7 +331,7 @@ public class RecoverySourceHandler { // operations. This ensures the shard engine is started and disables // garbage collection (not the JVM's GC!) of tombstone deletes transportService.submitRequest(request.targetNode(), RecoveryTarget.Actions.PREPARE_TRANSLOG, - new RecoveryPrepareForTranslogOperationsRequest(request.recoveryId(), request.shardId(), translogView.totalOperations()), + new RecoveryPrepareForTranslogOperationsRequest(request.recoveryId(), request.shardId(), totalTranslogOps), TransportRequestOptions.builder().withTimeout(recoverySettings.internalActionTimeout()).build(), EmptyTransportResponseHandler.INSTANCE_SAME).txGet(); } }); @@ -463,14 +462,14 @@ public class RecoverySourceHandler { // make sense to re-enable throttling in this phase cancellableThreads.execute(() -> { final RecoveryTranslogOperationsRequest translogOperationsRequest = new RecoveryTranslogOperationsRequest( - request.recoveryId(), request.shardId(), operations, snapshot.estimatedTotalOperations()); + request.recoveryId(), request.shardId(), operations, snapshot.totalOperations()); transportService.submitRequest(request.targetNode(), RecoveryTarget.Actions.TRANSLOG_OPS, translogOperationsRequest, recoveryOptions, EmptyTransportResponseHandler.INSTANCE_SAME).txGet(); }); if (logger.isTraceEnabled()) { logger.trace("[{}][{}] sent batch of [{}][{}] (total: [{}]) translog operations to {}", indexName, shardId, ops, new ByteSizeValue(size), - snapshot.estimatedTotalOperations(), + snapshot.totalOperations(), request.targetNode()); } @@ -488,7 +487,7 @@ public class RecoverySourceHandler { if (!operations.isEmpty()) { cancellableThreads.execute(() -> { RecoveryTranslogOperationsRequest translogOperationsRequest = new RecoveryTranslogOperationsRequest( - request.recoveryId(), request.shardId(), operations, snapshot.estimatedTotalOperations()); + request.recoveryId(), request.shardId(), operations, snapshot.totalOperations()); transportService.submitRequest(request.targetNode(), RecoveryTarget.Actions.TRANSLOG_OPS, translogOperationsRequest, recoveryOptions, EmptyTransportResponseHandler.INSTANCE_SAME).txGet(); }); @@ -497,7 +496,7 @@ public class RecoverySourceHandler { if (logger.isTraceEnabled()) { logger.trace("[{}][{}] sent final batch of [{}][{}] (total: [{}]) translog operations to {}", indexName, shardId, ops, new ByteSizeValue(size), - snapshot.estimatedTotalOperations(), + snapshot.totalOperations(), request.targetNode()); } return totalOperations; diff --git a/core/src/main/java/org/elasticsearch/indices/recovery/SharedFSRecoverySourceHandler.java b/core/src/main/java/org/elasticsearch/indices/recovery/SharedFSRecoverySourceHandler.java index e849580b2c4d..16bd1d46553d 100644 --- a/core/src/main/java/org/elasticsearch/indices/recovery/SharedFSRecoverySourceHandler.java +++ b/core/src/main/java/org/elasticsearch/indices/recovery/SharedFSRecoverySourceHandler.java @@ -58,7 +58,7 @@ public class SharedFSRecoverySourceHandler extends RecoverySourceHandler { shard.failShard("failed to close engine (phase1)", e); } } - prepareTargetForTranslog(Translog.View.EMPTY_VIEW); + prepareTargetForTranslog(0); finalizeRecovery(); return response; } catch (Throwable t) { diff --git a/core/src/main/java/org/elasticsearch/monitor/os/OsStats.java b/core/src/main/java/org/elasticsearch/monitor/os/OsStats.java index 26f3afacb3f3..c419c4f2608a 100644 --- a/core/src/main/java/org/elasticsearch/monitor/os/OsStats.java +++ b/core/src/main/java/org/elasticsearch/monitor/os/OsStats.java @@ -65,6 +65,9 @@ public class OsStats implements Streamable, ToXContent { static final XContentBuilderString CPU = new XContentBuilderString("cpu"); static final XContentBuilderString PERCENT = new XContentBuilderString("percent"); static final XContentBuilderString LOAD_AVERAGE = new XContentBuilderString("load_average"); + static final XContentBuilderString LOAD_AVERAGE_1M = new XContentBuilderString("1m"); + static final XContentBuilderString LOAD_AVERAGE_5M = new XContentBuilderString("5m"); + static final XContentBuilderString LOAD_AVERAGE_15M = new XContentBuilderString("15m"); static final XContentBuilderString MEM = new XContentBuilderString("mem"); static final XContentBuilderString SWAP = new XContentBuilderString("swap"); @@ -77,7 +80,6 @@ public class OsStats implements Streamable, ToXContent { static final XContentBuilderString FREE_PERCENT = new XContentBuilderString("free_percent"); static final XContentBuilderString USED_PERCENT = new XContentBuilderString("used_percent"); - } @Override @@ -88,11 +90,17 @@ public class OsStats implements Streamable, ToXContent { builder.startObject(Fields.CPU); builder.field(Fields.PERCENT, cpu.getPercent()); if (cpu.getLoadAverage() != null) { - builder.startArray(Fields.LOAD_AVERAGE); - builder.value(cpu.getLoadAverage()[0]); - builder.value(cpu.getLoadAverage()[1]); - builder.value(cpu.getLoadAverage()[2]); - builder.endArray(); + builder.startObject(Fields.LOAD_AVERAGE); + if (cpu.getLoadAverage()[0] != -1) { + builder.field(Fields.LOAD_AVERAGE_1M, cpu.getLoadAverage()[0]); + } + if (cpu.getLoadAverage()[1] != -1) { + builder.field(Fields.LOAD_AVERAGE_5M, cpu.getLoadAverage()[1]); + } + if (cpu.getLoadAverage()[2] != -1) { + builder.field(Fields.LOAD_AVERAGE_15M, cpu.getLoadAverage()[2]); + } + builder.endObject(); } builder.endObject(); } diff --git a/core/src/main/java/org/elasticsearch/node/Node.java b/core/src/main/java/org/elasticsearch/node/Node.java index da3d1e94e879..c5cf53defaae 100644 --- a/core/src/main/java/org/elasticsearch/node/Node.java +++ b/core/src/main/java/org/elasticsearch/node/Node.java @@ -37,6 +37,7 @@ import org.elasticsearch.common.component.LifecycleComponent; import org.elasticsearch.common.inject.Injector; import org.elasticsearch.common.inject.Module; import org.elasticsearch.common.inject.ModulesBuilder; +import org.elasticsearch.common.io.stream.NamedWriteableRegistry; import org.elasticsearch.common.lease.Releasable; import org.elasticsearch.common.lease.Releasables; import org.elasticsearch.common.logging.ESLogger; @@ -164,6 +165,7 @@ public class Node implements Releasable { final NetworkService networkService = new NetworkService(settings); final SettingsFilter settingsFilter = new SettingsFilter(settings); final ThreadPool threadPool = new ThreadPool(settings); + NamedWriteableRegistry namedWriteableRegistry = new NamedWriteableRegistry(); boolean success = false; try { final MonitorService monitorService = new MonitorService(settings, nodeEnvironment, threadPool); @@ -178,7 +180,7 @@ public class Node implements Releasable { modules.add(new SettingsModule(this.settings, settingsFilter)); modules.add(new EnvironmentModule(environment)); modules.add(new NodeModule(this, monitorService)); - modules.add(new NetworkModule(networkService, settings, false)); + modules.add(new NetworkModule(networkService, settings, false, namedWriteableRegistry)); modules.add(new ScriptModule(this.settings)); modules.add(new NodeEnvironmentModule(nodeEnvironment)); modules.add(new ClusterNameModule(this.settings)); @@ -186,7 +188,7 @@ public class Node implements Releasable { modules.add(new DiscoveryModule(this.settings)); modules.add(new ClusterModule(this.settings)); modules.add(new IndicesModule()); - modules.add(new SearchModule()); + modules.add(new SearchModule(settings, namedWriteableRegistry)); modules.add(new ActionModule(false)); modules.add(new GatewayModule(settings)); modules.add(new NodeClientModule()); diff --git a/core/src/main/java/org/elasticsearch/rest/BaseRestHandler.java b/core/src/main/java/org/elasticsearch/rest/BaseRestHandler.java index 294338c05017..bb99218855f5 100644 --- a/core/src/main/java/org/elasticsearch/rest/BaseRestHandler.java +++ b/core/src/main/java/org/elasticsearch/rest/BaseRestHandler.java @@ -71,7 +71,7 @@ public abstract class BaseRestHandler extends AbstractComponent implements RestH this.headers = headers; } - private static void copyHeadersAndContext(ActionRequest actionRequest, RestRequest restRequest, Set headers) { + private static void copyHeadersAndContext(ActionRequest actionRequest, RestRequest restRequest, Set headers) { for (String usefulHeader : headers) { String headerValue = restRequest.header(usefulHeader); if (headerValue != null) { @@ -82,7 +82,8 @@ public abstract class BaseRestHandler extends AbstractComponent implements RestH } @Override - protected > void doExecute(Action action, Request request, ActionListener listener) { + protected , Response extends ActionResponse, RequestBuilder extends ActionRequestBuilder> void doExecute( + Action action, Request request, ActionListener listener) { copyHeadersAndContext(request, restRequest, headers); super.doExecute(action, request, listener); } diff --git a/core/src/main/java/org/elasticsearch/rest/action/cat/RestNodesAction.java b/core/src/main/java/org/elasticsearch/rest/action/cat/RestNodesAction.java index 7bbc5b9eff99..c7eab8ca14b1 100644 --- a/core/src/main/java/org/elasticsearch/rest/action/cat/RestNodesAction.java +++ b/core/src/main/java/org/elasticsearch/rest/action/cat/RestNodesAction.java @@ -266,9 +266,9 @@ public class RestNodesAction extends AbstractCatAction { table.addCell(osStats == null ? null : Short.toString(osStats.getCpu().getPercent())); boolean hasLoadAverage = osStats != null && osStats.getCpu().getLoadAverage() != null; - table.addCell(!hasLoadAverage ? null : String.format(Locale.ROOT, "%.2f", osStats.getCpu().getLoadAverage()[0])); - table.addCell(!hasLoadAverage ? null : String.format(Locale.ROOT, "%.2f", osStats.getCpu().getLoadAverage()[1])); - table.addCell(!hasLoadAverage ? null : String.format(Locale.ROOT, "%.2f", osStats.getCpu().getLoadAverage()[2])); + table.addCell(!hasLoadAverage || osStats.getCpu().getLoadAverage()[0] == -1 ? null : String.format(Locale.ROOT, "%.2f", osStats.getCpu().getLoadAverage()[0])); + table.addCell(!hasLoadAverage || osStats.getCpu().getLoadAverage()[1] == -1 ? null : String.format(Locale.ROOT, "%.2f", osStats.getCpu().getLoadAverage()[1])); + table.addCell(!hasLoadAverage || osStats.getCpu().getLoadAverage()[2] == -1 ? null : String.format(Locale.ROOT, "%.2f", osStats.getCpu().getLoadAverage()[2])); table.addCell(jvmStats == null ? null : jvmStats.getUptime()); table.addCell(node.clientNode() ? "c" : node.dataNode() ? "d" : "-"); table.addCell(masterId == null ? "x" : masterId.equals(node.id()) ? "*" : node.masterNode() ? "m" : "-"); diff --git a/core/src/main/java/org/elasticsearch/search/SearchModule.java b/core/src/main/java/org/elasticsearch/search/SearchModule.java index 6f16d13bd922..e5c3e90739e9 100644 --- a/core/src/main/java/org/elasticsearch/search/SearchModule.java +++ b/core/src/main/java/org/elasticsearch/search/SearchModule.java @@ -19,10 +19,89 @@ package org.elasticsearch.search; +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 java.util.function.Supplier; + +import org.apache.lucene.search.BooleanQuery; +import org.elasticsearch.common.geo.ShapesAvailability; +import org.elasticsearch.common.geo.builders.CircleBuilder; +import org.elasticsearch.common.geo.builders.EnvelopeBuilder; +import org.elasticsearch.common.geo.builders.GeometryCollectionBuilder; +import org.elasticsearch.common.geo.builders.LineStringBuilder; +import org.elasticsearch.common.geo.builders.MultiLineStringBuilder; +import org.elasticsearch.common.geo.builders.MultiPointBuilder; +import org.elasticsearch.common.geo.builders.MultiPolygonBuilder; +import org.elasticsearch.common.geo.builders.PointBuilder; +import org.elasticsearch.common.geo.builders.PolygonBuilder; +import org.elasticsearch.common.geo.builders.ShapeBuilder; import org.elasticsearch.common.inject.AbstractModule; import org.elasticsearch.common.inject.multibindings.Multibinder; +import org.elasticsearch.common.io.stream.NamedWriteableRegistry; +import org.elasticsearch.common.settings.Settings; +import org.elasticsearch.index.query.BoolQueryParser; +import org.elasticsearch.index.query.BoostingQueryParser; +import org.elasticsearch.index.query.CommonTermsQueryParser; +import org.elasticsearch.index.query.ConstantScoreQueryParser; +import org.elasticsearch.index.query.DisMaxQueryParser; +import org.elasticsearch.index.query.EmptyQueryBuilder; +import org.elasticsearch.index.query.ExistsQueryParser; +import org.elasticsearch.index.query.FieldMaskingSpanQueryParser; +import org.elasticsearch.index.query.FuzzyQueryParser; +import org.elasticsearch.index.query.GeoBoundingBoxQueryParser; +import org.elasticsearch.index.query.GeoDistanceQueryParser; +import org.elasticsearch.index.query.GeoDistanceRangeQueryParser; +import org.elasticsearch.index.query.GeoPolygonQueryParser; +import org.elasticsearch.index.query.GeoShapeQueryParser; +import org.elasticsearch.index.query.GeohashCellQuery; +import org.elasticsearch.index.query.HasChildQueryParser; +import org.elasticsearch.index.query.HasParentQueryParser; +import org.elasticsearch.index.query.IdsQueryParser; +import org.elasticsearch.index.query.IndicesQueryParser; +import org.elasticsearch.index.query.MatchAllQueryParser; +import org.elasticsearch.index.query.MatchNoneQueryParser; +import org.elasticsearch.index.query.MatchQueryParser; +import org.elasticsearch.index.query.MoreLikeThisQueryParser; +import org.elasticsearch.index.query.MultiMatchQueryParser; +import org.elasticsearch.index.query.NestedQueryParser; +import org.elasticsearch.index.query.PrefixQueryParser; +import org.elasticsearch.index.query.QueryBuilder; +import org.elasticsearch.index.query.QueryParser; +import org.elasticsearch.index.query.QueryStringQueryParser; +import org.elasticsearch.index.query.RangeQueryParser; +import org.elasticsearch.index.query.RegexpQueryParser; +import org.elasticsearch.index.query.ScriptQueryParser; +import org.elasticsearch.index.query.SimpleQueryStringParser; +import org.elasticsearch.index.query.SpanContainingQueryParser; +import org.elasticsearch.index.query.SpanFirstQueryParser; +import org.elasticsearch.index.query.SpanMultiTermQueryParser; +import org.elasticsearch.index.query.SpanNearQueryParser; +import org.elasticsearch.index.query.SpanNotQueryParser; +import org.elasticsearch.index.query.SpanOrQueryParser; +import org.elasticsearch.index.query.SpanTermQueryParser; +import org.elasticsearch.index.query.SpanWithinQueryParser; +import org.elasticsearch.index.query.TemplateQueryParser; +import org.elasticsearch.index.query.TermQueryParser; +import org.elasticsearch.index.query.TermsQueryParser; +import org.elasticsearch.index.query.TypeQueryParser; +import org.elasticsearch.index.query.WildcardQueryParser; +import org.elasticsearch.index.query.WrapperQueryParser; +import org.elasticsearch.index.query.functionscore.FunctionScoreQueryParser; +import org.elasticsearch.index.query.functionscore.ScoreFunctionBuilder; import org.elasticsearch.index.query.functionscore.ScoreFunctionParser; import org.elasticsearch.index.query.functionscore.ScoreFunctionParserMapper; +import org.elasticsearch.index.query.functionscore.exp.ExponentialDecayFunctionParser; +import org.elasticsearch.index.query.functionscore.fieldvaluefactor.FieldValueFactorFunctionParser; +import org.elasticsearch.index.query.functionscore.gauss.GaussDecayFunctionParser; +import org.elasticsearch.index.query.functionscore.lin.LinearDecayFunctionParser; +import org.elasticsearch.index.query.functionscore.random.RandomScoreFunctionParser; +import org.elasticsearch.index.query.functionscore.script.ScriptScoreFunctionParser; +import org.elasticsearch.index.query.functionscore.weight.WeightBuilder; +import org.elasticsearch.indices.query.IndicesQueriesRegistry; import org.elasticsearch.search.action.SearchServiceTransportAction; import org.elasticsearch.search.aggregations.AggregationParseElement; import org.elasticsearch.search.aggregations.AggregationPhase; @@ -149,9 +228,6 @@ import org.elasticsearch.search.query.QueryPhase; import org.elasticsearch.search.suggest.Suggester; import org.elasticsearch.search.suggest.Suggesters; -import java.util.HashSet; -import java.util.Set; - /** * */ @@ -161,14 +237,35 @@ public class SearchModule extends AbstractModule { private final Set> pipelineAggParsers = new HashSet<>(); private final Highlighters highlighters = new Highlighters(); private final Suggesters suggesters = new Suggesters(); - private final Set> functionScoreParsers = new HashSet<>(); + /** + * Function score parsers constructed on registration. This is ok because + * they don't have any dependencies. + */ + private final Map> functionScoreParsers = new HashMap<>(); + /** + * Query parsers constructed at configure time. These have to be constructed + * at configure time because they depend on things that are registered by + * plugins (function score parsers). + */ + private final List>> queryParsers = new ArrayList<>(); private final Set> fetchSubPhases = new HashSet<>(); private final Set> heuristicParsers = new HashSet<>(); private final Set> modelParsers = new HashSet<>(); + private final Settings settings; + private final NamedWriteableRegistry namedWriteableRegistry; + // pkg private so tests can mock Class searchServiceImpl = SearchService.class; + public SearchModule(Settings settings, NamedWriteableRegistry namedWriteableRegistry) { + this.settings = settings; + this.namedWriteableRegistry = namedWriteableRegistry; + + registerBuiltinFunctionScoreParsers(); + registerBuiltinQueryParsers(); + } + public void registerHighlighter(String key, Class clazz) { highlighters.registerExtension(key, clazz); } @@ -177,8 +274,21 @@ public class SearchModule extends AbstractModule { suggesters.registerExtension(key, suggester); } - public void registerFunctionScoreParser(Class parser) { - functionScoreParsers.add(parser); + /** + * Register a new ScoreFunctionParser. + */ + public void registerFunctionScoreParser(ScoreFunctionParser parser) { + for (String name: parser.getNames()) { + Object oldValue = functionScoreParsers.putIfAbsent(name, parser); + if (oldValue != null) { + throw new IllegalArgumentException("Function score parser [" + oldValue + "] already registered for name [" + name + "]"); + } + } + namedWriteableRegistry.registerPrototype(ScoreFunctionBuilder.class, parser.getBuilderPrototype()); + } + + public void registerQueryParser(Supplier> parser) { + queryParsers.add(parser); } public void registerFetchSubPhase(Class subPhase) { @@ -212,8 +322,9 @@ public class SearchModule extends AbstractModule { configureAggs(); configureHighlighters(); configureSuggesters(); - configureFunctionScore(); + bind(IndicesQueriesRegistry.class).toInstance(buildQueryParserRegistry()); configureFetchSubPhase(); + configureShapes(); } protected void configureFetchSubPhase() { @@ -231,16 +342,23 @@ public class SearchModule extends AbstractModule { bind(InnerHitsFetchSubPhase.class).asEagerSingleton(); } - protected void configureSuggesters() { - suggesters.bind(binder()); + public IndicesQueriesRegistry buildQueryParserRegistry() { + Map> queryParsersMap = new HashMap<>(); + for (Supplier> parserSupplier : queryParsers) { + QueryParser parser = parserSupplier.get(); + for (String name: parser.names()) { + Object oldValue = queryParsersMap.putIfAbsent(name, parser); + if (oldValue != null) { + throw new IllegalArgumentException("Query parser [" + oldValue + "] already registered for name [" + name + "] while trying to register [" + parser + "]"); + } + } + namedWriteableRegistry.registerPrototype(QueryBuilder.class, parser.getBuilderPrototype()); + } + return new IndicesQueriesRegistry(settings, queryParsersMap); } - protected void configureFunctionScore() { - Multibinder parserMapBinder = Multibinder.newSetBinder(binder(), ScoreFunctionParser.class); - for (Class clazz : functionScoreParsers) { - parserMapBinder.addBinding().to(clazz); - } - bind(ScoreFunctionParserMapper.class).asEagerSingleton(); + protected void configureSuggesters() { + suggesters.bind(binder()); } protected void configureHighlighters() { @@ -332,6 +450,87 @@ public class SearchModule extends AbstractModule { } } + private void configureShapes() { + if (ShapesAvailability.JTS_AVAILABLE && ShapesAvailability.SPATIAL4J_AVAILABLE) { + namedWriteableRegistry.registerPrototype(ShapeBuilder.class, PointBuilder.PROTOTYPE); + namedWriteableRegistry.registerPrototype(ShapeBuilder.class, CircleBuilder.PROTOTYPE); + namedWriteableRegistry.registerPrototype(ShapeBuilder.class, EnvelopeBuilder.PROTOTYPE); + namedWriteableRegistry.registerPrototype(ShapeBuilder.class, MultiPointBuilder.PROTOTYPE); + namedWriteableRegistry.registerPrototype(ShapeBuilder.class, LineStringBuilder.PROTOTYPE); + namedWriteableRegistry.registerPrototype(ShapeBuilder.class, MultiLineStringBuilder.PROTOTYPE); + namedWriteableRegistry.registerPrototype(ShapeBuilder.class, PolygonBuilder.PROTOTYPE); + namedWriteableRegistry.registerPrototype(ShapeBuilder.class, MultiPolygonBuilder.PROTOTYPE); + namedWriteableRegistry.registerPrototype(ShapeBuilder.class, GeometryCollectionBuilder.PROTOTYPE); + } + } + + private void registerBuiltinFunctionScoreParsers() { + registerFunctionScoreParser(new ScriptScoreFunctionParser()); + registerFunctionScoreParser(new GaussDecayFunctionParser()); + registerFunctionScoreParser(new LinearDecayFunctionParser()); + registerFunctionScoreParser(new ExponentialDecayFunctionParser()); + registerFunctionScoreParser(new RandomScoreFunctionParser()); + registerFunctionScoreParser(new FieldValueFactorFunctionParser()); + //weight doesn't have its own parser, so every function supports it out of the box. + //Can be a single function too when not associated to any other function, which is why it needs to be registered manually here. + namedWriteableRegistry.registerPrototype(ScoreFunctionBuilder.class, new WeightBuilder()); + } + + private void registerBuiltinQueryParsers() { + registerQueryParser(MatchQueryParser::new); + registerQueryParser(MultiMatchQueryParser::new); + registerQueryParser(NestedQueryParser::new); + registerQueryParser(HasChildQueryParser::new); + registerQueryParser(HasParentQueryParser::new); + registerQueryParser(DisMaxQueryParser::new); + registerQueryParser(IdsQueryParser::new); + registerQueryParser(MatchAllQueryParser::new); + registerQueryParser(QueryStringQueryParser::new); + registerQueryParser(BoostingQueryParser::new); + BooleanQuery.setMaxClauseCount(settings.getAsInt("index.query.bool.max_clause_count", settings.getAsInt("indices.query.bool.max_clause_count", BooleanQuery.getMaxClauseCount()))); + registerQueryParser(BoolQueryParser::new); + registerQueryParser(TermQueryParser::new); + registerQueryParser(TermsQueryParser::new); + registerQueryParser(FuzzyQueryParser::new); + registerQueryParser(RegexpQueryParser::new); + registerQueryParser(RangeQueryParser::new); + registerQueryParser(PrefixQueryParser::new); + registerQueryParser(WildcardQueryParser::new); + registerQueryParser(ConstantScoreQueryParser::new); + registerQueryParser(SpanTermQueryParser::new); + registerQueryParser(SpanNotQueryParser::new); + registerQueryParser(SpanWithinQueryParser::new); + registerQueryParser(SpanContainingQueryParser::new); + registerQueryParser(FieldMaskingSpanQueryParser::new); + registerQueryParser(SpanFirstQueryParser::new); + registerQueryParser(SpanNearQueryParser::new); + registerQueryParser(SpanOrQueryParser::new); + registerQueryParser(MoreLikeThisQueryParser::new); + registerQueryParser(WrapperQueryParser::new); + registerQueryParser(IndicesQueryParser::new); + registerQueryParser(CommonTermsQueryParser::new); + registerQueryParser(SpanMultiTermQueryParser::new); + // This is delayed until configure time to give plugins a chance to register parsers + registerQueryParser(() -> new FunctionScoreQueryParser(new ScoreFunctionParserMapper(functionScoreParsers))); + registerQueryParser(SimpleQueryStringParser::new); + registerQueryParser(TemplateQueryParser::new); + registerQueryParser(TypeQueryParser::new); + registerQueryParser(ScriptQueryParser::new); + registerQueryParser(GeoDistanceQueryParser::new); + registerQueryParser(GeoDistanceRangeQueryParser::new); + registerQueryParser(GeoBoundingBoxQueryParser::new); + registerQueryParser(GeohashCellQuery.Parser::new); + registerQueryParser(GeoPolygonQueryParser::new); + registerQueryParser(ExistsQueryParser::new); + registerQueryParser(MatchNoneQueryParser::new); + if (ShapesAvailability.JTS_AVAILABLE && ShapesAvailability.SPATIAL4J_AVAILABLE) { + registerQueryParser(GeoShapeQueryParser::new); + } + // EmptyQueryBuilder is not registered as query parser but used internally. + // We need to register it with the NamedWriteableRegistry in order to serialize it + namedWriteableRegistry.registerPrototype(QueryBuilder.class, EmptyQueryBuilder.PROTOTYPE); + } + static { // calcs InternalAvg.registerStreams(); @@ -393,5 +592,4 @@ public class SearchModule extends AbstractModule { BucketSelectorPipelineAggregator.registerStreams(); SerialDiffPipelineAggregator.registerStreams(); } - } diff --git a/core/src/main/java/org/elasticsearch/search/aggregations/bucket/geogrid/GeoHashGridParser.java b/core/src/main/java/org/elasticsearch/search/aggregations/bucket/geogrid/GeoHashGridParser.java index 6473b5ae7f2a..60302f2da421 100644 --- a/core/src/main/java/org/elasticsearch/search/aggregations/bucket/geogrid/GeoHashGridParser.java +++ b/core/src/main/java/org/elasticsearch/search/aggregations/bucket/geogrid/GeoHashGridParser.java @@ -62,7 +62,8 @@ public class GeoHashGridParser implements Aggregator.Parser { @Override public AggregatorFactory parse(String aggregationName, XContentParser parser, SearchContext context) throws IOException { - ValuesSourceParser vsParser = ValuesSourceParser.geoPoint(aggregationName, InternalGeoHashGrid.TYPE, context).build(); + ValuesSourceParser vsParser = ValuesSourceParser.geoPoint(aggregationName, InternalGeoHashGrid.TYPE, context) + .build(); int precision = GeoHashGridParams.DEFAULT_PRECISION; int requiredSize = GeoHashGridParams.DEFAULT_MAX_NUM_CELLS; @@ -131,6 +132,7 @@ public class GeoHashGridParser implements Aggregator.Parser { final InternalAggregation aggregation = new InternalGeoHashGrid(name, requiredSize, Collections. emptyList(), pipelineAggregators, metaData); return new NonCollectingAggregator(name, aggregationContext, parent, pipelineAggregators, metaData) { + @Override public InternalAggregation buildEmptyAggregation() { return aggregation; } diff --git a/core/src/main/java/org/elasticsearch/search/aggregations/bucket/histogram/DateHistogramParser.java b/core/src/main/java/org/elasticsearch/search/aggregations/bucket/histogram/DateHistogramParser.java index 694abf26e66b..52d77e1594ca 100644 --- a/core/src/main/java/org/elasticsearch/search/aggregations/bucket/histogram/DateHistogramParser.java +++ b/core/src/main/java/org/elasticsearch/search/aggregations/bucket/histogram/DateHistogramParser.java @@ -28,6 +28,7 @@ import org.elasticsearch.search.SearchParseException; import org.elasticsearch.search.aggregations.Aggregator; import org.elasticsearch.search.aggregations.AggregatorFactory; import org.elasticsearch.search.aggregations.support.ValueType; +import org.elasticsearch.search.aggregations.support.ValuesSource.Numeric; import org.elasticsearch.search.aggregations.support.ValuesSourceConfig; import org.elasticsearch.search.aggregations.support.ValuesSourceParser; import org.elasticsearch.search.internal.SearchContext; @@ -78,7 +79,7 @@ public class DateHistogramParser implements Aggregator.Parser { @Override public AggregatorFactory parse(String aggregationName, XContentParser parser, SearchContext context) throws IOException { - ValuesSourceParser vsParser = ValuesSourceParser.numeric(aggregationName, InternalDateHistogram.TYPE, context) + ValuesSourceParser vsParser = ValuesSourceParser.numeric(aggregationName, InternalDateHistogram.TYPE, context) .targetValueType(ValueType.DATE) .formattable(true) .timezoneAware(true) @@ -190,7 +191,7 @@ public class DateHistogramParser implements Aggregator.Parser { .timeZone(vsParser.input().timezone()) .offset(offset).build(); - ValuesSourceConfig config = vsParser.config(); + ValuesSourceConfig config = vsParser.config(); return new HistogramAggregator.Factory(aggregationName, config, rounding, order, keyed, minDocCount, extendedBounds, new InternalDateHistogram.Factory()); diff --git a/core/src/main/java/org/elasticsearch/search/aggregations/bucket/histogram/HistogramParser.java b/core/src/main/java/org/elasticsearch/search/aggregations/bucket/histogram/HistogramParser.java index c738251d0e04..31ee6681d4ec 100644 --- a/core/src/main/java/org/elasticsearch/search/aggregations/bucket/histogram/HistogramParser.java +++ b/core/src/main/java/org/elasticsearch/search/aggregations/bucket/histogram/HistogramParser.java @@ -25,6 +25,7 @@ import org.elasticsearch.search.SearchParseException; import org.elasticsearch.search.aggregations.Aggregator; import org.elasticsearch.search.aggregations.AggregatorFactory; import org.elasticsearch.search.aggregations.support.ValueType; +import org.elasticsearch.search.aggregations.support.ValuesSource.Numeric; import org.elasticsearch.search.aggregations.support.ValuesSourceParser; import org.elasticsearch.search.aggregations.support.format.ValueParser; import org.elasticsearch.search.internal.SearchContext; @@ -46,7 +47,7 @@ public class HistogramParser implements Aggregator.Parser { @Override public AggregatorFactory parse(String aggregationName, XContentParser parser, SearchContext context) throws IOException { - ValuesSourceParser vsParser = ValuesSourceParser.numeric(aggregationName, InternalHistogram.TYPE, context) + ValuesSourceParser vsParser = ValuesSourceParser.numeric(aggregationName, InternalHistogram.TYPE, context) .targetValueType(ValueType.NUMERIC) .formattable(true) .build(); @@ -127,7 +128,7 @@ public class HistogramParser implements Aggregator.Parser { Rounding rounding = new Rounding.Interval(interval); if (offset != 0) { - rounding = new Rounding.OffsetRounding((Rounding.Interval) rounding, offset); + rounding = new Rounding.OffsetRounding(rounding, offset); } if (extendedBounds != null) { @@ -136,7 +137,7 @@ public class HistogramParser implements Aggregator.Parser { } return new HistogramAggregator.Factory(aggregationName, vsParser.config(), rounding, order, keyed, minDocCount, extendedBounds, - new InternalHistogram.Factory()); + new InternalHistogram.Factory<>()); } diff --git a/core/src/main/java/org/elasticsearch/search/aggregations/bucket/missing/MissingAggregator.java b/core/src/main/java/org/elasticsearch/search/aggregations/bucket/missing/MissingAggregator.java index 1ae7341f6110..38e15e216c27 100644 --- a/core/src/main/java/org/elasticsearch/search/aggregations/bucket/missing/MissingAggregator.java +++ b/core/src/main/java/org/elasticsearch/search/aggregations/bucket/missing/MissingAggregator.java @@ -81,9 +81,9 @@ public class MissingAggregator extends SingleBucketAggregator { return new InternalMissing(name, 0, buildEmptySubAggregations(), pipelineAggregators(), metaData()); } - public static class Factory extends ValuesSourceAggregatorFactory { + public static class Factory extends ValuesSourceAggregatorFactory { - public Factory(String name, ValuesSourceConfig valueSourceConfig) { + public Factory(String name, ValuesSourceConfig valueSourceConfig) { super(name, InternalMissing.TYPE.name(), valueSourceConfig); } diff --git a/core/src/main/java/org/elasticsearch/search/aggregations/bucket/missing/MissingParser.java b/core/src/main/java/org/elasticsearch/search/aggregations/bucket/missing/MissingParser.java index 6ecdc129dd0d..4210e020d8c2 100644 --- a/core/src/main/java/org/elasticsearch/search/aggregations/bucket/missing/MissingParser.java +++ b/core/src/main/java/org/elasticsearch/search/aggregations/bucket/missing/MissingParser.java @@ -22,6 +22,7 @@ import org.elasticsearch.common.xcontent.XContentParser; import org.elasticsearch.search.SearchParseException; import org.elasticsearch.search.aggregations.Aggregator; import org.elasticsearch.search.aggregations.AggregatorFactory; +import org.elasticsearch.search.aggregations.support.ValuesSource; import org.elasticsearch.search.aggregations.support.ValuesSourceParser; import org.elasticsearch.search.internal.SearchContext; @@ -39,8 +40,7 @@ public class MissingParser implements Aggregator.Parser { @Override public AggregatorFactory parse(String aggregationName, XContentParser parser, SearchContext context) throws IOException { - - ValuesSourceParser vsParser = ValuesSourceParser.any(aggregationName, InternalMissing.TYPE, context) + ValuesSourceParser vsParser = ValuesSourceParser.any(aggregationName, InternalMissing.TYPE, context) .scriptable(false) .build(); diff --git a/core/src/main/java/org/elasticsearch/search/aggregations/bucket/sampler/SamplerAggregator.java b/core/src/main/java/org/elasticsearch/search/aggregations/bucket/sampler/SamplerAggregator.java index 8cb980954cb4..4541aa9d142c 100644 --- a/core/src/main/java/org/elasticsearch/search/aggregations/bucket/sampler/SamplerAggregator.java +++ b/core/src/main/java/org/elasticsearch/search/aggregations/bucket/sampler/SamplerAggregator.java @@ -203,7 +203,8 @@ public class SamplerAggregator extends SingleBucketAggregator { private int maxDocsPerValue; private String executionHint; - public DiversifiedFactory(String name, int shardSize, String executionHint, ValuesSourceConfig vsConfig, int maxDocsPerValue) { + public DiversifiedFactory(String name, int shardSize, String executionHint, ValuesSourceConfig vsConfig, + int maxDocsPerValue) { super(name, InternalSampler.TYPE.name(), vsConfig); this.shardSize = shardSize; this.maxDocsPerValue = maxDocsPerValue; diff --git a/core/src/main/java/org/elasticsearch/search/aggregations/bucket/sampler/SamplerParser.java b/core/src/main/java/org/elasticsearch/search/aggregations/bucket/sampler/SamplerParser.java index 498a7cb5c66a..d51f43657f72 100644 --- a/core/src/main/java/org/elasticsearch/search/aggregations/bucket/sampler/SamplerParser.java +++ b/core/src/main/java/org/elasticsearch/search/aggregations/bucket/sampler/SamplerParser.java @@ -23,6 +23,7 @@ import org.elasticsearch.common.xcontent.XContentParser; import org.elasticsearch.search.SearchParseException; import org.elasticsearch.search.aggregations.Aggregator; import org.elasticsearch.search.aggregations.AggregatorFactory; +import org.elasticsearch.search.aggregations.support.ValuesSource; import org.elasticsearch.search.aggregations.support.ValuesSourceConfig; import org.elasticsearch.search.aggregations.support.ValuesSourceParser; import org.elasticsearch.search.internal.SearchContext; @@ -55,10 +56,10 @@ public class SamplerParser implements Aggregator.Parser { String executionHint = null; int shardSize = DEFAULT_SHARD_SAMPLE_SIZE; int maxDocsPerValue = MAX_DOCS_PER_VALUE_DEFAULT; - ValuesSourceParser vsParser = null; boolean diversityChoiceMade = false; - vsParser = ValuesSourceParser.any(aggregationName, InternalSampler.TYPE, context).scriptable(true).formattable(false).build(); + ValuesSourceParser vsParser = ValuesSourceParser.any(aggregationName, InternalSampler.TYPE, context).scriptable(true) + .formattable(false).build(); while ((token = parser.nextToken()) != XContentParser.Token.END_OBJECT) { if (token == XContentParser.Token.FIELD_NAME) { @@ -88,7 +89,7 @@ public class SamplerParser implements Aggregator.Parser { } } - ValuesSourceConfig vsConfig = vsParser.config(); + ValuesSourceConfig vsConfig = vsParser.config(); if (vsConfig.valid()) { return new SamplerAggregator.DiversifiedFactory(aggregationName, shardSize, executionHint, vsConfig, maxDocsPerValue); } else { diff --git a/core/src/main/java/org/elasticsearch/search/aggregations/bucket/significant/SignificantTermsAggregatorFactory.java b/core/src/main/java/org/elasticsearch/search/aggregations/bucket/significant/SignificantTermsAggregatorFactory.java index 399e85728af4..9b66fe01b674 100644 --- a/core/src/main/java/org/elasticsearch/search/aggregations/bucket/significant/SignificantTermsAggregatorFactory.java +++ b/core/src/main/java/org/elasticsearch/search/aggregations/bucket/significant/SignificantTermsAggregatorFactory.java @@ -20,7 +20,6 @@ package org.elasticsearch.search.aggregations.bucket.significant; import org.apache.lucene.index.IndexReader; import org.apache.lucene.index.PostingsEnum; -import org.apache.lucene.search.IndexSearcher; import org.apache.lucene.search.Query; import org.apache.lucene.util.BytesRef; import org.elasticsearch.ElasticsearchException; @@ -80,8 +79,6 @@ public class SignificantTermsAggregatorFactory extends ValuesSourceAggregatorFac TermsAggregator.BucketCountThresholds bucketCountThresholds, IncludeExclude includeExclude, AggregationContext aggregationContext, Aggregator parent, SignificantTermsAggregatorFactory termsAggregatorFactory, List pipelineAggregators, Map metaData) throws IOException { - ValuesSource.Bytes.WithOrdinals valueSourceWithOrdinals = (ValuesSource.Bytes.WithOrdinals) valuesSource; - IndexSearcher indexSearcher = aggregationContext.searchContext().searcher(); final IncludeExclude.OrdinalsFilter filter = includeExclude == null ? null : includeExclude.convertToOrdinalsFilter(); return new GlobalOrdinalsSignificantTermsAggregator(name, factories, (ValuesSource.Bytes.WithOrdinals.FieldData) valuesSource, bucketCountThresholds, filter, aggregationContext, @@ -98,9 +95,8 @@ public class SignificantTermsAggregatorFactory extends ValuesSourceAggregatorFac List pipelineAggregators, Map metaData) throws IOException { final IncludeExclude.OrdinalsFilter filter = includeExclude == null ? null : includeExclude.convertToOrdinalsFilter(); return new GlobalOrdinalsSignificantTermsAggregator.WithHash(name, factories, - (ValuesSource.Bytes.WithOrdinals.FieldData) valuesSource, bucketCountThresholds, filter, - aggregationContext, - parent, termsAggregatorFactory, pipelineAggregators, metaData); + (ValuesSource.Bytes.WithOrdinals.FieldData) valuesSource, bucketCountThresholds, filter, aggregationContext, parent, + termsAggregatorFactory, pipelineAggregators, metaData); } }; @@ -143,7 +139,7 @@ public class SignificantTermsAggregatorFactory extends ValuesSourceAggregatorFac return new TermsAggregator.BucketCountThresholds(bucketCountThresholds); } - public SignificantTermsAggregatorFactory(String name, ValuesSourceConfig valueSourceConfig, TermsAggregator.BucketCountThresholds bucketCountThresholds, IncludeExclude includeExclude, + public SignificantTermsAggregatorFactory(String name, ValuesSourceConfig valueSourceConfig, TermsAggregator.BucketCountThresholds bucketCountThresholds, IncludeExclude includeExclude, String executionHint, Query filter, SignificanceHeuristic significanceHeuristic) { super(name, SignificantStringTerms.TYPE.name(), valueSourceConfig); diff --git a/core/src/main/java/org/elasticsearch/search/aggregations/bucket/significant/SignificantTermsParser.java b/core/src/main/java/org/elasticsearch/search/aggregations/bucket/significant/SignificantTermsParser.java index 28e0fb5a8125..b4b89c29c33d 100644 --- a/core/src/main/java/org/elasticsearch/search/aggregations/bucket/significant/SignificantTermsParser.java +++ b/core/src/main/java/org/elasticsearch/search/aggregations/bucket/significant/SignificantTermsParser.java @@ -28,6 +28,7 @@ import org.elasticsearch.search.aggregations.bucket.significant.heuristics.Signi import org.elasticsearch.search.aggregations.bucket.significant.heuristics.SignificanceHeuristicParserMapper; import org.elasticsearch.search.aggregations.bucket.terms.TermsAggregator; import org.elasticsearch.search.aggregations.bucket.terms.support.IncludeExclude; +import org.elasticsearch.search.aggregations.support.ValuesSource; import org.elasticsearch.search.aggregations.support.ValuesSourceParser; import org.elasticsearch.search.internal.SearchContext; @@ -53,7 +54,7 @@ public class SignificantTermsParser implements Aggregator.Parser { @Override public AggregatorFactory parse(String aggregationName, XContentParser parser, SearchContext context) throws IOException { SignificantTermsParametersParser aggParser = new SignificantTermsParametersParser(significanceHeuristicParserMapper); - ValuesSourceParser vsParser = ValuesSourceParser.any(aggregationName, SignificantStringTerms.TYPE, context) + ValuesSourceParser vsParser = ValuesSourceParser.any(aggregationName, SignificantStringTerms.TYPE, context) .scriptable(false) .formattable(true) .build(); diff --git a/core/src/main/java/org/elasticsearch/search/aggregations/bucket/terms/AbstractTermsParametersParser.java b/core/src/main/java/org/elasticsearch/search/aggregations/bucket/terms/AbstractTermsParametersParser.java index 891526c33c10..ecd9d3bc6930 100644 --- a/core/src/main/java/org/elasticsearch/search/aggregations/bucket/terms/AbstractTermsParametersParser.java +++ b/core/src/main/java/org/elasticsearch/search/aggregations/bucket/terms/AbstractTermsParametersParser.java @@ -36,13 +36,13 @@ public abstract class AbstractTermsParametersParser { public static final ParseField SHARD_MIN_DOC_COUNT_FIELD_NAME = new ParseField("shard_min_doc_count"); public static final ParseField REQUIRED_SIZE_FIELD_NAME = new ParseField("size"); public static final ParseField SHOW_TERM_DOC_COUNT_ERROR = new ParseField("show_term_doc_count_error"); - + //These are the results of the parsing. private TermsAggregator.BucketCountThresholds bucketCountThresholds = new TermsAggregator.BucketCountThresholds(); private String executionHint = null; - + private SubAggCollectionMode collectMode = SubAggCollectionMode.DEPTH_FIRST; @@ -59,12 +59,12 @@ public abstract class AbstractTermsParametersParser { public IncludeExclude getIncludeExclude() { return includeExclude; } - + public SubAggCollectionMode getCollectionMode() { return collectMode; } - public void parse(String aggregationName, XContentParser parser, SearchContext context, ValuesSourceParser vsParser, IncludeExclude.Parser incExcParser) throws IOException { + public void parse(String aggregationName, XContentParser parser, SearchContext context, ValuesSourceParser vsParser, IncludeExclude.Parser incExcParser) throws IOException { bucketCountThresholds = getDefaultBucketCountThresholds(); XContentParser.Token token; String currentFieldName = null; diff --git a/core/src/main/java/org/elasticsearch/search/aggregations/bucket/terms/TermsAggregatorFactory.java b/core/src/main/java/org/elasticsearch/search/aggregations/bucket/terms/TermsAggregatorFactory.java index 270dc009af2d..04f7adf27b71 100644 --- a/core/src/main/java/org/elasticsearch/search/aggregations/bucket/terms/TermsAggregatorFactory.java +++ b/core/src/main/java/org/elasticsearch/search/aggregations/bucket/terms/TermsAggregatorFactory.java @@ -165,7 +165,7 @@ public class TermsAggregatorFactory extends ValuesSourceAggregatorFactory config, Terms.Order order, TermsAggregator.BucketCountThresholds bucketCountThresholds, IncludeExclude includeExclude, String executionHint, SubAggCollectionMode executionMode, boolean showTermDocCountError) { super(name, StringTerms.TYPE.name(), config); diff --git a/core/src/main/java/org/elasticsearch/search/aggregations/bucket/terms/TermsParser.java b/core/src/main/java/org/elasticsearch/search/aggregations/bucket/terms/TermsParser.java index 478309d1bc04..a7b60e930ff0 100644 --- a/core/src/main/java/org/elasticsearch/search/aggregations/bucket/terms/TermsParser.java +++ b/core/src/main/java/org/elasticsearch/search/aggregations/bucket/terms/TermsParser.java @@ -25,6 +25,7 @@ import org.elasticsearch.search.aggregations.bucket.BucketUtils; import org.elasticsearch.search.aggregations.bucket.terms.Terms.Order; import org.elasticsearch.search.aggregations.bucket.terms.TermsParametersParser.OrderElement; import org.elasticsearch.search.aggregations.bucket.terms.support.IncludeExclude; +import org.elasticsearch.search.aggregations.support.ValuesSource; import org.elasticsearch.search.aggregations.support.ValuesSourceParser; import org.elasticsearch.search.internal.SearchContext; @@ -45,7 +46,8 @@ public class TermsParser implements Aggregator.Parser { @Override public AggregatorFactory parse(String aggregationName, XContentParser parser, SearchContext context) throws IOException { TermsParametersParser aggParser = new TermsParametersParser(); - ValuesSourceParser vsParser = ValuesSourceParser.any(aggregationName, StringTerms.TYPE, context).scriptable(true).formattable(true).build(); + ValuesSourceParser vsParser = ValuesSourceParser.any(aggregationName, StringTerms.TYPE, context).scriptable(true) + .formattable(true).build(); IncludeExclude.Parser incExcParser = new IncludeExclude.Parser(); aggParser.parse(aggregationName, parser, context, vsParser, incExcParser); diff --git a/core/src/main/java/org/elasticsearch/search/aggregations/metrics/ValuesSourceMetricsAggregationBuilder.java b/core/src/main/java/org/elasticsearch/search/aggregations/metrics/ValuesSourceMetricsAggregationBuilder.java index e67554862258..81933144a5a6 100644 --- a/core/src/main/java/org/elasticsearch/search/aggregations/metrics/ValuesSourceMetricsAggregationBuilder.java +++ b/core/src/main/java/org/elasticsearch/search/aggregations/metrics/ValuesSourceMetricsAggregationBuilder.java @@ -62,6 +62,7 @@ public abstract class ValuesSourceMetricsAggregationBuilder config, long precisionThreshold) { super(name, InternalCardinality.TYPE.name(), config); this.precisionThreshold = precisionThreshold; } diff --git a/core/src/main/java/org/elasticsearch/search/aggregations/metrics/cardinality/CardinalityParser.java b/core/src/main/java/org/elasticsearch/search/aggregations/metrics/cardinality/CardinalityParser.java index 68339457fe79..315523264505 100644 --- a/core/src/main/java/org/elasticsearch/search/aggregations/metrics/cardinality/CardinalityParser.java +++ b/core/src/main/java/org/elasticsearch/search/aggregations/metrics/cardinality/CardinalityParser.java @@ -24,6 +24,7 @@ import org.elasticsearch.common.xcontent.XContentParser; import org.elasticsearch.search.SearchParseException; import org.elasticsearch.search.aggregations.Aggregator; import org.elasticsearch.search.aggregations.AggregatorFactory; +import org.elasticsearch.search.aggregations.support.ValuesSource; import org.elasticsearch.search.aggregations.support.ValuesSourceParser; import org.elasticsearch.search.internal.SearchContext; @@ -43,7 +44,7 @@ public class CardinalityParser implements Aggregator.Parser { @Override public AggregatorFactory parse(String name, XContentParser parser, SearchContext context) throws IOException { - ValuesSourceParser vsParser = ValuesSourceParser.any(name, InternalCardinality.TYPE, context).formattable(false).build(); + ValuesSourceParser vsParser = ValuesSourceParser.any(name, InternalCardinality.TYPE, context).formattable(false).build(); long precisionThreshold = -1; diff --git a/core/src/main/java/org/elasticsearch/search/aggregations/metrics/valuecount/ValueCountParser.java b/core/src/main/java/org/elasticsearch/search/aggregations/metrics/valuecount/ValueCountParser.java index 764f6ce9384e..0a9ea4a90723 100644 --- a/core/src/main/java/org/elasticsearch/search/aggregations/metrics/valuecount/ValueCountParser.java +++ b/core/src/main/java/org/elasticsearch/search/aggregations/metrics/valuecount/ValueCountParser.java @@ -40,7 +40,7 @@ public class ValueCountParser implements Aggregator.Parser { @Override public AggregatorFactory parse(String aggregationName, XContentParser parser, SearchContext context) throws IOException { - ValuesSourceParser vsParser = ValuesSourceParser.any(aggregationName, InternalValueCount.TYPE, context) + ValuesSourceParser vsParser = ValuesSourceParser.any(aggregationName, InternalValueCount.TYPE, context) .build(); XContentParser.Token token; @@ -54,6 +54,6 @@ public class ValueCountParser implements Aggregator.Parser { } } - return new ValueCountAggregator.Factory(aggregationName, vsParser.config()); + return new ValueCountAggregator.Factory<>(aggregationName, vsParser.config()); } } diff --git a/core/src/main/java/org/elasticsearch/search/aggregations/support/ValuesSource.java b/core/src/main/java/org/elasticsearch/search/aggregations/support/ValuesSource.java index b03bc8d68331..d9fe3ad66c02 100644 --- a/core/src/main/java/org/elasticsearch/search/aggregations/support/ValuesSource.java +++ b/core/src/main/java/org/elasticsearch/search/aggregations/support/ValuesSource.java @@ -53,6 +53,9 @@ import org.elasticsearch.search.aggregations.support.values.ScriptLongValues; import java.io.IOException; +/** + * How to load values for an aggregation. + */ public abstract class ValuesSource { /** @@ -528,6 +531,7 @@ public abstract class ValuesSource { return indexFieldData.load(context).getBytesValues(); } + @Override public org.elasticsearch.index.fielddata.MultiGeoPointValues geoPointValues(LeafReaderContext context) { return indexFieldData.load(context).getGeoPointValues(); } diff --git a/core/src/main/java/org/elasticsearch/search/aggregations/support/ValuesSourceAggregatorFactory.java b/core/src/main/java/org/elasticsearch/search/aggregations/support/ValuesSourceAggregatorFactory.java index d0eaec2d8bc3..3f56162a2f2d 100644 --- a/core/src/main/java/org/elasticsearch/search/aggregations/support/ValuesSourceAggregatorFactory.java +++ b/core/src/main/java/org/elasticsearch/search/aggregations/support/ValuesSourceAggregatorFactory.java @@ -78,19 +78,20 @@ public abstract class ValuesSourceAggregatorFactory ext boolean collectsFromSingleBucket, List pipelineAggregators, Map metaData) throws IOException; + @SuppressWarnings("unchecked") // Safe because we check the types with isAssignableFrom private void resolveValuesSourceConfigFromAncestors(String aggName, AggregatorFactory parent, Class requiredValuesSourceType) { - ValuesSourceConfig config; + ValuesSourceConfig config; while (parent != null) { if (parent instanceof ValuesSourceAggregatorFactory) { - config = ((ValuesSourceAggregatorFactory) parent).config; + config = ((ValuesSourceAggregatorFactory) parent).config; if (config != null && config.valid()) { if (requiredValuesSourceType == null || requiredValuesSourceType.isAssignableFrom(config.valueSourceType)) { ValueFormat format = config.format; - this.config = config; + this.config = (ValuesSourceConfig) config; // if the user explicitly defined a format pattern, we'll do our best to keep it even when we inherit the // value source form one of the ancestor aggregations if (this.config.formatPattern != null && format != null && format instanceof ValueFormat.Patternable) { - this.config.format = ((ValueFormat.Patternable) format).create(this.config.formatPattern); + this.config.format = ((ValueFormat.Patternable) format).create(this.config.formatPattern); } return; } diff --git a/core/src/main/java/org/elasticsearch/search/aggregations/support/ValuesSourceParser.java b/core/src/main/java/org/elasticsearch/search/aggregations/support/ValuesSourceParser.java index fced5fdc913c..7c2606191731 100644 --- a/core/src/main/java/org/elasticsearch/search/aggregations/support/ValuesSourceParser.java +++ b/core/src/main/java/org/elasticsearch/search/aggregations/support/ValuesSourceParser.java @@ -48,13 +48,16 @@ import java.util.HashMap; import java.util.Map; /** - * + * Parses a description of where to load the value sent by a user into a + * ValuesSourceConfig which can be used to work with the values in various ways, + * one of which is to create an actual ValueSource (done with the help of + * AggregationContext). */ public class ValuesSourceParser { static final ParseField TIME_ZONE = new ParseField("time_zone"); - public static Builder any(String aggName, InternalAggregation.Type aggType, SearchContext context) { + public static Builder any(String aggName, InternalAggregation.Type aggType, SearchContext context) { return new Builder<>(aggName, aggType, context, ValuesSource.class); } diff --git a/core/src/main/java/org/elasticsearch/search/profile/CollectorResult.java b/core/src/main/java/org/elasticsearch/search/profile/CollectorResult.java index 8da14d23d966..d0c006eb5dc5 100644 --- a/core/src/main/java/org/elasticsearch/search/profile/CollectorResult.java +++ b/core/src/main/java/org/elasticsearch/search/profile/CollectorResult.java @@ -36,7 +36,7 @@ import java.util.Locale; * Collectors used in the search. Children CollectorResult's may be * embedded inside of a parent CollectorResult */ -public class CollectorResult implements ToXContent, Writeable { +public class CollectorResult implements ToXContent, Writeable { public static final String REASON_SEARCH_COUNT = "search_count"; public static final String REASON_SEARCH_TOP_HITS = "search_top_hits"; @@ -125,7 +125,7 @@ public class CollectorResult implements ToXContent, Writeable { builder = builder.startObject() .field(NAME.getPreferredName(), getName()) .field(REASON.getPreferredName(), getReason()) - .field(TIME.getPreferredName(), String.format(Locale.US, "%.10gms", (double) (getTime() / 1000000.0))); + .field(TIME.getPreferredName(), String.format(Locale.US, "%.10gms", getTime() / 1000000.0)); if (!children.isEmpty()) { builder = builder.startArray(CHILDREN.getPreferredName()); @@ -150,7 +150,7 @@ public class CollectorResult implements ToXContent, Writeable { } @Override - public Object readFrom(StreamInput in) throws IOException { + public CollectorResult readFrom(StreamInput in) throws IOException { return new CollectorResult(in); } } diff --git a/core/src/test/java/org/elasticsearch/action/admin/cluster/node/tasks/TransportTasksActionTests.java b/core/src/test/java/org/elasticsearch/action/admin/cluster/node/tasks/TransportTasksActionTests.java index 55c10aa298e2..d8d4f2656c89 100644 --- a/core/src/test/java/org/elasticsearch/action/admin/cluster/node/tasks/TransportTasksActionTests.java +++ b/core/src/test/java/org/elasticsearch/action/admin/cluster/node/tasks/TransportTasksActionTests.java @@ -20,6 +20,7 @@ package org.elasticsearch.action.admin.cluster.node.tasks; import org.elasticsearch.Version; import org.elasticsearch.action.ActionFuture; +import org.elasticsearch.action.ActionListener; import org.elasticsearch.action.FailedNodeException; import org.elasticsearch.action.TaskOperationFailure; import org.elasticsearch.action.admin.cluster.node.tasks.list.ListTasksRequest; @@ -27,6 +28,7 @@ import org.elasticsearch.action.admin.cluster.node.tasks.list.ListTasksResponse; import org.elasticsearch.action.admin.cluster.node.tasks.list.TaskInfo; import org.elasticsearch.action.admin.cluster.node.tasks.list.TransportListTasksAction; import org.elasticsearch.action.support.ActionFilters; +import org.elasticsearch.action.support.PlainActionFuture; import org.elasticsearch.action.support.nodes.BaseNodeRequest; import org.elasticsearch.action.support.nodes.BaseNodeResponse; import org.elasticsearch.action.support.nodes.BaseNodesRequest; @@ -67,8 +69,10 @@ import java.util.Map; import java.util.concurrent.CountDownLatch; import java.util.concurrent.ExecutionException; import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicReference; import java.util.concurrent.atomic.AtomicReferenceArray; +import static org.elasticsearch.action.support.PlainActionFuture.newFuture; import static org.hamcrest.Matchers.containsString; import static org.hamcrest.Matchers.endsWith; import static org.hamcrest.Matchers.not; @@ -418,7 +422,17 @@ public class TransportTasksActionTests extends ESTestCase { return startBlockingTestNodesAction(checkLatch, new NodesRequest("Test Request")); } - private ActionFuture startBlockingTestNodesAction(CountDownLatch checkLatch, NodesRequest request) throws InterruptedException { + private ActionFuture startBlockingTestNodesAction(CountDownLatch checkLatch, NodesRequest request) throws InterruptedException { + PlainActionFuture future = newFuture(); + startBlockingTestNodesAction(checkLatch, request, future); + return future; + } + + private Task startBlockingTestNodesAction(CountDownLatch checkLatch, ActionListener listener) throws InterruptedException { + return startBlockingTestNodesAction(checkLatch, new NodesRequest("Test Request"), listener); + } + + private Task startBlockingTestNodesAction(CountDownLatch checkLatch, NodesRequest request, ActionListener listener) throws InterruptedException { CountDownLatch actionLatch = new CountDownLatch(nodesCount); TestNodesAction[] actions = new TestNodesAction[nodesCount]; for (int i = 0; i < testNodes.length; i++) { @@ -442,17 +456,31 @@ public class TransportTasksActionTests extends ESTestCase { for (TestNode node : testNodes) { assertEquals(0, node.transportService.getTaskManager().getTasks().size()); } - ActionFuture future = actions[0].execute(request); + Task task = actions[0].execute(request, listener); logger.info("Awaiting for all actions to start"); actionLatch.await(); logger.info("Done waiting for all actions to start"); - return future; + return task; } public void testRunningTasksCount() throws Exception { connectNodes(testNodes); CountDownLatch checkLatch = new CountDownLatch(1); - ActionFuture future = startBlockingTestNodesAction(checkLatch); + CountDownLatch responseLatch = new CountDownLatch(1); + final AtomicReference responseReference = new AtomicReference<>(); + Task mainTask = startBlockingTestNodesAction(checkLatch, new ActionListener() { + @Override + public void onResponse(NodesResponse listTasksResponse) { + responseReference.set(listTasksResponse); + responseLatch.countDown(); + } + + @Override + public void onFailure(Throwable e) { + logger.warn("Couldn't get list of tasks", e); + responseLatch.countDown(); + } + }); // Check task counts using taskManager Map localTasks = testNodes[0].transportService.getTaskManager().getTasks(); @@ -505,9 +533,17 @@ public class TransportTasksActionTests extends ESTestCase { assertEquals("NodeRequest[Test Request, true]", entry.getValue().get(0).getDescription()); } + // Make sure that the main task on coordinating node is the task that was returned to us by execute() + listTasksRequest.actions("testAction"); // only pick the main task + response = testNode.transportListTasksAction.execute(listTasksRequest).get(); + assertEquals(1, response.getTasks().size()); + assertEquals(mainTask.getId(), response.getTasks().get(0).getId()); + // Release all tasks and wait for response checkLatch.countDown(); - NodesResponse responses = future.get(); + assertTrue(responseLatch.await(10, TimeUnit.SECONDS)); + + NodesResponse responses = responseReference.get(); assertEquals(0, responses.failureCount()); // Make sure that we don't have any lingering tasks diff --git a/core/src/test/java/org/elasticsearch/action/bulk/BulkRequestTests.java b/core/src/test/java/org/elasticsearch/action/bulk/BulkRequestTests.java index 81eb832be9a6..66fb8aa3f21e 100644 --- a/core/src/test/java/org/elasticsearch/action/bulk/BulkRequestTests.java +++ b/core/src/test/java/org/elasticsearch/action/bulk/BulkRequestTests.java @@ -110,7 +110,7 @@ public class BulkRequestTests extends ESTestCase { public void testBulkAddIterable() { BulkRequest bulkRequest = Requests.bulkRequest(); - List requests = new ArrayList<>(); + List> requests = new ArrayList<>(); requests.add(new IndexRequest("test", "test", "id").source("field", "value")); requests.add(new UpdateRequest("test", "test", "id").doc("field", "value")); requests.add(new DeleteRequest("test", "test", "id")); diff --git a/core/src/test/java/org/elasticsearch/action/search/MultiSearchRequestTests.java b/core/src/test/java/org/elasticsearch/action/search/MultiSearchRequestTests.java index ee0ceef17213..7301d37488b6 100644 --- a/core/src/test/java/org/elasticsearch/action/search/MultiSearchRequestTests.java +++ b/core/src/test/java/org/elasticsearch/action/search/MultiSearchRequestTests.java @@ -19,10 +19,11 @@ package org.elasticsearch.action.search; +import java.io.IOException; + import org.elasticsearch.action.support.IndicesOptions; import org.elasticsearch.common.ParseFieldMatcher; import org.elasticsearch.common.bytes.BytesArray; -import org.elasticsearch.common.io.stream.NamedWriteableRegistry; import org.elasticsearch.common.settings.Settings; import org.elasticsearch.common.xcontent.ToXContent; import org.elasticsearch.common.xcontent.XContentBuilder; @@ -34,15 +35,14 @@ import org.elasticsearch.script.ScriptService; import org.elasticsearch.test.ESTestCase; import org.elasticsearch.test.StreamsUtils; -import java.io.IOException; -import java.util.Collections; +import static java.util.Collections.singletonMap; import static org.hamcrest.Matchers.equalTo; import static org.hamcrest.Matchers.nullValue; public class MultiSearchRequestTests extends ESTestCase { public void testSimpleAdd() throws Exception { - IndicesQueriesRegistry registry = new IndicesQueriesRegistry(Settings.EMPTY, Collections.singleton(new MatchAllQueryParser()), new NamedWriteableRegistry()); + IndicesQueriesRegistry registry = new IndicesQueriesRegistry(Settings.EMPTY, singletonMap("match_all", new MatchAllQueryParser())); byte[] data = StreamsUtils.copyToBytesFromClasspath("/org/elasticsearch/action/search/simple-msearch1.json"); MultiSearchRequest request = RestMultiSearchAction.parseRequest(new MultiSearchRequest(), new BytesArray(data), false, null, null, null, null, IndicesOptions.strictExpandOpenAndForbidClosed(), true, registry, ParseFieldMatcher.EMPTY); @@ -69,7 +69,7 @@ public class MultiSearchRequestTests extends ESTestCase { } public void testSimpleAdd2() throws Exception { - IndicesQueriesRegistry registry = new IndicesQueriesRegistry(Settings.EMPTY, Collections.singleton(new MatchAllQueryParser()), new NamedWriteableRegistry()); + IndicesQueriesRegistry registry = new IndicesQueriesRegistry(Settings.EMPTY, singletonMap("match_all", new MatchAllQueryParser())); byte[] data = StreamsUtils.copyToBytesFromClasspath("/org/elasticsearch/action/search/simple-msearch2.json"); MultiSearchRequest request = RestMultiSearchAction.parseRequest(new MultiSearchRequest(), new BytesArray(data), false, null, null, null, null, IndicesOptions.strictExpandOpenAndForbidClosed(), true, registry, ParseFieldMatcher.EMPTY); @@ -88,7 +88,7 @@ public class MultiSearchRequestTests extends ESTestCase { } public void testSimpleAdd3() throws Exception { - IndicesQueriesRegistry registry = new IndicesQueriesRegistry(Settings.EMPTY, Collections.singleton(new MatchAllQueryParser()), new NamedWriteableRegistry()); + IndicesQueriesRegistry registry = new IndicesQueriesRegistry(Settings.EMPTY, singletonMap("match_all", new MatchAllQueryParser())); byte[] data = StreamsUtils.copyToBytesFromClasspath("/org/elasticsearch/action/search/simple-msearch3.json"); MultiSearchRequest request = RestMultiSearchAction.parseRequest(new MultiSearchRequest(), new BytesArray(data), false, null, null, null, null, IndicesOptions.strictExpandOpenAndForbidClosed(), true, registry, ParseFieldMatcher.EMPTY); @@ -108,7 +108,7 @@ public class MultiSearchRequestTests extends ESTestCase { } public void testSimpleAdd4() throws Exception { - IndicesQueriesRegistry registry = new IndicesQueriesRegistry(Settings.EMPTY, Collections.singleton(new MatchAllQueryParser()), new NamedWriteableRegistry()); + IndicesQueriesRegistry registry = new IndicesQueriesRegistry(Settings.EMPTY, singletonMap("match_all", new MatchAllQueryParser())); byte[] data = StreamsUtils.copyToBytesFromClasspath("/org/elasticsearch/action/search/simple-msearch4.json"); MultiSearchRequest request = RestMultiSearchAction.parseRequest(new MultiSearchRequest(), new BytesArray(data), false, null, null, null, null, IndicesOptions.strictExpandOpenAndForbidClosed(), true, registry, ParseFieldMatcher.EMPTY); @@ -130,7 +130,7 @@ public class MultiSearchRequestTests extends ESTestCase { } public void testSimpleAdd5() throws Exception { - IndicesQueriesRegistry registry = new IndicesQueriesRegistry(Settings.EMPTY, Collections.singleton(new MatchAllQueryParser()), new NamedWriteableRegistry()); + IndicesQueriesRegistry registry = new IndicesQueriesRegistry(Settings.EMPTY, singletonMap("match_all", new MatchAllQueryParser())); byte[] data = StreamsUtils.copyToBytesFromClasspath("/org/elasticsearch/action/search/simple-msearch5.json"); MultiSearchRequest request = RestMultiSearchAction.parseRequest(new MultiSearchRequest(), new BytesArray(data), true, null, null, null, null, IndicesOptions.strictExpandOpenAndForbidClosed(), true, registry, ParseFieldMatcher.EMPTY); diff --git a/core/src/test/java/org/elasticsearch/action/support/TransportActionFilterChainTests.java b/core/src/test/java/org/elasticsearch/action/support/TransportActionFilterChainTests.java index 6a14989be1ad..fed4e1d6384d 100644 --- a/core/src/test/java/org/elasticsearch/action/support/TransportActionFilterChainTests.java +++ b/core/src/test/java/org/elasticsearch/action/support/TransportActionFilterChainTests.java @@ -442,7 +442,7 @@ public class TransportActionFilterChainTests extends ESTestCase { void execute(String action, ActionResponse response, ActionListener listener, ActionFilterChain chain); } - public static class TestRequest extends ActionRequest { + public static class TestRequest extends ActionRequest { @Override public ActionRequestValidationException validate() { return null; diff --git a/core/src/test/java/org/elasticsearch/action/support/replication/BroadcastReplicationTests.java b/core/src/test/java/org/elasticsearch/action/support/replication/BroadcastReplicationTests.java index d1abe8653f0d..c3084b93eb85 100644 --- a/core/src/test/java/org/elasticsearch/action/support/replication/BroadcastReplicationTests.java +++ b/core/src/test/java/org/elasticsearch/action/support/replication/BroadcastReplicationTests.java @@ -22,6 +22,7 @@ import org.elasticsearch.Version; import org.elasticsearch.action.ActionListener; import org.elasticsearch.action.NoShardAvailableActionException; import org.elasticsearch.action.ReplicationResponse; +import org.elasticsearch.action.ShardOperationFailedException; import org.elasticsearch.action.UnavailableShardsException; import org.elasticsearch.action.admin.indices.flush.FlushRequest; import org.elasticsearch.action.admin.indices.flush.FlushResponse; @@ -100,7 +101,7 @@ public class BroadcastReplicationTests extends ESTestCase { clusterService.setState(state(index, randomBoolean(), randomBoolean() ? ShardRoutingState.INITIALIZING : ShardRoutingState.UNASSIGNED, ShardRoutingState.UNASSIGNED)); logger.debug("--> using initial state:\n{}", clusterService.state().prettyPrint()); - Future response = (broadcastReplicationAction.execute(new BroadcastRequest().indices(index))); + Future response = (broadcastReplicationAction.execute(new DummyBroadcastRequest().indices(index))); for (Tuple> shardRequests : broadcastReplicationAction.capturedShardRequests) { if (randomBoolean()) { shardRequests.v2().onFailure(new NoShardAvailableActionException(shardRequests.v1())); @@ -119,7 +120,7 @@ public class BroadcastReplicationTests extends ESTestCase { clusterService.setState(state(index, randomBoolean(), ShardRoutingState.STARTED)); logger.debug("--> using initial state:\n{}", clusterService.state().prettyPrint()); - Future response = (broadcastReplicationAction.execute(new BroadcastRequest().indices(index))); + Future response = (broadcastReplicationAction.execute(new DummyBroadcastRequest().indices(index))); for (Tuple> shardRequests : broadcastReplicationAction.capturedShardRequests) { ReplicationResponse replicationResponse = new ReplicationResponse(); replicationResponse.setShardInfo(new ReplicationResponse.ShardInfo(1, 1, new ReplicationResponse.ShardInfo.Failure[0])); @@ -134,7 +135,7 @@ public class BroadcastReplicationTests extends ESTestCase { int numShards = randomInt(3); clusterService.setState(stateWithAssignedPrimariesAndOneReplica(index, numShards)); logger.debug("--> using initial state:\n{}", clusterService.state().prettyPrint()); - Future response = (broadcastReplicationAction.execute(new BroadcastRequest().indices(index))); + Future response = (broadcastReplicationAction.execute(new DummyBroadcastRequest().indices(index))); int succeeded = 0; int failed = 0; for (Tuple> shardRequests : broadcastReplicationAction.capturedShardRequests) { @@ -164,7 +165,7 @@ public class BroadcastReplicationTests extends ESTestCase { public void testNoShards() throws InterruptedException, ExecutionException, IOException { clusterService.setState(stateWithNoShard()); logger.debug("--> using initial state:\n{}", clusterService.state().prettyPrint()); - BroadcastResponse response = executeAndAssertImmediateResponse(broadcastReplicationAction, new BroadcastRequest()); + BroadcastResponse response = executeAndAssertImmediateResponse(broadcastReplicationAction, new DummyBroadcastRequest()); assertBroadcastResponse(0, 0, 0, response, null); } @@ -174,16 +175,19 @@ public class BroadcastReplicationTests extends ESTestCase { ClusterState clusterState = state(index, randomBoolean(), randomBoolean() ? ShardRoutingState.INITIALIZING : ShardRoutingState.UNASSIGNED, ShardRoutingState.UNASSIGNED); logger.debug("--> using initial state:\n{}", clusterService.state().prettyPrint()); - List shards = broadcastReplicationAction.shards(new BroadcastRequest().indices(shardId.index().name()), clusterState); + List shards = broadcastReplicationAction.shards(new DummyBroadcastRequest().indices(shardId.index().name()), clusterState); assertThat(shards.size(), equalTo(1)); assertThat(shards.get(0), equalTo(shardId)); } - private class TestBroadcastReplicationAction extends TransportBroadcastReplicationAction { + private class TestBroadcastReplicationAction extends TransportBroadcastReplicationAction { protected final Set>> capturedShardRequests = ConcurrentCollections.newConcurrentSet(); - public TestBroadcastReplicationAction(Settings settings, ThreadPool threadPool, ClusterService clusterService, TransportService transportService, ActionFilters actionFilters, IndexNameExpressionResolver indexNameExpressionResolver, TransportReplicationAction replicatedBroadcastShardAction) { - super("test-broadcast-replication-action", BroadcastRequest::new, settings, threadPool, clusterService, transportService, actionFilters, indexNameExpressionResolver, replicatedBroadcastShardAction); + public TestBroadcastReplicationAction(Settings settings, ThreadPool threadPool, ClusterService clusterService, + TransportService transportService, ActionFilters actionFilters, IndexNameExpressionResolver indexNameExpressionResolver, + TransportReplicationAction replicatedBroadcastShardAction) { + super("test-broadcast-replication-action", DummyBroadcastRequest::new, settings, threadPool, clusterService, transportService, + actionFilters, indexNameExpressionResolver, replicatedBroadcastShardAction); } @Override @@ -192,17 +196,18 @@ public class BroadcastReplicationTests extends ESTestCase { } @Override - protected ReplicationRequest newShardRequest(BroadcastRequest request, ShardId shardId) { - return new ReplicationRequest().setShardId(shardId); + protected BasicReplicationRequest newShardRequest(DummyBroadcastRequest request, ShardId shardId) { + return new BasicReplicationRequest().setShardId(shardId); } @Override - protected BroadcastResponse newResponse(int successfulShards, int failedShards, int totalNumCopies, List shardFailures) { + protected BroadcastResponse newResponse(int successfulShards, int failedShards, int totalNumCopies, + List shardFailures) { return new BroadcastResponse(totalNumCopies, successfulShards, failedShards, shardFailures); } @Override - protected void shardExecute(BroadcastRequest request, ShardId shardId, ActionListener shardActionListener) { + protected void shardExecute(DummyBroadcastRequest request, ShardId shardId, ActionListener shardActionListener) { capturedShardRequests.add(new Tuple<>(shardId, shardActionListener)); } } @@ -216,7 +221,7 @@ public class BroadcastReplicationTests extends ESTestCase { return flushResponse; } - public BroadcastResponse executeAndAssertImmediateResponse(TransportBroadcastReplicationAction broadcastAction, BroadcastRequest request) throws InterruptedException, ExecutionException { + public BroadcastResponse executeAndAssertImmediateResponse(TransportBroadcastReplicationAction broadcastAction, DummyBroadcastRequest request) throws InterruptedException, ExecutionException { return (BroadcastResponse) broadcastAction.execute(request).actionGet("5s"); } @@ -228,4 +233,8 @@ public class BroadcastReplicationTests extends ESTestCase { assertThat(response.getShardFailures()[0].getCause().getCause(), instanceOf(exceptionClass)); } } + + public static class DummyBroadcastRequest extends BroadcastRequest { + + } } diff --git a/core/src/test/java/org/elasticsearch/common/geo/builders/AbstractShapeBuilderTestCase.java b/core/src/test/java/org/elasticsearch/common/geo/builders/AbstractShapeBuilderTestCase.java index 03294b8e49ce..279e31aadd44 100644 --- a/core/src/test/java/org/elasticsearch/common/geo/builders/AbstractShapeBuilderTestCase.java +++ b/core/src/test/java/org/elasticsearch/common/geo/builders/AbstractShapeBuilderTestCase.java @@ -40,7 +40,7 @@ import static org.hamcrest.Matchers.not; public abstract class AbstractShapeBuilderTestCase extends ESTestCase { - private static final int NUMBER_OF_TESTBUILDERS = 100; + private static final int NUMBER_OF_TESTBUILDERS = 20; private static NamedWriteableRegistry namedWriteableRegistry; /** diff --git a/core/src/test/java/org/elasticsearch/common/lucene/ShardCoreKeyMapTests.java b/core/src/test/java/org/elasticsearch/common/lucene/ShardCoreKeyMapTests.java index 61660f967425..0c14e1a0bcba 100644 --- a/core/src/test/java/org/elasticsearch/common/lucene/ShardCoreKeyMapTests.java +++ b/core/src/test/java/org/elasticsearch/common/lucene/ShardCoreKeyMapTests.java @@ -22,8 +22,10 @@ package org.elasticsearch.common.lucene; import org.apache.lucene.document.Document; import org.apache.lucene.index.DirectoryReader; import org.apache.lucene.index.IndexReader; +import org.apache.lucene.index.LeafReader; import org.apache.lucene.index.LeafReaderContext; import org.apache.lucene.index.RandomIndexWriter; +import org.apache.lucene.store.AlreadyClosedException; import org.apache.lucene.store.Directory; import org.elasticsearch.common.lucene.index.ElasticsearchDirectoryReader; import org.elasticsearch.index.shard.ShardId; @@ -55,6 +57,25 @@ public class ShardCoreKeyMapTests extends ESTestCase { } } + public void testAddingAClosedReader() throws Exception { + LeafReader reader; + try (Directory dir = newDirectory(); + RandomIndexWriter writer = new RandomIndexWriter(random(), dir)) { + writer.addDocument(new Document()); + try (DirectoryReader dirReader = ElasticsearchDirectoryReader.wrap(writer.getReader(), new ShardId("index1", 1))) { + reader = dirReader.leaves().get(0).reader(); + } + } + ShardCoreKeyMap map = new ShardCoreKeyMap(); + try { + map.add(reader); + fail("Expected AlreadyClosedException"); + } catch (AlreadyClosedException e) { + // What we wanted + } + assertEquals(0, map.size()); + } + public void testBasics() throws IOException { Directory dir1 = newDirectory(); RandomIndexWriter w1 = new RandomIndexWriter(random(), dir1); diff --git a/core/src/test/java/org/elasticsearch/common/network/NetworkModuleTests.java b/core/src/test/java/org/elasticsearch/common/network/NetworkModuleTests.java index 798e82a979e2..82cabf7ec583 100644 --- a/core/src/test/java/org/elasticsearch/common/network/NetworkModuleTests.java +++ b/core/src/test/java/org/elasticsearch/common/network/NetworkModuleTests.java @@ -104,36 +104,36 @@ public class NetworkModuleTests extends ModuleTestCase { public void testRegisterTransportService() { Settings settings = Settings.builder().put(NetworkModule.TRANSPORT_SERVICE_TYPE_KEY, "custom").build(); - NetworkModule module = new NetworkModule(new NetworkService(settings), settings, false); + NetworkModule module = new NetworkModule(new NetworkService(settings), settings, false, null); module.registerTransportService("custom", FakeTransportService.class); assertBinding(module, TransportService.class, FakeTransportService.class); // check it works with transport only as well - module = new NetworkModule(new NetworkService(settings), settings, true); + module = new NetworkModule(new NetworkService(settings), settings, true, null); module.registerTransportService("custom", FakeTransportService.class); assertBinding(module, TransportService.class, FakeTransportService.class); } public void testRegisterTransport() { Settings settings = Settings.builder().put(NetworkModule.TRANSPORT_TYPE_KEY, "custom").build(); - NetworkModule module = new NetworkModule(new NetworkService(settings), settings, false); + NetworkModule module = new NetworkModule(new NetworkService(settings), settings, false, null); module.registerTransport("custom", FakeTransport.class); assertBinding(module, Transport.class, FakeTransport.class); // check it works with transport only as well - module = new NetworkModule(new NetworkService(settings), settings, true); + module = new NetworkModule(new NetworkService(settings), settings, true, null); module.registerTransport("custom", FakeTransport.class); assertBinding(module, Transport.class, FakeTransport.class); } public void testRegisterHttpTransport() { Settings settings = Settings.builder().put(NetworkModule.HTTP_TYPE_KEY, "custom").build(); - NetworkModule module = new NetworkModule(new NetworkService(settings), settings, false); + NetworkModule module = new NetworkModule(new NetworkService(settings), settings, false, null); module.registerHttpTransport("custom", FakeHttpTransport.class); assertBinding(module, HttpServerTransport.class, FakeHttpTransport.class); // check registration not allowed for transport only - module = new NetworkModule(new NetworkService(settings), settings, true); + module = new NetworkModule(new NetworkService(settings), settings, true, null); try { module.registerHttpTransport("custom", FakeHttpTransport.class); fail(); @@ -144,19 +144,19 @@ public class NetworkModuleTests extends ModuleTestCase { // not added if http is disabled settings = Settings.builder().put(NetworkModule.HTTP_ENABLED, false).build(); - module = new NetworkModule(new NetworkService(settings), settings, false); + module = new NetworkModule(new NetworkService(settings), settings, false, null); assertNotBound(module, HttpServerTransport.class); } public void testRegisterRestHandler() { Settings settings = Settings.EMPTY; - NetworkModule module = new NetworkModule(new NetworkService(settings), settings, false); + NetworkModule module = new NetworkModule(new NetworkService(settings), settings, false, null); module.registerRestHandler(FakeRestHandler.class); // also check a builtin is bound assertSetMultiBinding(module, RestHandler.class, FakeRestHandler.class, RestMainAction.class); // check registration not allowed for transport only - module = new NetworkModule(new NetworkService(settings), settings, true); + module = new NetworkModule(new NetworkService(settings), settings, true, null); try { module.registerRestHandler(FakeRestHandler.class); fail(); @@ -168,7 +168,7 @@ public class NetworkModuleTests extends ModuleTestCase { public void testRegisterCatRestHandler() { Settings settings = Settings.EMPTY; - NetworkModule module = new NetworkModule(new NetworkService(settings), settings, false); + NetworkModule module = new NetworkModule(new NetworkService(settings), settings, false, null); module.registerRestHandler(FakeCatRestHandler.class); // also check a builtin is bound assertSetMultiBinding(module, AbstractCatAction.class, FakeCatRestHandler.class, RestNodesAction.class); diff --git a/core/src/test/java/org/elasticsearch/common/settings/SettingTests.java b/core/src/test/java/org/elasticsearch/common/settings/SettingTests.java index 719144447258..cccfa373200d 100644 --- a/core/src/test/java/org/elasticsearch/common/settings/SettingTests.java +++ b/core/src/test/java/org/elasticsearch/common/settings/SettingTests.java @@ -45,7 +45,7 @@ public class SettingTests extends ESTestCase { ByteSizeValue byteSizeValue = byteSizeValueSetting.get(Settings.EMPTY); assertEquals(byteSizeValue.bytes(), 1024); AtomicReference value = new AtomicReference<>(null); - ClusterSettings.SettingUpdater settingUpdater = byteSizeValueSetting.newUpdater(value::set, logger); + ClusterSettings.SettingUpdater settingUpdater = byteSizeValueSetting.newUpdater(value::set, logger); try { settingUpdater.apply(Settings.builder().put("a.byte.size", 12).build(), Settings.EMPTY); fail("no unit"); @@ -60,7 +60,7 @@ public class SettingTests extends ESTestCase { public void testSimpleUpdate() { Setting booleanSetting = Setting.boolSetting("foo.bar", false, true, Setting.Scope.CLUSTER); AtomicReference atomicBoolean = new AtomicReference<>(null); - ClusterSettings.SettingUpdater settingUpdater = booleanSetting.newUpdater(atomicBoolean::set, logger); + ClusterSettings.SettingUpdater settingUpdater = booleanSetting.newUpdater(atomicBoolean::set, logger); Settings build = Settings.builder().put("foo.bar", false).build(); settingUpdater.apply(build, Settings.EMPTY); assertNull(atomicBoolean.get()); @@ -94,8 +94,7 @@ public class SettingTests extends ESTestCase { Setting booleanSetting = Setting.boolSetting("foo.bar", false, true, Setting.Scope.CLUSTER); AtomicReference ab1 = new AtomicReference<>(null); AtomicReference ab2 = new AtomicReference<>(null); - ClusterSettings.SettingUpdater settingUpdater = booleanSetting.newUpdater(ab1::set, logger); - ClusterSettings.SettingUpdater settingUpdater2 = booleanSetting.newUpdater(ab2::set, logger); + ClusterSettings.SettingUpdater settingUpdater = booleanSetting.newUpdater(ab1::set, logger); settingUpdater.apply(Settings.builder().put("foo.bar", true).build(), Settings.EMPTY); assertTrue(ab1.get()); assertNull(ab2.get()); @@ -120,7 +119,7 @@ public class SettingTests extends ESTestCase { assertFalse(setting.isGroupSetting()); ref.set(setting.get(Settings.EMPTY)); ComplexType type = ref.get(); - ClusterSettings.SettingUpdater settingUpdater = setting.newUpdater(ref::set, logger); + ClusterSettings.SettingUpdater settingUpdater = setting.newUpdater(ref::set, logger); assertFalse(settingUpdater.apply(Settings.EMPTY, Settings.EMPTY)); assertSame("no update - type has not changed", type, ref.get()); @@ -147,7 +146,7 @@ public class SettingTests extends ESTestCase { AtomicReference ref = new AtomicReference<>(null); Setting setting = Setting.groupSetting("foo.bar.", true, Setting.Scope.CLUSTER); assertTrue(setting.isGroupSetting()); - ClusterSettings.SettingUpdater settingUpdater = setting.newUpdater(ref::set, logger); + ClusterSettings.SettingUpdater settingUpdater = setting.newUpdater(ref::set, logger); Settings currentInput = Settings.builder().put("foo.bar.1.value", "1").put("foo.bar.2.value", "2").put("foo.bar.3.value", "3").build(); Settings previousInput = Settings.EMPTY; @@ -191,7 +190,7 @@ public class SettingTests extends ESTestCase { assertTrue(setting.match("foo.bar.baz")); assertFalse(setting.match("foo.baz.bar")); - ClusterSettings.SettingUpdater predicateSettingUpdater = setting.newUpdater(ref::set, logger,(s) -> assertFalse(true)); + ClusterSettings.SettingUpdater predicateSettingUpdater = setting.newUpdater(ref::set, logger,(s) -> assertFalse(true)); try { predicateSettingUpdater.apply(Settings.builder().put("foo.bar.1.value", "1").put("foo.bar.2.value", "2").build(), Settings.EMPTY); fail("not accepted"); @@ -273,7 +272,7 @@ public class SettingTests extends ESTestCase { assertArrayEquals(value.toArray(new String[0]), input.toArray(new String[0])); AtomicReference> ref = new AtomicReference<>(); - AbstractScopedSettings.SettingUpdater settingUpdater = listSetting.newUpdater(ref::set, logger); + AbstractScopedSettings.SettingUpdater> settingUpdater = listSetting.newUpdater(ref::set, logger); assertTrue(settingUpdater.hasChanged(builder.build(), Settings.EMPTY)); settingUpdater.apply(builder.build(), Settings.EMPTY); assertEquals(input.size(), ref.get().size()); diff --git a/core/src/test/java/org/elasticsearch/index/IndexModuleTests.java b/core/src/test/java/org/elasticsearch/index/IndexModuleTests.java index d38248f983d4..eae3e65c406a 100644 --- a/core/src/test/java/org/elasticsearch/index/IndexModuleTests.java +++ b/core/src/test/java/org/elasticsearch/index/IndexModuleTests.java @@ -18,6 +18,15 @@ */ package org.elasticsearch.index; +import java.io.IOException; +import java.util.Arrays; +import java.util.Collections; +import java.util.HashSet; +import java.util.Set; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicBoolean; +import java.util.function.Consumer; + import org.apache.lucene.index.AssertingDirectoryReader; import org.apache.lucene.index.DirectoryReader; import org.apache.lucene.index.FieldInvertState; @@ -33,7 +42,6 @@ import org.elasticsearch.Version; import org.elasticsearch.cache.recycler.PageCacheRecycler; import org.elasticsearch.client.Client; import org.elasticsearch.cluster.metadata.IndexMetaData; -import org.elasticsearch.common.io.stream.NamedWriteableRegistry; import org.elasticsearch.common.settings.Setting; import org.elasticsearch.common.settings.Settings; import org.elasticsearch.common.util.BigArrays; @@ -52,7 +60,6 @@ import org.elasticsearch.index.similarity.SimilarityProvider; import org.elasticsearch.index.similarity.SimilarityService; import org.elasticsearch.index.store.IndexStore; import org.elasticsearch.index.store.IndexStoreConfig; -import org.elasticsearch.indices.IndexingMemoryController; import org.elasticsearch.indices.IndicesModule; import org.elasticsearch.indices.IndicesWarmer; import org.elasticsearch.indices.breaker.CircuitBreakerService; @@ -71,14 +78,7 @@ import org.elasticsearch.test.engine.MockEngineFactory; import org.elasticsearch.threadpool.ThreadPool; import org.elasticsearch.watcher.ResourceWatcherService; -import java.io.IOException; -import java.util.Arrays; -import java.util.Collections; -import java.util.HashSet; -import java.util.Set; -import java.util.concurrent.TimeUnit; -import java.util.concurrent.atomic.AtomicBoolean; -import java.util.function.Consumer; +import static java.util.Collections.emptyMap; public class IndexModuleTests extends ESTestCase { private Index index; @@ -109,7 +109,7 @@ public class IndexModuleTests extends ESTestCase { Set scriptEngines = new HashSet<>(); scriptEngines.addAll(Arrays.asList(scriptEngineServices)); ScriptService scriptService = new ScriptService(settings, environment, scriptEngines, new ResourceWatcherService(settings, threadPool), new ScriptContextRegistry(Collections.emptyList())); - IndicesQueriesRegistry indicesQueriesRegistry = new IndicesQueriesRegistry(settings, Collections.emptySet(), new NamedWriteableRegistry()); + IndicesQueriesRegistry indicesQueriesRegistry = new IndicesQueriesRegistry(settings, emptyMap()); return new NodeServicesProvider(threadPool, indicesQueryCache, null, warmer, bigArrays, client, scriptService, indicesQueriesRegistry, indicesFieldDataCache, circuitBreakerService); } @@ -367,8 +367,6 @@ public class IndexModuleTests extends ESTestCase { } } - - public static final class FooStore extends IndexStore { public FooStore(IndexSettings indexSettings, IndexStoreConfig config) { diff --git a/core/src/test/java/org/elasticsearch/index/fielddata/ParentChildFieldDataTests.java b/core/src/test/java/org/elasticsearch/index/fielddata/ParentChildFieldDataTests.java index eefe8c891833..0187ababfe01 100644 --- a/core/src/test/java/org/elasticsearch/index/fielddata/ParentChildFieldDataTests.java +++ b/core/src/test/java/org/elasticsearch/index/fielddata/ParentChildFieldDataTests.java @@ -36,6 +36,7 @@ import org.apache.lucene.util.BytesRef; import org.elasticsearch.action.admin.indices.mapping.put.PutMappingRequest; import org.elasticsearch.common.compress.CompressedXContent; import org.elasticsearch.index.fielddata.plain.ParentChildIndexFieldData; +import org.elasticsearch.index.mapper.MapperService; import org.elasticsearch.index.mapper.Uid; import org.elasticsearch.index.mapper.internal.ParentFieldMapper; import org.elasticsearch.index.mapper.internal.UidFieldMapper; @@ -60,10 +61,10 @@ public class ParentChildFieldDataTests extends AbstractFieldDataTestCase { @Before public void before() throws Exception { mapperService.merge( - childType, new CompressedXContent(PutMappingRequest.buildFromSimplifiedDef(childType, "_parent", "type=" + parentType).string()), true, false + childType, new CompressedXContent(PutMappingRequest.buildFromSimplifiedDef(childType, "_parent", "type=" + parentType).string()), MapperService.MergeReason.MAPPING_UPDATE, false ); mapperService.merge( - grandChildType, new CompressedXContent(PutMappingRequest.buildFromSimplifiedDef(grandChildType, "_parent", "type=" + childType).string()), true, false + grandChildType, new CompressedXContent(PutMappingRequest.buildFromSimplifiedDef(grandChildType, "_parent", "type=" + childType).string()), MapperService.MergeReason.MAPPING_UPDATE, false ); Document d = new Document(); diff --git a/core/src/test/java/org/elasticsearch/index/mapper/DynamicMappingTests.java b/core/src/test/java/org/elasticsearch/index/mapper/DynamicMappingTests.java index 22a10ab8229c..1a4fb0d9c4ce 100644 --- a/core/src/test/java/org/elasticsearch/index/mapper/DynamicMappingTests.java +++ b/core/src/test/java/org/elasticsearch/index/mapper/DynamicMappingTests.java @@ -482,7 +482,7 @@ public class DynamicMappingTests extends ESSingleNodeTestCase { .endObject() .endArray() .endObject().endObject(); - indexService.mapperService().merge("type1", new CompressedXContent(mappings1.bytes()), true, false); + indexService.mapperService().merge("type1", new CompressedXContent(mappings1.bytes()), MapperService.MergeReason.MAPPING_UPDATE, false); XContentBuilder mappings2 = jsonBuilder().startObject() .startObject("type2") .startObject("properties") @@ -491,7 +491,7 @@ public class DynamicMappingTests extends ESSingleNodeTestCase { .endObject() .endObject() .endObject().endObject(); - indexService.mapperService().merge("type2", new CompressedXContent(mappings2.bytes()), true, false); + indexService.mapperService().merge("type2", new CompressedXContent(mappings2.bytes()), MapperService.MergeReason.MAPPING_UPDATE, false); XContentBuilder json = XContentFactory.jsonBuilder().startObject() .field("field", "foo") @@ -502,7 +502,7 @@ public class DynamicMappingTests extends ESSingleNodeTestCase { ParsedDocument parsed = mapper.parse(source); assertNotNull(parsed.dynamicMappingsUpdate()); - indexService.mapperService().merge("type1", new CompressedXContent(parsed.dynamicMappingsUpdate().toString()), false, false); + indexService.mapperService().merge("type1", new CompressedXContent(parsed.dynamicMappingsUpdate().toString()), MapperService.MergeReason.MAPPING_UPDATE, false); mapper = indexService.mapperService().documentMapper("type1"); assertNotNull(mapper.mappers().getMapper("field.raw")); parsed = mapper.parse(source); diff --git a/core/src/test/java/org/elasticsearch/index/mapper/MapperServiceTests.java b/core/src/test/java/org/elasticsearch/index/mapper/MapperServiceTests.java index 94445d3aadba..83eeaf36ffd3 100644 --- a/core/src/test/java/org/elasticsearch/index/mapper/MapperServiceTests.java +++ b/core/src/test/java/org/elasticsearch/index/mapper/MapperServiceTests.java @@ -83,15 +83,15 @@ public class MapperServiceTests extends ESSingleNodeTestCase { MapperService mapperService = indexService1.mapperService(); assertEquals(Collections.emptySet(), mapperService.types()); - mapperService.merge("type1", new CompressedXContent("{\"type1\":{}}"), true, false); + mapperService.merge("type1", new CompressedXContent("{\"type1\":{}}"), MapperService.MergeReason.MAPPING_UPDATE, false); assertNull(mapperService.documentMapper(MapperService.DEFAULT_MAPPING)); assertEquals(Collections.singleton("type1"), mapperService.types()); - mapperService.merge(MapperService.DEFAULT_MAPPING, new CompressedXContent("{\"_default_\":{}}"), true, false); + mapperService.merge(MapperService.DEFAULT_MAPPING, new CompressedXContent("{\"_default_\":{}}"), MapperService.MergeReason.MAPPING_UPDATE, false); assertNotNull(mapperService.documentMapper(MapperService.DEFAULT_MAPPING)); assertEquals(Collections.singleton("type1"), mapperService.types()); - mapperService.merge("type2", new CompressedXContent("{\"type2\":{}}"), true, false); + mapperService.merge("type2", new CompressedXContent("{\"type2\":{}}"), MapperService.MergeReason.MAPPING_UPDATE, false); assertNotNull(mapperService.documentMapper(MapperService.DEFAULT_MAPPING)); assertEquals(new HashSet<>(Arrays.asList("type1", "type2")), mapperService.types()); } diff --git a/core/src/test/java/org/elasticsearch/index/mapper/copyto/CopyToMapperTests.java b/core/src/test/java/org/elasticsearch/index/mapper/copyto/CopyToMapperTests.java index daf54d501d73..d1de843e38f8 100644 --- a/core/src/test/java/org/elasticsearch/index/mapper/copyto/CopyToMapperTests.java +++ b/core/src/test/java/org/elasticsearch/index/mapper/copyto/CopyToMapperTests.java @@ -312,11 +312,11 @@ public class CopyToMapperTests extends ESSingleNodeTestCase { .endObject().endObject().endObject().string(); MapperService mapperService = createIndex("test").mapperService(); - DocumentMapper docMapperBefore = mapperService.merge("type1", new CompressedXContent(mappingBefore), true, false); + DocumentMapper docMapperBefore = mapperService.merge("type1", new CompressedXContent(mappingBefore), MapperService.MergeReason.MAPPING_UPDATE, false); assertEquals(Arrays.asList("foo", "bar"), docMapperBefore.mappers().getMapper("copy_test").copyTo().copyToFields()); - DocumentMapper docMapperAfter = mapperService.merge("type1", new CompressedXContent(mappingAfter), false, false); + DocumentMapper docMapperAfter = mapperService.merge("type1", new CompressedXContent(mappingAfter), MapperService.MergeReason.MAPPING_UPDATE, false); assertEquals(Arrays.asList("baz", "bar"), docMapperAfter.mappers().getMapper("copy_test").copyTo().copyToFields()); assertEquals(Arrays.asList("foo", "bar"), docMapperBefore.mappers().getMapper("copy_test").copyTo().copyToFields()); diff --git a/core/src/test/java/org/elasticsearch/index/mapper/core/BooleanFieldMapperTests.java b/core/src/test/java/org/elasticsearch/index/mapper/core/BooleanFieldMapperTests.java index 3aa04ba0f018..2175f2ce3e75 100644 --- a/core/src/test/java/org/elasticsearch/index/mapper/core/BooleanFieldMapperTests.java +++ b/core/src/test/java/org/elasticsearch/index/mapper/core/BooleanFieldMapperTests.java @@ -37,6 +37,7 @@ import org.elasticsearch.index.IndexService; import org.elasticsearch.index.mapper.DocumentMapper; import org.elasticsearch.index.mapper.DocumentMapperParser; import org.elasticsearch.index.mapper.FieldMapper; +import org.elasticsearch.index.mapper.MapperService; import org.elasticsearch.index.mapper.ParsedDocument; import org.elasticsearch.test.ESSingleNodeTestCase; import org.junit.Before; @@ -125,7 +126,7 @@ public class BooleanFieldMapperTests extends ESSingleNodeTestCase { .endObject() .endObject().endObject() .endObject().endObject().string(); - DocumentMapper mapper = indexService.mapperService().merge("type", new CompressedXContent(mapping), true, false); + DocumentMapper mapper = indexService.mapperService().merge("type", new CompressedXContent(mapping), MapperService.MergeReason.MAPPING_UPDATE, false); assertEquals(mapping, mapper.mappingSource().toString()); BytesReference source = XContentFactory.jsonBuilder() .startObject() diff --git a/core/src/test/java/org/elasticsearch/index/mapper/core/TokenCountFieldMapperTests.java b/core/src/test/java/org/elasticsearch/index/mapper/core/TokenCountFieldMapperTests.java index a746717b73a0..e8fe39e9bbb9 100644 --- a/core/src/test/java/org/elasticsearch/index/mapper/core/TokenCountFieldMapperTests.java +++ b/core/src/test/java/org/elasticsearch/index/mapper/core/TokenCountFieldMapperTests.java @@ -52,7 +52,7 @@ public class TokenCountFieldMapperTests extends ESSingleNodeTestCase { .endObject() .endObject().endObject().string(); MapperService mapperService = createIndex("test").mapperService(); - DocumentMapper stage1 = mapperService.merge("person", new CompressedXContent(stage1Mapping), true, false); + DocumentMapper stage1 = mapperService.merge("person", new CompressedXContent(stage1Mapping), MapperService.MergeReason.MAPPING_UPDATE, false); String stage2Mapping = XContentFactory.jsonBuilder().startObject() .startObject("person") @@ -63,7 +63,7 @@ public class TokenCountFieldMapperTests extends ESSingleNodeTestCase { .endObject() .endObject() .endObject().endObject().string(); - DocumentMapper stage2 = mapperService.merge("person", new CompressedXContent(stage2Mapping), false, false); + DocumentMapper stage2 = mapperService.merge("person", new CompressedXContent(stage2Mapping), MapperService.MergeReason.MAPPING_UPDATE, false); // previous mapper has not been modified assertThat(((TokenCountFieldMapper) stage1.mappers().smartNameFieldMapper("tc")).analyzer(), equalTo("keyword")); diff --git a/core/src/test/java/org/elasticsearch/index/mapper/geo/GeoPointFieldMapperTests.java b/core/src/test/java/org/elasticsearch/index/mapper/geo/GeoPointFieldMapperTests.java index b756036bc266..2ea19b024505 100644 --- a/core/src/test/java/org/elasticsearch/index/mapper/geo/GeoPointFieldMapperTests.java +++ b/core/src/test/java/org/elasticsearch/index/mapper/geo/GeoPointFieldMapperTests.java @@ -671,12 +671,12 @@ public class GeoPointFieldMapperTests extends ESSingleNodeTestCase { .startObject("properties").startObject("point").field("type", "geo_point").field("lat_lon", true) .field("geohash", true).endObject().endObject().endObject().endObject().string(); MapperService mapperService = createIndex("test", settings).mapperService(); - DocumentMapper stage1 = mapperService.merge("type", new CompressedXContent(stage1Mapping), true, false); + DocumentMapper stage1 = mapperService.merge("type", new CompressedXContent(stage1Mapping), MapperService.MergeReason.MAPPING_UPDATE, false); String stage2Mapping = XContentFactory.jsonBuilder().startObject().startObject("type") .startObject("properties").startObject("point").field("type", "geo_point").field("lat_lon", false) .field("geohash", false).endObject().endObject().endObject().endObject().string(); try { - mapperService.merge("type", new CompressedXContent(stage2Mapping), false, false); + mapperService.merge("type", new CompressedXContent(stage2Mapping), MapperService.MergeReason.MAPPING_UPDATE, false); fail(); } catch (IllegalArgumentException e) { assertThat(e.getMessage(), containsString("mapper [point] has different [lat_lon]")); @@ -688,7 +688,7 @@ public class GeoPointFieldMapperTests extends ESSingleNodeTestCase { stage2Mapping = XContentFactory.jsonBuilder().startObject().startObject("type") .startObject("properties").startObject("point").field("type", "geo_point").field("lat_lon", true) .field("geohash", true).endObject().endObject().endObject().endObject().string(); - mapperService.merge("type", new CompressedXContent(stage2Mapping), false, false); + mapperService.merge("type", new CompressedXContent(stage2Mapping), MapperService.MergeReason.MAPPING_UPDATE, false); } public void testGeoHashSearch() throws Exception { diff --git a/core/src/test/java/org/elasticsearch/index/mapper/geo/GeoShapeFieldMapperTests.java b/core/src/test/java/org/elasticsearch/index/mapper/geo/GeoShapeFieldMapperTests.java index af03d3accb1b..0ad4dbd87bab 100644 --- a/core/src/test/java/org/elasticsearch/index/mapper/geo/GeoShapeFieldMapperTests.java +++ b/core/src/test/java/org/elasticsearch/index/mapper/geo/GeoShapeFieldMapperTests.java @@ -376,13 +376,13 @@ public class GeoShapeFieldMapperTests extends ESSingleNodeTestCase { .field("precision", "1m").field("tree_levels", 8).field("distance_error_pct", 0.01).field("orientation", "ccw") .endObject().endObject().endObject().endObject().string(); MapperService mapperService = createIndex("test").mapperService(); - DocumentMapper docMapper = mapperService.merge("type", new CompressedXContent(stage1Mapping), true, false); + DocumentMapper docMapper = mapperService.merge("type", new CompressedXContent(stage1Mapping), MapperService.MergeReason.MAPPING_UPDATE, false); String stage2Mapping = XContentFactory.jsonBuilder().startObject().startObject("type") .startObject("properties").startObject("shape").field("type", "geo_shape").field("tree", "quadtree") .field("strategy", "term").field("precision", "1km").field("tree_levels", 26).field("distance_error_pct", 26) .field("orientation", "cw").endObject().endObject().endObject().endObject().string(); try { - mapperService.merge("type", new CompressedXContent(stage2Mapping), false, false); + mapperService.merge("type", new CompressedXContent(stage2Mapping), MapperService.MergeReason.MAPPING_UPDATE, false); fail(); } catch (IllegalArgumentException e) { assertThat(e.getMessage(), containsString("mapper [shape] has different [strategy]")); @@ -408,7 +408,7 @@ public class GeoShapeFieldMapperTests extends ESSingleNodeTestCase { stage2Mapping = XContentFactory.jsonBuilder().startObject().startObject("type") .startObject("properties").startObject("shape").field("type", "geo_shape").field("precision", "1m") .field("tree_levels", 8).field("distance_error_pct", 0.001).field("orientation", "cw").endObject().endObject().endObject().endObject().string(); - docMapper = mapperService.merge("type", new CompressedXContent(stage2Mapping), false, false); + docMapper = mapperService.merge("type", new CompressedXContent(stage2Mapping), MapperService.MergeReason.MAPPING_UPDATE, false); fieldMapper = docMapper.mappers().getMapper("shape"); assertThat(fieldMapper, instanceOf(GeoShapeFieldMapper.class)); diff --git a/core/src/test/java/org/elasticsearch/index/mapper/internal/FieldNamesFieldMapperTests.java b/core/src/test/java/org/elasticsearch/index/mapper/internal/FieldNamesFieldMapperTests.java index 2e28f60e7d98..e23ea15b64f4 100644 --- a/core/src/test/java/org/elasticsearch/index/mapper/internal/FieldNamesFieldMapperTests.java +++ b/core/src/test/java/org/elasticsearch/index/mapper/internal/FieldNamesFieldMapperTests.java @@ -148,11 +148,11 @@ public class FieldNamesFieldMapperTests extends ESSingleNodeTestCase { .endObject().endObject().string(); MapperService mapperService = createIndex("test").mapperService(); - DocumentMapper mapperEnabled = mapperService.merge("type", new CompressedXContent(enabledMapping), true, false); - DocumentMapper mapperDisabled = mapperService.merge("type", new CompressedXContent(disabledMapping), false, false); + DocumentMapper mapperEnabled = mapperService.merge("type", new CompressedXContent(enabledMapping), MapperService.MergeReason.MAPPING_UPDATE, false); + DocumentMapper mapperDisabled = mapperService.merge("type", new CompressedXContent(disabledMapping), MapperService.MergeReason.MAPPING_UPDATE, false); assertFalse(mapperDisabled.metadataMapper(FieldNamesFieldMapper.class).fieldType().isEnabled()); - mapperEnabled = mapperService.merge("type", new CompressedXContent(enabledMapping), false, false); + mapperEnabled = mapperService.merge("type", new CompressedXContent(enabledMapping), MapperService.MergeReason.MAPPING_UPDATE, false); assertTrue(mapperEnabled.metadataMapper(FieldNamesFieldMapper.class).fieldType().isEnabled()); } diff --git a/core/src/test/java/org/elasticsearch/index/mapper/merge/TestMergeMapperTests.java b/core/src/test/java/org/elasticsearch/index/mapper/merge/TestMergeMapperTests.java index 80f7942bbcce..4614909568c6 100644 --- a/core/src/test/java/org/elasticsearch/index/mapper/merge/TestMergeMapperTests.java +++ b/core/src/test/java/org/elasticsearch/index/mapper/merge/TestMergeMapperTests.java @@ -34,6 +34,7 @@ import org.elasticsearch.index.mapper.core.StringFieldMapper; import org.elasticsearch.index.mapper.object.ObjectMapper; import org.elasticsearch.test.ESSingleNodeTestCase; +import java.io.IOException; import java.util.concurrent.CyclicBarrier; import java.util.concurrent.atomic.AtomicBoolean; import java.util.concurrent.atomic.AtomicReference; @@ -135,8 +136,8 @@ public class TestMergeMapperTests extends ESSingleNodeTestCase { .startObject("properties").startObject("field").field("type", "string").field("analyzer", "standard").field("ignore_above", 14).endObject().endObject() .endObject().endObject().string(); - DocumentMapper existing = mapperService.merge("type", new CompressedXContent(mapping1), true, false); - DocumentMapper merged = mapperService.merge("type", new CompressedXContent(mapping2), false, false); + DocumentMapper existing = mapperService.merge("type", new CompressedXContent(mapping1), MapperService.MergeReason.MAPPING_UPDATE, false); + DocumentMapper merged = mapperService.merge("type", new CompressedXContent(mapping2), MapperService.MergeReason.MAPPING_UPDATE, false); assertThat(((NamedAnalyzer) existing.mappers().getMapper("field").fieldType().searchAnalyzer()).name(), equalTo("whitespace")); @@ -146,7 +147,7 @@ public class TestMergeMapperTests extends ESSingleNodeTestCase { public void testConcurrentMergeTest() throws Throwable { final MapperService mapperService = createIndex("test").mapperService(); - mapperService.merge("test", new CompressedXContent("{\"test\":{}}"), true, false); + mapperService.merge("test", new CompressedXContent("{\"test\":{}}"), MapperService.MergeReason.MAPPING_UPDATE, false); final DocumentMapper documentMapper = mapperService.documentMapper("test"); DocumentFieldMappers dfm = documentMapper.mappers(); @@ -172,7 +173,7 @@ public class TestMergeMapperTests extends ESSingleNodeTestCase { Mapping update = doc.dynamicMappingsUpdate(); assert update != null; lastIntroducedFieldName.set(fieldName); - mapperService.merge("test", new CompressedXContent(update.toString()), false, false); + mapperService.merge("test", new CompressedXContent(update.toString()), MapperService.MergeReason.MAPPING_UPDATE, false); } } catch (Throwable t) { error.set(t); @@ -203,4 +204,28 @@ public class TestMergeMapperTests extends ESSingleNodeTestCase { throw error.get(); } } + + public void testDoNotRepeatOriginalMapping() throws IOException { + CompressedXContent mapping = new CompressedXContent(XContentFactory.jsonBuilder().startObject() + .startObject("type") + .startObject("_source") + .field("enabled", false) + .endObject() + .endObject().endObject().bytes()); + MapperService mapperService = createIndex("test").mapperService(); + mapperService.merge("type", mapping, MapperService.MergeReason.MAPPING_UPDATE, false); + + CompressedXContent update = new CompressedXContent(XContentFactory.jsonBuilder().startObject() + .startObject("type") + .startObject("properties") + .startObject("foo") + .field("type", "string") + .endObject() + .endObject() + .endObject().endObject().bytes()); + DocumentMapper mapper = mapperService.merge("type", update, MapperService.MergeReason.MAPPING_UPDATE, false); + + assertNotNull(mapper.mappers().getMapper("foo")); + assertFalse(mapper.sourceMapper().enabled()); + } } diff --git a/core/src/test/java/org/elasticsearch/index/mapper/multifield/merge/JavaMultiFieldMergeTests.java b/core/src/test/java/org/elasticsearch/index/mapper/multifield/merge/JavaMultiFieldMergeTests.java index 651b8c45d55d..e08d610d3f2b 100644 --- a/core/src/test/java/org/elasticsearch/index/mapper/multifield/merge/JavaMultiFieldMergeTests.java +++ b/core/src/test/java/org/elasticsearch/index/mapper/multifield/merge/JavaMultiFieldMergeTests.java @@ -42,7 +42,7 @@ public class JavaMultiFieldMergeTests extends ESSingleNodeTestCase { String mapping = copyToStringFromClasspath("/org/elasticsearch/index/mapper/multifield/merge/test-mapping1.json"); MapperService mapperService = createIndex("test").mapperService(); - DocumentMapper docMapper = mapperService.merge("person", new CompressedXContent(mapping), true, false); + DocumentMapper docMapper = mapperService.merge("person", new CompressedXContent(mapping), MapperService.MergeReason.MAPPING_UPDATE, false); assertNotSame(IndexOptions.NONE, docMapper.mappers().getMapper("name").fieldType().indexOptions()); assertThat(docMapper.mappers().getMapper("name.indexed"), nullValue()); @@ -55,7 +55,7 @@ public class JavaMultiFieldMergeTests extends ESSingleNodeTestCase { assertThat(f, nullValue()); mapping = copyToStringFromClasspath("/org/elasticsearch/index/mapper/multifield/merge/test-mapping2.json"); - docMapper = mapperService.merge("person", new CompressedXContent(mapping), false, false); + docMapper = mapperService.merge("person", new CompressedXContent(mapping), MapperService.MergeReason.MAPPING_UPDATE, false); assertNotSame(IndexOptions.NONE, docMapper.mappers().getMapper("name").fieldType().indexOptions()); @@ -72,7 +72,7 @@ public class JavaMultiFieldMergeTests extends ESSingleNodeTestCase { assertThat(f, notNullValue()); mapping = copyToStringFromClasspath("/org/elasticsearch/index/mapper/multifield/merge/test-mapping3.json"); - docMapper = mapperService.merge("person", new CompressedXContent(mapping), false, false); + docMapper = mapperService.merge("person", new CompressedXContent(mapping), MapperService.MergeReason.MAPPING_UPDATE, false); assertNotSame(IndexOptions.NONE, docMapper.mappers().getMapper("name").fieldType().indexOptions()); @@ -83,7 +83,7 @@ public class JavaMultiFieldMergeTests extends ESSingleNodeTestCase { assertThat(docMapper.mappers().getMapper("name.not_indexed3"), nullValue()); mapping = copyToStringFromClasspath("/org/elasticsearch/index/mapper/multifield/merge/test-mapping4.json"); - docMapper = mapperService.merge("person", new CompressedXContent(mapping), false, false); + docMapper = mapperService.merge("person", new CompressedXContent(mapping), MapperService.MergeReason.MAPPING_UPDATE, false); assertNotSame(IndexOptions.NONE, docMapper.mappers().getMapper("name").fieldType().indexOptions()); @@ -98,7 +98,7 @@ public class JavaMultiFieldMergeTests extends ESSingleNodeTestCase { String mapping = copyToStringFromClasspath("/org/elasticsearch/index/mapper/multifield/merge/test-mapping1.json"); MapperService mapperService = createIndex("test").mapperService(); - DocumentMapper docMapper = mapperService.merge("person", new CompressedXContent(mapping), true, false); + DocumentMapper docMapper = mapperService.merge("person", new CompressedXContent(mapping), MapperService.MergeReason.MAPPING_UPDATE, false); assertNotSame(IndexOptions.NONE, docMapper.mappers().getMapper("name").fieldType().indexOptions()); assertThat(docMapper.mappers().getMapper("name.indexed"), nullValue()); @@ -112,7 +112,7 @@ public class JavaMultiFieldMergeTests extends ESSingleNodeTestCase { mapping = copyToStringFromClasspath("/org/elasticsearch/index/mapper/multifield/merge/upgrade1.json"); - docMapper = mapperService.merge("person", new CompressedXContent(mapping), false, false); + docMapper = mapperService.merge("person", new CompressedXContent(mapping), MapperService.MergeReason.MAPPING_UPDATE, false); assertNotSame(IndexOptions.NONE, docMapper.mappers().getMapper("name").fieldType().indexOptions()); @@ -129,7 +129,7 @@ public class JavaMultiFieldMergeTests extends ESSingleNodeTestCase { assertThat(f, notNullValue()); mapping = copyToStringFromClasspath("/org/elasticsearch/index/mapper/multifield/merge/upgrade2.json"); - docMapper = mapperService.merge("person", new CompressedXContent(mapping), false, false); + docMapper = mapperService.merge("person", new CompressedXContent(mapping), MapperService.MergeReason.MAPPING_UPDATE, false); assertNotSame(IndexOptions.NONE, docMapper.mappers().getMapper("name").fieldType().indexOptions()); @@ -142,7 +142,7 @@ public class JavaMultiFieldMergeTests extends ESSingleNodeTestCase { mapping = copyToStringFromClasspath("/org/elasticsearch/index/mapper/multifield/merge/upgrade3.json"); try { - mapperService.merge("person", new CompressedXContent(mapping), false, false); + mapperService.merge("person", new CompressedXContent(mapping), MapperService.MergeReason.MAPPING_UPDATE, false); fail(); } catch (IllegalArgumentException e) { assertThat(e.getMessage(), containsString("mapper [name] has different [index] values")); diff --git a/core/src/test/java/org/elasticsearch/index/mapper/source/DefaultSourceMappingTests.java b/core/src/test/java/org/elasticsearch/index/mapper/source/DefaultSourceMappingTests.java index b5cca4ed95fc..f82dcb6eeccd 100644 --- a/core/src/test/java/org/elasticsearch/index/mapper/source/DefaultSourceMappingTests.java +++ b/core/src/test/java/org/elasticsearch/index/mapper/source/DefaultSourceMappingTests.java @@ -175,7 +175,7 @@ public class DefaultSourceMappingTests extends ESSingleNodeTestCase { .endObject().endObject().string(); MapperService mapperService = createIndex("test").mapperService(); - mapperService.merge(MapperService.DEFAULT_MAPPING, new CompressedXContent(defaultMapping), true, false); + mapperService.merge(MapperService.DEFAULT_MAPPING, new CompressedXContent(defaultMapping), MapperService.MergeReason.MAPPING_UPDATE, false); DocumentMapper mapper = mapperService.documentMapperWithAutoCreate("my_type").getDocumentMapper(); assertThat(mapper.type(), equalTo("my_type")); @@ -188,12 +188,12 @@ public class DefaultSourceMappingTests extends ESSingleNodeTestCase { .endObject().endObject().string(); MapperService mapperService = createIndex("test").mapperService(); - mapperService.merge(MapperService.DEFAULT_MAPPING, new CompressedXContent(defaultMapping), true, false); + mapperService.merge(MapperService.DEFAULT_MAPPING, new CompressedXContent(defaultMapping), MapperService.MergeReason.MAPPING_UPDATE, false); String mapping = XContentFactory.jsonBuilder().startObject().startObject("my_type") .startObject("_source").field("enabled", true).endObject() .endObject().endObject().string(); - mapperService.merge("my_type", new CompressedXContent(mapping), true, false); + mapperService.merge("my_type", new CompressedXContent(mapping), MapperService.MergeReason.MAPPING_UPDATE, false); DocumentMapper mapper = mapperService.documentMapper("my_type"); assertThat(mapper.type(), equalTo("my_type")); diff --git a/core/src/test/java/org/elasticsearch/index/mapper/string/SimpleStringMappingTests.java b/core/src/test/java/org/elasticsearch/index/mapper/string/SimpleStringMappingTests.java index 7bd4d9a78c33..d32dcad54340 100644 --- a/core/src/test/java/org/elasticsearch/index/mapper/string/SimpleStringMappingTests.java +++ b/core/src/test/java/org/elasticsearch/index/mapper/string/SimpleStringMappingTests.java @@ -480,7 +480,7 @@ public class SimpleStringMappingTests extends ESSingleNodeTestCase { .endObject().endObject().string(); MapperService mapperService = indexService.mapperService(); - DocumentMapper defaultMapper = mapperService.merge("type", new CompressedXContent(mapping), true, false); + DocumentMapper defaultMapper = mapperService.merge("type", new CompressedXContent(mapping), MapperService.MergeReason.MAPPING_UPDATE, false); ParsedDocument doc = defaultMapper.parse("test", "type", "1", XContentFactory.jsonBuilder() .startObject() @@ -494,7 +494,7 @@ public class SimpleStringMappingTests extends ESSingleNodeTestCase { String updatedMapping = XContentFactory.jsonBuilder().startObject().startObject("type") .startObject("properties").startObject("field").field("type", "string").startObject("norms").field("enabled", false).endObject() .endObject().endObject().endObject().endObject().string(); - defaultMapper = mapperService.merge("type", new CompressedXContent(updatedMapping), false, false); + defaultMapper = mapperService.merge("type", new CompressedXContent(updatedMapping), MapperService.MergeReason.MAPPING_UPDATE, false); doc = defaultMapper.parse("test", "type", "1", XContentFactory.jsonBuilder() .startObject() @@ -509,7 +509,7 @@ public class SimpleStringMappingTests extends ESSingleNodeTestCase { .startObject("properties").startObject("field").field("type", "string").startObject("norms").field("enabled", true).endObject() .endObject().endObject().endObject().endObject().string(); try { - mapperService.merge("type", new CompressedXContent(updatedMapping), false, false); + mapperService.merge("type", new CompressedXContent(updatedMapping), MapperService.MergeReason.MAPPING_UPDATE, false); fail(); } catch (IllegalArgumentException e) { assertThat(e.getMessage(), containsString("different [omit_norms]")); diff --git a/core/src/test/java/org/elasticsearch/index/mapper/timestamp/TimestampMappingTests.java b/core/src/test/java/org/elasticsearch/index/mapper/timestamp/TimestampMappingTests.java index f62d1fca7823..ed58bb63b650 100644 --- a/core/src/test/java/org/elasticsearch/index/mapper/timestamp/TimestampMappingTests.java +++ b/core/src/test/java/org/elasticsearch/index/mapper/timestamp/TimestampMappingTests.java @@ -123,12 +123,12 @@ public class TimestampMappingTests extends ESSingleNodeTestCase { .startObject("_timestamp").field("enabled", true).endObject() .endObject().endObject().string(); MapperService mapperService = createIndex("test").mapperService(); - DocumentMapper enabledMapper = mapperService.merge("type", new CompressedXContent(enabledMapping), true, false); + DocumentMapper enabledMapper = mapperService.merge("type", new CompressedXContent(enabledMapping), MapperService.MergeReason.MAPPING_UPDATE, false); String disabledMapping = XContentFactory.jsonBuilder().startObject().startObject("type") .startObject("_timestamp").field("enabled", false).endObject() .endObject().endObject().string(); - DocumentMapper disabledMapper = mapperService.merge("type", new CompressedXContent(disabledMapping), false, false); + DocumentMapper disabledMapper = mapperService.merge("type", new CompressedXContent(disabledMapping), MapperService.MergeReason.MAPPING_UPDATE, false); assertThat(enabledMapper.timestampFieldMapper().enabled(), is(true)); assertThat(disabledMapper.timestampFieldMapper().enabled(), is(false)); @@ -374,9 +374,9 @@ public class TimestampMappingTests extends ESSingleNodeTestCase { } void assertConflict(MapperService mapperService, String type, String mapping1, String mapping2, String conflict) throws IOException { - mapperService.merge("type", new CompressedXContent(mapping1), true, false); + mapperService.merge("type", new CompressedXContent(mapping1), MapperService.MergeReason.MAPPING_UPDATE, false); try { - mapperService.merge("type", new CompressedXContent(mapping2), false, false); + mapperService.merge("type", new CompressedXContent(mapping2), MapperService.MergeReason.MAPPING_UPDATE, false); assertNull(conflict); } catch (IllegalArgumentException e) { assertNotNull(conflict); diff --git a/core/src/test/java/org/elasticsearch/index/mapper/ttl/TTLMappingTests.java b/core/src/test/java/org/elasticsearch/index/mapper/ttl/TTLMappingTests.java index fa27e9bcfb75..335d3a06e320 100644 --- a/core/src/test/java/org/elasticsearch/index/mapper/ttl/TTLMappingTests.java +++ b/core/src/test/java/org/elasticsearch/index/mapper/ttl/TTLMappingTests.java @@ -22,10 +22,6 @@ package org.elasticsearch.index.mapper.ttl; import org.apache.lucene.index.IndexOptions; import org.elasticsearch.Version; import org.elasticsearch.action.admin.indices.mapping.get.GetMappingsResponse; -import org.elasticsearch.action.index.IndexRequest; -import org.elasticsearch.cluster.metadata.IndexMetaData; -import org.elasticsearch.cluster.metadata.MappingMetaData; -import org.elasticsearch.cluster.metadata.MetaData; import org.elasticsearch.common.bytes.BytesReference; import org.elasticsearch.common.compress.CompressedXContent; import org.elasticsearch.common.settings.Settings; @@ -99,8 +95,8 @@ public class TTLMappingTests extends ESSingleNodeTestCase { .endObject().endObject().string(); MapperService mapperService = createIndex("test").mapperService(); - DocumentMapper mapperWithoutTtl = mapperService.merge("type", new CompressedXContent(mappingWithoutTtl), true, false); - DocumentMapper mapperWithTtl = mapperService.merge("type", new CompressedXContent(mappingWithTtl), false, false); + DocumentMapper mapperWithoutTtl = mapperService.merge("type", new CompressedXContent(mappingWithoutTtl), MapperService.MergeReason.MAPPING_UPDATE, false); + DocumentMapper mapperWithTtl = mapperService.merge("type", new CompressedXContent(mappingWithTtl), MapperService.MergeReason.MAPPING_UPDATE, false); assertThat(mapperWithoutTtl.TTLFieldMapper().enabled(), equalTo(false)); assertThat(mapperWithTtl.TTLFieldMapper().enabled(), equalTo(true)); @@ -122,8 +118,8 @@ public class TTLMappingTests extends ESSingleNodeTestCase { .endObject().endObject().string(); MapperService mapperService = createIndex("test").mapperService(); - DocumentMapper initialMapper = mapperService.merge("type", new CompressedXContent(mappingWithTtl), true, false); - DocumentMapper updatedMapper = mapperService.merge("type", new CompressedXContent(updatedMapping), false, false); + DocumentMapper initialMapper = mapperService.merge("type", new CompressedXContent(mappingWithTtl), MapperService.MergeReason.MAPPING_UPDATE, false); + DocumentMapper updatedMapper = mapperService.merge("type", new CompressedXContent(updatedMapping), MapperService.MergeReason.MAPPING_UPDATE, false); assertThat(initialMapper.TTLFieldMapper().enabled(), equalTo(true)); assertThat(updatedMapper.TTLFieldMapper().enabled(), equalTo(true)); @@ -133,10 +129,10 @@ public class TTLMappingTests extends ESSingleNodeTestCase { String mappingWithTtl = getMappingWithTtlEnabled().string(); String mappingWithTtlDisabled = getMappingWithTtlDisabled().string(); MapperService mapperService = createIndex("test").mapperService(); - DocumentMapper initialMapper = mapperService.merge("type", new CompressedXContent(mappingWithTtl), true, false); + DocumentMapper initialMapper = mapperService.merge("type", new CompressedXContent(mappingWithTtl), MapperService.MergeReason.MAPPING_UPDATE, false); try { - mapperService.merge("type", new CompressedXContent(mappingWithTtlDisabled), false, false); + mapperService.merge("type", new CompressedXContent(mappingWithTtlDisabled), MapperService.MergeReason.MAPPING_UPDATE, false); fail(); } catch (IllegalArgumentException e) { // expected @@ -174,20 +170,20 @@ public class TTLMappingTests extends ESSingleNodeTestCase { public void testNoConflictIfNothingSetAndDisabledLater() throws Exception { IndexService indexService = createIndex("testindex", Settings.settingsBuilder().build(), "type"); XContentBuilder mappingWithTtlDisabled = getMappingWithTtlDisabled("7d"); - indexService.mapperService().merge("type", new CompressedXContent(mappingWithTtlDisabled.string()), randomBoolean(), false); + indexService.mapperService().merge("type", new CompressedXContent(mappingWithTtlDisabled.string()), MapperService.MergeReason.MAPPING_UPDATE, false); } public void testNoConflictIfNothingSetAndEnabledLater() throws Exception { IndexService indexService = createIndex("testindex", Settings.settingsBuilder().build(), "type"); XContentBuilder mappingWithTtlEnabled = getMappingWithTtlEnabled("7d"); - indexService.mapperService().merge("type", new CompressedXContent(mappingWithTtlEnabled.string()), randomBoolean(), false); + indexService.mapperService().merge("type", new CompressedXContent(mappingWithTtlEnabled.string()), MapperService.MergeReason.MAPPING_UPDATE, false); } public void testMergeWithOnlyDefaultSet() throws Exception { XContentBuilder mappingWithTtlEnabled = getMappingWithTtlEnabled("7d"); IndexService indexService = createIndex("testindex", Settings.settingsBuilder().build(), "type", mappingWithTtlEnabled); XContentBuilder mappingWithOnlyDefaultSet = getMappingWithOnlyTtlDefaultSet("6m"); - indexService.mapperService().merge("type", new CompressedXContent(mappingWithOnlyDefaultSet.string()), false, false); + indexService.mapperService().merge("type", new CompressedXContent(mappingWithOnlyDefaultSet.string()), MapperService.MergeReason.MAPPING_UPDATE, false); CompressedXContent mappingAfterMerge = indexService.mapperService().documentMapper("type").mappingSource(); assertThat(mappingAfterMerge, equalTo(new CompressedXContent("{\"type\":{\"_ttl\":{\"enabled\":true,\"default\":360000},\"properties\":{\"field\":{\"type\":\"string\"}}}}"))); } @@ -198,7 +194,7 @@ public class TTLMappingTests extends ESSingleNodeTestCase { CompressedXContent mappingAfterCreation = indexService.mapperService().documentMapper("type").mappingSource(); assertThat(mappingAfterCreation, equalTo(new CompressedXContent("{\"type\":{\"_ttl\":{\"enabled\":false},\"properties\":{\"field\":{\"type\":\"string\"}}}}"))); XContentBuilder mappingWithOnlyDefaultSet = getMappingWithOnlyTtlDefaultSet("6m"); - indexService.mapperService().merge("type", new CompressedXContent(mappingWithOnlyDefaultSet.string()), false, false); + indexService.mapperService().merge("type", new CompressedXContent(mappingWithOnlyDefaultSet.string()), MapperService.MergeReason.MAPPING_UPDATE, false); CompressedXContent mappingAfterMerge = indexService.mapperService().documentMapper("type").mappingSource(); assertThat(mappingAfterMerge, equalTo(new CompressedXContent("{\"type\":{\"_ttl\":{\"enabled\":false},\"properties\":{\"field\":{\"type\":\"string\"}}}}"))); } diff --git a/core/src/test/java/org/elasticsearch/index/mapper/update/UpdateMappingTests.java b/core/src/test/java/org/elasticsearch/index/mapper/update/UpdateMappingTests.java index 55847c27ed43..4e65c4a5ab7c 100644 --- a/core/src/test/java/org/elasticsearch/index/mapper/update/UpdateMappingTests.java +++ b/core/src/test/java/org/elasticsearch/index/mapper/update/UpdateMappingTests.java @@ -84,7 +84,7 @@ public class UpdateMappingTests extends ESSingleNodeTestCase { private void testNoConflictWhileMergingAndMappingChanged(XContentBuilder mapping, XContentBuilder mappingUpdate, XContentBuilder expectedMapping) throws IOException { IndexService indexService = createIndex("test", Settings.settingsBuilder().build(), "type", mapping); // simulate like in MetaDataMappingService#putMapping - indexService.mapperService().merge("type", new CompressedXContent(mappingUpdate.bytes()), false, false); + indexService.mapperService().merge("type", new CompressedXContent(mappingUpdate.bytes()), MapperService.MergeReason.MAPPING_UPDATE, false); // make sure mappings applied CompressedXContent mappingAfterUpdate = indexService.mapperService().documentMapper("type").mappingSource(); assertThat(mappingAfterUpdate.toString(), equalTo(expectedMapping.string())); @@ -107,7 +107,7 @@ public class UpdateMappingTests extends ESSingleNodeTestCase { CompressedXContent mappingBeforeUpdate = indexService.mapperService().documentMapper("type").mappingSource(); // simulate like in MetaDataMappingService#putMapping try { - indexService.mapperService().merge("type", new CompressedXContent(mappingUpdate.bytes()), true, false); + indexService.mapperService().merge("type", new CompressedXContent(mappingUpdate.bytes()), MapperService.MergeReason.MAPPING_UPDATE, false); fail(); } catch (IllegalArgumentException e) { // expected @@ -128,14 +128,14 @@ public class UpdateMappingTests extends ESSingleNodeTestCase { .endObject().endObject().endObject(); try { - mapperService.merge("type", new CompressedXContent(update.string()), false, false); + mapperService.merge("type", new CompressedXContent(update.string()), MapperService.MergeReason.MAPPING_UPDATE, false); fail(); } catch (IllegalArgumentException e) { assertThat(e.getMessage(), containsString("mapper [foo] of different type, current_type [long], merged_type [double]")); } try { - mapperService.merge("type", new CompressedXContent(update.string()), false, false); + mapperService.merge("type", new CompressedXContent(update.string()), MapperService.MergeReason.MAPPING_UPDATE, false); fail(); } catch (IllegalArgumentException e) { assertThat(e.getMessage(), containsString("mapper [foo] of different type, current_type [long], merged_type [double]")); @@ -155,7 +155,7 @@ public class UpdateMappingTests extends ESSingleNodeTestCase { .endObject().endObject().endObject(); try { - mapperService.merge("type2", new CompressedXContent(update.string()), false, false); + mapperService.merge("type2", new CompressedXContent(update.string()), MapperService.MergeReason.MAPPING_UPDATE, false); fail(); } catch (IllegalArgumentException e) { // expected @@ -163,7 +163,7 @@ public class UpdateMappingTests extends ESSingleNodeTestCase { } try { - mapperService.merge("type2", new CompressedXContent(update.string()), false, false); + mapperService.merge("type2", new CompressedXContent(update.string()), MapperService.MergeReason.MAPPING_UPDATE, false); fail(); } catch (IllegalArgumentException e) { // expected @@ -182,15 +182,15 @@ public class UpdateMappingTests extends ESSingleNodeTestCase { XContentBuilder mapping2 = XContentFactory.jsonBuilder().startObject().startObject("type2").endObject().endObject(); MapperService mapperService = createIndex("test", Settings.settingsBuilder().build()).mapperService(); - mapperService.merge("type1", new CompressedXContent(mapping1.string()), false, false); - mapperService.merge("type2", new CompressedXContent(mapping2.string()), false, false); + mapperService.merge("type1", new CompressedXContent(mapping1.string()), MapperService.MergeReason.MAPPING_UPDATE, false); + mapperService.merge("type2", new CompressedXContent(mapping2.string()), MapperService.MergeReason.MAPPING_UPDATE, false); XContentBuilder update = XContentFactory.jsonBuilder().startObject().startObject("type2") .startObject("properties").startObject("foo").field("type", "double").endObject() .endObject().endObject().endObject(); try { - mapperService.merge("type2", new CompressedXContent(update.string()), false, false); + mapperService.merge("type2", new CompressedXContent(update.string()), MapperService.MergeReason.MAPPING_UPDATE, false); fail(); } catch (IllegalArgumentException e) { // expected @@ -198,7 +198,7 @@ public class UpdateMappingTests extends ESSingleNodeTestCase { } try { - mapperService.merge("type2", new CompressedXContent(update.string()), false, false); + mapperService.merge("type2", new CompressedXContent(update.string()), MapperService.MergeReason.MAPPING_UPDATE, false); fail(); } catch (IllegalArgumentException e) { // expected @@ -217,14 +217,14 @@ public class UpdateMappingTests extends ESSingleNodeTestCase { MapperService mapperService = createIndex("test", Settings.settingsBuilder().build()).mapperService(); try { - mapperService.merge("type", new CompressedXContent(mapping.string()), false, false); + mapperService.merge("type", new CompressedXContent(mapping.string()), MapperService.MergeReason.MAPPING_UPDATE, false); fail(); } catch (IllegalArgumentException e) { assertTrue(e.getMessage().contains("Field [_id] is defined twice in [type]")); } try { - mapperService.merge("type", new CompressedXContent(mapping.string()), false, false); + mapperService.merge("type", new CompressedXContent(mapping.string()), MapperService.MergeReason.MAPPING_UPDATE, false); fail(); } catch (IllegalArgumentException e) { assertTrue(e.getMessage().contains("Field [_id] is defined twice in [type]")); @@ -241,14 +241,14 @@ public class UpdateMappingTests extends ESSingleNodeTestCase { MapperService mapperService = createIndex("test", settings).mapperService(); try { - mapperService.merge("type", new CompressedXContent(mapping.string()), false, false); + mapperService.merge("type", new CompressedXContent(mapping.string()), MapperService.MergeReason.MAPPING_UPDATE, false); fail(); } catch (IllegalArgumentException e) { assertTrue(e.getMessage().contains("Field [_id] is defined twice in [type]")); } try { - mapperService.merge("type", new CompressedXContent(mapping.string()), false, false); + mapperService.merge("type", new CompressedXContent(mapping.string()), MapperService.MergeReason.MAPPING_UPDATE, false); fail(); } catch (IllegalArgumentException e) { assertTrue(e.getMessage().contains("Field [_id] is defined twice in [type]")); diff --git a/core/src/test/java/org/elasticsearch/index/percolator/PercolatorFieldMapperTests.java b/core/src/test/java/org/elasticsearch/index/percolator/PercolatorFieldMapperTests.java index 5ce841540d13..9e46ce95041a 100644 --- a/core/src/test/java/org/elasticsearch/index/percolator/PercolatorFieldMapperTests.java +++ b/core/src/test/java/org/elasticsearch/index/percolator/PercolatorFieldMapperTests.java @@ -46,12 +46,12 @@ public class PercolatorFieldMapperTests extends ESSingleNodeTestCase { String mapper = XContentFactory.jsonBuilder().startObject().startObject("type") .startObject("properties").startObject("field").field("type", "string").endObject().endObject() .endObject().endObject().string(); - mapperService.merge("type", new CompressedXContent(mapper), true, true); + mapperService.merge("type", new CompressedXContent(mapper), MapperService.MergeReason.MAPPING_UPDATE, true); String percolatorMapper = XContentFactory.jsonBuilder().startObject().startObject(PercolatorService.TYPE_NAME) .startObject("properties").startObject("query").field("type", "percolator").endObject().endObject() .endObject().endObject().string(); - mapperService.merge(PercolatorService.TYPE_NAME, new CompressedXContent(percolatorMapper), true, true); + mapperService.merge(PercolatorService.TYPE_NAME, new CompressedXContent(percolatorMapper), MapperService.MergeReason.MAPPING_UPDATE, true); } public void testPercolatorFieldMapper() throws Exception { @@ -85,7 +85,7 @@ public class PercolatorFieldMapperTests extends ESSingleNodeTestCase { .startObject("properties").startObject("query").field("type", "percolator").field("index", "no").endObject().endObject() .endObject().endObject().string(); try { - mapperService.merge(PercolatorService.TYPE_NAME, new CompressedXContent(percolatorMapper), true, true); + mapperService.merge(PercolatorService.TYPE_NAME, new CompressedXContent(percolatorMapper), MapperService.MergeReason.MAPPING_UPDATE, true); fail("MapperParsingException expected"); } catch (MapperParsingException e) { assertThat(e.getMessage(), equalTo("Mapping definition for [query] has unsupported parameters: [index : no]")); diff --git a/core/src/test/java/org/elasticsearch/index/query/AbstractQueryTestCase.java b/core/src/test/java/org/elasticsearch/index/query/AbstractQueryTestCase.java index 62639c1a6ed9..cc54122d446a 100644 --- a/core/src/test/java/org/elasticsearch/index/query/AbstractQueryTestCase.java +++ b/core/src/test/java/org/elasticsearch/index/query/AbstractQueryTestCase.java @@ -22,6 +22,7 @@ package org.elasticsearch.index.query; import com.carrotsearch.randomizedtesting.generators.CodepointSetGenerator; import com.fasterxml.jackson.core.JsonParseException; import com.fasterxml.jackson.core.io.JsonStringEncoder; + import org.apache.lucene.search.BoostQuery; import org.apache.lucene.search.Query; import org.apache.lucene.search.TermQuery; @@ -46,7 +47,6 @@ import org.elasticsearch.common.bytes.BytesArray; import org.elasticsearch.common.bytes.BytesReference; import org.elasticsearch.common.collect.Tuple; import org.elasticsearch.common.compress.CompressedXContent; -import org.elasticsearch.common.geo.builders.ShapeBuilderRegistry; import org.elasticsearch.common.inject.AbstractModule; import org.elasticsearch.common.inject.Injector; import org.elasticsearch.common.inject.ModulesBuilder; @@ -74,8 +74,6 @@ import org.elasticsearch.index.analysis.AnalysisService; import org.elasticsearch.index.cache.bitset.BitsetFilterCache; import org.elasticsearch.index.fielddata.IndexFieldDataService; import org.elasticsearch.index.mapper.MapperService; -import org.elasticsearch.index.query.functionscore.ScoreFunctionParser; -import org.elasticsearch.index.query.functionscore.ScoreFunctionParserMapper; import org.elasticsearch.index.query.support.QueryParsers; import org.elasticsearch.index.shard.ShardId; import org.elasticsearch.index.similarity.SimilarityService; @@ -93,6 +91,7 @@ import org.elasticsearch.script.ScriptContextRegistry; import org.elasticsearch.script.ScriptEngineService; import org.elasticsearch.script.ScriptModule; import org.elasticsearch.script.ScriptService; +import org.elasticsearch.search.SearchModule; import org.elasticsearch.search.internal.SearchContext; import org.elasticsearch.test.ESTestCase; import org.elasticsearch.test.IndexSettingsModule; @@ -203,6 +202,7 @@ public abstract class AbstractQueryTestCase> Client.class.getClassLoader(), new Class[]{Client.class}, clientInvocationHandler); + namedWriteableRegistry = new NamedWriteableRegistry(); injector = new ModulesBuilder().add( new EnvironmentModule(new Environment(settings)), settingsModule, @@ -211,9 +211,7 @@ public abstract class AbstractQueryTestCase> @Override public void configure() { // skip services - bindQueryParsersExtension(); bindMapperExtension(); - bind(ShapeBuilderRegistry.class).asEagerSingleton(); } }, new ScriptModule(settings) { @@ -237,20 +235,26 @@ public abstract class AbstractQueryTestCase> } catch(IOException e) { throw new IllegalStateException("error while binding ScriptService", e); } - - } }, new IndexSettingsModule(index, indexSettings), + new SearchModule(settings, namedWriteableRegistry) { + @Override + protected void configureSearch() { + // Skip me + } + @Override + protected void configureSuggesters() { + // Skip me + } + }, new AbstractModule() { @Override protected void configure() { bind(Client.class).toInstance(proxy); - Multibinder.newSetBinder(binder(), ScoreFunctionParser.class); - bind(ScoreFunctionParserMapper.class).asEagerSingleton(); bind(ClusterService.class).toProvider(Providers.of(clusterService)); bind(CircuitBreakerService.class).to(NoneCircuitBreakerService.class); - bind(NamedWriteableRegistry.class).asEagerSingleton(); + bind(NamedWriteableRegistry.class).toInstance(namedWriteableRegistry); } } ).createInjector(); @@ -287,10 +291,11 @@ public abstract class AbstractQueryTestCase> OBJECT_FIELD_NAME, "type=object", GEO_POINT_FIELD_NAME, "type=geo_point,lat_lon=true,geohash=true,geohash_prefix=true", GEO_SHAPE_FIELD_NAME, "type=geo_shape" - ).string()), false, false); + ).string()), MapperService.MergeReason.MAPPING_UPDATE, false); // also add mappings for two inner field in the object field mapperService.merge(type, new CompressedXContent("{\"properties\":{\""+OBJECT_FIELD_NAME+"\":{\"type\":\"object\"," - + "\"properties\":{\""+DATE_FIELD_NAME+"\":{\"type\":\"date\"},\""+INT_FIELD_NAME+"\":{\"type\":\"integer\"}}}}}"), false, false); + + "\"properties\":{\""+DATE_FIELD_NAME+"\":{\"type\":\"date\"},\""+INT_FIELD_NAME+"\":{\"type\":\"integer\"}}}}}"), + MapperService.MergeReason.MAPPING_UPDATE, false); currentTypes[i] = type; } namedWriteableRegistry = injector.getInstance(NamedWriteableRegistry.class); @@ -486,7 +491,7 @@ public abstract class AbstractQueryTestCase> QueryParseContext context = createParseContext(); context.reset(parser); context.parseFieldMatcher(matcher); - QueryBuilder parseInnerQueryBuilder = context.parseInnerQueryBuilder(); + QueryBuilder parseInnerQueryBuilder = context.parseInnerQueryBuilder(); assertTrue(parser.nextToken() == null); return parseInnerQueryBuilder; } @@ -616,7 +621,7 @@ public abstract class AbstractQueryTestCase> testQuery.writeTo(output); try (StreamInput in = new NamedWriteableAwareStreamInput(StreamInput.wrap(output.bytes()), namedWriteableRegistry)) { QueryBuilder prototype = queryParser(testQuery.getName()).getBuilderPrototype(); - QueryBuilder deserializedQuery = prototype.readFrom(in); + QueryBuilder deserializedQuery = prototype.readFrom(in); assertEquals(deserializedQuery, testQuery); assertEquals(deserializedQuery.hashCode(), testQuery.hashCode()); assertNotSame(deserializedQuery, testQuery); @@ -848,7 +853,7 @@ public abstract class AbstractQueryTestCase> private static final List TIMEZONE_IDS = new ArrayList<>(DateTimeZone.getAvailableIDs()); private static class ClientInvocationHandler implements InvocationHandler { - AbstractQueryTestCase delegate; + AbstractQueryTestCase delegate; @Override public Object invoke(Object proxy, Method method, Object[] args) throws Throwable { if (method.equals(Client.class.getMethod("get", GetRequest.class))) { diff --git a/core/src/test/java/org/elasticsearch/index/query/BoolQueryBuilderTests.java b/core/src/test/java/org/elasticsearch/index/query/BoolQueryBuilderTests.java index 1c407fbaa0e2..d7740c765a73 100644 --- a/core/src/test/java/org/elasticsearch/index/query/BoolQueryBuilderTests.java +++ b/core/src/test/java/org/elasticsearch/index/query/BoolQueryBuilderTests.java @@ -106,16 +106,16 @@ public class BoolQueryBuilderTests extends AbstractQueryTestCase clauseIterator = clauses.iterator(); - for (BooleanClause booleanClause : booleanQuery.getClauses()) { + for (BooleanClause booleanClause : booleanQuery.clauses()) { assertThat(booleanClause, instanceOf(clauseIterator.next().getClass())); } } } } - private static List getBooleanClauses(List queryBuilders, BooleanClause.Occur occur, QueryShardContext context) throws IOException { + private static List getBooleanClauses(List> queryBuilders, BooleanClause.Occur occur, QueryShardContext context) throws IOException { List clauses = new ArrayList<>(); - for (QueryBuilder query : queryBuilders) { + for (QueryBuilder query : queryBuilders) { Query innerQuery = query.toQuery(context); if (innerQuery != null) { clauses.add(new BooleanClause(innerQuery, occur)); @@ -132,22 +132,22 @@ public class BoolQueryBuilderTests extends AbstractQueryTestCase 0) { - QueryBuilder must = tempQueryBuilder.must().get(0); + QueryBuilder must = tempQueryBuilder.must().get(0); contentString += "must: " + must.toString() + ","; expectedQuery.must(must); } if (tempQueryBuilder.mustNot().size() > 0) { - QueryBuilder mustNot = tempQueryBuilder.mustNot().get(0); + QueryBuilder mustNot = tempQueryBuilder.mustNot().get(0); contentString += (randomBoolean() ? "must_not: " : "mustNot: ") + mustNot.toString() + ","; expectedQuery.mustNot(mustNot); } if (tempQueryBuilder.should().size() > 0) { - QueryBuilder should = tempQueryBuilder.should().get(0); + QueryBuilder should = tempQueryBuilder.should().get(0); contentString += "should: " + should.toString() + ","; expectedQuery.should(should); } if (tempQueryBuilder.filter().size() > 0) { - QueryBuilder filter = tempQueryBuilder.filter().get(0); + QueryBuilder filter = tempQueryBuilder.filter().get(0); contentString += "filter: " + filter.toString() + ","; expectedQuery.filter(filter); } diff --git a/core/src/test/java/org/elasticsearch/index/query/DisMaxQueryBuilderTests.java b/core/src/test/java/org/elasticsearch/index/query/DisMaxQueryBuilderTests.java index f58d0f926a12..810cca0b240e 100644 --- a/core/src/test/java/org/elasticsearch/index/query/DisMaxQueryBuilderTests.java +++ b/core/src/test/java/org/elasticsearch/index/query/DisMaxQueryBuilderTests.java @@ -74,7 +74,7 @@ public class DisMaxQueryBuilderTests extends AbstractQueryTestCase getAlternateVersions() { Map alternateVersions = new HashMap<>(); - QueryBuilder innerQuery = createTestQueryBuilder().innerQueries().get(0); + QueryBuilder innerQuery = createTestQueryBuilder().innerQueries().get(0); DisMaxQueryBuilder expectedQuery = new DisMaxQueryBuilder(); expectedQuery.add(innerQuery); String contentString = "{\n" + diff --git a/core/src/test/java/org/elasticsearch/index/query/HasChildQueryBuilderTests.java b/core/src/test/java/org/elasticsearch/index/query/HasChildQueryBuilderTests.java index f2b3a1a50263..15d37150b3f1 100644 --- a/core/src/test/java/org/elasticsearch/index/query/HasChildQueryBuilderTests.java +++ b/core/src/test/java/org/elasticsearch/index/query/HasChildQueryBuilderTests.java @@ -45,6 +45,7 @@ import org.elasticsearch.search.fetch.innerhits.InnerHitsContext; import org.elasticsearch.search.internal.SearchContext; import org.elasticsearch.search.sort.SortOrder; import org.elasticsearch.test.TestSearchContext; +import org.junit.BeforeClass; import java.io.IOException; import java.util.Collections; @@ -57,9 +58,8 @@ public class HasChildQueryBuilderTests extends AbstractQueryTestCase spanQueryBuilderIterator = queryBuilder.clauses().iterator(); + Iterator> spanQueryBuilderIterator = queryBuilder.clauses().iterator(); for (SpanQuery spanQuery : spanNearQuery.getClauses()) { assertThat(spanQuery, equalTo(spanQueryBuilderIterator.next().toQuery(context))); } diff --git a/core/src/test/java/org/elasticsearch/index/query/SpanOrQueryBuilderTests.java b/core/src/test/java/org/elasticsearch/index/query/SpanOrQueryBuilderTests.java index 7a8b66139e37..c3d7be9ae9e4 100644 --- a/core/src/test/java/org/elasticsearch/index/query/SpanOrQueryBuilderTests.java +++ b/core/src/test/java/org/elasticsearch/index/query/SpanOrQueryBuilderTests.java @@ -45,7 +45,7 @@ public class SpanOrQueryBuilderTests extends AbstractQueryTestCase spanQueryBuilderIterator = queryBuilder.clauses().iterator(); + Iterator> spanQueryBuilderIterator = queryBuilder.clauses().iterator(); for (SpanQuery spanQuery : spanOrQuery.getClauses()) { assertThat(spanQuery, equalTo(spanQueryBuilderIterator.next().toQuery(context))); } @@ -70,32 +70,32 @@ public class SpanOrQueryBuilderTests extends AbstractQueryTestCase)queryBuilder).buildAsBytes(XContentType.values()[i])); } } } @@ -414,7 +415,7 @@ public class FunctionScoreQueryBuilderTests extends AbstractQueryTestCase)queryBuilder).buildAsBytes(XContentType.values()[i])); } } } diff --git a/core/src/test/java/org/elasticsearch/index/query/plugin/DummyQueryParserPlugin.java b/core/src/test/java/org/elasticsearch/index/query/plugin/DummyQueryParserPlugin.java index c72470c0cce5..98dbf67de574 100644 --- a/core/src/test/java/org/elasticsearch/index/query/plugin/DummyQueryParserPlugin.java +++ b/core/src/test/java/org/elasticsearch/index/query/plugin/DummyQueryParserPlugin.java @@ -19,6 +19,8 @@ package org.elasticsearch.index.query.plugin; +import java.io.IOException; + import org.apache.lucene.search.IndexSearcher; import org.apache.lucene.search.MatchAllDocsQuery; import org.apache.lucene.search.Query; @@ -31,10 +33,8 @@ import org.elasticsearch.index.query.AbstractQueryBuilder; import org.elasticsearch.index.query.QueryParseContext; import org.elasticsearch.index.query.QueryParser; import org.elasticsearch.index.query.QueryShardContext; -import org.elasticsearch.indices.IndicesModule; import org.elasticsearch.plugins.Plugin; - -import java.io.IOException; +import org.elasticsearch.search.SearchModule; public class DummyQueryParserPlugin extends Plugin { @@ -48,8 +48,8 @@ public class DummyQueryParserPlugin extends Plugin { return "dummy query"; } - public void onModule(IndicesModule module) { - module.registerQueryParser(DummyQueryParser.class); + public void onModule(SearchModule module) { + module.registerQueryParser(DummyQueryParser::new); } public static class DummyQueryBuilder extends AbstractQueryBuilder { diff --git a/core/src/test/java/org/elasticsearch/index/search/MultiMatchQueryTests.java b/core/src/test/java/org/elasticsearch/index/search/MultiMatchQueryTests.java index 236198261d77..46c8d3745cab 100644 --- a/core/src/test/java/org/elasticsearch/index/search/MultiMatchQueryTests.java +++ b/core/src/test/java/org/elasticsearch/index/search/MultiMatchQueryTests.java @@ -64,7 +64,7 @@ public class MultiMatchQueryTests extends ESSingleNodeTestCase { " }\n" + " }\n" + "}"; - mapperService.merge("person", new CompressedXContent(mapping), true, false); + mapperService.merge("person", new CompressedXContent(mapping), MapperService.MergeReason.MAPPING_UPDATE, false); this.indexService = indexService; } diff --git a/core/src/test/java/org/elasticsearch/index/translog/TranslogTests.java b/core/src/test/java/org/elasticsearch/index/translog/TranslogTests.java index da5e5c8aa08e..e1935328232f 100644 --- a/core/src/test/java/org/elasticsearch/index/translog/TranslogTests.java +++ b/core/src/test/java/org/elasticsearch/index/translog/TranslogTests.java @@ -138,8 +138,8 @@ public class TranslogTests extends ESTestCase { private TranslogConfig getTranslogConfig(Path path) { Settings build = Settings.settingsBuilder() - .put(IndexMetaData.SETTING_VERSION_CREATED, org.elasticsearch.Version.CURRENT) - .build(); + .put(IndexMetaData.SETTING_VERSION_CREATED, org.elasticsearch.Version.CURRENT) + .build(); ByteSizeValue bufferSize = randomBoolean() ? TranslogConfig.DEFAULT_BUFFER_SIZE : new ByteSizeValue(10 + randomInt(128 * 1024), ByteSizeUnit.BYTES); return new TranslogConfig(shardId, path, IndexSettingsModule.newIndexSettings(shardId.index(), build), BigArrays.NON_RECYCLING_INSTANCE, bufferSize); } @@ -234,19 +234,16 @@ public class TranslogTests extends ESTestCase { ArrayList ops = new ArrayList<>(); Translog.Snapshot snapshot = translog.newSnapshot(); assertThat(snapshot, SnapshotMatchers.size(0)); - snapshot.close(); addToTranslogAndList(translog, ops, new Translog.Index("test", "1", new byte[]{1})); snapshot = translog.newSnapshot(); assertThat(snapshot, SnapshotMatchers.equalsTo(ops)); - assertThat(snapshot.estimatedTotalOperations(), equalTo(ops.size())); - snapshot.close(); + assertThat(snapshot.totalOperations(), equalTo(ops.size())); addToTranslogAndList(translog, ops, new Translog.Delete(newUid("2"))); snapshot = translog.newSnapshot(); assertThat(snapshot, SnapshotMatchers.equalsTo(ops)); - assertThat(snapshot.estimatedTotalOperations(), equalTo(ops.size())); - snapshot.close(); + assertThat(snapshot.totalOperations(), equalTo(ops.size())); snapshot = translog.newSnapshot(); @@ -260,22 +257,18 @@ public class TranslogTests extends ESTestCase { assertThat(snapshot.next(), equalTo(null)); - snapshot.close(); - long firstId = translog.currentFileGeneration(); translog.prepareCommit(); assertThat(translog.currentFileGeneration(), Matchers.not(equalTo(firstId))); snapshot = translog.newSnapshot(); assertThat(snapshot, SnapshotMatchers.equalsTo(ops)); - assertThat(snapshot.estimatedTotalOperations(), equalTo(ops.size())); - snapshot.close(); + assertThat(snapshot.totalOperations(), equalTo(ops.size())); translog.commit(); snapshot = translog.newSnapshot(); assertThat(snapshot, SnapshotMatchers.size(0)); - assertThat(snapshot.estimatedTotalOperations(), equalTo(0)); - snapshot.close(); + assertThat(snapshot.totalOperations(), equalTo(0)); } protected TranslogStats stats() throws IOException { @@ -337,9 +330,9 @@ public class TranslogTests extends ESTestCase { assertEquals(6, copy.estimatedNumberOfOperations()); assertEquals(431, copy.getTranslogSizeInBytes()); assertEquals("\"translog\"{\n" + - " \"operations\" : 6,\n" + - " \"size_in_bytes\" : 431\n" + - "}", copy.toString().trim()); + " \"operations\" : 6,\n" + + " \"size_in_bytes\" : 431\n" + + "}", copy.toString().trim()); try { new TranslogStats(1, -1); @@ -359,51 +352,43 @@ public class TranslogTests extends ESTestCase { ArrayList ops = new ArrayList<>(); Translog.Snapshot snapshot = translog.newSnapshot(); assertThat(snapshot, SnapshotMatchers.size(0)); - snapshot.close(); addToTranslogAndList(translog, ops, new Translog.Index("test", "1", new byte[]{1})); snapshot = translog.newSnapshot(); assertThat(snapshot, SnapshotMatchers.equalsTo(ops)); - assertThat(snapshot.estimatedTotalOperations(), equalTo(1)); - snapshot.close(); + assertThat(snapshot.totalOperations(), equalTo(1)); snapshot = translog.newSnapshot(); - assertThat(snapshot, SnapshotMatchers.equalsTo(ops)); - assertThat(snapshot.estimatedTotalOperations(), equalTo(1)); - - // snapshot while another is open Translog.Snapshot snapshot1 = translog.newSnapshot(); - assertThat(snapshot1, SnapshotMatchers.size(1)); - assertThat(snapshot1.estimatedTotalOperations(), equalTo(1)); + assertThat(snapshot, SnapshotMatchers.equalsTo(ops)); + assertThat(snapshot.totalOperations(), equalTo(1)); - snapshot.close(); - snapshot1.close(); + assertThat(snapshot1, SnapshotMatchers.size(1)); + assertThat(snapshot1.totalOperations(), equalTo(1)); } public void testSnapshotWithNewTranslog() throws IOException { ArrayList ops = new ArrayList<>(); Translog.Snapshot snapshot = translog.newSnapshot(); assertThat(snapshot, SnapshotMatchers.size(0)); - snapshot.close(); addToTranslogAndList(translog, ops, new Translog.Index("test", "1", new byte[]{1})); Translog.Snapshot snapshot1 = translog.newSnapshot(); addToTranslogAndList(translog, ops, new Translog.Index("test", "2", new byte[]{2})); + assertThat(snapshot1, SnapshotMatchers.equalsTo(ops.get(0))); + translog.prepareCommit(); addToTranslogAndList(translog, ops, new Translog.Index("test", "3", new byte[]{3})); - Translog.Snapshot snapshot2 = translog.newSnapshot(); - translog.commit(); - assertThat(snapshot2, SnapshotMatchers.equalsTo(ops)); - assertThat(snapshot2.estimatedTotalOperations(), equalTo(ops.size())); - - - assertThat(snapshot1, SnapshotMatchers.equalsTo(ops.get(0))); - snapshot1.close(); - snapshot2.close(); + try (Translog.View view = translog.newView()) { + Translog.Snapshot snapshot2 = translog.newSnapshot(); + translog.commit(); + assertThat(snapshot2, SnapshotMatchers.equalsTo(ops)); + assertThat(snapshot2.totalOperations(), equalTo(ops.size())); + } } public void testSnapshotOnClosedTranslog() throws IOException { @@ -418,39 +403,6 @@ public class TranslogTests extends ESTestCase { } } - public void testDeleteOnSnapshotRelease() throws Exception { - ArrayList firstOps = new ArrayList<>(); - addToTranslogAndList(translog, firstOps, new Translog.Index("test", "1", new byte[]{1})); - - Translog.Snapshot firstSnapshot = translog.newSnapshot(); - assertThat(firstSnapshot.estimatedTotalOperations(), equalTo(1)); - translog.commit(); - assertFileIsPresent(translog, 1); - - - ArrayList secOps = new ArrayList<>(); - addToTranslogAndList(translog, secOps, new Translog.Index("test", "2", new byte[]{2})); - assertThat(firstSnapshot.estimatedTotalOperations(), equalTo(1)); - - Translog.Snapshot secondSnapshot = translog.newSnapshot(); - translog.add(new Translog.Index("test", "3", new byte[]{3})); - assertThat(secondSnapshot, SnapshotMatchers.equalsTo(secOps)); - assertThat(secondSnapshot.estimatedTotalOperations(), equalTo(1)); - assertFileIsPresent(translog, 1); - assertFileIsPresent(translog, 2); - - firstSnapshot.close(); - assertFileDeleted(translog, 1); - assertFileIsPresent(translog, 2); - secondSnapshot.close(); - assertFileIsPresent(translog, 2); // it's the current nothing should be deleted - translog.commit(); - assertFileIsPresent(translog, 3); // it's the current nothing should be deleted - assertFileDeleted(translog, 2); - - } - - public void assertFileIsPresent(Translog translog, long id) { if (Files.exists(translogDir.resolve(Translog.getFilename(id)))) { return; @@ -624,14 +576,8 @@ public class TranslogTests extends ESTestCase { Translog.Snapshot snapshot = translog.newSnapshot(); assertThat(snapshot, SnapshotMatchers.size(1)); assertFileIsPresent(translog, 1); - assertThat(snapshot.estimatedTotalOperations(), equalTo(1)); - if (randomBoolean()) { - translog.close(); - snapshot.close(); - } else { - snapshot.close(); - translog.close(); - } + assertThat(snapshot.totalOperations(), equalTo(1)); + translog.close(); assertFileIsPresent(translog, 1); } @@ -708,16 +654,21 @@ public class TranslogTests extends ESTestCase { public void onFailure(Throwable t) { logger.error("--> reader [{}] had an error", t, threadId); errors.add(t); - closeView(); + try { + closeView(); + } catch (IOException e) { + logger.error("unexpected error while closing view, after failure"); + t.addSuppressed(e); + } } - void closeView() { + void closeView() throws IOException { if (view != null) { view.close(); } } - void newView() { + void newView() throws IOException { closeView(); view = translog.newView(); // captures the currently written ops so we know what to expect from the view @@ -738,17 +689,16 @@ public class TranslogTests extends ESTestCase { // these are what we expect the snapshot to return (and potentially some more). Set expectedOps = new HashSet<>(writtenOps.keySet()); expectedOps.removeAll(writtenOpsAtView); - try (Translog.Snapshot snapshot = view.snapshot()) { - Translog.Operation op; - while ((op = snapshot.next()) != null) { - expectedOps.remove(op); - } + Translog.Snapshot snapshot = view.snapshot(); + Translog.Operation op; + while ((op = snapshot.next()) != null) { + expectedOps.remove(op); } if (expectedOps.isEmpty() == false) { StringBuilder missed = new StringBuilder("missed ").append(expectedOps.size()).append(" operations"); boolean failed = false; - for (Translog.Operation op : expectedOps) { - final Translog.Location loc = writtenOps.get(op); + for (Translog.Operation expectedOp : expectedOps) { + final Translog.Location loc = writtenOps.get(expectedOp); if (loc.generation < view.minTranslogGeneration()) { // writtenOps is only updated after the op was written to the translog. This mean // that ops written to the translog before the view was taken (and will be missing from the view) @@ -756,7 +706,7 @@ public class TranslogTests extends ESTestCase { continue; } failed = true; - missed.append("\n --> [").append(op).append("] written at ").append(loc); + missed.append("\n --> [").append(expectedOp).append("] written at ").append(loc); } if (failed) { fail(missed.toString()); @@ -803,7 +753,6 @@ public class TranslogTests extends ESTestCase { } } - public void testSyncUpTo() throws IOException { int translogOperations = randomIntBetween(10, 100); int count = 0; @@ -875,7 +824,7 @@ public class TranslogTests extends ESTestCase { final Translog.Location lastLocation = translog.add(new Translog.Index("test", "" + translogOperations, Integer.toString(translogOperations).getBytes(Charset.forName("UTF-8")))); final Checkpoint checkpoint = Checkpoint.read(translog.location().resolve(Translog.CHECKPOINT_FILE_NAME)); - try (final ImmutableTranslogReader reader = translog.openReader(translog.location().resolve(Translog.getFilename(translog.currentFileGeneration())), checkpoint)) { + try (final TranslogReader reader = translog.openReader(translog.location().resolve(Translog.getFilename(translog.currentFileGeneration())), checkpoint)) { assertEquals(lastSynced + 1, reader.totalOperations()); for (int op = 0; op < translogOperations; op++) { Translog.Location location = locations.get(op); @@ -913,7 +862,7 @@ public class TranslogTests extends ESTestCase { } writer.sync(); - final TranslogReader reader = randomBoolean() ? writer : translog.openReader(writer.path(), Checkpoint.read(translog.location().resolve(Translog.CHECKPOINT_FILE_NAME))); + final BaseTranslogReader reader = randomBoolean() ? writer : translog.openReader(writer.path(), Checkpoint.read(translog.location().resolve(Translog.CHECKPOINT_FILE_NAME))); for (int i = 0; i < numOps; i++) { ByteBuffer buffer = ByteBuffer.allocate(4); reader.readBytes(buffer, reader.getFirstOperationOffset() + 4 * i); @@ -926,7 +875,7 @@ public class TranslogTests extends ESTestCase { out.writeInt(2048); writer.add(new BytesArray(bytes)); - if (reader instanceof ImmutableTranslogReader) { + if (reader instanceof TranslogReader) { ByteBuffer buffer = ByteBuffer.allocate(4); try { reader.readBytes(buffer, reader.getFirstOperationOffset() + 4 * numOps); @@ -934,6 +883,7 @@ public class TranslogTests extends ESTestCase { } catch (EOFException ex) { // expected } + ((TranslogReader) reader).close(); } else { // live reader! ByteBuffer buffer = ByteBuffer.allocate(4); @@ -943,7 +893,7 @@ public class TranslogTests extends ESTestCase { final int value = buffer.getInt(); assertEquals(2048, value); } - IOUtils.close(writer, reader); + IOUtils.close(writer); } public void testBasicRecovery() throws IOException { @@ -971,19 +921,17 @@ public class TranslogTests extends ESTestCase { assertEquals(0, translog.stats().estimatedNumberOfOperations()); assertEquals(1, translog.currentFileGeneration()); assertFalse(translog.syncNeeded()); - try (Translog.Snapshot snapshot = translog.newSnapshot()) { - assertNull(snapshot.next()); - } + Translog.Snapshot snapshot = translog.newSnapshot(); + assertNull(snapshot.next()); } else { assertEquals("lastCommitted must be 1 less than current", translogGeneration.translogFileGeneration + 1, translog.currentFileGeneration()); assertFalse(translog.syncNeeded()); - try (Translog.Snapshot snapshot = translog.newSnapshot()) { - for (int i = minUncommittedOp; i < translogOperations; i++) { - assertEquals("expected operation" + i + " to be in the previous translog but wasn't", translog.currentFileGeneration() - 1, locations.get(i).generation); - Translog.Operation next = snapshot.next(); - assertNotNull("operation " + i + " must be non-null", next); - assertEquals(i, Integer.parseInt(next.getSource().source.toUtf8())); - } + Translog.Snapshot snapshot = translog.newSnapshot(); + for (int i = minUncommittedOp; i < translogOperations; i++) { + assertEquals("expected operation" + i + " to be in the previous translog but wasn't", translog.currentFileGeneration() - 1, locations.get(i).generation); + Translog.Operation next = snapshot.next(); + assertNotNull("operation " + i + " must be non-null", next); + assertEquals(i, Integer.parseInt(next.getSource().source.toUtf8())); } } } @@ -1014,13 +962,12 @@ public class TranslogTests extends ESTestCase { assertNotNull(translogGeneration); assertEquals("lastCommitted must be 2 less than current - we never finished the commit", translogGeneration.translogFileGeneration + 2, translog.currentFileGeneration()); assertFalse(translog.syncNeeded()); - try (Translog.Snapshot snapshot = translog.newSnapshot()) { - int upTo = sync ? translogOperations : prepareOp; - for (int i = 0; i < upTo; i++) { - Translog.Operation next = snapshot.next(); - assertNotNull("operation " + i + " must be non-null synced: " + sync, next); - assertEquals("payload missmatch, synced: " + sync, i, Integer.parseInt(next.getSource().source.toUtf8())); - } + Translog.Snapshot snapshot = translog.newSnapshot(); + int upTo = sync ? translogOperations : prepareOp; + for (int i = 0; i < upTo; i++) { + Translog.Operation next = snapshot.next(); + assertNotNull("operation " + i + " must be non-null synced: " + sync, next); + assertEquals("payload missmatch, synced: " + sync, i, Integer.parseInt(next.getSource().source.toUtf8())); } } if (randomBoolean()) { // recover twice @@ -1028,13 +975,12 @@ public class TranslogTests extends ESTestCase { assertNotNull(translogGeneration); assertEquals("lastCommitted must be 3 less than current - we never finished the commit and run recovery twice", translogGeneration.translogFileGeneration + 3, translog.currentFileGeneration()); assertFalse(translog.syncNeeded()); - try (Translog.Snapshot snapshot = translog.newSnapshot()) { - int upTo = sync ? translogOperations : prepareOp; - for (int i = 0; i < upTo; i++) { - Translog.Operation next = snapshot.next(); - assertNotNull("operation " + i + " must be non-null synced: " + sync, next); - assertEquals("payload missmatch, synced: " + sync, i, Integer.parseInt(next.getSource().source.toUtf8())); - } + Translog.Snapshot snapshot = translog.newSnapshot(); + int upTo = sync ? translogOperations : prepareOp; + for (int i = 0; i < upTo; i++) { + Translog.Operation next = snapshot.next(); + assertNotNull("operation " + i + " must be non-null synced: " + sync, next); + assertEquals("payload missmatch, synced: " + sync, i, Integer.parseInt(next.getSource().source.toUtf8())); } } } @@ -1071,14 +1017,14 @@ public class TranslogTests extends ESTestCase { assertNotNull(translogGeneration); assertEquals("lastCommitted must be 2 less than current - we never finished the commit", translogGeneration.translogFileGeneration + 2, translog.currentFileGeneration()); assertFalse(translog.syncNeeded()); - try (Translog.Snapshot snapshot = translog.newSnapshot()) { - int upTo = sync ? translogOperations : prepareOp; - for (int i = 0; i < upTo; i++) { - Translog.Operation next = snapshot.next(); - assertNotNull("operation " + i + " must be non-null synced: " + sync, next); - assertEquals("payload missmatch, synced: " + sync, i, Integer.parseInt(next.getSource().source.toUtf8())); - } + Translog.Snapshot snapshot = translog.newSnapshot(); + int upTo = sync ? translogOperations : prepareOp; + for (int i = 0; i < upTo; i++) { + Translog.Operation next = snapshot.next(); + assertNotNull("operation " + i + " must be non-null synced: " + sync, next); + assertEquals("payload missmatch, synced: " + sync, i, Integer.parseInt(next.getSource().source.toUtf8())); } + } if (randomBoolean()) { // recover twice @@ -1086,13 +1032,12 @@ public class TranslogTests extends ESTestCase { assertNotNull(translogGeneration); assertEquals("lastCommitted must be 3 less than current - we never finished the commit and run recovery twice", translogGeneration.translogFileGeneration + 3, translog.currentFileGeneration()); assertFalse(translog.syncNeeded()); - try (Translog.Snapshot snapshot = translog.newSnapshot()) { - int upTo = sync ? translogOperations : prepareOp; - for (int i = 0; i < upTo; i++) { - Translog.Operation next = snapshot.next(); - assertNotNull("operation " + i + " must be non-null synced: " + sync, next); - assertEquals("payload missmatch, synced: " + sync, i, Integer.parseInt(next.getSource().source.toUtf8())); - } + Translog.Snapshot snapshot = translog.newSnapshot(); + int upTo = sync ? translogOperations : prepareOp; + for (int i = 0; i < upTo; i++) { + Translog.Operation next = snapshot.next(); + assertNotNull("operation " + i + " must be non-null synced: " + sync, next); + assertEquals("payload missmatch, synced: " + sync, i, Integer.parseInt(next.getSource().source.toUtf8())); } } } @@ -1132,13 +1077,12 @@ public class TranslogTests extends ESTestCase { assertNotNull(translogGeneration); assertEquals("lastCommitted must be 2 less than current - we never finished the commit", translogGeneration.translogFileGeneration + 2, translog.currentFileGeneration()); assertFalse(translog.syncNeeded()); - try (Translog.Snapshot snapshot = translog.newSnapshot()) { - int upTo = sync ? translogOperations : prepareOp; - for (int i = 0; i < upTo; i++) { - Translog.Operation next = snapshot.next(); - assertNotNull("operation " + i + " must be non-null synced: " + sync, next); - assertEquals("payload missmatch, synced: " + sync, i, Integer.parseInt(next.getSource().source.toUtf8())); - } + Translog.Snapshot snapshot = translog.newSnapshot(); + int upTo = sync ? translogOperations : prepareOp; + for (int i = 0; i < upTo; i++) { + Translog.Operation next = snapshot.next(); + assertNotNull("operation " + i + " must be non-null synced: " + sync, next); + assertEquals("payload missmatch, synced: " + sync, i, Integer.parseInt(next.getSource().source.toUtf8())); } } } @@ -1209,14 +1153,13 @@ public class TranslogTests extends ESTestCase { } config.setTranslogGeneration(translogGeneration); this.translog = new Translog(config); - try (Translog.Snapshot snapshot = this.translog.newSnapshot()) { - for (int i = firstUncommitted; i < translogOperations; i++) { - Translog.Operation next = snapshot.next(); - assertNotNull("" + i, next); - assertEquals(Integer.parseInt(next.getSource().source.toUtf8()), i); - } - assertNull(snapshot.next()); + Translog.Snapshot snapshot = this.translog.newSnapshot(); + for (int i = firstUncommitted; i < translogOperations; i++) { + Translog.Operation next = snapshot.next(); + assertNotNull("" + i, next); + assertEquals(Integer.parseInt(next.getSource().source.toUtf8()), i); } + assertNull(snapshot.next()); } public void testFailOnClosedWrite() throws IOException { @@ -1287,12 +1230,12 @@ public class TranslogTests extends ESTestCase { case CREATE: case INDEX: op = new Translog.Index("test", threadId + "_" + opCount, - randomUnicodeOfLengthBetween(1, 20 * 1024).getBytes("UTF-8")); + randomUnicodeOfLengthBetween(1, 20 * 1024).getBytes("UTF-8")); break; case DELETE: op = new Translog.Delete(new Term("_uid", threadId + "_" + opCount), - 1 + randomInt(100000), - randomFrom(VersionType.values())); + 1 + randomInt(100000), + randomFrom(VersionType.values())); break; default: throw new ElasticsearchException("not supported op type"); @@ -1383,14 +1326,13 @@ public class TranslogTests extends ESTestCase { assertEquals("lastCommitted must be 1 less than current", translogGeneration.translogFileGeneration + 1, tlog.currentFileGeneration()); assertFalse(tlog.syncNeeded()); - try (Translog.Snapshot snapshot = tlog.newSnapshot()) { - assertEquals(opsSynced, snapshot.estimatedTotalOperations()); - for (int i = 0; i < opsSynced; i++) { - assertEquals("expected operation" + i + " to be in the previous translog but wasn't", tlog.currentFileGeneration() - 1, locations.get(i).generation); - Translog.Operation next = snapshot.next(); - assertNotNull("operation " + i + " must be non-null", next); - assertEquals(i, Integer.parseInt(next.getSource().source.toUtf8())); - } + Translog.Snapshot snapshot = tlog.newSnapshot(); + assertEquals(opsSynced, snapshot.totalOperations()); + for (int i = 0; i < opsSynced; i++) { + assertEquals("expected operation" + i + " to be in the previous translog but wasn't", tlog.currentFileGeneration() - 1, locations.get(i).generation); + Translog.Operation next = snapshot.next(); + assertNotNull("operation " + i + " must be non-null", next); + assertEquals(i, Integer.parseInt(next.getSource().source.toUtf8())); } } } @@ -1401,13 +1343,12 @@ public class TranslogTests extends ESTestCase { LineFileDocs lineFileDocs = new LineFileDocs(random()); // writes pretty big docs so we cross buffer boarders regularly for (int opsAdded = 0; opsAdded < numOps; opsAdded++) { locations.add(translog.add(new Translog.Index("test", "" + opsAdded, lineFileDocs.nextDoc().toString().getBytes(Charset.forName("UTF-8"))))); - try (Translog.Snapshot snapshot = translog.newSnapshot()) { - assertEquals(opsAdded + 1, snapshot.estimatedTotalOperations()); - for (int i = 0; i < opsAdded; i++) { - assertEquals("expected operation" + i + " to be in the current translog but wasn't", translog.currentFileGeneration(), locations.get(i).generation); - Translog.Operation next = snapshot.next(); - assertNotNull("operation " + i + " must be non-null", next); - } + Translog.Snapshot snapshot = this.translog.newSnapshot(); + assertEquals(opsAdded + 1, snapshot.totalOperations()); + for (int i = 0; i < opsAdded; i++) { + assertEquals("expected operation" + i + " to be in the current translog but wasn't", translog.currentFileGeneration(), locations.get(i).generation); + Translog.Operation next = snapshot.next(); + assertNotNull("operation " + i + " must be non-null", next); } } } @@ -1511,20 +1452,20 @@ public class TranslogTests extends ESTestCase { } config.setTranslogGeneration(translog.getGeneration()); try (Translog tlog = new Translog(config)) { - try (Translog.Snapshot snapshot = tlog.newSnapshot()) { - if (writtenOperations.size() != snapshot.estimatedTotalOperations()) { - for (int i = 0; i < threadCount; i++) { - if (threadExceptions[i] != null) - threadExceptions[i].printStackTrace(); + Translog.Snapshot snapshot = tlog.newSnapshot(); + if (writtenOperations.size() != snapshot.totalOperations()) { + for (int i = 0; i < threadCount; i++) { + if (threadExceptions[i] != null) { + threadExceptions[i].printStackTrace(); } } - assertEquals(writtenOperations.size(), snapshot.estimatedTotalOperations()); - for (int i = 0; i < writtenOperations.size(); i++) { - assertEquals("expected operation" + i + " to be in the previous translog but wasn't", tlog.currentFileGeneration() - 1, writtenOperations.get(i).location.generation); - Translog.Operation next = snapshot.next(); - assertNotNull("operation " + i + " must be non-null", next); - assertEquals(next, writtenOperations.get(i).operation); - } + } + assertEquals(writtenOperations.size(), snapshot.totalOperations()); + for (int i = 0; i < writtenOperations.size(); i++) { + assertEquals("expected operation" + i + " to be in the previous translog but wasn't", tlog.currentFileGeneration() - 1, writtenOperations.get(i).location.generation); + Translog.Operation next = snapshot.next(); + assertNotNull("operation " + i + " must be non-null", next); + assertEquals(next, writtenOperations.get(i).operation); } } } @@ -1537,6 +1478,7 @@ public class TranslogTests extends ESTestCase { private static class FailSwitch { private volatile int failRate; private volatile boolean onceFailedFailAlways = false; + public boolean fail() { boolean fail = randomIntBetween(1, 100) <= failRate; if (fail && onceFailedFailAlways) { @@ -1716,24 +1658,22 @@ public class TranslogTests extends ESTestCase { try (Translog tlog = new Translog(config)) { assertNotNull(translogGeneration); assertFalse(tlog.syncNeeded()); - try (Translog.Snapshot snapshot = tlog.newSnapshot()) { - for (int i = 0; i < 1; i++) { - Translog.Operation next = snapshot.next(); - assertNotNull("operation " + i + " must be non-null", next); - assertEquals("payload missmatch", i, Integer.parseInt(next.getSource().source.toUtf8())); - } + Translog.Snapshot snapshot = tlog.newSnapshot(); + for (int i = 0; i < 1; i++) { + Translog.Operation next = snapshot.next(); + assertNotNull("operation " + i + " must be non-null", next); + assertEquals("payload missmatch", i, Integer.parseInt(next.getSource().source.toUtf8())); } tlog.add(new Translog.Index("test", "" + 1, Integer.toString(1).getBytes(Charset.forName("UTF-8")))); } try (Translog tlog = new Translog(config)) { assertNotNull(translogGeneration); assertFalse(tlog.syncNeeded()); - try (Translog.Snapshot snapshot = tlog.newSnapshot()) { - for (int i = 0; i < 2; i++) { - Translog.Operation next = snapshot.next(); - assertNotNull("operation " + i + " must be non-null", next); - assertEquals("payload missmatch", i, Integer.parseInt(next.getSource().source.toUtf8())); - } + Translog.Snapshot snapshot = tlog.newSnapshot(); + for (int i = 0; i < 2; i++) { + Translog.Operation next = snapshot.next(); + assertNotNull("operation " + i + " must be non-null", next); + assertEquals("payload missmatch", i, Integer.parseInt(next.getSource().source.toUtf8())); } } } @@ -1749,7 +1689,7 @@ public class TranslogTests extends ESTestCase { Files.createFile(config.getTranslogPath().resolve("translog-" + (read.generation + 1) + ".tlog")); config.setTranslogGeneration(translogGeneration); - try { + try { Translog tlog = new Translog(config); fail("file already exists?"); } catch (TranslogException ex) { @@ -1758,6 +1698,7 @@ public class TranslogTests extends ESTestCase { assertEquals(ex.getCause().getClass(), FileAlreadyExistsException.class); } } + public void testRecoverWithUnbackedNextGenAndFutureFile() throws IOException { translog.add(new Translog.Index("test", "" + 0, Integer.toString(0).getBytes(Charset.forName("UTF-8")))); Translog.TranslogGeneration translogGeneration = translog.getGeneration(); @@ -1774,17 +1715,16 @@ public class TranslogTests extends ESTestCase { try (Translog tlog = new Translog(config)) { assertNotNull(translogGeneration); assertFalse(tlog.syncNeeded()); - try (Translog.Snapshot snapshot = tlog.newSnapshot()) { - for (int i = 0; i < 1; i++) { - Translog.Operation next = snapshot.next(); - assertNotNull("operation " + i + " must be non-null", next); - assertEquals("payload missmatch", i, Integer.parseInt(next.getSource().source.toUtf8())); - } + Translog.Snapshot snapshot = tlog.newSnapshot(); + for (int i = 0; i < 1; i++) { + Translog.Operation next = snapshot.next(); + assertNotNull("operation " + i + " must be non-null", next); + assertEquals("payload missmatch", i, Integer.parseInt(next.getSource().source.toUtf8())); } tlog.add(new Translog.Index("test", "" + 1, Integer.toString(1).getBytes(Charset.forName("UTF-8")))); } - try { + try { Translog tlog = new Translog(config); fail("file already exists?"); } catch (TranslogException ex) { @@ -1863,13 +1803,12 @@ public class TranslogTests extends ESTestCase { } try (Translog translog = new Translog(config)) { - try (Translog.Snapshot snapshot = translog.newSnapshot()) { - assertEquals(syncedDocs.size(), snapshot.estimatedTotalOperations()); - for (int i = 0; i < syncedDocs.size(); i++) { - Translog.Operation next = snapshot.next(); - assertEquals(syncedDocs.get(i), next.getSource().source.toUtf8()); - assertNotNull("operation " + i + " must be non-null", next); - } + Translog.Snapshot snapshot = translog.newSnapshot(); + assertEquals(syncedDocs.size(), snapshot.totalOperations()); + for (int i = 0; i < syncedDocs.size(); i++) { + Translog.Operation next = snapshot.next(); + assertEquals(syncedDocs.get(i), next.getSource().source.toUtf8()); + assertNotNull("operation " + i + " must be non-null", next); } } } diff --git a/core/src/test/java/org/elasticsearch/index/translog/TranslogVersionTests.java b/core/src/test/java/org/elasticsearch/index/translog/TranslogVersionTests.java index 68f26c504fb5..8ae7117d4838 100644 --- a/core/src/test/java/org/elasticsearch/index/translog/TranslogVersionTests.java +++ b/core/src/test/java/org/elasticsearch/index/translog/TranslogVersionTests.java @@ -20,7 +20,6 @@ package org.elasticsearch.index.translog; import org.apache.lucene.util.IOUtils; -import org.elasticsearch.index.VersionType; import org.elasticsearch.test.ESTestCase; import java.io.IOException; @@ -29,66 +28,32 @@ import java.nio.file.Files; import java.nio.file.Path; import java.nio.file.StandardOpenOption; +import static org.hamcrest.Matchers.containsString; import static org.hamcrest.Matchers.equalTo; /** * Tests for reading old and new translog files */ public class TranslogVersionTests extends ESTestCase { - public void testV0LegacyTranslogVersion() throws Exception { - Path translogFile = getDataPath("/org/elasticsearch/index/translog/translog-v0.binary"); - assertThat("test file should exist", Files.exists(translogFile), equalTo(true)); - try (ImmutableTranslogReader reader = openReader(translogFile, 0)) { - assertThat("a version0 stream is returned", reader instanceof LegacyTranslogReader, equalTo(true)); - try (final Translog.Snapshot snapshot = reader.newSnapshot()) { - final Translog.Operation operation = snapshot.next(); - assertThat("operation is the correct type correctly", operation.opType() == Translog.Operation.Type.INDEX, equalTo(true)); - Translog.Index op = (Translog.Index) operation; - assertThat(op.id(), equalTo("1")); - assertThat(op.type(), equalTo("doc")); - assertThat(op.source().toUtf8(), equalTo("{\"body\": \"worda wordb wordc wordd \\\"worde\\\" wordf\"}")); - assertThat(op.routing(), equalTo(null)); - assertThat(op.parent(), equalTo(null)); - assertThat(op.version(), equalTo(1L)); - assertThat(op.timestamp(), equalTo(1407312091791L)); - assertThat(op.ttl(), equalTo(-1L)); - assertThat(op.versionType(), equalTo(VersionType.INTERNAL)); - assertNull(snapshot.next()); - } + private void checkFailsToOpen(String file, String expectedMessage) throws IOException { + Path translogFile = getDataPath(file); + assertThat("test file should exist", Files.exists(translogFile), equalTo(true)); + try { + openReader(translogFile, 0); + fail("should be able to open an old translog"); + } catch (IllegalStateException e) { + assertThat(e.getMessage(), containsString(expectedMessage)); } + + } + + public void testV0LegacyTranslogVersion() throws Exception { + checkFailsToOpen("/org/elasticsearch/index/translog/translog-v0.binary", "pre-1.4 translog"); } public void testV1ChecksummedTranslogVersion() throws Exception { - Path translogFile = getDataPath("/org/elasticsearch/index/translog/translog-v1.binary"); - assertThat("test file should exist", Files.exists(translogFile), equalTo(true)); - try (ImmutableTranslogReader reader = openReader(translogFile, 0)) { - try (final Translog.Snapshot snapshot = reader.newSnapshot()) { - - assertThat("a version1 stream is returned", reader instanceof ImmutableTranslogReader, equalTo(true)); - - Translog.Operation operation = snapshot.next(); - - assertThat("operation is the correct type correctly", operation.opType() == Translog.Operation.Type.INDEX, equalTo(true)); - Translog.Index op = (Translog.Index) operation; - assertThat(op.id(), equalTo("Bwiq98KFSb6YjJQGeSpeiw")); - assertThat(op.type(), equalTo("doc")); - assertThat(op.source().toUtf8(), equalTo("{\"body\": \"foo\"}")); - assertThat(op.routing(), equalTo(null)); - assertThat(op.parent(), equalTo(null)); - assertThat(op.version(), equalTo(1L)); - assertThat(op.timestamp(), equalTo(1408627184844L)); - assertThat(op.ttl(), equalTo(-1L)); - assertThat(op.versionType(), equalTo(VersionType.INTERNAL)); - - // There are more operations - int opNum = 1; - while (snapshot.next() != null) { - opNum++; - } - assertThat("there should be 5 translog operations", opNum, equalTo(5)); - } - } + checkFailsToOpen("/org/elasticsearch/index/translog/translog-v1.binary", "pre-2.0 translog"); } public void testCorruptedTranslogs() throws Exception { @@ -112,47 +77,17 @@ public class TranslogVersionTests extends ESTestCase { e.getMessage().contains("Invalid first byte in translog file, got: 1, expected 0x00 or 0x3f"), equalTo(true)); } - try { - Path translogFile = getDataPath("/org/elasticsearch/index/translog/translog-v1-corrupted-body.binary"); - assertThat("test file should exist", Files.exists(translogFile), equalTo(true)); - try (ImmutableTranslogReader reader = openReader(translogFile, 0)) { - try (final Translog.Snapshot snapshot = reader.newSnapshot()) { - while(snapshot.next() != null) { - - } - } - } - fail("should have thrown an exception about the body being corrupted"); - } catch (TranslogCorruptedException e) { - assertThat("translog corruption from body: " + e.getMessage(), - e.getMessage().contains("translog corruption while reading from stream"), equalTo(true)); - } - + checkFailsToOpen("/org/elasticsearch/index/translog/translog-v1-corrupted-body.binary", "pre-2.0 translog"); } public void testTruncatedTranslog() throws Exception { - try { - Path translogFile = getDataPath("/org/elasticsearch/index/translog/translog-v1-truncated.binary"); - assertThat("test file should exist", Files.exists(translogFile), equalTo(true)); - try (ImmutableTranslogReader reader = openReader(translogFile, 0)) { - try (final Translog.Snapshot snapshot = reader.newSnapshot()) { - while(snapshot.next() != null) { - - } - } - } - fail("should have thrown an exception about the body being truncated"); - } catch (TranslogCorruptedException e) { - assertThat("translog truncated: " + e.getMessage(), - e.getMessage().contains("operation size is corrupted must be"), equalTo(true)); - } + checkFailsToOpen("/org/elasticsearch/index/translog/translog-v1-truncated.binary", "pre-2.0 translog"); } - public ImmutableTranslogReader openReader(Path path, long id) throws IOException { + public TranslogReader openReader(Path path, long id) throws IOException { FileChannel channel = FileChannel.open(path, StandardOpenOption.READ); try { - final ChannelReference raf = new ChannelReference(path, id, channel, null); - ImmutableTranslogReader reader = ImmutableTranslogReader.open(raf, new Checkpoint(Files.size(path), TranslogReader.UNKNOWN_OP_COUNT, id), null); + TranslogReader reader = TranslogReader.open(channel, path, new Checkpoint(Files.size(path), 1, id), null); channel = null; return reader; } finally { diff --git a/core/src/test/java/org/elasticsearch/indices/IndicesModuleTests.java b/core/src/test/java/org/elasticsearch/indices/IndicesModuleTests.java deleted file mode 100644 index ec182a69890e..000000000000 --- a/core/src/test/java/org/elasticsearch/indices/IndicesModuleTests.java +++ /dev/null @@ -1,73 +0,0 @@ -/* - * Licensed to Elasticsearch under one or more contributor - * license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright - * ownership. Elasticsearch licenses this file to you under - * the Apache License, Version 2.0 (the "License"); you may - * not use this file except in compliance with the License. - * You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, - * software distributed under the License is distributed on an - * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY - * KIND, either express or implied. See the License for the - * specific language governing permissions and limitations - * under the License. - */ - -package org.elasticsearch.indices; - -import org.elasticsearch.common.inject.ModuleTestCase; -import org.elasticsearch.index.query.QueryBuilder; -import org.elasticsearch.index.query.QueryParseContext; -import org.elasticsearch.index.query.QueryParser; -import org.elasticsearch.index.query.TermQueryParser; - -import java.io.IOException; - -public class IndicesModuleTests extends ModuleTestCase { - - static class FakeQueryParser implements QueryParser { - @Override - public String[] names() { - return new String[] {"fake-query-parser"}; - } - - @Override - public QueryBuilder fromXContent(QueryParseContext parseContext) throws IOException { - return null; - } - - @Override - public QueryBuilder getBuilderPrototype() { - return null; - } - } - - public void testRegisterQueryParser() { - IndicesModule module = new IndicesModule(); - module.registerQueryParser(FakeQueryParser.class); - assertSetMultiBinding(module, QueryParser.class, FakeQueryParser.class); - } - - public void testRegisterQueryParserDuplicate() { - IndicesModule module = new IndicesModule(); - try { - module.registerQueryParser(TermQueryParser.class); - } catch (IllegalArgumentException e) { - assertEquals(e.getMessage(), "Can't register the same [query_parser] more than once for [" + TermQueryParser.class.getName() + "]"); - } - } - - public void testRegisterHunspellDictionaryDuplicate() { - IndicesModule module = new IndicesModule(); - try { - module.registerQueryParser(TermQueryParser.class); - } catch (IllegalArgumentException e) { - assertEquals(e.getMessage(), "Can't register the same [query_parser] more than once for [" + TermQueryParser.class.getName() + "]"); - } - } - -} diff --git a/core/src/test/java/org/elasticsearch/percolator/PercolateDocumentParserTests.java b/core/src/test/java/org/elasticsearch/percolator/PercolateDocumentParserTests.java index 0c206ea1c930..c50c191f8f46 100644 --- a/core/src/test/java/org/elasticsearch/percolator/PercolateDocumentParserTests.java +++ b/core/src/test/java/org/elasticsearch/percolator/PercolateDocumentParserTests.java @@ -25,7 +25,6 @@ import org.elasticsearch.Version; import org.elasticsearch.action.percolate.PercolateShardRequest; import org.elasticsearch.cluster.action.index.MappingUpdatedAction; import org.elasticsearch.cluster.metadata.IndexMetaData; -import org.elasticsearch.common.io.stream.NamedWriteableRegistry; import org.elasticsearch.common.settings.Settings; import org.elasticsearch.common.xcontent.XContentBuilder; import org.elasticsearch.index.Index; @@ -57,8 +56,9 @@ import org.junit.Before; import org.mockito.Mockito; import java.util.Collections; -import java.util.Set; +import java.util.Map; +import static java.util.Collections.singletonMap; import static org.elasticsearch.common.xcontent.XContentFactory.jsonBuilder; import static org.hamcrest.Matchers.equalTo; import static org.hamcrest.Matchers.is; @@ -82,8 +82,8 @@ public class PercolateDocumentParserTests extends ESTestCase { IndicesModule indicesModule = new IndicesModule(); mapperService = new MapperService(indexSettings, analysisService, new SimilarityService(indexSettings, Collections.emptyMap()), indicesModule.getMapperRegistry(), () -> null); - Set parsers = Collections.singleton(new TermQueryParser()); - IndicesQueriesRegistry indicesQueriesRegistry = new IndicesQueriesRegistry(indexSettings.getSettings(), parsers, new NamedWriteableRegistry()); + Map> parsers = singletonMap("term", new TermQueryParser()); + IndicesQueriesRegistry indicesQueriesRegistry = new IndicesQueriesRegistry(indexSettings.getSettings(), parsers); queryShardContext = new QueryShardContext(indexSettings, null, null, null, mapperService, null, null, indicesQueriesRegistry); diff --git a/core/src/test/java/org/elasticsearch/rest/NoOpClient.java b/core/src/test/java/org/elasticsearch/rest/NoOpClient.java index 245bdb96a336..492c2cd43ed6 100644 --- a/core/src/test/java/org/elasticsearch/rest/NoOpClient.java +++ b/core/src/test/java/org/elasticsearch/rest/NoOpClient.java @@ -39,7 +39,7 @@ public class NoOpClient extends AbstractClient { } @Override - protected > void doExecute(Action action, Request request, ActionListener listener) { + protected , Response extends ActionResponse, RequestBuilder extends ActionRequestBuilder> void doExecute(Action action, Request request, ActionListener listener) { listener.onResponse(null); } diff --git a/core/src/test/java/org/elasticsearch/search/SearchModuleTests.java b/core/src/test/java/org/elasticsearch/search/SearchModuleTests.java index 15313095650e..b75d9634f691 100644 --- a/core/src/test/java/org/elasticsearch/search/SearchModuleTests.java +++ b/core/src/test/java/org/elasticsearch/search/SearchModuleTests.java @@ -18,19 +18,29 @@ */ package org.elasticsearch.search; +import java.io.IOException; + import org.elasticsearch.common.inject.ModuleTestCase; +import org.elasticsearch.common.io.stream.NamedWriteableRegistry; +import org.elasticsearch.common.settings.Settings; +import org.elasticsearch.index.query.QueryBuilder; +import org.elasticsearch.index.query.QueryParseContext; +import org.elasticsearch.index.query.QueryParser; +import org.elasticsearch.index.query.TermQueryParser; import org.elasticsearch.search.highlight.CustomHighlighter; import org.elasticsearch.search.highlight.Highlighter; import org.elasticsearch.search.highlight.PlainHighlighter; import org.elasticsearch.search.suggest.CustomSuggester; import org.elasticsearch.search.suggest.Suggester; import org.elasticsearch.search.suggest.phrase.PhraseSuggester; + +import static org.hamcrest.Matchers.containsString; /** */ public class SearchModuleTests extends ModuleTestCase { public void testDoubleRegister() { - SearchModule module = new SearchModule(); + SearchModule module = new SearchModule(Settings.EMPTY, new NamedWriteableRegistry()); try { module.registerHighlighter("fvh", PlainHighlighter.class); } catch (IllegalArgumentException e) { @@ -45,7 +55,7 @@ public class SearchModuleTests extends ModuleTestCase { } public void testRegisterSuggester() { - SearchModule module = new SearchModule(); + SearchModule module = new SearchModule(Settings.EMPTY, new NamedWriteableRegistry()); module.registerSuggester("custom", CustomSuggester.class); try { module.registerSuggester("custom", CustomSuggester.class); @@ -56,7 +66,7 @@ public class SearchModuleTests extends ModuleTestCase { } public void testRegisterHighlighter() { - SearchModule module = new SearchModule(); + SearchModule module = new SearchModule(Settings.EMPTY, new NamedWriteableRegistry()); module.registerHighlighter("custom", CustomHighlighter.class); try { module.registerHighlighter("custom", CustomHighlighter.class); @@ -65,4 +75,32 @@ public class SearchModuleTests extends ModuleTestCase { } assertMapMultiBinding(module, Highlighter.class, CustomHighlighter.class); } + + public void testRegisterQueryParserDuplicate() { + SearchModule module = new SearchModule(Settings.EMPTY, new NamedWriteableRegistry()); + module.registerQueryParser(TermQueryParser::new); + try { + module.buildQueryParserRegistry(); + } catch (IllegalArgumentException e) { + assertThat(e.getMessage(), containsString("already registered for name [term] while trying to register [org.elasticsearch.index.")); + } + } + + static class FakeQueryParser implements QueryParser { + @Override + public String[] names() { + return new String[] {"fake-query-parser"}; + } + + @Override + public QueryBuilder fromXContent(QueryParseContext parseContext) throws IOException { + return null; + } + + @Override + public QueryBuilder getBuilderPrototype() { + return null; + } + } + } diff --git a/core/src/test/java/org/elasticsearch/search/aggregations/bucket/nested/NestedAggregatorTests.java b/core/src/test/java/org/elasticsearch/search/aggregations/bucket/nested/NestedAggregatorTests.java index 17e8fd350732..0cb799d2ac0f 100644 --- a/core/src/test/java/org/elasticsearch/search/aggregations/bucket/nested/NestedAggregatorTests.java +++ b/core/src/test/java/org/elasticsearch/search/aggregations/bucket/nested/NestedAggregatorTests.java @@ -38,6 +38,7 @@ import org.elasticsearch.common.lucene.index.ElasticsearchDirectoryReader; import org.elasticsearch.common.lucene.search.Queries; import org.elasticsearch.index.Index; import org.elasticsearch.index.IndexService; +import org.elasticsearch.index.mapper.MapperService; import org.elasticsearch.index.mapper.internal.TypeFieldMapper; import org.elasticsearch.index.mapper.internal.UidFieldMapper; import org.elasticsearch.index.shard.ShardId; @@ -118,7 +119,7 @@ public class NestedAggregatorTests extends ESSingleNodeTestCase { IndexSearcher searcher = new IndexSearcher(directoryReader); IndexService indexService = createIndex("test"); - indexService.mapperService().merge("test", new CompressedXContent(PutMappingRequest.buildFromSimplifiedDef("test", "nested_field", "type=nested").string()), true, false); + indexService.mapperService().merge("test", new CompressedXContent(PutMappingRequest.buildFromSimplifiedDef("test", "nested_field", "type=nested").string()), MapperService.MergeReason.MAPPING_UPDATE, false); SearchContext searchContext = createSearchContext(indexService); AggregationContext context = new AggregationContext(searchContext); diff --git a/core/src/test/java/org/elasticsearch/search/aggregations/pipeline/PipelineAggregationHelperTests.java b/core/src/test/java/org/elasticsearch/search/aggregations/pipeline/PipelineAggregationHelperTests.java index e962e90830f7..65d5fbafbea8 100644 --- a/core/src/test/java/org/elasticsearch/search/aggregations/pipeline/PipelineAggregationHelperTests.java +++ b/core/src/test/java/org/elasticsearch/search/aggregations/pipeline/PipelineAggregationHelperTests.java @@ -109,7 +109,7 @@ public class PipelineAggregationHelperTests extends ESTestCase { * @param values Array of values to compute metric for * @param metric A metric builder which defines what kind of metric should be returned for the values */ - public static double calculateMetric(double[] values, ValuesSourceMetricsAggregationBuilder metric) { + public static double calculateMetric(double[] values, ValuesSourceMetricsAggregationBuilder metric) { if (metric instanceof MinBuilder) { double accumulator = Double.POSITIVE_INFINITY; diff --git a/core/src/test/java/org/elasticsearch/search/aggregations/pipeline/moving/avg/MovAvgIT.java b/core/src/test/java/org/elasticsearch/search/aggregations/pipeline/moving/avg/MovAvgIT.java index 90d4437fcea2..6184cb9fd68f 100644 --- a/core/src/test/java/org/elasticsearch/search/aggregations/pipeline/moving/avg/MovAvgIT.java +++ b/core/src/test/java/org/elasticsearch/search/aggregations/pipeline/moving/avg/MovAvgIT.java @@ -77,7 +77,7 @@ public class MovAvgIT extends ESIntegTestCase { static int period; static HoltWintersModel.SeasonalityType seasonalityType; static BucketHelpers.GapPolicy gapPolicy; - static ValuesSourceMetricsAggregationBuilder metric; + static ValuesSourceMetricsAggregationBuilder metric; static List mockHisto; static Map> testValues; @@ -864,7 +864,7 @@ public class MovAvgIT extends ESIntegTestCase { public void testHoltWintersNotEnoughData() { try { - SearchResponse response = client() + client() .prepareSearch("idx").setTypes("type") .addAggregation( histogram("histo").field(INTERVAL_FIELD).interval(interval) @@ -1003,7 +1003,7 @@ public class MovAvgIT extends ESIntegTestCase { public void testBadModelParams() { try { - SearchResponse response = client() + client() .prepareSearch("idx").setTypes("type") .addAggregation( histogram("histo").field(INTERVAL_FIELD).interval(interval) @@ -1248,7 +1248,7 @@ public class MovAvgIT extends ESIntegTestCase { for (MovAvgModelBuilder builder : builders) { try { - SearchResponse response = client() + client() .prepareSearch("idx").setTypes("type") .addAggregation( histogram("histo").field(INTERVAL_FIELD).interval(interval) @@ -1265,14 +1265,10 @@ public class MovAvgIT extends ESIntegTestCase { // All good } } - - - - } - private void assertValidIterators(Iterator expectedBucketIter, Iterator expectedCountsIter, Iterator expectedValuesIter) { + private void assertValidIterators(Iterator expectedBucketIter, Iterator expectedCountsIter, Iterator expectedValuesIter) { if (!expectedBucketIter.hasNext()) { fail("`expectedBucketIter` iterator ended before `actual` iterator, size mismatch"); } @@ -1355,7 +1351,7 @@ public class MovAvgIT extends ESIntegTestCase { } } - private ValuesSourceMetricsAggregationBuilder randomMetric(String name, String field) { + private ValuesSourceMetricsAggregationBuilder randomMetric(String name, String field) { int rand = randomIntBetween(0,3); switch (rand) { diff --git a/core/src/test/java/org/elasticsearch/search/aggregations/pipeline/serialdiff/SerialDiffIT.java b/core/src/test/java/org/elasticsearch/search/aggregations/pipeline/serialdiff/SerialDiffIT.java index aebd6a7e780e..145587a4b27c 100644 --- a/core/src/test/java/org/elasticsearch/search/aggregations/pipeline/serialdiff/SerialDiffIT.java +++ b/core/src/test/java/org/elasticsearch/search/aggregations/pipeline/serialdiff/SerialDiffIT.java @@ -60,7 +60,7 @@ public class SerialDiffIT extends ESIntegTestCase { static int numBuckets; static int lag; static BucketHelpers.GapPolicy gapPolicy; - static ValuesSourceMetricsAggregationBuilder metric; + static ValuesSourceMetricsAggregationBuilder metric; static List mockHisto; static Map> testValues; @@ -80,7 +80,7 @@ public class SerialDiffIT extends ESIntegTestCase { } } - private ValuesSourceMetricsAggregationBuilder randomMetric(String name, String field) { + private ValuesSourceMetricsAggregationBuilder randomMetric(String name, String field) { int rand = randomIntBetween(0,3); switch (rand) { @@ -95,7 +95,7 @@ public class SerialDiffIT extends ESIntegTestCase { } } - private void assertValidIterators(Iterator expectedBucketIter, Iterator expectedCountsIter, Iterator expectedValuesIter) { + private void assertValidIterators(Iterator expectedBucketIter, Iterator expectedCountsIter, Iterator expectedValuesIter) { if (!expectedBucketIter.hasNext()) { fail("`expectedBucketIter` iterator ended before `actual` iterator, size mismatch"); } diff --git a/core/src/test/java/org/elasticsearch/search/builder/SearchSourceBuilderTests.java b/core/src/test/java/org/elasticsearch/search/builder/SearchSourceBuilderTests.java index d7ede712447a..5a1b99fe05fc 100644 --- a/core/src/test/java/org/elasticsearch/search/builder/SearchSourceBuilderTests.java +++ b/core/src/test/java/org/elasticsearch/search/builder/SearchSourceBuilderTests.java @@ -19,6 +19,11 @@ package org.elasticsearch.search.builder; +import java.io.IOException; +import java.util.ArrayList; +import java.util.List; +import java.util.concurrent.TimeUnit; + import org.elasticsearch.common.ParseFieldMatcher; import org.elasticsearch.common.bytes.BytesArray; import org.elasticsearch.common.bytes.BytesReference; @@ -44,9 +49,9 @@ import org.elasticsearch.index.query.EmptyQueryBuilder; import org.elasticsearch.index.query.QueryBuilders; import org.elasticsearch.index.query.QueryParseContext; import org.elasticsearch.index.query.functionscore.ScoreFunctionParser; -import org.elasticsearch.indices.IndicesModule; import org.elasticsearch.indices.query.IndicesQueriesRegistry; import org.elasticsearch.script.Script; +import org.elasticsearch.search.SearchModule; import org.elasticsearch.search.aggregations.AggregationBuilders; import org.elasticsearch.search.fetch.innerhits.InnerHitsBuilder; import org.elasticsearch.search.fetch.innerhits.InnerHitsBuilder.InnerHit; @@ -63,11 +68,6 @@ import org.elasticsearch.threadpool.ThreadPoolModule; import org.junit.AfterClass; import org.junit.BeforeClass; -import java.io.IOException; -import java.util.ArrayList; -import java.util.List; -import java.util.concurrent.TimeUnit; - import static org.hamcrest.Matchers.equalTo; public class SearchSourceBuilderTests extends ESTestCase { @@ -83,26 +83,33 @@ public class SearchSourceBuilderTests extends ESTestCase { .put("name", SearchSourceBuilderTests.class.toString()) .put("path.home", createTempDir()) .build(); + namedWriteableRegistry = new NamedWriteableRegistry(); injector = new ModulesBuilder().add( new SettingsModule(settings, new SettingsFilter(settings)), new ThreadPoolModule(new ThreadPool(settings)), - new IndicesModule() { + new SearchModule(settings, namedWriteableRegistry) { @Override - public void configure() { - // skip services - bindQueryParsersExtension(); + protected void configureSearch() { + // skip me so we don't need transport + } + @Override + protected void configureAggs() { + // skip me so we don't need scripting + } + @Override + protected void configureSuggesters() { + // skip me so we don't need IndicesService } }, new AbstractModule() { @Override protected void configure() { Multibinder.newSetBinder(binder(), ScoreFunctionParser.class); - bind(NamedWriteableRegistry.class).asEagerSingleton(); + bind(NamedWriteableRegistry.class).toInstance(namedWriteableRegistry); } } ).createInjector(); indicesQueriesRegistry = injector.getInstance(IndicesQueriesRegistry.class); - namedWriteableRegistry = injector.getInstance(NamedWriteableRegistry.class); } @AfterClass diff --git a/core/src/test/java/org/elasticsearch/search/functionscore/FunctionScorePluginIT.java b/core/src/test/java/org/elasticsearch/search/functionscore/FunctionScorePluginIT.java index b428d911dd50..d1d8278b79b5 100644 --- a/core/src/test/java/org/elasticsearch/search/functionscore/FunctionScorePluginIT.java +++ b/core/src/test/java/org/elasticsearch/search/functionscore/FunctionScorePluginIT.java @@ -19,6 +19,8 @@ package org.elasticsearch.search.functionscore; +import java.util.Collection; + import org.apache.lucene.search.Explanation; import org.elasticsearch.action.ActionFuture; import org.elasticsearch.action.search.SearchResponse; @@ -36,8 +38,6 @@ import org.elasticsearch.test.ESIntegTestCase.ClusterScope; import org.elasticsearch.test.ESIntegTestCase.Scope; import org.elasticsearch.test.hamcrest.ElasticsearchAssertions; -import java.util.Collection; - import static org.elasticsearch.client.Requests.indexRequest; import static org.elasticsearch.client.Requests.searchRequest; import static org.elasticsearch.common.xcontent.XContentFactory.jsonBuilder; @@ -103,7 +103,7 @@ public class FunctionScorePluginIT extends ESIntegTestCase { } public void onModule(SearchModule scoreModule) { - scoreModule.registerFunctionScoreParser(FunctionScorePluginIT.CustomDistanceScoreParser.class); + scoreModule.registerFunctionScoreParser(new FunctionScorePluginIT.CustomDistanceScoreParser()); } } diff --git a/core/src/test/java/org/elasticsearch/search/highlight/HighlightBuilderTests.java b/core/src/test/java/org/elasticsearch/search/highlight/HighlightBuilderTests.java index 2ac5895c9ebc..383bde00b013 100644 --- a/core/src/test/java/org/elasticsearch/search/highlight/HighlightBuilderTests.java +++ b/core/src/test/java/org/elasticsearch/search/highlight/HighlightBuilderTests.java @@ -19,6 +19,13 @@ package org.elasticsearch.search.highlight; +import java.io.IOException; +import java.util.Arrays; +import java.util.HashMap; +import java.util.Iterator; +import java.util.List; +import java.util.Map; + import org.elasticsearch.Version; import org.elasticsearch.cluster.metadata.IndexMetaData; import org.elasticsearch.common.ParseFieldMatcher; @@ -42,16 +49,13 @@ import org.elasticsearch.index.mapper.Mapper; import org.elasticsearch.index.mapper.MapperBuilders; import org.elasticsearch.index.mapper.core.StringFieldMapper; import org.elasticsearch.index.query.IdsQueryBuilder; -import org.elasticsearch.index.query.IdsQueryParser; import org.elasticsearch.index.query.MatchAllQueryBuilder; -import org.elasticsearch.index.query.MatchAllQueryParser; import org.elasticsearch.index.query.QueryBuilder; import org.elasticsearch.index.query.QueryParseContext; -import org.elasticsearch.index.query.QueryParser; import org.elasticsearch.index.query.QueryShardContext; import org.elasticsearch.index.query.TermQueryBuilder; -import org.elasticsearch.index.query.TermQueryParser; import org.elasticsearch.indices.query.IndicesQueriesRegistry; +import org.elasticsearch.search.SearchModule; import org.elasticsearch.search.highlight.HighlightBuilder.Field; import org.elasticsearch.search.highlight.HighlightBuilder.Order; import org.elasticsearch.search.highlight.SearchContextHighlight.FieldOptions; @@ -60,15 +64,6 @@ import org.elasticsearch.test.IndexSettingsModule; import org.junit.AfterClass; import org.junit.BeforeClass; -import java.io.IOException; -import java.util.Arrays; -import java.util.HashMap; -import java.util.HashSet; -import java.util.Iterator; -import java.util.List; -import java.util.Map; -import java.util.Set; - import static org.hamcrest.Matchers.equalTo; import static org.hamcrest.Matchers.not; @@ -84,12 +79,7 @@ public class HighlightBuilderTests extends ESTestCase { @BeforeClass public static void init() { namedWriteableRegistry = new NamedWriteableRegistry(); - @SuppressWarnings("rawtypes") - Set injectedQueryParsers = new HashSet<>(); - injectedQueryParsers.add(new MatchAllQueryParser()); - injectedQueryParsers.add(new IdsQueryParser()); - injectedQueryParsers.add(new TermQueryParser()); - indicesQueriesRegistry = new IndicesQueriesRegistry(Settings.settingsBuilder().build(), injectedQueryParsers, namedWriteableRegistry); + indicesQueriesRegistry = new SearchModule(Settings.EMPTY, namedWriteableRegistry).buildQueryParserRegistry(); } @AfterClass diff --git a/core/src/test/java/org/elasticsearch/search/simple/SimpleSearchIT.java b/core/src/test/java/org/elasticsearch/search/simple/SimpleSearchIT.java index 205aacccf1ca..d14ea50838fe 100644 --- a/core/src/test/java/org/elasticsearch/search/simple/SimpleSearchIT.java +++ b/core/src/test/java/org/elasticsearch/search/simple/SimpleSearchIT.java @@ -41,6 +41,7 @@ import static org.elasticsearch.cluster.metadata.IndexMetaData.SETTING_NUMBER_OF import static org.elasticsearch.cluster.metadata.IndexMetaData.SETTING_NUMBER_OF_SHARDS; import static org.elasticsearch.common.xcontent.XContentFactory.jsonBuilder; import static org.elasticsearch.index.query.QueryBuilders.boolQuery; +import static org.elasticsearch.index.query.QueryBuilders.queryStringQuery; import static org.elasticsearch.index.query.QueryBuilders.rangeQuery; import static org.elasticsearch.test.hamcrest.ElasticsearchAssertions.assertAcked; import static org.elasticsearch.test.hamcrest.ElasticsearchAssertions.assertFailures; @@ -125,6 +126,16 @@ public class SimpleSearchIT extends ESIntegTestCase { refresh(); SearchResponse search = client().prepareSearch() + .setQuery(boolQuery().must(QueryBuilders.termQuery("ip", "192.168.0.1"))) + .execute().actionGet(); + assertHitCount(search, 1L); + + search = client().prepareSearch() + .setQuery(queryStringQuery("ip: 192.168.0.1")) + .execute().actionGet(); + assertHitCount(search, 1L); + + search = client().prepareSearch() .setQuery(boolQuery().must(QueryBuilders.termQuery("ip", "192.168.0.1/32"))) .execute().actionGet(); assertHitCount(search, 1l); diff --git a/core/src/test/java/org/elasticsearch/watcher/ResourceWatcherServiceTests.java b/core/src/test/java/org/elasticsearch/watcher/ResourceWatcherServiceTests.java index 6c6c45e9cfd0..fe36b7496254 100644 --- a/core/src/test/java/org/elasticsearch/watcher/ResourceWatcherServiceTests.java +++ b/core/src/test/java/org/elasticsearch/watcher/ResourceWatcherServiceTests.java @@ -79,7 +79,7 @@ public class ResourceWatcherServiceTests extends ESTestCase { }; // checking default freq - WatcherHandle handle = service.add(watcher); + WatcherHandle handle = service.add(watcher); assertThat(handle, notNullValue()); assertThat(handle.frequency(), equalTo(ResourceWatcherService.Frequency.MEDIUM)); assertThat(service.lowMonitor.watchers.size(), is(0)); diff --git a/docs/reference/cluster/nodes-stats.asciidoc b/docs/reference/cluster/nodes-stats.asciidoc index 144d6f72548e..bcef61d4ef71 100644 --- a/docs/reference/cluster/nodes-stats.asciidoc +++ b/docs/reference/cluster/nodes-stats.asciidoc @@ -131,9 +131,15 @@ the operating system: `os.cpu.percent`:: Recent CPU usage for the whole system, or -1 if not supported -`os.cpu.load_average`:: - Array of system load averages for the last one minute, five - minute and fifteen minutes (value of -1 indicates not supported) +`os.cpu.load_average.1m`:: + One-minute load average on the system (field is not present if + one-minute load average is not available) +`os.cpu.load_average.5m`:: + Five-minute load average on the system (field is not present if + five-minute load average is not available) +`os.cpu.load_average.15m`:: + Fifteen-minute load average on the system (field is not present if + fifteen-minute load average is not available) `os.mem.total_in_bytes`:: Total amount of physical memory in bytes diff --git a/docs/reference/migration/migrate_3_0.asciidoc b/docs/reference/migration/migrate_3_0.asciidoc index bffca162ed1f..d13e8cd78127 100644 --- a/docs/reference/migration/migrate_3_0.asciidoc +++ b/docs/reference/migration/migrate_3_0.asciidoc @@ -560,30 +560,30 @@ and high risk of being misused. The ability to change the thread pool type for a that it is still possible to adjust relevant thread pool parameters for each of the thread pools (e.g., depending on the thread pool type, `keep_alive`, `queue_size`, etc.). +[[breaking_30_cpu_stats]] === System CPU stats The recent CPU usage (as a percent) has been added to the OS stats reported under the node stats API and the cat nodes API. The breaking -change here is that there is a new object in the "os" object in the node -stats response. This object is called "cpu" and includes "percent" and -"load_average" as fields. This moves the "load_average" field that was -previously a top-level field in the "os" object to the "cpu" object. The -format of the "load_average" field has changed to an array of length -three representing the one-minute, five-minute and fifteen-minute load -averages (a value of -1 for any of array components indicates that the -corresponding metric is not available). +change here is that there is a new object in the `os` object in the node +stats response. This object is called `cpu` and includes "percent" and +`load_average` as fields. This moves the `load_average` field that was +previously a top-level field in the `os` object to the `cpu` object. The +format of the `load_average` field has changed to an object with fields +`1m`, `5m`, and `15m` representing the one-minute, five-minute and +fifteen-minute loads respectively. If any of these fields are not present, +it indicates that the corresponding value is not available. -In the cat nodes API response, the "cpu" field is output by default. The -previous "load" field has been removed and is replaced by "load_1m", -"load_5m", and "load_15m" which represent the one-minute, five-minute -and fifteen-minute loads respectively. These values are output by -default, and a value of -1 indicates that the corresponding metric is -not available. +In the cat nodes API response, the `cpu` field is output by default. The +previous `load` field has been removed and is replaced by `load_1m`, +`load_5m`, and `load_15m` which represent the one-minute, five-minute +and fifteen-minute loads respectively. The field will be null if the +corresponding value is not available. -Finally, the API for org.elasticsearch.monitor.os.OsStats has +Finally, the API for `org.elasticsearch.monitor.os.OsStats` has changed. The `getLoadAverage` method has been removed. The value for this can now be obtained from `OsStats.Cpu#getLoadAverage` but it is no -longer a double and is instead an object encapuslating the one-minute, +longer a double and is instead an object encapsulating the one-minute, five-minute and fifteen-minute load averages. Additionally, the recent CPU usage can be obtained from `OsStats.Cpu#getPercent`. diff --git a/modules/lang-expression/src/test/java/org/elasticsearch/script/expression/ExpressionRestIT.java b/modules/lang-expression/src/test/java/org/elasticsearch/script/expression/ExpressionRestIT.java index 8dcefce14786..3da22bf8a43b 100644 --- a/modules/lang-expression/src/test/java/org/elasticsearch/script/expression/ExpressionRestIT.java +++ b/modules/lang-expression/src/test/java/org/elasticsearch/script/expression/ExpressionRestIT.java @@ -21,21 +21,14 @@ package org.elasticsearch.script.expression; import com.carrotsearch.randomizedtesting.annotations.Name; import com.carrotsearch.randomizedtesting.annotations.ParametersFactory; -import org.elasticsearch.plugins.Plugin; import org.elasticsearch.test.rest.ESRestTestCase; import org.elasticsearch.test.rest.RestTestCandidate; import org.elasticsearch.test.rest.parser.RestTestParseException; import java.io.IOException; -import java.util.Collection; public class ExpressionRestIT extends ESRestTestCase { - @Override - protected Collection> nodePlugins() { - return pluginList(ExpressionPlugin.class); - } - public ExpressionRestIT(@Name("yaml") RestTestCandidate testCandidate) { super(testCandidate); } diff --git a/modules/lang-groovy/build.gradle b/modules/lang-groovy/build.gradle index 73ad6043f376..6f9b043d8bc2 100644 --- a/modules/lang-groovy/build.gradle +++ b/modules/lang-groovy/build.gradle @@ -26,9 +26,6 @@ dependencies { compile 'org.codehaus.groovy:groovy:2.4.4:indy' } -compileJava.options.compilerArgs << '-Xlint:-rawtypes,-unchecked,-cast' -compileTestJava.options.compilerArgs << '-Xlint:-rawtypes,-unchecked,-cast' - integTest { cluster { systemProperty 'es.script.inline', 'on' diff --git a/modules/lang-groovy/src/main/java/org/elasticsearch/script/groovy/GroovyScriptEngineService.java b/modules/lang-groovy/src/main/java/org/elasticsearch/script/groovy/GroovyScriptEngineService.java index 98ed5695973e..44ee9cdaf542 100644 --- a/modules/lang-groovy/src/main/java/org/elasticsearch/script/groovy/GroovyScriptEngineService.java +++ b/modules/lang-groovy/src/main/java/org/elasticsearch/script/groovy/GroovyScriptEngineService.java @@ -23,6 +23,7 @@ import groovy.lang.Binding; import groovy.lang.GroovyClassLoader; import groovy.lang.GroovyCodeSource; import groovy.lang.Script; + import org.apache.lucene.index.LeafReaderContext; import org.apache.lucene.search.Scorer; import org.codehaus.groovy.ast.ClassCodeExpressionTransformer; @@ -182,6 +183,7 @@ public class GroovyScriptEngineService extends AbstractComponent implements Scri String fake = MessageDigests.toHexString(MessageDigests.sha1().digest(script.getBytes(StandardCharsets.UTF_8))); // same logic as GroovyClassLoader.parseClass() but with a different codesource string: return AccessController.doPrivileged(new PrivilegedAction() { + @Override public Class run() { GroovyCodeSource gcs = new GroovyCodeSource(script, fake, BootstrapInfo.UNTRUSTED_CODEBASE); gcs.setCachable(false); @@ -203,7 +205,7 @@ public class GroovyScriptEngineService extends AbstractComponent implements Scri */ @SuppressWarnings("unchecked") private Script createScript(Object compiledScript, Map vars) throws InstantiationException, IllegalAccessException { - Class scriptClass = (Class) compiledScript; + Class scriptClass = (Class) compiledScript; Script scriptObject = (Script) scriptClass.newInstance(); Binding binding = new Binding(); binding.getVariables().putAll(vars); @@ -211,7 +213,6 @@ public class GroovyScriptEngineService extends AbstractComponent implements Scri return scriptObject; } - @SuppressWarnings({"unchecked"}) @Override public ExecutableScript executable(CompiledScript compiledScript, Map vars) { try { @@ -225,7 +226,6 @@ public class GroovyScriptEngineService extends AbstractComponent implements Scri } } - @SuppressWarnings({"unchecked"}) @Override public SearchScript search(final CompiledScript compiledScript, final SearchLookup lookup, @Nullable final Map vars) { return new SearchScript() { @@ -288,7 +288,6 @@ public class GroovyScriptEngineService extends AbstractComponent implements Scri } } - @SuppressWarnings({"unchecked"}) @Override public void setNextVar(String name, Object value) { variables.put(name, value); diff --git a/modules/lang-groovy/src/test/java/org/elasticsearch/messy/tests/ContextAndHeaderTransportTests.java b/modules/lang-groovy/src/test/java/org/elasticsearch/messy/tests/ContextAndHeaderTransportTests.java index 9b2e0041462c..2a6be52f3f4a 100644 --- a/modules/lang-groovy/src/test/java/org/elasticsearch/messy/tests/ContextAndHeaderTransportTests.java +++ b/modules/lang-groovy/src/test/java/org/elasticsearch/messy/tests/ContextAndHeaderTransportTests.java @@ -23,7 +23,6 @@ import org.apache.http.impl.client.CloseableHttpClient; import org.apache.http.impl.client.HttpClients; import org.elasticsearch.action.Action; import org.elasticsearch.action.ActionListener; -import org.elasticsearch.action.ActionModule; import org.elasticsearch.action.ActionRequest; import org.elasticsearch.action.ActionRequestBuilder; import org.elasticsearch.action.ActionResponse; @@ -35,13 +34,9 @@ import org.elasticsearch.action.indexedscripts.put.PutIndexedScriptResponse; import org.elasticsearch.action.percolate.PercolateResponse; import org.elasticsearch.action.search.SearchRequest; import org.elasticsearch.action.search.SearchResponse; -import org.elasticsearch.action.support.ActionFilter; import org.elasticsearch.action.termvectors.MultiTermVectorsRequest; import org.elasticsearch.client.Client; import org.elasticsearch.client.FilterClient; -import org.elasticsearch.common.inject.AbstractModule; -import org.elasticsearch.common.inject.Inject; -import org.elasticsearch.common.inject.Module; import org.elasticsearch.common.lucene.search.function.CombineFunction; import org.elasticsearch.common.settings.Settings; import org.elasticsearch.http.HttpServerTransport; @@ -59,6 +54,7 @@ import org.elasticsearch.script.Script; import org.elasticsearch.script.ScriptService.ScriptType; import org.elasticsearch.script.groovy.GroovyPlugin; import org.elasticsearch.script.groovy.GroovyScriptEngineService; +import org.elasticsearch.test.ActionRecordingPlugin; import org.elasticsearch.test.ESIntegTestCase; import org.elasticsearch.test.ESIntegTestCase.ClusterScope; import org.elasticsearch.test.rest.client.http.HttpRequestBuilder; @@ -66,12 +62,9 @@ import org.elasticsearch.test.rest.client.http.HttpResponse; import org.junit.After; import org.junit.Before; -import java.util.ArrayList; import java.util.Collection; -import java.util.Collections; import java.util.List; import java.util.Locale; -import java.util.concurrent.CopyOnWriteArrayList; import static org.elasticsearch.cluster.metadata.IndexMetaData.SETTING_NUMBER_OF_SHARDS; import static org.elasticsearch.common.settings.Settings.settingsBuilder; @@ -89,7 +82,6 @@ import static org.hamcrest.Matchers.is; @ClusterScope(scope = SUITE) public class ContextAndHeaderTransportTests extends ESIntegTestCase { - private static final List requests = new CopyOnWriteArrayList<>(); private String randomHeaderKey = randomAsciiOfLength(10); private String randomHeaderValue = randomAsciiOfLength(20); private String queryIndex = "query-" + randomAsciiOfLength(10).toLowerCase(Locale.ROOT); @@ -106,7 +98,7 @@ public class ContextAndHeaderTransportTests extends ESIntegTestCase { @Override protected Collection> nodePlugins() { - return pluginList(ActionLoggingPlugin.class, GroovyPlugin.class); + return pluginList(ActionRecordingPlugin.class, GroovyPlugin.class); } @Before @@ -128,7 +120,7 @@ public class ContextAndHeaderTransportTests extends ESIntegTestCase { .setSettings(settings).addMapping("type", mapping)); ensureGreen(queryIndex, lookupIndex); - requests.clear(); + ActionRecordingPlugin.clear(); } @After @@ -193,7 +185,7 @@ public class ContextAndHeaderTransportTests extends ESIntegTestCase { .get(); assertNoFailures(searchResponse); assertHitCount(searchResponse, 1); - assertThat(requests, hasSize(greaterThan(0))); + assertThat(ActionRecordingPlugin.allRequests(), hasSize(greaterThan(0))); assertGetRequestsContainHeaders(); } @@ -281,7 +273,7 @@ public class ContextAndHeaderTransportTests extends ESIntegTestCase { .execute(); assertThat(response, hasStatus(OK)); - List searchRequests = getRequests(SearchRequest.class); + List searchRequests = ActionRecordingPlugin.requestsOfType(SearchRequest.class); assertThat(searchRequests, hasSize(greaterThan(0))); for (SearchRequest searchRequest : searchRequests) { assertThat(searchRequest.hasHeader(releventHeaderName), is(true)); @@ -290,20 +282,9 @@ public class ContextAndHeaderTransportTests extends ESIntegTestCase { } } - private List getRequests(Class clazz) { - List results = new ArrayList<>(); - for (ActionRequest request : requests) { - if (request.getClass().equals(clazz)) { - results.add((T) request); - } - } - - return results; - } - - private void assertRequestsContainHeader(Class clazz) { - List classRequests = getRequests(clazz); - for (ActionRequest request : classRequests) { + private void assertRequestsContainHeader(Class> clazz) { + List> classRequests = ActionRecordingPlugin.requestsOfType(clazz); + for (ActionRequest request : classRequests) { assertRequestContainsHeader(request); } } @@ -313,7 +294,7 @@ public class ContextAndHeaderTransportTests extends ESIntegTestCase { } private void assertGetRequestsContainHeaders(String index) { - List getRequests = getRequests(GetRequest.class); + List getRequests = ActionRecordingPlugin.requestsOfType(GetRequest.class); assertThat(getRequests, hasSize(greaterThan(0))); for (GetRequest request : getRequests) { @@ -324,7 +305,7 @@ public class ContextAndHeaderTransportTests extends ESIntegTestCase { } } - private void assertRequestContainsHeader(ActionRequest request) { + private void assertRequestContainsHeader(ActionRequest request) { String msg = String.format(Locale.ROOT, "Expected header %s to be in request %s", randomHeaderKey, request.getClass().getName()); if (request instanceof IndexRequest) { IndexRequest indexRequest = (IndexRequest) request; @@ -342,7 +323,9 @@ public class ContextAndHeaderTransportTests extends ESIntegTestCase { Client transportClient = internalCluster().transportClient(); FilterClient filterClient = new FilterClient(transportClient) { @Override - protected > void doExecute(Action action, Request request, ActionListener listener) { + protected , Response extends ActionResponse, RequestBuilder extends ActionRequestBuilder> void doExecute( + Action action, Request request, + ActionListener listener) { request.putHeader(randomHeaderKey, randomHeaderValue); super.doExecute(action, request, listener); } @@ -350,58 +333,4 @@ public class ContextAndHeaderTransportTests extends ESIntegTestCase { return filterClient; } - - public static class ActionLoggingPlugin extends Plugin { - - @Override - public String name() { - return "test-action-logging"; - } - - @Override - public String description() { - return "Test action logging"; - } - - @Override - public Collection nodeModules() { - return Collections.singletonList(new ActionLoggingModule()); - } - - public void onModule(ActionModule module) { - module.registerFilter(LoggingFilter.class); - } - } - - public static class ActionLoggingModule extends AbstractModule { - @Override - protected void configure() { - bind(LoggingFilter.class).asEagerSingleton(); - } - - } - - public static class LoggingFilter extends ActionFilter.Simple { - - @Inject - public LoggingFilter(Settings settings) { - super(settings); - } - - @Override - public int order() { - return 999; - } - - @Override - protected boolean apply(String action, ActionRequest request, ActionListener listener) { - requests.add(request); - return true; - } - - @Override - protected boolean apply(String action, ActionResponse response, ActionListener listener) { - return true; - } - } } diff --git a/modules/lang-groovy/src/test/java/org/elasticsearch/messy/tests/MinDocCountTests.java b/modules/lang-groovy/src/test/java/org/elasticsearch/messy/tests/MinDocCountTests.java index a8f78c62c77d..9b3e1a342a8f 100644 --- a/modules/lang-groovy/src/test/java/org/elasticsearch/messy/tests/MinDocCountTests.java +++ b/modules/lang-groovy/src/test/java/org/elasticsearch/messy/tests/MinDocCountTests.java @@ -22,6 +22,7 @@ package org.elasticsearch.messy.tests; import com.carrotsearch.hppc.LongHashSet; import com.carrotsearch.hppc.LongSet; import com.carrotsearch.randomizedtesting.generators.RandomStrings; + import org.elasticsearch.action.index.IndexRequestBuilder; import org.elasticsearch.action.search.SearchRequest; import org.elasticsearch.action.search.SearchResponse; @@ -65,7 +66,7 @@ public class MinDocCountTests extends AbstractTermsTestCase { return Collections.singleton(GroovyPlugin.class); } - private static final QueryBuilder QUERY = QueryBuilders.termQuery("match", true); + private static final QueryBuilder QUERY = QueryBuilders.termQuery("match", true); private static int cardinality; @@ -77,7 +78,6 @@ public class MinDocCountTests extends AbstractTermsTestCase { final List indexRequests = new ArrayList<>(); final Set stringTerms = new HashSet<>(); final LongSet longTerms = new LongHashSet(); - final Set dateTerms = new HashSet<>(); for (int i = 0; i < cardinality; ++i) { String stringTerm; do { @@ -319,7 +319,6 @@ public class MinDocCountTests extends AbstractTermsTestCase { throw ae; } } - } public void testHistogramCountAsc() throws Exception { @@ -372,11 +371,9 @@ public class MinDocCountTests extends AbstractTermsTestCase { .execute().actionGet(); assertSubset(allHisto, (Histogram) response.getAggregations().get("histo"), minDocCount); } - } private void testMinDocCountOnDateHistogram(Histogram.Order order) throws Exception { - final int interval = randomIntBetween(1, 3); final SearchResponse allResponse = client().prepareSearch("idx").setTypes("type") .setSize(0) .setQuery(QUERY) @@ -393,7 +390,5 @@ public class MinDocCountTests extends AbstractTermsTestCase { .execute().actionGet(); assertSubset(allHisto, (Histogram) response.getAggregations().get("histo"), minDocCount); } - } - } diff --git a/modules/lang-groovy/src/test/java/org/elasticsearch/messy/tests/ScriptedMetricTests.java b/modules/lang-groovy/src/test/java/org/elasticsearch/messy/tests/ScriptedMetricTests.java index 98d53c851749..7e6dbd67f566 100644 --- a/modules/lang-groovy/src/test/java/org/elasticsearch/messy/tests/ScriptedMetricTests.java +++ b/modules/lang-groovy/src/test/java/org/elasticsearch/messy/tests/ScriptedMetricTests.java @@ -147,15 +147,15 @@ public class ScriptedMetricTests extends ESIntegTestCase { for (Object object : aggregationList) { assertThat(object, notNullValue()); assertThat(object, instanceOf(Map.class)); - Map map = (Map) object; + Map map = (Map) object; assertThat(map.size(), lessThanOrEqualTo(1)); if (map.size() == 1) { - assertThat(map.get("count"), notNullValue()); - assertThat(map.get("count"), instanceOf(Number.class)); - assertThat((Number) map.get("count"), equalTo((Number) 1)); + assertThat(map.get("count"), notNullValue()); + assertThat(map.get("count"), instanceOf(Number.class)); + assertThat((Number) map.get("count"), equalTo((Number) 1)); numShardsRun++; + } } - } // We don't know how many shards will have documents but we need to make // sure that at least one shard ran the map script assertThat(numShardsRun, greaterThan(0)); @@ -740,6 +740,7 @@ public class ScriptedMetricTests extends ESIntegTestCase { assertThat(scriptedMetric.getName(), equalTo("scripted")); assertThat(scriptedMetric.aggregation(), notNullValue()); assertThat(scriptedMetric.aggregation(), instanceOf(List.class)); + @SuppressWarnings("unchecked") // We'll just get a ClassCastException a couple lines down if we're wrong, its ok. List aggregationResult = (List) scriptedMetric.aggregation(); assertThat(aggregationResult.size(), equalTo(1)); assertThat(aggregationResult.get(0), equalTo(0)); diff --git a/modules/lang-groovy/src/test/java/org/elasticsearch/messy/tests/SearchFieldsTests.java b/modules/lang-groovy/src/test/java/org/elasticsearch/messy/tests/SearchFieldsTests.java index 5a56e0f69994..b78c1c264c38 100644 --- a/modules/lang-groovy/src/test/java/org/elasticsearch/messy/tests/SearchFieldsTests.java +++ b/modules/lang-groovy/src/test/java/org/elasticsearch/messy/tests/SearchFieldsTests.java @@ -339,9 +339,7 @@ public class SearchFieldsTests extends ESIntegTestCase { .execute().actionGet(); client().admin().indices().refresh(refreshRequest()).actionGet(); - SearchResponse response = client().prepareSearch() - .setQuery(matchAllQuery()) -.addScriptField("s_obj1", new Script("_source.obj1")) + SearchResponse response = client().prepareSearch().setQuery(matchAllQuery()).addScriptField("s_obj1", new Script("_source.obj1")) .addScriptField("s_obj1_test", new Script("_source.obj1.test")).addScriptField("s_obj2", new Script("_source.obj2")) .addScriptField("s_obj2_arr2", new Script("_source.obj2.arr2")).addScriptField("s_arr3", new Script("_source.arr3")) .execute().actionGet(); @@ -355,7 +353,7 @@ public class SearchFieldsTests extends ESIntegTestCase { assertThat(response.getHits().getAt(0).field("s_obj1_test").value().toString(), equalTo("something")); Map sObj2 = response.getHits().getAt(0).field("s_obj2").value(); - List sObj2Arr2 = (List) sObj2.get("arr2"); + List sObj2Arr2 = (List) sObj2.get("arr2"); assertThat(sObj2Arr2.size(), equalTo(2)); assertThat(sObj2Arr2.get(0).toString(), equalTo("arr_value1")); assertThat(sObj2Arr2.get(1).toString(), equalTo("arr_value2")); @@ -365,8 +363,8 @@ public class SearchFieldsTests extends ESIntegTestCase { assertThat(sObj2Arr2.get(0).toString(), equalTo("arr_value1")); assertThat(sObj2Arr2.get(1).toString(), equalTo("arr_value2")); - List sObj2Arr3 = response.getHits().getAt(0).field("s_arr3").values(); - assertThat(((Map) sObj2Arr3.get(0)).get("arr3_field1").toString(), equalTo("arr3_value1")); + List sObj2Arr3 = response.getHits().getAt(0).field("s_arr3").values(); + assertThat(((Map) sObj2Arr3.get(0)).get("arr3_field1").toString(), equalTo("arr3_value1")); } public void testPartialFields() throws Exception { diff --git a/modules/lang-groovy/src/test/java/org/elasticsearch/script/groovy/GroovyRestIT.java b/modules/lang-groovy/src/test/java/org/elasticsearch/script/groovy/GroovyRestIT.java index b96436dd77f7..b73ec250dafd 100644 --- a/modules/lang-groovy/src/test/java/org/elasticsearch/script/groovy/GroovyRestIT.java +++ b/modules/lang-groovy/src/test/java/org/elasticsearch/script/groovy/GroovyRestIT.java @@ -21,21 +21,14 @@ package org.elasticsearch.script.groovy; import com.carrotsearch.randomizedtesting.annotations.Name; import com.carrotsearch.randomizedtesting.annotations.ParametersFactory; -import org.elasticsearch.plugins.Plugin; import org.elasticsearch.test.rest.ESRestTestCase; import org.elasticsearch.test.rest.RestTestCandidate; import org.elasticsearch.test.rest.parser.RestTestParseException; import java.io.IOException; -import java.util.Collection; public class GroovyRestIT extends ESRestTestCase { - @Override - protected Collection> nodePlugins() { - return pluginList(GroovyPlugin.class); - } - public GroovyRestIT(@Name("yaml") RestTestCandidate testCandidate) { super(testCandidate); } diff --git a/modules/lang-mustache/build.gradle b/modules/lang-mustache/build.gradle index 4e8e9cc345dd..694ddc746068 100644 --- a/modules/lang-mustache/build.gradle +++ b/modules/lang-mustache/build.gradle @@ -26,8 +26,6 @@ dependencies { compile "com.github.spullara.mustache.java:compiler:0.9.1" } -compileTestJava.options.compilerArgs << '-Xlint:-rawtypes,-unchecked' - integTest { cluster { systemProperty 'es.script.inline', 'on' diff --git a/modules/lang-mustache/src/test/java/org/elasticsearch/messy/tests/ContextAndHeaderTransportTests.java b/modules/lang-mustache/src/test/java/org/elasticsearch/messy/tests/ContextAndHeaderTransportTests.java index 92d15332780f..485e687b4a58 100644 --- a/modules/lang-mustache/src/test/java/org/elasticsearch/messy/tests/ContextAndHeaderTransportTests.java +++ b/modules/lang-mustache/src/test/java/org/elasticsearch/messy/tests/ContextAndHeaderTransportTests.java @@ -21,7 +21,6 @@ package org.elasticsearch.messy.tests; import org.elasticsearch.action.Action; import org.elasticsearch.action.ActionListener; -import org.elasticsearch.action.ActionModule; import org.elasticsearch.action.ActionRequest; import org.elasticsearch.action.ActionRequestBuilder; import org.elasticsearch.action.ActionResponse; @@ -29,17 +28,12 @@ import org.elasticsearch.action.admin.indices.create.CreateIndexRequestBuilder; import org.elasticsearch.action.admin.indices.refresh.RefreshRequest; import org.elasticsearch.action.get.GetRequest; import org.elasticsearch.action.index.IndexRequest; -import org.elasticsearch.action.index.IndexRequestBuilder; import org.elasticsearch.action.indexedscripts.put.PutIndexedScriptRequest; import org.elasticsearch.action.indexedscripts.put.PutIndexedScriptResponse; import org.elasticsearch.action.search.SearchRequestBuilder; import org.elasticsearch.action.search.SearchResponse; -import org.elasticsearch.action.support.ActionFilter; import org.elasticsearch.client.Client; import org.elasticsearch.client.FilterClient; -import org.elasticsearch.common.inject.AbstractModule; -import org.elasticsearch.common.inject.Inject; -import org.elasticsearch.common.inject.Module; import org.elasticsearch.common.settings.Settings; import org.elasticsearch.common.xcontent.XContentBuilder; import org.elasticsearch.common.xcontent.XContentFactory; @@ -52,6 +46,7 @@ import org.elasticsearch.script.mustache.MustacheScriptEngineService; import org.elasticsearch.search.suggest.Suggest; import org.elasticsearch.search.suggest.SuggestBuilder; import org.elasticsearch.search.suggest.phrase.PhraseSuggestionBuilder; +import org.elasticsearch.test.ActionRecordingPlugin; import org.elasticsearch.test.ESIntegTestCase; import org.elasticsearch.test.ESIntegTestCase.ClusterScope; import org.junit.After; @@ -60,12 +55,10 @@ import org.junit.Before; import java.util.ArrayList; import java.util.Arrays; import java.util.Collection; -import java.util.Collections; import java.util.HashMap; import java.util.List; import java.util.Locale; import java.util.Map; -import java.util.concurrent.CopyOnWriteArrayList; import static org.elasticsearch.cluster.metadata.IndexMetaData.SETTING_NUMBER_OF_SHARDS; import static org.elasticsearch.common.settings.Settings.settingsBuilder; @@ -84,7 +77,6 @@ import static org.hamcrest.Matchers.is; @ClusterScope(scope = SUITE) public class ContextAndHeaderTransportTests extends ESIntegTestCase { - private static final List requests = new CopyOnWriteArrayList<>(); private String randomHeaderKey = randomAsciiOfLength(10); private String randomHeaderValue = randomAsciiOfLength(20); private String queryIndex = "query-" + randomAsciiOfLength(10).toLowerCase(Locale.ROOT); @@ -101,7 +93,7 @@ public class ContextAndHeaderTransportTests extends ESIntegTestCase { @Override protected Collection> nodePlugins() { - return pluginList(ActionLoggingPlugin.class, MustachePlugin.class); + return pluginList(ActionRecordingPlugin.class, MustachePlugin.class); } @Before @@ -122,14 +114,13 @@ public class ContextAndHeaderTransportTests extends ESIntegTestCase { assertAcked(transportClient().admin().indices().prepareCreate(queryIndex) .setSettings(settings).addMapping("type", mapping)); ensureGreen(queryIndex, lookupIndex); - - requests.clear(); } @After public void checkAllRequestsContainHeaders() { assertRequestsContainHeader(IndexRequest.class); assertRequestsContainHeader(RefreshRequest.class); + ActionRecordingPlugin.clear(); } public void testThatIndexedScriptGetRequestInTemplateQueryContainsContextAndHeaders() throws Exception { @@ -216,7 +207,6 @@ public class ContextAndHeaderTransportTests extends ESIntegTestCase { titles.add("Representative Government"); titles.add("Election"); - List builders = new ArrayList<>(); for (String title: titles) { transportClient().prepareIndex("test", "type1").setSource("title", title).get(); } @@ -272,30 +262,15 @@ public class ContextAndHeaderTransportTests extends ESIntegTestCase { assertRequestsContainHeader(PutIndexedScriptRequest.class); } - private List getRequests(Class clazz) { - List results = new ArrayList<>(); - for (ActionRequest request : requests) { - if (request.getClass().equals(clazz)) { - results.add((T) request); - } - } - - return results; - } - - private void assertRequestsContainHeader(Class clazz) { - List classRequests = getRequests(clazz); - for (ActionRequest request : classRequests) { + private void assertRequestsContainHeader(Class> clazz) { + List> classRequests = ActionRecordingPlugin.requestsOfType(clazz); + for (ActionRequest request : classRequests) { assertRequestContainsHeader(request); } } - private void assertGetRequestsContainHeaders() { - assertGetRequestsContainHeaders(this.lookupIndex); - } - private void assertGetRequestsContainHeaders(String index) { - List getRequests = getRequests(GetRequest.class); + List getRequests = ActionRecordingPlugin.requestsOfType(GetRequest.class); assertThat(getRequests, hasSize(greaterThan(0))); for (GetRequest request : getRequests) { @@ -306,7 +281,7 @@ public class ContextAndHeaderTransportTests extends ESIntegTestCase { } } - private void assertRequestContainsHeader(ActionRequest request) { + private void assertRequestContainsHeader(ActionRequest request) { String msg = String.format(Locale.ROOT, "Expected header %s to be in request %s", randomHeaderKey, request.getClass().getName()); if (request instanceof IndexRequest) { IndexRequest indexRequest = (IndexRequest) request; @@ -324,7 +299,9 @@ public class ContextAndHeaderTransportTests extends ESIntegTestCase { Client transportClient = internalCluster().transportClient(); FilterClient filterClient = new FilterClient(transportClient) { @Override - protected > void doExecute(Action action, Request request, ActionListener listener) { + protected , Response extends ActionResponse, RequestBuilder extends ActionRequestBuilder> void doExecute( + Action action, Request request, + ActionListener listener) { request.putHeader(randomHeaderKey, randomHeaderValue); super.doExecute(action, request, listener); } @@ -332,58 +309,4 @@ public class ContextAndHeaderTransportTests extends ESIntegTestCase { return filterClient; } - - public static class ActionLoggingPlugin extends Plugin { - - @Override - public String name() { - return "test-action-logging"; - } - - @Override - public String description() { - return "Test action logging"; - } - - @Override - public Collection nodeModules() { - return Collections.singletonList(new ActionLoggingModule()); - } - - public void onModule(ActionModule module) { - module.registerFilter(LoggingFilter.class); - } - } - - public static class ActionLoggingModule extends AbstractModule { - @Override - protected void configure() { - bind(LoggingFilter.class).asEagerSingleton(); - } - - } - - public static class LoggingFilter extends ActionFilter.Simple { - - @Inject - public LoggingFilter(Settings settings) { - super(settings); - } - - @Override - public int order() { - return 999; - } - - @Override - protected boolean apply(String action, ActionRequest request, ActionListener listener) { - requests.add(request); - return true; - } - - @Override - protected boolean apply(String action, ActionResponse response, ActionListener listener) { - return true; - } - } } diff --git a/modules/lang-mustache/src/test/java/org/elasticsearch/messy/tests/TemplateQueryParserTests.java b/modules/lang-mustache/src/test/java/org/elasticsearch/messy/tests/TemplateQueryParserTests.java index 4f4e5f21c019..9d1368070921 100644 --- a/modules/lang-mustache/src/test/java/org/elasticsearch/messy/tests/TemplateQueryParserTests.java +++ b/modules/lang-mustache/src/test/java/org/elasticsearch/messy/tests/TemplateQueryParserTests.java @@ -18,6 +18,10 @@ */ package org.elasticsearch.messy.tests; +import java.io.IOException; +import java.lang.reflect.Proxy; +import java.util.Collections; + import org.apache.lucene.search.MatchAllDocsQuery; import org.apache.lucene.search.Query; import org.apache.lucene.util.Accountable; @@ -31,6 +35,7 @@ import org.elasticsearch.common.inject.Injector; import org.elasticsearch.common.inject.ModulesBuilder; import org.elasticsearch.common.inject.multibindings.Multibinder; import org.elasticsearch.common.inject.util.Providers; +import org.elasticsearch.common.io.stream.NamedWriteableRegistry; import org.elasticsearch.common.settings.Settings; import org.elasticsearch.common.settings.SettingsFilter; import org.elasticsearch.common.settings.SettingsModule; @@ -60,6 +65,7 @@ import org.elasticsearch.indices.query.IndicesQueriesRegistry; import org.elasticsearch.script.ScriptModule; import org.elasticsearch.script.ScriptService; import org.elasticsearch.script.mustache.MustacheScriptEngineService; +import org.elasticsearch.search.SearchModule; import org.elasticsearch.test.ESTestCase; import org.elasticsearch.test.IndexSettingsModule; import org.elasticsearch.test.InternalSettingsPlugin; @@ -68,10 +74,6 @@ import org.elasticsearch.threadpool.ThreadPoolModule; import org.junit.After; import org.junit.Before; -import java.io.IOException; -import java.lang.reflect.Proxy; -import java.util.Collections; - import static org.hamcrest.Matchers.containsString; /** @@ -107,11 +109,14 @@ public class TemplateQueryParserTests extends ESTestCase { new EnvironmentModule(new Environment(settings)), settingsModule, new ThreadPoolModule(new ThreadPool(settings)), - new IndicesModule() { + new SearchModule(settings, new NamedWriteableRegistry()) { @Override - public void configure() { - // skip services - bindQueryParsersExtension(); + protected void configureSearch() { + // skip so we don't need transport + } + @Override + protected void configureSuggesters() { + // skip so we don't need IndicesService } }, scriptModule, diff --git a/modules/lang-mustache/src/test/java/org/elasticsearch/script/mustache/MustacheRestIT.java b/modules/lang-mustache/src/test/java/org/elasticsearch/script/mustache/MustacheRestIT.java index 0c489b3afb19..727d0c4316d3 100644 --- a/modules/lang-mustache/src/test/java/org/elasticsearch/script/mustache/MustacheRestIT.java +++ b/modules/lang-mustache/src/test/java/org/elasticsearch/script/mustache/MustacheRestIT.java @@ -21,21 +21,14 @@ package org.elasticsearch.script.mustache; import com.carrotsearch.randomizedtesting.annotations.Name; import com.carrotsearch.randomizedtesting.annotations.ParametersFactory; -import org.elasticsearch.plugins.Plugin; import org.elasticsearch.test.rest.ESRestTestCase; import org.elasticsearch.test.rest.RestTestCandidate; import org.elasticsearch.test.rest.parser.RestTestParseException; import java.io.IOException; -import java.util.Collection; public class MustacheRestIT extends ESRestTestCase { - @Override - protected Collection> nodePlugins() { - return pluginList(MustachePlugin.class); - } - public MustacheRestIT(@Name("yaml") RestTestCandidate testCandidate) { super(testCandidate); } diff --git a/plugins/analysis-icu/src/test/java/org/elasticsearch/index/analysis/AnalysisICURestIT.java b/plugins/analysis-icu/src/test/java/org/elasticsearch/index/analysis/AnalysisICURestIT.java index bfb2b96a5ba5..8da56d5a72b4 100644 --- a/plugins/analysis-icu/src/test/java/org/elasticsearch/index/analysis/AnalysisICURestIT.java +++ b/plugins/analysis-icu/src/test/java/org/elasticsearch/index/analysis/AnalysisICURestIT.java @@ -21,22 +21,14 @@ package org.elasticsearch.index.analysis; import com.carrotsearch.randomizedtesting.annotations.Name; import com.carrotsearch.randomizedtesting.annotations.ParametersFactory; -import org.elasticsearch.plugin.analysis.icu.AnalysisICUPlugin; -import org.elasticsearch.plugins.Plugin; import org.elasticsearch.test.rest.ESRestTestCase; import org.elasticsearch.test.rest.RestTestCandidate; import org.elasticsearch.test.rest.parser.RestTestParseException; import java.io.IOException; -import java.util.Collection; public class AnalysisICURestIT extends ESRestTestCase { - @Override - protected Collection> nodePlugins() { - return pluginList(AnalysisICUPlugin.class); - } - public AnalysisICURestIT(@Name("yaml") RestTestCandidate testCandidate) { super(testCandidate); } diff --git a/plugins/analysis-kuromoji/src/test/java/org/elasticsearch/index/analysis/AnalysisKuromojiRestIT.java b/plugins/analysis-kuromoji/src/test/java/org/elasticsearch/index/analysis/AnalysisKuromojiRestIT.java index cb9435e430ec..ae51e491d6b9 100644 --- a/plugins/analysis-kuromoji/src/test/java/org/elasticsearch/index/analysis/AnalysisKuromojiRestIT.java +++ b/plugins/analysis-kuromoji/src/test/java/org/elasticsearch/index/analysis/AnalysisKuromojiRestIT.java @@ -21,23 +21,14 @@ package org.elasticsearch.index.analysis; import com.carrotsearch.randomizedtesting.annotations.Name; import com.carrotsearch.randomizedtesting.annotations.ParametersFactory; -import org.elasticsearch.plugin.analysis.kuromoji.AnalysisKuromojiPlugin; -import org.elasticsearch.plugins.Plugin; import org.elasticsearch.test.rest.ESRestTestCase; import org.elasticsearch.test.rest.RestTestCandidate; import org.elasticsearch.test.rest.parser.RestTestParseException; import java.io.IOException; -import java.util.Collection; public class AnalysisKuromojiRestIT extends ESRestTestCase { - @Override - protected Collection> nodePlugins() { - return pluginList(AnalysisKuromojiPlugin.class); - } - - public AnalysisKuromojiRestIT(@Name("yaml") RestTestCandidate testCandidate) { super(testCandidate); } diff --git a/plugins/analysis-phonetic/src/test/java/org/elasticsearch/index/analysis/AnalysisPhoneticRestIT.java b/plugins/analysis-phonetic/src/test/java/org/elasticsearch/index/analysis/AnalysisPhoneticRestIT.java index 98380a07176e..9d66bf24357e 100644 --- a/plugins/analysis-phonetic/src/test/java/org/elasticsearch/index/analysis/AnalysisPhoneticRestIT.java +++ b/plugins/analysis-phonetic/src/test/java/org/elasticsearch/index/analysis/AnalysisPhoneticRestIT.java @@ -21,22 +21,14 @@ package org.elasticsearch.index.analysis; import com.carrotsearch.randomizedtesting.annotations.Name; import com.carrotsearch.randomizedtesting.annotations.ParametersFactory; -import org.elasticsearch.plugin.analysis.AnalysisPhoneticPlugin; -import org.elasticsearch.plugins.Plugin; import org.elasticsearch.test.rest.ESRestTestCase; import org.elasticsearch.test.rest.RestTestCandidate; import org.elasticsearch.test.rest.parser.RestTestParseException; import java.io.IOException; -import java.util.Collection; public class AnalysisPhoneticRestIT extends ESRestTestCase { - @Override - protected Collection> nodePlugins() { - return pluginList(AnalysisPhoneticPlugin.class); - } - public AnalysisPhoneticRestIT(@Name("yaml") RestTestCandidate testCandidate) { super(testCandidate); } diff --git a/plugins/analysis-smartcn/src/test/java/org/elasticsearch/index/analysis/AnalysisSmartChineseRestIT.java b/plugins/analysis-smartcn/src/test/java/org/elasticsearch/index/analysis/AnalysisSmartChineseRestIT.java index 2a76c21d353d..16113b2b7ac5 100644 --- a/plugins/analysis-smartcn/src/test/java/org/elasticsearch/index/analysis/AnalysisSmartChineseRestIT.java +++ b/plugins/analysis-smartcn/src/test/java/org/elasticsearch/index/analysis/AnalysisSmartChineseRestIT.java @@ -21,22 +21,14 @@ package org.elasticsearch.index.analysis; import com.carrotsearch.randomizedtesting.annotations.Name; import com.carrotsearch.randomizedtesting.annotations.ParametersFactory; -import org.elasticsearch.plugin.analysis.smartcn.AnalysisSmartChinesePlugin; -import org.elasticsearch.plugins.Plugin; import org.elasticsearch.test.rest.ESRestTestCase; import org.elasticsearch.test.rest.RestTestCandidate; import org.elasticsearch.test.rest.parser.RestTestParseException; import java.io.IOException; -import java.util.Collection; public class AnalysisSmartChineseRestIT extends ESRestTestCase { - @Override - protected Collection> nodePlugins() { - return pluginList(AnalysisSmartChinesePlugin.class); - } - public AnalysisSmartChineseRestIT(@Name("yaml") RestTestCandidate testCandidate) { super(testCandidate); } diff --git a/plugins/analysis-stempel/src/test/java/org/elasticsearch/index/analysis/AnalysisPolishRestIT.java b/plugins/analysis-stempel/src/test/java/org/elasticsearch/index/analysis/AnalysisPolishRestIT.java index c99ff75aebf5..330ad87af74d 100644 --- a/plugins/analysis-stempel/src/test/java/org/elasticsearch/index/analysis/AnalysisPolishRestIT.java +++ b/plugins/analysis-stempel/src/test/java/org/elasticsearch/index/analysis/AnalysisPolishRestIT.java @@ -21,22 +21,14 @@ package org.elasticsearch.index.analysis; import com.carrotsearch.randomizedtesting.annotations.Name; import com.carrotsearch.randomizedtesting.annotations.ParametersFactory; -import org.elasticsearch.plugin.analysis.stempel.AnalysisStempelPlugin; -import org.elasticsearch.plugins.Plugin; import org.elasticsearch.test.rest.ESRestTestCase; import org.elasticsearch.test.rest.RestTestCandidate; import org.elasticsearch.test.rest.parser.RestTestParseException; import java.io.IOException; -import java.util.Collection; public class AnalysisPolishRestIT extends ESRestTestCase { - @Override - protected Collection> nodePlugins() { - return pluginList(AnalysisStempelPlugin.class); - } - public AnalysisPolishRestIT(@Name("yaml") RestTestCandidate testCandidate) { super(testCandidate); } diff --git a/plugins/delete-by-query/src/test/java/org/elasticsearch/plugin/deletebyquery/test/rest/DeleteByQueryRestIT.java b/plugins/delete-by-query/src/test/java/org/elasticsearch/plugin/deletebyquery/test/rest/DeleteByQueryRestIT.java index 038f117b83ff..9674d541354e 100644 --- a/plugins/delete-by-query/src/test/java/org/elasticsearch/plugin/deletebyquery/test/rest/DeleteByQueryRestIT.java +++ b/plugins/delete-by-query/src/test/java/org/elasticsearch/plugin/deletebyquery/test/rest/DeleteByQueryRestIT.java @@ -21,22 +21,14 @@ package org.elasticsearch.plugin.deletebyquery.test.rest; import com.carrotsearch.randomizedtesting.annotations.Name; import com.carrotsearch.randomizedtesting.annotations.ParametersFactory; -import org.elasticsearch.plugin.deletebyquery.DeleteByQueryPlugin; -import org.elasticsearch.plugins.Plugin; import org.elasticsearch.test.rest.ESRestTestCase; import org.elasticsearch.test.rest.RestTestCandidate; import org.elasticsearch.test.rest.parser.RestTestParseException; import java.io.IOException; -import java.util.Collection; public class DeleteByQueryRestIT extends ESRestTestCase { - @Override - protected Collection> nodePlugins() { - return pluginList(DeleteByQueryPlugin.class); - } - public DeleteByQueryRestIT(@Name("yaml") RestTestCandidate testCandidate) { super(testCandidate); } diff --git a/plugins/discovery-azure/build.gradle b/plugins/discovery-azure/build.gradle index 2451672f1d1a..ec4ef7cb6254 100644 --- a/plugins/discovery-azure/build.gradle +++ b/plugins/discovery-azure/build.gradle @@ -56,8 +56,6 @@ dependencyLicenses { mapping from: /jaxb-.*/, to: 'jaxb' } -compileJava.options.compilerArgs << '-Xlint:-path,-unchecked' - thirdPartyAudit.excludes = [ // classes are missing 'javax.servlet.ServletContextEvent', diff --git a/plugins/discovery-azure/src/test/java/org/elasticsearch/discovery/azure/AzureDiscoveryRestIT.java b/plugins/discovery-azure/src/test/java/org/elasticsearch/discovery/azure/AzureDiscoveryRestIT.java index 63888837cdde..131f73d1ca9d 100644 --- a/plugins/discovery-azure/src/test/java/org/elasticsearch/discovery/azure/AzureDiscoveryRestIT.java +++ b/plugins/discovery-azure/src/test/java/org/elasticsearch/discovery/azure/AzureDiscoveryRestIT.java @@ -21,22 +21,14 @@ package org.elasticsearch.discovery.azure; import com.carrotsearch.randomizedtesting.annotations.Name; import com.carrotsearch.randomizedtesting.annotations.ParametersFactory; -import org.elasticsearch.plugin.discovery.azure.AzureDiscoveryPlugin; -import org.elasticsearch.plugins.Plugin; import org.elasticsearch.test.rest.ESRestTestCase; import org.elasticsearch.test.rest.RestTestCandidate; import org.elasticsearch.test.rest.parser.RestTestParseException; import java.io.IOException; -import java.util.Collection; public class AzureDiscoveryRestIT extends ESRestTestCase { - @Override - protected Collection> nodePlugins() { - return pluginList(AzureDiscoveryPlugin.class); - } - public AzureDiscoveryRestIT(@Name("yaml") RestTestCandidate testCandidate) { super(testCandidate); } diff --git a/plugins/discovery-ec2/src/test/java/org/elasticsearch/cloud/aws/DiscoveryEc2RestIT.java b/plugins/discovery-ec2/src/test/java/org/elasticsearch/cloud/aws/DiscoveryEc2RestIT.java index 57021a085e66..24ccf82a3d89 100644 --- a/plugins/discovery-ec2/src/test/java/org/elasticsearch/cloud/aws/DiscoveryEc2RestIT.java +++ b/plugins/discovery-ec2/src/test/java/org/elasticsearch/cloud/aws/DiscoveryEc2RestIT.java @@ -21,22 +21,14 @@ package org.elasticsearch.cloud.aws; import com.carrotsearch.randomizedtesting.annotations.Name; import com.carrotsearch.randomizedtesting.annotations.ParametersFactory; -import org.elasticsearch.plugin.discovery.ec2.Ec2DiscoveryPlugin; -import org.elasticsearch.plugins.Plugin; import org.elasticsearch.test.rest.ESRestTestCase; import org.elasticsearch.test.rest.RestTestCandidate; import org.elasticsearch.test.rest.parser.RestTestParseException; import java.io.IOException; -import java.util.Collection; public class DiscoveryEc2RestIT extends ESRestTestCase { - @Override - protected Collection> nodePlugins() { - return pluginList(Ec2DiscoveryPlugin.class); - } - public DiscoveryEc2RestIT(@Name("yaml") RestTestCandidate testCandidate) { super(testCandidate); } diff --git a/plugins/discovery-gce/src/test/java/org/elasticsearch/discovery/gce/DiscoveryGCERestIT.java b/plugins/discovery-gce/src/test/java/org/elasticsearch/discovery/gce/DiscoveryGCERestIT.java index 1a218394b7de..891dd156aac5 100644 --- a/plugins/discovery-gce/src/test/java/org/elasticsearch/discovery/gce/DiscoveryGCERestIT.java +++ b/plugins/discovery-gce/src/test/java/org/elasticsearch/discovery/gce/DiscoveryGCERestIT.java @@ -21,22 +21,14 @@ package org.elasticsearch.discovery.gce; import com.carrotsearch.randomizedtesting.annotations.Name; import com.carrotsearch.randomizedtesting.annotations.ParametersFactory; -import org.elasticsearch.plugin.discovery.gce.GceDiscoveryPlugin; -import org.elasticsearch.plugins.Plugin; import org.elasticsearch.test.rest.ESRestTestCase; import org.elasticsearch.test.rest.RestTestCandidate; import org.elasticsearch.test.rest.parser.RestTestParseException; import java.io.IOException; -import java.util.Collection; public class DiscoveryGCERestIT extends ESRestTestCase { - @Override - protected Collection> nodePlugins() { - return pluginList(GceDiscoveryPlugin.class); - } - public DiscoveryGCERestIT(@Name("yaml") RestTestCandidate testCandidate) { super(testCandidate); } diff --git a/plugins/discovery-multicast/src/test/java/org/elasticsearch/plugin/discovery/multicast/MulticastDiscoveryRestIT.java b/plugins/discovery-multicast/src/test/java/org/elasticsearch/plugin/discovery/multicast/MulticastDiscoveryRestIT.java index d75c038a5006..c6af20c011ec 100644 --- a/plugins/discovery-multicast/src/test/java/org/elasticsearch/plugin/discovery/multicast/MulticastDiscoveryRestIT.java +++ b/plugins/discovery-multicast/src/test/java/org/elasticsearch/plugin/discovery/multicast/MulticastDiscoveryRestIT.java @@ -21,21 +21,14 @@ package org.elasticsearch.plugin.discovery.multicast; import com.carrotsearch.randomizedtesting.annotations.Name; import com.carrotsearch.randomizedtesting.annotations.ParametersFactory; -import org.elasticsearch.plugins.Plugin; import org.elasticsearch.test.rest.ESRestTestCase; import org.elasticsearch.test.rest.RestTestCandidate; import org.elasticsearch.test.rest.parser.RestTestParseException; import java.io.IOException; -import java.util.Collection; public class MulticastDiscoveryRestIT extends ESRestTestCase { - @Override - protected Collection> nodePlugins() { - return pluginList(MulticastDiscoveryPlugin.class); - } - public MulticastDiscoveryRestIT(@Name("yaml") RestTestCandidate testCandidate) { super(testCandidate); } diff --git a/plugins/jvm-example/src/test/java/org/elasticsearch/plugin/example/JvmExampleRestIT.java b/plugins/jvm-example/src/test/java/org/elasticsearch/plugin/example/JvmExampleRestIT.java index 2e9039ed67fd..74573a79289f 100644 --- a/plugins/jvm-example/src/test/java/org/elasticsearch/plugin/example/JvmExampleRestIT.java +++ b/plugins/jvm-example/src/test/java/org/elasticsearch/plugin/example/JvmExampleRestIT.java @@ -21,21 +21,14 @@ package org.elasticsearch.plugin.example; import com.carrotsearch.randomizedtesting.annotations.Name; import com.carrotsearch.randomizedtesting.annotations.ParametersFactory; -import org.elasticsearch.plugins.Plugin; import org.elasticsearch.test.rest.ESRestTestCase; import org.elasticsearch.test.rest.RestTestCandidate; import org.elasticsearch.test.rest.parser.RestTestParseException; import java.io.IOException; -import java.util.Collection; public class JvmExampleRestIT extends ESRestTestCase { - @Override - protected Collection> nodePlugins() { - return pluginList(JvmExamplePlugin.class); - } - public JvmExampleRestIT(@Name("yaml") RestTestCandidate testCandidate) { super(testCandidate); } diff --git a/plugins/lang-javascript/src/test/java/org/elasticsearch/script/javascript/LangJavaScriptRestIT.java b/plugins/lang-javascript/src/test/java/org/elasticsearch/script/javascript/LangJavaScriptRestIT.java index 18f18372b90e..8039715c3d1c 100644 --- a/plugins/lang-javascript/src/test/java/org/elasticsearch/script/javascript/LangJavaScriptRestIT.java +++ b/plugins/lang-javascript/src/test/java/org/elasticsearch/script/javascript/LangJavaScriptRestIT.java @@ -21,22 +21,14 @@ package org.elasticsearch.script.javascript; import com.carrotsearch.randomizedtesting.annotations.Name; import com.carrotsearch.randomizedtesting.annotations.ParametersFactory; -import org.elasticsearch.plugin.javascript.JavaScriptPlugin; -import org.elasticsearch.plugins.Plugin; import org.elasticsearch.test.rest.ESRestTestCase; import org.elasticsearch.test.rest.RestTestCandidate; import org.elasticsearch.test.rest.parser.RestTestParseException; import java.io.IOException; -import java.util.Collection; public class LangJavaScriptRestIT extends ESRestTestCase { - @Override - protected Collection> nodePlugins() { - return pluginList(JavaScriptPlugin.class); - } - public LangJavaScriptRestIT(@Name("yaml") RestTestCandidate testCandidate) { super(testCandidate); } diff --git a/plugins/lang-plan-a/build.gradle b/plugins/lang-plan-a/build.gradle index 810f0df4e166..f40bf4fd8e7f 100644 --- a/plugins/lang-plan-a/build.gradle +++ b/plugins/lang-plan-a/build.gradle @@ -35,9 +35,6 @@ dependencyLicenses { mapping from: /asm-.*/, to: 'asm' } -compileJava.options.compilerArgs << '-Xlint:-cast,-rawtypes' -compileTestJava.options.compilerArgs << '-Xlint:-unchecked' - // regeneration logic, comes in via ant right now // don't port it to gradle, it works fine. diff --git a/plugins/lang-plan-a/src/main/java/org/elasticsearch/plan/a/Definition.java b/plugins/lang-plan-a/src/main/java/org/elasticsearch/plan/a/Definition.java index 5c52a202919e..901dd4f8d32c 100644 --- a/plugins/lang-plan-a/src/main/java/org/elasticsearch/plan/a/Definition.java +++ b/plugins/lang-plan-a/src/main/java/org/elasticsearch/plan/a/Definition.java @@ -778,7 +778,7 @@ class Definition { addMethod("Def", "DefToLong", null, true, longobjType, new Type[] {defType}, null, null); addMethod("Def", "DefToFloat", null, true, floatobjType, new Type[] {defType}, null, null); addMethod("Def", "DefToDouble", null, true, doubleobjType, new Type[] {defType}, null, null); - + addMethod("List", "addLast", "add", false, booleanType, new Type[] {objectType}, null, new Type[] {defType}); addMethod("List", "add", null, false, voidType, new Type[] {intType, objectType}, null, new Type[] {intType, defType}); addMethod("List", "get", null, false, objectType, new Type[] {intType}, defType, null); @@ -984,7 +984,7 @@ class Definition { addTransform(defType, longobjType, "Def", "DefToLong", true); addTransform(defType, floatobjType, "Def", "DefToFloat", true); addTransform(defType, doubleobjType, "Def", "DefToDouble", true); - + addTransform(numberType, booleanType, "Utility", "NumberToboolean", true); addTransform(numberType, byteType, "Number", "byteValue", false); addTransform(numberType, shortType, "Number", "shortValue", false); @@ -1255,7 +1255,7 @@ class Definition { " [" + name + "] within the same struct [" + owner.name + "]."); } - final Class[] classes = new Class[args.length]; + final Class[] classes = new Class[args.length]; for (int count = 0; count < classes.length; ++count) { if (genargs != null) { @@ -1343,7 +1343,7 @@ class Definition { " [" + args.length + "] within the struct [" + owner.name + "]."); } - final Class[] classes = new Class[args.length]; + final Class[] classes = new Class[args.length]; for (int count = 0; count < classes.length; ++count) { if (genargs != null) { diff --git a/plugins/lang-plan-a/src/main/java/org/elasticsearch/plan/a/Utility.java b/plugins/lang-plan-a/src/main/java/org/elasticsearch/plan/a/Utility.java index 3bb5ae463e72..c31bcb91f10e 100644 --- a/plugins/lang-plan-a/src/main/java/org/elasticsearch/plan/a/Utility.java +++ b/plugins/lang-plan-a/src/main/java/org/elasticsearch/plan/a/Utility.java @@ -157,7 +157,7 @@ public class Utility { } public static Character byteToCharacter(final byte value) { - return (char)(byte)value; + return (char)value; } public static Integer byteToInteger(final byte value) { @@ -193,7 +193,7 @@ public class Utility { } public static Character shortToCharacter(final short value) { - return (char)(short)value; + return (char)value; } public static Integer shortToInteger(final short value) { @@ -211,11 +211,11 @@ public class Utility { public static Double shortToDouble(final short value) { return (double)value; } - + public static boolean ShortToboolean(final Short value) { return value != 0; } - + public static char ShortTochar(final Short value) { return (char)value.shortValue(); } @@ -261,19 +261,19 @@ public class Utility { } public static int CharacterToint(final Character value) { - return (int)value; + return value; } public static long CharacterTolong(final Character value) { - return (long)value; + return value; } public static float CharacterTofloat(final Character value) { - return (float)value; + return value; } public static double CharacterTodouble(final Character value) { - return (double)value; + return value; } public static Boolean CharacterToBoolean(final Character value) { @@ -317,7 +317,7 @@ public class Utility { } public static Character intToCharacter(final int value) { - return (char)(int)value; + return (char)value; } public static Long intToLong(final int value) { @@ -331,7 +331,7 @@ public class Utility { public static Double intToDouble(final int value) { return (double)value; } - + public static boolean IntegerToboolean(final Integer value) { return value != 0; } @@ -353,7 +353,7 @@ public class Utility { } public static Character longToCharacter(final long value) { - return (char)(long)value; + return (char)value; } public static Integer longToInteger(final long value) { @@ -367,7 +367,7 @@ public class Utility { public static Double longToDouble(final long value) { return (double)value; } - + public static boolean LongToboolean(final Long value) { return value != 0; } @@ -389,7 +389,7 @@ public class Utility { } public static Character floatToCharacter(final float value) { - return (char)(float)value; + return (char)value; } public static Integer floatToInteger(final float value) { @@ -403,11 +403,11 @@ public class Utility { public static Double floatToDouble(final float value) { return (double)value; } - + public static boolean FloatToboolean(final Float value) { return value != 0; } - + public static char FloatTochar(final Float value) { return (char)value.floatValue(); } @@ -425,7 +425,7 @@ public class Utility { } public static Character doubleToCharacter(final double value) { - return (char)(double)value; + return (char)value; } public static Integer doubleToInteger(final double value) { @@ -435,23 +435,23 @@ public class Utility { public static Long doubleToLong(final double value) { return (long)value; } - + public static Float doubleToFloat(final double value) { return (float)value; } - + public static boolean DoubleToboolean(final Double value) { return value != 0; } - + public static char DoubleTochar(final Double value) { return (char)value.doubleValue(); } - + // although divide by zero is guaranteed, the special overflow case is not caught. // its not needed for remainder because it is not possible there. // see https://docs.oracle.com/javase/specs/jls/se8/html/jls-15.html#jls-15.17.2 - + /** * Integer divide without overflow * @throws ArithmeticException on overflow or divide-by-zero @@ -462,7 +462,7 @@ public class Utility { } return x / y; } - + /** * Long divide without overflow * @throws ArithmeticException on overflow or divide-by-zero @@ -667,7 +667,7 @@ public class Utility { } return z; } - + /** * Checks for NaN, result is NaN but operands are finite * @throws ArithmeticException if overflow occurred @@ -680,7 +680,7 @@ public class Utility { } return z; } - + /** * Checks for NaN, result is infinite but operands are finite * @throws ArithmeticException if overflow occurred @@ -693,7 +693,7 @@ public class Utility { } return z; } - + /** * Checks for NaN, result is NaN but operands are finite * @throws ArithmeticException if overflow occurred @@ -706,7 +706,7 @@ public class Utility { } return z; } - + /** * Adds two floats but throws {@code ArithmeticException} * if the result overflows. @@ -714,7 +714,7 @@ public class Utility { public static float addWithoutOverflow(float x, float y) { return checkInfFloat(x, y, x + y); } - + /** * Adds two doubles but throws {@code ArithmeticException} * if the result overflows. @@ -722,7 +722,7 @@ public class Utility { public static double addWithoutOverflow(double x, double y) { return checkInfDouble(x, y, x + y); } - + /** * Subtracts two floats but throws {@code ArithmeticException} * if the result overflows. @@ -730,7 +730,7 @@ public class Utility { public static float subtractWithoutOverflow(float x, float y) { return checkInfFloat(x, y, x - y); } - + /** * Subtracts two doubles but throws {@code ArithmeticException} * if the result overflows. @@ -738,7 +738,7 @@ public class Utility { public static double subtractWithoutOverflow(double x, double y) { return checkInfDouble(x, y , x - y); } - + /** * Multiplies two floats but throws {@code ArithmeticException} * if the result overflows. @@ -746,7 +746,7 @@ public class Utility { public static float multiplyWithoutOverflow(float x, float y) { return checkInfFloat(x, y, x * y); } - + /** * Multiplies two doubles but throws {@code ArithmeticException} * if the result overflows. @@ -754,7 +754,7 @@ public class Utility { public static double multiplyWithoutOverflow(double x, double y) { return checkInfDouble(x, y, x * y); } - + /** * Divides two floats but throws {@code ArithmeticException} * if the result overflows, or would create NaN from finite @@ -763,7 +763,7 @@ public class Utility { public static float divideWithoutOverflow(float x, float y) { return checkNaNFloat(x, y, checkInfFloat(x, y, x / y)); } - + /** * Divides two doubles but throws {@code ArithmeticException} * if the result overflows, or would create NaN from finite @@ -772,7 +772,7 @@ public class Utility { public static double divideWithoutOverflow(double x, double y) { return checkNaNDouble(x, y, checkInfDouble(x, y, x / y)); } - + /** * Takes remainder two floats but throws {@code ArithmeticException} * if the result would create NaN from finite inputs ({@code y == 0}) @@ -780,7 +780,7 @@ public class Utility { public static float remainderWithoutOverflow(float x, float y) { return checkNaNFloat(x, y, x % y); } - + /** * Divides two doubles but throws {@code ArithmeticException} * if the result would create NaN from finite inputs ({@code y == 0}) diff --git a/plugins/lang-plan-a/src/test/java/org/elasticsearch/plan/a/PlanARestIT.java b/plugins/lang-plan-a/src/test/java/org/elasticsearch/plan/a/PlanARestIT.java index c2c19ccb03ac..5e0b0035cebe 100644 --- a/plugins/lang-plan-a/src/test/java/org/elasticsearch/plan/a/PlanARestIT.java +++ b/plugins/lang-plan-a/src/test/java/org/elasticsearch/plan/a/PlanARestIT.java @@ -21,22 +21,15 @@ package org.elasticsearch.plan.a; import com.carrotsearch.randomizedtesting.annotations.Name; import com.carrotsearch.randomizedtesting.annotations.ParametersFactory; -import org.elasticsearch.plugins.Plugin; import org.elasticsearch.test.rest.ESRestTestCase; import org.elasticsearch.test.rest.RestTestCandidate; import org.elasticsearch.test.rest.parser.RestTestParseException; import java.io.IOException; -import java.util.Collection; /** Runs yaml rest tests */ public class PlanARestIT extends ESRestTestCase { - @Override - protected Collection> nodePlugins() { - return pluginList(PlanAPlugin.class); - } - public PlanARestIT(@Name("yaml") RestTestCandidate testCandidate) { super(testCandidate); } diff --git a/plugins/lang-plan-a/src/test/java/org/elasticsearch/plan/a/ScriptEngineTests.java b/plugins/lang-plan-a/src/test/java/org/elasticsearch/plan/a/ScriptEngineTests.java index e5084392f99e..8f2991c3d0ca 100644 --- a/plugins/lang-plan-a/src/test/java/org/elasticsearch/plan/a/ScriptEngineTests.java +++ b/plugins/lang-plan-a/src/test/java/org/elasticsearch/plan/a/ScriptEngineTests.java @@ -35,6 +35,7 @@ public class ScriptEngineTests extends ScriptTestCase { assertEquals(3, ((Number)value).intValue()); } + @SuppressWarnings("unchecked") // We know its Map because we put them there in the test public void testMapAccess() { Map vars = new HashMap<>(); Map obj2 = new HashMap<>(); @@ -54,6 +55,7 @@ public class ScriptEngineTests extends ScriptTestCase { assertEquals("2", value); } + @SuppressWarnings("unchecked") // We know its Map because we put them there ourselves public void testAccessListInScript() { Map vars = new HashMap<>(); Map obj2 = new HashMap<>(); diff --git a/plugins/lang-python/src/test/java/org/elasticsearch/script/python/LangPythonScriptRestIT.java b/plugins/lang-python/src/test/java/org/elasticsearch/script/python/LangPythonScriptRestIT.java index d7d0ca662c8f..ee0a707644f5 100644 --- a/plugins/lang-python/src/test/java/org/elasticsearch/script/python/LangPythonScriptRestIT.java +++ b/plugins/lang-python/src/test/java/org/elasticsearch/script/python/LangPythonScriptRestIT.java @@ -21,22 +21,14 @@ package org.elasticsearch.script.python; import com.carrotsearch.randomizedtesting.annotations.Name; import com.carrotsearch.randomizedtesting.annotations.ParametersFactory; -import org.elasticsearch.plugin.python.PythonPlugin; -import org.elasticsearch.plugins.Plugin; import org.elasticsearch.test.rest.ESRestTestCase; import org.elasticsearch.test.rest.RestTestCandidate; import org.elasticsearch.test.rest.parser.RestTestParseException; import java.io.IOException; -import java.util.Collection; public class LangPythonScriptRestIT extends ESRestTestCase { - @Override - protected Collection> nodePlugins() { - return pluginList(PythonPlugin.class); - } - public LangPythonScriptRestIT(@Name("yaml") RestTestCandidate testCandidate) { super(testCandidate); } diff --git a/plugins/mapper-attachments/src/test/java/org/elasticsearch/mapper/attachments/MapperAttachmentsRestIT.java b/plugins/mapper-attachments/src/test/java/org/elasticsearch/mapper/attachments/MapperAttachmentsRestIT.java index 1eecda65a05c..1eccb1a14453 100644 --- a/plugins/mapper-attachments/src/test/java/org/elasticsearch/mapper/attachments/MapperAttachmentsRestIT.java +++ b/plugins/mapper-attachments/src/test/java/org/elasticsearch/mapper/attachments/MapperAttachmentsRestIT.java @@ -21,7 +21,6 @@ package org.elasticsearch.mapper.attachments; import com.carrotsearch.randomizedtesting.annotations.Name; import com.carrotsearch.randomizedtesting.annotations.ParametersFactory; -import org.elasticsearch.common.settings.Settings; import org.elasticsearch.test.rest.ESRestTestCase; import org.elasticsearch.test.rest.RestTestCandidate; import org.elasticsearch.test.rest.parser.RestTestParseException; @@ -30,14 +29,6 @@ import java.io.IOException; public class MapperAttachmentsRestIT extends ESRestTestCase { - @Override - protected Settings nodeSettings(int nodeOrdinal) { - return Settings.builder() - .put(super.nodeSettings(nodeOrdinal)) - .put("plugin.types", MapperAttachmentsPlugin.class.getName()) - .build(); - } - public MapperAttachmentsRestIT(@Name("yaml") RestTestCandidate testCandidate) { super(testCandidate); } diff --git a/plugins/mapper-murmur3/src/test/java/org/elasticsearch/index/mapper/murmur3/MapperMurmur3RestIT.java b/plugins/mapper-murmur3/src/test/java/org/elasticsearch/index/mapper/murmur3/MapperMurmur3RestIT.java index 97c5ad994a46..bbe342c716c0 100644 --- a/plugins/mapper-murmur3/src/test/java/org/elasticsearch/index/mapper/murmur3/MapperMurmur3RestIT.java +++ b/plugins/mapper-murmur3/src/test/java/org/elasticsearch/index/mapper/murmur3/MapperMurmur3RestIT.java @@ -21,22 +21,14 @@ package org.elasticsearch.index.mapper.murmur3; import com.carrotsearch.randomizedtesting.annotations.Name; import com.carrotsearch.randomizedtesting.annotations.ParametersFactory; -import org.elasticsearch.plugin.mapper.MapperMurmur3Plugin; -import org.elasticsearch.plugins.Plugin; import org.elasticsearch.test.rest.ESRestTestCase; import org.elasticsearch.test.rest.RestTestCandidate; import org.elasticsearch.test.rest.parser.RestTestParseException; import java.io.IOException; -import java.util.Collection; public class MapperMurmur3RestIT extends ESRestTestCase { - @Override - protected Collection> nodePlugins() { - return pluginList(MapperMurmur3Plugin.class); - } - public MapperMurmur3RestIT(@Name("yaml") RestTestCandidate testCandidate) { super(testCandidate); } diff --git a/plugins/mapper-size/src/test/java/org/elasticsearch/index/mapper/size/MapperSizeRestIT.java b/plugins/mapper-size/src/test/java/org/elasticsearch/index/mapper/size/MapperSizeRestIT.java index 9899776f7ddc..84df085f2215 100644 --- a/plugins/mapper-size/src/test/java/org/elasticsearch/index/mapper/size/MapperSizeRestIT.java +++ b/plugins/mapper-size/src/test/java/org/elasticsearch/index/mapper/size/MapperSizeRestIT.java @@ -21,22 +21,14 @@ package org.elasticsearch.index.mapper.size; import com.carrotsearch.randomizedtesting.annotations.Name; import com.carrotsearch.randomizedtesting.annotations.ParametersFactory; -import org.elasticsearch.plugin.mapper.MapperSizePlugin; -import org.elasticsearch.plugins.Plugin; import org.elasticsearch.test.rest.ESRestTestCase; import org.elasticsearch.test.rest.RestTestCandidate; import org.elasticsearch.test.rest.parser.RestTestParseException; import java.io.IOException; -import java.util.Collection; public class MapperSizeRestIT extends ESRestTestCase { - @Override - protected Collection> nodePlugins() { - return pluginList(MapperSizePlugin.class); - } - public MapperSizeRestIT(@Name("yaml") RestTestCandidate testCandidate) { super(testCandidate); } diff --git a/plugins/mapper-size/src/test/java/org/elasticsearch/index/mapper/size/SizeMappingTests.java b/plugins/mapper-size/src/test/java/org/elasticsearch/index/mapper/size/SizeMappingTests.java index 3e4d92beae27..a44dddda3ed4 100644 --- a/plugins/mapper-size/src/test/java/org/elasticsearch/index/mapper/size/SizeMappingTests.java +++ b/plugins/mapper-size/src/test/java/org/elasticsearch/index/mapper/size/SizeMappingTests.java @@ -147,12 +147,12 @@ public class SizeMappingTests extends ESSingleNodeTestCase { String enabledMapping = XContentFactory.jsonBuilder().startObject().startObject("type") .startObject("_size").field("enabled", true).endObject() .endObject().endObject().string(); - DocumentMapper enabledMapper = mapperService.merge("type", new CompressedXContent(enabledMapping), true, false); + DocumentMapper enabledMapper = mapperService.merge("type", new CompressedXContent(enabledMapping), MapperService.MergeReason.MAPPING_UPDATE, false); String disabledMapping = XContentFactory.jsonBuilder().startObject().startObject("type") .startObject("_size").field("enabled", false).endObject() .endObject().endObject().string(); - DocumentMapper disabledMapper = mapperService.merge("type", new CompressedXContent(disabledMapping), false, false); + DocumentMapper disabledMapper = mapperService.merge("type", new CompressedXContent(disabledMapping), MapperService.MergeReason.MAPPING_UPDATE, false); assertThat(disabledMapper.metadataMapper(SizeFieldMapper.class).enabled(), is(false)); } diff --git a/plugins/repository-azure/src/test/java/org/elasticsearch/repositories/azure/AzureRepositoryRestIT.java b/plugins/repository-azure/src/test/java/org/elasticsearch/repositories/azure/AzureRepositoryRestIT.java index d0267d51b0ef..ad58838bf5bc 100644 --- a/plugins/repository-azure/src/test/java/org/elasticsearch/repositories/azure/AzureRepositoryRestIT.java +++ b/plugins/repository-azure/src/test/java/org/elasticsearch/repositories/azure/AzureRepositoryRestIT.java @@ -21,22 +21,14 @@ package org.elasticsearch.repositories.azure; import com.carrotsearch.randomizedtesting.annotations.Name; import com.carrotsearch.randomizedtesting.annotations.ParametersFactory; -import org.elasticsearch.plugin.repository.azure.AzureRepositoryPlugin; -import org.elasticsearch.plugins.Plugin; import org.elasticsearch.test.rest.ESRestTestCase; import org.elasticsearch.test.rest.RestTestCandidate; import org.elasticsearch.test.rest.parser.RestTestParseException; import java.io.IOException; -import java.util.Collection; public class AzureRepositoryRestIT extends ESRestTestCase { - @Override - protected Collection> nodePlugins() { - return pluginList(AzureRepositoryPlugin.class); - } - public AzureRepositoryRestIT(@Name("yaml") RestTestCandidate testCandidate) { super(testCandidate); } diff --git a/plugins/repository-hdfs/src/test/java/org/elasticsearch/repositories/hdfs/HdfsRepositoryRestIT.java b/plugins/repository-hdfs/src/test/java/org/elasticsearch/repositories/hdfs/HdfsRepositoryRestIT.java index db423cdd44f8..dea6e8b749fe 100644 --- a/plugins/repository-hdfs/src/test/java/org/elasticsearch/repositories/hdfs/HdfsRepositoryRestIT.java +++ b/plugins/repository-hdfs/src/test/java/org/elasticsearch/repositories/hdfs/HdfsRepositoryRestIT.java @@ -19,24 +19,15 @@ package org.elasticsearch.repositories.hdfs; import java.io.IOException; -import java.util.Collection; import com.carrotsearch.randomizedtesting.annotations.Name; import com.carrotsearch.randomizedtesting.annotations.ParametersFactory; - -import org.elasticsearch.plugins.Plugin; -import org.elasticsearch.repositories.hdfs.HdfsPlugin; import org.elasticsearch.test.rest.ESRestTestCase; import org.elasticsearch.test.rest.RestTestCandidate; import org.elasticsearch.test.rest.parser.RestTestParseException; public class HdfsRepositoryRestIT extends ESRestTestCase { - @Override - protected Collection> nodePlugins() { - return pluginList(HdfsPlugin.class); - } - public HdfsRepositoryRestIT(@Name("yaml") RestTestCandidate testCandidate) { super(testCandidate); } diff --git a/plugins/repository-s3/src/test/java/org/elasticsearch/repositories/s3/RepositoryS3RestIT.java b/plugins/repository-s3/src/test/java/org/elasticsearch/repositories/s3/RepositoryS3RestIT.java index 8521780b1d8d..d8e436b50bb3 100644 --- a/plugins/repository-s3/src/test/java/org/elasticsearch/repositories/s3/RepositoryS3RestIT.java +++ b/plugins/repository-s3/src/test/java/org/elasticsearch/repositories/s3/RepositoryS3RestIT.java @@ -21,22 +21,14 @@ package org.elasticsearch.repositories.s3; import com.carrotsearch.randomizedtesting.annotations.Name; import com.carrotsearch.randomizedtesting.annotations.ParametersFactory; -import org.elasticsearch.plugin.repository.s3.S3RepositoryPlugin; -import org.elasticsearch.plugins.Plugin; import org.elasticsearch.test.rest.ESRestTestCase; import org.elasticsearch.test.rest.RestTestCandidate; import org.elasticsearch.test.rest.parser.RestTestParseException; import java.io.IOException; -import java.util.Collection; public class RepositoryS3RestIT extends ESRestTestCase { - @Override - protected Collection> nodePlugins() { - return pluginList(S3RepositoryPlugin.class); - } - public RepositoryS3RestIT(@Name("yaml") RestTestCandidate testCandidate) { super(testCandidate); } diff --git a/plugins/store-smb/src/test/java/org/elasticsearch/index/store/SMBStoreRestIT.java b/plugins/store-smb/src/test/java/org/elasticsearch/index/store/SMBStoreRestIT.java index 649af1737635..af1b03729955 100644 --- a/plugins/store-smb/src/test/java/org/elasticsearch/index/store/SMBStoreRestIT.java +++ b/plugins/store-smb/src/test/java/org/elasticsearch/index/store/SMBStoreRestIT.java @@ -21,22 +21,14 @@ package org.elasticsearch.index.store; import com.carrotsearch.randomizedtesting.annotations.Name; import com.carrotsearch.randomizedtesting.annotations.ParametersFactory; -import org.elasticsearch.plugin.store.smb.SMBStorePlugin; -import org.elasticsearch.plugins.Plugin; import org.elasticsearch.test.rest.ESRestTestCase; import org.elasticsearch.test.rest.RestTestCandidate; import org.elasticsearch.test.rest.parser.RestTestParseException; import java.io.IOException; -import java.util.Collection; public class SMBStoreRestIT extends ESRestTestCase { - @Override - protected Collection> nodePlugins() { - return pluginList(SMBStorePlugin.class); - } - public SMBStoreRestIT(@Name("yaml") RestTestCandidate testCandidate) { super(testCandidate); } diff --git a/qa/smoke-test-client/src/test/java/org/elasticsearch/smoketest/ESSmokeClientTestCase.java b/qa/smoke-test-client/src/test/java/org/elasticsearch/smoketest/ESSmokeClientTestCase.java index cddea9fd7749..227936beb426 100644 --- a/qa/smoke-test-client/src/test/java/org/elasticsearch/smoketest/ESSmokeClientTestCase.java +++ b/qa/smoke-test-client/src/test/java/org/elasticsearch/smoketest/ESSmokeClientTestCase.java @@ -67,11 +67,6 @@ public abstract class ESSmokeClientTestCase extends LuceneTestCase { */ public static final String TESTS_CLUSTER = "tests.cluster"; - /** - * Defaults to localhost:9300 - */ - public static final String TESTS_CLUSTER_DEFAULT = "localhost:9300"; - protected static final ESLogger logger = ESLoggerFactory.getLogger(ESSmokeClientTestCase.class.getName()); private static final AtomicInteger counter = new AtomicInteger(); @@ -131,11 +126,10 @@ public abstract class ESSmokeClientTestCase extends LuceneTestCase { } @BeforeClass - public static void initializeSettings() throws UnknownHostException { + public static void initializeSettings() { clusterAddresses = System.getProperty(TESTS_CLUSTER); if (clusterAddresses == null || clusterAddresses.isEmpty()) { - clusterAddresses = TESTS_CLUSTER_DEFAULT; - logger.info("[{}] not set. Falling back to [{}]", TESTS_CLUSTER, TESTS_CLUSTER_DEFAULT); + fail("Must specify " + TESTS_CLUSTER + " for smoke client test"); } } diff --git a/test/framework/src/main/java/org/elasticsearch/test/ActionRecordingPlugin.java b/test/framework/src/main/java/org/elasticsearch/test/ActionRecordingPlugin.java new file mode 100644 index 000000000000..a51c3f9eb400 --- /dev/null +++ b/test/framework/src/main/java/org/elasticsearch/test/ActionRecordingPlugin.java @@ -0,0 +1,138 @@ +/* + * 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.test; + +import org.elasticsearch.action.ActionListener; +import org.elasticsearch.action.ActionModule; +import org.elasticsearch.action.ActionRequest; +import org.elasticsearch.action.ActionResponse; +import org.elasticsearch.action.support.ActionFilter; +import org.elasticsearch.common.inject.AbstractModule; +import org.elasticsearch.common.inject.Inject; +import org.elasticsearch.common.inject.Module; +import org.elasticsearch.common.settings.Settings; +import org.elasticsearch.plugins.Plugin; + +import java.util.ArrayList; +import java.util.Collection; +import java.util.Collections; +import java.util.List; +import java.util.concurrent.CopyOnWriteArrayList; + +import static java.util.Collections.unmodifiableList; + +/** + * Plugin that registers a filter that records actions. + */ +public class ActionRecordingPlugin extends Plugin { + /** + * Fetch all the requests recorded by the test plugin. The list is an + * immutable, moment in time snapshot. + */ + public static List> allRequests() { + List> requests = new ArrayList<>(); + for (RecordingFilter filter : ESIntegTestCase.internalCluster().getInstances(RecordingFilter.class)) { + requests.addAll(filter.requests); + } + return unmodifiableList(requests); + } + + /** + * Fetch all requests recorded by the test plugin of a certain type. The + * list is an immutable, moment in time snapshot. + */ + public static List requestsOfType(Class type) { + List requests = new ArrayList<>(); + for (RecordingFilter filter : ESIntegTestCase.internalCluster().getInstances(RecordingFilter.class)) { + for (ActionRequest request : filter.requests) { + if (type.isInstance(request)) { + requests.add(type.cast(request)); + } + } + } + return unmodifiableList(requests); + } + + /** + * Clear all the recorded requests. Use between test methods that shared a + * suite scoped cluster. + */ + public static void clear() { + for (RecordingFilter filter : ESIntegTestCase.internalCluster().getInstances(RecordingFilter.class)) { + filter.requests.clear(); + } + } + + @Override + public String name() { + return "test-action-logging"; + } + + @Override + public String description() { + return "Test action logging"; + } + + @Override + public Collection nodeModules() { + return Collections.singletonList(new ActionRecordingModule()); + } + + public void onModule(ActionModule module) { + module.registerFilter(RecordingFilter.class); + } + + public static class ActionRecordingModule extends AbstractModule { + @Override + protected void configure() { + bind(RecordingFilter.class).asEagerSingleton(); + } + + } + + public static class RecordingFilter extends ActionFilter.Simple { + private final List> requests = new CopyOnWriteArrayList<>(); + + @Inject + public RecordingFilter(Settings settings) { + super(settings); + } + + public List> getRequests() { + return new ArrayList<>(requests); + } + + @Override + public int order() { + return 999; + } + + @Override + protected boolean apply(String action, ActionRequest request, ActionListener listener) { + requests.add(request); + return true; + } + + @Override + protected boolean apply(String action, ActionResponse response, ActionListener listener) { + return true; + } + } +} diff --git a/test/framework/src/main/java/org/elasticsearch/test/junit/listeners/ReproduceInfoPrinter.java b/test/framework/src/main/java/org/elasticsearch/test/junit/listeners/ReproduceInfoPrinter.java index 969d59d885ec..1c9bddb6b456 100644 --- a/test/framework/src/main/java/org/elasticsearch/test/junit/listeners/ReproduceInfoPrinter.java +++ b/test/framework/src/main/java/org/elasticsearch/test/junit/listeners/ReproduceInfoPrinter.java @@ -24,6 +24,7 @@ import org.elasticsearch.common.logging.ESLogger; import org.elasticsearch.common.logging.Loggers; import org.elasticsearch.test.ESIntegTestCase; import org.elasticsearch.test.ESTestCase; +import org.elasticsearch.test.rest.ESRestTestCase; import org.junit.internal.AssumptionViolatedException; import org.junit.runner.Description; import org.junit.runner.notification.Failure; @@ -39,7 +40,6 @@ import static org.elasticsearch.test.ESIntegTestCase.TESTS_CLUSTER; import static org.elasticsearch.test.rest.ESRestTestCase.REST_TESTS_BLACKLIST; import static org.elasticsearch.test.rest.ESRestTestCase.REST_TESTS_SPEC; import static org.elasticsearch.test.rest.ESRestTestCase.REST_TESTS_SUITE; -import static org.elasticsearch.test.rest.ESRestTestCase.Rest; /** * A {@link RunListener} that emits to {@link System#err} a string with command @@ -82,7 +82,7 @@ public class ReproduceInfoPrinter extends RunListener { gradleMessageBuilder.appendAllOpts(failure.getDescription()); //Rest tests are a special case as they allow for additional parameters - if (failure.getDescription().getTestClass().isAnnotationPresent(Rest.class)) { + if (ESRestTestCase.class.isAssignableFrom(failure.getDescription().getTestClass())) { gradleMessageBuilder.appendRestTestsProperties(); } diff --git a/test/framework/src/main/java/org/elasticsearch/test/rest/ESRestTestCase.java b/test/framework/src/main/java/org/elasticsearch/test/rest/ESRestTestCase.java index b4aecd52a141..5684717342d1 100644 --- a/test/framework/src/main/java/org/elasticsearch/test/rest/ESRestTestCase.java +++ b/test/framework/src/main/java/org/elasticsearch/test/rest/ESRestTestCase.java @@ -20,20 +20,12 @@ package org.elasticsearch.test.rest; import com.carrotsearch.randomizedtesting.RandomizedTest; -import com.carrotsearch.randomizedtesting.annotations.TestGroup; -import com.carrotsearch.randomizedtesting.annotations.TimeoutSuite; import org.apache.lucene.util.IOUtils; -import org.apache.lucene.util.LuceneTestCase.SuppressCodecs; -import org.apache.lucene.util.LuceneTestCase.SuppressFsync; -import org.apache.lucene.util.TimeUnits; import org.elasticsearch.common.Strings; import org.elasticsearch.common.SuppressForbidden; import org.elasticsearch.common.settings.Settings; import org.elasticsearch.common.xcontent.XContentHelper; -import org.elasticsearch.node.Node; -import org.elasticsearch.repositories.uri.URLRepository; -import org.elasticsearch.test.ESIntegTestCase; -import org.elasticsearch.test.ESIntegTestCase.ClusterScope; +import org.elasticsearch.test.ESTestCase; import org.elasticsearch.test.rest.client.RestException; import org.elasticsearch.test.rest.parser.RestTestParseException; import org.elasticsearch.test.rest.parser.RestTestSuiteParser; @@ -45,17 +37,14 @@ import org.elasticsearch.test.rest.section.TestSection; import org.elasticsearch.test.rest.spec.RestApi; import org.elasticsearch.test.rest.spec.RestSpec; import org.elasticsearch.test.rest.support.FileUtils; +import org.junit.After; import org.junit.AfterClass; import org.junit.Before; import org.junit.BeforeClass; import java.io.IOException; import java.io.InputStream; -import java.lang.annotation.ElementType; -import java.lang.annotation.Inherited; -import java.lang.annotation.Retention; -import java.lang.annotation.RetentionPolicy; -import java.lang.annotation.Target; +import java.net.InetSocketAddress; import java.net.URI; import java.net.URISyntaxException; import java.net.URL; @@ -67,6 +56,7 @@ import java.nio.file.StandardCopyOption; import java.util.ArrayList; import java.util.Collections; import java.util.Comparator; +import java.util.HashMap; import java.util.List; import java.util.Map; import java.util.Set; @@ -74,27 +64,7 @@ import java.util.Set; /** * Runs the clients test suite against an elasticsearch cluster. */ -@ESRestTestCase.Rest -@SuppressFsync // we aren't trying to test this here, and it can make the test slow -@SuppressCodecs("*") // requires custom completion postings format -@ClusterScope(randomDynamicTemplates = false) -@TimeoutSuite(millis = 40 * TimeUnits.MINUTE) // timeout the suite after 40min and fail the test. -public abstract class ESRestTestCase extends ESIntegTestCase { - - /** - * Property that allows to control whether the REST tests are run (default) or not - */ - public static final String TESTS_REST = "tests.rest"; - - /** - * Annotation for REST tests - */ - @Inherited - @Retention(RetentionPolicy.RUNTIME) - @Target(ElementType.TYPE) - @TestGroup(enabled = true, sysProperty = ESRestTestCase.TESTS_REST) - public @interface Rest { - } +public abstract class ESRestTestCase extends ESTestCase { /** * Property that allows to control which REST tests get run. Supports comma separated list of tests @@ -132,7 +102,9 @@ public abstract class ESRestTestCase extends ESIntegTestCase { private static final String PATHS_SEPARATOR = "(? blacklistPathMatchers = new ArrayList<>(); + private final URL[] clusterUrls; private static RestTestExecutionContext restTestExecutionContext; + private static RestTestExecutionContext adminExecutionContext; private final RestTestCandidate testCandidate; @@ -142,6 +114,20 @@ public abstract class ESRestTestCase extends ESIntegTestCase { for (String entry : blacklist) { this.blacklistPathMatchers.add(new BlacklistedPathPatternMatcher(entry)); } + String cluster = System.getProperty("tests.rest.cluster"); + if (cluster == null) { + throw new RuntimeException("Must specify tests.rest.cluster for rest tests"); + } + String[] stringUrls = cluster.split(","); + clusterUrls = new URL[stringUrls.length]; + int i = 0; + try { + for (String stringUrl : stringUrls) { + clusterUrls[i++] = new URL("http://" + stringUrl); + } + } catch (IOException e) { + throw new RuntimeException("Failed to parse cluster addresses for rest test", e); + } } @Override @@ -150,28 +136,7 @@ public abstract class ESRestTestCase extends ESIntegTestCase { super.afterIfFailed(errors); } - @Override - protected Settings nodeSettings(int nodeOrdinal) { - return Settings.builder() - .putArray(URLRepository.ALLOWED_URLS_SETTING, "http://snapshot.test*") - .put(Node.HTTP_ENABLED, true) - .put("node.testattr", "test") - .put(super.nodeSettings(nodeOrdinal)).build(); - } - public static Iterable createParameters(int id, int count) throws IOException, RestTestParseException { - TestGroup testGroup = Rest.class.getAnnotation(TestGroup.class); - String sysProperty = TestGroup.Utilities.getSysProperty(Rest.class); - boolean enabled; - try { - enabled = RandomizedTest.systemPropertyAsBoolean(sysProperty, testGroup.enabled()); - } catch (IllegalArgumentException e) { - // Ignore malformed system property, disable the group if malformed though. - enabled = false; - } - if (!enabled) { - return new ArrayList<>(); - } //parse tests only if rest test group is enabled, otherwise rest tests might not even be available on file system List restTestCandidates = collectTestCandidates(id, count); List objects = new ArrayList<>(); @@ -274,6 +239,7 @@ public abstract class ESRestTestCase extends ESIntegTestCase { } validateSpec(restSpec); restTestExecutionContext = new RestTestExecutionContext(restSpec); + adminExecutionContext = new RestTestExecutionContext(restSpec); } private static void validateSpec(RestSpec restSpec) { @@ -293,27 +259,42 @@ public abstract class ESRestTestCase extends ESIntegTestCase { } } + @After + public void wipeCluster() throws Exception { + + // wipe indices + Map deleteIndicesArgs = new HashMap<>(); + deleteIndicesArgs.put("index", "*"); + try { + adminExecutionContext.callApi("indices.delete", deleteIndicesArgs, Collections.emptyList(), Collections.emptyMap()); + } catch (RestException e) { + // 404 here just means we had no indexes + if (e.statusCode() != 404) { + throw e; + } + } + + // wipe index templates + Map deleteTemplatesArgs = new HashMap<>(); + deleteTemplatesArgs.put("name", "*"); + adminExecutionContext.callApi("indices.delete_template", deleteTemplatesArgs, Collections.emptyList(), Collections.emptyMap()); + + // wipe snapshots + Map deleteSnapshotsArgs = new HashMap<>(); + deleteSnapshotsArgs.put("repository", "*"); + adminExecutionContext.callApi("snapshot.delete_repository", deleteSnapshotsArgs, Collections.emptyList(), Collections.emptyMap()); + } + @AfterClass public static void close() { if (restTestExecutionContext != null) { restTestExecutionContext.close(); + adminExecutionContext.close(); restTestExecutionContext = null; + adminExecutionContext = null; } } - @Override - protected int maximumNumberOfShards() { - return 3; // never go crazy in the REST tests - } - - @Override - protected int maximumNumberOfReplicas() { - // hardcoded 1 since this is what clients also do and our tests must expect that we have only node - // with replicas set to 1 ie. the cluster won't be green - return 1; - - } - /** * Used to obtain settings for the REST client that is used to send REST requests. */ @@ -321,15 +302,29 @@ public abstract class ESRestTestCase extends ESIntegTestCase { return Settings.EMPTY; } + /** Returns the REST client settings used for admin actions like cleaning up after the test has completed. */ + protected Settings restAdminSettings() { + return restClientSettings(); // default to the same client settings + } + + /** Returns the addresses the client uses to connect to the test cluster. */ + protected URL[] getClusterUrls() { + return clusterUrls; + } + @Before public void reset() throws IOException, RestException { + // admin context must be available for @After always, regardless of whether the test was blacklisted + adminExecutionContext.initClient(clusterUrls, restAdminSettings()); + adminExecutionContext.clear(); + //skip test if it matches one of the blacklist globs for (BlacklistedPathPatternMatcher blacklistedPathMatcher : blacklistPathMatchers) { String testPath = testCandidate.getSuitePath() + "/" + testCandidate.getTestSection().getName(); assumeFalse("[" + testCandidate.getTestPath() + "] skipped, reason: blacklisted", blacklistedPathMatcher.isSuffixMatch(testPath)); } //The client needs non static info to get initialized, therefore it can't be initialized in the before class - restTestExecutionContext.initClient(cluster().httpAddresses(), restClientSettings()); + restTestExecutionContext.initClient(clusterUrls, restClientSettings()); restTestExecutionContext.clear(); //skip test if the whole suite (yaml file) is disabled diff --git a/test/framework/src/main/java/org/elasticsearch/test/rest/RestTestExecutionContext.java b/test/framework/src/main/java/org/elasticsearch/test/rest/RestTestExecutionContext.java index 4054b8efce13..83860b18bd9b 100644 --- a/test/framework/src/main/java/org/elasticsearch/test/rest/RestTestExecutionContext.java +++ b/test/framework/src/main/java/org/elasticsearch/test/rest/RestTestExecutionContext.java @@ -31,6 +31,7 @@ import org.elasticsearch.test.rest.spec.RestSpec; import java.io.Closeable; import java.io.IOException; import java.net.InetSocketAddress; +import java.net.URL; import java.util.HashMap; import java.util.List; import java.util.Map; @@ -119,9 +120,9 @@ public class RestTestExecutionContext implements Closeable { /** * Creates the embedded REST client when needed. Needs to be called before each test. */ - public void initClient(InetSocketAddress[] addresses, Settings settings) throws IOException, RestException { + public void initClient(URL[] urls, Settings settings) throws IOException, RestException { if (restClient == null) { - restClient = new RestClient(restSpec, settings, addresses); + restClient = new RestClient(restSpec, settings, urls); } } diff --git a/test/framework/src/main/java/org/elasticsearch/test/rest/client/RestClient.java b/test/framework/src/main/java/org/elasticsearch/test/rest/client/RestClient.java index 63a8b397c450..2b6ded9a5f59 100644 --- a/test/framework/src/main/java/org/elasticsearch/test/rest/client/RestClient.java +++ b/test/framework/src/main/java/org/elasticsearch/test/rest/client/RestClient.java @@ -35,6 +35,7 @@ import org.elasticsearch.common.Strings; import org.elasticsearch.common.io.PathUtils; import org.elasticsearch.common.logging.ESLogger; import org.elasticsearch.common.logging.Loggers; +import org.elasticsearch.common.network.InetAddresses; import org.elasticsearch.common.network.NetworkAddress; import org.elasticsearch.common.settings.Settings; import org.elasticsearch.common.util.set.Sets; @@ -48,6 +49,7 @@ import java.io.Closeable; import java.io.IOException; import java.io.InputStream; import java.net.InetSocketAddress; +import java.net.URL; import java.nio.file.Files; import java.nio.file.Path; import java.security.KeyManagementException; @@ -80,18 +82,18 @@ public class RestClient implements Closeable { private final RestSpec restSpec; private final CloseableHttpClient httpClient; private final Headers headers; - private final InetSocketAddress[] addresses; + private final URL[] urls; private final Version esVersion; - public RestClient(RestSpec restSpec, Settings settings, InetSocketAddress[] addresses) throws IOException, RestException { - assert addresses.length > 0; + public RestClient(RestSpec restSpec, Settings settings, URL[] urls) throws IOException, RestException { + assert urls.length > 0; this.restSpec = restSpec; this.headers = new Headers(settings); this.protocol = settings.get(PROTOCOL, "http"); this.httpClient = createHttpClient(settings); - this.addresses = addresses; + this.urls = urls; this.esVersion = readAndCheckVersion(); - logger.info("REST client initialized {}, elasticsearch version: [{}]", addresses, esVersion); + logger.info("REST client initialized {}, elasticsearch version: [{}]", urls, esVersion); } private Version readAndCheckVersion() throws IOException, RestException { @@ -102,8 +104,8 @@ public class RestClient implements Closeable { assert restApi.getMethods().size() == 1; String version = null; - for (InetSocketAddress address : addresses) { - RestResponse restResponse = new RestResponse(httpRequestBuilder(address) + for (URL url : urls) { + RestResponse restResponse = new RestResponse(httpRequestBuilder(url) .path(restApi.getPaths().get(0)) .method(restApi.getMethods().get(0)).execute()); checkStatusCode(restResponse); @@ -152,6 +154,8 @@ public class RestClient implements Closeable { HttpRequestBuilder httpRequestBuilder = callApiBuilder(apiName, requestParams, body); for (Map.Entry header : headers.entrySet()) { + logger.error("Adding header " + header.getKey()); + logger.error(" with value " + header.getValue()); httpRequestBuilder.addHeader(header.getKey(), header.getValue()); } logger.debug("calling api [{}]", apiName); @@ -246,17 +250,18 @@ public class RestClient implements Closeable { return restApi; } - protected HttpRequestBuilder httpRequestBuilder(InetSocketAddress address) { + protected HttpRequestBuilder httpRequestBuilder(URL url) { return new HttpRequestBuilder(httpClient) .addHeaders(headers) .protocol(protocol) - .host(NetworkAddress.formatAddress(address.getAddress())).port(address.getPort()); + .host(url.getHost()) + .port(url.getPort()); } protected HttpRequestBuilder httpRequestBuilder() { //the address used is randomized between the available ones - InetSocketAddress address = RandomizedTest.randomFrom(addresses); - return httpRequestBuilder(address); + URL url = RandomizedTest.randomFrom(urls); + return httpRequestBuilder(url); } protected CloseableHttpClient createHttpClient(Settings settings) throws IOException { diff --git a/test/framework/src/main/java/org/elasticsearch/test/rest/parser/RestTestSuiteParseContext.java b/test/framework/src/main/java/org/elasticsearch/test/rest/parser/RestTestSuiteParseContext.java index 036310e4e471..b99bf3475da2 100644 --- a/test/framework/src/main/java/org/elasticsearch/test/rest/parser/RestTestSuiteParseContext.java +++ b/test/framework/src/main/java/org/elasticsearch/test/rest/parser/RestTestSuiteParseContext.java @@ -136,7 +136,7 @@ public class RestTestSuiteParseContext { token = parser.nextToken(); } if (token != XContentParser.Token.FIELD_NAME) { - throw new RestTestParseException("malformed test section: field name expected but found " + token); + throw new RestTestParseException("malformed test section: field name expected but found " + token + " at " + parser.getTokenLocation()); } } diff --git a/test/framework/src/main/java/org/elasticsearch/test/rest/parser/RestTestSuiteParser.java b/test/framework/src/main/java/org/elasticsearch/test/rest/parser/RestTestSuiteParser.java index e8422887ad44..ba1b99288e31 100644 --- a/test/framework/src/main/java/org/elasticsearch/test/rest/parser/RestTestSuiteParser.java +++ b/test/framework/src/main/java/org/elasticsearch/test/rest/parser/RestTestSuiteParser.java @@ -18,11 +18,6 @@ */ package org.elasticsearch.test.rest.parser; -import org.elasticsearch.common.xcontent.XContentParser; -import org.elasticsearch.common.xcontent.yaml.YamlXContent; -import org.elasticsearch.test.rest.section.RestTestSuite; -import org.elasticsearch.test.rest.section.TestSection; - import java.io.IOException; import java.nio.ByteBuffer; import java.nio.channels.FileChannel; @@ -30,6 +25,11 @@ import java.nio.file.Files; import java.nio.file.Path; import java.nio.file.StandardOpenOption; +import org.elasticsearch.common.xcontent.XContentParser; +import org.elasticsearch.common.xcontent.yaml.YamlXContent; +import org.elasticsearch.test.rest.section.RestTestSuite; +import org.elasticsearch.test.rest.section.TestSection; + /** * Parser for a complete test suite (yaml file) */