Merge branch 'master' into feature/query-refactoring

This commit is contained in:
Christoph Büscher 2015-07-24 17:51:31 +02:00
commit 081d264f40
110 changed files with 1599 additions and 1287 deletions

View File

@ -374,9 +374,9 @@ public class MapperQueryParser extends QueryParser {
Query rangeQuery;
if (currentFieldType instanceof DateFieldMapper.DateFieldType && settings.timeZone() != null) {
DateFieldMapper.DateFieldType dateFieldType = (DateFieldMapper.DateFieldType) this.currentFieldType;
rangeQuery = dateFieldType.rangeQuery(part1, part2, startInclusive, endInclusive, settings.timeZone(), null, parseContext);
rangeQuery = dateFieldType.rangeQuery(part1, part2, startInclusive, endInclusive, settings.timeZone(), null);
} else {
rangeQuery = currentFieldType.rangeQuery(part1, part2, startInclusive, endInclusive, parseContext);
rangeQuery = currentFieldType.rangeQuery(part1, part2, startInclusive, endInclusive);
}
return rangeQuery;
} catch (RuntimeException e) {

View File

@ -19,7 +19,7 @@
package org.elasticsearch.action.admin.indices.analyze;
import org.elasticsearch.action.ActionRequestValidationException;
import org.elasticsearch.action.support.single.custom.SingleCustomOperationRequest;
import org.elasticsearch.action.support.single.shard.SingleShardRequest;
import org.elasticsearch.common.Strings;
import org.elasticsearch.common.io.stream.StreamInput;
import org.elasticsearch.common.io.stream.StreamOutput;
@ -32,7 +32,7 @@ import static org.elasticsearch.action.ValidateActions.addValidationError;
* A request to analyze a text associated with a specific index. Allow to provide
* the actual analyzer name to perform the analysis with.
*/
public class AnalyzeRequest extends SingleCustomOperationRequest<AnalyzeRequest> {
public class AnalyzeRequest extends SingleShardRequest<AnalyzeRequest> {
private String[] text;
@ -114,7 +114,7 @@ public class AnalyzeRequest extends SingleCustomOperationRequest<AnalyzeRequest>
@Override
public ActionRequestValidationException validate() {
ActionRequestValidationException validationException = super.validate();
ActionRequestValidationException validationException = null;
if (text == null || text.length == 0) {
validationException = addValidationError("text is missing", validationException);
}

View File

@ -18,13 +18,13 @@
*/
package org.elasticsearch.action.admin.indices.analyze;
import org.elasticsearch.action.support.single.custom.SingleCustomOperationRequestBuilder;
import org.elasticsearch.action.support.single.shard.SingleShardOperationRequestBuilder;
import org.elasticsearch.client.ElasticsearchClient;
/**
*
*/
public class AnalyzeRequestBuilder extends SingleCustomOperationRequestBuilder<AnalyzeRequest, AnalyzeResponse, AnalyzeRequestBuilder> {
public class AnalyzeRequestBuilder extends SingleShardOperationRequestBuilder<AnalyzeRequest, AnalyzeResponse, AnalyzeRequestBuilder> {
public AnalyzeRequestBuilder(ElasticsearchClient client, AnalyzeAction action) {
super(client, action, new AnalyzeRequest());
@ -34,15 +34,6 @@ public class AnalyzeRequestBuilder extends SingleCustomOperationRequestBuilder<A
super(client, action, new AnalyzeRequest(index).text(text));
}
/**
* Sets the index to use to analyzer the text (for example, if it holds specific analyzers
* registered).
*/
public AnalyzeRequestBuilder setIndex(String index) {
request.index(index);
return this;
}
/**
* Sets the analyzer name to use in order to analyze the text.
*

View File

@ -29,7 +29,7 @@ import org.apache.lucene.util.IOUtils;
import org.elasticsearch.ElasticsearchException;
import org.elasticsearch.Version;
import org.elasticsearch.action.support.ActionFilters;
import org.elasticsearch.action.support.single.custom.TransportSingleCustomOperationAction;
import org.elasticsearch.action.support.single.shard.TransportSingleShardAction;
import org.elasticsearch.cluster.ClusterService;
import org.elasticsearch.cluster.ClusterState;
import org.elasticsearch.cluster.block.ClusterBlockException;
@ -54,7 +54,7 @@ import java.util.List;
/**
* Transport action used to execute analyze requests
*/
public class TransportAnalyzeAction extends TransportSingleCustomOperationAction<AnalyzeRequest, AnalyzeResponse> {
public class TransportAnalyzeAction extends TransportSingleShardAction<AnalyzeRequest, AnalyzeResponse> {
private final IndicesService indicesService;
private final IndicesAnalysisService indicesAnalysisService;

View File

@ -19,16 +19,17 @@
package org.elasticsearch.action.admin.indices.mapping.get;
import org.elasticsearch.action.ActionRequestValidationException;
import org.elasticsearch.action.OriginalIndices;
import org.elasticsearch.action.support.IndicesOptions;
import org.elasticsearch.action.support.single.custom.SingleCustomOperationRequest;
import org.elasticsearch.action.support.single.shard.SingleShardRequest;
import org.elasticsearch.common.Strings;
import org.elasticsearch.common.io.stream.StreamInput;
import org.elasticsearch.common.io.stream.StreamOutput;
import java.io.IOException;
class GetFieldMappingsIndexRequest extends SingleCustomOperationRequest<GetFieldMappingsIndexRequest> {
class GetFieldMappingsIndexRequest extends SingleShardRequest<GetFieldMappingsIndexRequest> {
private boolean probablySingleFieldRequest;
private boolean includeDefaults;
@ -42,7 +43,6 @@ class GetFieldMappingsIndexRequest extends SingleCustomOperationRequest<GetField
GetFieldMappingsIndexRequest(GetFieldMappingsRequest other, String index, boolean probablySingleFieldRequest) {
super(other);
this.preferLocal(other.local);
this.probablySingleFieldRequest = probablySingleFieldRequest;
this.includeDefaults = other.includeDefaults();
this.types = other.types();
@ -52,6 +52,11 @@ class GetFieldMappingsIndexRequest extends SingleCustomOperationRequest<GetField
this.originalIndices = new OriginalIndices(other);
}
@Override
public ActionRequestValidationException validate() {
return null;
}
public String[] types() {
return types;
}
@ -88,11 +93,6 @@ class GetFieldMappingsIndexRequest extends SingleCustomOperationRequest<GetField
OriginalIndices.writeOriginalIndices(originalIndices, out);
}
@Override
protected void writeIndex(StreamOutput out) throws IOException {
out.writeString(index());
}
@Override
public void readFrom(StreamInput in) throws IOException {
super.readFrom(in);
@ -103,8 +103,4 @@ class GetFieldMappingsIndexRequest extends SingleCustomOperationRequest<GetField
originalIndices = OriginalIndices.readOriginalIndices(in);
}
@Override
protected void readIndex(StreamInput in) throws IOException {
index(in.readString());
}
}

View File

@ -26,7 +26,7 @@ import com.google.common.collect.Lists;
import org.elasticsearch.ElasticsearchException;
import org.elasticsearch.action.admin.indices.mapping.get.GetFieldMappingsResponse.FieldMappingMetaData;
import org.elasticsearch.action.support.ActionFilters;
import org.elasticsearch.action.support.single.custom.TransportSingleCustomOperationAction;
import org.elasticsearch.action.support.single.shard.TransportSingleShardAction;
import org.elasticsearch.cluster.ClusterService;
import org.elasticsearch.cluster.ClusterState;
import org.elasticsearch.cluster.block.ClusterBlockException;
@ -58,7 +58,7 @@ import java.util.Iterator;
/**
* Transport action used to retrieve the mappings related to fields that belong to a specific index
*/
public class TransportGetFieldMappingsIndexAction extends TransportSingleCustomOperationAction<GetFieldMappingsIndexRequest, GetFieldMappingsResponse> {
public class TransportGetFieldMappingsIndexAction extends TransportSingleShardAction<GetFieldMappingsIndexRequest, GetFieldMappingsResponse> {
private static final String ACTION_NAME = GetFieldMappingsAction.NAME + "[index]";

View File

@ -152,7 +152,7 @@ public class ExplainRequest extends SingleShardRequest<ExplainRequest> {
@Override
public ActionRequestValidationException validate() {
ActionRequestValidationException validationException = super.validate();
ActionRequestValidationException validationException = super.validateNonNullIndex();
if (type == null) {
validationException = ValidateActions.addValidationError("type is missing", validationException);
}

View File

@ -88,7 +88,7 @@ public class TransportExplainAction extends TransportSingleShardAction<ExplainRe
}
@Override
protected boolean resolveIndex() {
protected boolean resolveIndex(ExplainRequest request) {
return true;
}

View File

@ -118,7 +118,7 @@ public class GetRequest extends SingleShardRequest<GetRequest> {
@Override
public ActionRequestValidationException validate() {
ActionRequestValidationException validationException = super.validate();
ActionRequestValidationException validationException = super.validateNonNullIndex();
if (type == null) {
validationException = ValidateActions.addValidationError("type is missing", validationException);
}

View File

@ -20,6 +20,7 @@
package org.elasticsearch.action.get;
import com.carrotsearch.hppc.IntArrayList;
import org.elasticsearch.action.ActionRequestValidationException;
import org.elasticsearch.action.support.single.shard.SingleShardRequest;
import org.elasticsearch.common.io.stream.StreamInput;
import org.elasticsearch.common.io.stream.StreamOutput;
@ -54,6 +55,11 @@ public class MultiGetShardRequest extends SingleShardRequest<MultiGetShardReques
ignoreErrorsOnGeneratedFields = multiGetRequest.ignoreErrorsOnGeneratedFields;
}
@Override
public ActionRequestValidationException validate() {
return super.validateNonNullIndex();
}
public int shardId() {
return this.shardId;
}

View File

@ -58,7 +58,7 @@ public class TransportGetAction extends TransportSingleShardAction<GetRequest, G
}
@Override
protected boolean resolveIndex() {
protected boolean resolveIndex(GetRequest request) {
return true;
}

View File

@ -20,7 +20,6 @@
package org.elasticsearch.action.get;
import org.elasticsearch.ElasticsearchException;
import org.elasticsearch.ExceptionsHelper;
import org.elasticsearch.action.support.ActionFilters;
import org.elasticsearch.action.support.TransportActions;
import org.elasticsearch.action.support.single.shard.TransportSingleShardAction;
@ -68,7 +67,7 @@ public class TransportShardMultiGetAction extends TransportSingleShardAction<Mul
}
@Override
protected boolean resolveIndex() {
protected boolean resolveIndex(MultiGetShardRequest request) {
return true;
}

View File

@ -21,6 +21,7 @@ package org.elasticsearch.action.percolate;
import org.elasticsearch.ElasticsearchException;
import org.elasticsearch.ExceptionsHelper;
import org.elasticsearch.action.ActionRequestValidationException;
import org.elasticsearch.action.ActionResponse;
import org.elasticsearch.action.IndicesRequest;
import org.elasticsearch.action.OriginalIndices;
@ -76,7 +77,7 @@ public class TransportShardMultiPercolateAction extends TransportSingleShardActi
}
@Override
protected boolean resolveIndex() {
protected boolean resolveIndex(Request request) {
return false;
}
@ -127,6 +128,11 @@ public class TransportShardMultiPercolateAction extends TransportSingleShardActi
this.items = new ArrayList<>();
}
@Override
public ActionRequestValidationException validate() {
return super.validateNonNullIndex();
}
@Override
public String[] indices() {
List<String> indices = new ArrayList<>();

View File

@ -1,142 +0,0 @@
/*
* Licensed to Elasticsearch under one or more contributor
* license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright
* ownership. Elasticsearch licenses this file to you under
* the Apache License, Version 2.0 (the "License"); you may
* not use this file except in compliance with the License.
* You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing,
* software distributed under the License is distributed on an
* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
* KIND, either express or implied. See the License for the
* specific language governing permissions and limitations
* under the License.
*/
package org.elasticsearch.action.support.single.custom;
import org.elasticsearch.action.ActionRequest;
import org.elasticsearch.action.ActionRequestValidationException;
import org.elasticsearch.action.IndicesRequest;
import org.elasticsearch.action.support.IndicesOptions;
import org.elasticsearch.common.Strings;
import org.elasticsearch.common.io.stream.StreamInput;
import org.elasticsearch.common.io.stream.StreamOutput;
import org.elasticsearch.index.shard.ShardId;
import java.io.IOException;
/**
*
*/
public abstract class SingleCustomOperationRequest<T extends SingleCustomOperationRequest> extends ActionRequest<T> implements IndicesRequest {
ShardId internalShardId;
private boolean threadedOperation = true;
private boolean preferLocal = true;
private String index;
protected SingleCustomOperationRequest() {
}
protected SingleCustomOperationRequest(ActionRequest request) {
super(request);
}
@Override
public ActionRequestValidationException validate() {
return null;
}
/**
* Controls if the operation will be executed on a separate thread when executed locally.
*/
public boolean operationThreaded() {
return threadedOperation;
}
/**
* Controls if the operation will be executed on a separate thread when executed locally.
*/
@SuppressWarnings("unchecked")
public final T operationThreaded(boolean threadedOperation) {
this.threadedOperation = threadedOperation;
return (T) this;
}
/**
* if this operation hits a node with a local relevant shard, should it be preferred
* to be executed on, or just do plain round robin. Defaults to <tt>true</tt>
*/
@SuppressWarnings("unchecked")
public final T preferLocal(boolean preferLocal) {
this.preferLocal = preferLocal;
return (T) this;
}
@SuppressWarnings("unchecked")
public T index(String index) {
this.index = index;
return (T)this;
}
public String index() {
return index;
}
@Override
public IndicesOptions indicesOptions() {
return IndicesOptions.strictSingleIndexNoExpandForbidClosed();
}
@Override
public String[] indices() {
if (index == null) {
return Strings.EMPTY_ARRAY;
}
return new String[]{index};
}
/**
* if this operation hits a node with a local relevant shard, should it be preferred
* to be executed on, or just do plain round robin. Defaults to <tt>true</tt>
*/
public boolean preferLocalShard() {
return this.preferLocal;
}
public void beforeLocalFork() {
}
@Override
public void readFrom(StreamInput in) throws IOException {
super.readFrom(in);
if (in.readBoolean()) {
internalShardId = ShardId.readShardId(in);
}
preferLocal = in.readBoolean();
readIndex(in);
}
protected void readIndex(StreamInput in) throws IOException {
index = in.readOptionalString();
}
@Override
public void writeTo(StreamOutput out) throws IOException {
super.writeTo(out);
out.writeOptionalStreamable(internalShardId);
out.writeBoolean(preferLocal);
writeIndex(out);
}
protected void writeIndex(StreamOutput out) throws IOException {
out.writeOptionalString(index);
}
}

View File

@ -1,56 +0,0 @@
/*
* Licensed to Elasticsearch under one or more contributor
* license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright
* ownership. Elasticsearch licenses this file to you under
* the Apache License, Version 2.0 (the "License"); you may
* not use this file except in compliance with the License.
* You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing,
* software distributed under the License is distributed on an
* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
* KIND, either express or implied. See the License for the
* specific language governing permissions and limitations
* under the License.
*/
package org.elasticsearch.action.support.single.custom;
import org.elasticsearch.action.Action;
import org.elasticsearch.action.ActionRequestBuilder;
import org.elasticsearch.action.ActionResponse;
import org.elasticsearch.client.Client;
import org.elasticsearch.client.ElasticsearchClient;
import org.elasticsearch.client.IndicesAdminClient;
/**
*/
public abstract class SingleCustomOperationRequestBuilder<Request extends SingleCustomOperationRequest<Request>, Response extends ActionResponse, RequestBuilder extends SingleCustomOperationRequestBuilder<Request, Response, RequestBuilder>>
extends ActionRequestBuilder<Request, Response, RequestBuilder> {
protected SingleCustomOperationRequestBuilder(ElasticsearchClient client, Action<Request, Response, RequestBuilder> action, Request request) {
super(client, action, request);
}
/**
* Controls if the operation will be executed on a separate thread when executed locally.
*/
@SuppressWarnings("unchecked")
public final RequestBuilder setOperationThreaded(boolean threadedOperation) {
request.operationThreaded(threadedOperation);
return (RequestBuilder) this;
}
/**
* if this operation hits a node with a local relevant shard, should it be preferred
* to be executed on, or just do plain round robin. Defaults to <tt>true</tt>
*/
@SuppressWarnings("unchecked")
public final RequestBuilder setPreferLocal(boolean preferLocal) {
request.preferLocal(preferLocal);
return (RequestBuilder) this;
}
}

View File

@ -1,314 +0,0 @@
/*
* Licensed to Elasticsearch under one or more contributor
* license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright
* ownership. Elasticsearch licenses this file to you under
* the Apache License, Version 2.0 (the "License"); you may
* not use this file except in compliance with the License.
* You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing,
* software distributed under the License is distributed on an
* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
* KIND, either express or implied. See the License for the
* specific language governing permissions and limitations
* under the License.
*/
package org.elasticsearch.action.support.single.custom;
import org.elasticsearch.action.ActionListener;
import org.elasticsearch.action.ActionResponse;
import org.elasticsearch.action.NoShardAvailableActionException;
import org.elasticsearch.action.support.ActionFilters;
import org.elasticsearch.action.support.HandledTransportAction;
import org.elasticsearch.cluster.ClusterService;
import org.elasticsearch.cluster.ClusterState;
import org.elasticsearch.cluster.block.ClusterBlockException;
import org.elasticsearch.cluster.block.ClusterBlockLevel;
import org.elasticsearch.cluster.metadata.IndexNameExpressionResolver;
import org.elasticsearch.cluster.node.DiscoveryNode;
import org.elasticsearch.cluster.node.DiscoveryNodes;
import org.elasticsearch.cluster.routing.ShardRouting;
import org.elasticsearch.cluster.routing.ShardsIterator;
import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.index.shard.ShardId;
import org.elasticsearch.threadpool.ThreadPool;
import org.elasticsearch.transport.*;
/**
* Transport action used to send a read request to one of the shards that belong to an index.
* Supports retrying another shard in case of failure.
*/
public abstract class TransportSingleCustomOperationAction<Request extends SingleCustomOperationRequest, Response extends ActionResponse> extends HandledTransportAction<Request, Response> {
protected final ClusterService clusterService;
protected final TransportService transportService;
final String transportShardAction;
final String executor;
protected TransportSingleCustomOperationAction(Settings settings, String actionName, ThreadPool threadPool,
ClusterService clusterService, TransportService transportService,
ActionFilters actionFilters, IndexNameExpressionResolver indexNameExpressionResolver,
Class<Request> request, String executor) {
super(settings, actionName, threadPool, transportService, actionFilters, indexNameExpressionResolver, request);
this.clusterService = clusterService;
this.transportService = transportService;
this.transportShardAction = actionName + "[s]";
this.executor = executor;
transportService.registerRequestHandler(transportShardAction, request, executor, new ShardTransportHandler());
}
@Override
protected void doExecute(Request request, ActionListener<Response> listener) {
new AsyncSingleAction(request, listener).start();
}
/**
* Can return null to execute on this local node.
*/
protected abstract ShardsIterator shards(ClusterState state, InternalRequest request);
/**
* Operation to be executed at the shard level. Can be called with shardId set to null, meaning that there is no
* shard involved and the operation just needs to be executed on the local node.
*/
protected abstract Response shardOperation(Request request, ShardId shardId);
protected abstract Response newResponse();
protected ClusterBlockException checkGlobalBlock(ClusterState state) {
return state.blocks().globalBlockedException(ClusterBlockLevel.READ);
}
protected ClusterBlockException checkRequestBlock(ClusterState state, InternalRequest request) {
return state.blocks().indexBlockedException(ClusterBlockLevel.READ, request.concreteIndex());
}
protected abstract boolean resolveIndex(Request request);
private class AsyncSingleAction {
private final ActionListener<Response> listener;
private final ShardsIterator shardsIt;
private final InternalRequest internalRequest;
private final DiscoveryNodes nodes;
private AsyncSingleAction(Request request, ActionListener<Response> listener) {
this.listener = listener;
ClusterState clusterState = clusterService.state();
nodes = clusterState.nodes();
ClusterBlockException blockException = checkGlobalBlock(clusterState);
if (blockException != null) {
throw blockException;
}
String concreteSingleIndex;
if (resolveIndex(request)) {
concreteSingleIndex = indexNameExpressionResolver.concreteSingleIndex(clusterState, request);
} else {
concreteSingleIndex = request.index();
}
this.internalRequest = new InternalRequest(request, concreteSingleIndex);
blockException = checkRequestBlock(clusterState, internalRequest);
if (blockException != null) {
throw blockException;
}
this.shardsIt = shards(clusterState, internalRequest);
}
public void start() {
performFirst();
}
private void onFailure(ShardRouting shardRouting, Throwable e) {
if (logger.isTraceEnabled() && e != null) {
logger.trace(shardRouting.shortSummary() + ": Failed to execute [" + internalRequest.request() + "]", e);
}
perform(e);
}
/**
* First get should try and use a shard that exists on a local node for better performance
*/
private void performFirst() {
if (shardsIt == null) {
// just execute it on the local node
if (internalRequest.request().operationThreaded()) {
internalRequest.request().beforeLocalFork();
threadPool.executor(executor).execute(new Runnable() {
@Override
public void run() {
try {
Response response = shardOperation(internalRequest.request(), null);
listener.onResponse(response);
} catch (Throwable e) {
onFailure(null, e);
}
}
});
return;
} else {
try {
final Response response = shardOperation(internalRequest.request(), null);
listener.onResponse(response);
return;
} catch (Throwable e) {
onFailure(null, e);
}
}
return;
}
if (internalRequest.request().preferLocalShard()) {
boolean foundLocal = false;
ShardRouting shardX;
while ((shardX = shardsIt.nextOrNull()) != null) {
final ShardRouting shard = shardX;
if (shard.currentNodeId().equals(nodes.localNodeId())) {
foundLocal = true;
if (internalRequest.request().operationThreaded()) {
internalRequest.request().beforeLocalFork();
threadPool.executor(executor).execute(new Runnable() {
@Override
public void run() {
try {
Response response = shardOperation(internalRequest.request(), shard.shardId());
listener.onResponse(response);
} catch (Throwable e) {
shardsIt.reset();
onFailure(shard, e);
}
}
});
return;
} else {
try {
final Response response = shardOperation(internalRequest.request(), shard.shardId());
listener.onResponse(response);
return;
} catch (Throwable e) {
shardsIt.reset();
onFailure(shard, e);
}
}
}
}
if (!foundLocal) {
// no local node get, go remote
shardsIt.reset();
perform(null);
}
} else {
perform(null);
}
}
private void perform(final Throwable lastException) {
final ShardRouting shard = shardsIt == null ? null : shardsIt.nextOrNull();
if (shard == null) {
Throwable failure = lastException;
if (failure == null) {
failure = new NoShardAvailableActionException(null, "No shard available for [" + internalRequest.request() + "]");
} else {
if (logger.isDebugEnabled()) {
logger.debug("failed to execute [" + internalRequest.request() + "]", failure);
}
}
listener.onFailure(failure);
} else {
if (shard.currentNodeId().equals(nodes.localNodeId())) {
// we don't prefer local shard, so try and do it here
if (!internalRequest.request().preferLocalShard()) {
try {
if (internalRequest.request().operationThreaded()) {
internalRequest.request().beforeLocalFork();
threadPool.executor(executor).execute(new Runnable() {
@Override
public void run() {
try {
Response response = shardOperation(internalRequest.request(), shard.shardId());
listener.onResponse(response);
} catch (Throwable e) {
onFailure(shard, e);
}
}
});
} else {
final Response response = shardOperation(internalRequest.request(), shard.shardId());
listener.onResponse(response);
}
} catch (Throwable e) {
onFailure(shard, e);
}
} else {
perform(lastException);
}
} else {
DiscoveryNode node = nodes.get(shard.currentNodeId());
internalRequest.request().internalShardId = shard.shardId();
transportService.sendRequest(node, transportShardAction, internalRequest.request(), new BaseTransportResponseHandler<Response>() {
@Override
public Response newInstance() {
return newResponse();
}
@Override
public String executor() {
return ThreadPool.Names.SAME;
}
@Override
public void handleResponse(final Response response) {
listener.onResponse(response);
}
@Override
public void handleException(TransportException exp) {
onFailure(shard, exp);
}
});
}
}
}
}
private class ShardTransportHandler implements TransportRequestHandler<Request> {
@Override
public void messageReceived(final Request request, final TransportChannel channel) throws Exception {
Response response = shardOperation(request, request.internalShardId);
channel.sendResponse(response);
}
}
/**
* Internal request class that gets built on each node. Holds the original request plus additional info.
*/
protected class InternalRequest {
final Request request;
final String concreteIndex;
InternalRequest(Request request, String concreteIndex) {
this.request = request;
this.concreteIndex = concreteIndex;
}
public Request request() {
return request;
}
public String concreteIndex() {
return concreteIndex;
}
}
}

View File

@ -24,6 +24,7 @@ import org.elasticsearch.action.ActionRequestValidationException;
import org.elasticsearch.action.IndicesRequest;
import org.elasticsearch.action.ValidateActions;
import org.elasticsearch.action.support.IndicesOptions;
import org.elasticsearch.common.Nullable;
import org.elasticsearch.common.io.stream.StreamInput;
import org.elasticsearch.common.io.stream.StreamOutput;
import org.elasticsearch.index.shard.ShardId;
@ -35,10 +36,17 @@ import java.io.IOException;
*/
public abstract class SingleShardRequest<T extends SingleShardRequest> extends ActionRequest<T> implements IndicesRequest {
ShardId internalShardId;
protected String index;
public static final IndicesOptions INDICES_OPTIONS = IndicesOptions.strictSingleIndexNoExpandForbidClosed();
/**
* The concrete index name
*
* Whether index property is optional depends on the concrete implementation. If index property is required the
* concrete implementation should use {@link #validateNonNullIndex()} to check if the index property has been set
*/
@Nullable
protected String index;
ShardId internalShardId;
private boolean threadedOperation = true;
protected SingleShardRequest() {
@ -57,8 +65,10 @@ public abstract class SingleShardRequest<T extends SingleShardRequest> extends A
this.index = index;
}
@Override
public ActionRequestValidationException validate() {
/**
* @return a validation exception if the index property hasn't been set
*/
protected ActionRequestValidationException validateNonNullIndex() {
ActionRequestValidationException validationException = null;
if (index == null) {
validationException = ValidateActions.addValidationError("index is missing", validationException);
@ -66,6 +76,13 @@ public abstract class SingleShardRequest<T extends SingleShardRequest> extends A
return validationException;
}
/**
* @return The concrete index this request is targeted for or <code>null</code> if index is optional.
* Whether index property is optional depends on the concrete implementation. If index property
* is required the concrete implementation should use {@link #validateNonNullIndex()} to check
* if the index property has been set
*/
@Nullable
public String index() {
return index;
}
@ -111,7 +128,7 @@ public abstract class SingleShardRequest<T extends SingleShardRequest> extends A
if (in.readBoolean()) {
internalShardId = ShardId.readShardId(in);
}
index = in.readString();
index = in.readOptionalString();
// no need to pass threading over the network, they are always false when coming throw a thread pool
}
@ -119,7 +136,7 @@ public abstract class SingleShardRequest<T extends SingleShardRequest> extends A
public void writeTo(StreamOutput out) throws IOException {
super.writeTo(out);
out.writeOptionalStreamable(internalShardId);
out.writeString(index);
out.writeOptionalString(index);
}
}

View File

@ -32,9 +32,10 @@ import org.elasticsearch.cluster.block.ClusterBlockLevel;
import org.elasticsearch.cluster.metadata.IndexNameExpressionResolver;
import org.elasticsearch.cluster.node.DiscoveryNode;
import org.elasticsearch.cluster.node.DiscoveryNodes;
import org.elasticsearch.cluster.routing.ShardIterator;
import org.elasticsearch.cluster.routing.ShardRouting;
import org.elasticsearch.cluster.routing.ShardsIterator;
import org.elasticsearch.common.Nullable;
import org.elasticsearch.common.logging.support.LoggerMessageFormat;
import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.index.shard.ShardId;
import org.elasticsearch.threadpool.ThreadPool;
@ -43,7 +44,9 @@ import org.elasticsearch.transport.*;
import static org.elasticsearch.action.support.TransportActions.isShardNotAvailableException;
/**
* A base class for single shard read operations.
* A base class for operations that need to perform a read operation on a single shard copy. If the operation fails,
* 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<Request extends SingleShardRequest, Response extends ActionResponse> extends TransportAction<Request, Response> {
@ -88,7 +91,7 @@ public abstract class TransportSingleShardAction<Request extends SingleShardRequ
protected abstract Response newResponse();
protected abstract boolean resolveIndex();
protected abstract boolean resolveIndex(Request request);
protected ClusterBlockException checkGlobalBlock(ClusterState state) {
return state.blocks().globalBlockedException(ClusterBlockLevel.READ);
@ -102,12 +105,17 @@ public abstract class TransportSingleShardAction<Request extends SingleShardRequ
}
protected abstract ShardIterator shards(ClusterState state, InternalRequest request);
/**
* Returns the candidate shards to execute the operation on or <code>null</code> the execute
* the operation locally (the node that received the request)
*/
@Nullable
protected abstract ShardsIterator shards(ClusterState state, InternalRequest request);
class AsyncSingleAction {
private final ActionListener<Response> listener;
private final ShardIterator shardIt;
private final ShardsIterator shardIt;
private final InternalRequest internalRequest;
private final DiscoveryNodes nodes;
private volatile Throwable lastFailure;
@ -126,7 +134,7 @@ public abstract class TransportSingleShardAction<Request extends SingleShardRequ
}
String concreteSingleIndex;
if (resolveIndex()) {
if (resolveIndex(request)) {
concreteSingleIndex = indexNameExpressionResolver.concreteSingleIndex(clusterState, request);
} else {
concreteSingleIndex = request.index();
@ -143,7 +151,32 @@ public abstract class TransportSingleShardAction<Request extends SingleShardRequ
}
public void start() {
perform(null);
if (shardIt == null) {
// just execute it on the local node
transportService.sendRequest(clusterService.localNode(), transportShardAction, internalRequest.request(), new BaseTransportResponseHandler<Response>() {
@Override
public Response newInstance() {
return newResponse();
}
@Override
public String executor() {
return ThreadPool.Names.SAME;
}
@Override
public void handleResponse(final Response response) {
listener.onResponse(response);
}
@Override
public void handleException(TransportException exp) {
perform(exp);
}
});
} else {
perform(null);
}
}
private void onFailure(ShardRouting shardRouting, Throwable e) {
@ -163,10 +196,10 @@ public abstract class TransportSingleShardAction<Request extends SingleShardRequ
if (shardRouting == null) {
Throwable failure = lastFailure;
if (failure == null || isShardNotAvailableException(failure)) {
failure = new NoShardAvailableActionException(shardIt.shardId(), null, failure);
failure = new NoShardAvailableActionException(null, LoggerMessageFormat.format("No shard available for [{}]", internalRequest.request()), failure);
} else {
if (logger.isDebugEnabled()) {
logger.debug("{}: failed to execute [{}]", failure, shardIt.shardId(), internalRequest.request());
logger.debug("{}: failed to execute [{}]", failure, null, internalRequest.request());
}
}
listener.onFailure(failure);
@ -174,7 +207,7 @@ public abstract class TransportSingleShardAction<Request extends SingleShardRequ
}
DiscoveryNode node = nodes.get(shardRouting.currentNodeId());
if (node == null) {
onFailure(shardRouting, new NoShardAvailableActionException(shardIt.shardId()));
onFailure(shardRouting, new NoShardAvailableActionException(shardRouting.shardId()));
} else {
internalRequest.request().internalShardId = shardRouting.shardId();
transportService.sendRequest(node, transportShardAction, internalRequest.request(), new BaseTransportResponseHandler<Response>() {

View File

@ -20,6 +20,7 @@
package org.elasticsearch.action.termvectors;
import com.carrotsearch.hppc.IntArrayList;
import org.elasticsearch.action.ActionRequestValidationException;
import org.elasticsearch.action.support.single.shard.SingleShardRequest;
import org.elasticsearch.common.io.stream.StreamInput;
import org.elasticsearch.common.io.stream.StreamOutput;
@ -47,6 +48,11 @@ public class MultiTermVectorsShardRequest extends SingleShardRequest<MultiTermVe
requests = new ArrayList<>();
}
@Override
public ActionRequestValidationException validate() {
return super.validateNonNullIndex();
}
public int shardId() {
return this.shardId;
}

View File

@ -167,6 +167,7 @@ public class TermVectorsRequest extends SingleShardRequest<TermVectorsRequest> i
this.version = other.version();
this.versionType = VersionType.fromValue(other.versionType().getValue());
this.startTime = other.startTime();
this.filterSettings = other.filterSettings();
}
public TermVectorsRequest(MultiGetRequest.Item item) {
@ -472,7 +473,7 @@ public class TermVectorsRequest extends SingleShardRequest<TermVectorsRequest> i
@Override
public ActionRequestValidationException validate() {
ActionRequestValidationException validationException = super.validate();
ActionRequestValidationException validationException = super.validateNonNullIndex();
if (type == null) {
validationException = ValidateActions.addValidationError("type is missing", validationException);
}

View File

@ -63,7 +63,7 @@ public class TransportShardMultiTermsVectorAction extends TransportSingleShardAc
}
@Override
protected boolean resolveIndex() {
protected boolean resolveIndex(MultiTermVectorsShardRequest request) {
return false;
}

View File

@ -65,7 +65,7 @@ public class TransportTermVectorsAction extends TransportSingleShardAction<TermV
}
@Override
protected boolean resolveIndex() {
protected boolean resolveIndex(TermVectorsRequest request) {
return true;
}

View File

@ -136,21 +136,9 @@ public class JarHell {
/** inspect manifest for sure incompatibilities */
static void checkManifest(Manifest manifest, Path jar) {
// give a nice error if jar requires a newer java version
String systemVersion = System.getProperty("java.specification.version");
String targetVersion = manifest.getMainAttributes().getValue("X-Compile-Target-JDK");
if (targetVersion != null) {
float current = Float.POSITIVE_INFINITY;
float target = Float.NEGATIVE_INFINITY;
try {
current = Float.parseFloat(systemVersion);
target = Float.parseFloat(targetVersion);
} catch (NumberFormatException e) {
// some spec changed, time for a more complex parser
}
if (current < target) {
throw new IllegalStateException(jar + " requires Java " + targetVersion
+ ", your system: " + systemVersion);
}
checkJavaVersion(jar.toString(), targetVersion);
}
// give a nice error if jar is compiled against different es version
@ -162,6 +150,26 @@ public class JarHell {
}
}
/**
* Checks that the java specification version {@code targetVersion}
* required by {@code resource} is compatible with the current installation.
*/
public static void checkJavaVersion(String resource, String targetVersion) {
String systemVersion = System.getProperty("java.specification.version");
float current = Float.POSITIVE_INFINITY;
float target = Float.NEGATIVE_INFINITY;
try {
current = Float.parseFloat(systemVersion);
target = Float.parseFloat(targetVersion);
} catch (NumberFormatException e) {
// some spec changed, time for a more complex parser
}
if (current < target) {
throw new IllegalStateException(resource + " requires Java " + targetVersion
+ ", your system: " + systemVersion);
}
}
static void checkClass(Map<String,Path> clazzes, String clazz, Path jarpath) {
Path previous = clazzes.put(clazz, jarpath);
if (previous != null) {

View File

@ -420,7 +420,10 @@ public class MetaDataCreateIndexService extends AbstractComponent {
.put(indexMetaData, false)
.build();
logger.info("[{}] creating index, cause [{}], templates {}, shards [{}]/[{}], mappings {}", request.index(), request.cause(), templateNames, indexMetaData.numberOfShards(), indexMetaData.numberOfReplicas(), mappings.keySet());
String maybeShadowIndicator = IndexMetaData.isIndexUsingShadowReplicas(indexMetaData.settings()) ? "s" : "";
logger.info("[{}] creating index, cause [{}], templates {}, shards [{}]/[{}{}], mappings {}",
request.index(), request.cause(), templateNames, indexMetaData.numberOfShards(),
indexMetaData.numberOfReplicas(), maybeShadowIndicator, mappings.keySet());
ClusterBlocks.Builder blocks = ClusterBlocks.builder().blocks(currentState.blocks());
if (!request.blocks().isEmpty()) {

View File

@ -83,10 +83,10 @@ public class AllocationId implements ToXContent {
/**
* Creates a new allocation id representing a cancelled relocation.
*
* <p/>
* Note that this is expected to be called on the allocation id
* of the *source* shard
* */
*/
public static AllocationId cancelRelocation(AllocationId allocationId) {
assert allocationId.getRelocationId() != null;
return new AllocationId(allocationId.getId(), null);
@ -94,7 +94,7 @@ public class AllocationId implements ToXContent {
/**
* Creates a new allocation id finalizing a relocation.
*
* <p/>
* Note that this is expected to be called on the allocation id
* of the *target* shard and thus it only needs to clear the relocating id.
*/
@ -120,9 +120,16 @@ public class AllocationId implements ToXContent {
@Override
public boolean equals(Object o) {
if (this == o) return true;
if (this == o) {
return true;
}
if (o == null) {
return false;
}
AllocationId that = (AllocationId) o;
if (!id.equals(that.id)) return false;
if (!id.equals(that.id)) {
return false;
}
return !(relocationId != null ? !relocationId.equals(that.relocationId) : that.relocationId != null);
}

View File

@ -88,7 +88,7 @@ public class RoutingNode implements Iterable<ShardRouting> {
void add(ShardRouting shard) {
// TODO use Set with ShardIds for faster lookup.
for (ShardRouting shardRouting : shards) {
if (shardRouting.shardId().equals(shard.shardId())) {
if (shardRouting.isSameShard(shard)) {
throw new IllegalStateException("Trying to add a shard [" + shard.shardId().index().name() + "][" + shard.shardId().id() + "] to a node [" + nodeId + "] where it already exists");
}
}

View File

@ -420,7 +420,7 @@ public final class ShardRouting implements Streamable, ToXContent {
void relocate(String relocatingNodeId) {
ensureNotFrozen();
version++;
assert state == ShardRoutingState.STARTED : this;
assert state == ShardRoutingState.STARTED : "current shard has to be started in order to be relocated " + this;
state = ShardRoutingState.RELOCATING;
this.relocatingNodeId = relocatingNodeId;
this.allocationId = AllocationId.newRelocation(allocationId);
@ -467,7 +467,7 @@ public final class ShardRouting implements Streamable, ToXContent {
restoreSource = null;
unassignedInfo = null; // we keep the unassigned data until the shard is started
if (allocationId.getRelocationId() != null) {
// target relocation
// relocation target
allocationId = AllocationId.finishRelocation(allocationId);
}
state = ShardRoutingState.STARTED;
@ -498,6 +498,106 @@ public final class ShardRouting implements Streamable, ToXContent {
primary = false;
}
/** returns true if this routing has the same shardId as another */
public boolean isSameShard(ShardRouting other) {
return index.equals(other.index) && shardId == other.shardId;
}
/**
* returns true if this routing has the same allocation ID as another.
* <p/>
* Note: if both shard routing has a null as their {@link #allocationId()}, this method returns false as the routing describe
* no allocation at all..
**/
public boolean isSameAllocation(ShardRouting other) {
boolean b = this.allocationId != null && other.allocationId != null && this.allocationId.getId().equals(other.allocationId.getId());
assert b == false || this.currentNodeId.equals(other.currentNodeId) : "ShardRoutings have the same allocation id but not the same node. This [" + this + "], other [" + other + "]";
return b;
}
/** returns true if the routing is the relocation target of the given routing */
public boolean isRelocationTargetOf(ShardRouting other) {
boolean b = this.allocationId != null && other.allocationId != null && this.state == ShardRoutingState.INITIALIZING &&
this.allocationId.getId().equals(other.allocationId.getRelocationId());
assert b == false || other.state == ShardRoutingState.RELOCATING :
"ShardRouting is a relocation target but the source shard state isn't relocating. This [" + this + "], other [" + other + "]";
assert b == false || other.allocationId.getId().equals(this.allocationId.getRelocationId()) :
"ShardRouting is a relocation target but the source id isn't equal to source's allocationId.getRelocationId. This [" + this + "], other [" + other + "]";
assert b == false || other.currentNodeId().equals(this.relocatingNodeId) :
"ShardRouting is a relocation target but source current node id isn't equal to target relocating node. This [" + this + "], other [" + other + "]";
assert b == false || this.currentNodeId().equals(other.relocatingNodeId) :
"ShardRouting is a relocation target but current node id isn't equal to source relocating node. This [" + this + "], other [" + other + "]";
assert b == false || isSameShard(other) :
"ShardRouting is a relocation target but both routings are not of the same shard. This [" + this + "], other [" + other + "]";
assert b == false || this.primary == other.primary :
"ShardRouting is a relocation target but primary flag is different. This [" + this + "], target [" + other + "]";
return b;
}
/** returns true if the routing is the relocation source for the given routing */
public boolean isRelocationSourceOf(ShardRouting other) {
boolean b = this.allocationId != null && other.allocationId != null && other.state == ShardRoutingState.INITIALIZING &&
other.allocationId.getId().equals(this.allocationId.getRelocationId());
assert b == false || this.state == ShardRoutingState.RELOCATING :
"ShardRouting is a relocation source but shard state isn't relocating. This [" + this + "], other [" + other + "]";
assert b == false || this.allocationId.getId().equals(other.allocationId.getRelocationId()) :
"ShardRouting is a relocation source but the allocation id isn't equal to other.allocationId.getRelocationId. This [" + this + "], other [" + other + "]";
assert b == false || this.currentNodeId().equals(other.relocatingNodeId) :
"ShardRouting is a relocation source but current node isn't equal to other's relocating node. This [" + this + "], other [" + other + "]";
assert b == false || other.currentNodeId().equals(this.relocatingNodeId) :
"ShardRouting is a relocation source but relocating node isn't equal to other's current node. This [" + this + "], other [" + other + "]";
assert b == false || isSameShard(other) :
"ShardRouting is a relocation source but both routings are not of the same shard. This [" + this + "], target [" + other + "]";
assert b == false || this.primary == other.primary :
"ShardRouting is a relocation source but primary flag is different. This [" + this + "], target [" + other + "]";
return b;
}
/** returns true if the current routing is identical to the other routing in all but meta fields, i.e., version and unassigned info */
public boolean equalsIgnoringMetaData(ShardRouting other) {
if (primary != other.primary) {
return false;
}
if (shardId != other.shardId) {
return false;
}
if (currentNodeId != null ? !currentNodeId.equals(other.currentNodeId) : other.currentNodeId != null) {
return false;
}
if (index != null ? !index.equals(other.index) : other.index != null) {
return false;
}
if (relocatingNodeId != null ? !relocatingNodeId.equals(other.relocatingNodeId) : other.relocatingNodeId != null) {
return false;
}
if (allocationId != null ? !allocationId.equals(other.allocationId) : other.allocationId != null) {
return false;
}
if (state != other.state) {
return false;
}
if (restoreSource != null ? !restoreSource.equals(other.restoreSource) : other.restoreSource != null) {
return false;
}
return true;
}
@Override
public boolean equals(Object o) {
if (this == o) {
@ -508,32 +608,8 @@ public final class ShardRouting implements Streamable, ToXContent {
return false;
}
ShardRouting that = (ShardRouting) o;
if (primary != that.primary) {
return false;
}
if (shardId != that.shardId) {
return false;
}
if (currentNodeId != null ? !currentNodeId.equals(that.currentNodeId) : that.currentNodeId != null) {
return false;
}
if (index != null ? !index.equals(that.index) : that.index != null) {
return false;
}
if (relocatingNodeId != null ? !relocatingNodeId.equals(that.relocatingNodeId) : that.relocatingNodeId != null) {
return false;
}
if (allocationId != null ? !allocationId.equals(that.allocationId) : that.allocationId != null) {
return false;
}
if (state != that.state) {
return false;
}
if (restoreSource != null ? !restoreSource.equals(that.restoreSource) : that.restoreSource != null) {
return false;
}
return true;
// TODO: add version + unassigned info check. see #12387
return equalsIgnoringMetaData(that);
}
private long hashVersion = version - 1;

View File

@ -95,7 +95,11 @@ public class UnassignedInfo implements ToXContent, Writeable<UnassignedInfo> {
/**
* When a shard moves from started back to initializing, for example, during shadow replica
*/
REINITIALIZED;
REINITIALIZED,
/**
* A better replica location is identified and causes the existing replica allocation to be cancelled.
*/
REALLOCATED_REPLICA;
}
private final Reason reason;

View File

@ -338,7 +338,7 @@ public class AllocationService extends AbstractComponent {
}
for (ShardRouting shard : currentRoutingNode) {
if (shard.allocationId().getId().equals(startedShard.allocationId().getId())) {
if (shard.isSameAllocation(startedShard)) {
if (shard.active()) {
logger.trace("{} shard is already started, ignoring (routing: {})", startedShard.shardId(), startedShard);
} else {
@ -363,8 +363,7 @@ public class AllocationService extends AbstractComponent {
if (sourceRoutingNode != null) {
while (sourceRoutingNode.hasNext()) {
ShardRouting shard = sourceRoutingNode.next();
if (shard.allocationId().getId().equals(startedShard.allocationId().getRelocationId())) {
assert shard.relocating() : "source shard for relocation is not marked as relocating. source " + shard + ", started target " + startedShard;
if (shard.isRelocationSourceOf(startedShard)) {
dirty = true;
sourceRoutingNode.remove();
break;
@ -397,7 +396,7 @@ public class AllocationService extends AbstractComponent {
boolean matchedShard = false;
while (matchedNode.hasNext()) {
ShardRouting routing = matchedNode.next();
if (routing.allocationId().getId().equals(failedShard.allocationId().getId())) {
if (routing.isSameAllocation(failedShard)) {
matchedShard = true;
logger.debug("{} failed shard {} found in routingNodes, failing it ({})", failedShard.shardId(), failedShard, unassignedInfo.shortSummary());
break;
@ -428,7 +427,7 @@ public class AllocationService extends AbstractComponent {
RoutingNode relocatingFromNode = routingNodes.node(failedShard.relocatingNodeId());
if (relocatingFromNode != null) {
for (ShardRouting shardRouting : relocatingFromNode) {
if (shardRouting.allocationId().getId().equals(failedShard.allocationId().getRelocationId())) {
if (shardRouting.isRelocationSourceOf(failedShard)) {
logger.trace("{}, resolved source to [{}]. canceling relocation ... ({})", failedShard.shardId(), shardRouting, unassignedInfo.shortSummary());
routingNodes.cancelRelocation(shardRouting);
break;
@ -441,7 +440,7 @@ public class AllocationService extends AbstractComponent {
// and the shard copy needs to be marked as unassigned
if (failedShard.relocatingNodeId() != null) {
// handle relocation source shards. we need to find the target initializing shard that is recovering from, and remove it...
// handle relocation source shards. we need to find the target initializing shard that is recovering, and remove it...
assert failedShard.initializing() == false; // should have been dealt with and returned
assert failedShard.relocating();
@ -449,10 +448,7 @@ public class AllocationService extends AbstractComponent {
if (initializingNode != null) {
while (initializingNode.hasNext()) {
ShardRouting shardRouting = initializingNode.next();
if (shardRouting.allocationId().getId().equals(failedShard.allocationId().getRelocationId())) {
assert shardRouting.initializing() : shardRouting;
assert failedShard.allocationId().getId().equals(shardRouting.allocationId().getRelocationId())
: "found target shard's allocation relocation id is different than source";
if (shardRouting.isRelocationTargetOf(failedShard)) {
logger.trace("{} is removed due to the failure of the source shard", shardRouting);
initializingNode.remove();
}

View File

@ -177,7 +177,7 @@ public class CancelAllocationCommand implements AllocationCommand {
RoutingNode relocatingFromNode = allocation.routingNodes().node(shardRouting.relocatingNodeId());
if (relocatingFromNode != null) {
for (ShardRouting fromShardRouting : relocatingFromNode) {
if (fromShardRouting.shardId().equals(shardRouting.shardId()) && fromShardRouting.state() == RELOCATING) {
if (fromShardRouting.isSameShard(shardRouting) && fromShardRouting.state() == RELOCATING) {
allocation.routingNodes().cancelRelocation(fromShardRouting);
break;
}
@ -201,7 +201,7 @@ public class CancelAllocationCommand implements AllocationCommand {
if (initializingNode != null) {
while (initializingNode.hasNext()) {
ShardRouting initializingShardRouting = initializingNode.next();
if (initializingShardRouting.shardId().equals(shardRouting.shardId()) && initializingShardRouting.initializing()) {
if (initializingShardRouting.isRelocationTargetOf(shardRouting)) {
initializingNode.remove();
}
}

View File

@ -38,7 +38,7 @@ import org.elasticsearch.node.settings.NodeSettingsService;
* node. The default is <tt>4</tt></li>
* <p/>
* <li><tt>cluster.routing.allocation.node_concurrent_recoveries</tt> -
* restricts the number of concurrent recovery operations on a single node. The
* restricts the number of total concurrent shards initializing on a single node. The
* default is <tt>2</tt></li>
* </ul>
* <p/>
@ -106,7 +106,7 @@ public class ThrottlingAllocationDecider extends AllocationDecider {
public Decision canAllocate(RoutingNode node, RoutingAllocation allocation) {
int currentRecoveries = 0;
for (ShardRouting shard : node) {
if (shard.initializing() || shard.relocating()) {
if (shard.initializing()) {
currentRecoveries++;
}
}

View File

@ -22,10 +22,7 @@ package org.elasticsearch.common.util.concurrent;
import java.util.concurrent.ConcurrentMap;
import java.util.concurrent.atomic.AtomicInteger;
import java.util.concurrent.locks.Lock;
import java.util.concurrent.locks.ReadWriteLock;
import java.util.concurrent.locks.ReentrantLock;
import java.util.concurrent.locks.ReentrantReadWriteLock;
/**
* This class manages locks. Locks can be accessed with an identifier and are
@ -115,58 +112,4 @@ public class KeyedLock<T> {
return !map.isEmpty();
}
/**
* A {@link KeyedLock} that allows to acquire a global lock that guarantees
* exclusive access to the resource the KeyedLock is guarding.
*/
public final static class GlobalLockable<T> extends KeyedLock<T> {
private final ReadWriteLock lock;
public GlobalLockable(boolean fair){
super(fair);
lock = new ReentrantReadWriteLock(fair);
}
public GlobalLockable() {
this(false);
}
@Override
public void acquire(T key) {
boolean success = false;
lock.readLock().lock();
try {
super.acquire(key);
success = true;
} finally {
if (!success) {
lock.readLock().unlock();
}
}
}
@Override
public void release(T key) {
KeyLock keyLock = threadLocal.get();
if (keyLock == null) {
throw new IllegalStateException("Lock not acquired");
}
try {
release(key, keyLock);
} finally {
lock.readLock().unlock();
}
}
/**
* Returns a global lock guaranteeing exclusive access to the resource
* this KeyedLock is guarding.
*/
public Lock globalLock() {
return lock.writeLock();
}
}
}

View File

@ -126,6 +126,7 @@ public class GatewayAllocator extends AbstractComponent {
}); // sort for priority ordering
changed |= primaryShardAllocator.allocateUnassigned(allocation);
changed |= replicaShardAllocator.processExistingRecoveries(allocation);
changed |= replicaShardAllocator.allocateUnassigned(allocation);
return changed;
}

View File

@ -25,11 +25,11 @@ import com.carrotsearch.hppc.cursors.ObjectCursor;
import com.carrotsearch.hppc.cursors.ObjectLongCursor;
import org.elasticsearch.cluster.ClusterChangedEvent;
import org.elasticsearch.cluster.metadata.IndexMetaData;
import org.elasticsearch.cluster.metadata.MetaData;
import org.elasticsearch.cluster.node.DiscoveryNode;
import org.elasticsearch.cluster.routing.RoutingNode;
import org.elasticsearch.cluster.routing.RoutingNodes;
import org.elasticsearch.cluster.routing.ShardRouting;
import org.elasticsearch.cluster.routing.UnassignedInfo;
import org.elasticsearch.cluster.routing.allocation.RoutingAllocation;
import org.elasticsearch.cluster.routing.allocation.decider.Decision;
import org.elasticsearch.common.Nullable;
@ -40,7 +40,6 @@ import org.elasticsearch.common.unit.TimeValue;
import org.elasticsearch.index.store.StoreFileMetaData;
import org.elasticsearch.indices.store.TransportNodesListShardStoreMetaData;
import java.util.Iterator;
import java.util.Map;
/**
@ -51,6 +50,62 @@ public abstract class ReplicaShardAllocator extends AbstractComponent {
super(settings);
}
/**
* Process existing recoveries of replicas and see if we need to cancel them if we find a better
* match. Today, a better match is one that has full sync id match compared to not having one in
* the previous recovery.
*/
public boolean processExistingRecoveries(RoutingAllocation allocation) {
boolean changed = false;
for (RoutingNodes.RoutingNodesIterator nodes = allocation.routingNodes().nodes(); nodes.hasNext(); ) {
nodes.next();
for (RoutingNodes.RoutingNodeIterator it = nodes.nodeShards(); it.hasNext(); ) {
ShardRouting shard = it.next();
if (shard.primary() == true) {
continue;
}
if (shard.initializing() == false) {
continue;
}
if (shard.relocatingNodeId() != null) {
continue;
}
AsyncShardFetch.FetchResult<TransportNodesListShardStoreMetaData.NodeStoreFilesMetaData> shardStores = fetchData(shard, allocation);
if (shardStores.hasData() == false) {
logger.trace("{}: fetching new stores for initializing shard", shard);
continue; // still fetching
}
ShardRouting primaryShard = allocation.routingNodes().activePrimary(shard);
assert primaryShard != null : "the replica shard can be allocated on at least one node, so there must be an active primary";
TransportNodesListShardStoreMetaData.StoreFilesMetaData primaryStore = findStore(primaryShard, allocation, shardStores);
if (primaryStore == null || primaryStore.allocated() == false) {
// if we can't find the primary data, it is probably because the primary shard is corrupted (and listing failed)
// just let the recovery find it out, no need to do anything about it for the initializing shard
logger.trace("{}: no primary shard store found or allocated, letting actual allocation figure it out", shard);
continue;
}
MatchingNodes matchingNodes = findMatchingNodes(shard, allocation, primaryStore, shardStores);
if (matchingNodes.getNodeWithHighestMatch() != null) {
DiscoveryNode currentNode = allocation.nodes().get(shard.currentNodeId());
DiscoveryNode nodeWithHighestMatch = matchingNodes.getNodeWithHighestMatch();
if (currentNode.equals(nodeWithHighestMatch) == false
&& matchingNodes.isNodeMatchBySyncID(currentNode) == false
&& matchingNodes.isNodeMatchBySyncID(nodeWithHighestMatch) == true) {
// we found a better match that has a full sync id match, the existing allocation is not fully synced
// so we found a better one, cancel this one
it.moveToUnassigned(new UnassignedInfo(UnassignedInfo.Reason.REALLOCATED_REPLICA,
"existing allocation of replica to [" + currentNode + "] cancelled, sync id match found on node [" + nodeWithHighestMatch + "]"));
changed = true;
}
}
}
}
return changed;
}
public boolean allocateUnassigned(RoutingAllocation allocation) {
boolean changed = false;
final RoutingNodes routingNodes = allocation.routingNodes();
@ -236,7 +291,7 @@ public abstract class ReplicaShardAllocator extends AbstractComponent {
highestMatchNode = cursor.key;
}
}
nodeWithHighestMatch = highestMatchNode;
this.nodeWithHighestMatch = highestMatchNode;
}
/**
@ -248,6 +303,10 @@ public abstract class ReplicaShardAllocator extends AbstractComponent {
return this.nodeWithHighestMatch;
}
public boolean isNodeMatchBySyncID(DiscoveryNode node) {
return nodesToSize.get(node) == Long.MAX_VALUE;
}
/**
* Did we manage to find any data, regardless how well they matched or not.
*/

View File

@ -20,7 +20,6 @@
package org.elasticsearch.index.engine;
import com.google.common.base.Preconditions;
import org.apache.lucene.index.*;
import org.apache.lucene.search.IndexSearcher;
import org.apache.lucene.search.Query;
@ -45,7 +44,6 @@ import org.elasticsearch.common.util.concurrent.ReleasableLock;
import org.elasticsearch.index.VersionType;
import org.elasticsearch.index.deletionpolicy.SnapshotDeletionPolicy;
import org.elasticsearch.index.deletionpolicy.SnapshotIndexCommit;
import org.elasticsearch.index.mapper.DocumentMapper;
import org.elasticsearch.index.mapper.ParseContext.Document;
import org.elasticsearch.index.mapper.ParsedDocument;
import org.elasticsearch.index.mapper.Uid;
@ -57,11 +55,7 @@ import org.elasticsearch.index.translog.Translog;
import java.io.Closeable;
import java.io.IOException;
import java.util.Arrays;
import java.util.Comparator;
import java.util.HashMap;
import java.util.List;
import java.util.Map;
import java.util.*;
import java.util.concurrent.TimeUnit;
import java.util.concurrent.atomic.AtomicBoolean;
import java.util.concurrent.locks.Condition;
@ -288,7 +282,7 @@ public abstract class Engine implements Closeable {
try {
final Searcher retVal = newSearcher(source, searcher, manager);
success = true;
return retVal;
return config().getWrappingService().wrap(engineConfig, retVal);
} finally {
if (!success) {
manager.release(searcher);

View File

@ -22,7 +22,6 @@ import org.apache.lucene.analysis.Analyzer;
import org.apache.lucene.codecs.Codec;
import org.apache.lucene.index.IndexWriterConfig;
import org.apache.lucene.index.MergePolicy;
import org.elasticsearch.index.shard.MergeSchedulerConfig;
import org.apache.lucene.search.QueryCache;
import org.apache.lucene.search.QueryCachingPolicy;
import org.apache.lucene.search.similarities.Similarity;
@ -35,6 +34,7 @@ import org.elasticsearch.common.util.concurrent.EsExecutors;
import org.elasticsearch.index.codec.CodecService;
import org.elasticsearch.index.deletionpolicy.SnapshotDeletionPolicy;
import org.elasticsearch.index.indexing.ShardIndexingService;
import org.elasticsearch.index.shard.MergeSchedulerConfig;
import org.elasticsearch.index.shard.ShardId;
import org.elasticsearch.index.shard.TranslogRecoveryPerformer;
import org.elasticsearch.index.store.Store;
@ -77,6 +77,7 @@ public final class EngineConfig {
private final boolean forceNewTranslog;
private final QueryCache queryCache;
private final QueryCachingPolicy queryCachingPolicy;
private final IndexSearcherWrappingService wrappingService;
/**
* Index setting for index concurrency / number of threadstates in the indexwriter.
@ -143,7 +144,7 @@ public final class EngineConfig {
Settings indexSettings, IndicesWarmer warmer, Store store, SnapshotDeletionPolicy deletionPolicy,
MergePolicy mergePolicy, MergeSchedulerConfig mergeSchedulerConfig, Analyzer analyzer,
Similarity similarity, CodecService codecService, Engine.FailedEngineListener failedEngineListener,
TranslogRecoveryPerformer translogRecoveryPerformer, QueryCache queryCache, QueryCachingPolicy queryCachingPolicy, TranslogConfig translogConfig) {
TranslogRecoveryPerformer translogRecoveryPerformer, QueryCache queryCache, QueryCachingPolicy queryCachingPolicy, IndexSearcherWrappingService wrappingService, TranslogConfig translogConfig) {
this.shardId = shardId;
this.indexSettings = indexSettings;
this.threadPool = threadPool;
@ -157,6 +158,7 @@ public final class EngineConfig {
this.similarity = similarity;
this.codecService = codecService;
this.failedEngineListener = failedEngineListener;
this.wrappingService = wrappingService;
this.optimizeAutoGenerateId = indexSettings.getAsBoolean(EngineConfig.INDEX_OPTIMIZE_AUTOGENERATED_ID_SETTING, false);
this.compoundOnFlush = indexSettings.getAsBoolean(EngineConfig.INDEX_COMPOUND_ON_FLUSH, compoundOnFlush);
this.indexConcurrency = indexSettings.getAsInt(EngineConfig.INDEX_CONCURRENCY_SETTING, Math.max(IndexWriterConfig.DEFAULT_MAX_THREAD_STATES, (int) (EsExecutors.boundedNumberOfProcessors(indexSettings) * 0.65)));
@ -421,6 +423,10 @@ public final class EngineConfig {
return queryCachingPolicy;
}
public IndexSearcherWrappingService getWrappingService() {
return wrappingService;
}
/**
* Returns the translog config for this engine
*/

View File

@ -0,0 +1,45 @@
/*
* Licensed to Elasticsearch under one or more contributor
* license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright
* ownership. Elasticsearch licenses this file to you under
* the Apache License, Version 2.0 (the "License"); you may
* not use this file except in compliance with the License.
* You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing,
* software distributed under the License is distributed on an
* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
* KIND, either express or implied. See the License for the
* specific language governing permissions and limitations
* under the License.
*/
package org.elasticsearch.index.engine;
import org.apache.lucene.index.DirectoryReader;
import org.apache.lucene.search.IndexSearcher;
/**
* Extension point to add custom functionality at request time to the {@link DirectoryReader}
* and {@link IndexSearcher} managed by the {@link Engine}.
*/
public interface IndexSearcherWrapper {
/**
* @param reader The provided directory reader to be wrapped to add custom functionality
* @return a new directory reader wrapping the provided directory reader or if no wrapping was performed
* the provided directory reader
*/
DirectoryReader wrap(DirectoryReader reader);
/**
* @param searcher The provided index searcher to be wrapped to add custom functionality
* @return a new index searcher wrapping the provided index searcher or if no wrapping was performed
* the provided index searcher
*/
IndexSearcher wrap(IndexSearcher searcher) throws EngineException;
}

View File

@ -0,0 +1,94 @@
/*
* Licensed to Elasticsearch under one or more contributor
* license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright
* ownership. Elasticsearch licenses this file to you under
* the Apache License, Version 2.0 (the "License"); you may
* not use this file except in compliance with the License.
* You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing,
* software distributed under the License is distributed on an
* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
* KIND, either express or implied. See the License for the
* specific language governing permissions and limitations
* under the License.
*/
package org.elasticsearch.index.engine;
import org.apache.lucene.index.DirectoryReader;
import org.apache.lucene.search.IndexSearcher;
import org.elasticsearch.ElasticsearchException;
import org.elasticsearch.common.inject.Inject;
import org.elasticsearch.index.engine.Engine.Searcher;
import java.util.Set;
/**
* Service responsible for wrapping the {@link DirectoryReader} and {@link IndexSearcher} of a {@link Searcher} via the
* configured {@link IndexSearcherWrapper} instance. This allows custom functionally to be added the {@link Searcher}
* before being used to do an operation (search, get, field stats etc.)
*/
// TODO: This needs extension point is a bit hacky now, because the IndexSearch from the engine can only be wrapped once,
// if we allowed the IndexSearcher to be wrapped multiple times then a custom IndexSearcherWrapper needs have good
// control over its location in the wrapping chain
public final class IndexSearcherWrappingService {
private final IndexSearcherWrapper wrapper;
// for unit tests:
IndexSearcherWrappingService() {
this.wrapper = null;
}
@Inject
// Use a Set parameter here, because constructor parameter can't be optional
// and I prefer to keep the `wrapper` field final.
public IndexSearcherWrappingService(Set<IndexSearcherWrapper> wrappers) {
if (wrappers.size() > 1) {
throw new IllegalStateException("wrapping of the index searcher by more than one wrappers is forbidden, found the following wrappers [" + wrappers + "]");
}
if (wrappers.isEmpty()) {
this.wrapper = null;
} else {
this.wrapper = wrappers.iterator().next();
}
}
/**
* If there are configured {@link IndexSearcherWrapper} instances, the {@link IndexSearcher} of the provided engine searcher
* gets wrapped and a new {@link Searcher} instances is returned, otherwise the provided {@link Searcher} is returned.
*
* This is invoked each time a {@link Searcher} is requested to do an operation. (for example search)
*/
public Searcher wrap(EngineConfig engineConfig, final Searcher engineSearcher) throws EngineException {
if (wrapper == null) {
return engineSearcher;
}
DirectoryReader reader = wrapper.wrap((DirectoryReader) engineSearcher.reader());
IndexSearcher innerIndexSearcher = new IndexSearcher(reader);
innerIndexSearcher.setQueryCache(engineConfig.getQueryCache());
innerIndexSearcher.setQueryCachingPolicy(engineConfig.getQueryCachingPolicy());
innerIndexSearcher.setSimilarity(engineConfig.getSimilarity());
// TODO: Right now IndexSearcher isn't wrapper friendly, when it becomes wrapper friendly we should revise this extension point
// For example if IndexSearcher#rewrite() is overwritten than also IndexSearcher#createNormalizedWeight needs to be overwritten
// This needs to be fixed before we can allow the IndexSearcher from Engine to be wrapped multiple times
IndexSearcher indexSearcher = wrapper.wrap(innerIndexSearcher);
if (reader == engineSearcher.reader() && indexSearcher == innerIndexSearcher) {
return engineSearcher;
} else {
return new Engine.Searcher(engineSearcher.source(), indexSearcher) {
@Override
public void close() throws ElasticsearchException {
engineSearcher.close();
}
};
}
}
}

View File

@ -21,12 +21,13 @@ package org.elasticsearch.index.fielddata;
import com.google.common.collect.ImmutableMap;
import com.google.common.collect.Maps;
import org.elasticsearch.ExceptionsHelper;
import org.elasticsearch.Version;
import org.elasticsearch.common.collect.MapBuilder;
import org.elasticsearch.common.collect.Tuple;
import org.elasticsearch.common.inject.Inject;
import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.common.util.concurrent.KeyedLock;
import org.elasticsearch.index.AbstractIndexComponent;
import org.elasticsearch.index.Index;
import org.elasticsearch.index.fielddata.plain.*;
@ -136,11 +137,16 @@ public class IndexFieldDataService extends AbstractIndexComponent {
}
private final IndicesFieldDataCache indicesFieldDataCache;
private final KeyedLock.GlobalLockable<String> fieldLoadingLock = new KeyedLock.GlobalLockable<>();
private final Map<String, IndexFieldDataCache> fieldDataCaches = Maps.newHashMap(); // no need for concurrency support, always used under lock
// the below map needs to be modified under a lock
private final Map<String, IndexFieldDataCache> fieldDataCaches = Maps.newHashMap();
IndexService indexService;
// We need to cache fielddata on the _parent field because of 1.x indices.
// When we don't support 1.x anymore (3.0) then remove this caching
// This variable needs to be read/written under lock
private IndexFieldData<?> parentIndexFieldData;
@Inject
public IndexFieldDataService(Index index, @IndexSettings Settings indexSettings, IndicesFieldDataCache indicesFieldDataCache,
CircuitBreakerService circuitBreakerService) {
@ -154,41 +160,35 @@ public class IndexFieldDataService extends AbstractIndexComponent {
this.indexService = indexService;
}
public void clear() {
fieldLoadingLock.globalLock().lock();
try {
List<Throwable> exceptions = new ArrayList<>(0);
final Collection<IndexFieldDataCache> fieldDataCacheValues = fieldDataCaches.values();
for (IndexFieldDataCache cache : fieldDataCacheValues) {
try {
cache.clear();
} catch (Throwable t) {
exceptions.add(t);
}
public synchronized void clear() {
parentIndexFieldData = null;
List<Throwable> exceptions = new ArrayList<>(0);
final Collection<IndexFieldDataCache> fieldDataCacheValues = fieldDataCaches.values();
for (IndexFieldDataCache cache : fieldDataCacheValues) {
try {
cache.clear();
} catch (Throwable t) {
exceptions.add(t);
}
fieldDataCacheValues.clear();
ExceptionsHelper.maybeThrowRuntimeAndSuppress(exceptions);
} finally {
fieldLoadingLock.globalLock().unlock();
}
fieldDataCacheValues.clear();
ExceptionsHelper.maybeThrowRuntimeAndSuppress(exceptions);
}
public void clearField(final String fieldName) {
fieldLoadingLock.acquire(fieldName);
try {
List<Throwable> exceptions = new ArrayList<>(0);
final IndexFieldDataCache cache = fieldDataCaches.remove(fieldName);
if (cache != null) {
try {
cache.clear();
} catch (Throwable t) {
exceptions.add(t);
}
}
ExceptionsHelper.maybeThrowRuntimeAndSuppress(exceptions);
} finally {
fieldLoadingLock.release(fieldName);
public synchronized void clearField(final String fieldName) {
if (ParentFieldMapper.NAME.equals(fieldName)) {
parentIndexFieldData = null;
}
List<Throwable> exceptions = new ArrayList<>(0);
final IndexFieldDataCache cache = fieldDataCaches.remove(fieldName);
if (cache != null) {
try {
cache.clear();
} catch (Throwable t) {
exceptions.add(t);
}
}
ExceptionsHelper.maybeThrowRuntimeAndSuppress(exceptions);
}
@SuppressWarnings("unchecked")
@ -199,32 +199,31 @@ public class IndexFieldDataService extends AbstractIndexComponent {
throw new IllegalArgumentException("found no fielddata type for field [" + fieldNames.fullName() + "]");
}
final boolean docValues = fieldType.hasDocValues();
final String key = fieldNames.indexName();
fieldLoadingLock.acquire(key);
try {
IndexFieldData.Builder builder = null;
String format = type.getFormat(indexSettings);
if (format != null && FieldDataType.DOC_VALUES_FORMAT_VALUE.equals(format) && !docValues) {
logger.warn("field [" + fieldNames.fullName() + "] has no doc values, will use default field data format");
format = null;
}
if (format != null) {
builder = buildersByTypeAndFormat.get(Tuple.tuple(type.getType(), format));
if (builder == null) {
logger.warn("failed to find format [" + format + "] for field [" + fieldNames.fullName() + "], will use default");
}
}
if (builder == null && docValues) {
builder = docValuesBuildersByType.get(type.getType());
}
IndexFieldData.Builder builder = null;
String format = type.getFormat(indexSettings);
if (format != null && FieldDataType.DOC_VALUES_FORMAT_VALUE.equals(format) && !docValues) {
logger.warn("field [" + fieldNames.fullName() + "] has no doc values, will use default field data format");
format = null;
}
if (format != null) {
builder = buildersByTypeAndFormat.get(Tuple.tuple(type.getType(), format));
if (builder == null) {
builder = buildersByType.get(type.getType());
}
if (builder == null) {
throw new IllegalArgumentException("failed to find field data builder for field " + fieldNames.fullName() + ", and type " + type.getType());
logger.warn("failed to find format [" + format + "] for field [" + fieldNames.fullName() + "], will use default");
}
}
if (builder == null && docValues) {
builder = docValuesBuildersByType.get(type.getType());
}
if (builder == null) {
builder = buildersByType.get(type.getType());
}
if (builder == null) {
throw new IllegalArgumentException("failed to find field data builder for field " + fieldNames.fullName() + ", and type " + type.getType());
}
IndexFieldDataCache cache = fieldDataCaches.get(fieldNames.indexName());
IndexFieldDataCache cache;
synchronized (this) {
cache = fieldDataCaches.get(fieldNames.indexName());
if (cache == null) {
// we default to node level cache, which in turn defaults to be unbounded
// this means changing the node level settings is simple, just set the bounds there
@ -239,10 +238,18 @@ public class IndexFieldDataService extends AbstractIndexComponent {
fieldDataCaches.put(fieldNames.indexName(), cache);
}
return (IFD) builder.build(index, indexSettings, fieldType, cache, circuitBreakerService, indexService.mapperService());
} finally {
fieldLoadingLock.release(key);
// Remove this in 3.0
final boolean isOldParentField = ParentFieldMapper.NAME.equals(fieldNames.indexName())
&& Version.indexCreated(indexSettings).before(Version.V_2_0_0_beta1);
if (isOldParentField) {
if (parentIndexFieldData == null) {
parentIndexFieldData = builder.build(index, indexSettings, fieldType, cache, circuitBreakerService, indexService.mapperService());
}
return (IFD) parentIndexFieldData;
}
}
return (IFD) builder.build(index, indexSettings, fieldType, cache, circuitBreakerService, indexService.mapperService());
}
}

View File

@ -64,7 +64,7 @@ public abstract class AbstractAtomicOrdinalsFieldData implements AtomicOrdinalsF
@Override
public RandomAccessOrds getOrdinalsValues() {
return (RandomAccessOrds) DocValues.emptySortedSet();
return DocValues.emptySortedSet();
}
};
}

View File

@ -19,8 +19,8 @@
package org.elasticsearch.index.fielddata.plain;
import org.apache.lucene.index.LeafReaderContext;
import org.apache.lucene.index.IndexReader;
import org.apache.lucene.index.LeafReaderContext;
import org.apache.lucene.index.Terms;
import org.apache.lucene.index.TermsEnum;
import org.apache.lucene.util.BytesRef;
@ -29,7 +29,6 @@ import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.index.AbstractIndexComponent;
import org.elasticsearch.index.Index;
import org.elasticsearch.index.fielddata.*;
import org.elasticsearch.index.mapper.FieldMapper;
import org.elasticsearch.index.mapper.MappedFieldType;
import org.elasticsearch.index.settings.IndexSettings;
@ -72,6 +71,11 @@ public abstract class AbstractIndexFieldData<FD extends AtomicFieldData> extends
@Override
public FD load(LeafReaderContext context) {
if (context.reader().getFieldInfos().fieldInfo(fieldNames.indexName()) == null) {
// If the field doesn't exist, then don't bother with loading and adding an empty instance to the field data cache
return empty(context.reader().maxDoc());
}
try {
FD fd = cache.load(context, this);
return fd;
@ -84,6 +88,12 @@ public abstract class AbstractIndexFieldData<FD extends AtomicFieldData> extends
}
}
/**
* @param maxDoc of the current reader
* @return an empty field data instances for field data lookups of empty segments (returning no values)
*/
protected abstract FD empty(int maxDoc);
/**
* A {@code PerValueEstimator} is a sub-class that can be used to estimate
* the memory overhead for loading the data. Each field data

View File

@ -80,4 +80,9 @@ abstract class AbstractIndexGeoPointFieldData extends AbstractIndexFieldData<Ato
throw new IllegalArgumentException("can't sort on geo_point field without using specific sorting feature, like geo_distance");
}
@Override
protected AtomicGeoPointFieldData empty(int maxDoc) {
return AbstractAtomicGeoPointFieldData.empty(maxDoc);
}
}

View File

@ -80,6 +80,11 @@ public abstract class AbstractIndexOrdinalsFieldData extends AbstractIndexFieldD
return GlobalOrdinalsBuilder.build(indexReader, this, indexSettings, breakerService, logger);
}
@Override
protected AtomicOrdinalsFieldData empty(int maxDoc) {
return AbstractAtomicOrdinalsFieldData.empty();
}
protected TermsEnum filter(Terms terms, LeafReader reader) throws IOException {
TermsEnum iterator = terms.iterator();
if (iterator == null) {

View File

@ -55,6 +55,11 @@ public final class DisabledIndexFieldData extends AbstractIndexFieldData<AtomicF
throw fail();
}
@Override
protected AtomicFieldData empty(int maxDoc) {
throw fail();
}
@Override
public IndexFieldData.XFieldComparatorSource comparatorSource(Object missingValue, MultiValueMode sortMode, Nested nested) {
throw fail();

View File

@ -19,40 +19,19 @@
package org.elasticsearch.index.fielddata.plain;
import org.apache.lucene.index.DocValues;
import org.apache.lucene.index.LeafReader;
import org.apache.lucene.index.LeafReaderContext;
import org.apache.lucene.index.RandomAccessOrds;
import org.apache.lucene.index.SortedDocValues;
import org.apache.lucene.index.SortedSetDocValues;
import org.apache.lucene.index.Terms;
import org.apache.lucene.util.Accountable;
import org.apache.lucene.util.Accountables;
import org.apache.lucene.util.BitSet;
import org.apache.lucene.util.Bits;
import org.apache.lucene.util.BytesRef;
import org.apache.lucene.util.BytesRefIterator;
import org.apache.lucene.util.NumericUtils;
import org.apache.lucene.util.RamUsageEstimator;
import org.apache.lucene.index.*;
import org.apache.lucene.util.*;
import org.elasticsearch.common.Nullable;
import org.elasticsearch.common.breaker.CircuitBreaker;
import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.common.util.BigArrays;
import org.elasticsearch.common.util.DoubleArray;
import org.elasticsearch.index.Index;
import org.elasticsearch.index.fielddata.AtomicNumericFieldData;
import org.elasticsearch.index.fielddata.FieldData;
import org.elasticsearch.index.fielddata.FieldDataType;
import org.elasticsearch.index.fielddata.IndexFieldData;
import org.elasticsearch.index.fielddata.*;
import org.elasticsearch.index.fielddata.IndexFieldData.XFieldComparatorSource.Nested;
import org.elasticsearch.index.fielddata.IndexFieldDataCache;
import org.elasticsearch.index.fielddata.IndexNumericFieldData;
import org.elasticsearch.index.fielddata.NumericDoubleValues;
import org.elasticsearch.index.fielddata.SortedNumericDoubleValues;
import org.elasticsearch.index.fielddata.fieldcomparator.DoubleValuesComparatorSource;
import org.elasticsearch.index.fielddata.ordinals.Ordinals;
import org.elasticsearch.index.fielddata.ordinals.OrdinalsBuilder;
import org.elasticsearch.index.mapper.FieldMapper;
import org.elasticsearch.index.mapper.MappedFieldType;
import org.elasticsearch.index.mapper.MapperService;
import org.elasticsearch.index.settings.IndexSettings;
@ -206,6 +185,11 @@ public class DoubleArrayIndexFieldData extends AbstractIndexFieldData<AtomicNume
}
@Override
protected AtomicNumericFieldData empty(int maxDoc) {
return AtomicDoubleFieldData.empty(maxDoc);
}
@Override
public XFieldComparatorSource comparatorSource(@Nullable Object missingValue, MultiValueMode sortMode, Nested nested) {
return new DoubleValuesComparatorSource(this, missingValue, sortMode, nested);

View File

@ -18,40 +18,19 @@
*/
package org.elasticsearch.index.fielddata.plain;
import org.apache.lucene.index.DocValues;
import org.apache.lucene.index.LeafReader;
import org.apache.lucene.index.LeafReaderContext;
import org.apache.lucene.index.RandomAccessOrds;
import org.apache.lucene.index.SortedDocValues;
import org.apache.lucene.index.SortedSetDocValues;
import org.apache.lucene.index.Terms;
import org.apache.lucene.util.Accountable;
import org.apache.lucene.util.Accountables;
import org.apache.lucene.util.BitSet;
import org.apache.lucene.util.Bits;
import org.apache.lucene.util.BytesRef;
import org.apache.lucene.util.BytesRefIterator;
import org.apache.lucene.util.NumericUtils;
import org.apache.lucene.util.RamUsageEstimator;
import org.apache.lucene.index.*;
import org.apache.lucene.util.*;
import org.elasticsearch.common.Nullable;
import org.elasticsearch.common.breaker.CircuitBreaker;
import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.common.util.BigArrays;
import org.elasticsearch.common.util.FloatArray;
import org.elasticsearch.index.Index;
import org.elasticsearch.index.fielddata.AtomicNumericFieldData;
import org.elasticsearch.index.fielddata.FieldData;
import org.elasticsearch.index.fielddata.FieldDataType;
import org.elasticsearch.index.fielddata.IndexFieldData;
import org.elasticsearch.index.fielddata.*;
import org.elasticsearch.index.fielddata.IndexFieldData.XFieldComparatorSource.Nested;
import org.elasticsearch.index.fielddata.IndexFieldDataCache;
import org.elasticsearch.index.fielddata.IndexNumericFieldData;
import org.elasticsearch.index.fielddata.NumericDoubleValues;
import org.elasticsearch.index.fielddata.SortedNumericDoubleValues;
import org.elasticsearch.index.fielddata.fieldcomparator.FloatValuesComparatorSource;
import org.elasticsearch.index.fielddata.ordinals.Ordinals;
import org.elasticsearch.index.fielddata.ordinals.OrdinalsBuilder;
import org.elasticsearch.index.mapper.FieldMapper;
import org.elasticsearch.index.mapper.MappedFieldType;
import org.elasticsearch.index.mapper.MapperService;
import org.elasticsearch.index.settings.IndexSettings;
@ -204,6 +183,11 @@ public class FloatArrayIndexFieldData extends AbstractIndexFieldData<AtomicNumer
}
@Override
protected AtomicNumericFieldData empty(int maxDoc) {
return AtomicDoubleFieldData.empty(maxDoc);
}
@Override
public XFieldComparatorSource comparatorSource(@Nullable Object missingValue, MultiValueMode sortMode, Nested nested) {
return new FloatValuesComparatorSource(this, missingValue, sortMode, nested);

View File

@ -20,24 +20,9 @@
package org.elasticsearch.index.fielddata.plain;
import com.google.common.base.Preconditions;
import org.apache.lucene.index.DocValues;
import org.apache.lucene.index.LeafReader;
import org.apache.lucene.index.LeafReaderContext;
import org.apache.lucene.index.NumericDocValues;
import org.apache.lucene.index.RandomAccessOrds;
import org.apache.lucene.index.SortedDocValues;
import org.apache.lucene.index.SortedNumericDocValues;
import org.apache.lucene.index.Terms;
import org.apache.lucene.index.TermsEnum;
import org.apache.lucene.util.Accountable;
import org.apache.lucene.util.Accountables;
import org.apache.lucene.index.*;
import org.apache.lucene.util.*;
import org.apache.lucene.util.BitSet;
import org.apache.lucene.util.Bits;
import org.apache.lucene.util.BytesRef;
import org.apache.lucene.util.BytesRefIterator;
import org.apache.lucene.util.LongValues;
import org.apache.lucene.util.RamUsageEstimator;
import org.apache.lucene.util.packed.PackedInts;
import org.apache.lucene.util.packed.PackedLongValues;
import org.elasticsearch.ElasticsearchException;
@ -45,18 +30,11 @@ import org.elasticsearch.common.Nullable;
import org.elasticsearch.common.breaker.CircuitBreaker;
import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.index.Index;
import org.elasticsearch.index.fielddata.AtomicNumericFieldData;
import org.elasticsearch.index.fielddata.FieldData;
import org.elasticsearch.index.fielddata.FieldDataType;
import org.elasticsearch.index.fielddata.IndexFieldData;
import org.elasticsearch.index.fielddata.*;
import org.elasticsearch.index.fielddata.IndexFieldData.XFieldComparatorSource.Nested;
import org.elasticsearch.index.fielddata.IndexFieldDataCache;
import org.elasticsearch.index.fielddata.IndexNumericFieldData;
import org.elasticsearch.index.fielddata.RamAccountingTermsEnum;
import org.elasticsearch.index.fielddata.fieldcomparator.LongValuesComparatorSource;
import org.elasticsearch.index.fielddata.ordinals.Ordinals;
import org.elasticsearch.index.fielddata.ordinals.OrdinalsBuilder;
import org.elasticsearch.index.mapper.FieldMapper;
import org.elasticsearch.index.mapper.MappedFieldType;
import org.elasticsearch.index.mapper.MapperService;
import org.elasticsearch.index.settings.IndexSettings;
@ -64,11 +42,7 @@ import org.elasticsearch.indices.breaker.CircuitBreakerService;
import org.elasticsearch.search.MultiValueMode;
import java.io.IOException;
import java.util.ArrayList;
import java.util.Collection;
import java.util.Collections;
import java.util.EnumSet;
import java.util.List;
import java.util.*;
/**
* Stores numeric data into bit-packed arrays for better memory efficiency.
@ -404,6 +378,11 @@ public class PackedArrayIndexFieldData extends AbstractIndexFieldData<AtomicNume
return pageMemorySize;
}
@Override
protected AtomicNumericFieldData empty(int maxDoc) {
return AtomicLongFieldData.empty(maxDoc);
}
@Override
public XFieldComparatorSource comparatorSource(@Nullable Object missingValue, MultiValueMode sortMode, Nested nested) {
return new LongValuesComparatorSource(this, missingValue, sortMode, nested);

View File

@ -23,6 +23,7 @@ import com.carrotsearch.hppc.ObjectObjectHashMap;
import com.carrotsearch.hppc.cursors.ObjectObjectCursor;
import com.google.common.collect.ImmutableSet;
import com.google.common.collect.ImmutableSortedSet;
import org.apache.lucene.index.*;
import org.apache.lucene.index.MultiDocValues.OrdinalMap;
import org.apache.lucene.search.DocIdSetIterator;
@ -79,12 +80,23 @@ public class ParentChildIndexFieldData extends AbstractIndexFieldData<AtomicPare
FieldDataType fieldDataType, IndexFieldDataCache cache, MapperService mapperService,
CircuitBreakerService breakerService) {
super(index, indexSettings, fieldNames, fieldDataType, cache);
parentTypes = new TreeSet<>();
this.breakerService = breakerService;
for (DocumentMapper documentMapper : mapperService.docMappers(false)) {
beforeCreate(documentMapper);
if (Version.indexCreated(indexSettings).before(Version.V_2_0_0_beta1)) {
parentTypes = new TreeSet<>();
for (DocumentMapper documentMapper : mapperService.docMappers(false)) {
beforeCreate(documentMapper);
}
mapperService.addTypeListener(this);
} else {
ImmutableSortedSet.Builder<String> builder = ImmutableSortedSet.naturalOrder();
for (DocumentMapper mapper : mapperService.docMappers(false)) {
ParentFieldMapper parentFieldMapper = mapper.parentFieldMapper();
if (parentFieldMapper.active()) {
builder.add(parentFieldMapper.type());
}
}
parentTypes = builder.build();
}
mapperService.addTypeListener(this);
}
@Override
@ -96,10 +108,6 @@ public class ParentChildIndexFieldData extends AbstractIndexFieldData<AtomicPare
public AtomicParentChildFieldData load(LeafReaderContext context) {
if (Version.indexCreated(indexSettings).onOrAfter(Version.V_2_0_0_beta1)) {
final LeafReader reader = context.reader();
final NavigableSet<String> parentTypes;
synchronized (lock) {
parentTypes = ImmutableSortedSet.copyOf(this.parentTypes);
}
return new AbstractAtomicParentChildFieldData() {
public Set<String> types() {
@ -131,12 +139,22 @@ public class ParentChildIndexFieldData extends AbstractIndexFieldData<AtomicPare
}
};
} else {
return super.load(context);
try {
return cache.load(context, this);
} catch (Throwable e) {
if (e instanceof ElasticsearchException) {
throw (ElasticsearchException) e;
} else {
throw new ElasticsearchException(e.getMessage(), e);
}
}
}
}
@Override
public AbstractAtomicParentChildFieldData loadDirect(LeafReaderContext context) throws Exception {
// Make this method throw an UnsupportedOperationException in 3.0, only
// needed for indices created BEFORE 2.0
LeafReader reader = context.reader();
final float acceptableTransientOverheadRatio = fieldDataType.getSettings().getAsFloat(
"acceptable_transient_overhead_ratio", OrdinalsBuilder.DEFAULT_ACCEPTABLE_OVERHEAD_RATIO
@ -211,6 +229,7 @@ public class ParentChildIndexFieldData extends AbstractIndexFieldData<AtomicPare
@Override
public void beforeCreate(DocumentMapper mapper) {
// Remove in 3.0
synchronized (lock) {
ParentFieldMapper parentFieldMapper = mapper.parentFieldMapper();
if (parentFieldMapper.active()) {
@ -224,13 +243,8 @@ public class ParentChildIndexFieldData extends AbstractIndexFieldData<AtomicPare
}
@Override
public void afterRemove(DocumentMapper mapper) {
synchronized (lock) {
ParentFieldMapper parentFieldMapper = mapper.parentFieldMapper();
if (parentFieldMapper.active()) {
parentTypes.remove(new BytesRef(parentFieldMapper.type()));
}
}
protected AtomicParentChildFieldData empty(int maxDoc) {
return new ParentChildAtomicFieldData(ImmutableOpenMap.<String, AtomicOrdinalsFieldData>of());
}
class TypeBuilder {
@ -345,8 +359,12 @@ public class ParentChildIndexFieldData extends AbstractIndexFieldData<AtomicPare
public IndexParentChildFieldData localGlobalDirect(IndexReader indexReader) throws Exception {
final long startTime = System.nanoTime();
final Set<String> parentTypes;
synchronized (lock) {
parentTypes = ImmutableSet.copyOf(this.parentTypes);
if (Version.indexCreated(indexSettings).before(Version.V_2_0_0_beta1)) {
synchronized (lock) {
parentTypes = ImmutableSet.copyOf(this.parentTypes);
}
} else {
parentTypes = this.parentTypes;
}
long ramBytesUsed = 0;

View File

@ -30,11 +30,4 @@ public interface DocumentTypeListener {
*/
void beforeCreate(DocumentMapper mapper);
/**
* Invoked just after an existing document type has been removed.
*
* @param mapper The existing document mapper of the type being removed
*/
void afterRemove(DocumentMapper mapper);
}

View File

@ -30,7 +30,6 @@ import org.apache.lucene.util.BytesRef;
import org.elasticsearch.action.fieldstats.FieldStats;
import org.elasticsearch.common.Nullable;
import org.elasticsearch.common.lucene.BytesRefs;
import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.common.unit.Fuzziness;
import org.elasticsearch.index.analysis.NamedAnalyzer;
import org.elasticsearch.index.fielddata.FieldDataType;
@ -186,6 +185,7 @@ public abstract class MappedFieldType extends FieldType {
fieldDataType = new FieldDataType(typeName());
}
@Override
public abstract MappedFieldType clone();
@Override
@ -449,7 +449,7 @@ public abstract class MappedFieldType extends FieldType {
return new TermsQuery(names.indexName(), bytesRefs);
}
public Query rangeQuery(Object lowerTerm, Object upperTerm, boolean includeLower, boolean includeUpper, @Nullable QueryParseContext context) {
public Query rangeQuery(Object lowerTerm, Object upperTerm, boolean includeLower, boolean includeUpper) {
return new TermRangeQuery(names().indexName(),
lowerTerm == null ? null : indexedValueForSearch(lowerTerm),
upperTerm == null ? null : indexedValueForSearch(upperTerm),

View File

@ -622,6 +622,10 @@ public class MapperService extends AbstractIndexComponent {
return META_FIELDS.contains(fieldName);
}
public static String[] getAllMetaFields() {
return META_FIELDS.toArray(String.class);
}
/** An analyzer wrapper that can lookup fields within the index mappings */
final class MapperAnalyzerWrapper extends DelegatingAnalyzerWrapper {

View File

@ -30,7 +30,6 @@ import org.apache.lucene.util.BytesRefBuilder;
import org.apache.lucene.util.NumericUtils;
import org.elasticsearch.action.fieldstats.FieldStats;
import org.elasticsearch.common.Explicit;
import org.elasticsearch.common.Nullable;
import org.elasticsearch.common.Strings;
import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.common.unit.Fuzziness;
@ -42,8 +41,6 @@ import org.elasticsearch.index.mapper.MappedFieldType;
import org.elasticsearch.index.mapper.Mapper;
import org.elasticsearch.index.mapper.MapperParsingException;
import org.elasticsearch.index.mapper.ParseContext;
import org.elasticsearch.index.query.QueryParseContext;
import java.io.IOException;
import java.util.Iterator;
import java.util.List;
@ -163,7 +160,7 @@ public class ByteFieldMapper extends NumberFieldMapper {
}
@Override
public Query rangeQuery(Object lowerTerm, Object upperTerm, boolean includeLower, boolean includeUpper, @Nullable QueryParseContext context) {
public Query rangeQuery(Object lowerTerm, Object upperTerm, boolean includeLower, boolean includeUpper) {
return NumericRangeQuery.newIntRange(names().indexName(), numericPrecisionStep(),
lowerTerm == null ? null : (int)parseValue(lowerTerm),
upperTerm == null ? null : (int)parseValue(upperTerm),

View File

@ -51,12 +51,15 @@ import org.elasticsearch.index.mapper.Mapper;
import org.elasticsearch.index.mapper.MapperParsingException;
import org.elasticsearch.index.mapper.ParseContext;
import org.elasticsearch.index.mapper.core.LongFieldMapper.CustomLongNumericField;
import org.elasticsearch.index.query.QueryParseContext;
import org.elasticsearch.search.internal.SearchContext;
import org.joda.time.DateTimeZone;
import java.io.IOException;
import java.util.*;
import java.util.Iterator;
import java.util.List;
import java.util.Locale;
import java.util.Map;
import java.util.Objects;
import java.util.concurrent.Callable;
import java.util.concurrent.TimeUnit;
@ -124,6 +127,7 @@ public class DateFieldMapper extends NumberFieldMapper {
return fieldMapper;
}
@Override
protected void setupFieldType(BuilderContext context) {
if (Version.indexCreated(context.indexSettings()).before(Version.V_2_0_0_beta1) &&
!fieldType().dateTimeFormatter().format().contains("epoch_")) {
@ -277,6 +281,7 @@ public class DateFieldMapper extends NumberFieldMapper {
this.dateMathParser = ref.dateMathParser;
}
@Override
public DateFieldType clone() {
return new DateFieldType(this);
}
@ -390,8 +395,8 @@ public class DateFieldMapper extends NumberFieldMapper {
}
@Override
public Query rangeQuery(Object lowerTerm, Object upperTerm, boolean includeLower, boolean includeUpper, @Nullable QueryParseContext context) {
return rangeQuery(lowerTerm, upperTerm, includeLower, includeUpper, null, null, context);
public Query rangeQuery(Object lowerTerm, Object upperTerm, boolean includeLower, boolean includeUpper) {
return rangeQuery(lowerTerm, upperTerm, includeLower, includeUpper, null, null);
}
@Override
@ -419,7 +424,7 @@ public class DateFieldMapper extends NumberFieldMapper {
);
}
public Query rangeQuery(Object lowerTerm, Object upperTerm, boolean includeLower, boolean includeUpper, @Nullable DateTimeZone timeZone, @Nullable DateMathParser forcedDateParser, @Nullable QueryParseContext context) {
public Query rangeQuery(Object lowerTerm, Object upperTerm, boolean includeLower, boolean includeUpper, @Nullable DateTimeZone timeZone, @Nullable DateMathParser forcedDateParser) {
return new LateParsingQuery(lowerTerm, upperTerm, includeLower, includeUpper, timeZone, forcedDateParser);
}

View File

@ -20,6 +20,7 @@
package org.elasticsearch.index.mapper.core;
import com.carrotsearch.hppc.DoubleArrayList;
import org.apache.lucene.analysis.Analyzer;
import org.apache.lucene.analysis.TokenStream;
import org.apache.lucene.document.Field;
@ -32,7 +33,6 @@ import org.apache.lucene.util.BytesRefBuilder;
import org.apache.lucene.util.NumericUtils;
import org.elasticsearch.action.fieldstats.FieldStats;
import org.elasticsearch.common.Explicit;
import org.elasticsearch.common.Nullable;
import org.elasticsearch.common.Numbers;
import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.common.unit.Fuzziness;
@ -46,7 +46,6 @@ import org.elasticsearch.index.mapper.MappedFieldType;
import org.elasticsearch.index.mapper.Mapper;
import org.elasticsearch.index.mapper.MapperParsingException;
import org.elasticsearch.index.mapper.ParseContext;
import org.elasticsearch.index.query.QueryParseContext;
import java.io.IOException;
import java.util.Iterator;
@ -169,7 +168,7 @@ public class DoubleFieldMapper extends NumberFieldMapper {
}
@Override
public Query rangeQuery(Object lowerTerm, Object upperTerm, boolean includeLower, boolean includeUpper, @Nullable QueryParseContext context) {
public Query rangeQuery(Object lowerTerm, Object upperTerm, boolean includeLower, boolean includeUpper) {
return NumericRangeQuery.newDoubleRange(names().indexName(), numericPrecisionStep(),
lowerTerm == null ? null : parseDoubleValue(lowerTerm),
upperTerm == null ? null : parseDoubleValue(upperTerm),

View File

@ -20,6 +20,7 @@
package org.elasticsearch.index.mapper.core;
import com.carrotsearch.hppc.FloatArrayList;
import org.apache.lucene.analysis.Analyzer;
import org.apache.lucene.analysis.TokenStream;
import org.apache.lucene.document.Field;
@ -32,7 +33,6 @@ import org.apache.lucene.util.BytesRefBuilder;
import org.apache.lucene.util.NumericUtils;
import org.elasticsearch.action.fieldstats.FieldStats;
import org.elasticsearch.common.Explicit;
import org.elasticsearch.common.Nullable;
import org.elasticsearch.common.Numbers;
import org.elasticsearch.common.Strings;
import org.elasticsearch.common.settings.Settings;
@ -47,7 +47,6 @@ import org.elasticsearch.index.mapper.MappedFieldType;
import org.elasticsearch.index.mapper.Mapper;
import org.elasticsearch.index.mapper.MapperParsingException;
import org.elasticsearch.index.mapper.ParseContext;
import org.elasticsearch.index.query.QueryParseContext;
import java.io.IOException;
import java.util.Iterator;
@ -170,7 +169,7 @@ public class FloatFieldMapper extends NumberFieldMapper {
}
@Override
public Query rangeQuery(Object lowerTerm, Object upperTerm, boolean includeLower, boolean includeUpper, @Nullable QueryParseContext context) {
public Query rangeQuery(Object lowerTerm, Object upperTerm, boolean includeLower, boolean includeUpper) {
return NumericRangeQuery.newFloatRange(names().indexName(), numericPrecisionStep(),
lowerTerm == null ? null : parseValue(lowerTerm),
upperTerm == null ? null : parseValue(upperTerm),

View File

@ -31,7 +31,6 @@ import org.apache.lucene.util.BytesRefBuilder;
import org.apache.lucene.util.NumericUtils;
import org.elasticsearch.action.fieldstats.FieldStats;
import org.elasticsearch.common.Explicit;
import org.elasticsearch.common.Nullable;
import org.elasticsearch.common.Numbers;
import org.elasticsearch.common.Strings;
import org.elasticsearch.common.settings.Settings;
@ -44,8 +43,6 @@ import org.elasticsearch.index.mapper.MappedFieldType;
import org.elasticsearch.index.mapper.Mapper;
import org.elasticsearch.index.mapper.MapperParsingException;
import org.elasticsearch.index.mapper.ParseContext;
import org.elasticsearch.index.query.QueryParseContext;
import java.io.IOException;
import java.util.Iterator;
import java.util.List;
@ -172,7 +169,7 @@ public class IntegerFieldMapper extends NumberFieldMapper {
}
@Override
public Query rangeQuery(Object lowerTerm, Object upperTerm, boolean includeLower, boolean includeUpper, @Nullable QueryParseContext context) {
public Query rangeQuery(Object lowerTerm, Object upperTerm, boolean includeLower, boolean includeUpper) {
return NumericRangeQuery.newIntRange(names().indexName(), numericPrecisionStep(),
lowerTerm == null ? null : parseValue(lowerTerm),
upperTerm == null ? null : parseValue(upperTerm),

View File

@ -31,7 +31,6 @@ import org.apache.lucene.util.BytesRefBuilder;
import org.apache.lucene.util.NumericUtils;
import org.elasticsearch.action.fieldstats.FieldStats;
import org.elasticsearch.common.Explicit;
import org.elasticsearch.common.Nullable;
import org.elasticsearch.common.Numbers;
import org.elasticsearch.common.Strings;
import org.elasticsearch.common.settings.Settings;
@ -44,8 +43,6 @@ import org.elasticsearch.index.mapper.MappedFieldType;
import org.elasticsearch.index.mapper.Mapper;
import org.elasticsearch.index.mapper.MapperParsingException;
import org.elasticsearch.index.mapper.ParseContext;
import org.elasticsearch.index.query.QueryParseContext;
import java.io.IOException;
import java.util.Iterator;
import java.util.List;
@ -170,7 +167,7 @@ public class LongFieldMapper extends NumberFieldMapper {
}
@Override
public Query rangeQuery(Object lowerTerm, Object upperTerm, boolean includeLower, boolean includeUpper, @Nullable QueryParseContext context) {
public Query rangeQuery(Object lowerTerm, Object upperTerm, boolean includeLower, boolean includeUpper) {
return NumericRangeQuery.newLongRange(names().indexName(), numericPrecisionStep(),
lowerTerm == null ? null : parseLongValue(lowerTerm),
upperTerm == null ? null : parseLongValue(upperTerm),

View File

@ -31,7 +31,6 @@ import org.apache.lucene.util.BytesRefBuilder;
import org.apache.lucene.util.NumericUtils;
import org.elasticsearch.action.fieldstats.FieldStats;
import org.elasticsearch.common.Explicit;
import org.elasticsearch.common.Nullable;
import org.elasticsearch.common.Numbers;
import org.elasticsearch.common.Strings;
import org.elasticsearch.common.settings.Settings;
@ -44,8 +43,6 @@ import org.elasticsearch.index.mapper.MappedFieldType;
import org.elasticsearch.index.mapper.Mapper;
import org.elasticsearch.index.mapper.MapperParsingException;
import org.elasticsearch.index.mapper.ParseContext;
import org.elasticsearch.index.query.QueryParseContext;
import java.io.IOException;
import java.util.Iterator;
import java.util.List;
@ -168,7 +165,7 @@ public class ShortFieldMapper extends NumberFieldMapper {
}
@Override
public Query rangeQuery(Object lowerTerm, Object upperTerm, boolean includeLower, boolean includeUpper, @Nullable QueryParseContext context) {
public Query rangeQuery(Object lowerTerm, Object upperTerm, boolean includeLower, boolean includeUpper) {
return NumericRangeQuery.newIntRange(names().indexName(), numericPrecisionStep(),
lowerTerm == null ? null : (int)parseValue(lowerTerm),
upperTerm == null ? null : (int)parseValue(upperTerm),

View File

@ -29,7 +29,6 @@ import org.apache.lucene.util.BytesRef;
import org.apache.lucene.util.BytesRefBuilder;
import org.apache.lucene.util.NumericUtils;
import org.elasticsearch.common.Explicit;
import org.elasticsearch.common.Nullable;
import org.elasticsearch.common.Numbers;
import org.elasticsearch.common.Strings;
import org.elasticsearch.common.settings.Settings;
@ -47,8 +46,6 @@ import org.elasticsearch.index.mapper.ParseContext;
import org.elasticsearch.index.mapper.core.LongFieldMapper;
import org.elasticsearch.index.mapper.core.LongFieldMapper.CustomLongNumericField;
import org.elasticsearch.index.mapper.core.NumberFieldMapper;
import org.elasticsearch.index.query.QueryParseContext;
import java.io.IOException;
import java.util.Iterator;
import java.util.List;
@ -209,7 +206,7 @@ public class IpFieldMapper extends NumberFieldMapper {
}
@Override
public Query rangeQuery(Object lowerTerm, Object upperTerm, boolean includeLower, boolean includeUpper, @Nullable QueryParseContext context) {
public Query rangeQuery(Object lowerTerm, Object upperTerm, boolean includeLower, boolean includeUpper) {
return NumericRangeQuery.newLongRange(names().indexName(), numericPrecisionStep(),
lowerTerm == null ? null : parseValue(lowerTerm),
upperTerm == null ? null : parseValue(upperTerm),

View File

@ -226,14 +226,6 @@ public class PercolatorQueriesRegistry extends AbstractIndexShardComponent imple
}
}
@Override
public void afterRemove(DocumentMapper mapper) {
if (PercolatorService.TYPE_NAME.equals(mapper.type())) {
disableRealTimePercolator();
clear();
}
}
}
private class ShardLifecycleListener extends IndicesLifecycle.Listener {

View File

@ -107,7 +107,7 @@ public class ExistsQueryBuilder extends AbstractQueryBuilder<ExistsQueryBuilder>
}
// if _field_names are not indexed, we need to go the slow way
if (filter == null && fieldType != null) {
filter = fieldType.rangeQuery(null, null, true, true, parseContext);
filter = fieldType.rangeQuery(null, null, true, true);
}
if (filter == null) {
filter = new TermRangeQuery(field, null, null, true, true);

View File

@ -55,11 +55,11 @@ public class MissingQueryBuilder extends AbstractQueryBuilder<MissingQueryBuilde
public MissingQueryBuilder(String fieldPattern) {
this.fieldPattern = fieldPattern;
}
public String fieldPattern() {
return this.fieldPattern;
}
/**
* Should the missing filter automatically include fields with null value configured in the
* mappings. Defaults to <tt>false</tt>.
@ -159,7 +159,7 @@ public class MissingQueryBuilder extends AbstractQueryBuilder<MissingQueryBuilde
}
// if _field_names are not indexed, we need to go the slow way
if (filter == null && fieldType != null) {
filter = fieldType.rangeQuery(null, null, true, true, parseContext);
filter = fieldType.rangeQuery(null, null, true, true);
}
if (filter == null) {
filter = new TermRangeQuery(field, null, null, true, true);

View File

@ -256,14 +256,14 @@ public class RangeQueryBuilder extends AbstractQueryBuilder<RangeQueryBuilder> i
if (this.timeZone != null) {
dateTimeZone = DateTimeZone.forID(this.timeZone);
}
query = ((DateFieldMapper.DateFieldType) mapper).rangeQuery(from, to, includeLower, includeUpper, dateTimeZone, forcedDateParser, parseContext);
query = ((DateFieldMapper.DateFieldType) mapper).rangeQuery(from, to, includeLower, includeUpper, dateTimeZone, forcedDateParser);
} else {
if (timeZone != null) {
throw new QueryParsingException(parseContext, "[range] time_zone can not be applied to non date field ["
+ fieldName + "]");
}
//LUCENE 4 UPGRADE Mapper#rangeQuery should use bytesref as well?
query = mapper.rangeQuery(from, to, includeLower, includeUpper, parseContext);
query = mapper.rangeQuery(from, to, includeLower, includeUpper);
}
} else {
if (timeZone != null) {

View File

@ -45,16 +45,16 @@ public class IndexedGeoBoundingBoxQuery {
private static Query westGeoBoundingBoxFilter(GeoPoint topLeft, GeoPoint bottomRight, GeoPointFieldMapper.GeoPointFieldType fieldType) {
BooleanQuery filter = new BooleanQuery();
filter.setMinimumNumberShouldMatch(1);
filter.add(fieldType.lonFieldType().rangeQuery(null, bottomRight.lon(), true, true, null), Occur.SHOULD);
filter.add(fieldType.lonFieldType().rangeQuery(topLeft.lon(), null, true, true, null), Occur.SHOULD);
filter.add(fieldType.latFieldType().rangeQuery(bottomRight.lat(), topLeft.lat(), true, true, null), Occur.MUST);
filter.add(fieldType.lonFieldType().rangeQuery(null, bottomRight.lon(), true, true), Occur.SHOULD);
filter.add(fieldType.lonFieldType().rangeQuery(topLeft.lon(), null, true, true), Occur.SHOULD);
filter.add(fieldType.latFieldType().rangeQuery(bottomRight.lat(), topLeft.lat(), true, true), Occur.MUST);
return new ConstantScoreQuery(filter);
}
private static Query eastGeoBoundingBoxFilter(GeoPoint topLeft, GeoPoint bottomRight, GeoPointFieldMapper.GeoPointFieldType fieldType) {
BooleanQuery filter = new BooleanQuery();
filter.add(fieldType.lonFieldType().rangeQuery(topLeft.lon(), bottomRight.lon(), true, true, null), Occur.MUST);
filter.add(fieldType.latFieldType().rangeQuery(bottomRight.lat(), topLeft.lat(), true, true, null), Occur.MUST);
filter.add(fieldType.lonFieldType().rangeQuery(topLeft.lon(), bottomRight.lon(), true, true), Occur.MUST);
filter.add(fieldType.latFieldType().rangeQuery(bottomRight.lat(), topLeft.lat(), true, true), Occur.MUST);
return new ConstantScoreQuery(filter);
}
}

View File

@ -21,7 +21,6 @@ package org.elasticsearch.index.shard;
import com.google.common.base.Charsets;
import com.google.common.base.Preconditions;
import org.apache.lucene.codecs.PostingsFormat;
import org.apache.lucene.index.CheckIndex;
import org.apache.lucene.search.QueryCachingPolicy;
@ -169,6 +168,7 @@ public class IndexShard extends AbstractIndexShardComponent {
protected volatile IndexShardState state;
protected final AtomicReference<Engine> currentEngineReference = new AtomicReference<>();
protected final EngineFactory engineFactory;
private final IndexSearcherWrappingService wrappingService;
@Nullable
private RecoveryState recoveryState;
@ -198,12 +198,13 @@ public class IndexShard extends AbstractIndexShardComponent {
IndicesQueryCache indicesQueryCache, ShardPercolateService shardPercolateService, CodecService codecService,
ShardTermVectorsService termVectorsService, IndexFieldDataService indexFieldDataService, IndexService indexService,
@Nullable IndicesWarmer warmer, SnapshotDeletionPolicy deletionPolicy, SimilarityService similarityService, EngineFactory factory,
ClusterService clusterService, ShardPath path, BigArrays bigArrays) {
ClusterService clusterService, ShardPath path, BigArrays bigArrays, IndexSearcherWrappingService wrappingService) {
super(shardId, indexSettingsService.getSettings());
this.codecService = codecService;
this.warmer = warmer;
this.deletionPolicy = deletionPolicy;
this.similarityService = similarityService;
this.wrappingService = wrappingService;
Preconditions.checkNotNull(store, "Store must be provided to the index shard");
Preconditions.checkNotNull(deletionPolicy, "Snapshot deletion policy must be provided to the index shard");
this.engineFactory = factory;
@ -337,14 +338,16 @@ public class IndexShard extends AbstractIndexShardComponent {
if (!newRouting.shardId().equals(shardId())) {
throw new IllegalArgumentException("Trying to set a routing entry with shardId [" + newRouting.shardId() + "] on a shard with shardId [" + shardId() + "]");
}
if ((currentRouting == null || newRouting.isSameAllocation(currentRouting)) == false) {
throw new IllegalArgumentException("Trying to set a routing entry with a different allocation. Current " + currentRouting + ", new " + newRouting);
}
try {
if (currentRouting != null) {
assert newRouting.version() > currentRouting.version() : "expected: " + newRouting.version() + " > " + currentRouting.version();
if (!newRouting.primary() && currentRouting.primary()) {
logger.warn("suspect illegal state: trying to move shard from primary mode to replica mode");
}
// if its the same routing, return
if (currentRouting.equals(newRouting)) {
// if its the same routing except for some metadata info, return
if (currentRouting.equalsIgnoringMetaData(newRouting)) {
this.shardRouting = newRouting; // might have a new version
return;
}
@ -723,12 +726,12 @@ public class IndexShard extends AbstractIndexShardComponent {
public org.apache.lucene.util.Version minimumCompatibleVersion() {
org.apache.lucene.util.Version luceneVersion = null;
for(Segment segment : engine().segments(false)) {
for (Segment segment : engine().segments(false)) {
if (luceneVersion == null || luceneVersion.onOrAfter(segment.getVersion())) {
luceneVersion = segment.getVersion();
}
}
return luceneVersion == null ? Version.indexCreated(indexSettings).luceneVersion : luceneVersion;
return luceneVersion == null ? Version.indexCreated(indexSettings).luceneVersion : luceneVersion;
}
public SnapshotIndexCommit snapshotIndex(boolean flushFirst) throws EngineException {
@ -1113,7 +1116,7 @@ public class IndexShard extends AbstractIndexShardComponent {
}
final int maxMergeCount = settings.getAsInt(MergeSchedulerConfig.MAX_MERGE_COUNT, mergeSchedulerConfig.getMaxMergeCount());
if (maxMergeCount != mergeSchedulerConfig.getMaxMergeCount()) {
if (maxMergeCount != mergeSchedulerConfig.getMaxMergeCount()) {
logger.info("updating [{}] from [{}] to [{}]", MergeSchedulerConfig.MAX_MERGE_COUNT, mergeSchedulerConfig.getMaxMergeCount(), maxMergeCount);
mergeSchedulerConfig.setMaxMergeCount(maxMergeCount);
change = true;
@ -1360,7 +1363,7 @@ public class IndexShard extends AbstractIndexShardComponent {
};
return new EngineConfig(shardId,
threadPool, indexingService, indexSettingsService.indexSettings(), warmer, store, deletionPolicy, mergePolicyConfig.getMergePolicy(), mergeSchedulerConfig,
mapperService.indexAnalyzer(), similarityService.similarity(), codecService, failedEngineListener, translogRecoveryPerformer, indexCache.query(), cachingPolicy, translogConfig);
mapperService.indexAnalyzer(), similarityService.similarity(), codecService, failedEngineListener, translogRecoveryPerformer, indexCache.query(), cachingPolicy, wrappingService, translogConfig);
}
private static class IndexShardOperationCounter extends AbstractRefCounted {

View File

@ -21,7 +21,10 @@ package org.elasticsearch.index.shard;
import org.elasticsearch.cluster.metadata.IndexMetaData;
import org.elasticsearch.common.inject.AbstractModule;
import org.elasticsearch.common.inject.multibindings.Multibinder;
import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.index.engine.IndexSearcherWrapper;
import org.elasticsearch.index.engine.IndexSearcherWrappingService;
import org.elasticsearch.index.engine.EngineFactory;
import org.elasticsearch.index.engine.InternalEngineFactory;
import org.elasticsearch.index.percolator.stats.ShardPercolateService;
@ -73,6 +76,10 @@ public class IndexShardModule extends AbstractModule {
bind(StoreRecoveryService.class).asEagerSingleton();
bind(ShardPercolateService.class).asEagerSingleton();
bind(ShardTermVectorsService.class).asEagerSingleton();
bind(IndexSearcherWrappingService.class).asEagerSingleton();
// this injects an empty set in IndexSearcherWrappingService, otherwise guice can't construct IndexSearcherWrappingService
Multibinder<IndexSearcherWrapper> multibinder
= Multibinder.newSetBinder(binder(), IndexSearcherWrapper.class);
}

View File

@ -28,6 +28,7 @@ import org.elasticsearch.index.aliases.IndexAliasesService;
import org.elasticsearch.index.cache.IndexCache;
import org.elasticsearch.index.codec.CodecService;
import org.elasticsearch.index.deletionpolicy.SnapshotDeletionPolicy;
import org.elasticsearch.index.engine.IndexSearcherWrappingService;
import org.elasticsearch.index.engine.Engine;
import org.elasticsearch.index.engine.EngineConfig;
import org.elasticsearch.index.engine.EngineFactory;
@ -66,13 +67,13 @@ public final class ShadowIndexShard extends IndexShard {
IndexService indexService, @Nullable IndicesWarmer warmer,
SnapshotDeletionPolicy deletionPolicy, SimilarityService similarityService,
EngineFactory factory, ClusterService clusterService,
ShardPath path, BigArrays bigArrays) throws IOException {
ShardPath path, BigArrays bigArrays, IndexSearcherWrappingService wrappingService) throws IOException {
super(shardId, indexSettingsService, indicesLifecycle, store, storeRecoveryService,
threadPool, mapperService, queryParserService, indexCache, indexAliasesService,
indicesQueryCache, shardPercolateService, codecService,
termVectorsService, indexFieldDataService, indexService,
warmer, deletionPolicy, similarityService,
factory, clusterService, path, bigArrays);
factory, clusterService, path, bigArrays, wrappingService);
}
/**

View File

@ -63,7 +63,11 @@ public final class ShardUtils {
if (reader instanceof ElasticsearchLeafReader) {
return (ElasticsearchLeafReader) reader;
} else {
return getElasticsearchLeafReader(FilterLeafReader.unwrap(reader));
// We need to use FilterLeafReader#getDelegate and not FilterLeafReader#unwrap, because
// If there are multiple levels of filtered leaf readers then with the unwrap() method it immediately
// returns the most inner leaf reader and thus skipping of over any other filtered leaf reader that
// may be instance of ElasticsearchLeafReader. This can cause us to miss the shardId.
return getElasticsearchLeafReader(((FilterLeafReader) reader).getDelegate());
}
}
return null;

View File

@ -507,7 +507,7 @@ public class IndicesClusterStateService extends AbstractLifecycleComponent<Indic
// for example: a shard that recovers from one node and now needs to recover to another node,
// or a replica allocated and then allocating a primary because the primary failed on another node
boolean shardHasBeenRemoved = false;
if (currentRoutingEntry.initializing() && shardRouting.initializing() && !currentRoutingEntry.equals(shardRouting)) {
if (currentRoutingEntry.isSameAllocation(shardRouting) == false) {
logger.debug("[{}][{}] removing shard (different instance of it allocated on this node, current [{}], global [{}])", shardRouting.index(), shardRouting.id(), currentRoutingEntry, shardRouting);
// closing the shard will also cancel any ongoing recovery.
indexService.removeShard(shardRouting.id(), "removing shard (different instance of it allocated on this node)");
@ -526,22 +526,19 @@ public class IndicesClusterStateService extends AbstractLifecycleComponent<Indic
// closing the shard will also cancel any ongoing recovery.
indexService.removeShard(shardRouting.id(), "removing shard (recovery source node changed)");
shardHasBeenRemoved = true;
}
}
if (shardHasBeenRemoved == false && (shardRouting.equals(indexShard.routingEntry()) == false || shardRouting.version() > indexShard.routingEntry().version())) {
if (shardRouting.primary() && indexShard.routingEntry().primary() == false && shardRouting.initializing() && indexShard.allowsPrimaryPromotion() == false) {
logger.debug("{} reinitialize shard on primary promotion", indexShard.shardId());
indexService.removeShard(shardId, "promoted to primary");
} else {
// if we happen to remove the shardRouting by id above we don't need to jump in here!
indexShard.updateRoutingEntry(shardRouting, event.state().blocks().disableStatePersistence() == false);
}
if (shardHasBeenRemoved == false) {
// shadow replicas do not support primary promotion. The master would reinitialize the shard, giving it a new allocation, meaning we should be there.
assert (shardRouting.primary() && currentRoutingEntry.primary() == false) == false || indexShard.allowsPrimaryPromotion() :
"shard for doesn't support primary promotion but master promoted it with changing allocation. New routing " + shardRouting + ", current routing " + currentRoutingEntry;
indexShard.updateRoutingEntry(shardRouting, event.state().blocks().disableStatePersistence() == false);
}
}
if (shardRouting.initializing()) {
applyInitializingShard(event.state(),indexMetaData, shardRouting);
applyInitializingShard(event.state(), indexMetaData, shardRouting);
}
}
}

View File

@ -196,7 +196,7 @@ public class IndicesTTLService extends AbstractLifecycleComponent<IndicesTTLServ
private void purgeShards(List<IndexShard> shardsToPurge) {
for (IndexShard shardToPurge : shardsToPurge) {
Query query = shardToPurge.indexService().mapperService().smartNameFieldType(TTLFieldMapper.NAME).rangeQuery(null, System.currentTimeMillis(), false, true, null);
Query query = shardToPurge.indexService().mapperService().smartNameFieldType(TTLFieldMapper.NAME).rangeQuery(null, System.currentTimeMillis(), false, true);
Engine.Searcher searcher = shardToPurge.acquireSearcher("indices_ttl");
try {
logger.debug("[{}][{}] purging shard", shardToPurge.routingEntry().index(), shardToPurge.routingEntry().id());

View File

@ -180,7 +180,7 @@ public class InternalSettingsPreparer {
static Settings replacePromptPlaceholders(Settings settings, Terminal terminal) {
UnmodifiableIterator<Map.Entry<String, String>> iter = settings.getAsMap().entrySet().iterator();
Settings.Builder builder = Settings.builder();
Settings.Builder builder = Settings.builder().classLoader(settings.getClassLoaderIfSet());
while (iter.hasNext()) {
Map.Entry<String, String> entry = iter.next();

View File

@ -19,6 +19,7 @@
package org.elasticsearch.plugins;
import org.elasticsearch.Version;
import org.elasticsearch.bootstrap.JarHell;
import org.elasticsearch.common.io.stream.StreamInput;
import org.elasticsearch.common.io.stream.StreamOutput;
import org.elasticsearch.common.io.stream.Streamable;
@ -110,12 +111,23 @@ public class PluginInfo implements Streamable, ToXContent {
if (esVersion.equals(Version.CURRENT) == false) {
throw new IllegalArgumentException("Elasticsearch version [" + esVersionString + "] is too old for plugin [" + name + "]");
}
String javaVersionString = props.getProperty("java.version");
if (javaVersionString == null) {
throw new IllegalArgumentException("Property [java.version] is missing for jvm plugin [" + name + "]");
}
JarHell.checkJavaVersion(name, javaVersionString);
isolated = Boolean.parseBoolean(props.getProperty("isolated", "true"));
classname = props.getProperty("classname");
if (classname == null) {
throw new IllegalArgumentException("Property [classname] is missing for jvm plugin [" + name + "]");
}
}
if (site) {
if (!Files.exists(dir.resolve("_site"))) {
throw new IllegalArgumentException("Plugin [" + name + "] is a site plugin but has no '_site/' directory");
}
}
return new PluginInfo(name, description, site, version, jvm, classname, isolated);
}

View File

@ -22,6 +22,7 @@ package org.elasticsearch.plugins;
import com.google.common.base.Strings;
import com.google.common.collect.ImmutableSet;
import com.google.common.collect.Iterators;
import org.apache.lucene.util.IOUtils;
import org.elasticsearch.ElasticsearchTimeoutException;
import org.elasticsearch.ExceptionsHelper;
@ -32,6 +33,7 @@ import org.elasticsearch.common.http.client.HttpDownloadHelper;
import org.elasticsearch.common.io.FileSystemUtils;
import org.elasticsearch.common.unit.TimeValue;
import org.elasticsearch.env.Environment;
import org.elasticsearch.plugins.PluginsService.Bundle;
import java.io.IOException;
import java.io.OutputStream;
@ -98,14 +100,6 @@ public class PluginManager {
if (name == null) {
throw new IllegalArgumentException("plugin name must be supplied with install [name].");
}
HttpDownloadHelper downloadHelper = new HttpDownloadHelper();
boolean downloaded = false;
HttpDownloadHelper.DownloadProgress progress;
if (outputMode == OutputMode.SILENT) {
progress = new HttpDownloadHelper.NullProgress();
} else {
progress = new HttpDownloadHelper.VerboseProgress(terminal.writer());
}
if (!Files.exists(environment.pluginsFile())) {
terminal.println("Plugins directory [%s] does not exist. Creating...", environment.pluginsFile());
@ -119,11 +113,20 @@ public class PluginManager {
PluginHandle pluginHandle = PluginHandle.parse(name);
checkForForbiddenName(pluginHandle.name);
Path pluginFile = pluginHandle.distroFile(environment);
// extract the plugin
final Path extractLocation = pluginHandle.extractedDir(environment);
if (Files.exists(extractLocation)) {
throw new IOException("plugin directory " + extractLocation.toAbsolutePath() + " already exists. To update the plugin, uninstall it first using remove " + name + " command");
Path pluginFile = download(pluginHandle, terminal);
extract(pluginHandle, terminal, pluginFile);
}
private Path download(PluginHandle pluginHandle, Terminal terminal) throws IOException {
Path pluginFile = pluginHandle.newDistroFile(environment);
HttpDownloadHelper downloadHelper = new HttpDownloadHelper();
boolean downloaded = false;
HttpDownloadHelper.DownloadProgress progress;
if (outputMode == OutputMode.SILENT) {
progress = new HttpDownloadHelper.NullProgress();
} else {
progress = new HttpDownloadHelper.VerboseProgress(terminal.writer());
}
// first, try directly from the URL provided
@ -162,97 +165,44 @@ public class PluginManager {
}
if (!downloaded) {
// try to cleanup what we downloaded
IOUtils.deleteFilesIgnoringExceptions(pluginFile);
throw new IOException("failed to download out of all possible locations..., use --verbose to get detailed information");
}
return pluginFile;
}
// unzip plugin to a temp dir
Path tmp = unzipToTemporary(pluginFile);
// create list of current jars in classpath
final List<URL> jars = new ArrayList<>();
ClassLoader loader = PluginManager.class.getClassLoader();
if (loader instanceof URLClassLoader) {
Collections.addAll(jars, ((URLClassLoader) loader).getURLs());
private void extract(PluginHandle pluginHandle, Terminal terminal, Path pluginFile) throws IOException {
final Path extractLocation = pluginHandle.extractedDir(environment);
if (Files.exists(extractLocation)) {
throw new IOException("plugin directory " + extractLocation.toAbsolutePath() + " already exists. To update the plugin, uninstall it first using 'remove " + pluginHandle.name + "' command");
}
// add any jars we find in the plugin to the list
Files.walkFileTree(tmp, new SimpleFileVisitor<Path>() {
@Override
public FileVisitResult visitFile(Path file, BasicFileAttributes attrs) throws IOException {
if (file.toString().endsWith(".jar")) {
jars.add(file.toUri().toURL());
}
return FileVisitResult.CONTINUE;
}
});
// unzip plugin to a staging temp dir, named for the plugin
Path tmp = Files.createTempDirectory(environment.tmpFile(), null);
Path root = tmp.resolve(pluginHandle.name);
unzipPlugin(pluginFile, root);
// check combined (current classpath + new jars to-be-added)
try {
JarHell.checkJarHell(jars.toArray(new URL[jars.size()]));
} catch (Exception ex) {
throw new RuntimeException(ex);
// find the actual root (in case its unzipped with extra directory wrapping)
root = findPluginRoot(root);
// read and validate the plugin descriptor
PluginInfo info = PluginInfo.readFromProperties(root);
terminal.println("%s", info);
// check for jar hell before any copying
if (info.isJvm()) {
jarHellCheck(root, info.isIsolated());
}
// install plugin
FileSystemUtils.copyDirectoryRecursively(root, extractLocation);
terminal.println("Installed %s into %s", pluginHandle.name, extractLocation.toAbsolutePath());
// cleanup
IOUtils.rm(tmp);
IOUtils.rm(tmp, pluginFile);
// TODO: we have a tmpdir made above, so avoid zipfilesystem
try (FileSystem zipFile = FileSystems.newFileSystem(pluginFile, null)) {
for (final Path root : zipFile.getRootDirectories() ) {
final Path[] topLevelFiles = FileSystemUtils.files(root);
//we check whether we need to remove the top-level folder while extracting
//sometimes (e.g. github) the downloaded archive contains a top-level folder which needs to be removed
final boolean stripTopLevelDirectory;
if (topLevelFiles.length == 1 && Files.isDirectory(topLevelFiles[0])) {
// valid names if the zip has only one top level directory
switch (topLevelFiles[0].getFileName().toString()) {
case "_site/":
case "bin/":
case "config/":
case "_dict/":
stripTopLevelDirectory = false;
break;
default:
stripTopLevelDirectory = true;
}
} else {
stripTopLevelDirectory = false;
}
Files.walkFileTree(root, new SimpleFileVisitor<Path>() {
@Override
public FileVisitResult visitFile(Path file, BasicFileAttributes attrs) throws IOException {
Path target = FileSystemUtils.append(extractLocation, file, stripTopLevelDirectory ? 1 : 0);
Files.createDirectories(target);
Files.copy(file, target, StandardCopyOption.REPLACE_EXISTING);
return FileVisitResult.CONTINUE;
}
});
}
terminal.println("Installed %s into %s", name, extractLocation.toAbsolutePath());
} catch (Exception e) {
terminal.printError("failed to extract plugin [%s]: %s", pluginFile, ExceptionsHelper.detailedMessage(e));
return;
} finally {
try {
Files.delete(pluginFile);
} catch (Exception ex) {
terminal.printError("Failed to delete plugin file %s %s", pluginFile, ex);
}
}
if (FileSystemUtils.hasExtensions(extractLocation, ".java")) {
terminal.printError("Plugin installation assumed to be site plugin, but contains source code, aborting installation...");
try {
IOUtils.rm(extractLocation);
} catch(Exception ex) {
terminal.printError("Failed to remove site plugin from path %s - %s", extractLocation, ex.getMessage());
}
throw new IllegalArgumentException("Plugin installation assumed to be site plugin, but contains source code, aborting installation.");
}
// It could potentially be a non explicit _site plugin
boolean potentialSitePlugin = true;
// take care of bin/ by moving and applying permissions if needed
Path binFile = extractLocation.resolve("bin");
if (Files.isDirectory(binFile)) {
Path toLocation = pluginHandle.binDir(environment);
@ -289,8 +239,7 @@ public class PluginManager {
} else {
terminal.println(VERBOSE, "Skipping posix permissions - filestore doesn't support posix permission");
}
terminal.println(VERBOSE, "Installed %s into %s", name, toLocation.toAbsolutePath());
potentialSitePlugin = false;
terminal.println(VERBOSE, "Installed %s into %s", pluginHandle.name, toLocation.toAbsolutePath());
}
Path configFile = extractLocation.resolve("config");
@ -298,33 +247,68 @@ public class PluginManager {
Path configDestLocation = pluginHandle.configDir(environment);
terminal.println(VERBOSE, "Found config, moving to %s", configDestLocation.toAbsolutePath());
moveFilesWithoutOverwriting(configFile, configDestLocation, ".new");
terminal.println(VERBOSE, "Installed %s into %s", name, configDestLocation.toAbsolutePath());
potentialSitePlugin = false;
}
// try and identify the plugin type, see if it has no .class or .jar files in it
// so its probably a _site, and it it does not have a _site in it, move everything to _site
if (!Files.exists(extractLocation.resolve("_site"))) {
if (potentialSitePlugin && !FileSystemUtils.hasExtensions(extractLocation, ".class", ".jar")) {
terminal.println(VERBOSE, "Identified as a _site plugin, moving to _site structure ...");
Path site = extractLocation.resolve("_site");
Path tmpLocation = environment.pluginsFile().resolve(extractLocation.getFileName() + ".tmp");
Files.move(extractLocation, tmpLocation);
Files.createDirectories(extractLocation);
Files.move(tmpLocation, site);
terminal.println(VERBOSE, "Installed " + name + " into " + site.toAbsolutePath());
}
terminal.println(VERBOSE, "Installed %s into %s", pluginHandle.name, configDestLocation.toAbsolutePath());
}
}
private Path unzipToTemporary(Path zip) throws IOException {
Path tmp = Files.createTempDirectory(environment.tmpFile(), null);
/** we check whether we need to remove the top-level folder while extracting
* sometimes (e.g. github) the downloaded archive contains a top-level folder which needs to be removed
*/
private Path findPluginRoot(Path dir) throws IOException {
if (Files.exists(dir.resolve(PluginInfo.ES_PLUGIN_PROPERTIES))) {
return dir;
} else {
final Path[] topLevelFiles = FileSystemUtils.files(dir);
if (topLevelFiles.length == 1 && Files.isDirectory(topLevelFiles[0])) {
Path subdir = topLevelFiles[0];
if (Files.exists(subdir.resolve(PluginInfo.ES_PLUGIN_PROPERTIES))) {
return subdir;
}
}
}
throw new RuntimeException("Could not find plugin descriptor '" + PluginInfo.ES_PLUGIN_PROPERTIES + "' in plugin zip");
}
/** check a candidate plugin for jar hell before installing it */
private void jarHellCheck(Path candidate, boolean isolated) throws IOException {
// create list of current jars in classpath
final List<URL> jars = new ArrayList<>();
ClassLoader loader = PluginManager.class.getClassLoader();
if (loader instanceof URLClassLoader) {
Collections.addAll(jars, ((URLClassLoader) loader).getURLs());
}
// read existing bundles. this does some checks on the installation too.
List<Bundle> bundles = PluginsService.getPluginBundles(environment);
// if we aren't isolated, we need to jarhellcheck against any other non-isolated plugins
// thats always the first bundle
if (isolated == false) {
jars.addAll(bundles.get(0).urls);
}
// add plugin jars to the list
Path pluginJars[] = FileSystemUtils.files(candidate, "*.jar");
for (Path jar : pluginJars) {
jars.add(jar.toUri().toURL());
}
// check combined (current classpath + new jars to-be-added)
try {
JarHell.checkJarHell(jars.toArray(new URL[jars.size()]));
} catch (Exception ex) {
throw new RuntimeException(ex);
}
}
private void unzipPlugin(Path zip, Path target) throws IOException {
Files.createDirectories(target);
try (ZipInputStream zipInput = new ZipInputStream(Files.newInputStream(zip))) {
ZipEntry entry;
byte[] buffer = new byte[8192];
while ((entry = zipInput.getNextEntry()) != null) {
Path targetFile = tmp.resolve(entry.getName());
Path targetFile = target.resolve(entry.getName());
// be on the safe side: do not rely on that directories are always extracted
// before their children (although this makes sense, but is it guaranteed?)
@ -340,8 +324,6 @@ public class PluginManager {
zipInput.closeEntry();
}
}
return tmp;
}
public void removePlugin(String name, Terminal terminal) throws IOException {
@ -363,17 +345,6 @@ public class PluginManager {
}
removed = true;
}
pluginToDelete = pluginHandle.distroFile(environment);
if (Files.exists(pluginToDelete)) {
terminal.println(VERBOSE, "Removing: %s", pluginToDelete);
try {
Files.delete(pluginToDelete);
} catch (Exception ex) {
throw new IOException("Unable to remove " + pluginHandle.name + ". Check file permissions on " +
pluginToDelete.toString(), ex);
}
removed = true;
}
Path binLocation = pluginHandle.binDir(environment);
if (Files.exists(binLocation)) {
terminal.println(VERBOSE, "Removing: %s", binLocation);
@ -482,8 +453,8 @@ public class PluginManager {
}
}
Path distroFile(Environment env) {
return env.pluginsFile().resolve(name + ".zip");
Path newDistroFile(Environment env) throws IOException {
return Files.createTempFile(env.tmpFile(), name, ".zip");
}
Path extractedDir(Environment env) {

View File

@ -60,7 +60,6 @@ public class RestAnalyzeAction extends BaseRestHandler {
AnalyzeRequest analyzeRequest = new AnalyzeRequest(request.param("index"));
analyzeRequest.text(texts);
analyzeRequest.preferLocal(request.paramAsBoolean("prefer_local", analyzeRequest.preferLocalShard()));
analyzeRequest.analyzer(request.param("analyzer"));
analyzeRequest.field(request.param("field"));
analyzeRequest.tokenizer(request.param("tokenizer"));
@ -93,8 +92,6 @@ public class RestAnalyzeAction extends BaseRestHandler {
while ((token = parser.nextToken()) != XContentParser.Token.END_OBJECT) {
if (token == XContentParser.Token.FIELD_NAME) {
currentFieldName = parser.currentName();
} else if ("prefer_local".equals(currentFieldName) && token == XContentParser.Token.VALUE_BOOLEAN) {
analyzeRequest.preferLocal(parser.booleanValue());
} else if ("text".equals(currentFieldName) && token == XContentParser.Token.VALUE_STRING) {
analyzeRequest.text(parser.text());
} else if ("text".equals(currentFieldName) && token == XContentParser.Token.START_ARRAY) {

View File

@ -50,7 +50,7 @@ public class ResourceWatcherService extends AbstractLifecycleComponent<ResourceW
/**
* Defaults to 30 seconds
*/
MEDIUM(TimeValue.timeValueSeconds(25)),
MEDIUM(TimeValue.timeValueSeconds(30)),
/**
* Defaults to 60 seconds

View File

@ -302,8 +302,8 @@ public class TermVectorsUnitTests extends ElasticsearchTestCase {
request = new MultiTermVectorsRequest();
request.add(new TermVectorsRequest(), bytes);
checkParsedParameters(request);
}
void checkParsedParameters(MultiTermVectorsRequest request) {
Set<String> ids = new HashSet<>();
ids.add("1");
@ -324,5 +324,31 @@ public class TermVectorsUnitTests extends ElasticsearchTestCase {
assertThat(singleRequest.selectedFields(), equalTo(fields));
}
}
@Test // issue #12311
public void testMultiParserFilter() throws Exception {
byte[] data = Streams.copyToBytesFromClasspath("/org/elasticsearch/action/termvectors/multiRequest3.json");
BytesReference bytes = new BytesArray(data);
MultiTermVectorsRequest request = new MultiTermVectorsRequest();
request.add(new TermVectorsRequest(), bytes);
checkParsedFilterParameters(request);
}
void checkParsedFilterParameters(MultiTermVectorsRequest multiRequest) {
int id = 1;
for (TermVectorsRequest request : multiRequest.requests) {
assertThat(request.index(), equalTo("testidx"));
assertThat(request.type(), equalTo("test"));
assertThat(request.id(), equalTo(id+""));
assertNotNull(request.filterSettings());
assertThat(request.filterSettings().maxNumTerms, equalTo(20));
assertThat(request.filterSettings().minTermFreq, equalTo(1));
assertThat(request.filterSettings().maxTermFreq, equalTo(20));
assertThat(request.filterSettings().minDocFreq, equalTo(1));
assertThat(request.filterSettings().maxDocFreq, equalTo(20));
assertThat(request.filterSettings().minWordLength, equalTo(1));
assertThat(request.filterSettings().maxWordLength, equalTo(20));
id++;
}
}
}

View File

@ -0,0 +1,16 @@
{
"ids": ["1","2"],
"parameters": {
"_index": "testidx",
"_type": "test",
"filter": {
"max_num_terms": 20,
"min_term_freq": 1,
"max_term_freq": 20,
"min_doc_freq": 1,
"max_doc_freq": 20,
"min_word_length": 1,
"max_word_length": 20
}
}
}

View File

@ -23,6 +23,7 @@ import org.elasticsearch.Version;
import org.elasticsearch.cluster.ClusterState;
import org.elasticsearch.cluster.metadata.IndexMetaData;
import org.elasticsearch.cluster.metadata.MetaData;
import org.elasticsearch.cluster.metadata.SnapshotId;
import org.elasticsearch.common.io.stream.BytesStreamOutput;
import org.elasticsearch.common.io.stream.StreamInput;
import org.elasticsearch.test.ElasticsearchTestCase;
@ -48,6 +49,177 @@ public class ShardRoutingTests extends ElasticsearchTestCase {
}
}
public void testIsSameAllocation() {
ShardRouting unassignedShard0 = TestShardRouting.newShardRouting("test", 0, null, false, ShardRoutingState.UNASSIGNED, 1);
ShardRouting unassignedShard1 = TestShardRouting.newShardRouting("test", 1, null, false, ShardRoutingState.UNASSIGNED, 1);
ShardRouting initializingShard0 = TestShardRouting.newShardRouting("test", 0, "1", randomBoolean(), ShardRoutingState.INITIALIZING, 1);
ShardRouting initializingShard1 = TestShardRouting.newShardRouting("test", 1, "1", randomBoolean(), ShardRoutingState.INITIALIZING, 1);
ShardRouting startedShard0 = new ShardRouting(initializingShard0);
startedShard0.moveToStarted();
ShardRouting startedShard1 = new ShardRouting(initializingShard1);
startedShard1.moveToStarted();
// test identity
assertTrue(initializingShard0.isSameAllocation(initializingShard0));
// test same allocation different state
assertTrue(initializingShard0.isSameAllocation(startedShard0));
// test unassigned is false even to itself
assertFalse(unassignedShard0.isSameAllocation(unassignedShard0));
// test different shards/nodes/state
assertFalse(unassignedShard0.isSameAllocation(unassignedShard1));
assertFalse(unassignedShard0.isSameAllocation(initializingShard0));
assertFalse(unassignedShard0.isSameAllocation(initializingShard1));
assertFalse(unassignedShard0.isSameAllocation(startedShard1));
}
public void testIsSameShard() {
ShardRouting index1Shard0a = randomShardRouting("index1", 0);
ShardRouting index1Shard0b = randomShardRouting("index1", 0);
ShardRouting index1Shard1 = randomShardRouting("index1", 1);
ShardRouting index2Shard0 = randomShardRouting("index2", 0);
ShardRouting index2Shard1 = randomShardRouting("index2", 1);
assertTrue(index1Shard0a.isSameShard(index1Shard0a));
assertTrue(index1Shard0a.isSameShard(index1Shard0b));
assertFalse(index1Shard0a.isSameShard(index1Shard1));
assertFalse(index1Shard0a.isSameShard(index2Shard0));
assertFalse(index1Shard0a.isSameShard(index2Shard1));
}
private ShardRouting randomShardRouting(String index, int shard) {
ShardRoutingState state = randomFrom(ShardRoutingState.values());
return TestShardRouting.newShardRouting(index, shard, state == ShardRoutingState.UNASSIGNED ? null : "1", state != ShardRoutingState.UNASSIGNED && randomBoolean(), state, randomInt(5));
}
public void testIsSourceTargetRelocation() {
ShardRouting unassignedShard0 = TestShardRouting.newShardRouting("test", 0, null, false, ShardRoutingState.UNASSIGNED, 1);
ShardRouting initializingShard0 = TestShardRouting.newShardRouting("test", 0, "node1", randomBoolean(), ShardRoutingState.INITIALIZING, 1);
ShardRouting initializingShard1 = TestShardRouting.newShardRouting("test", 1, "node1", randomBoolean(), ShardRoutingState.INITIALIZING, 1);
ShardRouting startedShard0 = new ShardRouting(initializingShard0);
startedShard0.moveToStarted();
ShardRouting startedShard1 = new ShardRouting(initializingShard1);
startedShard1.moveToStarted();
ShardRouting sourceShard0a = new ShardRouting(startedShard0);
sourceShard0a.relocate("node2");
ShardRouting targetShard0a = sourceShard0a.buildTargetRelocatingShard();
ShardRouting sourceShard0b = new ShardRouting(startedShard0);
sourceShard0b.relocate("node2");
ShardRouting sourceShard1 = new ShardRouting(startedShard1);
sourceShard1.relocate("node2");
// test true scenarios
assertTrue(targetShard0a.isRelocationTargetOf(sourceShard0a));
assertTrue(sourceShard0a.isRelocationSourceOf(targetShard0a));
// test two shards are not mixed
assertFalse(targetShard0a.isRelocationTargetOf(sourceShard1));
assertFalse(sourceShard1.isRelocationSourceOf(targetShard0a));
// test two allocations are not mixed
assertFalse(targetShard0a.isRelocationTargetOf(sourceShard0b));
assertFalse(sourceShard0b.isRelocationSourceOf(targetShard0a));
// test different shard states
assertFalse(targetShard0a.isRelocationTargetOf(unassignedShard0));
assertFalse(sourceShard0a.isRelocationTargetOf(unassignedShard0));
assertFalse(unassignedShard0.isRelocationSourceOf(targetShard0a));
assertFalse(unassignedShard0.isRelocationSourceOf(sourceShard0a));
assertFalse(targetShard0a.isRelocationTargetOf(initializingShard0));
assertFalse(sourceShard0a.isRelocationTargetOf(initializingShard0));
assertFalse(initializingShard0.isRelocationSourceOf(targetShard0a));
assertFalse(initializingShard0.isRelocationSourceOf(sourceShard0a));
assertFalse(targetShard0a.isRelocationTargetOf(startedShard0));
assertFalse(sourceShard0a.isRelocationTargetOf(startedShard0));
assertFalse(startedShard0.isRelocationSourceOf(targetShard0a));
assertFalse(startedShard0.isRelocationSourceOf(sourceShard0a));
}
public void testEqualsIgnoringVersion() {
ShardRouting routing = randomShardRouting("test", 0);
ShardRouting otherRouting = new ShardRouting(routing);
assertTrue("expected equality\nthis " + routing + ",\nother " + otherRouting, routing.equalsIgnoringMetaData(otherRouting));
otherRouting = new ShardRouting(routing, 1);
assertTrue("expected equality\nthis " + routing + ",\nother " + otherRouting, routing.equalsIgnoringMetaData(otherRouting));
otherRouting = new ShardRouting(routing);
Integer[] changeIds = new Integer[]{0, 1, 2, 3, 4, 5, 6};
for (int changeId : randomSubsetOf(randomIntBetween(1, changeIds.length), changeIds)) {
switch (changeId) {
case 0:
// change index
otherRouting = TestShardRouting.newShardRouting(otherRouting.index() + "a", otherRouting.id(), otherRouting.currentNodeId(), otherRouting.relocatingNodeId(),
otherRouting.restoreSource(), otherRouting.primary(), otherRouting.state(), otherRouting.version(), otherRouting.unassignedInfo());
break;
case 1:
// change shard id
otherRouting = TestShardRouting.newShardRouting(otherRouting.index(), otherRouting.id() + 1, otherRouting.currentNodeId(), otherRouting.relocatingNodeId(),
otherRouting.restoreSource(), otherRouting.primary(), otherRouting.state(), otherRouting.version(), otherRouting.unassignedInfo());
break;
case 2:
// change current node
otherRouting = TestShardRouting.newShardRouting(otherRouting.index(), otherRouting.id(), otherRouting.currentNodeId() == null ? "1" : otherRouting.currentNodeId() + "_1", otherRouting.relocatingNodeId(),
otherRouting.restoreSource(), otherRouting.primary(), otherRouting.state(), otherRouting.version(), otherRouting.unassignedInfo());
break;
case 3:
// change relocating node
otherRouting = TestShardRouting.newShardRouting(otherRouting.index(), otherRouting.id(), otherRouting.currentNodeId(),
otherRouting.relocatingNodeId() == null ? "1" : otherRouting.relocatingNodeId() + "_1",
otherRouting.restoreSource(), otherRouting.primary(), otherRouting.state(), otherRouting.version(), otherRouting.unassignedInfo());
break;
case 4:
// change restore source
otherRouting = TestShardRouting.newShardRouting(otherRouting.index(), otherRouting.id(), otherRouting.currentNodeId(), otherRouting.relocatingNodeId(),
otherRouting.restoreSource() == null ? new RestoreSource(new SnapshotId("test", "s1"), Version.CURRENT, "test") :
new RestoreSource(otherRouting.restoreSource().snapshotId(), Version.CURRENT, otherRouting.index() + "_1"),
otherRouting.primary(), otherRouting.state(), otherRouting.version(), otherRouting.unassignedInfo());
break;
case 5:
// change primary flag
otherRouting = TestShardRouting.newShardRouting(otherRouting.index(), otherRouting.id(), otherRouting.currentNodeId(), otherRouting.relocatingNodeId(),
otherRouting.restoreSource(), otherRouting.primary() == false, otherRouting.state(), otherRouting.version(), otherRouting.unassignedInfo());
break;
case 6:
// change state
ShardRoutingState newState;
do {
newState = randomFrom(ShardRoutingState.values());
} while (newState == otherRouting.state());
UnassignedInfo unassignedInfo = otherRouting.unassignedInfo();
if (unassignedInfo == null && (newState == ShardRoutingState.UNASSIGNED || newState == ShardRoutingState.INITIALIZING)) {
unassignedInfo = new UnassignedInfo(UnassignedInfo.Reason.INDEX_CREATED, "test");
}
otherRouting = TestShardRouting.newShardRouting(otherRouting.index(), otherRouting.id(), otherRouting.currentNodeId(), otherRouting.relocatingNodeId(),
otherRouting.restoreSource(), otherRouting.primary(), newState, otherRouting.version(), unassignedInfo);
break;
}
if (randomBoolean()) {
// change version
otherRouting = new ShardRouting(otherRouting, otherRouting.version() + 1);
}
if (randomBoolean()) {
// change unassigned info
otherRouting = TestShardRouting.newShardRouting(otherRouting.index(), otherRouting.id(), otherRouting.currentNodeId(), otherRouting.relocatingNodeId(),
otherRouting.restoreSource(), otherRouting.primary(), otherRouting.state(), otherRouting.version(),
otherRouting.unassignedInfo() == null ? new UnassignedInfo(UnassignedInfo.Reason.INDEX_CREATED, "test") :
new UnassignedInfo(UnassignedInfo.Reason.INDEX_CREATED, otherRouting.unassignedInfo().getMessage() + "_1"));
}
logger.debug("comparing\nthis {} to\nother {}", routing, otherRouting);
assertFalse("expected non-equality\nthis " + routing + ",\nother " + otherRouting, routing.equalsIgnoringMetaData(otherRouting));
}
}
public void testFrozenOnRoutingTable() {
MetaData metaData = MetaData.builder()

View File

@ -19,6 +19,8 @@
package org.elasticsearch.cluster.routing;
import org.elasticsearch.test.ElasticsearchTestCase;
/**
* A helper that allows to create shard routing instances within tests, while not requiring to expose
* different simplified constructors on the ShardRouting itself.
@ -26,19 +28,19 @@ package org.elasticsearch.cluster.routing;
public class TestShardRouting {
public static ShardRouting newShardRouting(String index, int shardId, String currentNodeId, boolean primary, ShardRoutingState state, long version) {
return new ShardRouting(index, shardId, currentNodeId, null, null, primary, state, version, null, buildAllocationId(state), true);
return new ShardRouting(index, shardId, currentNodeId, null, null, primary, state, version, buildUnassignedInfo(state), buildAllocationId(state), true);
}
public static ShardRouting newShardRouting(String index, int shardId, String currentNodeId, String relocatingNodeId, boolean primary, ShardRoutingState state, long version) {
return new ShardRouting(index, shardId, currentNodeId, relocatingNodeId, null, primary, state, version, null, buildAllocationId(state), true);
return new ShardRouting(index, shardId, currentNodeId, relocatingNodeId, null, primary, state, version, buildUnassignedInfo(state), buildAllocationId(state), true);
}
public static ShardRouting newShardRouting(String index, int shardId, String currentNodeId, String relocatingNodeId, boolean primary, ShardRoutingState state, AllocationId allocationId, long version) {
return new ShardRouting(index, shardId, currentNodeId, relocatingNodeId, null, primary, state, version, null, allocationId, true);
return new ShardRouting(index, shardId, currentNodeId, relocatingNodeId, null, primary, state, version, buildUnassignedInfo(state), allocationId, true);
}
public static ShardRouting newShardRouting(String index, int shardId, String currentNodeId, String relocatingNodeId, RestoreSource restoreSource, boolean primary, ShardRoutingState state, long version) {
return new ShardRouting(index, shardId, currentNodeId, relocatingNodeId, restoreSource, primary, state, version, null, buildAllocationId(state), true);
return new ShardRouting(index, shardId, currentNodeId, relocatingNodeId, restoreSource, primary, state, version, buildUnassignedInfo(state), buildAllocationId(state), true);
}
public static ShardRouting newShardRouting(String index, int shardId, String currentNodeId,
@ -61,4 +63,17 @@ public class TestShardRouting {
throw new IllegalStateException("illegal state");
}
}
private static UnassignedInfo buildUnassignedInfo(ShardRoutingState state) {
switch (state) {
case UNASSIGNED:
case INITIALIZING:
return new UnassignedInfo(ElasticsearchTestCase.randomFrom(UnassignedInfo.Reason.values()), "auto generated for test");
case STARTED:
case RELOCATING:
return null;
default:
throw new IllegalStateException("illegal state");
}
}
}

View File

@ -60,7 +60,8 @@ public class UnassignedInfoTests extends ElasticsearchAllocationTestCase {
UnassignedInfo.Reason.ALLOCATION_FAILED,
UnassignedInfo.Reason.NODE_LEFT,
UnassignedInfo.Reason.REROUTE_CANCELLED,
UnassignedInfo.Reason.REINITIALIZED};
UnassignedInfo.Reason.REINITIALIZED,
UnassignedInfo.Reason.REALLOCATED_REPLICA};
for (int i = 0; i < order.length; i++) {
assertThat(order[i].ordinal(), equalTo(i));
}

View File

@ -219,6 +219,39 @@ public class ReplicaShardAllocatorTests extends ElasticsearchAllocationTestCase
assertThat(allocation.routingNodes().shardsWithState(ShardRoutingState.INITIALIZING).get(0).currentNodeId(), equalTo(node2.id()));
}
@Test
public void testCancelRecoveryBetterSyncId() {
RoutingAllocation allocation = onePrimaryOnNode1And1ReplicaRecovering(yesAllocationDeciders());
testAllocator.addData(node1, true, "MATCH", new StoreFileMetaData("file1", 10, "MATCH_CHECKSUM"))
.addData(node2, false, "NO_MATCH", new StoreFileMetaData("file1", 10, "MATCH_CHECKSUM"))
.addData(node3, false, "MATCH", new StoreFileMetaData("file1", 10, "MATCH_CHECKSUM"));
boolean changed = testAllocator.processExistingRecoveries(allocation);
assertThat(changed, equalTo(true));
assertThat(allocation.routingNodes().shardsWithState(ShardRoutingState.UNASSIGNED).size(), equalTo(1));
assertThat(allocation.routingNodes().shardsWithState(ShardRoutingState.UNASSIGNED).get(0).shardId(), equalTo(shardId));
}
@Test
public void testNotCancellingRecoveryIfSyncedOnExistingRecovery() {
RoutingAllocation allocation = onePrimaryOnNode1And1ReplicaRecovering(yesAllocationDeciders());
testAllocator.addData(node1, true, "MATCH", new StoreFileMetaData("file1", 10, "MATCH_CHECKSUM"))
.addData(node2, false, "MATCH", new StoreFileMetaData("file1", 10, "MATCH_CHECKSUM"))
.addData(node3, false, randomBoolean() ? "MATCH" : "NO_MATCH", new StoreFileMetaData("file1", 10, "MATCH_CHECKSUM"));
boolean changed = testAllocator.processExistingRecoveries(allocation);
assertThat(changed, equalTo(false));
assertThat(allocation.routingNodes().shardsWithState(ShardRoutingState.UNASSIGNED).size(), equalTo(0));
}
@Test
public void testNotCancellingRecovery() {
RoutingAllocation allocation = onePrimaryOnNode1And1ReplicaRecovering(yesAllocationDeciders());
testAllocator.addData(node1, true, "MATCH", new StoreFileMetaData("file1", 10, "MATCH_CHECKSUM"))
.addData(node2, false, "MATCH", new StoreFileMetaData("file1", 10, "MATCH_CHECKSUM"));
boolean changed = testAllocator.processExistingRecoveries(allocation);
assertThat(changed, equalTo(false));
assertThat(allocation.routingNodes().shardsWithState(ShardRoutingState.UNASSIGNED).size(), equalTo(0));
}
private RoutingAllocation onePrimaryOnNode1And1Replica(AllocationDeciders deciders) {
return onePrimaryOnNode1And1Replica(deciders, Settings.EMPTY, UnassignedInfo.Reason.INDEX_CREATED);
}
@ -242,6 +275,25 @@ public class ReplicaShardAllocatorTests extends ElasticsearchAllocationTestCase
return new RoutingAllocation(deciders, state.routingNodes(), state.nodes(), null);
}
private RoutingAllocation onePrimaryOnNode1And1ReplicaRecovering(AllocationDeciders deciders) {
MetaData metaData = MetaData.builder()
.put(IndexMetaData.builder(shardId.getIndex()).settings(settings(Version.CURRENT)).numberOfShards(1).numberOfReplicas(0))
.build();
RoutingTable routingTable = RoutingTable.builder()
.add(IndexRoutingTable.builder(shardId.getIndex())
.addIndexShard(new IndexShardRoutingTable.Builder(shardId)
.addShard(TestShardRouting.newShardRouting(shardId.getIndex(), shardId.getId(), node1.id(), true, ShardRoutingState.STARTED, 10))
.addShard(TestShardRouting.newShardRouting(shardId.getIndex(), shardId.getId(), node2.id(), false, ShardRoutingState.INITIALIZING, 10))
.build())
)
.build();
ClusterState state = ClusterState.builder(org.elasticsearch.cluster.ClusterName.DEFAULT)
.metaData(metaData)
.routingTable(routingTable)
.nodes(DiscoveryNodes.builder().put(node1).put(node2).put(node3)).build();
return new RoutingAllocation(deciders, state.routingNodes(), state.nodes(), null);
}
class TestAllocator extends ReplicaShardAllocator {
private Map<DiscoveryNode, TransportNodesListShardStoreMetaData.StoreFilesMetaData> data = null;

View File

@ -236,15 +236,15 @@ public class InternalEngineTests extends ElasticsearchTestCase {
}
protected InternalEngine createEngine(Store store, Path translogPath) {
return createEngine(defaultSettings, store, translogPath, new MergeSchedulerConfig(defaultSettings), newMergePolicy());
protected InternalEngine createEngine(Store store, Path translogPath, IndexSearcherWrapper... wrappers) {
return createEngine(defaultSettings, store, translogPath, new MergeSchedulerConfig(defaultSettings), newMergePolicy(), wrappers);
}
protected InternalEngine createEngine(Settings indexSettings, Store store, Path translogPath, MergeSchedulerConfig mergeSchedulerConfig, MergePolicy mergePolicy) {
return new InternalEngine(config(indexSettings, store, translogPath, mergeSchedulerConfig, mergePolicy), false);
protected InternalEngine createEngine(Settings indexSettings, Store store, Path translogPath, MergeSchedulerConfig mergeSchedulerConfig, MergePolicy mergePolicy, IndexSearcherWrapper... wrappers) {
return new InternalEngine(config(indexSettings, store, translogPath, mergeSchedulerConfig, mergePolicy, wrappers), false);
}
public EngineConfig config(Settings indexSettings, Store store, Path translogPath, MergeSchedulerConfig mergeSchedulerConfig, MergePolicy mergePolicy) {
public EngineConfig config(Settings indexSettings, Store store, Path translogPath, MergeSchedulerConfig mergeSchedulerConfig, MergePolicy mergePolicy, IndexSearcherWrapper... wrappers) {
IndexWriterConfig iwc = newIndexWriterConfig();
TranslogConfig translogConfig = new TranslogConfig(shardId, translogPath, indexSettings, Translog.Durabilty.REQUEST, BigArrays.NON_RECYCLING_INSTANCE, threadPool);
@ -255,7 +255,7 @@ public class InternalEngineTests extends ElasticsearchTestCase {
public void onFailedEngine(ShardId shardId, String reason, @Nullable Throwable t) {
// we don't need to notify anybody in this test
}
}, new TranslogHandler(shardId.index().getName()), IndexSearcher.getDefaultQueryCache(), IndexSearcher.getDefaultQueryCachingPolicy(), translogConfig);
}, new TranslogHandler(shardId.index().getName()), IndexSearcher.getDefaultQueryCache(), IndexSearcher.getDefaultQueryCachingPolicy(), new IndexSearcherWrappingService(new HashSet<>(Arrays.asList(wrappers))), translogConfig);
return config;
}
@ -493,6 +493,32 @@ public class InternalEngineTests extends ElasticsearchTestCase {
;
}
@Test
public void testIndexSearcherWrapper() throws Exception {
final AtomicInteger counter = new AtomicInteger();
IndexSearcherWrapper wrapper = new IndexSearcherWrapper() {
@Override
public DirectoryReader wrap(DirectoryReader reader) {
counter.incrementAndGet();
return reader;
}
@Override
public IndexSearcher wrap(IndexSearcher searcher) throws EngineException {
counter.incrementAndGet();
return searcher;
}
};
Store store = createStore();
Path translog = createTempDir("translog-test");
InternalEngine engine = createEngine(store, translog, wrapper);
Engine.Searcher searcher = engine.acquireSearcher("test");
assertThat(counter.get(), equalTo(2));
searcher.close();
IOUtils.close(store, engine);
}
@Test
public void testSimpleOperations() throws Exception {
Engine.Searcher searchResult = engine.acquireSearcher("test");
@ -1985,7 +2011,7 @@ public class InternalEngineTests extends ElasticsearchTestCase {
EngineConfig brokenConfig = new EngineConfig(shardId, threadPool, config.getIndexingService(), config.getIndexSettings()
, null, store, createSnapshotDeletionPolicy(), newMergePolicy(), config.getMergeSchedulerConfig(),
config.getAnalyzer(), config.getSimilarity(), new CodecService(shardId.index()), config.getFailedEngineListener()
, config.getTranslogRecoveryPerformer(), IndexSearcher.getDefaultQueryCache(), IndexSearcher.getDefaultQueryCachingPolicy(), translogConfig);
, config.getTranslogRecoveryPerformer(), IndexSearcher.getDefaultQueryCache(), IndexSearcher.getDefaultQueryCachingPolicy(), new IndexSearcherWrappingService(), translogConfig);
try {
new InternalEngine(brokenConfig, false);

View File

@ -47,7 +47,6 @@ import org.elasticsearch.index.mapper.ParseContext;
import org.elasticsearch.index.mapper.ParsedDocument;
import org.elasticsearch.index.mapper.internal.SourceFieldMapper;
import org.elasticsearch.index.mapper.internal.UidFieldMapper;
import org.elasticsearch.index.settings.IndexSettingsService;
import org.elasticsearch.index.shard.MergeSchedulerConfig;
import org.elasticsearch.index.shard.ShardId;
import org.elasticsearch.index.shard.ShardUtils;
@ -226,7 +225,7 @@ public class ShadowEngineTests extends ElasticsearchTestCase {
@Override
public void onFailedEngine(ShardId shardId, String reason, @Nullable Throwable t) {
// we don't need to notify anybody in this test
}}, null, IndexSearcher.getDefaultQueryCache(), IndexSearcher.getDefaultQueryCachingPolicy(), translogConfig);
}}, null, IndexSearcher.getDefaultQueryCache(), IndexSearcher.getDefaultQueryCachingPolicy(), new IndexSearcherWrappingService(), translogConfig);
return config;
}

View File

@ -19,21 +19,21 @@
package org.elasticsearch.index.fielddata;
import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.index.cache.bitset.BitsetFilterCache;
import org.apache.lucene.analysis.standard.StandardAnalyzer;
import org.apache.lucene.document.Document;
import org.apache.lucene.document.Field;
import org.apache.lucene.document.StringField;
import org.apache.lucene.index.*;
import org.apache.lucene.search.Filter;
import org.apache.lucene.store.RAMDirectory;
import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.index.IndexService;
import org.elasticsearch.index.cache.bitset.BitsetFilterCache;
import org.elasticsearch.index.mapper.ContentPath;
import org.elasticsearch.index.mapper.FieldMapper;
import org.elasticsearch.index.mapper.MappedFieldType;
import org.elasticsearch.index.mapper.Mapper.BuilderContext;
import org.elasticsearch.index.mapper.MapperBuilders;
import org.elasticsearch.index.mapper.MapperService;
import org.elasticsearch.index.IndexService;
import org.elasticsearch.index.mapper.internal.ParentFieldMapper;
import org.elasticsearch.indices.fielddata.cache.IndicesFieldDataCache;
import org.elasticsearch.test.ElasticsearchSingleNodeTest;
@ -41,6 +41,9 @@ import org.junit.After;
import org.junit.Before;
import static org.elasticsearch.index.fielddata.IndexFieldData.XFieldComparatorSource.Nested;
import static org.hamcrest.Matchers.equalTo;
import static org.hamcrest.Matchers.not;
import static org.hamcrest.Matchers.sameInstance;
public abstract class AbstractFieldDataTests extends ElasticsearchSingleNodeTest {
@ -130,4 +133,25 @@ public abstract class AbstractFieldDataTests extends ElasticsearchSingleNodeTest
return new Nested(s.getBitDocIdSetFilter(parentFilter), s.getBitDocIdSetFilter(childFilter));
}
public void testEmpty() throws Exception {
Document d = new Document();
d.add(new StringField("field", "value", Field.Store.NO));
writer.addDocument(d);
refreshReader();
IndexFieldData fieldData = getForField("non_existing_field");
int max = randomInt(7);
AtomicFieldData previous = null;
for (int i = 0; i < max; i++) {
AtomicFieldData current = fieldData.load(readerContext);
assertThat(current.ramBytesUsed(), equalTo(0l));
if (previous != null) {
assertThat(current, not(sameInstance(previous)));
}
previous = current;
}
}
}

View File

@ -465,6 +465,11 @@ public class DuelFieldDataTests extends AbstractFieldDataTests {
}
}
@Override
public void testEmpty() throws Exception {
// No need to test empty usage here
}
private int[] getNumbers(Random random, int margin) {
if (random.nextInt(20) == 0) {
int[] num = new int[1 + random.nextInt(10)];

View File

@ -180,5 +180,8 @@ public class FilterFieldDataTest extends AbstractFieldDataTests {
}
@Override
public void testEmpty() throws Exception {
// No need to test empty usage here
}
}

View File

@ -61,7 +61,7 @@ import static org.elasticsearch.index.mapper.string.SimpleStringMappingTests.doc
import static org.hamcrest.Matchers.*;
public class SimpleDateMappingTests extends ElasticsearchSingleNodeTest {
public void testAutomaticDateParser() throws Exception {
String mapping = XContentFactory.jsonBuilder().startObject().startObject("type")
.startObject("properties").endObject()
@ -93,12 +93,12 @@ public class SimpleDateMappingTests extends ElasticsearchSingleNodeTest {
fieldMapper = defaultMapper.mappers().smartNameFieldMapper("wrong_date3");
assertThat(fieldMapper, instanceOf(StringFieldMapper.class));
}
public void testParseLocal() {
assertThat(Locale.GERMAN, equalTo(LocaleUtils.parse("de")));
assertThat(Locale.GERMANY, equalTo(LocaleUtils.parse("de_DE")));
assertThat(new Locale("de","DE","DE"), equalTo(LocaleUtils.parse("de_DE_DE")));
try {
LocaleUtils.parse("de_DE_DE_DE");
fail();
@ -108,7 +108,7 @@ public class SimpleDateMappingTests extends ElasticsearchSingleNodeTest {
assertThat(Locale.ROOT, equalTo(LocaleUtils.parse("")));
assertThat(Locale.ROOT, equalTo(LocaleUtils.parse("ROOT")));
}
public void testLocale() throws IOException {
assumeFalse("Locals are buggy on JDK9EA", Constants.JRE_IS_MINIMUM_JAVA9 && systemPropertyAsBoolean("tests.security.manager", false));
String mapping = XContentFactory.jsonBuilder()
@ -169,7 +169,7 @@ public class SimpleDateMappingTests extends ElasticsearchSingleNodeTest {
private void assertNumericTokensEqual(ParsedDocument doc, DocumentMapper defaultMapper, String fieldA, String fieldB) throws IOException {
assertThat(doc.rootDoc().getField(fieldA).tokenStream(defaultMapper.mappers().indexAnalyzer(), null), notNullValue());
assertThat(doc.rootDoc().getField(fieldB).tokenStream(defaultMapper.mappers().indexAnalyzer(), null), notNullValue());
TokenStream tokenStream = doc.rootDoc().getField(fieldA).tokenStream(defaultMapper.mappers().indexAnalyzer(), null);
tokenStream.reset();
NumericTermAttribute nta = tokenStream.addAttribute(NumericTermAttribute.class);
@ -177,7 +177,7 @@ public class SimpleDateMappingTests extends ElasticsearchSingleNodeTest {
while(tokenStream.incrementToken()) {
values.add(nta.getRawValue());
}
tokenStream = doc.rootDoc().getField(fieldB).tokenStream(defaultMapper.mappers().indexAnalyzer(), null);
tokenStream.reset();
nta = tokenStream.addAttribute(NumericTermAttribute.class);
@ -223,7 +223,7 @@ public class SimpleDateMappingTests extends ElasticsearchSingleNodeTest {
assertThat(doc.rootDoc().get("date_field"), equalTo("1262304000000"));
assertThat(doc.rootDoc().get("date_field_x"), equalTo("2010-01-01"));
}
public void testHourFormat() throws Exception {
String mapping = XContentFactory.jsonBuilder().startObject().startObject("type")
.field("date_detection", false)
@ -242,14 +242,14 @@ public class SimpleDateMappingTests extends ElasticsearchSingleNodeTest {
NumericRangeQuery<Long> rangeQuery;
try {
SearchContext.setCurrent(new TestSearchContext());
rangeQuery = (NumericRangeQuery<Long>) defaultMapper.mappers().smartNameFieldMapper("date_field").fieldType().rangeQuery("10:00:00", "11:00:00", true, true, null).rewrite(null);
rangeQuery = (NumericRangeQuery<Long>) defaultMapper.mappers().smartNameFieldMapper("date_field").fieldType().rangeQuery("10:00:00", "11:00:00", true, true).rewrite(null);
} finally {
SearchContext.removeCurrent();
}
assertThat(rangeQuery.getMax(), equalTo(new DateTime(TimeValue.timeValueHours(11).millis(), DateTimeZone.UTC).getMillis()));
assertThat(rangeQuery.getMin(), equalTo(new DateTime(TimeValue.timeValueHours(10).millis(), DateTimeZone.UTC).getMillis()));
}
public void testDayWithoutYearFormat() throws Exception {
String mapping = XContentFactory.jsonBuilder().startObject().startObject("type")
.field("date_detection", false)
@ -268,14 +268,14 @@ public class SimpleDateMappingTests extends ElasticsearchSingleNodeTest {
NumericRangeQuery<Long> rangeQuery;
try {
SearchContext.setCurrent(new TestSearchContext());
rangeQuery = (NumericRangeQuery<Long>) defaultMapper.mappers().smartNameFieldMapper("date_field").fieldType().rangeQuery("Jan 02 10:00:00", "Jan 02 11:00:00", true, true, null).rewrite(null);
rangeQuery = (NumericRangeQuery<Long>) defaultMapper.mappers().smartNameFieldMapper("date_field").fieldType().rangeQuery("Jan 02 10:00:00", "Jan 02 11:00:00", true, true).rewrite(null);
} finally {
SearchContext.removeCurrent();
}
assertThat(rangeQuery.getMax(), equalTo(new DateTime(TimeValue.timeValueHours(35).millis(), DateTimeZone.UTC).getMillis()));
assertThat(rangeQuery.getMin(), equalTo(new DateTime(TimeValue.timeValueHours(34).millis(), DateTimeZone.UTC).getMillis()));
}
public void testIgnoreMalformedOption() throws Exception {
String mapping = XContentFactory.jsonBuilder().startObject().startObject("type")
.startObject("properties")
@ -377,7 +377,7 @@ public class SimpleDateMappingTests extends ElasticsearchSingleNodeTest {
Map<String, String> mergedConfig = getConfigurationViaXContent(mergedFieldMapper);
assertThat(mergedConfig.get("format"), is("EEE MMM dd HH:mm:ss.S Z yyyy||EEE MMM dd HH:mm:ss.SSS Z yyyy||yyyy-MM-dd'T'HH:mm:ss.SSSZZ"));
}
public void testDefaultDocValues() throws Exception {
String mapping = XContentFactory.jsonBuilder().startObject().startObject("type")
.startObject("properties").startObject("date_field").field("type", "date").endObject().endObject()

View File

@ -58,7 +58,7 @@ public class ExistsQueryBuilderTest extends BaseQueryTestCase<ExistsQueryBuilder
if (fieldType == null) {
boolFilter.add(new TermRangeQuery(field, null, null, true, true), BooleanClause.Occur.SHOULD);
} else {
boolFilter.add(fieldType.rangeQuery(null, null, true, true, context), BooleanClause.Occur.SHOULD);
boolFilter.add(fieldType.rangeQuery(null, null, true, true), BooleanClause.Occur.SHOULD);
}
}
}

View File

@ -72,7 +72,7 @@ public class MissingQueryBuilderTest extends BaseQueryTestCase<MissingQueryBuild
final boolean existence = queryBuilder.existence();
final boolean nullValue = queryBuilder.nullValue();
String fieldPattern = queryBuilder.fieldPattern();
if (!existence && !nullValue) {
throw new QueryParsingException(context, "missing must have either existence, or null_value, or both set to true");
}
@ -117,7 +117,7 @@ public class MissingQueryBuilderTest extends BaseQueryTestCase<MissingQueryBuild
}
// if _field_names are not indexed, we need to go the slow way
if (filter == null && fieldType != null) {
filter = fieldType.rangeQuery(null, null, true, true, context);
filter = fieldType.rangeQuery(null, null, true, true);
}
if (filter == null) {
filter = new TermRangeQuery(field, null, null, true, true);
@ -174,7 +174,7 @@ public class MissingQueryBuilderTest extends BaseQueryTestCase<MissingQueryBuild
missingQueryBuilder = new MissingQueryBuilder("field");
assertNull(missingQueryBuilder.validate());
}
@Test(expected = QueryParsingException.class)
public void testBothNullValueAndExistenceFalse() throws IOException {
QueryParseContext context = createContext();

View File

@ -111,7 +111,7 @@ public class RangeQueryBuilderTest extends BaseQueryTestCase<RangeQueryBuilder>
}
MappedFieldType mapper = context.fieldMapper(queryBuilder.fieldName());
expectedQuery = ((DateFieldMapper.DateFieldType) mapper).rangeQuery(BytesRefs.toBytesRef(queryBuilder.from()), BytesRefs.toBytesRef(queryBuilder.to()),
queryBuilder.includeLower(), queryBuilder.includeUpper(), dateTimeZone, forcedDateParser, context);
queryBuilder.includeLower(), queryBuilder.includeUpper(), dateTimeZone, forcedDateParser);
} else if (queryBuilder.fieldName().equals(INT_FIELD_NAME)) {
expectedQuery = NumericRangeQuery.newIntRange(INT_FIELD_NAME, (Integer) queryBuilder.from(), (Integer) queryBuilder.to(),
queryBuilder.includeLower(), queryBuilder.includeUpper());

View File

@ -205,7 +205,7 @@ public class IndexShardTests extends ElasticsearchSingleNodeTest {
ShardStateMetaData shardStateMetaData = load(logger, env.availableShardPaths(shard.shardId));
assertEquals(shardStateMetaData, getShardStateMetadata(shard));
routing = TestShardRouting.newShardRouting(shard.shardId.index().getName(), shard.shardId.id(), routing.currentNodeId(), null, null, routing.primary(), ShardRoutingState.INITIALIZING, shard.shardRouting.version() + 1);
routing = TestShardRouting.newShardRouting(shard.shardId.index().getName(), shard.shardId.id(), routing.currentNodeId(), null, routing.primary(), ShardRoutingState.INITIALIZING, shard.shardRouting.allocationId(), shard.shardRouting.version() + 1);
shard.updateRoutingEntry(routing, true);
shard.deleteShardState();

View File

@ -29,6 +29,8 @@ import org.junit.After;
import org.junit.Before;
import org.junit.Test;
import java.net.URL;
import java.net.URLClassLoader;
import java.util.ArrayList;
import java.util.List;
import java.util.concurrent.atomic.AtomicInteger;
@ -220,4 +222,19 @@ public class InternalSettingsPreparerTests extends ElasticsearchTestCase {
assertThat(settings.get("name"), is("prompted name 0"));
assertThat(settings.get("node.name"), is("prompted name 0"));
}
@Test
public void testPreserveSettingsClassloader() {
final ClassLoader classLoader = URLClassLoader.newInstance(new URL[0]);
Settings settings = settingsBuilder()
.put("foo", "bar")
.put("path.home", createTempDir())
.classLoader(classLoader)
.build();
Tuple<Settings, Environment> tuple = InternalSettingsPreparer.prepareSettings(settings, randomBoolean());
Settings preparedSettings = tuple.v1();
assertThat(preparedSettings.getClassLoaderIfSet(), is(classLoader));
}
}

View File

@ -38,7 +38,7 @@ import static org.hamcrest.Matchers.contains;
public class PluginInfoTests extends ElasticsearchTestCase {
void writeProperties(Path pluginDir, String... stringProps) throws IOException {
static void writeProperties(Path pluginDir, String... stringProps) throws IOException {
assert stringProps.length % 2 == 0;
Files.createDirectories(pluginDir);
Path propertiesFile = pluginDir.resolve(PluginInfo.ES_PLUGIN_PROPERTIES);
@ -57,6 +57,7 @@ public class PluginInfoTests extends ElasticsearchTestCase {
"description", "fake desc",
"version", "1.0",
"elasticsearch.version", Version.CURRENT.toString(),
"java.version", System.getProperty("java.specification.version"),
"jvm", "true",
"classname", "FakePlugin");
PluginInfo info = PluginInfo.readFromProperties(pluginDir);
@ -119,6 +120,38 @@ public class PluginInfoTests extends ElasticsearchTestCase {
}
}
public void testReadFromPropertiesJavaVersionMissing() throws Exception {
Path pluginDir = createTempDir().resolve("fake-plugin");
writeProperties(pluginDir,
"description", "fake desc",
"elasticsearch.version", Version.CURRENT.toString(),
"version", "1.0",
"jvm", "true");
try {
PluginInfo.readFromProperties(pluginDir);
fail("expected missing java version exception");
} catch (IllegalArgumentException e) {
assertTrue(e.getMessage().contains("[java.version] is missing"));
}
}
public void testReadFromPropertiesJavaVersionIncompatible() throws Exception {
Path pluginDir = createTempDir().resolve("fake-plugin");
writeProperties(pluginDir,
"description", "fake desc",
"elasticsearch.version", Version.CURRENT.toString(),
"java.version", "1000000.0",
"classname", "FakePlugin",
"version", "1.0",
"jvm", "true");
try {
PluginInfo.readFromProperties(pluginDir);
fail("expected incompatible java version exception");
} catch (IllegalStateException e) {
assertTrue(e.getMessage(), e.getMessage().contains("fake-plugin requires Java"));
}
}
public void testReadFromPropertiesBogusElasticsearchVersion() throws Exception {
Path pluginDir = createTempDir().resolve("fake-plugin");
writeProperties(pluginDir,
@ -155,6 +188,7 @@ public class PluginInfoTests extends ElasticsearchTestCase {
"description", "fake desc",
"version", "1.0",
"elasticsearch.version", Version.CURRENT.toString(),
"java.version", System.getProperty("java.specification.version"),
"jvm", "true");
try {
PluginInfo.readFromProperties(pluginDir);
@ -166,16 +200,30 @@ public class PluginInfoTests extends ElasticsearchTestCase {
public void testReadFromPropertiesSitePlugin() throws Exception {
Path pluginDir = createTempDir().resolve("fake-plugin");
Files.createDirectories(pluginDir.resolve("_site"));
writeProperties(pluginDir,
"description", "fake desc",
"version", "1.0",
"elasticsearch.version", Version.CURRENT.toString(),
"site", "true");
PluginInfo info = PluginInfo.readFromProperties(pluginDir);
assertTrue(info.isSite());
assertFalse(info.isJvm());
assertEquals("NA", info.getClassname());
}
public void testReadFromPropertiesSitePluginWithoutSite() throws Exception {
Path pluginDir = createTempDir().resolve("fake-plugin");
writeProperties(pluginDir,
"description", "fake desc",
"version", "1.0",
"site", "true");
try {
PluginInfo.readFromProperties(pluginDir);
fail("didn't get expected exception");
} catch (IllegalArgumentException e) {
assertTrue(e.getMessage().contains("site plugin but has no '_site"));
}
}
public void testPluginListSorted() {
PluginsInfo pluginsInfo = new PluginsInfo(5);

View File

@ -20,6 +20,7 @@ package org.elasticsearch.plugins;
import org.apache.http.impl.client.HttpClients;
import org.apache.lucene.util.LuceneTestCase;
import org.elasticsearch.Version;
import org.elasticsearch.common.cli.CliTool;
import org.elasticsearch.common.cli.CliToolTestCase.CaptureOutputTerminal;
import org.elasticsearch.common.collect.Tuple;
@ -36,13 +37,18 @@ import org.junit.Before;
import org.junit.Test;
import java.io.IOException;
import java.net.URI;
import java.nio.charset.StandardCharsets;
import java.nio.file.FileVisitResult;
import java.nio.file.Files;
import java.nio.file.Path;
import java.nio.file.SimpleFileVisitor;
import java.nio.file.attribute.BasicFileAttributes;
import java.nio.file.attribute.PosixFileAttributeView;
import java.nio.file.attribute.PosixFileAttributes;
import java.nio.file.attribute.PosixFilePermission;
import java.util.Locale;
import java.util.zip.ZipEntry;
import java.util.zip.ZipOutputStream;
import static org.elasticsearch.common.cli.CliTool.ExitStatus.USAGE;
import static org.elasticsearch.common.cli.CliToolTestCase.args;
@ -52,6 +58,7 @@ import static org.elasticsearch.test.ElasticsearchIntegrationTest.Scope;
import static org.elasticsearch.test.hamcrest.ElasticsearchAssertions.assertDirectoryExists;
import static org.elasticsearch.test.hamcrest.ElasticsearchAssertions.assertFileExists;
import static org.hamcrest.Matchers.*;
import static org.elasticsearch.plugins.PluginInfoTests.writeProperties;
@ClusterScope(scope = Scope.TEST, numDataNodes = 0, transportClientRatio = 0.0)
@LuceneTestCase.SuppressFileSystems("*") // TODO: clean up this test to allow extra files
@ -74,29 +81,65 @@ public class PluginManagerTests extends ElasticsearchIntegrationTest {
if (!Files.exists(configDir)) {
Files.createDirectories(configDir);
}
}
@After
public void clearPathHome() {
System.clearProperty("es.default.path.home");
}
/** creates a plugin .zip and returns the url for testing */
private String createPlugin(final Path structure, String... properties) throws IOException {
writeProperties(structure, properties);
Path zip = createTempDir().resolve(structure.getFileName() + ".zip");
try (ZipOutputStream stream = new ZipOutputStream(Files.newOutputStream(zip))) {
Files.walkFileTree(structure, new SimpleFileVisitor<Path>() {
@Override
public FileVisitResult visitFile(Path file, BasicFileAttributes attrs) throws IOException {
stream.putNextEntry(new ZipEntry(structure.relativize(file).toString()));
Files.copy(file, stream);
return FileVisitResult.CONTINUE;
}
});
}
return zip.toUri().toURL().toString();
}
@Test
public void testThatPluginNameMustBeSupplied() throws IOException {
String pluginUrl = getPluginUrlForResource("plugin_with_bin_and_config.zip");
Path pluginDir = createTempDir().resolve("fake-plugin");
String pluginUrl = createPlugin(pluginDir,
"description", "fake desc",
"version", "1.0",
"elasticsearch.version", Version.CURRENT.toString(),
"jvm", "true",
"classname", "FakePlugin");
assertStatus("install --url " + pluginUrl, USAGE);
}
@Test
public void testLocalPluginInstallWithBinAndConfig() throws Exception {
String pluginName = "plugin-test";
String pluginName = "fake-plugin";
Path pluginDir = createTempDir().resolve(pluginName);
// create bin/tool and config/file
Files.createDirectories(pluginDir.resolve("bin"));
Files.createFile(pluginDir.resolve("bin").resolve("tool"));
Files.createDirectories(pluginDir.resolve("config"));
Files.createFile(pluginDir.resolve("config").resolve("file"));
String pluginUrl = createPlugin(pluginDir,
"description", "fake desc",
"version", "1.0",
"elasticsearch.version", Version.CURRENT.toString(),
"java.version", System.getProperty("java.specification.version"),
"jvm", "true",
"classname", "FakePlugin");
Environment env = initialSettings.v2();
Path binDir = env.homeFile().resolve("bin");
Path pluginBinDir = binDir.resolve(pluginName);
Path pluginConfigDir = env.configFile().resolve(pluginName);
String pluginUrl = getPluginUrlForResource("plugin_with_bin_and_config.zip");
assertStatusOk("install " + pluginName + " --url " + pluginUrl + " --verbose");
terminal.getTerminalOutput().clear();
@ -123,23 +166,36 @@ public class PluginManagerTests extends ElasticsearchIntegrationTest {
*/
@Test
public void testLocalPluginInstallWithBinAndConfigInAlreadyExistingConfigDir_7890() throws Exception {
String pluginName = "plugin-test";
String pluginName = "fake-plugin";
Path pluginDir = createTempDir().resolve(pluginName);
// create config/test.txt with contents 'version1'
Files.createDirectories(pluginDir.resolve("config"));
Files.write(pluginDir.resolve("config").resolve("test.txt"), "version1".getBytes(StandardCharsets.UTF_8));
String pluginUrl = createPlugin(pluginDir,
"description", "fake desc",
"version", "1.0",
"elasticsearch.version", Version.CURRENT.toString(),
"java.version", System.getProperty("java.specification.version"),
"jvm", "true",
"classname", "FakePlugin");
Environment env = initialSettings.v2();
Path pluginConfigDir = env.configFile().resolve(pluginName);
assertStatusOk(String.format(Locale.ROOT, "install %s --url %s --verbose", pluginName, getPluginUrlForResource("plugin_with_config_v1.zip")));
assertStatusOk(String.format(Locale.ROOT, "install %s --url %s --verbose", pluginName, pluginUrl));
/*
First time, our plugin contains:
- config/test.txt (version1)
*/
assertFileContent(pluginConfigDir, "test.txt", "version1\n");
assertFileContent(pluginConfigDir, "test.txt", "version1");
// We now remove the plugin
assertStatusOk("remove " + pluginName);
// We should still have test.txt
assertFileContent(pluginConfigDir, "test.txt", "version1\n");
assertFileContent(pluginConfigDir, "test.txt", "version1");
// Installing a new plugin version
/*
@ -148,19 +204,31 @@ public class PluginManagerTests extends ElasticsearchIntegrationTest {
- config/dir/testdir.txt (version1)
- config/dir/subdir/testsubdir.txt (version1)
*/
assertStatusOk(String.format(Locale.ROOT, "install %s --url %s --verbose", pluginName, getPluginUrlForResource("plugin_with_config_v2.zip")));
Files.write(pluginDir.resolve("config").resolve("test.txt"), "version2".getBytes(StandardCharsets.UTF_8));
Files.createDirectories(pluginDir.resolve("config").resolve("dir").resolve("subdir"));
Files.write(pluginDir.resolve("config").resolve("dir").resolve("testdir.txt"), "version1".getBytes(StandardCharsets.UTF_8));
Files.write(pluginDir.resolve("config").resolve("dir").resolve("subdir").resolve("testsubdir.txt"), "version1".getBytes(StandardCharsets.UTF_8));
pluginUrl = createPlugin(pluginDir,
"description", "fake desc",
"version", "2.0",
"elasticsearch.version", Version.CURRENT.toString(),
"java.version", System.getProperty("java.specification.version"),
"jvm", "true",
"classname", "FakePlugin");
assertStatusOk(String.format(Locale.ROOT, "install %s --url %s --verbose", pluginName, pluginUrl));
assertFileContent(pluginConfigDir, "test.txt", "version1\n");
assertFileContent(pluginConfigDir, "test.txt.new", "version2\n");
assertFileContent(pluginConfigDir, "dir/testdir.txt", "version1\n");
assertFileContent(pluginConfigDir, "dir/subdir/testsubdir.txt", "version1\n");
assertFileContent(pluginConfigDir, "test.txt", "version1");
assertFileContent(pluginConfigDir, "test.txt.new", "version2");
assertFileContent(pluginConfigDir, "dir/testdir.txt", "version1");
assertFileContent(pluginConfigDir, "dir/subdir/testsubdir.txt", "version1");
// Removing
assertStatusOk("remove " + pluginName);
assertFileContent(pluginConfigDir, "test.txt", "version1\n");
assertFileContent(pluginConfigDir, "test.txt.new", "version2\n");
assertFileContent(pluginConfigDir, "dir/testdir.txt", "version1\n");
assertFileContent(pluginConfigDir, "dir/subdir/testsubdir.txt", "version1\n");
assertFileContent(pluginConfigDir, "test.txt", "version1");
assertFileContent(pluginConfigDir, "test.txt.new", "version2");
assertFileContent(pluginConfigDir, "dir/testdir.txt", "version1");
assertFileContent(pluginConfigDir, "dir/subdir/testsubdir.txt", "version1");
// Installing a new plugin version
/*
@ -171,40 +239,56 @@ public class PluginManagerTests extends ElasticsearchIntegrationTest {
- config/dir/testdir2.txt (version1)
- config/dir/subdir/testsubdir.txt (version2)
*/
assertStatusOk(String.format(Locale.ROOT, "install %s --url %s --verbose", pluginName, getPluginUrlForResource("plugin_with_config_v3.zip")));
Files.write(pluginDir.resolve("config").resolve("test.txt"), "version3".getBytes(StandardCharsets.UTF_8));
Files.write(pluginDir.resolve("config").resolve("test2.txt"), "version1".getBytes(StandardCharsets.UTF_8));
Files.write(pluginDir.resolve("config").resolve("dir").resolve("testdir.txt"), "version2".getBytes(StandardCharsets.UTF_8));
Files.write(pluginDir.resolve("config").resolve("dir").resolve("testdir2.txt"), "version1".getBytes(StandardCharsets.UTF_8));
Files.write(pluginDir.resolve("config").resolve("dir").resolve("subdir").resolve("testsubdir.txt"), "version2".getBytes(StandardCharsets.UTF_8));
pluginUrl = createPlugin(pluginDir,
"description", "fake desc",
"version", "3.0",
"elasticsearch.version", Version.CURRENT.toString(),
"java.version", System.getProperty("java.specification.version"),
"jvm", "true",
"classname", "FakePlugin");
assertFileContent(pluginConfigDir, "test.txt", "version1\n");
assertFileContent(pluginConfigDir, "test2.txt", "version1\n");
assertFileContent(pluginConfigDir, "test.txt.new", "version3\n");
assertFileContent(pluginConfigDir, "dir/testdir.txt", "version1\n");
assertFileContent(pluginConfigDir, "dir/testdir.txt.new", "version2\n");
assertFileContent(pluginConfigDir, "dir/testdir2.txt", "version1\n");
assertFileContent(pluginConfigDir, "dir/subdir/testsubdir.txt", "version1\n");
assertFileContent(pluginConfigDir, "dir/subdir/testsubdir.txt.new", "version2\n");
assertStatusOk(String.format(Locale.ROOT, "install %s --url %s --verbose", pluginName, pluginUrl));
assertFileContent(pluginConfigDir, "test.txt", "version1");
assertFileContent(pluginConfigDir, "test2.txt", "version1");
assertFileContent(pluginConfigDir, "test.txt.new", "version3");
assertFileContent(pluginConfigDir, "dir/testdir.txt", "version1");
assertFileContent(pluginConfigDir, "dir/testdir.txt.new", "version2");
assertFileContent(pluginConfigDir, "dir/testdir2.txt", "version1");
assertFileContent(pluginConfigDir, "dir/subdir/testsubdir.txt", "version1");
assertFileContent(pluginConfigDir, "dir/subdir/testsubdir.txt.new", "version2");
}
// For #7152
@Test
public void testLocalPluginInstallWithBinOnly_7152() throws Exception {
String pluginName = "plugin-test";
String pluginName = "fake-plugin";
Path pluginDir = createTempDir().resolve(pluginName);
// create bin/tool
Files.createDirectories(pluginDir.resolve("bin"));
Files.createFile(pluginDir.resolve("bin").resolve("tool"));;
String pluginUrl = createPlugin(pluginDir,
"description", "fake desc",
"version", "1.0",
"elasticsearch.version", Version.CURRENT.toString(),
"java.version", System.getProperty("java.specification.version"),
"jvm", "true",
"classname", "FakePlugin");
Environment env = initialSettings.v2();
Path binDir = env.homeFile().resolve("bin");
Path pluginBinDir = binDir.resolve(pluginName);
assertStatusOk(String.format(Locale.ROOT, "install %s --url %s --verbose", pluginName, getPluginUrlForResource("plugin_with_bin_only.zip")));
assertStatusOk(String.format(Locale.ROOT, "install %s --url %s --verbose", pluginName, pluginUrl));
assertThatPluginIsListed(pluginName);
assertDirectoryExists(pluginBinDir);
}
@Test
public void testSitePluginWithSourceDoesNotInstall() throws Exception {
String pluginName = "plugin-with-source";
String cmd = String.format(Locale.ROOT, "install %s --url %s --verbose", pluginName, getPluginUrlForResource("plugin_with_sourcefiles.zip"));
int status = new PluginManagerCliParser(terminal).execute(args(cmd));
assertThat(status, is(USAGE.status()));
assertThat(terminal.getTerminalOutput(), hasItem(containsString("Plugin installation assumed to be site plugin, but contains source code, aborting installation")));
}
@Test
public void testListInstalledEmpty() throws IOException {
assertStatusOk("list");
@ -220,18 +304,33 @@ public class PluginManagerTests extends ElasticsearchIntegrationTest {
@Test
public void testInstallPlugin() throws IOException {
String pluginName = "plugin-classfile";
assertStatusOk(String.format(Locale.ROOT, "install %s --url %s --verbose", pluginName, getPluginUrlForResource("plugin_with_classfile.zip")));
assertThatPluginIsListed("plugin-classfile");
String pluginName = "fake-plugin";
Path pluginDir = createTempDir().resolve(pluginName);
String pluginUrl = createPlugin(pluginDir,
"description", "fake desc",
"version", "1.0",
"elasticsearch.version", Version.CURRENT.toString(),
"java.version", System.getProperty("java.specification.version"),
"jvm", "true",
"classname", "FakePlugin");
assertStatusOk(String.format(Locale.ROOT, "install %s --url %s --verbose", pluginName, pluginUrl));
assertThatPluginIsListed(pluginName);
}
@Test
public void testInstallSitePlugin() throws IOException {
String pluginName = "plugin-site";
assertStatusOk(String.format(Locale.ROOT, "install %s --url %s --verbose", pluginName, getPluginUrlForResource("plugin_without_folders.zip")));
String pluginName = "fake-plugin";
Path pluginDir = createTempDir().resolve(pluginName);
Files.createDirectories(pluginDir.resolve("_site"));
Files.createFile(pluginDir.resolve("_site").resolve("somefile"));
String pluginUrl = createPlugin(pluginDir,
"description", "fake desc",
"version", "1.0",
"site", "true");
assertStatusOk(String.format(Locale.ROOT, "install %s --url %s --verbose", pluginName, pluginUrl));
assertThatPluginIsListed(pluginName);
// We want to check that Plugin Manager moves content to _site
assertFileExists(initialSettings.v2().pluginsFile().resolve("plugin-site/_site"));
assertFileExists(initialSettings.v2().pluginsFile().resolve(pluginName).resolve("_site"));
}
@ -313,14 +412,24 @@ public class PluginManagerTests extends ElasticsearchIntegrationTest {
@Test
public void testRemovePlugin() throws Exception {
String pluginName = "plugintest";
Path pluginDir = createTempDir().resolve(pluginName);
String pluginUrl = createPlugin(pluginDir,
"description", "fake desc",
"version", "1.0.0",
"elasticsearch.version", Version.CURRENT.toString(),
"java.version", System.getProperty("java.specification.version"),
"jvm", "true",
"classname", "FakePlugin");
// We want to remove plugin with plugin short name
singlePluginInstallAndRemove("plugintest", "plugintest", getPluginUrlForResource("plugin_without_folders.zip"));
singlePluginInstallAndRemove("plugintest", "plugintest", pluginUrl);
// We want to remove plugin with groupid/artifactid/version form
singlePluginInstallAndRemove("groupid/plugintest/1.0.0", "plugintest", getPluginUrlForResource("plugin_without_folders.zip"));
singlePluginInstallAndRemove("groupid/plugintest/1.0.0", "plugintest", pluginUrl);
// We want to remove plugin with groupid/artifactid form
singlePluginInstallAndRemove("groupid/plugintest", "plugintest", getPluginUrlForResource("plugin_without_folders.zip"));
singlePluginInstallAndRemove("groupid/plugintest", "plugintest", pluginUrl);
}
@Test
@ -370,18 +479,6 @@ public class PluginManagerTests extends ElasticsearchIntegrationTest {
}
}
/**
* Retrieve a URL string that represents the resource with the given {@code resourceName}.
* @param resourceName The resource name relative to {@link PluginManagerTests}.
* @return Never {@code null}.
* @throws NullPointerException if {@code resourceName} does not point to a valid resource.
*/
private String getPluginUrlForResource(String resourceName) {
URI uri = URI.create(PluginManagerTests.class.getResource(resourceName).toString());
return "file://" + uri.getPath();
}
private Tuple<Settings, Environment> buildInitialSettings() throws IOException {
Settings settings = settingsBuilder()
.put("discovery.zen.ping.multicast.enabled", false)

View File

@ -54,6 +54,7 @@ import org.elasticsearch.env.Environment;
import org.elasticsearch.env.NodeEnvironment;
import org.elasticsearch.test.cache.recycler.MockBigArrays;
import org.elasticsearch.test.cache.recycler.MockPageCacheRecycler;
import org.elasticsearch.test.junit.listeners.AssertionErrorThreadDumpPrinter;
import org.elasticsearch.test.junit.listeners.LoggingListener;
import org.elasticsearch.test.junit.listeners.ReproduceInfoPrinter;
import org.elasticsearch.test.search.MockSearchService;
@ -78,7 +79,8 @@ import static com.google.common.collect.Lists.newArrayList;
*/
@Listeners({
ReproduceInfoPrinter.class,
LoggingListener.class
LoggingListener.class,
AssertionErrorThreadDumpPrinter.class
})
// remove this entire annotation on upgrade to 5.3!
@ThreadLeakFilters(defaultFilters = true, filters = {
@ -550,44 +552,7 @@ public abstract class ElasticsearchTestCase extends LuceneTestCase {
protected static final void printStackDump(ESLogger logger) {
// print stack traces if we can't create any native thread anymore
Map<Thread, StackTraceElement[]> allStackTraces = Thread.getAllStackTraces();
logger.error(formatThreadStacks(allStackTraces));
}
/** Dump threads and their current stack trace. */
private static String formatThreadStacks(Map<Thread, StackTraceElement[]> threads) {
StringBuilder message = new StringBuilder();
int cnt = 1;
final Formatter f = new Formatter(message, Locale.ENGLISH);
for (Map.Entry<Thread, StackTraceElement[]> e : threads.entrySet()) {
if (e.getKey().isAlive()) {
f.format(Locale.ENGLISH, "\n %2d) %s", cnt++, threadName(e.getKey())).flush();
}
if (e.getValue().length == 0) {
message.append("\n at (empty stack)");
} else {
for (StackTraceElement ste : e.getValue()) {
message.append("\n at ").append(ste);
}
}
}
return message.toString();
}
private static String threadName(Thread t) {
return "Thread[" +
"id=" + t.getId() +
", name=" + t.getName() +
", state=" + t.getState() +
", group=" + groupName(t.getThreadGroup()) +
"]";
}
private static String groupName(ThreadGroup threadGroup) {
if (threadGroup == null) {
return "{null group}";
} else {
return threadGroup.getName();
}
logger.error(StackTraces.formatThreadStacks(allStackTraces));
}
/**

View File

@ -0,0 +1,63 @@
/*
* Licensed to Elasticsearch under one or more contributor
* license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright
* ownership. Elasticsearch licenses this file to you under
* the Apache License, Version 2.0 (the "License"); you may
* not use this file except in compliance with the License.
* You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing,
* software distributed under the License is distributed on an
* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
* KIND, either express or implied. See the License for the
* specific language governing permissions and limitations
* under the License.
*/
package org.elasticsearch.test;
import java.util.Formatter;
import java.util.Locale;
import java.util.Map;
public class StackTraces {
/** Dump threads and their current stack trace. */
public static String formatThreadStacks(Map<Thread, StackTraceElement[]> threads) {
StringBuilder message = new StringBuilder();
int cnt = 1;
final Formatter f = new Formatter(message, Locale.ENGLISH);
for (Map.Entry<Thread, StackTraceElement[]> e : threads.entrySet()) {
if (e.getKey().isAlive()) {
f.format(Locale.ENGLISH, "\n %2d) %s", cnt++, threadName(e.getKey())).flush();
}
if (e.getValue().length == 0) {
message.append("\n at (empty stack)");
} else {
for (StackTraceElement ste : e.getValue()) {
message.append("\n at ").append(ste);
}
}
}
return message.toString();
}
private static String groupName(ThreadGroup threadGroup) {
if (threadGroup == null) {
return "{null group}";
} else {
return threadGroup.getName();
}
}
private static String threadName(Thread t) {
return "Thread[" +
"id=" + t.getId() +
", name=" + t.getName() +
", state=" + t.getState() +
", group=" + groupName(t.getThreadGroup()) +
"]";
}
}

View File

@ -0,0 +1,37 @@
/*
* Licensed to Elasticsearch under one or more contributor
* license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright
* ownership. Elasticsearch licenses this file to you under
* the Apache License, Version 2.0 (the "License"); you may
* not use this file except in compliance with the License.
* You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing,
* software distributed under the License is distributed on an
* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
* KIND, either express or implied. See the License for the
* specific language governing permissions and limitations
* under the License.
*/
package org.elasticsearch.test.junit.listeners;
import org.elasticsearch.test.StackTraces;
import org.junit.runner.notification.Failure;
import org.junit.runner.notification.RunListener;
import java.util.Map;
public class AssertionErrorThreadDumpPrinter extends RunListener {
@Override
public void testFailure(Failure failure) throws Exception {
if (failure.getException() instanceof AssertionError) {
Map<Thread, StackTraceElement[]> allStackTraces = Thread.getAllStackTraces();
String threadStacks = StackTraces.formatThreadStacks(allStackTraces);
System.err.println(threadStacks);
}
}
}

View File

@ -39,7 +39,7 @@ public class KeyedLockTests extends ElasticsearchTestCase {
public void checkIfMapEmptyAfterLotsOfAcquireAndReleases() throws InterruptedException {
ConcurrentHashMap<String, Integer> counter = new ConcurrentHashMap<>();
ConcurrentHashMap<String, AtomicInteger> safeCounter = new ConcurrentHashMap<>();
KeyedLock<String> connectionLock = randomBoolean() ? new KeyedLock.GlobalLockable<String>(randomBoolean()) : new KeyedLock<String>(randomBoolean());
KeyedLock<String> connectionLock = new KeyedLock<String>(randomBoolean());
String[] names = new String[randomIntBetween(1, 40)];
for (int i = 0; i < names.length; i++) {
names[i] = randomRealisticUnicodeOfLengthBetween(10, 20);
@ -54,11 +54,6 @@ public class KeyedLockTests extends ElasticsearchTestCase {
threads[i].start();
}
startLatch.countDown();
for (int i = 0; i < numThreads; i++) {
if (randomBoolean()) {
threads[i].incWithGlobal();
}
}
for (int i = 0; i < numThreads; i++) {
threads[i].join();
@ -74,23 +69,9 @@ public class KeyedLockTests extends ElasticsearchTestCase {
}
}
@Test(expected = IllegalStateException.class)
public void checkCannotAcquireTwoLocksGlobal() throws InterruptedException {
KeyedLock.GlobalLockable<String> connectionLock = new KeyedLock.GlobalLockable<>();
String name = randomRealisticUnicodeOfLength(scaledRandomIntBetween(10, 50));
connectionLock.acquire(name);
try {
connectionLock.acquire(name);
} finally {
connectionLock.release(name);
connectionLock.globalLock().lock();
connectionLock.globalLock().unlock();
}
}
@Test(expected = IllegalStateException.class)
public void checkCannotAcquireTwoLocks() throws InterruptedException {
KeyedLock<String> connectionLock = randomBoolean() ? new KeyedLock.GlobalLockable<String>() : new KeyedLock<String>();
KeyedLock<String> connectionLock = new KeyedLock<String>();
String name = randomRealisticUnicodeOfLength(scaledRandomIntBetween(10, 50));
connectionLock.acquire(name);
connectionLock.acquire(name);
@ -98,7 +79,7 @@ public class KeyedLockTests extends ElasticsearchTestCase {
@Test(expected = IllegalStateException.class)
public void checkCannotReleaseUnacquiredLock() throws InterruptedException {
KeyedLock<String> connectionLock = randomBoolean() ? new KeyedLock.GlobalLockable<String>() : new KeyedLock<String>();
KeyedLock<String> connectionLock = new KeyedLock<String>();
String name = randomRealisticUnicodeOfLength(scaledRandomIntBetween(10, 50));
connectionLock.release(name);
}
@ -149,32 +130,5 @@ public class KeyedLockTests extends ElasticsearchTestCase {
}
}
}
public void incWithGlobal() {
if (connectionLock instanceof KeyedLock.GlobalLockable) {
final int iters = randomIntBetween(10, 200);
for (int i = 0; i < iters; i++) {
((KeyedLock.GlobalLockable) connectionLock).globalLock().lock();
try {
String curName = names[randomInt(names.length - 1)];
Integer integer = counter.get(curName);
if (integer == null) {
counter.put(curName, 1);
} else {
counter.put(curName, integer.intValue() + 1);
}
AtomicInteger atomicInteger = new AtomicInteger(0);
AtomicInteger value = safeCounter.putIfAbsent(curName, atomicInteger);
if (value == null) {
atomicInteger.incrementAndGet();
} else {
value.incrementAndGet();
}
} finally {
((KeyedLock.GlobalLockable) connectionLock).globalLock().unlock();
}
}
}
}
}
}

Some files were not shown because too many files have changed in this diff Show More