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; Query rangeQuery;
if (currentFieldType instanceof DateFieldMapper.DateFieldType && settings.timeZone() != null) { if (currentFieldType instanceof DateFieldMapper.DateFieldType && settings.timeZone() != null) {
DateFieldMapper.DateFieldType dateFieldType = (DateFieldMapper.DateFieldType) this.currentFieldType; 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 { } else {
rangeQuery = currentFieldType.rangeQuery(part1, part2, startInclusive, endInclusive, parseContext); rangeQuery = currentFieldType.rangeQuery(part1, part2, startInclusive, endInclusive);
} }
return rangeQuery; return rangeQuery;
} catch (RuntimeException e) { } catch (RuntimeException e) {

View File

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

View File

@ -18,13 +18,13 @@
*/ */
package org.elasticsearch.action.admin.indices.analyze; 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; 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) { public AnalyzeRequestBuilder(ElasticsearchClient client, AnalyzeAction action) {
super(client, action, new AnalyzeRequest()); super(client, action, new AnalyzeRequest());
@ -34,15 +34,6 @@ public class AnalyzeRequestBuilder extends SingleCustomOperationRequestBuilder<A
super(client, action, new AnalyzeRequest(index).text(text)); 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. * 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.ElasticsearchException;
import org.elasticsearch.Version; import org.elasticsearch.Version;
import org.elasticsearch.action.support.ActionFilters; 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.ClusterService;
import org.elasticsearch.cluster.ClusterState; import org.elasticsearch.cluster.ClusterState;
import org.elasticsearch.cluster.block.ClusterBlockException; import org.elasticsearch.cluster.block.ClusterBlockException;
@ -54,7 +54,7 @@ import java.util.List;
/** /**
* Transport action used to execute analyze requests * 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 IndicesService indicesService;
private final IndicesAnalysisService indicesAnalysisService; private final IndicesAnalysisService indicesAnalysisService;

View File

@ -19,16 +19,17 @@
package org.elasticsearch.action.admin.indices.mapping.get; package org.elasticsearch.action.admin.indices.mapping.get;
import org.elasticsearch.action.ActionRequestValidationException;
import org.elasticsearch.action.OriginalIndices; import org.elasticsearch.action.OriginalIndices;
import org.elasticsearch.action.support.IndicesOptions; 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.Strings;
import org.elasticsearch.common.io.stream.StreamInput; import org.elasticsearch.common.io.stream.StreamInput;
import org.elasticsearch.common.io.stream.StreamOutput; import org.elasticsearch.common.io.stream.StreamOutput;
import java.io.IOException; import java.io.IOException;
class GetFieldMappingsIndexRequest extends SingleCustomOperationRequest<GetFieldMappingsIndexRequest> { class GetFieldMappingsIndexRequest extends SingleShardRequest<GetFieldMappingsIndexRequest> {
private boolean probablySingleFieldRequest; private boolean probablySingleFieldRequest;
private boolean includeDefaults; private boolean includeDefaults;
@ -42,7 +43,6 @@ class GetFieldMappingsIndexRequest extends SingleCustomOperationRequest<GetField
GetFieldMappingsIndexRequest(GetFieldMappingsRequest other, String index, boolean probablySingleFieldRequest) { GetFieldMappingsIndexRequest(GetFieldMappingsRequest other, String index, boolean probablySingleFieldRequest) {
super(other); super(other);
this.preferLocal(other.local);
this.probablySingleFieldRequest = probablySingleFieldRequest; this.probablySingleFieldRequest = probablySingleFieldRequest;
this.includeDefaults = other.includeDefaults(); this.includeDefaults = other.includeDefaults();
this.types = other.types(); this.types = other.types();
@ -52,6 +52,11 @@ class GetFieldMappingsIndexRequest extends SingleCustomOperationRequest<GetField
this.originalIndices = new OriginalIndices(other); this.originalIndices = new OriginalIndices(other);
} }
@Override
public ActionRequestValidationException validate() {
return null;
}
public String[] types() { public String[] types() {
return types; return types;
} }
@ -88,11 +93,6 @@ class GetFieldMappingsIndexRequest extends SingleCustomOperationRequest<GetField
OriginalIndices.writeOriginalIndices(originalIndices, out); OriginalIndices.writeOriginalIndices(originalIndices, out);
} }
@Override
protected void writeIndex(StreamOutput out) throws IOException {
out.writeString(index());
}
@Override @Override
public void readFrom(StreamInput in) throws IOException { public void readFrom(StreamInput in) throws IOException {
super.readFrom(in); super.readFrom(in);
@ -103,8 +103,4 @@ class GetFieldMappingsIndexRequest extends SingleCustomOperationRequest<GetField
originalIndices = OriginalIndices.readOriginalIndices(in); 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.ElasticsearchException;
import org.elasticsearch.action.admin.indices.mapping.get.GetFieldMappingsResponse.FieldMappingMetaData; import org.elasticsearch.action.admin.indices.mapping.get.GetFieldMappingsResponse.FieldMappingMetaData;
import org.elasticsearch.action.support.ActionFilters; 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.ClusterService;
import org.elasticsearch.cluster.ClusterState; import org.elasticsearch.cluster.ClusterState;
import org.elasticsearch.cluster.block.ClusterBlockException; 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 * 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]"; private static final String ACTION_NAME = GetFieldMappingsAction.NAME + "[index]";

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

@ -21,6 +21,7 @@ package org.elasticsearch.action.percolate;
import org.elasticsearch.ElasticsearchException; import org.elasticsearch.ElasticsearchException;
import org.elasticsearch.ExceptionsHelper; import org.elasticsearch.ExceptionsHelper;
import org.elasticsearch.action.ActionRequestValidationException;
import org.elasticsearch.action.ActionResponse; import org.elasticsearch.action.ActionResponse;
import org.elasticsearch.action.IndicesRequest; import org.elasticsearch.action.IndicesRequest;
import org.elasticsearch.action.OriginalIndices; import org.elasticsearch.action.OriginalIndices;
@ -76,7 +77,7 @@ public class TransportShardMultiPercolateAction extends TransportSingleShardActi
} }
@Override @Override
protected boolean resolveIndex() { protected boolean resolveIndex(Request request) {
return false; return false;
} }
@ -127,6 +128,11 @@ public class TransportShardMultiPercolateAction extends TransportSingleShardActi
this.items = new ArrayList<>(); this.items = new ArrayList<>();
} }
@Override
public ActionRequestValidationException validate() {
return super.validateNonNullIndex();
}
@Override @Override
public String[] indices() { public String[] indices() {
List<String> indices = new ArrayList<>(); 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.IndicesRequest;
import org.elasticsearch.action.ValidateActions; import org.elasticsearch.action.ValidateActions;
import org.elasticsearch.action.support.IndicesOptions; import org.elasticsearch.action.support.IndicesOptions;
import org.elasticsearch.common.Nullable;
import org.elasticsearch.common.io.stream.StreamInput; import org.elasticsearch.common.io.stream.StreamInput;
import org.elasticsearch.common.io.stream.StreamOutput; import org.elasticsearch.common.io.stream.StreamOutput;
import org.elasticsearch.index.shard.ShardId; 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 { 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(); 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; private boolean threadedOperation = true;
protected SingleShardRequest() { protected SingleShardRequest() {
@ -57,8 +65,10 @@ public abstract class SingleShardRequest<T extends SingleShardRequest> extends A
this.index = index; 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; ActionRequestValidationException validationException = null;
if (index == null) { if (index == null) {
validationException = ValidateActions.addValidationError("index is missing", validationException); validationException = ValidateActions.addValidationError("index is missing", validationException);
@ -66,6 +76,13 @@ public abstract class SingleShardRequest<T extends SingleShardRequest> extends A
return validationException; 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() { public String index() {
return index; return index;
} }
@ -111,7 +128,7 @@ public abstract class SingleShardRequest<T extends SingleShardRequest> extends A
if (in.readBoolean()) { if (in.readBoolean()) {
internalShardId = ShardId.readShardId(in); 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 // 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 { public void writeTo(StreamOutput out) throws IOException {
super.writeTo(out); super.writeTo(out);
out.writeOptionalStreamable(internalShardId); 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.metadata.IndexNameExpressionResolver;
import org.elasticsearch.cluster.node.DiscoveryNode; import org.elasticsearch.cluster.node.DiscoveryNode;
import org.elasticsearch.cluster.node.DiscoveryNodes; import org.elasticsearch.cluster.node.DiscoveryNodes;
import org.elasticsearch.cluster.routing.ShardIterator;
import org.elasticsearch.cluster.routing.ShardRouting; import org.elasticsearch.cluster.routing.ShardRouting;
import org.elasticsearch.cluster.routing.ShardsIterator;
import org.elasticsearch.common.Nullable; import org.elasticsearch.common.Nullable;
import org.elasticsearch.common.logging.support.LoggerMessageFormat;
import org.elasticsearch.common.settings.Settings; import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.index.shard.ShardId; import org.elasticsearch.index.shard.ShardId;
import org.elasticsearch.threadpool.ThreadPool; import org.elasticsearch.threadpool.ThreadPool;
@ -43,7 +44,9 @@ import org.elasticsearch.transport.*;
import static org.elasticsearch.action.support.TransportActions.isShardNotAvailableException; 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> { 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 Response newResponse();
protected abstract boolean resolveIndex(); protected abstract boolean resolveIndex(Request request);
protected ClusterBlockException checkGlobalBlock(ClusterState state) { protected ClusterBlockException checkGlobalBlock(ClusterState state) {
return state.blocks().globalBlockedException(ClusterBlockLevel.READ); 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 { class AsyncSingleAction {
private final ActionListener<Response> listener; private final ActionListener<Response> listener;
private final ShardIterator shardIt; private final ShardsIterator shardIt;
private final InternalRequest internalRequest; private final InternalRequest internalRequest;
private final DiscoveryNodes nodes; private final DiscoveryNodes nodes;
private volatile Throwable lastFailure; private volatile Throwable lastFailure;
@ -126,7 +134,7 @@ public abstract class TransportSingleShardAction<Request extends SingleShardRequ
} }
String concreteSingleIndex; String concreteSingleIndex;
if (resolveIndex()) { if (resolveIndex(request)) {
concreteSingleIndex = indexNameExpressionResolver.concreteSingleIndex(clusterState, request); concreteSingleIndex = indexNameExpressionResolver.concreteSingleIndex(clusterState, request);
} else { } else {
concreteSingleIndex = request.index(); concreteSingleIndex = request.index();
@ -143,7 +151,32 @@ public abstract class TransportSingleShardAction<Request extends SingleShardRequ
} }
public void start() { 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) { private void onFailure(ShardRouting shardRouting, Throwable e) {
@ -163,10 +196,10 @@ public abstract class TransportSingleShardAction<Request extends SingleShardRequ
if (shardRouting == null) { if (shardRouting == null) {
Throwable failure = lastFailure; Throwable failure = lastFailure;
if (failure == null || isShardNotAvailableException(failure)) { 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 { } else {
if (logger.isDebugEnabled()) { 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); listener.onFailure(failure);
@ -174,7 +207,7 @@ public abstract class TransportSingleShardAction<Request extends SingleShardRequ
} }
DiscoveryNode node = nodes.get(shardRouting.currentNodeId()); DiscoveryNode node = nodes.get(shardRouting.currentNodeId());
if (node == null) { if (node == null) {
onFailure(shardRouting, new NoShardAvailableActionException(shardIt.shardId())); onFailure(shardRouting, new NoShardAvailableActionException(shardRouting.shardId()));
} else { } else {
internalRequest.request().internalShardId = shardRouting.shardId(); internalRequest.request().internalShardId = shardRouting.shardId();
transportService.sendRequest(node, transportShardAction, internalRequest.request(), new BaseTransportResponseHandler<Response>() { transportService.sendRequest(node, transportShardAction, internalRequest.request(), new BaseTransportResponseHandler<Response>() {

View File

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

View File

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

View File

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

View File

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

View File

@ -136,21 +136,9 @@ public class JarHell {
/** inspect manifest for sure incompatibilities */ /** inspect manifest for sure incompatibilities */
static void checkManifest(Manifest manifest, Path jar) { static void checkManifest(Manifest manifest, Path jar) {
// give a nice error if jar requires a newer java version // 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"); String targetVersion = manifest.getMainAttributes().getValue("X-Compile-Target-JDK");
if (targetVersion != null) { if (targetVersion != null) {
float current = Float.POSITIVE_INFINITY; checkJavaVersion(jar.toString(), targetVersion);
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);
}
} }
// give a nice error if jar is compiled against different es version // 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) { static void checkClass(Map<String,Path> clazzes, String clazz, Path jarpath) {
Path previous = clazzes.put(clazz, jarpath); Path previous = clazzes.put(clazz, jarpath);
if (previous != null) { if (previous != null) {

View File

@ -420,7 +420,10 @@ public class MetaDataCreateIndexService extends AbstractComponent {
.put(indexMetaData, false) .put(indexMetaData, false)
.build(); .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()); ClusterBlocks.Builder blocks = ClusterBlocks.builder().blocks(currentState.blocks());
if (!request.blocks().isEmpty()) { if (!request.blocks().isEmpty()) {

View File

@ -83,10 +83,10 @@ public class AllocationId implements ToXContent {
/** /**
* Creates a new allocation id representing a cancelled relocation. * Creates a new allocation id representing a cancelled relocation.
* * <p/>
* Note that this is expected to be called on the allocation id * Note that this is expected to be called on the allocation id
* of the *source* shard * of the *source* shard
* */ */
public static AllocationId cancelRelocation(AllocationId allocationId) { public static AllocationId cancelRelocation(AllocationId allocationId) {
assert allocationId.getRelocationId() != null; assert allocationId.getRelocationId() != null;
return new AllocationId(allocationId.getId(), null); return new AllocationId(allocationId.getId(), null);
@ -94,7 +94,7 @@ public class AllocationId implements ToXContent {
/** /**
* Creates a new allocation id finalizing a relocation. * Creates a new allocation id finalizing a relocation.
* * <p/>
* Note that this is expected to be called on the allocation id * 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. * of the *target* shard and thus it only needs to clear the relocating id.
*/ */
@ -120,9 +120,16 @@ public class AllocationId implements ToXContent {
@Override @Override
public boolean equals(Object o) { public boolean equals(Object o) {
if (this == o) return true; if (this == o) {
return true;
}
if (o == null) {
return false;
}
AllocationId that = (AllocationId) o; 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); 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) { void add(ShardRouting shard) {
// TODO use Set with ShardIds for faster lookup. // TODO use Set with ShardIds for faster lookup.
for (ShardRouting shardRouting : shards) { 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"); 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) { void relocate(String relocatingNodeId) {
ensureNotFrozen(); ensureNotFrozen();
version++; 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; state = ShardRoutingState.RELOCATING;
this.relocatingNodeId = relocatingNodeId; this.relocatingNodeId = relocatingNodeId;
this.allocationId = AllocationId.newRelocation(allocationId); this.allocationId = AllocationId.newRelocation(allocationId);
@ -467,7 +467,7 @@ public final class ShardRouting implements Streamable, ToXContent {
restoreSource = null; restoreSource = null;
unassignedInfo = null; // we keep the unassigned data until the shard is started unassignedInfo = null; // we keep the unassigned data until the shard is started
if (allocationId.getRelocationId() != null) { if (allocationId.getRelocationId() != null) {
// target relocation // relocation target
allocationId = AllocationId.finishRelocation(allocationId); allocationId = AllocationId.finishRelocation(allocationId);
} }
state = ShardRoutingState.STARTED; state = ShardRoutingState.STARTED;
@ -498,6 +498,106 @@ public final class ShardRouting implements Streamable, ToXContent {
primary = false; 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 @Override
public boolean equals(Object o) { public boolean equals(Object o) {
if (this == o) { if (this == o) {
@ -508,32 +608,8 @@ public final class ShardRouting implements Streamable, ToXContent {
return false; return false;
} }
ShardRouting that = (ShardRouting) o; ShardRouting that = (ShardRouting) o;
// TODO: add version + unassigned info check. see #12387
if (primary != that.primary) { return equalsIgnoringMetaData(that);
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;
} }
private long hashVersion = version - 1; 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 * 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; private final Reason reason;

View File

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

View File

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

View File

@ -38,7 +38,7 @@ import org.elasticsearch.node.settings.NodeSettingsService;
* node. The default is <tt>4</tt></li> * node. The default is <tt>4</tt></li>
* <p/> * <p/>
* <li><tt>cluster.routing.allocation.node_concurrent_recoveries</tt> - * <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> * default is <tt>2</tt></li>
* </ul> * </ul>
* <p/> * <p/>
@ -106,7 +106,7 @@ public class ThrottlingAllocationDecider extends AllocationDecider {
public Decision canAllocate(RoutingNode node, RoutingAllocation allocation) { public Decision canAllocate(RoutingNode node, RoutingAllocation allocation) {
int currentRecoveries = 0; int currentRecoveries = 0;
for (ShardRouting shard : node) { for (ShardRouting shard : node) {
if (shard.initializing() || shard.relocating()) { if (shard.initializing()) {
currentRecoveries++; currentRecoveries++;
} }
} }

View File

@ -22,10 +22,7 @@ package org.elasticsearch.common.util.concurrent;
import java.util.concurrent.ConcurrentMap; import java.util.concurrent.ConcurrentMap;
import java.util.concurrent.atomic.AtomicInteger; 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.ReentrantLock;
import java.util.concurrent.locks.ReentrantReadWriteLock;
/** /**
* This class manages locks. Locks can be accessed with an identifier and are * This class manages locks. Locks can be accessed with an identifier and are
@ -115,58 +112,4 @@ public class KeyedLock<T> {
return !map.isEmpty(); 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 }); // sort for priority ordering
changed |= primaryShardAllocator.allocateUnassigned(allocation); changed |= primaryShardAllocator.allocateUnassigned(allocation);
changed |= replicaShardAllocator.processExistingRecoveries(allocation);
changed |= replicaShardAllocator.allocateUnassigned(allocation); changed |= replicaShardAllocator.allocateUnassigned(allocation);
return changed; return changed;
} }

View File

@ -25,11 +25,11 @@ import com.carrotsearch.hppc.cursors.ObjectCursor;
import com.carrotsearch.hppc.cursors.ObjectLongCursor; import com.carrotsearch.hppc.cursors.ObjectLongCursor;
import org.elasticsearch.cluster.ClusterChangedEvent; import org.elasticsearch.cluster.ClusterChangedEvent;
import org.elasticsearch.cluster.metadata.IndexMetaData; import org.elasticsearch.cluster.metadata.IndexMetaData;
import org.elasticsearch.cluster.metadata.MetaData;
import org.elasticsearch.cluster.node.DiscoveryNode; import org.elasticsearch.cluster.node.DiscoveryNode;
import org.elasticsearch.cluster.routing.RoutingNode; import org.elasticsearch.cluster.routing.RoutingNode;
import org.elasticsearch.cluster.routing.RoutingNodes; import org.elasticsearch.cluster.routing.RoutingNodes;
import org.elasticsearch.cluster.routing.ShardRouting; 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.RoutingAllocation;
import org.elasticsearch.cluster.routing.allocation.decider.Decision; import org.elasticsearch.cluster.routing.allocation.decider.Decision;
import org.elasticsearch.common.Nullable; import org.elasticsearch.common.Nullable;
@ -40,7 +40,6 @@ import org.elasticsearch.common.unit.TimeValue;
import org.elasticsearch.index.store.StoreFileMetaData; import org.elasticsearch.index.store.StoreFileMetaData;
import org.elasticsearch.indices.store.TransportNodesListShardStoreMetaData; import org.elasticsearch.indices.store.TransportNodesListShardStoreMetaData;
import java.util.Iterator;
import java.util.Map; import java.util.Map;
/** /**
@ -51,6 +50,62 @@ public abstract class ReplicaShardAllocator extends AbstractComponent {
super(settings); 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) { public boolean allocateUnassigned(RoutingAllocation allocation) {
boolean changed = false; boolean changed = false;
final RoutingNodes routingNodes = allocation.routingNodes(); final RoutingNodes routingNodes = allocation.routingNodes();
@ -236,7 +291,7 @@ public abstract class ReplicaShardAllocator extends AbstractComponent {
highestMatchNode = cursor.key; highestMatchNode = cursor.key;
} }
} }
nodeWithHighestMatch = highestMatchNode; this.nodeWithHighestMatch = highestMatchNode;
} }
/** /**
@ -248,6 +303,10 @@ public abstract class ReplicaShardAllocator extends AbstractComponent {
return this.nodeWithHighestMatch; 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. * 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; package org.elasticsearch.index.engine;
import com.google.common.base.Preconditions; import com.google.common.base.Preconditions;
import org.apache.lucene.index.*; import org.apache.lucene.index.*;
import org.apache.lucene.search.IndexSearcher; import org.apache.lucene.search.IndexSearcher;
import org.apache.lucene.search.Query; 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.VersionType;
import org.elasticsearch.index.deletionpolicy.SnapshotDeletionPolicy; import org.elasticsearch.index.deletionpolicy.SnapshotDeletionPolicy;
import org.elasticsearch.index.deletionpolicy.SnapshotIndexCommit; import org.elasticsearch.index.deletionpolicy.SnapshotIndexCommit;
import org.elasticsearch.index.mapper.DocumentMapper;
import org.elasticsearch.index.mapper.ParseContext.Document; import org.elasticsearch.index.mapper.ParseContext.Document;
import org.elasticsearch.index.mapper.ParsedDocument; import org.elasticsearch.index.mapper.ParsedDocument;
import org.elasticsearch.index.mapper.Uid; import org.elasticsearch.index.mapper.Uid;
@ -57,11 +55,7 @@ import org.elasticsearch.index.translog.Translog;
import java.io.Closeable; import java.io.Closeable;
import java.io.IOException; import java.io.IOException;
import java.util.Arrays; import java.util.*;
import java.util.Comparator;
import java.util.HashMap;
import java.util.List;
import java.util.Map;
import java.util.concurrent.TimeUnit; import java.util.concurrent.TimeUnit;
import java.util.concurrent.atomic.AtomicBoolean; import java.util.concurrent.atomic.AtomicBoolean;
import java.util.concurrent.locks.Condition; import java.util.concurrent.locks.Condition;
@ -288,7 +282,7 @@ public abstract class Engine implements Closeable {
try { try {
final Searcher retVal = newSearcher(source, searcher, manager); final Searcher retVal = newSearcher(source, searcher, manager);
success = true; success = true;
return retVal; return config().getWrappingService().wrap(engineConfig, retVal);
} finally { } finally {
if (!success) { if (!success) {
manager.release(searcher); 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.codecs.Codec;
import org.apache.lucene.index.IndexWriterConfig; import org.apache.lucene.index.IndexWriterConfig;
import org.apache.lucene.index.MergePolicy; import org.apache.lucene.index.MergePolicy;
import org.elasticsearch.index.shard.MergeSchedulerConfig;
import org.apache.lucene.search.QueryCache; import org.apache.lucene.search.QueryCache;
import org.apache.lucene.search.QueryCachingPolicy; import org.apache.lucene.search.QueryCachingPolicy;
import org.apache.lucene.search.similarities.Similarity; 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.codec.CodecService;
import org.elasticsearch.index.deletionpolicy.SnapshotDeletionPolicy; import org.elasticsearch.index.deletionpolicy.SnapshotDeletionPolicy;
import org.elasticsearch.index.indexing.ShardIndexingService; import org.elasticsearch.index.indexing.ShardIndexingService;
import org.elasticsearch.index.shard.MergeSchedulerConfig;
import org.elasticsearch.index.shard.ShardId; import org.elasticsearch.index.shard.ShardId;
import org.elasticsearch.index.shard.TranslogRecoveryPerformer; import org.elasticsearch.index.shard.TranslogRecoveryPerformer;
import org.elasticsearch.index.store.Store; import org.elasticsearch.index.store.Store;
@ -77,6 +77,7 @@ public final class EngineConfig {
private final boolean forceNewTranslog; private final boolean forceNewTranslog;
private final QueryCache queryCache; private final QueryCache queryCache;
private final QueryCachingPolicy queryCachingPolicy; private final QueryCachingPolicy queryCachingPolicy;
private final IndexSearcherWrappingService wrappingService;
/** /**
* Index setting for index concurrency / number of threadstates in the indexwriter. * 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, Settings indexSettings, IndicesWarmer warmer, Store store, SnapshotDeletionPolicy deletionPolicy,
MergePolicy mergePolicy, MergeSchedulerConfig mergeSchedulerConfig, Analyzer analyzer, MergePolicy mergePolicy, MergeSchedulerConfig mergeSchedulerConfig, Analyzer analyzer,
Similarity similarity, CodecService codecService, Engine.FailedEngineListener failedEngineListener, 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.shardId = shardId;
this.indexSettings = indexSettings; this.indexSettings = indexSettings;
this.threadPool = threadPool; this.threadPool = threadPool;
@ -157,6 +158,7 @@ public final class EngineConfig {
this.similarity = similarity; this.similarity = similarity;
this.codecService = codecService; this.codecService = codecService;
this.failedEngineListener = failedEngineListener; this.failedEngineListener = failedEngineListener;
this.wrappingService = wrappingService;
this.optimizeAutoGenerateId = indexSettings.getAsBoolean(EngineConfig.INDEX_OPTIMIZE_AUTOGENERATED_ID_SETTING, false); this.optimizeAutoGenerateId = indexSettings.getAsBoolean(EngineConfig.INDEX_OPTIMIZE_AUTOGENERATED_ID_SETTING, false);
this.compoundOnFlush = indexSettings.getAsBoolean(EngineConfig.INDEX_COMPOUND_ON_FLUSH, compoundOnFlush); 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))); 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; return queryCachingPolicy;
} }
public IndexSearcherWrappingService getWrappingService() {
return wrappingService;
}
/** /**
* Returns the translog config for this engine * 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.ImmutableMap;
import com.google.common.collect.Maps; import com.google.common.collect.Maps;
import org.elasticsearch.ExceptionsHelper; import org.elasticsearch.ExceptionsHelper;
import org.elasticsearch.Version;
import org.elasticsearch.common.collect.MapBuilder; import org.elasticsearch.common.collect.MapBuilder;
import org.elasticsearch.common.collect.Tuple; import org.elasticsearch.common.collect.Tuple;
import org.elasticsearch.common.inject.Inject; import org.elasticsearch.common.inject.Inject;
import org.elasticsearch.common.settings.Settings; import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.common.util.concurrent.KeyedLock;
import org.elasticsearch.index.AbstractIndexComponent; import org.elasticsearch.index.AbstractIndexComponent;
import org.elasticsearch.index.Index; import org.elasticsearch.index.Index;
import org.elasticsearch.index.fielddata.plain.*; import org.elasticsearch.index.fielddata.plain.*;
@ -136,11 +137,16 @@ public class IndexFieldDataService extends AbstractIndexComponent {
} }
private final IndicesFieldDataCache indicesFieldDataCache; private final IndicesFieldDataCache indicesFieldDataCache;
private final KeyedLock.GlobalLockable<String> fieldLoadingLock = new KeyedLock.GlobalLockable<>(); // the below map needs to be modified under a lock
private final Map<String, IndexFieldDataCache> fieldDataCaches = Maps.newHashMap(); // no need for concurrency support, always used under lock private final Map<String, IndexFieldDataCache> fieldDataCaches = Maps.newHashMap();
IndexService indexService; 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 @Inject
public IndexFieldDataService(Index index, @IndexSettings Settings indexSettings, IndicesFieldDataCache indicesFieldDataCache, public IndexFieldDataService(Index index, @IndexSettings Settings indexSettings, IndicesFieldDataCache indicesFieldDataCache,
CircuitBreakerService circuitBreakerService) { CircuitBreakerService circuitBreakerService) {
@ -154,41 +160,35 @@ public class IndexFieldDataService extends AbstractIndexComponent {
this.indexService = indexService; this.indexService = indexService;
} }
public void clear() { public synchronized void clear() {
fieldLoadingLock.globalLock().lock(); parentIndexFieldData = null;
try { List<Throwable> exceptions = new ArrayList<>(0);
List<Throwable> exceptions = new ArrayList<>(0); final Collection<IndexFieldDataCache> fieldDataCacheValues = fieldDataCaches.values();
final Collection<IndexFieldDataCache> fieldDataCacheValues = fieldDataCaches.values(); for (IndexFieldDataCache cache : fieldDataCacheValues) {
for (IndexFieldDataCache cache : fieldDataCacheValues) { try {
try { cache.clear();
cache.clear(); } catch (Throwable t) {
} catch (Throwable t) { exceptions.add(t);
exceptions.add(t);
}
} }
fieldDataCacheValues.clear();
ExceptionsHelper.maybeThrowRuntimeAndSuppress(exceptions);
} finally {
fieldLoadingLock.globalLock().unlock();
} }
fieldDataCacheValues.clear();
ExceptionsHelper.maybeThrowRuntimeAndSuppress(exceptions);
} }
public void clearField(final String fieldName) { public synchronized void clearField(final String fieldName) {
fieldLoadingLock.acquire(fieldName); if (ParentFieldMapper.NAME.equals(fieldName)) {
try { 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);
} finally {
fieldLoadingLock.release(fieldName);
} }
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") @SuppressWarnings("unchecked")
@ -199,32 +199,31 @@ public class IndexFieldDataService extends AbstractIndexComponent {
throw new IllegalArgumentException("found no fielddata type for field [" + fieldNames.fullName() + "]"); throw new IllegalArgumentException("found no fielddata type for field [" + fieldNames.fullName() + "]");
} }
final boolean docValues = fieldType.hasDocValues(); final boolean docValues = fieldType.hasDocValues();
final String key = fieldNames.indexName(); IndexFieldData.Builder builder = null;
fieldLoadingLock.acquire(key); String format = type.getFormat(indexSettings);
try { if (format != null && FieldDataType.DOC_VALUES_FORMAT_VALUE.equals(format) && !docValues) {
IndexFieldData.Builder builder = null; logger.warn("field [" + fieldNames.fullName() + "] has no doc values, will use default field data format");
String format = type.getFormat(indexSettings); format = null;
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"); if (format != null) {
format = null; builder = buildersByTypeAndFormat.get(Tuple.tuple(type.getType(), format));
}
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());
}
if (builder == null) { if (builder == null) {
builder = buildersByType.get(type.getType()); logger.warn("failed to find format [" + format + "] for field [" + fieldNames.fullName() + "], will use default");
}
if (builder == null) {
throw new IllegalArgumentException("failed to find field data builder for field " + fieldNames.fullName() + ", and type " + type.getType());
} }
}
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) { if (cache == null) {
// we default to node level cache, which in turn defaults to be unbounded // 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 // 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); fieldDataCaches.put(fieldNames.indexName(), cache);
} }
return (IFD) builder.build(index, indexSettings, fieldType, cache, circuitBreakerService, indexService.mapperService()); // Remove this in 3.0
} finally { final boolean isOldParentField = ParentFieldMapper.NAME.equals(fieldNames.indexName())
fieldLoadingLock.release(key); && 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 @Override
public RandomAccessOrds getOrdinalsValues() { public RandomAccessOrds getOrdinalsValues() {
return (RandomAccessOrds) DocValues.emptySortedSet(); return DocValues.emptySortedSet();
} }
}; };
} }

View File

@ -19,8 +19,8 @@
package org.elasticsearch.index.fielddata.plain; package org.elasticsearch.index.fielddata.plain;
import org.apache.lucene.index.LeafReaderContext;
import org.apache.lucene.index.IndexReader; import org.apache.lucene.index.IndexReader;
import org.apache.lucene.index.LeafReaderContext;
import org.apache.lucene.index.Terms; import org.apache.lucene.index.Terms;
import org.apache.lucene.index.TermsEnum; import org.apache.lucene.index.TermsEnum;
import org.apache.lucene.util.BytesRef; 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.AbstractIndexComponent;
import org.elasticsearch.index.Index; import org.elasticsearch.index.Index;
import org.elasticsearch.index.fielddata.*; import org.elasticsearch.index.fielddata.*;
import org.elasticsearch.index.mapper.FieldMapper;
import org.elasticsearch.index.mapper.MappedFieldType; import org.elasticsearch.index.mapper.MappedFieldType;
import org.elasticsearch.index.settings.IndexSettings; import org.elasticsearch.index.settings.IndexSettings;
@ -72,6 +71,11 @@ public abstract class AbstractIndexFieldData<FD extends AtomicFieldData> extends
@Override @Override
public FD load(LeafReaderContext context) { 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 { try {
FD fd = cache.load(context, this); FD fd = cache.load(context, this);
return fd; 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 * A {@code PerValueEstimator} is a sub-class that can be used to estimate
* the memory overhead for loading the data. Each field data * 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"); 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); 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 { protected TermsEnum filter(Terms terms, LeafReader reader) throws IOException {
TermsEnum iterator = terms.iterator(); TermsEnum iterator = terms.iterator();
if (iterator == null) { if (iterator == null) {

View File

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

View File

@ -19,40 +19,19 @@
package org.elasticsearch.index.fielddata.plain; package org.elasticsearch.index.fielddata.plain;
import org.apache.lucene.index.DocValues; import org.apache.lucene.index.*;
import org.apache.lucene.index.LeafReader; import org.apache.lucene.util.*;
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.elasticsearch.common.Nullable; import org.elasticsearch.common.Nullable;
import org.elasticsearch.common.breaker.CircuitBreaker; import org.elasticsearch.common.breaker.CircuitBreaker;
import org.elasticsearch.common.settings.Settings; import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.common.util.BigArrays; import org.elasticsearch.common.util.BigArrays;
import org.elasticsearch.common.util.DoubleArray; import org.elasticsearch.common.util.DoubleArray;
import org.elasticsearch.index.Index; import org.elasticsearch.index.Index;
import org.elasticsearch.index.fielddata.AtomicNumericFieldData; import org.elasticsearch.index.fielddata.*;
import org.elasticsearch.index.fielddata.FieldData;
import org.elasticsearch.index.fielddata.FieldDataType;
import org.elasticsearch.index.fielddata.IndexFieldData;
import org.elasticsearch.index.fielddata.IndexFieldData.XFieldComparatorSource.Nested; 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.fieldcomparator.DoubleValuesComparatorSource;
import org.elasticsearch.index.fielddata.ordinals.Ordinals; import org.elasticsearch.index.fielddata.ordinals.Ordinals;
import org.elasticsearch.index.fielddata.ordinals.OrdinalsBuilder; import org.elasticsearch.index.fielddata.ordinals.OrdinalsBuilder;
import org.elasticsearch.index.mapper.FieldMapper;
import org.elasticsearch.index.mapper.MappedFieldType; import org.elasticsearch.index.mapper.MappedFieldType;
import org.elasticsearch.index.mapper.MapperService; import org.elasticsearch.index.mapper.MapperService;
import org.elasticsearch.index.settings.IndexSettings; 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 @Override
public XFieldComparatorSource comparatorSource(@Nullable Object missingValue, MultiValueMode sortMode, Nested nested) { public XFieldComparatorSource comparatorSource(@Nullable Object missingValue, MultiValueMode sortMode, Nested nested) {
return new DoubleValuesComparatorSource(this, missingValue, sortMode, nested); return new DoubleValuesComparatorSource(this, missingValue, sortMode, nested);

View File

@ -18,40 +18,19 @@
*/ */
package org.elasticsearch.index.fielddata.plain; package org.elasticsearch.index.fielddata.plain;
import org.apache.lucene.index.DocValues; import org.apache.lucene.index.*;
import org.apache.lucene.index.LeafReader; import org.apache.lucene.util.*;
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.elasticsearch.common.Nullable; import org.elasticsearch.common.Nullable;
import org.elasticsearch.common.breaker.CircuitBreaker; import org.elasticsearch.common.breaker.CircuitBreaker;
import org.elasticsearch.common.settings.Settings; import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.common.util.BigArrays; import org.elasticsearch.common.util.BigArrays;
import org.elasticsearch.common.util.FloatArray; import org.elasticsearch.common.util.FloatArray;
import org.elasticsearch.index.Index; import org.elasticsearch.index.Index;
import org.elasticsearch.index.fielddata.AtomicNumericFieldData; import org.elasticsearch.index.fielddata.*;
import org.elasticsearch.index.fielddata.FieldData;
import org.elasticsearch.index.fielddata.FieldDataType;
import org.elasticsearch.index.fielddata.IndexFieldData;
import org.elasticsearch.index.fielddata.IndexFieldData.XFieldComparatorSource.Nested; 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.fieldcomparator.FloatValuesComparatorSource;
import org.elasticsearch.index.fielddata.ordinals.Ordinals; import org.elasticsearch.index.fielddata.ordinals.Ordinals;
import org.elasticsearch.index.fielddata.ordinals.OrdinalsBuilder; import org.elasticsearch.index.fielddata.ordinals.OrdinalsBuilder;
import org.elasticsearch.index.mapper.FieldMapper;
import org.elasticsearch.index.mapper.MappedFieldType; import org.elasticsearch.index.mapper.MappedFieldType;
import org.elasticsearch.index.mapper.MapperService; import org.elasticsearch.index.mapper.MapperService;
import org.elasticsearch.index.settings.IndexSettings; 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 @Override
public XFieldComparatorSource comparatorSource(@Nullable Object missingValue, MultiValueMode sortMode, Nested nested) { public XFieldComparatorSource comparatorSource(@Nullable Object missingValue, MultiValueMode sortMode, Nested nested) {
return new FloatValuesComparatorSource(this, missingValue, sortMode, nested); return new FloatValuesComparatorSource(this, missingValue, sortMode, nested);

View File

@ -20,24 +20,9 @@
package org.elasticsearch.index.fielddata.plain; package org.elasticsearch.index.fielddata.plain;
import com.google.common.base.Preconditions; import com.google.common.base.Preconditions;
import org.apache.lucene.index.*;
import org.apache.lucene.index.DocValues; import org.apache.lucene.util.*;
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.util.BitSet; 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.PackedInts;
import org.apache.lucene.util.packed.PackedLongValues; import org.apache.lucene.util.packed.PackedLongValues;
import org.elasticsearch.ElasticsearchException; import org.elasticsearch.ElasticsearchException;
@ -45,18 +30,11 @@ import org.elasticsearch.common.Nullable;
import org.elasticsearch.common.breaker.CircuitBreaker; import org.elasticsearch.common.breaker.CircuitBreaker;
import org.elasticsearch.common.settings.Settings; import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.index.Index; import org.elasticsearch.index.Index;
import org.elasticsearch.index.fielddata.AtomicNumericFieldData; import org.elasticsearch.index.fielddata.*;
import org.elasticsearch.index.fielddata.FieldData;
import org.elasticsearch.index.fielddata.FieldDataType;
import org.elasticsearch.index.fielddata.IndexFieldData;
import org.elasticsearch.index.fielddata.IndexFieldData.XFieldComparatorSource.Nested; 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.fieldcomparator.LongValuesComparatorSource;
import org.elasticsearch.index.fielddata.ordinals.Ordinals; import org.elasticsearch.index.fielddata.ordinals.Ordinals;
import org.elasticsearch.index.fielddata.ordinals.OrdinalsBuilder; import org.elasticsearch.index.fielddata.ordinals.OrdinalsBuilder;
import org.elasticsearch.index.mapper.FieldMapper;
import org.elasticsearch.index.mapper.MappedFieldType; import org.elasticsearch.index.mapper.MappedFieldType;
import org.elasticsearch.index.mapper.MapperService; import org.elasticsearch.index.mapper.MapperService;
import org.elasticsearch.index.settings.IndexSettings; import org.elasticsearch.index.settings.IndexSettings;
@ -64,11 +42,7 @@ import org.elasticsearch.indices.breaker.CircuitBreakerService;
import org.elasticsearch.search.MultiValueMode; import org.elasticsearch.search.MultiValueMode;
import java.io.IOException; import java.io.IOException;
import java.util.ArrayList; import java.util.*;
import java.util.Collection;
import java.util.Collections;
import java.util.EnumSet;
import java.util.List;
/** /**
* Stores numeric data into bit-packed arrays for better memory efficiency. * Stores numeric data into bit-packed arrays for better memory efficiency.
@ -404,6 +378,11 @@ public class PackedArrayIndexFieldData extends AbstractIndexFieldData<AtomicNume
return pageMemorySize; return pageMemorySize;
} }
@Override
protected AtomicNumericFieldData empty(int maxDoc) {
return AtomicLongFieldData.empty(maxDoc);
}
@Override @Override
public XFieldComparatorSource comparatorSource(@Nullable Object missingValue, MultiValueMode sortMode, Nested nested) { public XFieldComparatorSource comparatorSource(@Nullable Object missingValue, MultiValueMode sortMode, Nested nested) {
return new LongValuesComparatorSource(this, missingValue, sortMode, 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.carrotsearch.hppc.cursors.ObjectObjectCursor;
import com.google.common.collect.ImmutableSet; import com.google.common.collect.ImmutableSet;
import com.google.common.collect.ImmutableSortedSet; import com.google.common.collect.ImmutableSortedSet;
import org.apache.lucene.index.*; import org.apache.lucene.index.*;
import org.apache.lucene.index.MultiDocValues.OrdinalMap; import org.apache.lucene.index.MultiDocValues.OrdinalMap;
import org.apache.lucene.search.DocIdSetIterator; import org.apache.lucene.search.DocIdSetIterator;
@ -79,12 +80,23 @@ public class ParentChildIndexFieldData extends AbstractIndexFieldData<AtomicPare
FieldDataType fieldDataType, IndexFieldDataCache cache, MapperService mapperService, FieldDataType fieldDataType, IndexFieldDataCache cache, MapperService mapperService,
CircuitBreakerService breakerService) { CircuitBreakerService breakerService) {
super(index, indexSettings, fieldNames, fieldDataType, cache); super(index, indexSettings, fieldNames, fieldDataType, cache);
parentTypes = new TreeSet<>();
this.breakerService = breakerService; this.breakerService = breakerService;
for (DocumentMapper documentMapper : mapperService.docMappers(false)) { if (Version.indexCreated(indexSettings).before(Version.V_2_0_0_beta1)) {
beforeCreate(documentMapper); 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 @Override
@ -96,10 +108,6 @@ public class ParentChildIndexFieldData extends AbstractIndexFieldData<AtomicPare
public AtomicParentChildFieldData load(LeafReaderContext context) { public AtomicParentChildFieldData load(LeafReaderContext context) {
if (Version.indexCreated(indexSettings).onOrAfter(Version.V_2_0_0_beta1)) { if (Version.indexCreated(indexSettings).onOrAfter(Version.V_2_0_0_beta1)) {
final LeafReader reader = context.reader(); final LeafReader reader = context.reader();
final NavigableSet<String> parentTypes;
synchronized (lock) {
parentTypes = ImmutableSortedSet.copyOf(this.parentTypes);
}
return new AbstractAtomicParentChildFieldData() { return new AbstractAtomicParentChildFieldData() {
public Set<String> types() { public Set<String> types() {
@ -131,12 +139,22 @@ public class ParentChildIndexFieldData extends AbstractIndexFieldData<AtomicPare
} }
}; };
} else { } 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 @Override
public AbstractAtomicParentChildFieldData loadDirect(LeafReaderContext context) throws Exception { 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(); LeafReader reader = context.reader();
final float acceptableTransientOverheadRatio = fieldDataType.getSettings().getAsFloat( final float acceptableTransientOverheadRatio = fieldDataType.getSettings().getAsFloat(
"acceptable_transient_overhead_ratio", OrdinalsBuilder.DEFAULT_ACCEPTABLE_OVERHEAD_RATIO "acceptable_transient_overhead_ratio", OrdinalsBuilder.DEFAULT_ACCEPTABLE_OVERHEAD_RATIO
@ -211,6 +229,7 @@ public class ParentChildIndexFieldData extends AbstractIndexFieldData<AtomicPare
@Override @Override
public void beforeCreate(DocumentMapper mapper) { public void beforeCreate(DocumentMapper mapper) {
// Remove in 3.0
synchronized (lock) { synchronized (lock) {
ParentFieldMapper parentFieldMapper = mapper.parentFieldMapper(); ParentFieldMapper parentFieldMapper = mapper.parentFieldMapper();
if (parentFieldMapper.active()) { if (parentFieldMapper.active()) {
@ -224,13 +243,8 @@ public class ParentChildIndexFieldData extends AbstractIndexFieldData<AtomicPare
} }
@Override @Override
public void afterRemove(DocumentMapper mapper) { protected AtomicParentChildFieldData empty(int maxDoc) {
synchronized (lock) { return new ParentChildAtomicFieldData(ImmutableOpenMap.<String, AtomicOrdinalsFieldData>of());
ParentFieldMapper parentFieldMapper = mapper.parentFieldMapper();
if (parentFieldMapper.active()) {
parentTypes.remove(new BytesRef(parentFieldMapper.type()));
}
}
} }
class TypeBuilder { class TypeBuilder {
@ -345,8 +359,12 @@ public class ParentChildIndexFieldData extends AbstractIndexFieldData<AtomicPare
public IndexParentChildFieldData localGlobalDirect(IndexReader indexReader) throws Exception { public IndexParentChildFieldData localGlobalDirect(IndexReader indexReader) throws Exception {
final long startTime = System.nanoTime(); final long startTime = System.nanoTime();
final Set<String> parentTypes; final Set<String> parentTypes;
synchronized (lock) { if (Version.indexCreated(indexSettings).before(Version.V_2_0_0_beta1)) {
parentTypes = ImmutableSet.copyOf(this.parentTypes); synchronized (lock) {
parentTypes = ImmutableSet.copyOf(this.parentTypes);
}
} else {
parentTypes = this.parentTypes;
} }
long ramBytesUsed = 0; long ramBytesUsed = 0;

View File

@ -30,11 +30,4 @@ public interface DocumentTypeListener {
*/ */
void beforeCreate(DocumentMapper mapper); 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.action.fieldstats.FieldStats;
import org.elasticsearch.common.Nullable; import org.elasticsearch.common.Nullable;
import org.elasticsearch.common.lucene.BytesRefs; import org.elasticsearch.common.lucene.BytesRefs;
import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.common.unit.Fuzziness; import org.elasticsearch.common.unit.Fuzziness;
import org.elasticsearch.index.analysis.NamedAnalyzer; import org.elasticsearch.index.analysis.NamedAnalyzer;
import org.elasticsearch.index.fielddata.FieldDataType; import org.elasticsearch.index.fielddata.FieldDataType;
@ -186,6 +185,7 @@ public abstract class MappedFieldType extends FieldType {
fieldDataType = new FieldDataType(typeName()); fieldDataType = new FieldDataType(typeName());
} }
@Override
public abstract MappedFieldType clone(); public abstract MappedFieldType clone();
@Override @Override
@ -449,7 +449,7 @@ public abstract class MappedFieldType extends FieldType {
return new TermsQuery(names.indexName(), bytesRefs); 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(), return new TermRangeQuery(names().indexName(),
lowerTerm == null ? null : indexedValueForSearch(lowerTerm), lowerTerm == null ? null : indexedValueForSearch(lowerTerm),
upperTerm == null ? null : indexedValueForSearch(upperTerm), upperTerm == null ? null : indexedValueForSearch(upperTerm),

View File

@ -622,6 +622,10 @@ public class MapperService extends AbstractIndexComponent {
return META_FIELDS.contains(fieldName); 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 */ /** An analyzer wrapper that can lookup fields within the index mappings */
final class MapperAnalyzerWrapper extends DelegatingAnalyzerWrapper { 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.apache.lucene.util.NumericUtils;
import org.elasticsearch.action.fieldstats.FieldStats; import org.elasticsearch.action.fieldstats.FieldStats;
import org.elasticsearch.common.Explicit; import org.elasticsearch.common.Explicit;
import org.elasticsearch.common.Nullable;
import org.elasticsearch.common.Strings; import org.elasticsearch.common.Strings;
import org.elasticsearch.common.settings.Settings; import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.common.unit.Fuzziness; 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.Mapper;
import org.elasticsearch.index.mapper.MapperParsingException; import org.elasticsearch.index.mapper.MapperParsingException;
import org.elasticsearch.index.mapper.ParseContext; import org.elasticsearch.index.mapper.ParseContext;
import org.elasticsearch.index.query.QueryParseContext;
import java.io.IOException; import java.io.IOException;
import java.util.Iterator; import java.util.Iterator;
import java.util.List; import java.util.List;
@ -163,7 +160,7 @@ public class ByteFieldMapper extends NumberFieldMapper {
} }
@Override @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(), return NumericRangeQuery.newIntRange(names().indexName(), numericPrecisionStep(),
lowerTerm == null ? null : (int)parseValue(lowerTerm), lowerTerm == null ? null : (int)parseValue(lowerTerm),
upperTerm == null ? null : (int)parseValue(upperTerm), 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.MapperParsingException;
import org.elasticsearch.index.mapper.ParseContext; import org.elasticsearch.index.mapper.ParseContext;
import org.elasticsearch.index.mapper.core.LongFieldMapper.CustomLongNumericField; import org.elasticsearch.index.mapper.core.LongFieldMapper.CustomLongNumericField;
import org.elasticsearch.index.query.QueryParseContext;
import org.elasticsearch.search.internal.SearchContext; import org.elasticsearch.search.internal.SearchContext;
import org.joda.time.DateTimeZone; import org.joda.time.DateTimeZone;
import java.io.IOException; 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.Callable;
import java.util.concurrent.TimeUnit; import java.util.concurrent.TimeUnit;
@ -124,6 +127,7 @@ public class DateFieldMapper extends NumberFieldMapper {
return fieldMapper; return fieldMapper;
} }
@Override
protected void setupFieldType(BuilderContext context) { protected void setupFieldType(BuilderContext context) {
if (Version.indexCreated(context.indexSettings()).before(Version.V_2_0_0_beta1) && if (Version.indexCreated(context.indexSettings()).before(Version.V_2_0_0_beta1) &&
!fieldType().dateTimeFormatter().format().contains("epoch_")) { !fieldType().dateTimeFormatter().format().contains("epoch_")) {
@ -277,6 +281,7 @@ public class DateFieldMapper extends NumberFieldMapper {
this.dateMathParser = ref.dateMathParser; this.dateMathParser = ref.dateMathParser;
} }
@Override
public DateFieldType clone() { public DateFieldType clone() {
return new DateFieldType(this); return new DateFieldType(this);
} }
@ -390,8 +395,8 @@ public class DateFieldMapper extends NumberFieldMapper {
} }
@Override @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 rangeQuery(lowerTerm, upperTerm, includeLower, includeUpper, null, null, context); return rangeQuery(lowerTerm, upperTerm, includeLower, includeUpper, null, null);
} }
@Override @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); return new LateParsingQuery(lowerTerm, upperTerm, includeLower, includeUpper, timeZone, forcedDateParser);
} }

View File

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

View File

@ -20,6 +20,7 @@
package org.elasticsearch.index.mapper.core; package org.elasticsearch.index.mapper.core;
import com.carrotsearch.hppc.FloatArrayList; import com.carrotsearch.hppc.FloatArrayList;
import org.apache.lucene.analysis.Analyzer; import org.apache.lucene.analysis.Analyzer;
import org.apache.lucene.analysis.TokenStream; import org.apache.lucene.analysis.TokenStream;
import org.apache.lucene.document.Field; import org.apache.lucene.document.Field;
@ -32,7 +33,6 @@ import org.apache.lucene.util.BytesRefBuilder;
import org.apache.lucene.util.NumericUtils; import org.apache.lucene.util.NumericUtils;
import org.elasticsearch.action.fieldstats.FieldStats; import org.elasticsearch.action.fieldstats.FieldStats;
import org.elasticsearch.common.Explicit; import org.elasticsearch.common.Explicit;
import org.elasticsearch.common.Nullable;
import org.elasticsearch.common.Numbers; import org.elasticsearch.common.Numbers;
import org.elasticsearch.common.Strings; import org.elasticsearch.common.Strings;
import org.elasticsearch.common.settings.Settings; 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.Mapper;
import org.elasticsearch.index.mapper.MapperParsingException; import org.elasticsearch.index.mapper.MapperParsingException;
import org.elasticsearch.index.mapper.ParseContext; import org.elasticsearch.index.mapper.ParseContext;
import org.elasticsearch.index.query.QueryParseContext;
import java.io.IOException; import java.io.IOException;
import java.util.Iterator; import java.util.Iterator;
@ -170,7 +169,7 @@ public class FloatFieldMapper extends NumberFieldMapper {
} }
@Override @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(), return NumericRangeQuery.newFloatRange(names().indexName(), numericPrecisionStep(),
lowerTerm == null ? null : parseValue(lowerTerm), lowerTerm == null ? null : parseValue(lowerTerm),
upperTerm == null ? null : parseValue(upperTerm), 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.apache.lucene.util.NumericUtils;
import org.elasticsearch.action.fieldstats.FieldStats; import org.elasticsearch.action.fieldstats.FieldStats;
import org.elasticsearch.common.Explicit; import org.elasticsearch.common.Explicit;
import org.elasticsearch.common.Nullable;
import org.elasticsearch.common.Numbers; import org.elasticsearch.common.Numbers;
import org.elasticsearch.common.Strings; import org.elasticsearch.common.Strings;
import org.elasticsearch.common.settings.Settings; 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.Mapper;
import org.elasticsearch.index.mapper.MapperParsingException; import org.elasticsearch.index.mapper.MapperParsingException;
import org.elasticsearch.index.mapper.ParseContext; import org.elasticsearch.index.mapper.ParseContext;
import org.elasticsearch.index.query.QueryParseContext;
import java.io.IOException; import java.io.IOException;
import java.util.Iterator; import java.util.Iterator;
import java.util.List; import java.util.List;
@ -172,7 +169,7 @@ public class IntegerFieldMapper extends NumberFieldMapper {
} }
@Override @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(), return NumericRangeQuery.newIntRange(names().indexName(), numericPrecisionStep(),
lowerTerm == null ? null : parseValue(lowerTerm), lowerTerm == null ? null : parseValue(lowerTerm),
upperTerm == null ? null : parseValue(upperTerm), 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.apache.lucene.util.NumericUtils;
import org.elasticsearch.action.fieldstats.FieldStats; import org.elasticsearch.action.fieldstats.FieldStats;
import org.elasticsearch.common.Explicit; import org.elasticsearch.common.Explicit;
import org.elasticsearch.common.Nullable;
import org.elasticsearch.common.Numbers; import org.elasticsearch.common.Numbers;
import org.elasticsearch.common.Strings; import org.elasticsearch.common.Strings;
import org.elasticsearch.common.settings.Settings; 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.Mapper;
import org.elasticsearch.index.mapper.MapperParsingException; import org.elasticsearch.index.mapper.MapperParsingException;
import org.elasticsearch.index.mapper.ParseContext; import org.elasticsearch.index.mapper.ParseContext;
import org.elasticsearch.index.query.QueryParseContext;
import java.io.IOException; import java.io.IOException;
import java.util.Iterator; import java.util.Iterator;
import java.util.List; import java.util.List;
@ -170,7 +167,7 @@ public class LongFieldMapper extends NumberFieldMapper {
} }
@Override @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(), return NumericRangeQuery.newLongRange(names().indexName(), numericPrecisionStep(),
lowerTerm == null ? null : parseLongValue(lowerTerm), lowerTerm == null ? null : parseLongValue(lowerTerm),
upperTerm == null ? null : parseLongValue(upperTerm), 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.apache.lucene.util.NumericUtils;
import org.elasticsearch.action.fieldstats.FieldStats; import org.elasticsearch.action.fieldstats.FieldStats;
import org.elasticsearch.common.Explicit; import org.elasticsearch.common.Explicit;
import org.elasticsearch.common.Nullable;
import org.elasticsearch.common.Numbers; import org.elasticsearch.common.Numbers;
import org.elasticsearch.common.Strings; import org.elasticsearch.common.Strings;
import org.elasticsearch.common.settings.Settings; 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.Mapper;
import org.elasticsearch.index.mapper.MapperParsingException; import org.elasticsearch.index.mapper.MapperParsingException;
import org.elasticsearch.index.mapper.ParseContext; import org.elasticsearch.index.mapper.ParseContext;
import org.elasticsearch.index.query.QueryParseContext;
import java.io.IOException; import java.io.IOException;
import java.util.Iterator; import java.util.Iterator;
import java.util.List; import java.util.List;
@ -168,7 +165,7 @@ public class ShortFieldMapper extends NumberFieldMapper {
} }
@Override @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(), return NumericRangeQuery.newIntRange(names().indexName(), numericPrecisionStep(),
lowerTerm == null ? null : (int)parseValue(lowerTerm), lowerTerm == null ? null : (int)parseValue(lowerTerm),
upperTerm == null ? null : (int)parseValue(upperTerm), 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.BytesRefBuilder;
import org.apache.lucene.util.NumericUtils; import org.apache.lucene.util.NumericUtils;
import org.elasticsearch.common.Explicit; import org.elasticsearch.common.Explicit;
import org.elasticsearch.common.Nullable;
import org.elasticsearch.common.Numbers; import org.elasticsearch.common.Numbers;
import org.elasticsearch.common.Strings; import org.elasticsearch.common.Strings;
import org.elasticsearch.common.settings.Settings; 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;
import org.elasticsearch.index.mapper.core.LongFieldMapper.CustomLongNumericField; import org.elasticsearch.index.mapper.core.LongFieldMapper.CustomLongNumericField;
import org.elasticsearch.index.mapper.core.NumberFieldMapper; import org.elasticsearch.index.mapper.core.NumberFieldMapper;
import org.elasticsearch.index.query.QueryParseContext;
import java.io.IOException; import java.io.IOException;
import java.util.Iterator; import java.util.Iterator;
import java.util.List; import java.util.List;
@ -209,7 +206,7 @@ public class IpFieldMapper extends NumberFieldMapper {
} }
@Override @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(), return NumericRangeQuery.newLongRange(names().indexName(), numericPrecisionStep(),
lowerTerm == null ? null : parseValue(lowerTerm), lowerTerm == null ? null : parseValue(lowerTerm),
upperTerm == null ? null : parseValue(upperTerm), 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 { 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 _field_names are not indexed, we need to go the slow way
if (filter == null && fieldType != null) { if (filter == null && fieldType != null) {
filter = fieldType.rangeQuery(null, null, true, true, parseContext); filter = fieldType.rangeQuery(null, null, true, true);
} }
if (filter == null) { if (filter == null) {
filter = new TermRangeQuery(field, null, null, true, true); filter = new TermRangeQuery(field, null, null, true, true);

View File

@ -55,11 +55,11 @@ public class MissingQueryBuilder extends AbstractQueryBuilder<MissingQueryBuilde
public MissingQueryBuilder(String fieldPattern) { public MissingQueryBuilder(String fieldPattern) {
this.fieldPattern = fieldPattern; this.fieldPattern = fieldPattern;
} }
public String fieldPattern() { public String fieldPattern() {
return this.fieldPattern; return this.fieldPattern;
} }
/** /**
* Should the missing filter automatically include fields with null value configured in the * Should the missing filter automatically include fields with null value configured in the
* mappings. Defaults to <tt>false</tt>. * 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 _field_names are not indexed, we need to go the slow way
if (filter == null && fieldType != null) { if (filter == null && fieldType != null) {
filter = fieldType.rangeQuery(null, null, true, true, parseContext); filter = fieldType.rangeQuery(null, null, true, true);
} }
if (filter == null) { if (filter == null) {
filter = new TermRangeQuery(field, null, null, true, true); 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) { if (this.timeZone != null) {
dateTimeZone = DateTimeZone.forID(this.timeZone); 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 { } else {
if (timeZone != null) { if (timeZone != null) {
throw new QueryParsingException(parseContext, "[range] time_zone can not be applied to non date field [" throw new QueryParsingException(parseContext, "[range] time_zone can not be applied to non date field ["
+ fieldName + "]"); + fieldName + "]");
} }
//LUCENE 4 UPGRADE Mapper#rangeQuery should use bytesref as well? //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 { } else {
if (timeZone != null) { if (timeZone != null) {

View File

@ -45,16 +45,16 @@ public class IndexedGeoBoundingBoxQuery {
private static Query westGeoBoundingBoxFilter(GeoPoint topLeft, GeoPoint bottomRight, GeoPointFieldMapper.GeoPointFieldType fieldType) { private static Query westGeoBoundingBoxFilter(GeoPoint topLeft, GeoPoint bottomRight, GeoPointFieldMapper.GeoPointFieldType fieldType) {
BooleanQuery filter = new BooleanQuery(); BooleanQuery filter = new BooleanQuery();
filter.setMinimumNumberShouldMatch(1); filter.setMinimumNumberShouldMatch(1);
filter.add(fieldType.lonFieldType().rangeQuery(null, bottomRight.lon(), true, true, null), Occur.SHOULD); filter.add(fieldType.lonFieldType().rangeQuery(null, bottomRight.lon(), true, true), Occur.SHOULD);
filter.add(fieldType.lonFieldType().rangeQuery(topLeft.lon(), null, true, true, null), 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, null), Occur.MUST); filter.add(fieldType.latFieldType().rangeQuery(bottomRight.lat(), topLeft.lat(), true, true), Occur.MUST);
return new ConstantScoreQuery(filter); return new ConstantScoreQuery(filter);
} }
private static Query eastGeoBoundingBoxFilter(GeoPoint topLeft, GeoPoint bottomRight, GeoPointFieldMapper.GeoPointFieldType fieldType) { private static Query eastGeoBoundingBoxFilter(GeoPoint topLeft, GeoPoint bottomRight, GeoPointFieldMapper.GeoPointFieldType fieldType) {
BooleanQuery filter = new BooleanQuery(); BooleanQuery filter = new BooleanQuery();
filter.add(fieldType.lonFieldType().rangeQuery(topLeft.lon(), bottomRight.lon(), 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, null), Occur.MUST); filter.add(fieldType.latFieldType().rangeQuery(bottomRight.lat(), topLeft.lat(), true, true), Occur.MUST);
return new ConstantScoreQuery(filter); 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.Charsets;
import com.google.common.base.Preconditions; import com.google.common.base.Preconditions;
import org.apache.lucene.codecs.PostingsFormat; import org.apache.lucene.codecs.PostingsFormat;
import org.apache.lucene.index.CheckIndex; import org.apache.lucene.index.CheckIndex;
import org.apache.lucene.search.QueryCachingPolicy; import org.apache.lucene.search.QueryCachingPolicy;
@ -169,6 +168,7 @@ public class IndexShard extends AbstractIndexShardComponent {
protected volatile IndexShardState state; protected volatile IndexShardState state;
protected final AtomicReference<Engine> currentEngineReference = new AtomicReference<>(); protected final AtomicReference<Engine> currentEngineReference = new AtomicReference<>();
protected final EngineFactory engineFactory; protected final EngineFactory engineFactory;
private final IndexSearcherWrappingService wrappingService;
@Nullable @Nullable
private RecoveryState recoveryState; private RecoveryState recoveryState;
@ -198,12 +198,13 @@ public class IndexShard extends AbstractIndexShardComponent {
IndicesQueryCache indicesQueryCache, ShardPercolateService shardPercolateService, CodecService codecService, IndicesQueryCache indicesQueryCache, ShardPercolateService shardPercolateService, CodecService codecService,
ShardTermVectorsService termVectorsService, IndexFieldDataService indexFieldDataService, IndexService indexService, ShardTermVectorsService termVectorsService, IndexFieldDataService indexFieldDataService, IndexService indexService,
@Nullable IndicesWarmer warmer, SnapshotDeletionPolicy deletionPolicy, SimilarityService similarityService, EngineFactory factory, @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()); super(shardId, indexSettingsService.getSettings());
this.codecService = codecService; this.codecService = codecService;
this.warmer = warmer; this.warmer = warmer;
this.deletionPolicy = deletionPolicy; this.deletionPolicy = deletionPolicy;
this.similarityService = similarityService; this.similarityService = similarityService;
this.wrappingService = wrappingService;
Preconditions.checkNotNull(store, "Store must be provided to the index shard"); Preconditions.checkNotNull(store, "Store must be provided to the index shard");
Preconditions.checkNotNull(deletionPolicy, "Snapshot deletion policy must be provided to the index shard"); Preconditions.checkNotNull(deletionPolicy, "Snapshot deletion policy must be provided to the index shard");
this.engineFactory = factory; this.engineFactory = factory;
@ -337,14 +338,16 @@ public class IndexShard extends AbstractIndexShardComponent {
if (!newRouting.shardId().equals(shardId())) { if (!newRouting.shardId().equals(shardId())) {
throw new IllegalArgumentException("Trying to set a routing entry with shardId [" + newRouting.shardId() + "] on a shard with shardId [" + 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 { try {
if (currentRouting != null) { if (currentRouting != null) {
assert newRouting.version() > currentRouting.version() : "expected: " + newRouting.version() + " > " + currentRouting.version();
if (!newRouting.primary() && currentRouting.primary()) { if (!newRouting.primary() && currentRouting.primary()) {
logger.warn("suspect illegal state: trying to move shard from primary mode to replica mode"); logger.warn("suspect illegal state: trying to move shard from primary mode to replica mode");
} }
// if its the same routing, return // if its the same routing except for some metadata info, return
if (currentRouting.equals(newRouting)) { if (currentRouting.equalsIgnoringMetaData(newRouting)) {
this.shardRouting = newRouting; // might have a new version this.shardRouting = newRouting; // might have a new version
return; return;
} }
@ -723,12 +726,12 @@ public class IndexShard extends AbstractIndexShardComponent {
public org.apache.lucene.util.Version minimumCompatibleVersion() { public org.apache.lucene.util.Version minimumCompatibleVersion() {
org.apache.lucene.util.Version luceneVersion = null; 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())) { if (luceneVersion == null || luceneVersion.onOrAfter(segment.getVersion())) {
luceneVersion = 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 { 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()); 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); logger.info("updating [{}] from [{}] to [{}]", MergeSchedulerConfig.MAX_MERGE_COUNT, mergeSchedulerConfig.getMaxMergeCount(), maxMergeCount);
mergeSchedulerConfig.setMaxMergeCount(maxMergeCount); mergeSchedulerConfig.setMaxMergeCount(maxMergeCount);
change = true; change = true;
@ -1360,7 +1363,7 @@ public class IndexShard extends AbstractIndexShardComponent {
}; };
return new EngineConfig(shardId, return new EngineConfig(shardId,
threadPool, indexingService, indexSettingsService.indexSettings(), warmer, store, deletionPolicy, mergePolicyConfig.getMergePolicy(), mergeSchedulerConfig, 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 { 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.cluster.metadata.IndexMetaData;
import org.elasticsearch.common.inject.AbstractModule; import org.elasticsearch.common.inject.AbstractModule;
import org.elasticsearch.common.inject.multibindings.Multibinder;
import org.elasticsearch.common.settings.Settings; 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.EngineFactory;
import org.elasticsearch.index.engine.InternalEngineFactory; import org.elasticsearch.index.engine.InternalEngineFactory;
import org.elasticsearch.index.percolator.stats.ShardPercolateService; import org.elasticsearch.index.percolator.stats.ShardPercolateService;
@ -73,6 +76,10 @@ public class IndexShardModule extends AbstractModule {
bind(StoreRecoveryService.class).asEagerSingleton(); bind(StoreRecoveryService.class).asEagerSingleton();
bind(ShardPercolateService.class).asEagerSingleton(); bind(ShardPercolateService.class).asEagerSingleton();
bind(ShardTermVectorsService.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.cache.IndexCache;
import org.elasticsearch.index.codec.CodecService; import org.elasticsearch.index.codec.CodecService;
import org.elasticsearch.index.deletionpolicy.SnapshotDeletionPolicy; import org.elasticsearch.index.deletionpolicy.SnapshotDeletionPolicy;
import org.elasticsearch.index.engine.IndexSearcherWrappingService;
import org.elasticsearch.index.engine.Engine; import org.elasticsearch.index.engine.Engine;
import org.elasticsearch.index.engine.EngineConfig; import org.elasticsearch.index.engine.EngineConfig;
import org.elasticsearch.index.engine.EngineFactory; import org.elasticsearch.index.engine.EngineFactory;
@ -66,13 +67,13 @@ public final class ShadowIndexShard extends IndexShard {
IndexService indexService, @Nullable IndicesWarmer warmer, IndexService indexService, @Nullable IndicesWarmer warmer,
SnapshotDeletionPolicy deletionPolicy, SimilarityService similarityService, SnapshotDeletionPolicy deletionPolicy, SimilarityService similarityService,
EngineFactory factory, ClusterService clusterService, EngineFactory factory, ClusterService clusterService,
ShardPath path, BigArrays bigArrays) throws IOException { ShardPath path, BigArrays bigArrays, IndexSearcherWrappingService wrappingService) throws IOException {
super(shardId, indexSettingsService, indicesLifecycle, store, storeRecoveryService, super(shardId, indexSettingsService, indicesLifecycle, store, storeRecoveryService,
threadPool, mapperService, queryParserService, indexCache, indexAliasesService, threadPool, mapperService, queryParserService, indexCache, indexAliasesService,
indicesQueryCache, shardPercolateService, codecService, indicesQueryCache, shardPercolateService, codecService,
termVectorsService, indexFieldDataService, indexService, termVectorsService, indexFieldDataService, indexService,
warmer, deletionPolicy, similarityService, 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) { if (reader instanceof ElasticsearchLeafReader) {
return (ElasticsearchLeafReader) reader; return (ElasticsearchLeafReader) reader;
} else { } 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; 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, // 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 // or a replica allocated and then allocating a primary because the primary failed on another node
boolean shardHasBeenRemoved = false; 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); 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. // closing the shard will also cancel any ongoing recovery.
indexService.removeShard(shardRouting.id(), "removing shard (different instance of it allocated on this node)"); 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. // closing the shard will also cancel any ongoing recovery.
indexService.removeShard(shardRouting.id(), "removing shard (recovery source node changed)"); indexService.removeShard(shardRouting.id(), "removing shard (recovery source node changed)");
shardHasBeenRemoved = true; 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) { if (shardHasBeenRemoved == false) {
logger.debug("{} reinitialize shard on primary promotion", indexShard.shardId()); // shadow replicas do not support primary promotion. The master would reinitialize the shard, giving it a new allocation, meaning we should be there.
indexService.removeShard(shardId, "promoted to primary"); assert (shardRouting.primary() && currentRoutingEntry.primary() == false) == false || indexShard.allowsPrimaryPromotion() :
} else { "shard for doesn't support primary promotion but master promoted it with changing allocation. New routing " + shardRouting + ", current routing " + currentRoutingEntry;
// 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);
indexShard.updateRoutingEntry(shardRouting, event.state().blocks().disableStatePersistence() == false);
}
} }
} }
if (shardRouting.initializing()) { 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) { private void purgeShards(List<IndexShard> shardsToPurge) {
for (IndexShard shardToPurge : 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"); Engine.Searcher searcher = shardToPurge.acquireSearcher("indices_ttl");
try { try {
logger.debug("[{}][{}] purging shard", shardToPurge.routingEntry().index(), shardToPurge.routingEntry().id()); 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) { static Settings replacePromptPlaceholders(Settings settings, Terminal terminal) {
UnmodifiableIterator<Map.Entry<String, String>> iter = settings.getAsMap().entrySet().iterator(); 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()) { while (iter.hasNext()) {
Map.Entry<String, String> entry = iter.next(); Map.Entry<String, String> entry = iter.next();

View File

@ -19,6 +19,7 @@
package org.elasticsearch.plugins; package org.elasticsearch.plugins;
import org.elasticsearch.Version; import org.elasticsearch.Version;
import org.elasticsearch.bootstrap.JarHell;
import org.elasticsearch.common.io.stream.StreamInput; import org.elasticsearch.common.io.stream.StreamInput;
import org.elasticsearch.common.io.stream.StreamOutput; import org.elasticsearch.common.io.stream.StreamOutput;
import org.elasticsearch.common.io.stream.Streamable; import org.elasticsearch.common.io.stream.Streamable;
@ -110,12 +111,23 @@ public class PluginInfo implements Streamable, ToXContent {
if (esVersion.equals(Version.CURRENT) == false) { if (esVersion.equals(Version.CURRENT) == false) {
throw new IllegalArgumentException("Elasticsearch version [" + esVersionString + "] is too old for plugin [" + name + "]"); 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")); isolated = Boolean.parseBoolean(props.getProperty("isolated", "true"));
classname = props.getProperty("classname"); classname = props.getProperty("classname");
if (classname == null) { if (classname == null) {
throw new IllegalArgumentException("Property [classname] is missing for jvm plugin [" + name + "]"); 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); 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.base.Strings;
import com.google.common.collect.ImmutableSet; import com.google.common.collect.ImmutableSet;
import com.google.common.collect.Iterators; import com.google.common.collect.Iterators;
import org.apache.lucene.util.IOUtils; import org.apache.lucene.util.IOUtils;
import org.elasticsearch.ElasticsearchTimeoutException; import org.elasticsearch.ElasticsearchTimeoutException;
import org.elasticsearch.ExceptionsHelper; 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.io.FileSystemUtils;
import org.elasticsearch.common.unit.TimeValue; import org.elasticsearch.common.unit.TimeValue;
import org.elasticsearch.env.Environment; import org.elasticsearch.env.Environment;
import org.elasticsearch.plugins.PluginsService.Bundle;
import java.io.IOException; import java.io.IOException;
import java.io.OutputStream; import java.io.OutputStream;
@ -98,14 +100,6 @@ public class PluginManager {
if (name == null) { if (name == null) {
throw new IllegalArgumentException("plugin name must be supplied with install [name]."); 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())) { if (!Files.exists(environment.pluginsFile())) {
terminal.println("Plugins directory [%s] does not exist. Creating...", 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); PluginHandle pluginHandle = PluginHandle.parse(name);
checkForForbiddenName(pluginHandle.name); checkForForbiddenName(pluginHandle.name);
Path pluginFile = pluginHandle.distroFile(environment); Path pluginFile = download(pluginHandle, terminal);
// extract the plugin extract(pluginHandle, terminal, pluginFile);
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"); 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 // first, try directly from the URL provided
@ -162,97 +165,44 @@ public class PluginManager {
} }
if (!downloaded) { 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"); 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 private void extract(PluginHandle pluginHandle, Terminal terminal, Path pluginFile) throws IOException {
Path tmp = unzipToTemporary(pluginFile); final Path extractLocation = pluginHandle.extractedDir(environment);
if (Files.exists(extractLocation)) {
// create list of current jars in classpath throw new IOException("plugin directory " + extractLocation.toAbsolutePath() + " already exists. To update the plugin, uninstall it first using 'remove " + pluginHandle.name + "' command");
final List<URL> jars = new ArrayList<>();
ClassLoader loader = PluginManager.class.getClassLoader();
if (loader instanceof URLClassLoader) {
Collections.addAll(jars, ((URLClassLoader) loader).getURLs());
} }
// add any jars we find in the plugin to the list // unzip plugin to a staging temp dir, named for the plugin
Files.walkFileTree(tmp, new SimpleFileVisitor<Path>() { Path tmp = Files.createTempDirectory(environment.tmpFile(), null);
@Override Path root = tmp.resolve(pluginHandle.name);
public FileVisitResult visitFile(Path file, BasicFileAttributes attrs) throws IOException { unzipPlugin(pluginFile, root);
if (file.toString().endsWith(".jar")) {
jars.add(file.toUri().toURL());
}
return FileVisitResult.CONTINUE;
}
});
// check combined (current classpath + new jars to-be-added) // find the actual root (in case its unzipped with extra directory wrapping)
try { root = findPluginRoot(root);
JarHell.checkJarHell(jars.toArray(new URL[jars.size()]));
} catch (Exception ex) { // read and validate the plugin descriptor
throw new RuntimeException(ex); 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 // cleanup
IOUtils.rm(tmp); IOUtils.rm(tmp, pluginFile);
// TODO: we have a tmpdir made above, so avoid zipfilesystem // take care of bin/ by moving and applying permissions if needed
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;
Path binFile = extractLocation.resolve("bin"); Path binFile = extractLocation.resolve("bin");
if (Files.isDirectory(binFile)) { if (Files.isDirectory(binFile)) {
Path toLocation = pluginHandle.binDir(environment); Path toLocation = pluginHandle.binDir(environment);
@ -289,8 +239,7 @@ public class PluginManager {
} else { } else {
terminal.println(VERBOSE, "Skipping posix permissions - filestore doesn't support posix permission"); terminal.println(VERBOSE, "Skipping posix permissions - filestore doesn't support posix permission");
} }
terminal.println(VERBOSE, "Installed %s into %s", name, toLocation.toAbsolutePath()); terminal.println(VERBOSE, "Installed %s into %s", pluginHandle.name, toLocation.toAbsolutePath());
potentialSitePlugin = false;
} }
Path configFile = extractLocation.resolve("config"); Path configFile = extractLocation.resolve("config");
@ -298,33 +247,68 @@ public class PluginManager {
Path configDestLocation = pluginHandle.configDir(environment); Path configDestLocation = pluginHandle.configDir(environment);
terminal.println(VERBOSE, "Found config, moving to %s", configDestLocation.toAbsolutePath()); terminal.println(VERBOSE, "Found config, moving to %s", configDestLocation.toAbsolutePath());
moveFilesWithoutOverwriting(configFile, configDestLocation, ".new"); moveFilesWithoutOverwriting(configFile, configDestLocation, ".new");
terminal.println(VERBOSE, "Installed %s into %s", name, configDestLocation.toAbsolutePath()); terminal.println(VERBOSE, "Installed %s into %s", pluginHandle.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());
}
} }
} }
private Path unzipToTemporary(Path zip) throws IOException { /** we check whether we need to remove the top-level folder while extracting
Path tmp = Files.createTempDirectory(environment.tmpFile(), null); * 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))) { try (ZipInputStream zipInput = new ZipInputStream(Files.newInputStream(zip))) {
ZipEntry entry; ZipEntry entry;
byte[] buffer = new byte[8192]; byte[] buffer = new byte[8192];
while ((entry = zipInput.getNextEntry()) != null) { 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 // 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?) // before their children (although this makes sense, but is it guaranteed?)
@ -340,8 +324,6 @@ public class PluginManager {
zipInput.closeEntry(); zipInput.closeEntry();
} }
} }
return tmp;
} }
public void removePlugin(String name, Terminal terminal) throws IOException { public void removePlugin(String name, Terminal terminal) throws IOException {
@ -363,17 +345,6 @@ public class PluginManager {
} }
removed = true; 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); Path binLocation = pluginHandle.binDir(environment);
if (Files.exists(binLocation)) { if (Files.exists(binLocation)) {
terminal.println(VERBOSE, "Removing: %s", binLocation); terminal.println(VERBOSE, "Removing: %s", binLocation);
@ -482,8 +453,8 @@ public class PluginManager {
} }
} }
Path distroFile(Environment env) { Path newDistroFile(Environment env) throws IOException {
return env.pluginsFile().resolve(name + ".zip"); return Files.createTempFile(env.tmpFile(), name, ".zip");
} }
Path extractedDir(Environment env) { Path extractedDir(Environment env) {

View File

@ -60,7 +60,6 @@ public class RestAnalyzeAction extends BaseRestHandler {
AnalyzeRequest analyzeRequest = new AnalyzeRequest(request.param("index")); AnalyzeRequest analyzeRequest = new AnalyzeRequest(request.param("index"));
analyzeRequest.text(texts); analyzeRequest.text(texts);
analyzeRequest.preferLocal(request.paramAsBoolean("prefer_local", analyzeRequest.preferLocalShard()));
analyzeRequest.analyzer(request.param("analyzer")); analyzeRequest.analyzer(request.param("analyzer"));
analyzeRequest.field(request.param("field")); analyzeRequest.field(request.param("field"));
analyzeRequest.tokenizer(request.param("tokenizer")); analyzeRequest.tokenizer(request.param("tokenizer"));
@ -93,8 +92,6 @@ public class RestAnalyzeAction extends BaseRestHandler {
while ((token = parser.nextToken()) != XContentParser.Token.END_OBJECT) { while ((token = parser.nextToken()) != XContentParser.Token.END_OBJECT) {
if (token == XContentParser.Token.FIELD_NAME) { if (token == XContentParser.Token.FIELD_NAME) {
currentFieldName = parser.currentName(); 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) { } else if ("text".equals(currentFieldName) && token == XContentParser.Token.VALUE_STRING) {
analyzeRequest.text(parser.text()); analyzeRequest.text(parser.text());
} else if ("text".equals(currentFieldName) && token == XContentParser.Token.START_ARRAY) { } 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 * Defaults to 30 seconds
*/ */
MEDIUM(TimeValue.timeValueSeconds(25)), MEDIUM(TimeValue.timeValueSeconds(30)),
/** /**
* Defaults to 60 seconds * Defaults to 60 seconds

View File

@ -302,8 +302,8 @@ public class TermVectorsUnitTests extends ElasticsearchTestCase {
request = new MultiTermVectorsRequest(); request = new MultiTermVectorsRequest();
request.add(new TermVectorsRequest(), bytes); request.add(new TermVectorsRequest(), bytes);
checkParsedParameters(request); checkParsedParameters(request);
} }
void checkParsedParameters(MultiTermVectorsRequest request) { void checkParsedParameters(MultiTermVectorsRequest request) {
Set<String> ids = new HashSet<>(); Set<String> ids = new HashSet<>();
ids.add("1"); ids.add("1");
@ -324,5 +324,31 @@ public class TermVectorsUnitTests extends ElasticsearchTestCase {
assertThat(singleRequest.selectedFields(), equalTo(fields)); 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.ClusterState;
import org.elasticsearch.cluster.metadata.IndexMetaData; import org.elasticsearch.cluster.metadata.IndexMetaData;
import org.elasticsearch.cluster.metadata.MetaData; 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.BytesStreamOutput;
import org.elasticsearch.common.io.stream.StreamInput; import org.elasticsearch.common.io.stream.StreamInput;
import org.elasticsearch.test.ElasticsearchTestCase; 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() { public void testFrozenOnRoutingTable() {
MetaData metaData = MetaData.builder() MetaData metaData = MetaData.builder()

View File

@ -19,6 +19,8 @@
package org.elasticsearch.cluster.routing; 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 * A helper that allows to create shard routing instances within tests, while not requiring to expose
* different simplified constructors on the ShardRouting itself. * different simplified constructors on the ShardRouting itself.
@ -26,19 +28,19 @@ package org.elasticsearch.cluster.routing;
public class TestShardRouting { public class TestShardRouting {
public static ShardRouting newShardRouting(String index, int shardId, String currentNodeId, boolean primary, ShardRoutingState state, long version) { 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) { 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) { 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) { 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, public static ShardRouting newShardRouting(String index, int shardId, String currentNodeId,
@ -61,4 +63,17 @@ public class TestShardRouting {
throw new IllegalStateException("illegal state"); 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.ALLOCATION_FAILED,
UnassignedInfo.Reason.NODE_LEFT, UnassignedInfo.Reason.NODE_LEFT,
UnassignedInfo.Reason.REROUTE_CANCELLED, UnassignedInfo.Reason.REROUTE_CANCELLED,
UnassignedInfo.Reason.REINITIALIZED}; UnassignedInfo.Reason.REINITIALIZED,
UnassignedInfo.Reason.REALLOCATED_REPLICA};
for (int i = 0; i < order.length; i++) { for (int i = 0; i < order.length; i++) {
assertThat(order[i].ordinal(), equalTo(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())); 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) { private RoutingAllocation onePrimaryOnNode1And1Replica(AllocationDeciders deciders) {
return onePrimaryOnNode1And1Replica(deciders, Settings.EMPTY, UnassignedInfo.Reason.INDEX_CREATED); 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); 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 { class TestAllocator extends ReplicaShardAllocator {
private Map<DiscoveryNode, TransportNodesListShardStoreMetaData.StoreFilesMetaData> data = null; 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) { protected InternalEngine createEngine(Store store, Path translogPath, IndexSearcherWrapper... wrappers) {
return createEngine(defaultSettings, store, translogPath, new MergeSchedulerConfig(defaultSettings), newMergePolicy()); return createEngine(defaultSettings, store, translogPath, new MergeSchedulerConfig(defaultSettings), newMergePolicy(), wrappers);
} }
protected InternalEngine createEngine(Settings indexSettings, Store store, Path translogPath, MergeSchedulerConfig mergeSchedulerConfig, MergePolicy mergePolicy) { protected InternalEngine createEngine(Settings indexSettings, Store store, Path translogPath, MergeSchedulerConfig mergeSchedulerConfig, MergePolicy mergePolicy, IndexSearcherWrapper... wrappers) {
return new InternalEngine(config(indexSettings, store, translogPath, mergeSchedulerConfig, mergePolicy), false); 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(); IndexWriterConfig iwc = newIndexWriterConfig();
TranslogConfig translogConfig = new TranslogConfig(shardId, translogPath, indexSettings, Translog.Durabilty.REQUEST, BigArrays.NON_RECYCLING_INSTANCE, threadPool); 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) { public void onFailedEngine(ShardId shardId, String reason, @Nullable Throwable t) {
// we don't need to notify anybody in this test // 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; 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 @Test
public void testSimpleOperations() throws Exception { public void testSimpleOperations() throws Exception {
Engine.Searcher searchResult = engine.acquireSearcher("test"); 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() EngineConfig brokenConfig = new EngineConfig(shardId, threadPool, config.getIndexingService(), config.getIndexSettings()
, null, store, createSnapshotDeletionPolicy(), newMergePolicy(), config.getMergeSchedulerConfig(), , null, store, createSnapshotDeletionPolicy(), newMergePolicy(), config.getMergeSchedulerConfig(),
config.getAnalyzer(), config.getSimilarity(), new CodecService(shardId.index()), config.getFailedEngineListener() 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 { try {
new InternalEngine(brokenConfig, false); 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.ParsedDocument;
import org.elasticsearch.index.mapper.internal.SourceFieldMapper; import org.elasticsearch.index.mapper.internal.SourceFieldMapper;
import org.elasticsearch.index.mapper.internal.UidFieldMapper; import org.elasticsearch.index.mapper.internal.UidFieldMapper;
import org.elasticsearch.index.settings.IndexSettingsService;
import org.elasticsearch.index.shard.MergeSchedulerConfig; import org.elasticsearch.index.shard.MergeSchedulerConfig;
import org.elasticsearch.index.shard.ShardId; import org.elasticsearch.index.shard.ShardId;
import org.elasticsearch.index.shard.ShardUtils; import org.elasticsearch.index.shard.ShardUtils;
@ -226,7 +225,7 @@ public class ShadowEngineTests extends ElasticsearchTestCase {
@Override @Override
public void onFailedEngine(ShardId shardId, String reason, @Nullable Throwable t) { public void onFailedEngine(ShardId shardId, String reason, @Nullable Throwable t) {
// we don't need to notify anybody in this test // 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; return config;
} }

View File

@ -19,21 +19,21 @@
package org.elasticsearch.index.fielddata; 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.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.index.*;
import org.apache.lucene.search.Filter; import org.apache.lucene.search.Filter;
import org.apache.lucene.store.RAMDirectory; import org.apache.lucene.store.RAMDirectory;
import org.elasticsearch.common.settings.Settings; 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.ContentPath;
import org.elasticsearch.index.mapper.FieldMapper;
import org.elasticsearch.index.mapper.MappedFieldType; import org.elasticsearch.index.mapper.MappedFieldType;
import org.elasticsearch.index.mapper.Mapper.BuilderContext; import org.elasticsearch.index.mapper.Mapper.BuilderContext;
import org.elasticsearch.index.mapper.MapperBuilders; import org.elasticsearch.index.mapper.MapperBuilders;
import org.elasticsearch.index.mapper.MapperService; import org.elasticsearch.index.mapper.MapperService;
import org.elasticsearch.index.IndexService;
import org.elasticsearch.index.mapper.internal.ParentFieldMapper; import org.elasticsearch.index.mapper.internal.ParentFieldMapper;
import org.elasticsearch.indices.fielddata.cache.IndicesFieldDataCache; import org.elasticsearch.indices.fielddata.cache.IndicesFieldDataCache;
import org.elasticsearch.test.ElasticsearchSingleNodeTest; import org.elasticsearch.test.ElasticsearchSingleNodeTest;
@ -41,6 +41,9 @@ import org.junit.After;
import org.junit.Before; import org.junit.Before;
import static org.elasticsearch.index.fielddata.IndexFieldData.XFieldComparatorSource.Nested; 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 { 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)); 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) { private int[] getNumbers(Random random, int margin) {
if (random.nextInt(20) == 0) { if (random.nextInt(20) == 0) {
int[] num = new int[1 + random.nextInt(10)]; 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.*; import static org.hamcrest.Matchers.*;
public class SimpleDateMappingTests extends ElasticsearchSingleNodeTest { public class SimpleDateMappingTests extends ElasticsearchSingleNodeTest {
public void testAutomaticDateParser() throws Exception { public void testAutomaticDateParser() throws Exception {
String mapping = XContentFactory.jsonBuilder().startObject().startObject("type") String mapping = XContentFactory.jsonBuilder().startObject().startObject("type")
.startObject("properties").endObject() .startObject("properties").endObject()
@ -93,12 +93,12 @@ public class SimpleDateMappingTests extends ElasticsearchSingleNodeTest {
fieldMapper = defaultMapper.mappers().smartNameFieldMapper("wrong_date3"); fieldMapper = defaultMapper.mappers().smartNameFieldMapper("wrong_date3");
assertThat(fieldMapper, instanceOf(StringFieldMapper.class)); assertThat(fieldMapper, instanceOf(StringFieldMapper.class));
} }
public void testParseLocal() { public void testParseLocal() {
assertThat(Locale.GERMAN, equalTo(LocaleUtils.parse("de"))); assertThat(Locale.GERMAN, equalTo(LocaleUtils.parse("de")));
assertThat(Locale.GERMANY, equalTo(LocaleUtils.parse("de_DE"))); assertThat(Locale.GERMANY, equalTo(LocaleUtils.parse("de_DE")));
assertThat(new Locale("de","DE","DE"), equalTo(LocaleUtils.parse("de_DE_DE"))); assertThat(new Locale("de","DE","DE"), equalTo(LocaleUtils.parse("de_DE_DE")));
try { try {
LocaleUtils.parse("de_DE_DE_DE"); LocaleUtils.parse("de_DE_DE_DE");
fail(); fail();
@ -108,7 +108,7 @@ public class SimpleDateMappingTests extends ElasticsearchSingleNodeTest {
assertThat(Locale.ROOT, equalTo(LocaleUtils.parse(""))); assertThat(Locale.ROOT, equalTo(LocaleUtils.parse("")));
assertThat(Locale.ROOT, equalTo(LocaleUtils.parse("ROOT"))); assertThat(Locale.ROOT, equalTo(LocaleUtils.parse("ROOT")));
} }
public void testLocale() throws IOException { public void testLocale() throws IOException {
assumeFalse("Locals are buggy on JDK9EA", Constants.JRE_IS_MINIMUM_JAVA9 && systemPropertyAsBoolean("tests.security.manager", false)); assumeFalse("Locals are buggy on JDK9EA", Constants.JRE_IS_MINIMUM_JAVA9 && systemPropertyAsBoolean("tests.security.manager", false));
String mapping = XContentFactory.jsonBuilder() 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 { 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(fieldA).tokenStream(defaultMapper.mappers().indexAnalyzer(), null), notNullValue());
assertThat(doc.rootDoc().getField(fieldB).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 tokenStream = doc.rootDoc().getField(fieldA).tokenStream(defaultMapper.mappers().indexAnalyzer(), null);
tokenStream.reset(); tokenStream.reset();
NumericTermAttribute nta = tokenStream.addAttribute(NumericTermAttribute.class); NumericTermAttribute nta = tokenStream.addAttribute(NumericTermAttribute.class);
@ -177,7 +177,7 @@ public class SimpleDateMappingTests extends ElasticsearchSingleNodeTest {
while(tokenStream.incrementToken()) { while(tokenStream.incrementToken()) {
values.add(nta.getRawValue()); values.add(nta.getRawValue());
} }
tokenStream = doc.rootDoc().getField(fieldB).tokenStream(defaultMapper.mappers().indexAnalyzer(), null); tokenStream = doc.rootDoc().getField(fieldB).tokenStream(defaultMapper.mappers().indexAnalyzer(), null);
tokenStream.reset(); tokenStream.reset();
nta = tokenStream.addAttribute(NumericTermAttribute.class); 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"), equalTo("1262304000000"));
assertThat(doc.rootDoc().get("date_field_x"), equalTo("2010-01-01")); assertThat(doc.rootDoc().get("date_field_x"), equalTo("2010-01-01"));
} }
public void testHourFormat() throws Exception { public void testHourFormat() throws Exception {
String mapping = XContentFactory.jsonBuilder().startObject().startObject("type") String mapping = XContentFactory.jsonBuilder().startObject().startObject("type")
.field("date_detection", false) .field("date_detection", false)
@ -242,14 +242,14 @@ public class SimpleDateMappingTests extends ElasticsearchSingleNodeTest {
NumericRangeQuery<Long> rangeQuery; NumericRangeQuery<Long> rangeQuery;
try { try {
SearchContext.setCurrent(new TestSearchContext()); 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 { } finally {
SearchContext.removeCurrent(); SearchContext.removeCurrent();
} }
assertThat(rangeQuery.getMax(), equalTo(new DateTime(TimeValue.timeValueHours(11).millis(), DateTimeZone.UTC).getMillis())); 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())); assertThat(rangeQuery.getMin(), equalTo(new DateTime(TimeValue.timeValueHours(10).millis(), DateTimeZone.UTC).getMillis()));
} }
public void testDayWithoutYearFormat() throws Exception { public void testDayWithoutYearFormat() throws Exception {
String mapping = XContentFactory.jsonBuilder().startObject().startObject("type") String mapping = XContentFactory.jsonBuilder().startObject().startObject("type")
.field("date_detection", false) .field("date_detection", false)
@ -268,14 +268,14 @@ public class SimpleDateMappingTests extends ElasticsearchSingleNodeTest {
NumericRangeQuery<Long> rangeQuery; NumericRangeQuery<Long> rangeQuery;
try { try {
SearchContext.setCurrent(new TestSearchContext()); 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 { } finally {
SearchContext.removeCurrent(); SearchContext.removeCurrent();
} }
assertThat(rangeQuery.getMax(), equalTo(new DateTime(TimeValue.timeValueHours(35).millis(), DateTimeZone.UTC).getMillis())); 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())); assertThat(rangeQuery.getMin(), equalTo(new DateTime(TimeValue.timeValueHours(34).millis(), DateTimeZone.UTC).getMillis()));
} }
public void testIgnoreMalformedOption() throws Exception { public void testIgnoreMalformedOption() throws Exception {
String mapping = XContentFactory.jsonBuilder().startObject().startObject("type") String mapping = XContentFactory.jsonBuilder().startObject().startObject("type")
.startObject("properties") .startObject("properties")
@ -377,7 +377,7 @@ public class SimpleDateMappingTests extends ElasticsearchSingleNodeTest {
Map<String, String> mergedConfig = getConfigurationViaXContent(mergedFieldMapper); 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")); 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 { public void testDefaultDocValues() throws Exception {
String mapping = XContentFactory.jsonBuilder().startObject().startObject("type") String mapping = XContentFactory.jsonBuilder().startObject().startObject("type")
.startObject("properties").startObject("date_field").field("type", "date").endObject().endObject() .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) { if (fieldType == null) {
boolFilter.add(new TermRangeQuery(field, null, null, true, true), BooleanClause.Occur.SHOULD); boolFilter.add(new TermRangeQuery(field, null, null, true, true), BooleanClause.Occur.SHOULD);
} else { } 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 existence = queryBuilder.existence();
final boolean nullValue = queryBuilder.nullValue(); final boolean nullValue = queryBuilder.nullValue();
String fieldPattern = queryBuilder.fieldPattern(); String fieldPattern = queryBuilder.fieldPattern();
if (!existence && !nullValue) { if (!existence && !nullValue) {
throw new QueryParsingException(context, "missing must have either existence, or null_value, or both set to true"); 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 _field_names are not indexed, we need to go the slow way
if (filter == null && fieldType != null) { if (filter == null && fieldType != null) {
filter = fieldType.rangeQuery(null, null, true, true, context); filter = fieldType.rangeQuery(null, null, true, true);
} }
if (filter == null) { if (filter == null) {
filter = new TermRangeQuery(field, null, null, true, true); filter = new TermRangeQuery(field, null, null, true, true);
@ -174,7 +174,7 @@ public class MissingQueryBuilderTest extends BaseQueryTestCase<MissingQueryBuild
missingQueryBuilder = new MissingQueryBuilder("field"); missingQueryBuilder = new MissingQueryBuilder("field");
assertNull(missingQueryBuilder.validate()); assertNull(missingQueryBuilder.validate());
} }
@Test(expected = QueryParsingException.class) @Test(expected = QueryParsingException.class)
public void testBothNullValueAndExistenceFalse() throws IOException { public void testBothNullValueAndExistenceFalse() throws IOException {
QueryParseContext context = createContext(); QueryParseContext context = createContext();

View File

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

View File

@ -205,7 +205,7 @@ public class IndexShardTests extends ElasticsearchSingleNodeTest {
ShardStateMetaData shardStateMetaData = load(logger, env.availableShardPaths(shard.shardId)); ShardStateMetaData shardStateMetaData = load(logger, env.availableShardPaths(shard.shardId));
assertEquals(shardStateMetaData, getShardStateMetadata(shard)); 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.updateRoutingEntry(routing, true);
shard.deleteShardState(); shard.deleteShardState();

View File

@ -29,6 +29,8 @@ import org.junit.After;
import org.junit.Before; import org.junit.Before;
import org.junit.Test; import org.junit.Test;
import java.net.URL;
import java.net.URLClassLoader;
import java.util.ArrayList; import java.util.ArrayList;
import java.util.List; import java.util.List;
import java.util.concurrent.atomic.AtomicInteger; 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("name"), is("prompted name 0"));
assertThat(settings.get("node.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 { 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; assert stringProps.length % 2 == 0;
Files.createDirectories(pluginDir); Files.createDirectories(pluginDir);
Path propertiesFile = pluginDir.resolve(PluginInfo.ES_PLUGIN_PROPERTIES); Path propertiesFile = pluginDir.resolve(PluginInfo.ES_PLUGIN_PROPERTIES);
@ -57,6 +57,7 @@ public class PluginInfoTests extends ElasticsearchTestCase {
"description", "fake desc", "description", "fake desc",
"version", "1.0", "version", "1.0",
"elasticsearch.version", Version.CURRENT.toString(), "elasticsearch.version", Version.CURRENT.toString(),
"java.version", System.getProperty("java.specification.version"),
"jvm", "true", "jvm", "true",
"classname", "FakePlugin"); "classname", "FakePlugin");
PluginInfo info = PluginInfo.readFromProperties(pluginDir); 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 { public void testReadFromPropertiesBogusElasticsearchVersion() throws Exception {
Path pluginDir = createTempDir().resolve("fake-plugin"); Path pluginDir = createTempDir().resolve("fake-plugin");
writeProperties(pluginDir, writeProperties(pluginDir,
@ -155,6 +188,7 @@ public class PluginInfoTests extends ElasticsearchTestCase {
"description", "fake desc", "description", "fake desc",
"version", "1.0", "version", "1.0",
"elasticsearch.version", Version.CURRENT.toString(), "elasticsearch.version", Version.CURRENT.toString(),
"java.version", System.getProperty("java.specification.version"),
"jvm", "true"); "jvm", "true");
try { try {
PluginInfo.readFromProperties(pluginDir); PluginInfo.readFromProperties(pluginDir);
@ -166,16 +200,30 @@ public class PluginInfoTests extends ElasticsearchTestCase {
public void testReadFromPropertiesSitePlugin() throws Exception { public void testReadFromPropertiesSitePlugin() throws Exception {
Path pluginDir = createTempDir().resolve("fake-plugin"); Path pluginDir = createTempDir().resolve("fake-plugin");
Files.createDirectories(pluginDir.resolve("_site"));
writeProperties(pluginDir, writeProperties(pluginDir,
"description", "fake desc", "description", "fake desc",
"version", "1.0", "version", "1.0",
"elasticsearch.version", Version.CURRENT.toString(),
"site", "true"); "site", "true");
PluginInfo info = PluginInfo.readFromProperties(pluginDir); PluginInfo info = PluginInfo.readFromProperties(pluginDir);
assertTrue(info.isSite()); assertTrue(info.isSite());
assertFalse(info.isJvm()); assertFalse(info.isJvm());
assertEquals("NA", info.getClassname()); 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() { public void testPluginListSorted() {
PluginsInfo pluginsInfo = new PluginsInfo(5); 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.http.impl.client.HttpClients;
import org.apache.lucene.util.LuceneTestCase; import org.apache.lucene.util.LuceneTestCase;
import org.elasticsearch.Version;
import org.elasticsearch.common.cli.CliTool; import org.elasticsearch.common.cli.CliTool;
import org.elasticsearch.common.cli.CliToolTestCase.CaptureOutputTerminal; import org.elasticsearch.common.cli.CliToolTestCase.CaptureOutputTerminal;
import org.elasticsearch.common.collect.Tuple; import org.elasticsearch.common.collect.Tuple;
@ -36,13 +37,18 @@ import org.junit.Before;
import org.junit.Test; import org.junit.Test;
import java.io.IOException; 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.Files;
import java.nio.file.Path; 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.PosixFileAttributeView;
import java.nio.file.attribute.PosixFileAttributes; import java.nio.file.attribute.PosixFileAttributes;
import java.nio.file.attribute.PosixFilePermission; import java.nio.file.attribute.PosixFilePermission;
import java.util.Locale; 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.CliTool.ExitStatus.USAGE;
import static org.elasticsearch.common.cli.CliToolTestCase.args; 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.assertDirectoryExists;
import static org.elasticsearch.test.hamcrest.ElasticsearchAssertions.assertFileExists; import static org.elasticsearch.test.hamcrest.ElasticsearchAssertions.assertFileExists;
import static org.hamcrest.Matchers.*; import static org.hamcrest.Matchers.*;
import static org.elasticsearch.plugins.PluginInfoTests.writeProperties;
@ClusterScope(scope = Scope.TEST, numDataNodes = 0, transportClientRatio = 0.0) @ClusterScope(scope = Scope.TEST, numDataNodes = 0, transportClientRatio = 0.0)
@LuceneTestCase.SuppressFileSystems("*") // TODO: clean up this test to allow extra files @LuceneTestCase.SuppressFileSystems("*") // TODO: clean up this test to allow extra files
@ -74,29 +81,65 @@ public class PluginManagerTests extends ElasticsearchIntegrationTest {
if (!Files.exists(configDir)) { if (!Files.exists(configDir)) {
Files.createDirectories(configDir); Files.createDirectories(configDir);
} }
} }
@After @After
public void clearPathHome() { public void clearPathHome() {
System.clearProperty("es.default.path.home"); 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 @Test
public void testThatPluginNameMustBeSupplied() throws IOException { 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); assertStatus("install --url " + pluginUrl, USAGE);
} }
@Test @Test
public void testLocalPluginInstallWithBinAndConfig() throws Exception { 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(); Environment env = initialSettings.v2();
Path binDir = env.homeFile().resolve("bin"); Path binDir = env.homeFile().resolve("bin");
Path pluginBinDir = binDir.resolve(pluginName); Path pluginBinDir = binDir.resolve(pluginName);
Path pluginConfigDir = env.configFile().resolve(pluginName); Path pluginConfigDir = env.configFile().resolve(pluginName);
String pluginUrl = getPluginUrlForResource("plugin_with_bin_and_config.zip");
assertStatusOk("install " + pluginName + " --url " + pluginUrl + " --verbose"); assertStatusOk("install " + pluginName + " --url " + pluginUrl + " --verbose");
terminal.getTerminalOutput().clear(); terminal.getTerminalOutput().clear();
@ -123,23 +166,36 @@ public class PluginManagerTests extends ElasticsearchIntegrationTest {
*/ */
@Test @Test
public void testLocalPluginInstallWithBinAndConfigInAlreadyExistingConfigDir_7890() throws Exception { 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(); Environment env = initialSettings.v2();
Path pluginConfigDir = env.configFile().resolve(pluginName); 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: First time, our plugin contains:
- config/test.txt (version1) - config/test.txt (version1)
*/ */
assertFileContent(pluginConfigDir, "test.txt", "version1\n"); assertFileContent(pluginConfigDir, "test.txt", "version1");
// We now remove the plugin // We now remove the plugin
assertStatusOk("remove " + pluginName); assertStatusOk("remove " + pluginName);
// We should still have test.txt // We should still have test.txt
assertFileContent(pluginConfigDir, "test.txt", "version1\n"); assertFileContent(pluginConfigDir, "test.txt", "version1");
// Installing a new plugin version // Installing a new plugin version
/* /*
@ -148,19 +204,31 @@ public class PluginManagerTests extends ElasticsearchIntegrationTest {
- config/dir/testdir.txt (version1) - config/dir/testdir.txt (version1)
- config/dir/subdir/testsubdir.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", "version1");
assertFileContent(pluginConfigDir, "test.txt.new", "version2\n"); assertFileContent(pluginConfigDir, "test.txt.new", "version2");
assertFileContent(pluginConfigDir, "dir/testdir.txt", "version1\n"); assertFileContent(pluginConfigDir, "dir/testdir.txt", "version1");
assertFileContent(pluginConfigDir, "dir/subdir/testsubdir.txt", "version1\n"); assertFileContent(pluginConfigDir, "dir/subdir/testsubdir.txt", "version1");
// Removing // Removing
assertStatusOk("remove " + pluginName); assertStatusOk("remove " + pluginName);
assertFileContent(pluginConfigDir, "test.txt", "version1\n"); assertFileContent(pluginConfigDir, "test.txt", "version1");
assertFileContent(pluginConfigDir, "test.txt.new", "version2\n"); assertFileContent(pluginConfigDir, "test.txt.new", "version2");
assertFileContent(pluginConfigDir, "dir/testdir.txt", "version1\n"); assertFileContent(pluginConfigDir, "dir/testdir.txt", "version1");
assertFileContent(pluginConfigDir, "dir/subdir/testsubdir.txt", "version1\n"); assertFileContent(pluginConfigDir, "dir/subdir/testsubdir.txt", "version1");
// Installing a new plugin version // Installing a new plugin version
/* /*
@ -171,40 +239,56 @@ public class PluginManagerTests extends ElasticsearchIntegrationTest {
- config/dir/testdir2.txt (version1) - config/dir/testdir2.txt (version1)
- config/dir/subdir/testsubdir.txt (version2) - 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"); assertStatusOk(String.format(Locale.ROOT, "install %s --url %s --verbose", pluginName, pluginUrl));
assertFileContent(pluginConfigDir, "test2.txt", "version1\n");
assertFileContent(pluginConfigDir, "test.txt.new", "version3\n"); assertFileContent(pluginConfigDir, "test.txt", "version1");
assertFileContent(pluginConfigDir, "dir/testdir.txt", "version1\n"); assertFileContent(pluginConfigDir, "test2.txt", "version1");
assertFileContent(pluginConfigDir, "dir/testdir.txt.new", "version2\n"); assertFileContent(pluginConfigDir, "test.txt.new", "version3");
assertFileContent(pluginConfigDir, "dir/testdir2.txt", "version1\n"); assertFileContent(pluginConfigDir, "dir/testdir.txt", "version1");
assertFileContent(pluginConfigDir, "dir/subdir/testsubdir.txt", "version1\n"); assertFileContent(pluginConfigDir, "dir/testdir.txt.new", "version2");
assertFileContent(pluginConfigDir, "dir/subdir/testsubdir.txt.new", "version2\n"); assertFileContent(pluginConfigDir, "dir/testdir2.txt", "version1");
assertFileContent(pluginConfigDir, "dir/subdir/testsubdir.txt", "version1");
assertFileContent(pluginConfigDir, "dir/subdir/testsubdir.txt.new", "version2");
} }
// For #7152 // For #7152
@Test @Test
public void testLocalPluginInstallWithBinOnly_7152() throws Exception { 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(); Environment env = initialSettings.v2();
Path binDir = env.homeFile().resolve("bin"); Path binDir = env.homeFile().resolve("bin");
Path pluginBinDir = binDir.resolve(pluginName); 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); assertThatPluginIsListed(pluginName);
assertDirectoryExists(pluginBinDir); 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 @Test
public void testListInstalledEmpty() throws IOException { public void testListInstalledEmpty() throws IOException {
assertStatusOk("list"); assertStatusOk("list");
@ -220,18 +304,33 @@ public class PluginManagerTests extends ElasticsearchIntegrationTest {
@Test @Test
public void testInstallPlugin() throws IOException { public void testInstallPlugin() throws IOException {
String pluginName = "plugin-classfile"; String pluginName = "fake-plugin";
assertStatusOk(String.format(Locale.ROOT, "install %s --url %s --verbose", pluginName, getPluginUrlForResource("plugin_with_classfile.zip"))); Path pluginDir = createTempDir().resolve(pluginName);
assertThatPluginIsListed("plugin-classfile"); 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 @Test
public void testInstallSitePlugin() throws IOException { public void testInstallSitePlugin() throws IOException {
String pluginName = "plugin-site"; String pluginName = "fake-plugin";
assertStatusOk(String.format(Locale.ROOT, "install %s --url %s --verbose", pluginName, getPluginUrlForResource("plugin_without_folders.zip"))); 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); assertThatPluginIsListed(pluginName);
// We want to check that Plugin Manager moves content to _site // 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 @Test
public void testRemovePlugin() throws Exception { 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 // 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 // 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 // We want to remove plugin with groupid/artifactid form
singlePluginInstallAndRemove("groupid/plugintest", "plugintest", getPluginUrlForResource("plugin_without_folders.zip")); singlePluginInstallAndRemove("groupid/plugintest", "plugintest", pluginUrl);
} }
@Test @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 { private Tuple<Settings, Environment> buildInitialSettings() throws IOException {
Settings settings = settingsBuilder() Settings settings = settingsBuilder()
.put("discovery.zen.ping.multicast.enabled", false) .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.env.NodeEnvironment;
import org.elasticsearch.test.cache.recycler.MockBigArrays; import org.elasticsearch.test.cache.recycler.MockBigArrays;
import org.elasticsearch.test.cache.recycler.MockPageCacheRecycler; 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.LoggingListener;
import org.elasticsearch.test.junit.listeners.ReproduceInfoPrinter; import org.elasticsearch.test.junit.listeners.ReproduceInfoPrinter;
import org.elasticsearch.test.search.MockSearchService; import org.elasticsearch.test.search.MockSearchService;
@ -78,7 +79,8 @@ import static com.google.common.collect.Lists.newArrayList;
*/ */
@Listeners({ @Listeners({
ReproduceInfoPrinter.class, ReproduceInfoPrinter.class,
LoggingListener.class LoggingListener.class,
AssertionErrorThreadDumpPrinter.class
}) })
// remove this entire annotation on upgrade to 5.3! // remove this entire annotation on upgrade to 5.3!
@ThreadLeakFilters(defaultFilters = true, filters = { @ThreadLeakFilters(defaultFilters = true, filters = {
@ -550,44 +552,7 @@ public abstract class ElasticsearchTestCase extends LuceneTestCase {
protected static final void printStackDump(ESLogger logger) { protected static final void printStackDump(ESLogger logger) {
// print stack traces if we can't create any native thread anymore // print stack traces if we can't create any native thread anymore
Map<Thread, StackTraceElement[]> allStackTraces = Thread.getAllStackTraces(); Map<Thread, StackTraceElement[]> allStackTraces = Thread.getAllStackTraces();
logger.error(formatThreadStacks(allStackTraces)); logger.error(StackTraces.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();
}
} }
/** /**

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 { public void checkIfMapEmptyAfterLotsOfAcquireAndReleases() throws InterruptedException {
ConcurrentHashMap<String, Integer> counter = new ConcurrentHashMap<>(); ConcurrentHashMap<String, Integer> counter = new ConcurrentHashMap<>();
ConcurrentHashMap<String, AtomicInteger> safeCounter = 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)]; String[] names = new String[randomIntBetween(1, 40)];
for (int i = 0; i < names.length; i++) { for (int i = 0; i < names.length; i++) {
names[i] = randomRealisticUnicodeOfLengthBetween(10, 20); names[i] = randomRealisticUnicodeOfLengthBetween(10, 20);
@ -54,11 +54,6 @@ public class KeyedLockTests extends ElasticsearchTestCase {
threads[i].start(); threads[i].start();
} }
startLatch.countDown(); startLatch.countDown();
for (int i = 0; i < numThreads; i++) {
if (randomBoolean()) {
threads[i].incWithGlobal();
}
}
for (int i = 0; i < numThreads; i++) { for (int i = 0; i < numThreads; i++) {
threads[i].join(); 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) @Test(expected = IllegalStateException.class)
public void checkCannotAcquireTwoLocks() throws InterruptedException { 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)); String name = randomRealisticUnicodeOfLength(scaledRandomIntBetween(10, 50));
connectionLock.acquire(name); connectionLock.acquire(name);
connectionLock.acquire(name); connectionLock.acquire(name);
@ -98,7 +79,7 @@ public class KeyedLockTests extends ElasticsearchTestCase {
@Test(expected = IllegalStateException.class) @Test(expected = IllegalStateException.class)
public void checkCannotReleaseUnacquiredLock() throws InterruptedException { 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)); String name = randomRealisticUnicodeOfLength(scaledRandomIntBetween(10, 50));
connectionLock.release(name); 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