Internal: changed every single index operation to not replace the index within the original request

An anti-pattern that we have in our code, noticeable for java API users, is that we modify incoming requests by replacing the index or alias with the concrete index. This way not only the request has changed, but all following communications that use that request will lose the information on whether the original request was performed against an alias or an index.

Refactored the following base classes: `TransportShardReplicationOperationAction`, `TransportShardSingleOperationAction`, `TransportSingleCustomOperationAction`, `TransportInstanceSingleOperationAction` and all subclasses by introduced an InternalRequest object that contains the original request plus additional info (e.g. the concrete index). This internal request doesn't get sent over the transport but rebuilt on each node on demand (not different to what currently happens anyway, as concrete index gets set on each node). When the request becomes a shard level request, instead of using the only int shardId we serialize the ShardId that contains both concrete index name (which might then differ ffrom the original one within the request) and shard id.

Using this pattern we can move get, multi_get, explain, analyze, term_vector, multi_term_vector, index, delete, update, bulk to not replace the index name with the concrete one within the request. The index name within the original request will stay the same.

Made it also clearer within the different transport actions when the index needs to be resolved and when that's not needed (e.g. shard level request), by exposing `resolveIndex` method. Moved check block methods to parent classes as their content was always the same on every subclass.

Improved existing tests by randomly introducing the use of an alias, and verifying that the responses always contain the concrete index name and not the original one, as that's the expected behaviour.

Added backwards compatibility tests to make sure that the change is applied in a backwards compatible manner.

Closes #7223
This commit is contained in:
javanna 2014-07-23 21:36:24 +02:00 committed by Luca Cavanna
parent 371d6021e7
commit 5d987ad5e2
38 changed files with 1284 additions and 1026 deletions

View File

@ -41,6 +41,7 @@ import org.elasticsearch.index.analysis.*;
import org.elasticsearch.index.mapper.FieldMapper; import org.elasticsearch.index.mapper.FieldMapper;
import org.elasticsearch.index.mapper.internal.AllFieldMapper; import org.elasticsearch.index.mapper.internal.AllFieldMapper;
import org.elasticsearch.index.service.IndexService; import org.elasticsearch.index.service.IndexService;
import org.elasticsearch.index.shard.ShardId;
import org.elasticsearch.indices.IndicesService; import org.elasticsearch.indices.IndicesService;
import org.elasticsearch.indices.analysis.IndicesAnalysisService; import org.elasticsearch.indices.analysis.IndicesAnalysisService;
import org.elasticsearch.threadpool.ThreadPool; import org.elasticsearch.threadpool.ThreadPool;
@ -85,28 +86,32 @@ public class TransportAnalyzeAction extends TransportSingleCustomOperationAction
} }
@Override @Override
protected ClusterBlockException checkRequestBlock(ClusterState state, AnalyzeRequest request) { protected boolean resolveIndex(AnalyzeRequest request) {
if (request.index() != null) { return request.index() != null;
request.index(state.metaData().concreteSingleIndex(request.index(), request.indicesOptions())); }
@Override
protected ClusterBlockException checkRequestBlock(ClusterState state, InternalRequest request) {
if (request.concreteIndex() != null) {
return super.checkRequestBlock(state, request); return super.checkRequestBlock(state, request);
} }
return null; return null;
} }
@Override @Override
protected ShardsIterator shards(ClusterState state, AnalyzeRequest request) { protected ShardsIterator shards(ClusterState state, InternalRequest request) {
if (request.index() == null) { if (request.concreteIndex() == null) {
// just execute locally.... // just execute locally....
return null; return null;
} }
return state.routingTable().index(request.index()).randomAllActiveShardsIt(); return state.routingTable().index(request.concreteIndex()).randomAllActiveShardsIt();
} }
@Override @Override
protected AnalyzeResponse shardOperation(AnalyzeRequest request, int shardId) throws ElasticsearchException { protected AnalyzeResponse shardOperation(AnalyzeRequest request, ShardId shardId) throws ElasticsearchException {
IndexService indexService = null; IndexService indexService = null;
if (request.index() != null) { if (shardId != null) {
indexService = indicesService.indexServiceSafe(request.index()); indexService = indicesService.indexServiceSafe(shardId.getIndex());
} }
Analyzer analyzer = null; Analyzer analyzer = null;
boolean closeAnalyzer = false; boolean closeAnalyzer = false;

View File

@ -37,10 +37,10 @@ import org.elasticsearch.common.xcontent.ToXContent;
import org.elasticsearch.common.xcontent.XContentBuilder; import org.elasticsearch.common.xcontent.XContentBuilder;
import org.elasticsearch.common.xcontent.XContentFactory; import org.elasticsearch.common.xcontent.XContentFactory;
import org.elasticsearch.common.xcontent.XContentType; import org.elasticsearch.common.xcontent.XContentType;
import org.elasticsearch.index.Index;
import org.elasticsearch.index.mapper.DocumentMapper; import org.elasticsearch.index.mapper.DocumentMapper;
import org.elasticsearch.index.mapper.FieldMapper; import org.elasticsearch.index.mapper.FieldMapper;
import org.elasticsearch.index.service.IndexService; import org.elasticsearch.index.service.IndexService;
import org.elasticsearch.index.shard.ShardId;
import org.elasticsearch.indices.IndicesService; import org.elasticsearch.indices.IndicesService;
import org.elasticsearch.indices.TypeMissingException; import org.elasticsearch.indices.TypeMissingException;
import org.elasticsearch.threadpool.ThreadPool; import org.elasticsearch.threadpool.ThreadPool;
@ -76,14 +76,21 @@ public class TransportGetFieldMappingsIndexAction extends TransportSingleCustomO
} }
@Override @Override
protected ShardsIterator shards(ClusterState state, GetFieldMappingsIndexRequest request) { protected boolean resolveIndex(GetFieldMappingsIndexRequest request) {
// Will balance requests between shards //internal action, index already resolved
return state.routingTable().index(request.index()).randomAllActiveShardsIt(); return false;
} }
@Override @Override
protected GetFieldMappingsResponse shardOperation(final GetFieldMappingsIndexRequest request, int shardId) throws ElasticsearchException { protected ShardsIterator shards(ClusterState state, InternalRequest request) {
IndexService indexService = indicesService.indexServiceSafe(request.index()); // Will balance requests between shards
return state.routingTable().index(request.concreteIndex()).randomAllActiveShardsIt();
}
@Override
protected GetFieldMappingsResponse shardOperation(final GetFieldMappingsIndexRequest request, ShardId shardId) throws ElasticsearchException {
assert shardId != null;
IndexService indexService = indicesService.indexServiceSafe(shardId.getIndex());
Collection<String> typeIntersection; Collection<String> typeIntersection;
if (request.types().length == 0) { if (request.types().length == 0) {
typeIntersection = indexService.mapperService().types(); typeIntersection = indexService.mapperService().types();
@ -98,7 +105,7 @@ public class TransportGetFieldMappingsIndexAction extends TransportSingleCustomO
}); });
if (typeIntersection.isEmpty()) { if (typeIntersection.isEmpty()) {
throw new TypeMissingException(new Index(request.index()), request.types()); throw new TypeMissingException(shardId.index(), request.types());
} }
} }
@ -111,7 +118,7 @@ public class TransportGetFieldMappingsIndexAction extends TransportSingleCustomO
} }
} }
return new GetFieldMappingsResponse(ImmutableMap.of(request.index(), typeMappings.immutableMap())); return new GetFieldMappingsResponse(ImmutableMap.of(shardId.getIndex(), typeMappings.immutableMap()));
} }
@Override @Override

View File

@ -35,7 +35,7 @@ import org.elasticsearch.action.index.IndexRequest;
import org.elasticsearch.action.support.ActionFilters; import org.elasticsearch.action.support.ActionFilters;
import org.elasticsearch.action.support.AutoCreateIndex; import org.elasticsearch.action.support.AutoCreateIndex;
import org.elasticsearch.action.support.HandledTransportAction; import org.elasticsearch.action.support.HandledTransportAction;
import org.elasticsearch.action.support.TransportAction; import org.elasticsearch.action.support.IndicesOptions;
import org.elasticsearch.action.update.UpdateRequest; import org.elasticsearch.action.update.UpdateRequest;
import org.elasticsearch.cluster.ClusterService; import org.elasticsearch.cluster.ClusterService;
import org.elasticsearch.cluster.ClusterState; import org.elasticsearch.cluster.ClusterState;
@ -51,14 +51,9 @@ import org.elasticsearch.index.shard.ShardId;
import org.elasticsearch.indices.IndexAlreadyExistsException; import org.elasticsearch.indices.IndexAlreadyExistsException;
import org.elasticsearch.rest.RestStatus; import org.elasticsearch.rest.RestStatus;
import org.elasticsearch.threadpool.ThreadPool; import org.elasticsearch.threadpool.ThreadPool;
import org.elasticsearch.transport.BaseTransportRequestHandler;
import org.elasticsearch.transport.TransportChannel;
import org.elasticsearch.transport.TransportService; import org.elasticsearch.transport.TransportService;
import java.util.List; import java.util.*;
import java.util.Locale;
import java.util.Map;
import java.util.Set;
import java.util.concurrent.atomic.AtomicInteger; import java.util.concurrent.atomic.AtomicInteger;
/** /**
@ -201,26 +196,25 @@ public class TransportBulkAction extends HandledTransportAction<BulkRequest, Bul
} }
private void executeBulk(final BulkRequest bulkRequest, final long startTime, final ActionListener<BulkResponse> listener, final AtomicArray<BulkItemResponse> responses ) { private void executeBulk(final BulkRequest bulkRequest, final long startTime, final ActionListener<BulkResponse> listener, final AtomicArray<BulkItemResponse> responses ) {
ClusterState clusterState = clusterService.state(); final ClusterState clusterState = clusterService.state();
// TODO use timeout to wait here if its blocked... // TODO use timeout to wait here if its blocked...
clusterState.blocks().globalBlockedRaiseException(ClusterBlockLevel.WRITE); clusterState.blocks().globalBlockedRaiseException(ClusterBlockLevel.WRITE);
final ConcreteIndices concreteIndices = new ConcreteIndices(clusterState.metaData());
MetaData metaData = clusterState.metaData(); MetaData metaData = clusterState.metaData();
for (int i = 0; i < bulkRequest.requests.size(); i++) { for (int i = 0; i < bulkRequest.requests.size(); i++) {
ActionRequest request = bulkRequest.requests.get(i); ActionRequest request = bulkRequest.requests.get(i);
if (request instanceof IndexRequest) { if (request instanceof IndexRequest) {
IndexRequest indexRequest = (IndexRequest) request; IndexRequest indexRequest = (IndexRequest) request;
String aliasOrIndex = indexRequest.index(); String concreteIndex = concreteIndices.resolveIfAbsent(indexRequest.index(), indexRequest.indicesOptions());
indexRequest.index(clusterState.metaData().concreteSingleIndex(indexRequest.index(), indexRequest.indicesOptions()));
MappingMetaData mappingMd = null; MappingMetaData mappingMd = null;
if (metaData.hasIndex(indexRequest.index())) { if (metaData.hasIndex(concreteIndex)) {
mappingMd = metaData.index(indexRequest.index()).mappingOrDefault(indexRequest.type()); mappingMd = metaData.index(concreteIndex).mappingOrDefault(indexRequest.type());
} }
try { try {
indexRequest.process(metaData, aliasOrIndex, mappingMd, allowIdGeneration); indexRequest.process(metaData, mappingMd, allowIdGeneration, concreteIndex);
} catch (ElasticsearchParseException e) { } catch (ElasticsearchParseException e) {
BulkItemResponse.Failure failure = new BulkItemResponse.Failure(indexRequest.index(), indexRequest.type(), indexRequest.id(), e); BulkItemResponse.Failure failure = new BulkItemResponse.Failure(concreteIndex, indexRequest.type(), indexRequest.id(), e);
BulkItemResponse bulkItemResponse = new BulkItemResponse(i, "index", failure); BulkItemResponse bulkItemResponse = new BulkItemResponse(i, "index", failure);
responses.set(i, bulkItemResponse); responses.set(i, bulkItemResponse);
// make sure the request gets never processed again // make sure the request gets never processed again
@ -228,12 +222,12 @@ public class TransportBulkAction extends HandledTransportAction<BulkRequest, Bul
} }
} else if (request instanceof DeleteRequest) { } else if (request instanceof DeleteRequest) {
DeleteRequest deleteRequest = (DeleteRequest) request; DeleteRequest deleteRequest = (DeleteRequest) request;
concreteIndices.resolveIfAbsent(deleteRequest.index(), deleteRequest.indicesOptions());
deleteRequest.routing(clusterState.metaData().resolveIndexRouting(deleteRequest.routing(), deleteRequest.index())); deleteRequest.routing(clusterState.metaData().resolveIndexRouting(deleteRequest.routing(), deleteRequest.index()));
deleteRequest.index(clusterState.metaData().concreteSingleIndex(deleteRequest.index(), deleteRequest.indicesOptions()));
} else if (request instanceof UpdateRequest) { } else if (request instanceof UpdateRequest) {
UpdateRequest updateRequest = (UpdateRequest) request; UpdateRequest updateRequest = (UpdateRequest) request;
concreteIndices.resolveIfAbsent(updateRequest.index(), updateRequest.indicesOptions());
updateRequest.routing(clusterState.metaData().resolveIndexRouting(updateRequest.routing(), updateRequest.index())); updateRequest.routing(clusterState.metaData().resolveIndexRouting(updateRequest.routing(), updateRequest.index()));
updateRequest.index(clusterState.metaData().concreteSingleIndex(updateRequest.index(), updateRequest.indicesOptions()));
} }
} }
@ -244,7 +238,8 @@ public class TransportBulkAction extends HandledTransportAction<BulkRequest, Bul
ActionRequest request = bulkRequest.requests.get(i); ActionRequest request = bulkRequest.requests.get(i);
if (request instanceof IndexRequest) { if (request instanceof IndexRequest) {
IndexRequest indexRequest = (IndexRequest) request; IndexRequest indexRequest = (IndexRequest) request;
ShardId shardId = clusterService.operationRouting().indexShards(clusterState, indexRequest.index(), indexRequest.type(), indexRequest.id(), indexRequest.routing()).shardId(); String concreteIndex = concreteIndices.getConcreteIndex(indexRequest.index());
ShardId shardId = clusterService.operationRouting().indexShards(clusterState, concreteIndex, indexRequest.type(), indexRequest.id(), indexRequest.routing()).shardId();
List<BulkItemRequest> list = requestsByShard.get(shardId); List<BulkItemRequest> list = requestsByShard.get(shardId);
if (list == null) { if (list == null) {
list = Lists.newArrayList(); list = Lists.newArrayList();
@ -253,10 +248,11 @@ public class TransportBulkAction extends HandledTransportAction<BulkRequest, Bul
list.add(new BulkItemRequest(i, request)); list.add(new BulkItemRequest(i, request));
} else if (request instanceof DeleteRequest) { } else if (request instanceof DeleteRequest) {
DeleteRequest deleteRequest = (DeleteRequest) request; DeleteRequest deleteRequest = (DeleteRequest) request;
MappingMetaData mappingMd = clusterState.metaData().index(deleteRequest.index()).mappingOrDefault(deleteRequest.type()); String concreteIndex = concreteIndices.getConcreteIndex(deleteRequest.index());
MappingMetaData mappingMd = clusterState.metaData().index(concreteIndex).mappingOrDefault(deleteRequest.type());
if (mappingMd != null && mappingMd.routing().required() && deleteRequest.routing() == null) { if (mappingMd != null && mappingMd.routing().required() && deleteRequest.routing() == null) {
// if routing is required, and no routing on the delete request, we need to broadcast it.... // if routing is required, and no routing on the delete request, we need to broadcast it....
GroupShardsIterator groupShards = clusterService.operationRouting().broadcastDeleteShards(clusterState, deleteRequest.index()); GroupShardsIterator groupShards = clusterService.operationRouting().broadcastDeleteShards(clusterState, concreteIndex);
for (ShardIterator shardIt : groupShards) { for (ShardIterator shardIt : groupShards) {
List<BulkItemRequest> list = requestsByShard.get(shardIt.shardId()); List<BulkItemRequest> list = requestsByShard.get(shardIt.shardId());
if (list == null) { if (list == null) {
@ -266,7 +262,7 @@ public class TransportBulkAction extends HandledTransportAction<BulkRequest, Bul
list.add(new BulkItemRequest(i, new DeleteRequest(deleteRequest))); list.add(new BulkItemRequest(i, new DeleteRequest(deleteRequest)));
} }
} else { } else {
ShardId shardId = clusterService.operationRouting().deleteShards(clusterState, deleteRequest.index(), deleteRequest.type(), deleteRequest.id(), deleteRequest.routing()).shardId(); ShardId shardId = clusterService.operationRouting().deleteShards(clusterState, concreteIndex, deleteRequest.type(), deleteRequest.id(), deleteRequest.routing()).shardId();
List<BulkItemRequest> list = requestsByShard.get(shardId); List<BulkItemRequest> list = requestsByShard.get(shardId);
if (list == null) { if (list == null) {
list = Lists.newArrayList(); list = Lists.newArrayList();
@ -276,11 +272,12 @@ public class TransportBulkAction extends HandledTransportAction<BulkRequest, Bul
} }
} else if (request instanceof UpdateRequest) { } else if (request instanceof UpdateRequest) {
UpdateRequest updateRequest = (UpdateRequest) request; UpdateRequest updateRequest = (UpdateRequest) request;
MappingMetaData mappingMd = clusterState.metaData().index(updateRequest.index()).mappingOrDefault(updateRequest.type()); String concreteIndex = concreteIndices.getConcreteIndex(updateRequest.index());
MappingMetaData mappingMd = clusterState.metaData().index(concreteIndex).mappingOrDefault(updateRequest.type());
if (mappingMd != null && mappingMd.routing().required() && updateRequest.routing() == null) { if (mappingMd != null && mappingMd.routing().required() && updateRequest.routing() == null) {
continue; // What to do? continue; // What to do?
} }
ShardId shardId = clusterService.operationRouting().indexShards(clusterState, updateRequest.index(), updateRequest.type(), updateRequest.id(), updateRequest.routing()).shardId(); ShardId shardId = clusterService.operationRouting().indexShards(clusterState, concreteIndex, updateRequest.type(), updateRequest.id(), updateRequest.routing()).shardId();
List<BulkItemRequest> list = requestsByShard.get(shardId); List<BulkItemRequest> list = requestsByShard.get(shardId);
if (list == null) { if (list == null) {
list = Lists.newArrayList(); list = Lists.newArrayList();
@ -323,15 +320,15 @@ public class TransportBulkAction extends HandledTransportAction<BulkRequest, Bul
if (request.request() instanceof IndexRequest) { if (request.request() instanceof IndexRequest) {
IndexRequest indexRequest = (IndexRequest) request.request(); IndexRequest indexRequest = (IndexRequest) request.request();
responses.set(request.id(), new BulkItemResponse(request.id(), indexRequest.opType().toString().toLowerCase(Locale.ENGLISH), responses.set(request.id(), new BulkItemResponse(request.id(), indexRequest.opType().toString().toLowerCase(Locale.ENGLISH),
new BulkItemResponse.Failure(indexRequest.index(), indexRequest.type(), indexRequest.id(), message, status))); new BulkItemResponse.Failure(concreteIndices.getConcreteIndex(indexRequest.index()), indexRequest.type(), indexRequest.id(), message, status)));
} else if (request.request() instanceof DeleteRequest) { } else if (request.request() instanceof DeleteRequest) {
DeleteRequest deleteRequest = (DeleteRequest) request.request(); DeleteRequest deleteRequest = (DeleteRequest) request.request();
responses.set(request.id(), new BulkItemResponse(request.id(), "delete", responses.set(request.id(), new BulkItemResponse(request.id(), "delete",
new BulkItemResponse.Failure(deleteRequest.index(), deleteRequest.type(), deleteRequest.id(), message, status))); new BulkItemResponse.Failure(concreteIndices.getConcreteIndex(deleteRequest.index()), deleteRequest.type(), deleteRequest.id(), message, status)));
} else if (request.request() instanceof UpdateRequest) { } else if (request.request() instanceof UpdateRequest) {
UpdateRequest updateRequest = (UpdateRequest) request.request(); UpdateRequest updateRequest = (UpdateRequest) request.request();
responses.set(request.id(), new BulkItemResponse(request.id(), "update", responses.set(request.id(), new BulkItemResponse(request.id(), "update",
new BulkItemResponse.Failure(updateRequest.index(), updateRequest.type(), updateRequest.id(), message, status))); new BulkItemResponse.Failure(concreteIndices.getConcreteIndex(updateRequest.index()), updateRequest.type(), updateRequest.id(), message, status)));
} }
} }
if (counter.decrementAndGet() == 0) { if (counter.decrementAndGet() == 0) {
@ -345,4 +342,27 @@ public class TransportBulkAction extends HandledTransportAction<BulkRequest, Bul
}); });
} }
} }
private static class ConcreteIndices {
private final Map<String, String> indices = new HashMap<>();
private final MetaData metaData;
ConcreteIndices(MetaData metaData) {
this.metaData = metaData;
}
String getConcreteIndex(String indexOrAlias) {
return indices.get(indexOrAlias);
}
String resolveIfAbsent(String indexOrAlias, IndicesOptions indicesOptions) {
String concreteIndex = indices.get(indexOrAlias);
if (concreteIndex == null) {
concreteIndex = metaData.concreteSingleIndex(indexOrAlias, indicesOptions);
indices.put(indexOrAlias, concreteIndex);
}
return concreteIndex;
}
}
} }

View File

@ -39,8 +39,6 @@ import org.elasticsearch.cluster.ClusterService;
import org.elasticsearch.cluster.ClusterState; import org.elasticsearch.cluster.ClusterState;
import org.elasticsearch.cluster.action.index.MappingUpdatedAction; import org.elasticsearch.cluster.action.index.MappingUpdatedAction;
import org.elasticsearch.cluster.action.shard.ShardStateAction; import org.elasticsearch.cluster.action.shard.ShardStateAction;
import org.elasticsearch.cluster.block.ClusterBlockException;
import org.elasticsearch.cluster.block.ClusterBlockLevel;
import org.elasticsearch.cluster.metadata.MappingMetaData; import org.elasticsearch.cluster.metadata.MappingMetaData;
import org.elasticsearch.cluster.routing.ShardIterator; import org.elasticsearch.cluster.routing.ShardIterator;
import org.elasticsearch.common.Nullable; import org.elasticsearch.common.Nullable;
@ -57,7 +55,6 @@ import org.elasticsearch.index.engine.VersionConflictEngineException;
import org.elasticsearch.index.mapper.DocumentMapper; import org.elasticsearch.index.mapper.DocumentMapper;
import org.elasticsearch.index.mapper.SourceToParse; import org.elasticsearch.index.mapper.SourceToParse;
import org.elasticsearch.index.service.IndexService; import org.elasticsearch.index.service.IndexService;
import org.elasticsearch.index.shard.ShardId;
import org.elasticsearch.index.shard.service.IndexShard; import org.elasticsearch.index.shard.service.IndexShard;
import org.elasticsearch.indices.IndicesService; import org.elasticsearch.indices.IndicesService;
import org.elasticsearch.rest.RestStatus; import org.elasticsearch.rest.RestStatus;
@ -123,25 +120,20 @@ public class TransportShardBulkAction extends TransportShardReplicationOperation
} }
@Override @Override
protected ClusterBlockException checkGlobalBlock(ClusterState state, BulkShardRequest request) { protected boolean resolveIndex() {
return state.blocks().globalBlockedException(ClusterBlockLevel.WRITE); return false;
} }
@Override @Override
protected ClusterBlockException checkRequestBlock(ClusterState state, BulkShardRequest request) { protected ShardIterator shards(ClusterState clusterState, InternalRequest request) {
return state.blocks().indexBlockedException(ClusterBlockLevel.WRITE, request.index()); return clusterState.routingTable().index(request.concreteIndex()).shard(request.request().shardId()).shardsIt();
}
@Override
protected ShardIterator shards(ClusterState clusterState, BulkShardRequest request) {
return clusterState.routingTable().index(request.index()).shard(request.shardId()).shardsIt();
} }
@Override @Override
protected PrimaryResponse<BulkShardResponse, BulkShardRequest> shardOperationOnPrimary(ClusterState clusterState, PrimaryOperationRequest shardRequest) { protected PrimaryResponse<BulkShardResponse, BulkShardRequest> shardOperationOnPrimary(ClusterState clusterState, PrimaryOperationRequest shardRequest) {
final BulkShardRequest request = shardRequest.request; final BulkShardRequest request = shardRequest.request;
IndexService indexService = indicesService.indexServiceSafe(shardRequest.request.index()); IndexService indexService = indicesService.indexServiceSafe(request.index());
IndexShard indexShard = indexService.shardSafe(shardRequest.shardId); IndexShard indexShard = indexService.shardSafe(shardRequest.shardId.id());
Engine.IndexingOperation[] ops = null; Engine.IndexingOperation[] ops = null;
final Set<String> mappingTypesToUpdate = Sets.newHashSet(); final Set<String> mappingTypesToUpdate = Sets.newHashSet();
@ -191,12 +183,12 @@ public class TransportShardBulkAction extends TransportShardReplicationOperation
throw (ElasticsearchException) e; throw (ElasticsearchException) e;
} }
if (e instanceof ElasticsearchException && ((ElasticsearchException) e).status() == RestStatus.CONFLICT) { if (e instanceof ElasticsearchException && ((ElasticsearchException) e).status() == RestStatus.CONFLICT) {
logger.trace("[{}][{}] failed to execute bulk item (index) {}", e, shardRequest.request.index(), shardRequest.shardId, indexRequest); logger.trace("{} failed to execute bulk item (index) {}", e, shardRequest.shardId, indexRequest);
} else { } else {
logger.debug("[{}][{}] failed to execute bulk item (index) {}", e, shardRequest.request.index(), shardRequest.shardId, indexRequest); logger.debug("{} failed to execute bulk item (index) {}", e, shardRequest.shardId, indexRequest);
} }
responses[requestIndex] = new BulkItemResponse(item.id(), indexRequest.opType().lowercase(), responses[requestIndex] = new BulkItemResponse(item.id(), indexRequest.opType().lowercase(),
new BulkItemResponse.Failure(indexRequest.index(), indexRequest.type(), indexRequest.id(), e)); new BulkItemResponse.Failure(request.index(), indexRequest.type(), indexRequest.id(), e));
// nullify the request so it won't execute on the replicas // nullify the request so it won't execute on the replicas
request.items()[requestIndex] = null; request.items()[requestIndex] = null;
} }
@ -207,7 +199,7 @@ public class TransportShardBulkAction extends TransportShardReplicationOperation
try { try {
// add the response // add the response
DeleteResponse deleteResponse = shardDeleteOperation(deleteRequest, indexShard).response(); DeleteResponse deleteResponse = shardDeleteOperation(request, deleteRequest, indexShard).response();
responses[requestIndex] = new BulkItemResponse(item.id(), OP_TYPE_DELETE, deleteResponse); responses[requestIndex] = new BulkItemResponse(item.id(), OP_TYPE_DELETE, deleteResponse);
} catch (Throwable e) { } catch (Throwable e) {
// rethrow the failure if we are going to retry on primary and let parent failure to handle it // rethrow the failure if we are going to retry on primary and let parent failure to handle it
@ -219,12 +211,12 @@ public class TransportShardBulkAction extends TransportShardReplicationOperation
throw (ElasticsearchException) e; throw (ElasticsearchException) e;
} }
if (e instanceof ElasticsearchException && ((ElasticsearchException) e).status() == RestStatus.CONFLICT) { if (e instanceof ElasticsearchException && ((ElasticsearchException) e).status() == RestStatus.CONFLICT) {
logger.trace("[{}][{}] failed to execute bulk item (delete) {}", e, shardRequest.request.index(), shardRequest.shardId, deleteRequest); logger.trace("{} failed to execute bulk item (delete) {}", e, shardRequest.shardId, deleteRequest);
} else { } else {
logger.debug("[{}][{}] failed to execute bulk item (delete) {}", e, shardRequest.request.index(), shardRequest.shardId, deleteRequest); logger.debug("{} failed to execute bulk item (delete) {}", e, shardRequest.shardId, deleteRequest);
} }
responses[requestIndex] = new BulkItemResponse(item.id(), OP_TYPE_DELETE, responses[requestIndex] = new BulkItemResponse(item.id(), OP_TYPE_DELETE,
new BulkItemResponse.Failure(deleteRequest.index(), deleteRequest.type(), deleteRequest.id(), e)); new BulkItemResponse.Failure(request.index(), deleteRequest.type(), deleteRequest.id(), e));
// nullify the request so it won't execute on the replicas // nullify the request so it won't execute on the replicas
request.items()[requestIndex] = null; request.items()[requestIndex] = null;
} }
@ -253,7 +245,7 @@ public class TransportShardBulkAction extends TransportShardReplicationOperation
UpdateResponse updateResponse = new UpdateResponse(indexResponse.getIndex(), indexResponse.getType(), indexResponse.getId(), indexResponse.getVersion(), indexResponse.isCreated()); UpdateResponse updateResponse = new UpdateResponse(indexResponse.getIndex(), indexResponse.getType(), indexResponse.getId(), indexResponse.getVersion(), indexResponse.isCreated());
if (updateRequest.fields() != null && updateRequest.fields().length > 0) { if (updateRequest.fields() != null && updateRequest.fields().length > 0) {
Tuple<XContentType, Map<String, Object>> sourceAndContent = XContentHelper.convertToMap(indexSourceAsBytes, true); Tuple<XContentType, Map<String, Object>> sourceAndContent = XContentHelper.convertToMap(indexSourceAsBytes, true);
updateResponse.setGetResult(updateHelper.extractGetResult(updateRequest, indexResponse.getVersion(), sourceAndContent.v2(), sourceAndContent.v1(), indexSourceAsBytes)); updateResponse.setGetResult(updateHelper.extractGetResult(updateRequest, shardRequest.request.index(), indexResponse.getVersion(), sourceAndContent.v2(), sourceAndContent.v1(), indexSourceAsBytes));
} }
responses[requestIndex] = new BulkItemResponse(item.id(), OP_TYPE_UPDATE, updateResponse); responses[requestIndex] = new BulkItemResponse(item.id(), OP_TYPE_UPDATE, updateResponse);
if (result.mappingTypeToUpdate != null) { if (result.mappingTypeToUpdate != null) {
@ -272,7 +264,7 @@ public class TransportShardBulkAction extends TransportShardReplicationOperation
DeleteResponse response = updateResult.writeResult.response(); DeleteResponse response = updateResult.writeResult.response();
DeleteRequest deleteRequest = updateResult.request(); DeleteRequest deleteRequest = updateResult.request();
updateResponse = new UpdateResponse(response.getIndex(), response.getType(), response.getId(), response.getVersion(), false); updateResponse = new UpdateResponse(response.getIndex(), response.getType(), response.getId(), response.getVersion(), false);
updateResponse.setGetResult(updateHelper.extractGetResult(updateRequest, response.getVersion(), updateResult.result.updatedSourceAsMap(), updateResult.result.updateSourceContentType(), null)); updateResponse.setGetResult(updateHelper.extractGetResult(updateRequest, shardRequest.request.index(), response.getVersion(), updateResult.result.updatedSourceAsMap(), updateResult.result.updateSourceContentType(), null));
responses[requestIndex] = new BulkItemResponse(item.id(), OP_TYPE_UPDATE, updateResponse); responses[requestIndex] = new BulkItemResponse(item.id(), OP_TYPE_UPDATE, updateResponse);
// Replace the update request to the translated delete request to execute on the replica. // Replace the update request to the translated delete request to execute on the replica.
request.items()[requestIndex] = new BulkItemRequest(request.items()[requestIndex].id(), deleteRequest); request.items()[requestIndex] = new BulkItemRequest(request.items()[requestIndex].id(), deleteRequest);
@ -291,7 +283,7 @@ public class TransportShardBulkAction extends TransportShardReplicationOperation
if (updateAttemptsCount >= updateRequest.retryOnConflict()) { if (updateAttemptsCount >= updateRequest.retryOnConflict()) {
// we can't try any more // we can't try any more
responses[requestIndex] = new BulkItemResponse(item.id(), OP_TYPE_UPDATE, responses[requestIndex] = new BulkItemResponse(item.id(), OP_TYPE_UPDATE,
new BulkItemResponse.Failure(updateRequest.index(), updateRequest.type(), updateRequest.id(), t)); new BulkItemResponse.Failure(request.index(), updateRequest.type(), updateRequest.id(), t));
request.items()[requestIndex] = null; // do not send to replicas request.items()[requestIndex] = null; // do not send to replicas
} }
} else { } else {
@ -304,29 +296,29 @@ public class TransportShardBulkAction extends TransportShardReplicationOperation
throw (ElasticsearchException) t; throw (ElasticsearchException) t;
} }
if (updateResult.result == null) { if (updateResult.result == null) {
responses[requestIndex] = new BulkItemResponse(item.id(), OP_TYPE_UPDATE, new BulkItemResponse.Failure(updateRequest.index(), updateRequest.type(), updateRequest.id(), t)); responses[requestIndex] = new BulkItemResponse(item.id(), OP_TYPE_UPDATE, new BulkItemResponse.Failure(shardRequest.request.index(), updateRequest.type(), updateRequest.id(), t));
} else { } else {
switch (updateResult.result.operation()) { switch (updateResult.result.operation()) {
case UPSERT: case UPSERT:
case INDEX: case INDEX:
IndexRequest indexRequest = updateResult.request(); IndexRequest indexRequest = updateResult.request();
if (t instanceof ElasticsearchException && ((ElasticsearchException) t).status() == RestStatus.CONFLICT) { if (t instanceof ElasticsearchException && ((ElasticsearchException) t).status() == RestStatus.CONFLICT) {
logger.trace("[{}][{}] failed to execute bulk item (index) {}", t, shardRequest.request.index(), shardRequest.shardId, indexRequest); logger.trace("{} failed to execute bulk item (index) {}", t, shardRequest.shardId, indexRequest);
} else { } else {
logger.debug("[{}][{}] failed to execute bulk item (index) {}", t, shardRequest.request.index(), shardRequest.shardId, indexRequest); logger.debug("{} failed to execute bulk item (index) {}", t, shardRequest.shardId, indexRequest);
} }
responses[requestIndex] = new BulkItemResponse(item.id(), OP_TYPE_UPDATE, responses[requestIndex] = new BulkItemResponse(item.id(), OP_TYPE_UPDATE,
new BulkItemResponse.Failure(indexRequest.index(), indexRequest.type(), indexRequest.id(), t)); new BulkItemResponse.Failure(request.index(), indexRequest.type(), indexRequest.id(), t));
break; break;
case DELETE: case DELETE:
DeleteRequest deleteRequest = updateResult.request(); DeleteRequest deleteRequest = updateResult.request();
if (t instanceof ElasticsearchException && ((ElasticsearchException) t).status() == RestStatus.CONFLICT) { if (t instanceof ElasticsearchException && ((ElasticsearchException) t).status() == RestStatus.CONFLICT) {
logger.trace("[{}][{}] failed to execute bulk item (delete) {}", t, shardRequest.request.index(), shardRequest.shardId, deleteRequest); logger.trace("{} failed to execute bulk item (delete) {}", t, shardRequest.shardId, deleteRequest);
} else { } else {
logger.debug("[{}][{}] failed to execute bulk item (delete) {}", t, shardRequest.request.index(), shardRequest.shardId, deleteRequest); logger.debug("{} failed to execute bulk item (delete) {}", t, shardRequest.shardId, deleteRequest);
} }
responses[requestIndex] = new BulkItemResponse(item.id(), OP_TYPE_DELETE, responses[requestIndex] = new BulkItemResponse(item.id(), OP_TYPE_DELETE,
new BulkItemResponse.Failure(deleteRequest.index(), deleteRequest.type(), deleteRequest.id(), t)); new BulkItemResponse.Failure(request.index(), deleteRequest.type(), deleteRequest.id(), t));
break; break;
} }
} }
@ -359,7 +351,7 @@ public class TransportShardBulkAction extends TransportShardReplicationOperation
// ignore // ignore
} }
} }
BulkShardResponse response = new BulkShardResponse(new ShardId(request.index(), request.shardId()), responses); BulkShardResponse response = new BulkShardResponse(shardRequest.shardId, responses);
return new PrimaryResponse<>(shardRequest.request, response, ops); return new PrimaryResponse<>(shardRequest.request, response, ops);
} }
@ -400,12 +392,12 @@ public class TransportShardBulkAction extends TransportShardReplicationOperation
MappingMetaData mappingMd = clusterState.metaData().index(request.index()).mappingOrDefault(indexRequest.type()); MappingMetaData mappingMd = clusterState.metaData().index(request.index()).mappingOrDefault(indexRequest.type());
if (mappingMd != null && mappingMd.routing().required()) { if (mappingMd != null && mappingMd.routing().required()) {
if (indexRequest.routing() == null) { if (indexRequest.routing() == null) {
throw new RoutingMissingException(indexRequest.index(), indexRequest.type(), indexRequest.id()); throw new RoutingMissingException(request.index(), indexRequest.type(), indexRequest.id());
} }
} }
if (!processed) { if (!processed) {
indexRequest.process(clusterState.metaData(), indexRequest.index(), mappingMd, allowIdGeneration); indexRequest.process(clusterState.metaData(), mappingMd, allowIdGeneration, request.index());
} }
SourceToParse sourceToParse = SourceToParse.source(SourceToParse.Origin.PRIMARY, indexRequest.source()).type(indexRequest.type()).id(indexRequest.id()) SourceToParse sourceToParse = SourceToParse.source(SourceToParse.Origin.PRIMARY, indexRequest.source()).type(indexRequest.type()).id(indexRequest.id())
@ -448,11 +440,11 @@ public class TransportShardBulkAction extends TransportShardReplicationOperation
assert indexRequest.versionType().validateVersionForWrites(indexRequest.version()); assert indexRequest.versionType().validateVersionForWrites(indexRequest.version());
IndexResponse indexResponse = new IndexResponse(indexRequest.index(), indexRequest.type(), indexRequest.id(), version, created); IndexResponse indexResponse = new IndexResponse(request.index(), indexRequest.type(), indexRequest.id(), version, created);
return new WriteResult(indexResponse, mappingTypeToUpdate, op); return new WriteResult(indexResponse, mappingTypeToUpdate, op);
} }
private WriteResult shardDeleteOperation(DeleteRequest deleteRequest, IndexShard indexShard) { private WriteResult shardDeleteOperation(BulkShardRequest request, DeleteRequest deleteRequest, IndexShard indexShard) {
Engine.Delete delete = indexShard.prepareDelete(deleteRequest.type(), deleteRequest.id(), deleteRequest.version(), deleteRequest.versionType(), Engine.Operation.Origin.PRIMARY); Engine.Delete delete = indexShard.prepareDelete(deleteRequest.type(), deleteRequest.id(), deleteRequest.version(), deleteRequest.versionType(), Engine.Operation.Origin.PRIMARY);
indexShard.delete(delete); indexShard.delete(delete);
// update the request with the version so it will go to the replicas // update the request with the version so it will go to the replicas
@ -461,7 +453,7 @@ public class TransportShardBulkAction extends TransportShardReplicationOperation
assert deleteRequest.versionType().validateVersionForWrites(deleteRequest.version()); assert deleteRequest.versionType().validateVersionForWrites(deleteRequest.version());
DeleteResponse deleteResponse = new DeleteResponse(deleteRequest.index(), deleteRequest.type(), deleteRequest.id(), delete.version(), delete.found()); DeleteResponse deleteResponse = new DeleteResponse(request.index(), deleteRequest.type(), deleteRequest.id(), delete.version(), delete.found());
return new WriteResult(deleteResponse, null, null); return new WriteResult(deleteResponse, null, null);
} }
@ -538,7 +530,7 @@ public class TransportShardBulkAction extends TransportShardReplicationOperation
case DELETE: case DELETE:
DeleteRequest deleteRequest = translate.action(); DeleteRequest deleteRequest = translate.action();
try { try {
WriteResult result = shardDeleteOperation(deleteRequest, indexShard); WriteResult result = shardDeleteOperation(bulkShardRequest, deleteRequest, indexShard);
return new UpdateResult(translate, deleteRequest, result); return new UpdateResult(translate, deleteRequest, result);
} catch (Throwable t) { } catch (Throwable t) {
t = ExceptionsHelper.unwrapCause(t); t = ExceptionsHelper.unwrapCause(t);
@ -559,7 +551,7 @@ public class TransportShardBulkAction extends TransportShardReplicationOperation
protected void shardOperationOnReplica(ReplicaOperationRequest shardRequest) { protected void shardOperationOnReplica(ReplicaOperationRequest shardRequest) {
IndexShard indexShard = indicesService.indexServiceSafe(shardRequest.request.index()).shardSafe(shardRequest.shardId); IndexShard indexShard = indicesService.indexServiceSafe(shardRequest.shardId.getIndex()).shardSafe(shardRequest.shardId.id());
final BulkShardRequest request = shardRequest.request; final BulkShardRequest request = shardRequest.request;
for (int i = 0; i < request.items().length; i++) { for (int i = 0; i < request.items().length; i++) {
BulkItemRequest item = request.items()[i]; BulkItemRequest item = request.items()[i];

View File

@ -32,8 +32,8 @@ class IndexDeleteRequest extends IndexReplicationOperationRequest<IndexDeleteReq
private final boolean refresh; private final boolean refresh;
private final long version; private final long version;
IndexDeleteRequest(DeleteRequest request) { IndexDeleteRequest(DeleteRequest request, String concreteIndex) {
super(request.index(), request.timeout(), request.replicationType(), request.consistencyLevel()); super(concreteIndex, request.timeout(), request.replicationType(), request.consistencyLevel());
this.type = request.type(); this.type = request.type();
this.id = request.id(); this.id = request.id();
this.refresh = request.refresh(); this.refresh = request.refresh();

View File

@ -31,8 +31,6 @@ import org.elasticsearch.action.support.replication.TransportShardReplicationOpe
import org.elasticsearch.cluster.ClusterService; import org.elasticsearch.cluster.ClusterService;
import org.elasticsearch.cluster.ClusterState; import org.elasticsearch.cluster.ClusterState;
import org.elasticsearch.cluster.action.shard.ShardStateAction; import org.elasticsearch.cluster.action.shard.ShardStateAction;
import org.elasticsearch.cluster.block.ClusterBlockException;
import org.elasticsearch.cluster.block.ClusterBlockLevel;
import org.elasticsearch.cluster.metadata.MappingMetaData; import org.elasticsearch.cluster.metadata.MappingMetaData;
import org.elasticsearch.cluster.routing.ShardIterator; import org.elasticsearch.cluster.routing.ShardIterator;
import org.elasticsearch.common.inject.Inject; import org.elasticsearch.common.inject.Inject;
@ -98,20 +96,24 @@ public class TransportDeleteAction extends TransportShardReplicationOperationAct
} }
@Override @Override
protected boolean resolveRequest(final ClusterState state, final DeleteRequest request, final ActionListener<DeleteResponse> listener) { protected boolean resolveIndex() {
request.routing(state.metaData().resolveIndexRouting(request.routing(), request.index())); return true;
request.index(state.metaData().concreteSingleIndex(request.index(), request.indicesOptions())); }
if (state.metaData().hasIndex(request.index())) {
@Override
protected boolean resolveRequest(final ClusterState state, final InternalRequest request, final ActionListener<DeleteResponse> listener) {
request.request().routing(state.metaData().resolveIndexRouting(request.request().routing(), request.request().index()));
if (state.metaData().hasIndex(request.concreteIndex())) {
// check if routing is required, if so, do a broadcast delete // check if routing is required, if so, do a broadcast delete
MappingMetaData mappingMd = state.metaData().index(request.index()).mappingOrDefault(request.type()); MappingMetaData mappingMd = state.metaData().index(request.concreteIndex()).mappingOrDefault(request.request().type());
if (mappingMd != null && mappingMd.routing().required()) { if (mappingMd != null && mappingMd.routing().required()) {
if (request.routing() == null) { if (request.request().routing() == null) {
if (request.versionType() != VersionType.INTERNAL) { if (request.request().versionType() != VersionType.INTERNAL) {
// TODO: implement this feature // TODO: implement this feature
throw new ElasticsearchIllegalArgumentException("routing value is required for deleting documents of type [" + request.type() throw new ElasticsearchIllegalArgumentException("routing value is required for deleting documents of type [" + request.request().type()
+ "] while using version_type [" + request.versionType() + "]"); + "] while using version_type [" + request.request().versionType() + "]");
} }
indexDeleteAction.execute(new IndexDeleteRequest(request), new ActionListener<IndexDeleteResponse>() { indexDeleteAction.execute(new IndexDeleteRequest(request.request(), request.concreteIndex()), new ActionListener<IndexDeleteResponse>() {
@Override @Override
public void onResponse(IndexDeleteResponse indexDeleteResponse) { public void onResponse(IndexDeleteResponse indexDeleteResponse) {
// go over the response, see if we have found one, and the version if found // go over the response, see if we have found one, and the version if found
@ -124,7 +126,7 @@ public class TransportDeleteAction extends TransportShardReplicationOperationAct
break; break;
} }
} }
listener.onResponse(new DeleteResponse(request.index(), request.type(), request.id(), version, found)); listener.onResponse(new DeleteResponse(request.concreteIndex(), request.request().type(), request.request().id(), version, found));
} }
@Override @Override
@ -163,20 +165,10 @@ public class TransportDeleteAction extends TransportShardReplicationOperationAct
return new DeleteResponse(); return new DeleteResponse();
} }
@Override
protected ClusterBlockException checkGlobalBlock(ClusterState state, DeleteRequest request) {
return state.blocks().globalBlockedException(ClusterBlockLevel.WRITE);
}
@Override
protected ClusterBlockException checkRequestBlock(ClusterState state, DeleteRequest request) {
return state.blocks().indexBlockedException(ClusterBlockLevel.WRITE, request.index());
}
@Override @Override
protected PrimaryResponse<DeleteResponse, DeleteRequest> shardOperationOnPrimary(ClusterState clusterState, PrimaryOperationRequest shardRequest) { protected PrimaryResponse<DeleteResponse, DeleteRequest> shardOperationOnPrimary(ClusterState clusterState, PrimaryOperationRequest shardRequest) {
DeleteRequest request = shardRequest.request; DeleteRequest request = shardRequest.request;
IndexShard indexShard = indicesService.indexServiceSafe(shardRequest.request.index()).shardSafe(shardRequest.shardId); IndexShard indexShard = indicesService.indexServiceSafe(shardRequest.shardId.getIndex()).shardSafe(shardRequest.shardId.id());
Engine.Delete delete = indexShard.prepareDelete(request.type(), request.id(), request.version(), request.versionType(), Engine.Operation.Origin.PRIMARY); Engine.Delete delete = indexShard.prepareDelete(request.type(), request.id(), request.version(), request.versionType(), Engine.Operation.Origin.PRIMARY);
indexShard.delete(delete); indexShard.delete(delete);
// update the request with teh version so it will go to the replicas // update the request with teh version so it will go to the replicas
@ -193,14 +185,14 @@ public class TransportDeleteAction extends TransportShardReplicationOperationAct
} }
} }
DeleteResponse response = new DeleteResponse(request.index(), request.type(), request.id(), delete.version(), delete.found()); DeleteResponse response = new DeleteResponse(shardRequest.shardId.getIndex(), request.type(), request.id(), delete.version(), delete.found());
return new PrimaryResponse<>(shardRequest.request, response, null); return new PrimaryResponse<>(shardRequest.request, response, null);
} }
@Override @Override
protected void shardOperationOnReplica(ReplicaOperationRequest shardRequest) { protected void shardOperationOnReplica(ReplicaOperationRequest shardRequest) {
DeleteRequest request = shardRequest.request; DeleteRequest request = shardRequest.request;
IndexShard indexShard = indicesService.indexServiceSafe(shardRequest.request.index()).shardSafe(shardRequest.shardId); IndexShard indexShard = indicesService.indexServiceSafe(shardRequest.shardId.getIndex()).shardSafe(shardRequest.shardId.id());
Engine.Delete delete = indexShard.prepareDelete(request.type(), request.id(), request.version(), request.versionType(), Engine.Operation.Origin.REPLICA); Engine.Delete delete = indexShard.prepareDelete(request.type(), request.id(), request.version(), request.versionType(), Engine.Operation.Origin.REPLICA);
indexShard.delete(delete); indexShard.delete(delete);
@ -212,12 +204,11 @@ public class TransportDeleteAction extends TransportShardReplicationOperationAct
// ignore // ignore
} }
} }
} }
@Override @Override
protected ShardIterator shards(ClusterState clusterState, DeleteRequest request) { protected ShardIterator shards(ClusterState clusterState, InternalRequest request) {
return clusterService.operationRouting() return clusterService.operationRouting()
.deleteShards(clusterService.state(), request.index(), request.type(), request.id(), request.routing()); .deleteShards(clusterService.state(), request.concreteIndex(), request.request().type(), request.request().id(), request.request().routing());
} }
} }

View File

@ -25,8 +25,6 @@ import org.elasticsearch.action.support.replication.TransportShardReplicationOpe
import org.elasticsearch.cluster.ClusterService; import org.elasticsearch.cluster.ClusterService;
import org.elasticsearch.cluster.ClusterState; import org.elasticsearch.cluster.ClusterState;
import org.elasticsearch.cluster.action.shard.ShardStateAction; import org.elasticsearch.cluster.action.shard.ShardStateAction;
import org.elasticsearch.cluster.block.ClusterBlockException;
import org.elasticsearch.cluster.block.ClusterBlockLevel;
import org.elasticsearch.cluster.routing.GroupShardsIterator; import org.elasticsearch.cluster.routing.GroupShardsIterator;
import org.elasticsearch.cluster.routing.ShardIterator; import org.elasticsearch.cluster.routing.ShardIterator;
import org.elasticsearch.common.inject.Inject; import org.elasticsearch.common.inject.Inject;
@ -78,19 +76,14 @@ public class TransportShardDeleteAction extends TransportShardReplicationOperati
} }
@Override @Override
protected ClusterBlockException checkGlobalBlock(ClusterState state, ShardDeleteRequest request) { protected boolean resolveIndex() {
return state.blocks().globalBlockedException(ClusterBlockLevel.WRITE); return false;
}
@Override
protected ClusterBlockException checkRequestBlock(ClusterState state, ShardDeleteRequest request) {
return state.blocks().indexBlockedException(ClusterBlockLevel.WRITE, request.index());
} }
@Override @Override
protected PrimaryResponse<ShardDeleteResponse, ShardDeleteRequest> shardOperationOnPrimary(ClusterState clusterState, PrimaryOperationRequest shardRequest) { protected PrimaryResponse<ShardDeleteResponse, ShardDeleteRequest> shardOperationOnPrimary(ClusterState clusterState, PrimaryOperationRequest shardRequest) {
ShardDeleteRequest request = shardRequest.request; ShardDeleteRequest request = shardRequest.request;
IndexShard indexShard = indicesService.indexServiceSafe(shardRequest.request.index()).shardSafe(shardRequest.shardId); IndexShard indexShard = indicesService.indexServiceSafe(shardRequest.shardId.getIndex()).shardSafe(shardRequest.shardId.id());
Engine.Delete delete = indexShard.prepareDelete(request.type(), request.id(), request.version(), VersionType.INTERNAL, Engine.Operation.Origin.PRIMARY); Engine.Delete delete = indexShard.prepareDelete(request.type(), request.id(), request.version(), VersionType.INTERNAL, Engine.Operation.Origin.PRIMARY);
indexShard.delete(delete); indexShard.delete(delete);
// update the version to happen on the replicas // update the version to happen on the replicas
@ -112,7 +105,7 @@ public class TransportShardDeleteAction extends TransportShardReplicationOperati
@Override @Override
protected void shardOperationOnReplica(ReplicaOperationRequest shardRequest) { protected void shardOperationOnReplica(ReplicaOperationRequest shardRequest) {
ShardDeleteRequest request = shardRequest.request; ShardDeleteRequest request = shardRequest.request;
IndexShard indexShard = indicesService.indexServiceSafe(shardRequest.request.index()).shardSafe(shardRequest.shardId); IndexShard indexShard = indicesService.indexServiceSafe(shardRequest.shardId.getIndex()).shardSafe(shardRequest.shardId.id());
Engine.Delete delete = indexShard.prepareDelete(request.type(), request.id(), request.version(), VersionType.INTERNAL, Engine.Operation.Origin.REPLICA); Engine.Delete delete = indexShard.prepareDelete(request.type(), request.id(), request.version(), VersionType.INTERNAL, Engine.Operation.Origin.REPLICA);
// IndexDeleteAction doesn't support version type at the moment. Hard coded for the INTERNAL version // IndexDeleteAction doesn't support version type at the moment. Hard coded for the INTERNAL version
@ -133,13 +126,13 @@ public class TransportShardDeleteAction extends TransportShardReplicationOperati
} }
@Override @Override
protected ShardIterator shards(ClusterState clusterState, ShardDeleteRequest request) { protected ShardIterator shards(ClusterState clusterState, InternalRequest request) {
GroupShardsIterator group = clusterService.operationRouting().broadcastDeleteShards(clusterService.state(), request.index()); GroupShardsIterator group = clusterService.operationRouting().broadcastDeleteShards(clusterService.state(), request.concreteIndex());
for (ShardIterator shardIt : group) { for (ShardIterator shardIt : group) {
if (shardIt.shardId().id() == request.shardId()) { if (shardIt.shardId().id() == request.request().shardId()) {
return shardIt; return shardIt;
} }
} }
throw new ElasticsearchIllegalStateException("No shards iterator found for shard [" + request.shardId() + "]"); throw new ElasticsearchIllegalStateException("No shards iterator found for shard [" + request.request().shardId() + "]");
} }
} }

View File

@ -29,8 +29,6 @@ import org.elasticsearch.cache.recycler.PageCacheRecycler;
import org.elasticsearch.cluster.ClusterService; import org.elasticsearch.cluster.ClusterService;
import org.elasticsearch.cluster.ClusterState; import org.elasticsearch.cluster.ClusterState;
import org.elasticsearch.cluster.action.shard.ShardStateAction; import org.elasticsearch.cluster.action.shard.ShardStateAction;
import org.elasticsearch.cluster.block.ClusterBlockException;
import org.elasticsearch.cluster.block.ClusterBlockLevel;
import org.elasticsearch.cluster.routing.GroupShardsIterator; import org.elasticsearch.cluster.routing.GroupShardsIterator;
import org.elasticsearch.cluster.routing.ShardIterator; import org.elasticsearch.cluster.routing.ShardIterator;
import org.elasticsearch.common.inject.Inject; import org.elasticsearch.common.inject.Inject;
@ -100,20 +98,15 @@ public class TransportShardDeleteByQueryAction extends TransportShardReplication
} }
@Override @Override
protected ClusterBlockException checkGlobalBlock(ClusterState state, ShardDeleteByQueryRequest request) { protected boolean resolveIndex() {
return state.blocks().globalBlockedException(ClusterBlockLevel.WRITE); return false;
}
@Override
protected ClusterBlockException checkRequestBlock(ClusterState state, ShardDeleteByQueryRequest request) {
return state.blocks().indexBlockedException(ClusterBlockLevel.WRITE, request.index());
} }
@Override @Override
protected PrimaryResponse<ShardDeleteByQueryResponse, ShardDeleteByQueryRequest> shardOperationOnPrimary(ClusterState clusterState, PrimaryOperationRequest shardRequest) { protected PrimaryResponse<ShardDeleteByQueryResponse, ShardDeleteByQueryRequest> shardOperationOnPrimary(ClusterState clusterState, PrimaryOperationRequest shardRequest) {
ShardDeleteByQueryRequest request = shardRequest.request; ShardDeleteByQueryRequest request = shardRequest.request;
IndexService indexService = indicesService.indexServiceSafe(shardRequest.request.index()); IndexService indexService = indicesService.indexServiceSafe(shardRequest.shardId.getIndex());
IndexShard indexShard = indexService.shardSafe(shardRequest.shardId); IndexShard indexShard = indexService.shardSafe(shardRequest.shardId.id());
SearchContext.setCurrent(new DefaultSearchContext(0, new ShardSearchRequest().types(request.types()).nowInMillis(request.nowInMillis()), null, SearchContext.setCurrent(new DefaultSearchContext(0, new ShardSearchRequest().types(request.types()).nowInMillis(request.nowInMillis()), null,
indexShard.acquireSearcher(DELETE_BY_QUERY_API), indexService, indexShard, scriptService, cacheRecycler, indexShard.acquireSearcher(DELETE_BY_QUERY_API), indexService, indexShard, scriptService, cacheRecycler,
@ -134,8 +127,8 @@ public class TransportShardDeleteByQueryAction extends TransportShardReplication
@Override @Override
protected void shardOperationOnReplica(ReplicaOperationRequest shardRequest) { protected void shardOperationOnReplica(ReplicaOperationRequest shardRequest) {
ShardDeleteByQueryRequest request = shardRequest.request; ShardDeleteByQueryRequest request = shardRequest.request;
IndexService indexService = indicesService.indexServiceSafe(shardRequest.request.index()); IndexService indexService = indicesService.indexServiceSafe(shardRequest.shardId.getIndex());
IndexShard indexShard = indexService.shardSafe(shardRequest.shardId); IndexShard indexShard = indexService.shardSafe(shardRequest.shardId.id());
SearchContext.setCurrent(new DefaultSearchContext(0, new ShardSearchRequest().types(request.types()).nowInMillis(request.nowInMillis()), null, SearchContext.setCurrent(new DefaultSearchContext(0, new ShardSearchRequest().types(request.types()).nowInMillis(request.nowInMillis()), null,
indexShard.acquireSearcher(DELETE_BY_QUERY_API, IndexShard.Mode.WRITE), indexService, indexShard, scriptService, indexShard.acquireSearcher(DELETE_BY_QUERY_API, IndexShard.Mode.WRITE), indexService, indexShard, scriptService,
@ -145,20 +138,20 @@ public class TransportShardDeleteByQueryAction extends TransportShardReplication
SearchContext.current().parsedQuery(new ParsedQuery(deleteByQuery.query(), ImmutableMap.<String, Filter>of())); SearchContext.current().parsedQuery(new ParsedQuery(deleteByQuery.query(), ImmutableMap.<String, Filter>of()));
indexShard.deleteByQuery(deleteByQuery); indexShard.deleteByQuery(deleteByQuery);
} finally { } finally {
try (SearchContext searchContext = SearchContext.current();) { try (SearchContext searchContext = SearchContext.current()) {
SearchContext.removeCurrent(); SearchContext.removeCurrent();
} }
} }
} }
@Override @Override
protected ShardIterator shards(ClusterState clusterState, ShardDeleteByQueryRequest request) { protected ShardIterator shards(ClusterState clusterState, InternalRequest request) {
GroupShardsIterator group = clusterService.operationRouting().deleteByQueryShards(clusterService.state(), request.index(), request.routing()); GroupShardsIterator group = clusterService.operationRouting().deleteByQueryShards(clusterService.state(), request.concreteIndex(), request.request().routing());
for (ShardIterator shardIt : group) { for (ShardIterator shardIt : group) {
if (shardIt.shardId().id() == request.shardId()) { if (shardIt.shardId().id() == request.request().shardId()) {
return shardIt; return shardIt;
} }
} }
throw new ElasticsearchIllegalStateException("No shards iterator found for shard [" + request.shardId() + "]"); throw new ElasticsearchIllegalStateException("No shards iterator found for shard [" + request.request().shardId() + "]");
} }
} }

View File

@ -30,8 +30,6 @@ import org.elasticsearch.cache.recycler.CacheRecycler;
import org.elasticsearch.cache.recycler.PageCacheRecycler; import org.elasticsearch.cache.recycler.PageCacheRecycler;
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.ClusterBlockLevel;
import org.elasticsearch.cluster.routing.ShardIterator; import org.elasticsearch.cluster.routing.ShardIterator;
import org.elasticsearch.common.inject.Inject; import org.elasticsearch.common.inject.Inject;
import org.elasticsearch.common.settings.Settings; import org.elasticsearch.common.settings.Settings;
@ -41,6 +39,7 @@ import org.elasticsearch.index.get.GetResult;
import org.elasticsearch.index.mapper.Uid; import org.elasticsearch.index.mapper.Uid;
import org.elasticsearch.index.mapper.internal.UidFieldMapper; import org.elasticsearch.index.mapper.internal.UidFieldMapper;
import org.elasticsearch.index.service.IndexService; import org.elasticsearch.index.service.IndexService;
import org.elasticsearch.index.shard.ShardId;
import org.elasticsearch.index.shard.service.IndexShard; import org.elasticsearch.index.shard.service.IndexShard;
import org.elasticsearch.indices.IndicesService; import org.elasticsearch.indices.IndicesService;
import org.elasticsearch.script.ScriptService; import org.elasticsearch.script.ScriptService;
@ -94,24 +93,27 @@ public class TransportExplainAction extends TransportShardSingleOperationAction<
} }
@Override @Override
protected void resolveRequest(ClusterState state, ExplainRequest request) { protected boolean resolveIndex() {
String concreteIndex = state.metaData().concreteSingleIndex(request.index(), request.indicesOptions()); return true;
request.filteringAlias(state.metaData().filteringAliases(concreteIndex, request.index())); }
request.index(state.metaData().concreteSingleIndex(request.index(), request.indicesOptions()));
@Override
protected void resolveRequest(ClusterState state, InternalRequest request) {
request.request().filteringAlias(state.metaData().filteringAliases(request.concreteIndex(), request.request().index()));
// Fail fast on the node that received the request. // Fail fast on the node that received the request.
if (request.routing() == null && state.getMetaData().routingRequired(request.index(), request.type())) { if (request.request().routing() == null && state.getMetaData().routingRequired(request.concreteIndex(), request.request().type())) {
throw new RoutingMissingException(request.index(), request.type(), request.id()); throw new RoutingMissingException(request.concreteIndex(), request.request().type(), request.request().id());
} }
} }
protected ExplainResponse shardOperation(ExplainRequest request, int shardId) throws ElasticsearchException { @Override
IndexService indexService = indicesService.indexServiceSafe(request.index()); protected ExplainResponse shardOperation(ExplainRequest request, ShardId shardId) throws ElasticsearchException {
IndexShard indexShard = indexService.shardSafe(shardId); IndexService indexService = indicesService.indexService(shardId.getIndex());
IndexShard indexShard = indexService.shardSafe(shardId.id());
Term uidTerm = new Term(UidFieldMapper.NAME, Uid.createUidAsBytes(request.type(), request.id())); Term uidTerm = new Term(UidFieldMapper.NAME, Uid.createUidAsBytes(request.type(), request.id()));
Engine.GetResult result = indexShard.get(new Engine.Get(false, uidTerm)); Engine.GetResult result = indexShard.get(new Engine.Get(false, uidTerm));
if (!result.exists()) { if (!result.exists()) {
return new ExplainResponse(request.index(), request.type(), request.id(), false); return new ExplainResponse(shardId.getIndex(), request.type(), request.id(), false);
} }
SearchContext context = new DefaultSearchContext( SearchContext context = new DefaultSearchContext(
@ -139,9 +141,9 @@ public class TransportExplainAction extends TransportShardSingleOperationAction<
// because we are working in the same searcher in engineGetResult we can be sure that a // because we are working in the same searcher in engineGetResult we can be sure that a
// doc isn't deleted between the initial get and this call. // doc isn't deleted between the initial get and this call.
GetResult getResult = indexShard.getService().get(result, request.id(), request.type(), request.fields(), request.fetchSourceContext(), false); GetResult getResult = indexShard.getService().get(result, request.id(), request.type(), request.fields(), request.fetchSourceContext(), false);
return new ExplainResponse(request.index(), request.type(), request.id(), true, explanation, getResult); return new ExplainResponse(shardId.getIndex(), request.type(), request.id(), true, explanation, getResult);
} else { } else {
return new ExplainResponse(request.index(), request.type(), request.id(), true, explanation); return new ExplainResponse(shardId.getIndex(), request.type(), request.id(), true, explanation);
} }
} catch (IOException e) { } catch (IOException e) {
throw new ElasticsearchException("Could not explain", e); throw new ElasticsearchException("Could not explain", e);
@ -159,17 +161,10 @@ public class TransportExplainAction extends TransportShardSingleOperationAction<
return new ExplainResponse(); return new ExplainResponse();
} }
protected ClusterBlockException checkGlobalBlock(ClusterState state, ExplainRequest request) { @Override
return state.blocks().globalBlockedException(ClusterBlockLevel.READ); protected ShardIterator shards(ClusterState state, InternalRequest request) throws ElasticsearchException {
}
protected ClusterBlockException checkRequestBlock(ClusterState state, ExplainRequest request) {
return state.blocks().indexBlockedException(ClusterBlockLevel.READ, request.index());
}
protected ShardIterator shards(ClusterState state, ExplainRequest request) throws ElasticsearchException {
return clusterService.operationRouting().getShards( return clusterService.operationRouting().getShards(
clusterService.state(), request.index(), request.type(), request.id(), request.routing(), request.preference() clusterService.state(), request.concreteIndex(), request.request().type(), request.request().id(), request.request().routing(), request.request().preference()
); );
} }
} }

View File

@ -25,14 +25,13 @@ import org.elasticsearch.action.support.ActionFilters;
import org.elasticsearch.action.support.single.shard.TransportShardSingleOperationAction; import org.elasticsearch.action.support.single.shard.TransportShardSingleOperationAction;
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.ClusterBlockLevel;
import org.elasticsearch.cluster.routing.ShardIterator; import org.elasticsearch.cluster.routing.ShardIterator;
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.index.engine.Engine; import org.elasticsearch.index.engine.Engine;
import org.elasticsearch.index.get.GetResult; import org.elasticsearch.index.get.GetResult;
import org.elasticsearch.index.service.IndexService; import org.elasticsearch.index.service.IndexService;
import org.elasticsearch.index.shard.ShardId;
import org.elasticsearch.index.shard.service.IndexShard; import org.elasticsearch.index.shard.service.IndexShard;
import org.elasticsearch.indices.IndicesService; import org.elasticsearch.indices.IndicesService;
import org.elasticsearch.threadpool.ThreadPool; import org.elasticsearch.threadpool.ThreadPool;
@ -63,40 +62,33 @@ public class TransportGetAction extends TransportShardSingleOperationAction<GetR
} }
@Override @Override
protected ClusterBlockException checkGlobalBlock(ClusterState state, GetRequest request) { protected boolean resolveIndex() {
return state.blocks().globalBlockedException(ClusterBlockLevel.READ); return true;
} }
@Override @Override
protected ClusterBlockException checkRequestBlock(ClusterState state, GetRequest request) { protected ShardIterator shards(ClusterState state, InternalRequest request) {
return state.blocks().indexBlockedException(ClusterBlockLevel.READ, request.index());
}
@Override
protected ShardIterator shards(ClusterState state, GetRequest request) {
return clusterService.operationRouting() return clusterService.operationRouting()
.getShards(clusterService.state(), request.index(), request.type(), request.id(), request.routing(), request.preference()); .getShards(clusterService.state(), request.concreteIndex(), request.request().type(), request.request().id(), request.request().routing(), request.request().preference());
} }
@Override @Override
protected void resolveRequest(ClusterState state, GetRequest request) { protected void resolveRequest(ClusterState state, InternalRequest request) {
if (request.realtime == null) { if (request.request().realtime == null) {
request.realtime = this.realtime; request.request().realtime = this.realtime;
} }
// update the routing (request#index here is possibly an alias) // update the routing (request#index here is possibly an alias)
request.routing(state.metaData().resolveIndexRouting(request.routing(), request.index())); request.request().routing(state.metaData().resolveIndexRouting(request.request().routing(), request.request().index()));
request.index(state.metaData().concreteSingleIndex(request.index(), request.indicesOptions()));
// Fail fast on the node that received the request. // Fail fast on the node that received the request.
if (request.routing() == null && state.getMetaData().routingRequired(request.index(), request.type())) { if (request.request().routing() == null && state.getMetaData().routingRequired(request.concreteIndex(), request.request().type())) {
throw new RoutingMissingException(request.index(), request.type(), request.id()); throw new RoutingMissingException(request.concreteIndex(), request.request().type(), request.request().id());
} }
} }
@Override @Override
protected GetResponse shardOperation(GetRequest request, int shardId) throws ElasticsearchException { protected GetResponse shardOperation(GetRequest request, ShardId shardId) throws ElasticsearchException {
IndexService indexService = indicesService.indexServiceSafe(request.index()); IndexService indexService = indicesService.indexServiceSafe(shardId.getIndex());
IndexShard indexShard = indexService.shardSafe(shardId); IndexShard indexShard = indexService.shardSafe(shardId.id());
if (request.refresh() && !request.realtime()) { if (request.refresh() && !request.realtime()) {
indexShard.refresh(new Engine.Refresh("refresh_flag_get").force(REFRESH_FORCE)); indexShard.refresh(new Engine.Refresh("refresh_flag_get").force(REFRESH_FORCE));

View File

@ -70,16 +70,15 @@ public class TransportMultiGetAction extends HandledTransportAction<MultiGetRequ
responses.set(i, new MultiGetItemResponse(null, new MultiGetResponse.Failure(item.index(), item.type(), item.id(), "[" + item.index() + "] missing"))); responses.set(i, new MultiGetItemResponse(null, new MultiGetResponse.Failure(item.index(), item.type(), item.id(), "[" + item.index() + "] missing")));
continue; continue;
} }
item.routing(clusterState.metaData().resolveIndexRouting(item.routing(), item.index())); item.routing(clusterState.metaData().resolveIndexRouting(item.routing(), item.index()));
item.index(clusterState.metaData().concreteSingleIndex(item.index(), item.indicesOptions())); String concreteSingleIndex = clusterState.metaData().concreteSingleIndex(item.index(), item.indicesOptions());
if (item.routing() == null && clusterState.getMetaData().routingRequired(item.index(), item.type())) { if (item.routing() == null && clusterState.getMetaData().routingRequired(concreteSingleIndex, item.type())) {
responses.set(i, new MultiGetItemResponse(null, new MultiGetResponse.Failure(item.index(), item.type(), item.id(), responses.set(i, new MultiGetItemResponse(null, new MultiGetResponse.Failure(concreteSingleIndex, item.type(), item.id(),
"routing is required for [" + item.index() + "]/[" + item.type() + "]/[" + item.id() + "]"))); "routing is required for [" + concreteSingleIndex + "]/[" + item.type() + "]/[" + item.id() + "]")));
continue; continue;
} }
ShardId shardId = clusterService.operationRouting() ShardId shardId = clusterService.operationRouting()
.getShards(clusterState, item.index(), item.type(), item.id(), item.routing(), null).shardId(); .getShards(clusterState, concreteSingleIndex, item.type(), item.id(), item.routing(), null).shardId();
MultiGetShardRequest shardRequest = shardRequests.get(shardId); MultiGetShardRequest shardRequest = shardRequests.get(shardId);
if (shardRequest == null) { if (shardRequest == null) {
shardRequest = new MultiGetShardRequest(shardId.index().name(), shardId.id()); shardRequest = new MultiGetShardRequest(shardId.index().name(), shardId.id());

View File

@ -26,8 +26,6 @@ import org.elasticsearch.action.support.TransportActions;
import org.elasticsearch.action.support.single.shard.TransportShardSingleOperationAction; import org.elasticsearch.action.support.single.shard.TransportShardSingleOperationAction;
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.ClusterBlockLevel;
import org.elasticsearch.cluster.routing.ShardIterator; import org.elasticsearch.cluster.routing.ShardIterator;
import org.elasticsearch.common.inject.Inject; import org.elasticsearch.common.inject.Inject;
import org.elasticsearch.common.settings.Settings; import org.elasticsearch.common.settings.Settings;
@ -35,6 +33,7 @@ import org.elasticsearch.index.VersionType;
import org.elasticsearch.index.engine.Engine; import org.elasticsearch.index.engine.Engine;
import org.elasticsearch.index.get.GetResult; import org.elasticsearch.index.get.GetResult;
import org.elasticsearch.index.service.IndexService; import org.elasticsearch.index.service.IndexService;
import org.elasticsearch.index.shard.ShardId;
import org.elasticsearch.index.shard.service.IndexShard; import org.elasticsearch.index.shard.service.IndexShard;
import org.elasticsearch.indices.IndicesService; import org.elasticsearch.indices.IndicesService;
import org.elasticsearch.search.fetch.source.FetchSourceContext; import org.elasticsearch.search.fetch.source.FetchSourceContext;
@ -74,34 +73,27 @@ public class TransportShardMultiGetAction extends TransportShardSingleOperationA
} }
@Override @Override
protected ClusterBlockException checkGlobalBlock(ClusterState state, MultiGetShardRequest request) { protected boolean resolveIndex() {
return state.blocks().globalBlockedException(ClusterBlockLevel.READ); return true;
} }
@Override @Override
protected ClusterBlockException checkRequestBlock(ClusterState state, MultiGetShardRequest request) { protected ShardIterator shards(ClusterState state, InternalRequest request) {
return state.blocks().indexBlockedException(ClusterBlockLevel.READ, request.index());
}
@Override
protected ShardIterator shards(ClusterState state, MultiGetShardRequest request) {
return clusterService.operationRouting() return clusterService.operationRouting()
.getShards(clusterService.state(), request.index(), request.shardId(), request.preference()); .getShards(state, request.request().index(), request.request().shardId(), request.request().preference());
} }
@Override @Override
protected void resolveRequest(ClusterState state, MultiGetShardRequest request) { protected void resolveRequest(ClusterState state, InternalRequest request) {
if (request.realtime == null) { if (request.request().realtime == null) {
request.realtime = this.realtime; request.request().realtime = this.realtime;
} }
// no need to set concrete index and routing here, it has already been set by the multi get action on the item
//request.index(state.metaData().concreteIndex(request.index()));
} }
@Override @Override
protected MultiGetShardResponse shardOperation(MultiGetShardRequest request, int shardId) throws ElasticsearchException { protected MultiGetShardResponse shardOperation(MultiGetShardRequest request, ShardId shardId) throws ElasticsearchException {
IndexService indexService = indicesService.indexServiceSafe(request.index()); IndexService indexService = indicesService.indexServiceSafe(shardId.getIndex());
IndexShard indexShard = indexService.shardSafe(shardId); IndexShard indexShard = indexService.shardSafe(shardId.id());
if (request.refresh() && !request.realtime()) { if (request.refresh() && !request.realtime()) {
indexShard.refresh(new Engine.Refresh("refresh_flag_mget").force(TransportGetAction.REFRESH_FORCE)); indexShard.refresh(new Engine.Refresh("refresh_flag_mget").force(TransportGetAction.REFRESH_FORCE));
@ -135,5 +127,4 @@ public class TransportShardMultiGetAction extends TransportShardSingleOperationA
return response; return response;
} }
} }

View File

@ -547,9 +547,9 @@ public class IndexRequest extends ShardReplicationOperationRequest<IndexRequest>
return this.autoGeneratedId; return this.autoGeneratedId;
} }
public void process(MetaData metaData, String aliasOrIndex, @Nullable MappingMetaData mappingMd, boolean allowIdGeneration) throws ElasticsearchException { public void process(MetaData metaData, @Nullable MappingMetaData mappingMd, boolean allowIdGeneration, String concreteIndex) throws ElasticsearchException {
// resolve the routing if needed // resolve the routing if needed
routing(metaData.resolveIndexRouting(routing, aliasOrIndex)); routing(metaData.resolveIndexRouting(routing, index));
// resolve timestamp if provided externally // resolve timestamp if provided externally
if (timestamp != null) { if (timestamp != null) {
timestamp = MappingMetaData.Timestamp.parseStringTimestamp(timestamp, timestamp = MappingMetaData.Timestamp.parseStringTimestamp(timestamp,
@ -592,7 +592,7 @@ public class IndexRequest extends ShardReplicationOperationRequest<IndexRequest>
// might as well check for routing here // might as well check for routing here
if (mappingMd.routing().required() && routing == null) { if (mappingMd.routing().required() && routing == null) {
throw new RoutingMissingException(index, type, id); throw new RoutingMissingException(concreteIndex, type, id);
} }
if (parent != null && !mappingMd.hasParentField()) { if (parent != null && !mappingMd.hasParentField()) {

View File

@ -32,8 +32,6 @@ import org.elasticsearch.cluster.ClusterService;
import org.elasticsearch.cluster.ClusterState; import org.elasticsearch.cluster.ClusterState;
import org.elasticsearch.cluster.action.index.MappingUpdatedAction; import org.elasticsearch.cluster.action.index.MappingUpdatedAction;
import org.elasticsearch.cluster.action.shard.ShardStateAction; import org.elasticsearch.cluster.action.shard.ShardStateAction;
import org.elasticsearch.cluster.block.ClusterBlockException;
import org.elasticsearch.cluster.block.ClusterBlockLevel;
import org.elasticsearch.cluster.metadata.IndexMetaData; import org.elasticsearch.cluster.metadata.IndexMetaData;
import org.elasticsearch.cluster.metadata.MappingMetaData; import org.elasticsearch.cluster.metadata.MappingMetaData;
import org.elasticsearch.cluster.metadata.MetaData; import org.elasticsearch.cluster.metadata.MetaData;
@ -111,15 +109,19 @@ public class TransportIndexAction extends TransportShardReplicationOperationActi
} }
@Override @Override
protected boolean resolveRequest(ClusterState state, IndexRequest request, ActionListener<IndexResponse> indexResponseActionListener) { protected boolean resolveIndex() {
return true;
}
@Override
protected boolean resolveRequest(ClusterState state, InternalRequest request, ActionListener<IndexResponse> indexResponseActionListener) {
MetaData metaData = clusterService.state().metaData(); MetaData metaData = clusterService.state().metaData();
String aliasOrIndex = request.index();
request.index(metaData.concreteSingleIndex(request.index(), request.indicesOptions()));
MappingMetaData mappingMd = null; MappingMetaData mappingMd = null;
if (metaData.hasIndex(request.index())) { if (metaData.hasIndex(request.concreteIndex())) {
mappingMd = metaData.index(request.index()).mappingOrDefault(request.type()); mappingMd = metaData.index(request.concreteIndex()).mappingOrDefault(request.request().type());
} }
request.process(metaData, aliasOrIndex, mappingMd, allowIdGeneration); request.request().process(metaData, mappingMd, allowIdGeneration, request.concreteIndex());
return true; return true;
} }
@ -153,19 +155,9 @@ public class TransportIndexAction extends TransportShardReplicationOperationActi
} }
@Override @Override
protected ClusterBlockException checkGlobalBlock(ClusterState state, IndexRequest request) { protected ShardIterator shards(ClusterState clusterState, InternalRequest request) {
return state.blocks().globalBlockedException(ClusterBlockLevel.WRITE);
}
@Override
protected ClusterBlockException checkRequestBlock(ClusterState state, IndexRequest request) {
return state.blocks().indexBlockedException(ClusterBlockLevel.WRITE, request.index());
}
@Override
protected ShardIterator shards(ClusterState clusterState, IndexRequest request) {
return clusterService.operationRouting() return clusterService.operationRouting()
.indexShards(clusterService.state(), request.index(), request.type(), request.id(), request.routing()); .indexShards(clusterService.state(), request.concreteIndex(), request.request().type(), request.request().id(), request.request().routing());
} }
@Override @Override
@ -173,16 +165,16 @@ public class TransportIndexAction extends TransportShardReplicationOperationActi
final IndexRequest request = shardRequest.request; final IndexRequest request = shardRequest.request;
// validate, if routing is required, that we got routing // validate, if routing is required, that we got routing
IndexMetaData indexMetaData = clusterState.metaData().index(request.index()); IndexMetaData indexMetaData = clusterState.metaData().index(shardRequest.shardId.getIndex());
MappingMetaData mappingMd = indexMetaData.mappingOrDefault(request.type()); MappingMetaData mappingMd = indexMetaData.mappingOrDefault(request.type());
if (mappingMd != null && mappingMd.routing().required()) { if (mappingMd != null && mappingMd.routing().required()) {
if (request.routing() == null) { if (request.routing() == null) {
throw new RoutingMissingException(request.index(), request.type(), request.id()); throw new RoutingMissingException(shardRequest.shardId.getIndex(), request.type(), request.id());
} }
} }
IndexService indexService = indicesService.indexServiceSafe(shardRequest.request.index()); IndexService indexService = indicesService.indexServiceSafe(shardRequest.shardId.getIndex());
IndexShard indexShard = indexService.shardSafe(shardRequest.shardId); IndexShard indexShard = indexService.shardSafe(shardRequest.shardId.id());
SourceToParse sourceToParse = SourceToParse.source(SourceToParse.Origin.PRIMARY, request.source()).type(request.type()).id(request.id()) SourceToParse sourceToParse = SourceToParse.source(SourceToParse.Origin.PRIMARY, request.source()).type(request.type()).id(request.id())
.routing(request.routing()).parent(request.parent()).timestamp(request.timestamp()).ttl(request.ttl()); .routing(request.routing()).parent(request.parent()).timestamp(request.timestamp()).ttl(request.ttl());
long version; long version;
@ -191,7 +183,7 @@ public class TransportIndexAction extends TransportShardReplicationOperationActi
if (request.opType() == IndexRequest.OpType.INDEX) { if (request.opType() == IndexRequest.OpType.INDEX) {
Engine.Index index = indexShard.prepareIndex(sourceToParse, request.version(), request.versionType(), Engine.Operation.Origin.PRIMARY, request.canHaveDuplicates()); Engine.Index index = indexShard.prepareIndex(sourceToParse, request.version(), request.versionType(), Engine.Operation.Origin.PRIMARY, request.canHaveDuplicates());
if (index.parsedDoc().mappingsModified()) { if (index.parsedDoc().mappingsModified()) {
mappingUpdatedAction.updateMappingOnMaster(request.index(), index.docMapper(), indexService.indexUUID()); mappingUpdatedAction.updateMappingOnMaster(shardRequest.shardId.getIndex(), index.docMapper(), indexService.indexUUID());
} }
indexShard.index(index); indexShard.index(index);
version = index.version(); version = index.version();
@ -201,7 +193,7 @@ public class TransportIndexAction extends TransportShardReplicationOperationActi
Engine.Create create = indexShard.prepareCreate(sourceToParse, Engine.Create create = indexShard.prepareCreate(sourceToParse,
request.version(), request.versionType(), Engine.Operation.Origin.PRIMARY, request.canHaveDuplicates(), request.autoGeneratedId()); request.version(), request.versionType(), Engine.Operation.Origin.PRIMARY, request.canHaveDuplicates(), request.autoGeneratedId());
if (create.parsedDoc().mappingsModified()) { if (create.parsedDoc().mappingsModified()) {
mappingUpdatedAction.updateMappingOnMaster(request.index(), create.docMapper(), indexService.indexUUID()); mappingUpdatedAction.updateMappingOnMaster(shardRequest.shardId.getIndex(), create.docMapper(), indexService.indexUUID());
} }
indexShard.create(create); indexShard.create(create);
version = create.version(); version = create.version();
@ -222,13 +214,13 @@ public class TransportIndexAction extends TransportShardReplicationOperationActi
assert request.versionType().validateVersionForWrites(request.version()); assert request.versionType().validateVersionForWrites(request.version());
IndexResponse response = new IndexResponse(request.index(), request.type(), request.id(), version, created); IndexResponse response = new IndexResponse(shardRequest.shardId.getIndex(), request.type(), request.id(), version, created);
return new PrimaryResponse<>(shardRequest.request, response, op); return new PrimaryResponse<>(shardRequest.request, response, op);
} }
@Override @Override
protected void shardOperationOnReplica(ReplicaOperationRequest shardRequest) { protected void shardOperationOnReplica(ReplicaOperationRequest shardRequest) {
IndexShard indexShard = indicesService.indexServiceSafe(shardRequest.request.index()).shardSafe(shardRequest.shardId); IndexShard indexShard = indicesService.indexServiceSafe(shardRequest.shardId.getIndex()).shardSafe(shardRequest.shardId.id());
IndexRequest request = shardRequest.request; IndexRequest request = shardRequest.request;
SourceToParse sourceToParse = SourceToParse.source(SourceToParse.Origin.REPLICA, request.source()).type(request.type()).id(request.id()) SourceToParse sourceToParse = SourceToParse.source(SourceToParse.Origin.REPLICA, request.source()).type(request.type()).id(request.id())
.routing(request.routing()).parent(request.parent()).timestamp(request.timestamp()).ttl(request.ttl()); .routing(request.routing()).parent(request.parent()).timestamp(request.timestamp()).ttl(request.ttl());

View File

@ -26,7 +26,6 @@ import org.elasticsearch.action.UnavailableShardsException;
import org.elasticsearch.action.get.*; import org.elasticsearch.action.get.*;
import org.elasticsearch.action.support.ActionFilters; import org.elasticsearch.action.support.ActionFilters;
import org.elasticsearch.action.support.HandledTransportAction; import org.elasticsearch.action.support.HandledTransportAction;
import org.elasticsearch.action.support.TransportAction;
import org.elasticsearch.action.support.broadcast.BroadcastShardOperationFailedException; import org.elasticsearch.action.support.broadcast.BroadcastShardOperationFailedException;
import org.elasticsearch.cluster.ClusterService; import org.elasticsearch.cluster.ClusterService;
import org.elasticsearch.cluster.ClusterState; import org.elasticsearch.cluster.ClusterState;
@ -41,8 +40,6 @@ import org.elasticsearch.index.shard.ShardId;
import org.elasticsearch.indices.IndexMissingException; import org.elasticsearch.indices.IndexMissingException;
import org.elasticsearch.percolator.PercolatorService; import org.elasticsearch.percolator.PercolatorService;
import org.elasticsearch.threadpool.ThreadPool; import org.elasticsearch.threadpool.ThreadPool;
import org.elasticsearch.transport.BaseTransportRequestHandler;
import org.elasticsearch.transport.TransportChannel;
import org.elasticsearch.transport.TransportService; import org.elasticsearch.transport.TransportService;
import java.util.*; import java.util.*;
@ -195,7 +192,7 @@ public class TransportMultiPercolateAction extends HandledTransportAction<MultiP
ShardId shardId = shard.shardId(); ShardId shardId = shard.shardId();
TransportShardMultiPercolateAction.Request requests = requestsByShard.get(shardId); TransportShardMultiPercolateAction.Request requests = requestsByShard.get(shardId);
if (requests == null) { if (requests == null) {
requestsByShard.put(shardId, requests = new TransportShardMultiPercolateAction.Request(shard.shardId().getIndex(), shardId.id(), percolateRequest.preference())); requestsByShard.put(shardId, requests = new TransportShardMultiPercolateAction.Request(shardId.getIndex(), shardId.getId(), percolateRequest.preference()));
} }
logger.trace("Adding shard[{}] percolate request for item[{}]", shardId, slot); logger.trace("Adding shard[{}] percolate request for item[{}]", shardId, slot);
requests.add(new TransportShardMultiPercolateAction.Request.Item(slot, new PercolateShardRequest(shardId, percolateRequest))); requests.add(new TransportShardMultiPercolateAction.Request.Item(slot, new PercolateShardRequest(shardId, percolateRequest)));

View File

@ -29,8 +29,6 @@ import org.elasticsearch.action.support.single.shard.SingleShardOperationRequest
import org.elasticsearch.action.support.single.shard.TransportShardSingleOperationAction; import org.elasticsearch.action.support.single.shard.TransportShardSingleOperationAction;
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.ClusterBlockLevel;
import org.elasticsearch.cluster.routing.ShardIterator; import org.elasticsearch.cluster.routing.ShardIterator;
import org.elasticsearch.common.inject.Inject; import org.elasticsearch.common.inject.Inject;
import org.elasticsearch.common.io.stream.StreamInput; import org.elasticsearch.common.io.stream.StreamInput;
@ -38,6 +36,7 @@ import org.elasticsearch.common.io.stream.StreamOutput;
import org.elasticsearch.common.settings.Settings; import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.common.text.StringText; import org.elasticsearch.common.text.StringText;
import org.elasticsearch.common.text.Text; import org.elasticsearch.common.text.Text;
import org.elasticsearch.index.shard.ShardId;
import org.elasticsearch.percolator.PercolatorService; import org.elasticsearch.percolator.PercolatorService;
import org.elasticsearch.threadpool.ThreadPool; import org.elasticsearch.threadpool.ThreadPool;
import org.elasticsearch.transport.TransportService; import org.elasticsearch.transport.TransportService;
@ -76,24 +75,19 @@ public class TransportShardMultiPercolateAction extends TransportShardSingleOper
} }
@Override @Override
protected ClusterBlockException checkGlobalBlock(ClusterState state, Request request) { protected boolean resolveIndex() {
return state.blocks().globalBlockedException(ClusterBlockLevel.READ); return false;
} }
@Override @Override
protected ClusterBlockException checkRequestBlock(ClusterState state, Request request) { protected ShardIterator shards(ClusterState state, InternalRequest request) throws ElasticsearchException {
return state.blocks().indexBlockedException(ClusterBlockLevel.READ, request.index());
}
@Override
protected ShardIterator shards(ClusterState state, Request request) throws ElasticsearchException {
return clusterService.operationRouting().getShards( return clusterService.operationRouting().getShards(
clusterService.state(), request.index(), request.shardId(), request.preference state, request.concreteIndex(), request.request().shardId(), request.request().preference
); );
} }
@Override @Override
protected Response shardOperation(Request request, int shardId) throws ElasticsearchException { protected Response shardOperation(Request request, ShardId shardId) throws ElasticsearchException {
// TODO: Look into combining the shard req's docs into one in memory index. // TODO: Look into combining the shard req's docs into one in memory index.
Response response = new Response(); Response response = new Response();
response.items = new ArrayList<>(request.items.size()); response.items = new ArrayList<>(request.items.size());
@ -106,7 +100,7 @@ public class TransportShardMultiPercolateAction extends TransportShardSingleOper
if (TransportActions.isShardNotAvailableException(t)) { if (TransportActions.isShardNotAvailableException(t)) {
throw (ElasticsearchException) t; throw (ElasticsearchException) t;
} else { } else {
logger.debug("[{}][{}] failed to multi percolate", t, request.index(), request.shardId()); logger.debug("{} failed to multi percolate", t, request.shardId());
responseItem = new Response.Item(slot, new StringText(ExceptionsHelper.detailedMessage(t))); responseItem = new Response.Item(slot, new StringText(ExceptionsHelper.detailedMessage(t)));
} }
} }

View File

@ -22,6 +22,7 @@ package org.elasticsearch.action.support.replication;
import org.elasticsearch.ElasticsearchException; import org.elasticsearch.ElasticsearchException;
import org.elasticsearch.ElasticsearchIllegalStateException; import org.elasticsearch.ElasticsearchIllegalStateException;
import org.elasticsearch.ExceptionsHelper; import org.elasticsearch.ExceptionsHelper;
import org.elasticsearch.Version;
import org.elasticsearch.action.ActionListener; import org.elasticsearch.action.ActionListener;
import org.elasticsearch.action.ActionResponse; import org.elasticsearch.action.ActionResponse;
import org.elasticsearch.action.UnavailableShardsException; import org.elasticsearch.action.UnavailableShardsException;
@ -34,6 +35,7 @@ import org.elasticsearch.cluster.ClusterState;
import org.elasticsearch.cluster.ClusterStateObserver; import org.elasticsearch.cluster.ClusterStateObserver;
import org.elasticsearch.cluster.action.shard.ShardStateAction; import org.elasticsearch.cluster.action.shard.ShardStateAction;
import org.elasticsearch.cluster.block.ClusterBlockException; import org.elasticsearch.cluster.block.ClusterBlockException;
import org.elasticsearch.cluster.block.ClusterBlockLevel;
import org.elasticsearch.cluster.metadata.IndexMetaData; import org.elasticsearch.cluster.metadata.IndexMetaData;
import org.elasticsearch.cluster.node.DiscoveryNode; import org.elasticsearch.cluster.node.DiscoveryNode;
import org.elasticsearch.cluster.routing.ShardIterator; import org.elasticsearch.cluster.routing.ShardIterator;
@ -42,12 +44,12 @@ import org.elasticsearch.cluster.routing.ShardRoutingState;
import org.elasticsearch.common.Nullable; 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.common.io.stream.Streamable;
import org.elasticsearch.common.settings.Settings; import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.common.unit.TimeValue; import org.elasticsearch.common.unit.TimeValue;
import org.elasticsearch.common.util.concurrent.AbstractRunnable; import org.elasticsearch.common.util.concurrent.AbstractRunnable;
import org.elasticsearch.index.engine.VersionConflictEngineException; import org.elasticsearch.index.engine.VersionConflictEngineException;
import org.elasticsearch.index.service.IndexService; import org.elasticsearch.index.service.IndexService;
import org.elasticsearch.index.shard.ShardId;
import org.elasticsearch.index.shard.service.IndexShard; import org.elasticsearch.index.shard.service.IndexShard;
import org.elasticsearch.indices.IndicesService; import org.elasticsearch.indices.IndicesService;
import org.elasticsearch.node.NodeClosedException; import org.elasticsearch.node.NodeClosedException;
@ -117,24 +119,29 @@ public abstract class TransportShardReplicationOperationAction<Request extends S
/** /**
* Called once replica operations have been dispatched on the * Called once replica operations have been dispatched on the
*/ */
protected void postPrimaryOperation(Request request, PrimaryResponse<Response, ReplicaRequest> response) { protected void postPrimaryOperation(InternalRequest request, PrimaryResponse<Response, ReplicaRequest> response) {
} }
protected abstract ShardIterator shards(ClusterState clusterState, Request request) throws ElasticsearchException; protected abstract ShardIterator shards(ClusterState clusterState, InternalRequest request) throws ElasticsearchException;
protected abstract boolean checkWriteConsistency(); protected abstract boolean checkWriteConsistency();
protected abstract ClusterBlockException checkGlobalBlock(ClusterState state, Request request); protected ClusterBlockException checkGlobalBlock(ClusterState state) {
return state.blocks().globalBlockedException(ClusterBlockLevel.WRITE);
}
protected abstract ClusterBlockException checkRequestBlock(ClusterState state, Request request); protected ClusterBlockException checkRequestBlock(ClusterState state, InternalRequest request) {
return state.blocks().indexBlockedException(ClusterBlockLevel.WRITE, request.concreteIndex());
}
protected abstract boolean resolveIndex();
/** /**
* Resolves the request, by default, simply setting the concrete index (if its aliased one). If the resolve * Resolves the request, by default doing nothing. If the resolve
* means a different execution, then return false here to indicate not to continue and execute this request. * means a different execution, then return false here to indicate not to continue and execute this request.
*/ */
protected boolean resolveRequest(ClusterState state, Request request, ActionListener<Response> listener) { protected boolean resolveRequest(ClusterState state, InternalRequest request, ActionListener<Response> listener) {
request.index(state.metaData().concreteSingleIndex(request.index(), request.indicesOptions()));
return true; return true;
} }
@ -237,50 +244,32 @@ public abstract class TransportShardReplicationOperationAction<Request extends S
try { try {
shardOperationOnReplica(request); shardOperationOnReplica(request);
} catch (Throwable t) { } catch (Throwable t) {
failReplicaIfNeeded(request.request.index(), request.shardId, t); failReplicaIfNeeded(request.shardId.getIndex(), request.shardId.id(), t);
throw t; throw t;
} }
channel.sendResponse(TransportResponse.Empty.INSTANCE); channel.sendResponse(TransportResponse.Empty.INSTANCE);
} }
} }
protected class PrimaryOperationRequest implements Streamable { protected class PrimaryOperationRequest {
public ShardId shardId;
public int shardId;
public Request request; public Request request;
public PrimaryOperationRequest() { public PrimaryOperationRequest(int shardId, String index, Request request) {
} this.shardId = new ShardId(index, shardId);
public PrimaryOperationRequest(int shardId, Request request) {
this.shardId = shardId;
this.request = request; this.request = request;
} }
@Override
public void readFrom(StreamInput in) throws IOException {
shardId = in.readVInt();
request = newRequestInstance();
request.readFrom(in);
}
@Override
public void writeTo(StreamOutput out) throws IOException {
out.writeVInt(shardId);
request.writeTo(out);
}
} }
protected class ReplicaOperationRequest extends TransportRequest { protected class ReplicaOperationRequest extends TransportRequest {
public int shardId; public ShardId shardId;
public ReplicaRequest request; public ReplicaRequest request;
public ReplicaOperationRequest() { public ReplicaOperationRequest() {
} }
public ReplicaOperationRequest(int shardId, ReplicaRequest request) { public ReplicaOperationRequest(ShardId shardId, ReplicaRequest request) {
super(request); super(request);
this.shardId = shardId; this.shardId = shardId;
this.request = request; this.request = request;
@ -289,30 +278,45 @@ public abstract class TransportShardReplicationOperationAction<Request extends S
@Override @Override
public void readFrom(StreamInput in) throws IOException { public void readFrom(StreamInput in) throws IOException {
super.readFrom(in); super.readFrom(in);
shardId = in.readVInt(); int shard = -1;
if (in.getVersion().onOrAfter(Version.V_1_4_0)) {
shardId = ShardId.readShardId(in);
} else {
shard = in.readVInt();
}
request = newReplicaRequestInstance(); request = newReplicaRequestInstance();
request.readFrom(in); request.readFrom(in);
if (in.getVersion().before(Version.V_1_4_0)) {
assert shard >= 0;
//older nodes will send the concrete index as part of the request
shardId = new ShardId(request.index(), shard);
}
} }
@Override @Override
public void writeTo(StreamOutput out) throws IOException { public void writeTo(StreamOutput out) throws IOException {
super.writeTo(out); super.writeTo(out);
out.writeVInt(shardId); if (out.getVersion().onOrAfter(Version.V_1_4_0)) {
shardId.writeTo(out);
} else {
out.writeVInt(shardId.id());
//older nodes expect the concrete index as part of the request
request.index(shardId.getIndex());
}
request.writeTo(out); request.writeTo(out);
} }
} }
protected class AsyncShardOperationAction { protected class AsyncShardOperationAction {
private final ActionListener<Response> listener; private final ActionListener<Response> listener;
private final Request request; private final InternalRequest internalRequest;
private volatile ShardIterator shardIt; private volatile ShardIterator shardIt;
private final AtomicBoolean primaryOperationStarted = new AtomicBoolean(); private final AtomicBoolean primaryOperationStarted = new AtomicBoolean();
private final ReplicationType replicationType; private final ReplicationType replicationType;
private volatile ClusterStateObserver observer; private volatile ClusterStateObserver observer;
AsyncShardOperationAction(Request request, ActionListener<Response> listener) { AsyncShardOperationAction(Request request, ActionListener<Response> listener) {
this.request = request; this.internalRequest = new InternalRequest(request);
this.listener = listener; this.listener = listener;
if (request.replicationType() != ReplicationType.DEFAULT) { if (request.replicationType() != ReplicationType.DEFAULT) {
@ -323,7 +327,7 @@ public abstract class TransportShardReplicationOperationAction<Request extends S
} }
public void start() { public void start() {
observer = new ClusterStateObserver(clusterService, request.timeout(), logger); this.observer = new ClusterStateObserver(clusterService, internalRequest.request().timeout(), logger);
doStart(); doStart();
} }
@ -332,7 +336,7 @@ public abstract class TransportShardReplicationOperationAction<Request extends S
*/ */
protected boolean doStart() throws ElasticsearchException { protected boolean doStart() throws ElasticsearchException {
try { try {
ClusterBlockException blockException = checkGlobalBlock(observer.observedState(), request); ClusterBlockException blockException = checkGlobalBlock(observer.observedState());
if (blockException != null) { if (blockException != null) {
if (blockException.retryable()) { if (blockException.retryable()) {
logger.trace("cluster is blocked ({}), scheduling a retry", blockException.getMessage()); logger.trace("cluster is blocked ({}), scheduling a retry", blockException.getMessage());
@ -342,11 +346,16 @@ public abstract class TransportShardReplicationOperationAction<Request extends S
throw blockException; throw blockException;
} }
} }
if (resolveIndex()) {
internalRequest.concreteIndex(observer.observedState().metaData().concreteSingleIndex(internalRequest.request().index(), internalRequest.request().indicesOptions()));
} else {
internalRequest.concreteIndex(internalRequest.request().index());
}
// check if we need to execute, and if not, return // check if we need to execute, and if not, return
if (!resolveRequest(observer.observedState(), request, listener)) { if (!resolveRequest(observer.observedState(), internalRequest, listener)) {
return true; return true;
} }
blockException = checkRequestBlock(observer.observedState(), request); blockException = checkRequestBlock(observer.observedState(), internalRequest);
if (blockException != null) { if (blockException != null) {
if (blockException.retryable()) { if (blockException.retryable()) {
logger.trace("cluster is blocked ({}), scheduling a retry", blockException.getMessage()); logger.trace("cluster is blocked ({}), scheduling a retry", blockException.getMessage());
@ -356,7 +365,7 @@ public abstract class TransportShardReplicationOperationAction<Request extends S
throw blockException; throw blockException;
} }
} }
shardIt = shards(observer.observedState(), request); shardIt = shards(observer.observedState(), internalRequest);
} catch (Throwable e) { } catch (Throwable e) {
listener.onFailure(e); listener.onFailure(e);
return true; return true;
@ -387,8 +396,8 @@ public abstract class TransportShardReplicationOperationAction<Request extends S
// check here for consistency // check here for consistency
if (checkWriteConsistency) { if (checkWriteConsistency) {
WriteConsistencyLevel consistencyLevel = defaultWriteConsistencyLevel; WriteConsistencyLevel consistencyLevel = defaultWriteConsistencyLevel;
if (request.consistencyLevel() != WriteConsistencyLevel.DEFAULT) { if (internalRequest.request().consistencyLevel() != WriteConsistencyLevel.DEFAULT) {
consistencyLevel = request.consistencyLevel(); consistencyLevel = internalRequest.request().consistencyLevel();
} }
int requiredNumber = 1; int requiredNumber = 1;
if (consistencyLevel == WriteConsistencyLevel.QUORUM && shardIt.size() > 2) { if (consistencyLevel == WriteConsistencyLevel.QUORUM && shardIt.size() > 2) {
@ -413,8 +422,8 @@ public abstract class TransportShardReplicationOperationAction<Request extends S
foundPrimary = true; foundPrimary = true;
if (shard.currentNodeId().equals(observer.observedState().nodes().localNodeId())) { if (shard.currentNodeId().equals(observer.observedState().nodes().localNodeId())) {
try { try {
if (request.operationThreaded()) { if (internalRequest.request().operationThreaded()) {
request.beforeLocalFork(); internalRequest.request().beforeLocalFork();
threadPool.executor(executor).execute(new Runnable() { threadPool.executor(executor).execute(new Runnable() {
@Override @Override
public void run() { public void run() {
@ -433,7 +442,7 @@ public abstract class TransportShardReplicationOperationAction<Request extends S
} }
} else { } else {
DiscoveryNode node = observer.observedState().nodes().get(shard.currentNodeId()); DiscoveryNode node = observer.observedState().nodes().get(shard.currentNodeId());
transportService.sendRequest(node, actionName, request, transportOptions, new BaseTransportResponseHandler<Response>() { transportService.sendRequest(node, actionName, internalRequest.request(), transportOptions, new BaseTransportResponseHandler<Response>() {
@Override @Override
public Response newInstance() { public Response newInstance() {
@ -483,8 +492,8 @@ public abstract class TransportShardReplicationOperationAction<Request extends S
return; return;
} }
// make it threaded operation so we fork on the discovery listener thread // make it threaded operation so we fork on the discovery listener thread
request.beforeLocalFork(); internalRequest.request().beforeLocalFork();
request.operationThreaded(true); internalRequest.request().operationThreaded(true);
observer.waitForNextChange(new ClusterStateObserver.Listener() { observer.waitForNextChange(new ClusterStateObserver.Listener() {
@Override @Override
@ -510,9 +519,9 @@ public abstract class TransportShardReplicationOperationAction<Request extends S
void raiseTimeoutFailure(TimeValue timeout, @Nullable Throwable failure) { void raiseTimeoutFailure(TimeValue timeout, @Nullable Throwable failure) {
if (failure == null) { if (failure == null) {
if (shardIt == null) { if (shardIt == null) {
failure = new UnavailableShardsException(null, "no available shards: Timeout waiting for [" + timeout + "], request: " + request.toString()); failure = new UnavailableShardsException(null, "no available shards: Timeout waiting for [" + timeout + "], request: " + internalRequest.request().toString());
} else { } else {
failure = new UnavailableShardsException(shardIt.shardId(), "[" + shardIt.size() + "] shardIt, [" + shardIt.sizeActive() + "] active : Timeout waiting for [" + timeout + "], request: " + request.toString()); failure = new UnavailableShardsException(shardIt.shardId(), "[" + shardIt.size() + "] shardIt, [" + shardIt.sizeActive() + "] active : Timeout waiting for [" + timeout + "], request: " + internalRequest.request().toString());
} }
} }
listener.onFailure(failure); listener.onFailure(failure);
@ -520,7 +529,7 @@ public abstract class TransportShardReplicationOperationAction<Request extends S
void performOnPrimary(int primaryShardId, final ShardRouting shard, ClusterState clusterState) { void performOnPrimary(int primaryShardId, final ShardRouting shard, ClusterState clusterState) {
try { try {
PrimaryResponse<Response, ReplicaRequest> response = shardOperationOnPrimary(clusterState, new PrimaryOperationRequest(primaryShardId, request)); PrimaryResponse<Response, ReplicaRequest> response = shardOperationOnPrimary(clusterState, new PrimaryOperationRequest(primaryShardId, internalRequest.concreteIndex(), internalRequest.request()));
performReplicas(response); performReplicas(response);
} catch (Throwable e) { } catch (Throwable e) {
// shard has not been allocated yet, retry it here // shard has not been allocated yet, retry it here
@ -532,11 +541,11 @@ public abstract class TransportShardReplicationOperationAction<Request extends S
} }
if (e instanceof ElasticsearchException && ((ElasticsearchException) e).status() == RestStatus.CONFLICT) { if (e instanceof ElasticsearchException && ((ElasticsearchException) e).status() == RestStatus.CONFLICT) {
if (logger.isTraceEnabled()) { if (logger.isTraceEnabled()) {
logger.trace(shard.shortSummary() + ": Failed to execute [" + request + "]", e); logger.trace(shard.shortSummary() + ": Failed to execute [" + internalRequest.request() + "]", e);
} }
} else { } else {
if (logger.isDebugEnabled()) { if (logger.isDebugEnabled()) {
logger.debug(shard.shortSummary() + ": Failed to execute [" + request + "]", e); logger.debug(shard.shortSummary() + ": Failed to execute [" + internalRequest.request() + "]", e);
} }
} }
listener.onFailure(e); listener.onFailure(e);
@ -545,7 +554,7 @@ public abstract class TransportShardReplicationOperationAction<Request extends S
void performReplicas(final PrimaryResponse<Response, ReplicaRequest> response) { void performReplicas(final PrimaryResponse<Response, ReplicaRequest> response) {
if (ignoreReplicas()) { if (ignoreReplicas()) {
postPrimaryOperation(request, response); postPrimaryOperation(internalRequest, response);
listener.onResponse(response.response()); listener.onResponse(response.response());
return; return;
} }
@ -573,7 +582,7 @@ public abstract class TransportShardReplicationOperationAction<Request extends S
} }
observer.reset(newState); observer.reset(newState);
shardIt = shards(newState, request); shardIt = shards(newState, internalRequest);
while ((shard = shardIt.nextOrNull()) != null) { while ((shard = shardIt.nextOrNull()) != null) {
if (shard.primary()) { if (shard.primary()) {
if (originalPrimaryShard.currentNodeId().equals(shard.currentNodeId())) { if (originalPrimaryShard.currentNodeId().equals(shard.currentNodeId())) {
@ -585,12 +594,12 @@ public abstract class TransportShardReplicationOperationAction<Request extends S
} }
} }
shardIt.reset(); shardIt.reset();
request.setCanHaveDuplicates(); // safe side, cluster state changed, we might have dups internalRequest.request().setCanHaveDuplicates(); // safe side, cluster state changed, we might have dups
} else{ } else{
shardIt.reset(); shardIt.reset();
while ((shard = shardIt.nextOrNull()) != null) { while ((shard = shardIt.nextOrNull()) != null) {
if (shard.state() != ShardRoutingState.STARTED) { if (shard.state() != ShardRoutingState.STARTED) {
request.setCanHaveDuplicates(); internalRequest.request().setCanHaveDuplicates();
} }
} }
shardIt.reset(); shardIt.reset();
@ -604,13 +613,13 @@ public abstract class TransportShardReplicationOperationAction<Request extends S
} }
if (replicaCounter == 0) { if (replicaCounter == 0) {
postPrimaryOperation(request, response); postPrimaryOperation(internalRequest, response);
listener.onResponse(response.response()); listener.onResponse(response.response());
return; return;
} }
if (replicationType == ReplicationType.ASYNC) { if (replicationType == ReplicationType.ASYNC) {
postPrimaryOperation(request, response); postPrimaryOperation(internalRequest, response);
// async replication, notify the listener // async replication, notify the listener
listener.onResponse(response.response()); listener.onResponse(response.response());
// now, trick the counter so it won't decrease to 0 and notify the listeners // now, trick the counter so it won't decrease to 0 and notify the listeners
@ -622,7 +631,7 @@ public abstract class TransportShardReplicationOperationAction<Request extends S
AtomicInteger counter = new AtomicInteger(replicaCounter); AtomicInteger counter = new AtomicInteger(replicaCounter);
IndexMetaData indexMetaData = observer.observedState().metaData().index(request.index()); IndexMetaData indexMetaData = observer.observedState().metaData().index(internalRequest.concreteIndex());
if (newPrimaryShard != null) { if (newPrimaryShard != null) {
performOnReplica(response, counter, newPrimaryShard, newPrimaryShard.currentNodeId(), indexMetaData); performOnReplica(response, counter, newPrimaryShard, newPrimaryShard.currentNodeId(), indexMetaData);
@ -657,7 +666,7 @@ public abstract class TransportShardReplicationOperationAction<Request extends S
} }
// now do the postPrimary operation, and check if the listener needs to be invoked // now do the postPrimary operation, and check if the listener needs to be invoked
postPrimaryOperation(request, response); postPrimaryOperation(internalRequest, response);
// we also invoke here in case replicas finish before postPrimaryAction does // we also invoke here in case replicas finish before postPrimaryAction does
if (counter.decrementAndGet() == 0) { if (counter.decrementAndGet() == 0) {
listener.onResponse(response.response()); listener.onResponse(response.response());
@ -674,7 +683,7 @@ public abstract class TransportShardReplicationOperationAction<Request extends S
return; return;
} }
final ReplicaOperationRequest shardRequest = new ReplicaOperationRequest(shardIt.shardId().id(), response.replicaRequest()); final ReplicaOperationRequest shardRequest = new ReplicaOperationRequest(shardIt.shardId(), response.replicaRequest());
if (!nodeId.equals(observer.observedState().nodes().localNodeId())) { if (!nodeId.equals(observer.observedState().nodes().localNodeId())) {
final DiscoveryNode node = observer.observedState().nodes().get(nodeId); final DiscoveryNode node = observer.observedState().nodes().get(nodeId);
transportService.sendRequest(node, transportReplicaAction, shardRequest, transportOptions, new EmptyTransportResponseHandler(ThreadPool.Names.SAME) { transportService.sendRequest(node, transportReplicaAction, shardRequest, transportOptions, new EmptyTransportResponseHandler(ThreadPool.Names.SAME) {
@ -685,7 +694,7 @@ public abstract class TransportShardReplicationOperationAction<Request extends S
@Override @Override
public void handleException(TransportException exp) { public void handleException(TransportException exp) {
logger.trace("[{}] Transport failure during replica request [{}] ", exp, node, request); logger.trace("[{}] Transport failure during replica request [{}] ", exp, node, internalRequest.request());
if (!ignoreReplicaException(exp)) { if (!ignoreReplicaException(exp)) {
logger.warn("Failed to perform " + actionName + " on remote replica " + node + shardIt.shardId(), exp); logger.warn("Failed to perform " + actionName + " on remote replica " + node + shardIt.shardId(), exp);
shardStateAction.shardFailed(shard, indexMetaData.getUUID(), shardStateAction.shardFailed(shard, indexMetaData.getUUID(),
@ -701,8 +710,8 @@ public abstract class TransportShardReplicationOperationAction<Request extends S
} }
}); });
} else { } else {
if (request.operationThreaded()) { if (internalRequest.request().operationThreaded()) {
request.beforeLocalFork(); internalRequest.request().beforeLocalFork();
try { try {
threadPool.executor(executor).execute(new AbstractRunnable() { threadPool.executor(executor).execute(new AbstractRunnable() {
@Override @Override
@ -786,4 +795,28 @@ public abstract class TransportShardReplicationOperationAction<Request extends S
return payload; return payload;
} }
} }
/**
* Internal request class that gets built on each node. Holds the original request plus additional info.
*/
protected class InternalRequest {
final Request request;
String concreteIndex;
InternalRequest(Request request) {
this.request = request;
}
public Request request() {
return request;
}
void concreteIndex(String concreteIndex) {
this.concreteIndex = concreteIndex;
}
public String concreteIndex() {
return concreteIndex;
}
}
} }

View File

@ -19,6 +19,7 @@
package org.elasticsearch.action.support.single.custom; package org.elasticsearch.action.support.single.custom;
import org.elasticsearch.ElasticsearchException; import org.elasticsearch.ElasticsearchException;
import org.elasticsearch.Version;
import org.elasticsearch.action.ActionListener; import org.elasticsearch.action.ActionListener;
import org.elasticsearch.action.ActionResponse; import org.elasticsearch.action.ActionResponse;
import org.elasticsearch.action.NoShardAvailableActionException; import org.elasticsearch.action.NoShardAvailableActionException;
@ -35,6 +36,7 @@ import org.elasticsearch.cluster.routing.ShardsIterator;
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.settings.Settings; import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.index.shard.ShardId;
import org.elasticsearch.threadpool.ThreadPool; import org.elasticsearch.threadpool.ThreadPool;
import org.elasticsearch.transport.*; import org.elasticsearch.transport.*;
@ -74,9 +76,13 @@ public abstract class TransportSingleCustomOperationAction<Request extends Singl
/** /**
* Can return null to execute on this local node. * Can return null to execute on this local node.
*/ */
protected abstract ShardsIterator shards(ClusterState state, Request request); protected abstract ShardsIterator shards(ClusterState state, InternalRequest request);
protected abstract Response shardOperation(Request request, int shardId) throws ElasticsearchException; /**
* 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) throws ElasticsearchException;
protected abstract Request newRequest(); protected abstract Request newRequest();
@ -86,22 +92,23 @@ public abstract class TransportSingleCustomOperationAction<Request extends Singl
return state.blocks().globalBlockedException(ClusterBlockLevel.READ); return state.blocks().globalBlockedException(ClusterBlockLevel.READ);
} }
protected ClusterBlockException checkRequestBlock(ClusterState state, Request request) { protected ClusterBlockException checkRequestBlock(ClusterState state, InternalRequest request) {
return state.blocks().indexBlockedException(ClusterBlockLevel.READ, request.index()); return state.blocks().indexBlockedException(ClusterBlockLevel.READ, request.concreteIndex());
} }
protected abstract boolean resolveIndex(Request request);
private class AsyncSingleAction { private class AsyncSingleAction {
private final ActionListener<Response> listener; private final ActionListener<Response> listener;
private final ShardsIterator shardsIt; private final ShardsIterator shardsIt;
private final Request request; private final InternalRequest internalRequest;
private final DiscoveryNodes nodes; private final DiscoveryNodes nodes;
private AsyncSingleAction(Request request, ActionListener<Response> listener) { private AsyncSingleAction(Request request, ActionListener<Response> listener) {
this.request = request;
this.listener = listener; this.listener = listener;
ClusterState clusterState = clusterService.state(); ClusterState clusterState = clusterService.state();
@ -110,11 +117,20 @@ public abstract class TransportSingleCustomOperationAction<Request extends Singl
if (blockException != null) { if (blockException != null) {
throw blockException; throw blockException;
} }
blockException = checkRequestBlock(clusterState, request);
String concreteSingleIndex;
if (resolveIndex(request)) {
concreteSingleIndex = clusterState.metaData().concreteSingleIndex(request.index(), request.indicesOptions());
} else {
concreteSingleIndex = request.index();
}
this.internalRequest = new InternalRequest(request, concreteSingleIndex);
blockException = checkRequestBlock(clusterState, internalRequest);
if (blockException != null) { if (blockException != null) {
throw blockException; throw blockException;
} }
this.shardsIt = shards(clusterState, request); this.shardsIt = shards(clusterState, internalRequest);
} }
public void start() { public void start() {
@ -123,7 +139,7 @@ public abstract class TransportSingleCustomOperationAction<Request extends Singl
private void onFailure(ShardRouting shardRouting, Throwable e) { private void onFailure(ShardRouting shardRouting, Throwable e) {
if (logger.isTraceEnabled() && e != null) { if (logger.isTraceEnabled() && e != null) {
logger.trace(shardRouting.shortSummary() + ": Failed to execute [" + request + "]", e); logger.trace(shardRouting.shortSummary() + ": Failed to execute [" + internalRequest.request() + "]", e);
} }
perform(e); perform(e);
} }
@ -134,13 +150,13 @@ public abstract class TransportSingleCustomOperationAction<Request extends Singl
private void performFirst() { private void performFirst() {
if (shardsIt == null) { if (shardsIt == null) {
// just execute it on the local node // just execute it on the local node
if (request.operationThreaded()) { if (internalRequest.request().operationThreaded()) {
request.beforeLocalFork(); internalRequest.request().beforeLocalFork();
threadPool.executor(executor()).execute(new Runnable() { threadPool.executor(executor()).execute(new Runnable() {
@Override @Override
public void run() { public void run() {
try { try {
Response response = shardOperation(request, -1); Response response = shardOperation(internalRequest.request(), null);
listener.onResponse(response); listener.onResponse(response);
} catch (Throwable e) { } catch (Throwable e) {
onFailure(null, e); onFailure(null, e);
@ -150,7 +166,7 @@ public abstract class TransportSingleCustomOperationAction<Request extends Singl
return; return;
} else { } else {
try { try {
final Response response = shardOperation(request, -1); final Response response = shardOperation(internalRequest.request(), null);
listener.onResponse(response); listener.onResponse(response);
return; return;
} catch (Throwable e) { } catch (Throwable e) {
@ -160,20 +176,20 @@ public abstract class TransportSingleCustomOperationAction<Request extends Singl
return; return;
} }
if (request.preferLocalShard()) { if (internalRequest.request().preferLocalShard()) {
boolean foundLocal = false; boolean foundLocal = false;
ShardRouting shardX; ShardRouting shardX;
while ((shardX = shardsIt.nextOrNull()) != null) { while ((shardX = shardsIt.nextOrNull()) != null) {
final ShardRouting shard = shardX; final ShardRouting shard = shardX;
if (shard.currentNodeId().equals(nodes.localNodeId())) { if (shard.currentNodeId().equals(nodes.localNodeId())) {
foundLocal = true; foundLocal = true;
if (request.operationThreaded()) { if (internalRequest.request().operationThreaded()) {
request.beforeLocalFork(); internalRequest.request().beforeLocalFork();
threadPool.executor(executor()).execute(new Runnable() { threadPool.executor(executor()).execute(new Runnable() {
@Override @Override
public void run() { public void run() {
try { try {
Response response = shardOperation(request, shard.id()); Response response = shardOperation(internalRequest.request(), shard.shardId());
listener.onResponse(response); listener.onResponse(response);
} catch (Throwable e) { } catch (Throwable e) {
shardsIt.reset(); shardsIt.reset();
@ -184,7 +200,7 @@ public abstract class TransportSingleCustomOperationAction<Request extends Singl
return; return;
} else { } else {
try { try {
final Response response = shardOperation(request, shard.id()); final Response response = shardOperation(internalRequest.request(), shard.shardId());
listener.onResponse(response); listener.onResponse(response);
return; return;
} catch (Throwable e) { } catch (Throwable e) {
@ -209,25 +225,25 @@ public abstract class TransportSingleCustomOperationAction<Request extends Singl
if (shard == null) { if (shard == null) {
Throwable failure = lastException; Throwable failure = lastException;
if (failure == null) { if (failure == null) {
failure = new NoShardAvailableActionException(null, "No shard available for [" + request + "]"); failure = new NoShardAvailableActionException(null, "No shard available for [" + internalRequest.request() + "]");
} else { } else {
if (logger.isDebugEnabled()) { if (logger.isDebugEnabled()) {
logger.debug("failed to execute [" + request + "]", failure); logger.debug("failed to execute [" + internalRequest.request() + "]", failure);
} }
} }
listener.onFailure(failure); listener.onFailure(failure);
} else { } else {
if (shard.currentNodeId().equals(nodes.localNodeId())) { if (shard.currentNodeId().equals(nodes.localNodeId())) {
// we don't prefer local shard, so try and do it here // we don't prefer local shard, so try and do it here
if (!request.preferLocalShard()) { if (!internalRequest.request().preferLocalShard()) {
try { try {
if (request.operationThreaded()) { if (internalRequest.request().operationThreaded()) {
request.beforeLocalFork(); internalRequest.request().beforeLocalFork();
threadPool.executor(executor).execute(new Runnable() { threadPool.executor(executor).execute(new Runnable() {
@Override @Override
public void run() { public void run() {
try { try {
Response response = shardOperation(request, shard.id()); Response response = shardOperation(internalRequest.request(), shard.shardId());
listener.onResponse(response); listener.onResponse(response);
} catch (Throwable e) { } catch (Throwable e) {
onFailure(shard, e); onFailure(shard, e);
@ -235,7 +251,7 @@ public abstract class TransportSingleCustomOperationAction<Request extends Singl
} }
}); });
} else { } else {
final Response response = shardOperation(request, shard.id()); final Response response = shardOperation(internalRequest.request(), shard.shardId());
listener.onResponse(response); listener.onResponse(response);
} }
} catch (Throwable e) { } catch (Throwable e) {
@ -246,7 +262,7 @@ public abstract class TransportSingleCustomOperationAction<Request extends Singl
} }
} else { } else {
DiscoveryNode node = nodes.get(shard.currentNodeId()); DiscoveryNode node = nodes.get(shard.currentNodeId());
transportService.sendRequest(node, transportShardAction, new ShardSingleOperationRequest(request, shard.id()), new BaseTransportResponseHandler<Response>() { transportService.sendRequest(node, transportShardAction, new ShardSingleOperationRequest(internalRequest.request(), shard.shardId()), new BaseTransportResponseHandler<Response>() {
@Override @Override
public Response newInstance() { public Response newInstance() {
return newResponse(); return newResponse();
@ -294,12 +310,12 @@ public abstract class TransportSingleCustomOperationAction<Request extends Singl
protected class ShardSingleOperationRequest extends TransportRequest { protected class ShardSingleOperationRequest extends TransportRequest {
private Request request; private Request request;
private int shardId; private ShardId shardId;
ShardSingleOperationRequest() { ShardSingleOperationRequest() {
} }
public ShardSingleOperationRequest(Request request, int shardId) { public ShardSingleOperationRequest(Request request, ShardId shardId) {
super(request); super(request);
this.request = request; this.request = request;
this.shardId = shardId; this.shardId = shardId;
@ -309,7 +325,7 @@ public abstract class TransportSingleCustomOperationAction<Request extends Singl
return request; return request;
} }
public int shardId() { public ShardId shardId() {
return shardId; return shardId;
} }
@ -318,14 +334,48 @@ public abstract class TransportSingleCustomOperationAction<Request extends Singl
super.readFrom(in); super.readFrom(in);
request = newRequest(); request = newRequest();
request.readFrom(in); request.readFrom(in);
shardId = in.readVInt(); if (in.getVersion().onOrAfter(Version.V_1_4_0)) {
shardId = ShardId.readShardId(in);
} else {
//older nodes will send the concrete index as part of the request
shardId = new ShardId(request.index(), in.readVInt());
}
} }
@Override @Override
public void writeTo(StreamOutput out) throws IOException { public void writeTo(StreamOutput out) throws IOException {
super.writeTo(out); super.writeTo(out);
if (out.getVersion().before(Version.V_1_4_0)) {
//older nodes expect the concrete index as part of the request
request.index(shardId.getIndex());
}
request.writeTo(out); request.writeTo(out);
out.writeVInt(shardId); if (out.getVersion().onOrAfter(Version.V_1_4_0)) {
shardId.writeTo(out);
} else {
out.writeVInt(shardId.id());
}
}
}
/**
* 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

@ -29,6 +29,7 @@ import org.elasticsearch.cluster.ClusterService;
import org.elasticsearch.cluster.ClusterState; import org.elasticsearch.cluster.ClusterState;
import org.elasticsearch.cluster.ClusterStateObserver; import org.elasticsearch.cluster.ClusterStateObserver;
import org.elasticsearch.cluster.block.ClusterBlockException; import org.elasticsearch.cluster.block.ClusterBlockException;
import org.elasticsearch.cluster.block.ClusterBlockLevel;
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.ShardIterator;
@ -71,25 +72,25 @@ public abstract class TransportInstanceSingleOperationAction<Request extends Ins
protected abstract String executor(); protected abstract String executor();
protected abstract void shardOperation(Request request, ActionListener<Response> listener) throws ElasticsearchException; protected abstract void shardOperation(InternalRequest request, ActionListener<Response> listener) throws ElasticsearchException;
protected abstract Request newRequest(); protected abstract Request newRequest();
protected abstract Response newResponse(); protected abstract Response newResponse();
protected abstract ClusterBlockException checkGlobalBlock(ClusterState state, Request request); protected ClusterBlockException checkGlobalBlock(ClusterState state) {
return state.blocks().globalBlockedException(ClusterBlockLevel.WRITE);
protected abstract ClusterBlockException checkRequestBlock(ClusterState state, Request request);
/**
* Resolves the request, by default, simply setting the concrete index (if its aliased one). If the resolve
* means a different execution, then return false here to indicate not to continue and execute this request.
*/
protected boolean resolveRequest(ClusterState state, Request request, ActionListener<Response> listener) {
request.index(state.metaData().concreteSingleIndex(request.index(), request.indicesOptions()));
return true;
} }
protected ClusterBlockException checkRequestBlock(ClusterState state, InternalRequest request) {
return state.blocks().indexBlockedException(ClusterBlockLevel.WRITE, request.concreteIndex());
}
/**
* Resolves the request. If the resolve means a different execution, then return false
* here to indicate not to continue and execute this request.
*/
protected abstract boolean resolveRequest(ClusterState state, InternalRequest request, ActionListener<Response> listener);
protected boolean retryOnFailure(Throwable e) { protected boolean retryOnFailure(Throwable e) {
return false; return false;
} }
@ -101,36 +102,31 @@ public abstract class TransportInstanceSingleOperationAction<Request extends Ins
/** /**
* Should return an iterator with a single shard! * Should return an iterator with a single shard!
*/ */
protected abstract ShardIterator shards(ClusterState clusterState, Request request) throws ElasticsearchException; protected abstract ShardIterator shards(ClusterState clusterState, InternalRequest request) throws ElasticsearchException;
class AsyncSingleAction { class AsyncSingleAction {
private final ActionListener<Response> listener; private final ActionListener<Response> listener;
private final InternalRequest internalRequest;
private final Request request; private volatile ClusterStateObserver observer;
private ShardIterator shardIt; private ShardIterator shardIt;
private DiscoveryNodes nodes; private DiscoveryNodes nodes;
private final AtomicBoolean operationStarted = new AtomicBoolean(); private final AtomicBoolean operationStarted = new AtomicBoolean();
private volatile ClusterStateObserver observer;
private AsyncSingleAction(Request request, ActionListener<Response> listener) { private AsyncSingleAction(Request request, ActionListener<Response> listener) {
this.request = request; this.internalRequest = new InternalRequest(request);
this.listener = listener; this.listener = listener;
} }
public void start() { public void start() {
observer = new ClusterStateObserver(clusterService, request.timeout(), logger); this.observer = new ClusterStateObserver(clusterService, internalRequest.request().timeout(), logger);
doStart(); doStart();
} }
protected boolean doStart() throws ElasticsearchException { protected boolean doStart() throws ElasticsearchException {
nodes = observer.observedState().nodes(); nodes = observer.observedState().nodes();
try { try {
ClusterBlockException blockException = checkGlobalBlock(observer.observedState(), request); ClusterBlockException blockException = checkGlobalBlock(observer.observedState());
if (blockException != null) { if (blockException != null) {
if (blockException.retryable()) { if (blockException.retryable()) {
retry(blockException); retry(blockException);
@ -139,11 +135,12 @@ public abstract class TransportInstanceSingleOperationAction<Request extends Ins
throw blockException; throw blockException;
} }
} }
internalRequest.concreteIndex(observer.observedState().metaData().concreteSingleIndex(internalRequest.request().index(), internalRequest.request().indicesOptions()));
// check if we need to execute, and if not, return // check if we need to execute, and if not, return
if (!resolveRequest(observer.observedState(), request, listener)) { if (!resolveRequest(observer.observedState(), internalRequest, listener)) {
return true; return true;
} }
blockException = checkRequestBlock(observer.observedState(), request); blockException = checkRequestBlock(observer.observedState(), internalRequest);
if (blockException != null) { if (blockException != null) {
if (blockException.retryable()) { if (blockException.retryable()) {
retry(blockException); retry(blockException);
@ -152,7 +149,7 @@ public abstract class TransportInstanceSingleOperationAction<Request extends Ins
throw blockException; throw blockException;
} }
} }
shardIt = shards(observer.observedState(), request); shardIt = shards(observer.observedState(), internalRequest);
} catch (Throwable e) { } catch (Throwable e) {
listener.onFailure(e); listener.onFailure(e);
return true; return true;
@ -179,15 +176,15 @@ public abstract class TransportInstanceSingleOperationAction<Request extends Ins
return true; return true;
} }
request.shardId = shardIt.shardId().id(); internalRequest.request().shardId = shardIt.shardId().id();
if (shard.currentNodeId().equals(nodes.localNodeId())) { if (shard.currentNodeId().equals(nodes.localNodeId())) {
request.beforeLocalFork(); internalRequest.request().beforeLocalFork();
try { try {
threadPool.executor(executor).execute(new Runnable() { threadPool.executor(executor).execute(new Runnable() {
@Override @Override
public void run() { public void run() {
try { try {
shardOperation(request, listener); shardOperation(internalRequest, listener);
} catch (Throwable e) { } catch (Throwable e) {
if (retryOnFailure(e)) { if (retryOnFailure(e)) {
operationStarted.set(false); operationStarted.set(false);
@ -209,7 +206,7 @@ public abstract class TransportInstanceSingleOperationAction<Request extends Ins
} }
} else { } else {
DiscoveryNode node = nodes.get(shard.currentNodeId()); DiscoveryNode node = nodes.get(shard.currentNodeId());
transportService.sendRequest(node, actionName, request, transportOptions(), new BaseTransportResponseHandler<Response>() { transportService.sendRequest(node, actionName, internalRequest.request(), transportOptions(), new BaseTransportResponseHandler<Response>() {
@Override @Override
public Response newInstance() { public Response newInstance() {
@ -251,7 +248,7 @@ public abstract class TransportInstanceSingleOperationAction<Request extends Ins
} }
// make it threaded operation so we fork on the discovery listener thread // make it threaded operation so we fork on the discovery listener thread
request.beforeLocalFork(); internalRequest.request().beforeLocalFork();
observer.waitForNextChange(new ClusterStateObserver.Listener() { observer.waitForNextChange(new ClusterStateObserver.Listener() {
@Override @Override
public void onNewClusterState(ClusterState state) { public void onNewClusterState(ClusterState state) {
@ -270,15 +267,15 @@ public abstract class TransportInstanceSingleOperationAction<Request extends Ins
Throwable listenFailure = failure; Throwable listenFailure = failure;
if (listenFailure == null) { if (listenFailure == null) {
if (shardIt == null) { if (shardIt == null) {
listenFailure = new UnavailableShardsException(new ShardId(request.index(), -1), "Timeout waiting for [" + timeout + "], request: " + request.toString()); listenFailure = new UnavailableShardsException(new ShardId(internalRequest.concreteIndex(), -1), "Timeout waiting for [" + timeout + "], request: " + internalRequest.request().toString());
} else { } else {
listenFailure = new UnavailableShardsException(shardIt.shardId(), "[" + shardIt.size() + "] shardIt, [" + shardIt.sizeActive() + "] active : Timeout waiting for [" + timeout + "], request: " + request.toString()); listenFailure = new UnavailableShardsException(shardIt.shardId(), "[" + shardIt.size() + "] shardIt, [" + shardIt.sizeActive() + "] active : Timeout waiting for [" + timeout + "], request: " + internalRequest.request().toString());
} }
} }
listener.onFailure(listenFailure); listener.onFailure(listenFailure);
} }
} }
}, request.timeout()); }, internalRequest.request().timeout());
} }
} }
@ -319,4 +316,28 @@ public abstract class TransportInstanceSingleOperationAction<Request extends Ins
}); });
} }
} }
/**
* Internal request class that gets built on each node. Holds the original request plus additional info.
*/
protected class InternalRequest {
final Request request;
String concreteIndex;
InternalRequest(Request request) {
this.request = request;
}
public Request request() {
return request;
}
void concreteIndex(String concreteIndex) {
this.concreteIndex = concreteIndex;
}
public String concreteIndex() {
return concreteIndex;
}
}
} }

View File

@ -20,6 +20,7 @@
package org.elasticsearch.action.support.single.shard; package org.elasticsearch.action.support.single.shard;
import org.elasticsearch.ElasticsearchException; import org.elasticsearch.ElasticsearchException;
import org.elasticsearch.Version;
import org.elasticsearch.action.ActionListener; import org.elasticsearch.action.ActionListener;
import org.elasticsearch.action.ActionResponse; import org.elasticsearch.action.ActionResponse;
import org.elasticsearch.action.NoShardAvailableActionException; import org.elasticsearch.action.NoShardAvailableActionException;
@ -29,6 +30,7 @@ import org.elasticsearch.action.support.TransportActions;
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;
import org.elasticsearch.cluster.block.ClusterBlockLevel;
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.ShardIterator;
@ -37,6 +39,7 @@ 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.common.settings.Settings; import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.index.shard.ShardId;
import org.elasticsearch.threadpool.ThreadPool; import org.elasticsearch.threadpool.ThreadPool;
import org.elasticsearch.transport.*; import org.elasticsearch.transport.*;
@ -75,32 +78,37 @@ public abstract class TransportShardSingleOperationAction<Request extends Single
protected abstract String executor(); protected abstract String executor();
protected abstract Response shardOperation(Request request, int shardId) throws ElasticsearchException; protected abstract Response shardOperation(Request request, ShardId shardId) throws ElasticsearchException;
protected abstract Request newRequest(); protected abstract Request newRequest();
protected abstract Response newResponse(); protected abstract Response newResponse();
protected abstract ClusterBlockException checkGlobalBlock(ClusterState state, Request request); protected abstract boolean resolveIndex();
protected abstract ClusterBlockException checkRequestBlock(ClusterState state, Request request); protected ClusterBlockException checkGlobalBlock(ClusterState state) {
return state.blocks().globalBlockedException(ClusterBlockLevel.READ);
protected void resolveRequest(ClusterState state, Request request) {
request.index(state.metaData().concreteSingleIndex(request.index(), request.indicesOptions()));
} }
protected abstract ShardIterator shards(ClusterState state, Request request) throws ElasticsearchException; protected ClusterBlockException checkRequestBlock(ClusterState state, InternalRequest request) {
return state.blocks().indexBlockedException(ClusterBlockLevel.READ, request.concreteIndex());
}
protected void resolveRequest(ClusterState state, InternalRequest request) {
}
protected abstract ShardIterator shards(ClusterState state, InternalRequest request) throws ElasticsearchException;
class AsyncSingleAction { class AsyncSingleAction {
private final ActionListener<Response> listener; private final ActionListener<Response> listener;
private final ShardIterator shardIt; private final ShardIterator shardIt;
private final Request request; private final InternalRequest internalRequest;
private final DiscoveryNodes nodes; private final DiscoveryNodes nodes;
private volatile Throwable lastFailure; private volatile Throwable lastFailure;
private AsyncSingleAction(Request request, ActionListener<Response> listener) { private AsyncSingleAction(Request request, ActionListener<Response> listener) {
this.request = request;
this.listener = listener; this.listener = listener;
ClusterState clusterState = clusterService.state(); ClusterState clusterState = clusterService.state();
@ -108,17 +116,26 @@ public abstract class TransportShardSingleOperationAction<Request extends Single
logger.trace("executing [{}] based on cluster state version [{}]", request, clusterState.version()); logger.trace("executing [{}] based on cluster state version [{}]", request, clusterState.version());
} }
nodes = clusterState.nodes(); nodes = clusterState.nodes();
ClusterBlockException blockException = checkGlobalBlock(clusterState, request); ClusterBlockException blockException = checkGlobalBlock(clusterState);
if (blockException != null) {
throw blockException;
}
resolveRequest(clusterState, request);
blockException = checkRequestBlock(clusterState, request);
if (blockException != null) { if (blockException != null) {
throw blockException; throw blockException;
} }
this.shardIt = shards(clusterState, request); String concreteSingleIndex;
if (resolveIndex()) {
concreteSingleIndex = clusterState.metaData().concreteSingleIndex(request.index(), request.indicesOptions());
} else {
concreteSingleIndex = request.index();
}
this.internalRequest = new InternalRequest(request, concreteSingleIndex);
resolveRequest(clusterState, internalRequest);
blockException = checkRequestBlock(clusterState, internalRequest);
if (blockException != null) {
throw blockException;
}
this.shardIt = shards(clusterState, internalRequest);
} }
public void start() { public void start() {
@ -127,7 +144,7 @@ public abstract class TransportShardSingleOperationAction<Request extends Single
private void onFailure(ShardRouting shardRouting, Throwable e) { private void onFailure(ShardRouting shardRouting, Throwable e) {
if (logger.isTraceEnabled() && e != null) { if (logger.isTraceEnabled() && e != null) {
logger.trace("{}: failed to execute [{}]", e, shardRouting, request); logger.trace("{}: failed to execute [{}]", e, shardRouting, internalRequest.request());
} }
perform(e); perform(e);
} }
@ -145,7 +162,7 @@ public abstract class TransportShardSingleOperationAction<Request extends Single
failure = new NoShardAvailableActionException(shardIt.shardId()); failure = new NoShardAvailableActionException(shardIt.shardId());
} else { } else {
if (logger.isDebugEnabled()) { if (logger.isDebugEnabled()) {
logger.debug("{}: failed to execute [{}]", failure, shardIt.shardId(), request); logger.debug("{}: failed to execute [{}]", failure, shardIt.shardId(), internalRequest.request());
} }
} }
listener.onFailure(failure); listener.onFailure(failure);
@ -153,16 +170,16 @@ public abstract class TransportShardSingleOperationAction<Request extends Single
} }
if (shardRouting.currentNodeId().equals(nodes.localNodeId())) { if (shardRouting.currentNodeId().equals(nodes.localNodeId())) {
if (logger.isTraceEnabled()) { if (logger.isTraceEnabled()) {
logger.trace("executing [{}] on shard [{}]", request, shardRouting.shardId()); logger.trace("executing [{}] on shard [{}]", internalRequest.request(), shardRouting.shardId());
} }
try { try {
if (request.operationThreaded()) { if (internalRequest.request().operationThreaded()) {
request.beforeLocalFork(); internalRequest.request().beforeLocalFork();
threadPool.executor(executor).execute(new Runnable() { threadPool.executor(executor).execute(new Runnable() {
@Override @Override
public void run() { public void run() {
try { try {
Response response = shardOperation(request, shardRouting.id()); Response response = shardOperation(internalRequest.request(), shardRouting.shardId());
listener.onResponse(response); listener.onResponse(response);
} catch (Throwable e) { } catch (Throwable e) {
onFailure(shardRouting, e); onFailure(shardRouting, e);
@ -170,7 +187,7 @@ public abstract class TransportShardSingleOperationAction<Request extends Single
} }
}); });
} else { } else {
final Response response = shardOperation(request, shardRouting.id()); final Response response = shardOperation(internalRequest.request(), shardRouting.shardId());
listener.onResponse(response); listener.onResponse(response);
} }
} catch (Throwable e) { } catch (Throwable e) {
@ -181,7 +198,7 @@ public abstract class TransportShardSingleOperationAction<Request extends Single
if (node == null) { if (node == null) {
onFailure(shardRouting, new NoShardAvailableActionException(shardIt.shardId())); onFailure(shardRouting, new NoShardAvailableActionException(shardIt.shardId()));
} else { } else {
transportService.sendRequest(node, transportShardAction, new ShardSingleOperationRequest(request, shardRouting.id()), new BaseTransportResponseHandler<Response>() { transportService.sendRequest(node, transportShardAction, new ShardSingleOperationRequest(internalRequest.request(), shardRouting.shardId()), new BaseTransportResponseHandler<Response>() {
@Override @Override
public Response newInstance() { public Response newInstance() {
@ -274,12 +291,12 @@ public abstract class TransportShardSingleOperationAction<Request extends Single
private Request request; private Request request;
private int shardId; private ShardId shardId;
ShardSingleOperationRequest() { ShardSingleOperationRequest() {
} }
public ShardSingleOperationRequest(Request request, int shardId) { public ShardSingleOperationRequest(Request request, ShardId shardId) {
super(request); super(request);
this.request = request; this.request = request;
this.shardId = shardId; this.shardId = shardId;
@ -289,7 +306,7 @@ public abstract class TransportShardSingleOperationAction<Request extends Single
return request; return request;
} }
public int shardId() { public ShardId shardId() {
return shardId; return shardId;
} }
@ -298,14 +315,48 @@ public abstract class TransportShardSingleOperationAction<Request extends Single
super.readFrom(in); super.readFrom(in);
request = newRequest(); request = newRequest();
request.readFrom(in); request.readFrom(in);
shardId = in.readVInt(); if (in.getVersion().onOrAfter(Version.V_1_4_0)) {
shardId = ShardId.readShardId(in);
} else {
//older nodes will send the concrete index as part of the request
shardId = new ShardId(request.index(), in.readVInt());
}
} }
@Override @Override
public void writeTo(StreamOutput out) throws IOException { public void writeTo(StreamOutput out) throws IOException {
super.writeTo(out); super.writeTo(out);
if (out.getVersion().onOrBefore(Version.V_1_4_0)) {
//older nodes expect the concrete index as part of the request
request.index(shardId.getIndex());
}
request.writeTo(out); request.writeTo(out);
out.writeVInt(shardId); if (out.getVersion().onOrAfter(Version.V_1_4_0)) {
shardId.writeTo(out);
} else {
out.writeVInt(shardId.id());
}
}
}
/**
* 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

@ -68,16 +68,14 @@ public class TransportMultiTermVectorsAction extends HandledTransportAction<Mult
termVectorRequest.type(), termVectorRequest.id(), "[" + termVectorRequest.index() + "] missing"))); termVectorRequest.type(), termVectorRequest.id(), "[" + termVectorRequest.index() + "] missing")));
continue; continue;
} }
termVectorRequest.index(clusterState.metaData().concreteSingleIndex(termVectorRequest.index(), termVectorRequest.indicesOptions())); String concreteSingleIndex = clusterState.metaData().concreteSingleIndex(termVectorRequest.index(), termVectorRequest.indicesOptions());
if (termVectorRequest.routing() == null && clusterState.getMetaData().routingRequired(termVectorRequest.index(), termVectorRequest.type())) { if (termVectorRequest.routing() == null && clusterState.getMetaData().routingRequired(concreteSingleIndex, termVectorRequest.type())) {
responses.set(i, new MultiTermVectorsItemResponse(null, new MultiTermVectorsResponse.Failure(termVectorRequest.index(), termVectorRequest.type(), termVectorRequest.id(), responses.set(i, new MultiTermVectorsItemResponse(null, new MultiTermVectorsResponse.Failure(concreteSingleIndex, termVectorRequest.type(), termVectorRequest.id(),
"routing is required for [" + termVectorRequest.index() + "]/[" + termVectorRequest.type() + "]/[" + termVectorRequest.id() + "]"))); "routing is required for [" + concreteSingleIndex + "]/[" + termVectorRequest.type() + "]/[" + termVectorRequest.id() + "]")));
continue; continue;
} }
ShardId shardId = clusterService ShardId shardId = clusterService.operationRouting().getShards(clusterState, concreteSingleIndex,
.operationRouting() termVectorRequest.type(), termVectorRequest.id(), termVectorRequest.routing(), null).shardId();
.getShards(clusterState, termVectorRequest.index(), termVectorRequest.type(), termVectorRequest.id(),
termVectorRequest.routing(), null).shardId();
MultiTermVectorsShardRequest shardRequest = shardRequests.get(shardId); MultiTermVectorsShardRequest shardRequest = shardRequests.get(shardId);
if (shardRequest == null) { if (shardRequest == null) {
shardRequest = new MultiTermVectorsShardRequest(shardId.index().name(), shardId.id()); shardRequest = new MultiTermVectorsShardRequest(shardId.index().name(), shardId.id());

View File

@ -26,12 +26,11 @@ import org.elasticsearch.action.support.TransportActions;
import org.elasticsearch.action.support.single.shard.TransportShardSingleOperationAction; import org.elasticsearch.action.support.single.shard.TransportShardSingleOperationAction;
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.ClusterBlockLevel;
import org.elasticsearch.cluster.routing.ShardIterator; import org.elasticsearch.cluster.routing.ShardIterator;
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.index.service.IndexService; import org.elasticsearch.index.service.IndexService;
import org.elasticsearch.index.shard.ShardId;
import org.elasticsearch.index.shard.service.IndexShard; import org.elasticsearch.index.shard.service.IndexShard;
import org.elasticsearch.indices.IndicesService; import org.elasticsearch.indices.IndicesService;
import org.elasticsearch.threadpool.ThreadPool; import org.elasticsearch.threadpool.ThreadPool;
@ -66,44 +65,31 @@ public class TransportSingleShardMultiTermsVectorAction extends TransportShardSi
} }
@Override @Override
protected ClusterBlockException checkGlobalBlock(ClusterState state, MultiTermVectorsShardRequest request) { protected boolean resolveIndex() {
return state.blocks().globalBlockedException(ClusterBlockLevel.READ); return false;
} }
@Override @Override
protected ClusterBlockException checkRequestBlock(ClusterState state, MultiTermVectorsShardRequest request) { protected ShardIterator shards(ClusterState state, InternalRequest request) {
return state.blocks().indexBlockedException(ClusterBlockLevel.READ, request.index());
}
@Override
protected ShardIterator shards(ClusterState state, MultiTermVectorsShardRequest request) {
return clusterService.operationRouting() return clusterService.operationRouting()
.getShards(clusterService.state(), request.index(), request.shardId(), request.preference()); .getShards(state, request.concreteIndex(), request.request().shardId(), request.request().preference());
} }
@Override @Override
protected void resolveRequest(ClusterState state, MultiTermVectorsShardRequest request) { protected MultiTermVectorsShardResponse shardOperation(MultiTermVectorsShardRequest request, ShardId shardId) throws ElasticsearchException {
// no need to set concrete index and routing here, it has already been set by the multi term vectors action on the item
// request.index(state.metaData().concreteIndex(request.index()));
}
@Override
protected MultiTermVectorsShardResponse shardOperation(MultiTermVectorsShardRequest request, int shardId) throws ElasticsearchException {
MultiTermVectorsShardResponse response = new MultiTermVectorsShardResponse(); MultiTermVectorsShardResponse response = new MultiTermVectorsShardResponse();
for (int i = 0; i < request.locations.size(); i++) { for (int i = 0; i < request.locations.size(); i++) {
TermVectorRequest termVectorRequest = request.requests.get(i); TermVectorRequest termVectorRequest = request.requests.get(i);
try { try {
IndexService indexService = indicesService.indexServiceSafe(request.index()); IndexService indexService = indicesService.indexServiceSafe(request.index());
IndexShard indexShard = indexService.shardSafe(shardId); IndexShard indexShard = indexService.shardSafe(shardId.id());
TermVectorResponse termVectorResponse = indexShard.termVectorService().getTermVector(termVectorRequest); TermVectorResponse termVectorResponse = indexShard.termVectorService().getTermVector(termVectorRequest, shardId.getIndex());
response.add(request.locations.get(i), termVectorResponse); response.add(request.locations.get(i), termVectorResponse);
} catch (Throwable t) { } catch (Throwable t) {
if (TransportActions.isShardNotAvailableException(t)) { if (TransportActions.isShardNotAvailableException(t)) {
throw (ElasticsearchException) t; throw (ElasticsearchException) t;
} else { } else {
logger.debug("[{}][{}] failed to execute multi term vectors for [{}]/[{}]", t, request.index(), shardId, termVectorRequest.type(), termVectorRequest.id()); logger.debug("{} failed to execute multi term vectors for [{}]/[{}]", t, shardId, termVectorRequest.type(), termVectorRequest.id());
response.add(request.locations.get(i), response.add(request.locations.get(i),
new MultiTermVectorsResponse.Failure(request.index(), termVectorRequest.type(), termVectorRequest.id(), ExceptionsHelper.detailedMessage(t))); new MultiTermVectorsResponse.Failure(request.index(), termVectorRequest.type(), termVectorRequest.id(), ExceptionsHelper.detailedMessage(t)));
} }
@ -112,5 +98,4 @@ public class TransportSingleShardMultiTermsVectorAction extends TransportShardSi
return response; return response;
} }
} }

View File

@ -25,12 +25,11 @@ import org.elasticsearch.action.support.ActionFilters;
import org.elasticsearch.action.support.single.shard.TransportShardSingleOperationAction; import org.elasticsearch.action.support.single.shard.TransportShardSingleOperationAction;
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.ClusterBlockLevel;
import org.elasticsearch.cluster.routing.ShardIterator; import org.elasticsearch.cluster.routing.ShardIterator;
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.index.service.IndexService; import org.elasticsearch.index.service.IndexService;
import org.elasticsearch.index.shard.ShardId;
import org.elasticsearch.index.shard.service.IndexShard; import org.elasticsearch.index.shard.service.IndexShard;
import org.elasticsearch.indices.IndicesService; import org.elasticsearch.indices.IndicesService;
import org.elasticsearch.threadpool.ThreadPool; import org.elasticsearch.threadpool.ThreadPool;
@ -57,38 +56,31 @@ public class TransportSingleShardTermVectorAction extends TransportShardSingleOp
} }
@Override @Override
protected ClusterBlockException checkGlobalBlock(ClusterState state, TermVectorRequest request) { protected ShardIterator shards(ClusterState state, InternalRequest request) {
return state.blocks().globalBlockedException(ClusterBlockLevel.READ); return clusterService.operationRouting().getShards(state, request.concreteIndex(), request.request().type(), request.request().id(),
request.request().routing(), request.request().preference());
} }
@Override @Override
protected ClusterBlockException checkRequestBlock(ClusterState state, TermVectorRequest request) { protected boolean resolveIndex() {
return state.blocks().indexBlockedException(ClusterBlockLevel.READ, request.index()); return true;
} }
@Override @Override
protected ShardIterator shards(ClusterState state, TermVectorRequest request) { protected void resolveRequest(ClusterState state, InternalRequest request) {
return clusterService.operationRouting().getShards(clusterService.state(), request.index(), request.type(), request.id(),
request.routing(), request.preference());
}
@Override
protected void resolveRequest(ClusterState state, TermVectorRequest request) {
// update the routing (request#index here is possibly an alias) // update the routing (request#index here is possibly an alias)
request.routing(state.metaData().resolveIndexRouting(request.routing(), request.index())); request.request().routing(state.metaData().resolveIndexRouting(request.request().routing(), request.request().index()));
request.index(state.metaData().concreteSingleIndex(request.index(), request.indicesOptions()));
// Fail fast on the node that received the request. // Fail fast on the node that received the request.
if (request.routing() == null && state.getMetaData().routingRequired(request.index(), request.type())) { if (request.request().routing() == null && state.getMetaData().routingRequired(request.concreteIndex(), request.request().type())) {
throw new RoutingMissingException(request.index(), request.type(), request.id()); throw new RoutingMissingException(request.concreteIndex(), request.request().type(), request.request().id());
} }
} }
@Override @Override
protected TermVectorResponse shardOperation(TermVectorRequest request, int shardId) throws ElasticsearchException { protected TermVectorResponse shardOperation(TermVectorRequest request, ShardId shardId) throws ElasticsearchException {
IndexService indexService = indicesService.indexServiceSafe(request.index()); IndexService indexService = indicesService.indexServiceSafe(shardId.getIndex());
IndexShard indexShard = indexService.shardSafe(shardId); IndexShard indexShard = indexService.shardSafe(shardId.id());
return indexShard.termVectorService().getTermVector(request); return indexShard.termVectorService().getTermVector(request, shardId.getIndex());
} }
@Override @Override
@ -100,5 +92,4 @@ public class TransportSingleShardTermVectorAction extends TransportShardSingleOp
protected TermVectorResponse newResponse() { protected TermVectorResponse newResponse() {
return new TermVectorResponse(); return new TermVectorResponse();
} }
} }

View File

@ -41,9 +41,6 @@ import org.elasticsearch.action.support.TransportActions;
import org.elasticsearch.action.support.single.instance.TransportInstanceSingleOperationAction; import org.elasticsearch.action.support.single.instance.TransportInstanceSingleOperationAction;
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.ClusterBlockLevel;
import org.elasticsearch.cluster.metadata.MetaData;
import org.elasticsearch.cluster.routing.PlainShardIterator; import org.elasticsearch.cluster.routing.PlainShardIterator;
import org.elasticsearch.cluster.routing.ShardIterator; import org.elasticsearch.cluster.routing.ShardIterator;
import org.elasticsearch.cluster.routing.ShardRouting; import org.elasticsearch.cluster.routing.ShardRouting;
@ -103,31 +100,17 @@ public class TransportUpdateAction extends TransportInstanceSingleOperationActio
return new UpdateResponse(); return new UpdateResponse();
} }
@Override
protected ClusterBlockException checkGlobalBlock(ClusterState state, UpdateRequest request) {
return state.blocks().globalBlockedException(ClusterBlockLevel.WRITE);
}
@Override
protected ClusterBlockException checkRequestBlock(ClusterState state, UpdateRequest request) {
return state.blocks().indexBlockedException(ClusterBlockLevel.WRITE, request.index());
}
@Override @Override
protected boolean retryOnFailure(Throwable e) { protected boolean retryOnFailure(Throwable e) {
return TransportActions.isShardNotAvailableException(e); return TransportActions.isShardNotAvailableException(e);
} }
@Override @Override
protected boolean resolveRequest(ClusterState state, UpdateRequest request, ActionListener<UpdateResponse> listener) { protected boolean resolveRequest(ClusterState state, InternalRequest request, ActionListener<UpdateResponse> listener) {
MetaData metaData = clusterService.state().metaData(); request.request().routing((state.metaData().resolveIndexRouting(request.request().routing(), request.request().index())));
String aliasOrIndex = request.index();
request.routing((metaData.resolveIndexRouting(request.routing(), aliasOrIndex)));
request.index(metaData.concreteSingleIndex(request.index(), request.indicesOptions()));
// Fail fast on the node that received the request, rather than failing when translating on the index or delete request. // Fail fast on the node that received the request, rather than failing when translating on the index or delete request.
if (request.routing() == null && state.getMetaData().routingRequired(request.index(), request.type())) { if (request.request().routing() == null && state.getMetaData().routingRequired(request.concreteIndex(), request.request().type())) {
throw new RoutingMissingException(request.index(), request.type(), request.id()); throw new RoutingMissingException(request.concreteIndex(), request.request().type(), request.request().id());
} }
return true; return true;
} }
@ -167,12 +150,12 @@ public class TransportUpdateAction extends TransportInstanceSingleOperationActio
} }
@Override @Override
protected ShardIterator shards(ClusterState clusterState, UpdateRequest request) throws ElasticsearchException { protected ShardIterator shards(ClusterState clusterState, InternalRequest request) throws ElasticsearchException {
if (request.shardId() != -1) { if (request.request().shardId() != -1) {
return clusterState.routingTable().index(request.index()).shard(request.shardId()).primaryShardIt(); return clusterState.routingTable().index(request.concreteIndex()).shard(request.request().shardId()).primaryShardIt();
} }
ShardIterator shardIterator = clusterService.operationRouting() ShardIterator shardIterator = clusterService.operationRouting()
.indexShards(clusterState, request.index(), request.type(), request.id(), request.routing()); .indexShards(clusterState, request.concreteIndex(), request.request().type(), request.request().id(), request.request().routing());
ShardRouting shard; ShardRouting shard;
while ((shard = shardIterator.nextOrNull()) != null) { while ((shard = shardIterator.nextOrNull()) != null) {
if (shard.primary()) { if (shard.primary()) {
@ -183,12 +166,14 @@ public class TransportUpdateAction extends TransportInstanceSingleOperationActio
} }
@Override @Override
protected void shardOperation(final UpdateRequest request, final ActionListener<UpdateResponse> listener) throws ElasticsearchException { protected void shardOperation(final InternalRequest request, final ActionListener<UpdateResponse> listener) throws ElasticsearchException {
shardOperation(request, listener, 0); shardOperation(request, listener, 0);
} }
protected void shardOperation(final UpdateRequest request, final ActionListener<UpdateResponse> listener, final int retryCount) throws ElasticsearchException { protected void shardOperation(final InternalRequest request, final ActionListener<UpdateResponse> listener, final int retryCount) throws ElasticsearchException {
final UpdateHelper.Result result = updateHelper.prepare(request); IndexService indexService = indicesService.indexServiceSafe(request.concreteIndex());
IndexShard indexShard = indexService.shardSafe(request.request().shardId());
final UpdateHelper.Result result = updateHelper.prepare(request.request(), indexShard);
switch (result.operation()) { switch (result.operation()) {
case UPSERT: case UPSERT:
IndexRequest upsertRequest = result.action(); IndexRequest upsertRequest = result.action();
@ -198,9 +183,9 @@ public class TransportUpdateAction extends TransportInstanceSingleOperationActio
@Override @Override
public void onResponse(IndexResponse response) { public void onResponse(IndexResponse response) {
UpdateResponse update = new UpdateResponse(response.getIndex(), response.getType(), response.getId(), response.getVersion(), response.isCreated()); UpdateResponse update = new UpdateResponse(response.getIndex(), response.getType(), response.getId(), response.getVersion(), response.isCreated());
if (request.fields() != null && request.fields().length > 0) { if (request.request().fields() != null && request.request().fields().length > 0) {
Tuple<XContentType, Map<String, Object>> sourceAndContent = XContentHelper.convertToMap(upsertSourceBytes, true); Tuple<XContentType, Map<String, Object>> sourceAndContent = XContentHelper.convertToMap(upsertSourceBytes, true);
update.setGetResult(updateHelper.extractGetResult(request, response.getVersion(), sourceAndContent.v2(), sourceAndContent.v1(), upsertSourceBytes)); update.setGetResult(updateHelper.extractGetResult(request.request(), request.concreteIndex(), response.getVersion(), sourceAndContent.v2(), sourceAndContent.v1(), upsertSourceBytes));
} else { } else {
update.setGetResult(null); update.setGetResult(null);
} }
@ -211,7 +196,7 @@ public class TransportUpdateAction extends TransportInstanceSingleOperationActio
public void onFailure(Throwable e) { public void onFailure(Throwable e) {
e = ExceptionsHelper.unwrapCause(e); e = ExceptionsHelper.unwrapCause(e);
if (e instanceof VersionConflictEngineException || e instanceof DocumentAlreadyExistsException) { if (e instanceof VersionConflictEngineException || e instanceof DocumentAlreadyExistsException) {
if (retryCount < request.retryOnConflict()) { if (retryCount < request.request().retryOnConflict()) {
threadPool.executor(executor()).execute(new ActionRunnable<UpdateResponse>(listener) { threadPool.executor(executor()).execute(new ActionRunnable<UpdateResponse>(listener) {
@Override @Override
protected void doRun() { protected void doRun() {
@ -233,7 +218,7 @@ public class TransportUpdateAction extends TransportInstanceSingleOperationActio
@Override @Override
public void onResponse(IndexResponse response) { public void onResponse(IndexResponse response) {
UpdateResponse update = new UpdateResponse(response.getIndex(), response.getType(), response.getId(), response.getVersion(), response.isCreated()); UpdateResponse update = new UpdateResponse(response.getIndex(), response.getType(), response.getId(), response.getVersion(), response.isCreated());
update.setGetResult(updateHelper.extractGetResult(request, response.getVersion(), result.updatedSourceAsMap(), result.updateSourceContentType(), indexSourceBytes)); update.setGetResult(updateHelper.extractGetResult(request.request(), request.concreteIndex(), response.getVersion(), result.updatedSourceAsMap(), result.updateSourceContentType(), indexSourceBytes));
listener.onResponse(update); listener.onResponse(update);
} }
@ -241,7 +226,7 @@ public class TransportUpdateAction extends TransportInstanceSingleOperationActio
public void onFailure(Throwable e) { public void onFailure(Throwable e) {
e = ExceptionsHelper.unwrapCause(e); e = ExceptionsHelper.unwrapCause(e);
if (e instanceof VersionConflictEngineException) { if (e instanceof VersionConflictEngineException) {
if (retryCount < request.retryOnConflict()) { if (retryCount < request.request().retryOnConflict()) {
threadPool.executor(executor()).execute(new ActionRunnable<UpdateResponse>(listener) { threadPool.executor(executor()).execute(new ActionRunnable<UpdateResponse>(listener) {
@Override @Override
protected void doRun() { protected void doRun() {
@ -261,7 +246,7 @@ public class TransportUpdateAction extends TransportInstanceSingleOperationActio
@Override @Override
public void onResponse(DeleteResponse response) { public void onResponse(DeleteResponse response) {
UpdateResponse update = new UpdateResponse(response.getIndex(), response.getType(), response.getId(), response.getVersion(), false); UpdateResponse update = new UpdateResponse(response.getIndex(), response.getType(), response.getId(), response.getVersion(), false);
update.setGetResult(updateHelper.extractGetResult(request, response.getVersion(), result.updatedSourceAsMap(), result.updateSourceContentType(), null)); update.setGetResult(updateHelper.extractGetResult(request.request(), request.concreteIndex(), response.getVersion(), result.updatedSourceAsMap(), result.updateSourceContentType(), null));
listener.onResponse(update); listener.onResponse(update);
} }
@ -269,7 +254,7 @@ public class TransportUpdateAction extends TransportInstanceSingleOperationActio
public void onFailure(Throwable e) { public void onFailure(Throwable e) {
e = ExceptionsHelper.unwrapCause(e); e = ExceptionsHelper.unwrapCause(e);
if (e instanceof VersionConflictEngineException) { if (e instanceof VersionConflictEngineException) {
if (retryCount < request.retryOnConflict()) { if (retryCount < request.request().retryOnConflict()) {
threadPool.executor(executor()).execute(new ActionRunnable<UpdateResponse>(listener) { threadPool.executor(executor()).execute(new ActionRunnable<UpdateResponse>(listener) {
@Override @Override
protected void doRun() { protected void doRun() {
@ -286,10 +271,9 @@ public class TransportUpdateAction extends TransportInstanceSingleOperationActio
case NONE: case NONE:
UpdateResponse update = result.action(); UpdateResponse update = result.action();
listener.onResponse(update); listener.onResponse(update);
IndexService indexService = indicesService.indexService(request.index()); IndexService indexServiceOrNull = indicesService.indexService(request.concreteIndex());
if (indexService != null) { if (indexServiceOrNull != null) {
IndexShard indexShard = indexService.shard(request.shardId()); indexService.shard(request.request().shardId()).indexingService().noopUpdate(request.request().type());
indexShard.indexingService().noopUpdate(request.type());
} }
break; break;
default: default:

View File

@ -41,10 +41,8 @@ import org.elasticsearch.index.get.GetResult;
import org.elasticsearch.index.mapper.internal.ParentFieldMapper; import org.elasticsearch.index.mapper.internal.ParentFieldMapper;
import org.elasticsearch.index.mapper.internal.RoutingFieldMapper; import org.elasticsearch.index.mapper.internal.RoutingFieldMapper;
import org.elasticsearch.index.mapper.internal.TTLFieldMapper; import org.elasticsearch.index.mapper.internal.TTLFieldMapper;
import org.elasticsearch.index.service.IndexService;
import org.elasticsearch.index.shard.ShardId; import org.elasticsearch.index.shard.ShardId;
import org.elasticsearch.index.shard.service.IndexShard; import org.elasticsearch.index.shard.service.IndexShard;
import org.elasticsearch.indices.IndicesService;
import org.elasticsearch.script.ExecutableScript; import org.elasticsearch.script.ExecutableScript;
import org.elasticsearch.script.ScriptService; import org.elasticsearch.script.ScriptService;
import org.elasticsearch.search.fetch.source.FetchSourceContext; import org.elasticsearch.search.fetch.source.FetchSourceContext;
@ -61,25 +59,18 @@ import static com.google.common.collect.Maps.newHashMapWithExpectedSize;
*/ */
public class UpdateHelper extends AbstractComponent { public class UpdateHelper extends AbstractComponent {
private final IndicesService indicesService;
private final ScriptService scriptService; private final ScriptService scriptService;
@Inject @Inject
public UpdateHelper(Settings settings, IndicesService indicesService, ScriptService scriptService) { public UpdateHelper(Settings settings, ScriptService scriptService) {
super(settings); super(settings);
this.indicesService = indicesService;
this.scriptService = scriptService; this.scriptService = scriptService;
} }
/** /**
* Prepares an update request by converting it into an index or delete request or an update response (no action). * Prepares an update request by converting it into an index or delete request or an update response (no action).
*/ */
public Result prepare(UpdateRequest request) { @SuppressWarnings("unchecked")
IndexService indexService = indicesService.indexServiceSafe(request.index());
IndexShard indexShard = indexService.shardSafe(request.shardId());
return prepare(request, indexShard);
}
public Result prepare(UpdateRequest request, IndexShard indexShard) { public Result prepare(UpdateRequest request, IndexShard indexShard) {
long getDate = System.currentTimeMillis(); long getDate = System.currentTimeMillis();
final GetResult getResult = indexShard.getService().get(request.type(), request.id(), final GetResult getResult = indexShard.getService().get(request.type(), request.id(),
@ -88,7 +79,7 @@ public class UpdateHelper extends AbstractComponent {
if (!getResult.isExists()) { if (!getResult.isExists()) {
if (request.upsertRequest() == null && !request.docAsUpsert()) { if (request.upsertRequest() == null && !request.docAsUpsert()) {
throw new DocumentMissingException(new ShardId(request.index(), request.shardId()), request.type(), request.id()); throw new DocumentMissingException(new ShardId(indexShard.indexService().index().name(), request.shardId()), request.type(), request.id());
} }
Long ttl = null; Long ttl = null;
IndexRequest indexRequest = request.docAsUpsert() ? request.doc() : request.upsertRequest(); IndexRequest indexRequest = request.docAsUpsert() ? request.doc() : request.upsertRequest();
@ -152,12 +143,12 @@ public class UpdateHelper extends AbstractComponent {
if (getResult.internalSourceRef() == null) { if (getResult.internalSourceRef() == null) {
// no source, we can't do nothing, through a failure... // no source, we can't do nothing, through a failure...
throw new DocumentSourceMissingException(new ShardId(request.index(), request.shardId()), request.type(), request.id()); throw new DocumentSourceMissingException(new ShardId(indexShard.indexService().index().name(), request.shardId()), request.type(), request.id());
} }
Tuple<XContentType, Map<String, Object>> sourceAndContent = XContentHelper.convertToMap(getResult.internalSourceRef(), true); Tuple<XContentType, Map<String, Object>> sourceAndContent = XContentHelper.convertToMap(getResult.internalSourceRef(), true);
String operation = null; String operation = null;
String timestamp = null; String timestamp;
Long ttl = null; Long ttl = null;
final Map<String, Object> updatedSourceAsMap; final Map<String, Object> updatedSourceAsMap;
final XContentType updateSourceContentType = sourceAndContent.v1(); final XContentType updateSourceContentType = sourceAndContent.v1();
@ -232,7 +223,7 @@ public class UpdateHelper extends AbstractComponent {
return new Result(deleteRequest, Operation.DELETE, updatedSourceAsMap, updateSourceContentType); return new Result(deleteRequest, Operation.DELETE, updatedSourceAsMap, updateSourceContentType);
} else if ("none".equals(operation)) { } else if ("none".equals(operation)) {
UpdateResponse update = new UpdateResponse(getResult.getIndex(), getResult.getType(), getResult.getId(), getResult.getVersion(), false); UpdateResponse update = new UpdateResponse(getResult.getIndex(), getResult.getType(), getResult.getId(), getResult.getVersion(), false);
update.setGetResult(extractGetResult(request, getResult.getVersion(), updatedSourceAsMap, updateSourceContentType, getResult.internalSourceRef())); update.setGetResult(extractGetResult(request, indexShard.indexService().index().name(), getResult.getVersion(), updatedSourceAsMap, updateSourceContentType, getResult.internalSourceRef()));
return new Result(update, Operation.NONE, updatedSourceAsMap, updateSourceContentType); return new Result(update, Operation.NONE, updatedSourceAsMap, updateSourceContentType);
} else { } else {
logger.warn("Used update operation [{}] for script [{}], doing nothing...", operation, request.script); logger.warn("Used update operation [{}] for script [{}], doing nothing...", operation, request.script);
@ -257,7 +248,7 @@ public class UpdateHelper extends AbstractComponent {
/** /**
* Extracts the fields from the updated document to be returned in a update response * Extracts the fields from the updated document to be returned in a update response
*/ */
public GetResult extractGetResult(final UpdateRequest request, long version, final Map<String, Object> source, XContentType sourceContentType, @Nullable final BytesReference sourceAsBytes) { public GetResult extractGetResult(final UpdateRequest request, String concreteIndex, long version, final Map<String, Object> source, XContentType sourceContentType, @Nullable final BytesReference sourceAsBytes) {
if (request.fields() == null || request.fields().length == 0) { if (request.fields() == null || request.fields().length == 0) {
return null; return null;
} }
@ -287,7 +278,7 @@ public class UpdateHelper extends AbstractComponent {
} }
// TODO when using delete/none, we can still return the source as bytes by generating it (using the sourceContentType) // TODO when using delete/none, we can still return the source as bytes by generating it (using the sourceContentType)
return new GetResult(request.index(), request.type(), request.id(), version, true, sourceRequested ? sourceAsBytes : null, fields); return new GetResult(concreteIndex, request.type(), request.id(), version, true, sourceRequested ? sourceAsBytes : null, fields);
} }
public static class Result { public static class Result {

View File

@ -63,10 +63,10 @@ public class ShardTermVectorService extends AbstractIndexShardComponent {
return this; return this;
} }
public TermVectorResponse getTermVector(TermVectorRequest request) { public TermVectorResponse getTermVector(TermVectorRequest request, String concreteIndex) {
final Engine.Searcher searcher = indexShard.acquireSearcher("term_vector"); final Engine.Searcher searcher = indexShard.acquireSearcher("term_vector");
IndexReader topLevelReader = searcher.reader(); IndexReader topLevelReader = searcher.reader();
final TermVectorResponse termVectorResponse = new TermVectorResponse(request.index(), request.type(), request.id()); final TermVectorResponse termVectorResponse = new TermVectorResponse(concreteIndex, request.type(), request.id());
final Term uidTerm = new Term(UidFieldMapper.NAME, Uid.createUidAsBytes(request.type(), request.id())); final Term uidTerm = new Term(UidFieldMapper.NAME, Uid.createUidAsBytes(request.type(), request.id()));
try { try {
Fields topLevelFields = MultiFields.getFields(topLevelReader); Fields topLevelFields = MultiFields.getFields(topLevelReader);

View File

@ -37,6 +37,7 @@ import org.apache.lucene.search.TopDocs;
import org.apache.lucene.store.Directory; import org.apache.lucene.store.Directory;
import org.apache.lucene.store.RAMDirectory; import org.apache.lucene.store.RAMDirectory;
import org.elasticsearch.Version; import org.elasticsearch.Version;
import org.elasticsearch.action.admin.indices.alias.Alias;
import org.elasticsearch.common.inject.internal.Join; import org.elasticsearch.common.inject.internal.Join;
import org.elasticsearch.common.settings.ImmutableSettings; import org.elasticsearch.common.settings.ImmutableSettings;
import org.elasticsearch.common.xcontent.XContentBuilder; import org.elasticsearch.common.xcontent.XContentBuilder;
@ -114,6 +115,7 @@ public abstract class AbstractTermVectorTests extends ElasticsearchIntegrationTe
final public TestFieldSetting[] fieldSettings; final public TestFieldSetting[] fieldSettings;
final public String[] fieldContent; final public String[] fieldContent;
public String index = "test"; public String index = "test";
public String alias = "alias";
public String type = "type1"; public String type = "type1";
public TestDoc(String id, TestFieldSetting[] fieldSettings, String[] fieldContent) { public TestDoc(String id, TestFieldSetting[] fieldSettings, String[] fieldContent) {
@ -128,6 +130,11 @@ public abstract class AbstractTermVectorTests extends ElasticsearchIntegrationTe
return this; return this;
} }
public TestDoc alias(String alias) {
this.alias = alias;
return this;
}
@Override @Override
public String toString() { public String toString() {
@ -181,7 +188,7 @@ public abstract class AbstractTermVectorTests extends ElasticsearchIntegrationTe
} }
} }
protected void createIndexBasedOnFieldSettings(String index, TestFieldSetting[] fieldSettings) throws IOException { protected void createIndexBasedOnFieldSettings(String index, String alias, TestFieldSetting[] fieldSettings) throws IOException {
XContentBuilder mappingBuilder = jsonBuilder(); XContentBuilder mappingBuilder = jsonBuilder();
mappingBuilder.startObject().startObject("type1").startObject("properties"); mappingBuilder.startObject().startObject("type1").startObject("properties");
for (TestFieldSetting field : fieldSettings) { for (TestFieldSetting field : fieldSettings) {
@ -192,7 +199,7 @@ public abstract class AbstractTermVectorTests extends ElasticsearchIntegrationTe
.put(indexSettings()) .put(indexSettings())
.put("index.analysis.analyzer.tv_test.tokenizer", "standard") .put("index.analysis.analyzer.tv_test.tokenizer", "standard")
.putArray("index.analysis.analyzer.tv_test.filter", "type_as_payload", "lowercase"); .putArray("index.analysis.analyzer.tv_test.filter", "type_as_payload", "lowercase");
assertAcked(prepareCreate(index).addMapping("type1", mappingBuilder).setSettings(settings)); assertAcked(prepareCreate(index).addMapping("type1", mappingBuilder).setSettings(settings).addAlias(new Alias(alias)));
ensureYellow(); ensureYellow();
} }
@ -253,12 +260,12 @@ public abstract class AbstractTermVectorTests extends ElasticsearchIntegrationTe
configs.add(config); configs.add(config);
} }
// always adds a test that fails // always adds a test that fails
configs.add(new TestConfig(new TestDoc("doesnt_exist", new TestFieldSetting[]{}, new String[]{}).index("doesn't_exist"), configs.add(new TestConfig(new TestDoc("doesnt_exist", new TestFieldSetting[]{}, new String[]{}).index("doesn't_exist").alias("doesn't_exist"),
new String[]{"doesnt_exist"}, true, true, true).expectedException(IndexMissingException.class)); new String[]{"doesnt_exist"}, true, true, true).expectedException(IndexMissingException.class));
refresh(); refresh();
return configs.toArray(new TestConfig[]{}); return configs.toArray(new TestConfig[configs.size()]);
} }
protected TestFieldSetting[] getFieldSettings() { protected TestFieldSetting[] getFieldSettings() {
@ -267,12 +274,10 @@ public abstract class AbstractTermVectorTests extends ElasticsearchIntegrationTe
new TestFieldSetting("field_with_only_tv", false, false, false), new TestFieldSetting("field_with_only_tv", false, false, false),
new TestFieldSetting("field_with_positions_offsets", false, false, true), new TestFieldSetting("field_with_positions_offsets", false, false, true),
new TestFieldSetting("field_with_positions_payloads", false, true, true) new TestFieldSetting("field_with_positions_payloads", false, true, true)
}; };
} }
protected DirectoryReader indexDocsWithLucene(TestDoc[] testDocs) throws IOException { protected DirectoryReader indexDocsWithLucene(TestDoc[] testDocs) throws IOException {
Map<String, Analyzer> mapping = new HashMap<>(); Map<String, Analyzer> mapping = new HashMap<>();
for (TestFieldSetting field : testDocs[0].fieldSettings) { for (TestFieldSetting field : testDocs[0].fieldSettings) {
if (field.storedPayloads) { if (field.storedPayloads) {
@ -319,6 +324,7 @@ public abstract class AbstractTermVectorTests extends ElasticsearchIntegrationTe
} }
protected void validateResponse(TermVectorResponse esResponse, Fields luceneFields, TestConfig testConfig) throws IOException { protected void validateResponse(TermVectorResponse esResponse, Fields luceneFields, TestConfig testConfig) throws IOException {
assertThat(esResponse.getIndex(), equalTo(testConfig.doc.index));
TestDoc testDoc = testConfig.doc; TestDoc testDoc = testConfig.doc;
HashSet<String> selectedFields = testConfig.selectedFields == null ? null : new HashSet<>( HashSet<String> selectedFields = testConfig.selectedFields == null ? null : new HashSet<>(
Arrays.asList(testConfig.selectedFields)); Arrays.asList(testConfig.selectedFields));
@ -380,21 +386,16 @@ public abstract class AbstractTermVectorTests extends ElasticsearchIntegrationTe
} else { } else {
assertThat("Missing payload test failed" + failDesc, esDocsPosEnum.getPayload(), equalTo(null)); assertThat("Missing payload test failed" + failDesc, esDocsPosEnum.getPayload(), equalTo(null));
} }
} }
} }
assertNull("Es returned terms are done but lucene isn't", luceneTermEnum.next()); assertNull("Es returned terms are done but lucene isn't", luceneTermEnum.next());
} }
} }
protected TermVectorRequestBuilder getRequestForConfig(TestConfig config) { protected TermVectorRequestBuilder getRequestForConfig(TestConfig config) {
return client().prepareTermVector(config.doc.index, config.doc.type, config.doc.id).setPayloads(config.requestPayloads) return client().prepareTermVector(randomBoolean() ? config.doc.index : config.doc.alias, config.doc.type, config.doc.id).setPayloads(config.requestPayloads)
.setOffsets(config.requestOffsets).setPositions(config.requestPositions).setFieldStatistics(true).setTermStatistics(true) .setOffsets(config.requestOffsets).setPositions(config.requestPositions).setFieldStatistics(true).setTermStatistics(true)
.setSelectedFields(config.selectedFields); .setSelectedFields(config.selectedFields);
} }
protected Fields getTermVectorsFromLucene(DirectoryReader directoryReader, TestDoc doc) throws IOException { protected Fields getTermVectorsFromLucene(DirectoryReader directoryReader, TestDoc doc) throws IOException {
@ -405,5 +406,4 @@ public abstract class AbstractTermVectorTests extends ElasticsearchIntegrationTe
assertEquals(1, scoreDocs.length); assertEquals(1, scoreDocs.length);
return directoryReader.getTermVectors(scoreDocs[0].doc); return directoryReader.getTermVectors(scoreDocs[0].doc);
} }
} }

View File

@ -26,6 +26,7 @@ import org.apache.lucene.index.*;
import org.apache.lucene.util.BytesRef; import org.apache.lucene.util.BytesRef;
import org.elasticsearch.ElasticsearchException; import org.elasticsearch.ElasticsearchException;
import org.elasticsearch.action.ActionFuture; import org.elasticsearch.action.ActionFuture;
import org.elasticsearch.action.admin.indices.alias.Alias;
import org.elasticsearch.action.index.IndexRequestBuilder; import org.elasticsearch.action.index.IndexRequestBuilder;
import org.elasticsearch.common.settings.ImmutableSettings; import org.elasticsearch.common.settings.ImmutableSettings;
import org.elasticsearch.common.xcontent.XContentBuilder; import org.elasticsearch.common.xcontent.XContentBuilder;
@ -57,16 +58,17 @@ public class GetTermVectorTests extends AbstractTermVectorTests {
.endObject() .endObject()
.endObject() .endObject()
.endObject().endObject(); .endObject().endObject();
assertAcked(prepareCreate("test").addMapping("type1", mapping)); assertAcked(prepareCreate("test").addAlias(new Alias("alias")).addMapping("type1", mapping));
ensureYellow(); ensureYellow();
client().prepareIndex("test", "type1", "666").setSource("field", "foo bar").execute().actionGet(); client().prepareIndex("test", "type1", "666").setSource("field", "foo bar").execute().actionGet();
refresh(); refresh();
for (int i = 0; i < 20; i++) { for (int i = 0; i < 20; i++) {
ActionFuture<TermVectorResponse> termVector = client().termVector(new TermVectorRequest("test", "type1", "" + i)); ActionFuture<TermVectorResponse> termVector = client().termVector(new TermVectorRequest(indexOrAlias(), "type1", "" + i));
TermVectorResponse actionGet = termVector.actionGet(); TermVectorResponse actionGet = termVector.actionGet();
assertThat(actionGet, notNullValue()); assertThat(actionGet, notNullValue());
assertThat(actionGet.getIndex(), equalTo("test"));
assertThat(actionGet.isExists(), equalTo(false)); assertThat(actionGet.isExists(), equalTo(false));
} }
} }
@ -81,7 +83,7 @@ public class GetTermVectorTests extends AbstractTermVectorTests {
.endObject() .endObject()
.endObject() .endObject()
.endObject().endObject(); .endObject().endObject();
assertAcked(prepareCreate("test").addMapping("type1", mapping)); assertAcked(prepareCreate("test").addAlias(new Alias("alias")).addMapping("type1", mapping));
ensureYellow(); ensureYellow();
@ -89,13 +91,14 @@ public class GetTermVectorTests extends AbstractTermVectorTests {
// vectors will be null // vectors will be null
client().prepareIndex("test", "type1", "0").setSource("existingfield", "?").execute().actionGet(); client().prepareIndex("test", "type1", "0").setSource("existingfield", "?").execute().actionGet();
refresh(); refresh();
ActionFuture<TermVectorResponse> termVector = client().termVector(new TermVectorRequest("test", "type1", "0") ActionFuture<TermVectorResponse> termVector = client().termVector(new TermVectorRequest(indexOrAlias(), "type1", "0")
.selectedFields(new String[]{"existingfield"})); .selectedFields(new String[]{"existingfield"}));
// lets see if the null term vectors are caught... // lets see if the null term vectors are caught...
TermVectorResponse actionGet = termVector.actionGet(); TermVectorResponse actionGet = termVector.actionGet();
assertThat(actionGet, notNullValue()); assertThat(actionGet, notNullValue());
assertThat(actionGet.isExists(), equalTo(true)); assertThat(actionGet.isExists(), equalTo(true));
assertThat(actionGet.getIndex(), equalTo("test"));
assertThat(actionGet.getFields().terms("existingfield"), nullValue()); assertThat(actionGet.getFields().terms("existingfield"), nullValue());
} }
@ -109,7 +112,7 @@ public class GetTermVectorTests extends AbstractTermVectorTests {
.endObject() .endObject()
.endObject() .endObject()
.endObject().endObject(); .endObject().endObject();
assertAcked(prepareCreate("test").addMapping("type1", mapping)); assertAcked(prepareCreate("test").addAlias(new Alias("alias")).addMapping("type1", mapping));
ensureYellow(); ensureYellow();
@ -117,26 +120,29 @@ public class GetTermVectorTests extends AbstractTermVectorTests {
// vectors will be null // vectors will be null
client().prepareIndex("test", "type1", "0").setSource("anotherexistingfield", 1).execute().actionGet(); client().prepareIndex("test", "type1", "0").setSource("anotherexistingfield", 1).execute().actionGet();
refresh(); refresh();
ActionFuture<TermVectorResponse> termVector = client().termVector(new TermVectorRequest("test", "type1", "0") ActionFuture<TermVectorResponse> termVector = client().termVector(new TermVectorRequest(indexOrAlias(), "type1", "0")
.selectedFields(new String[]{"existingfield"})); .selectedFields(new String[]{"existingfield"}));
// lets see if the null term vectors are caught... // lets see if the null term vectors are caught...
TermVectorResponse actionGet = termVector.actionGet(); TermVectorResponse actionGet = termVector.actionGet();
assertThat(actionGet, notNullValue()); assertThat(actionGet, notNullValue());
assertThat(actionGet.isExists(), equalTo(true)); assertThat(actionGet.isExists(), equalTo(true));
assertThat(actionGet.getIndex(), equalTo("test"));
assertThat(actionGet.getFields().terms("existingfield"), nullValue()); assertThat(actionGet.getFields().terms("existingfield"), nullValue());
} }
@Test @Test
public void testNotIndexedField() throws Exception { public void testNotIndexedField() throws Exception {
// must be of type string and indexed. // must be of type string and indexed.
assertAcked(prepareCreate("test").addMapping("type1", assertAcked(prepareCreate("test")
"field0", "type=integer,", // no tvs .addAlias(new Alias("alias"))
"field1", "type=string,index=no", // no tvs .addMapping("type1",
"field2", "type=string,index=no,store=yes", // no tvs "field0", "type=integer,", // no tvs
"field3", "type=string,index=no,term_vector=yes", // no tvs "field1", "type=string,index=no", // no tvs
"field4", "type=string,index=not_analyzed", // yes tvs "field2", "type=string,index=no,store=yes", // no tvs
"field5", "type=string,index=analyzed")); // yes tvs "field3", "type=string,index=no,term_vector=yes", // no tvs
"field4", "type=string,index=not_analyzed", // yes tvs
"field5", "type=string,index=analyzed")); // yes tvs
ensureYellow(); ensureYellow();
@ -151,18 +157,19 @@ public class GetTermVectorTests extends AbstractTermVectorTests {
indexRandom(true, indexBuilders); indexRandom(true, indexBuilders);
for (int i = 0; i < 4; i++) { for (int i = 0; i < 4; i++) {
TermVectorResponse resp = client().prepareTermVector("test", "type1", String.valueOf(i)) TermVectorResponse resp = client().prepareTermVector(indexOrAlias(), "type1", String.valueOf(i))
.setSelectedFields("field" + i) .setSelectedFields("field" + i)
.get(); .get();
assertThat(resp, notNullValue()); assertThat(resp, notNullValue());
assertThat(resp.isExists(), equalTo(true)); assertThat(resp.isExists(), equalTo(true));
assertThat(resp.getIndex(), equalTo("test"));
assertThat("field" + i + " :", resp.getFields().terms("field" + i), nullValue()); assertThat("field" + i + " :", resp.getFields().terms("field" + i), nullValue());
} }
for (int i = 4; i < 6; i++) { for (int i = 4; i < 6; i++) {
TermVectorResponse resp = client().prepareTermVector("test", "type1", String.valueOf(i)) TermVectorResponse resp = client().prepareTermVector(indexOrAlias(), "type1", String.valueOf(i))
.setSelectedFields("field" + i) .setSelectedFields("field" + i).get();
.get(); assertThat(resp.getIndex(), equalTo("test"));
assertThat("field" + i + " :", resp.getFields().terms("field" + i), notNullValue()); assertThat("field" + i + " :", resp.getFields().terms("field" + i), notNullValue());
} }
} }
@ -179,6 +186,7 @@ public class GetTermVectorTests extends AbstractTermVectorTests {
.endObject() .endObject()
.endObject().endObject(); .endObject().endObject();
assertAcked(prepareCreate("test").addMapping("type1", mapping) assertAcked(prepareCreate("test").addMapping("type1", mapping)
.addAlias(new Alias("alias"))
.setSettings(settingsBuilder() .setSettings(settingsBuilder()
.put(indexSettings()) .put(indexSettings())
.put("index.analysis.analyzer.tv_test.tokenizer", "whitespace") .put("index.analysis.analyzer.tv_test.tokenizer", "whitespace")
@ -193,9 +201,10 @@ public class GetTermVectorTests extends AbstractTermVectorTests {
refresh(); refresh();
} }
for (int i = 0; i < 10; i++) { for (int i = 0; i < 10; i++) {
TermVectorRequestBuilder resp = client().prepareTermVector("test", "type1", Integer.toString(i)).setPayloads(true) TermVectorRequestBuilder resp = client().prepareTermVector(indexOrAlias(), "type1", Integer.toString(i)).setPayloads(true)
.setOffsets(true).setPositions(true).setSelectedFields(); .setOffsets(true).setPositions(true).setSelectedFields();
TermVectorResponse response = resp.execute().actionGet(); TermVectorResponse response = resp.execute().actionGet();
assertThat(response.getIndex(), equalTo("test"));
assertThat("doc id: " + i + " doesn't exists but should", response.isExists(), equalTo(true)); assertThat("doc id: " + i + " doesn't exists but should", response.isExists(), equalTo(true));
Fields fields = response.getFields(); Fields fields = response.getFields();
assertThat(fields.size(), equalTo(1)); assertThat(fields.size(), equalTo(1));
@ -367,7 +376,7 @@ public class GetTermVectorTests extends AbstractTermVectorTests {
@Test @Test
public void testDuelESLucene() throws Exception { public void testDuelESLucene() throws Exception {
TestFieldSetting[] testFieldSettings = getFieldSettings(); TestFieldSetting[] testFieldSettings = getFieldSettings();
createIndexBasedOnFieldSettings("test", testFieldSettings); createIndexBasedOnFieldSettings("test", "alias", testFieldSettings);
//we generate as many docs as many shards we have //we generate as many docs as many shards we have
TestDoc[] testDocs = generateTestDocs(getNumShards("test").numPrimaries, testFieldSettings); TestDoc[] testDocs = generateTestDocs(getNumShards("test").numPrimaries, testFieldSettings);
@ -764,14 +773,19 @@ public class GetTermVectorTests extends AbstractTermVectorTests {
source.endObject(); source.endObject();
mapping.endObject().endObject().endObject(); mapping.endObject().endObject().endObject();
assertAcked(prepareCreate("test").addMapping("type1", mapping)); assertAcked(prepareCreate("test").addAlias(new Alias("alias")).addMapping("type1", mapping));
ensureGreen(); ensureGreen();
client().prepareIndex("test", "type1", "0").setSource(source).get(); client().prepareIndex("test", "type1", "0").setSource(source).get();
refresh(); refresh();
TermVectorResponse response = client().prepareTermVector("test", "type1", "0").setSelectedFields("field*").get(); TermVectorResponse response = client().prepareTermVector(indexOrAlias(), "type1", "0").setSelectedFields("field*").get();
assertThat("Doc doesn't exists but should", response.isExists(), equalTo(true)); assertThat("Doc doesn't exists but should", response.isExists(), equalTo(true));
assertThat(response.getIndex(), equalTo("test"));
assertThat("All term vectors should have been generated", response.getFields().size(), equalTo(numFields)); assertThat("All term vectors should have been generated", response.getFields().size(), equalTo(numFields));
} }
private static String indexOrAlias() {
return randomBoolean() ? "test" : "alias";
}
} }

View File

@ -30,7 +30,7 @@ public class MultiTermVectorsTests extends AbstractTermVectorTests {
@Test @Test
public void testDuelESLucene() throws Exception { public void testDuelESLucene() throws Exception {
AbstractTermVectorTests.TestFieldSetting[] testFieldSettings = getFieldSettings(); AbstractTermVectorTests.TestFieldSetting[] testFieldSettings = getFieldSettings();
createIndexBasedOnFieldSettings("test", testFieldSettings); createIndexBasedOnFieldSettings("test", "alias", testFieldSettings);
//we generate as many docs as many shards we have //we generate as many docs as many shards we have
TestDoc[] testDocs = generateTestDocs(getNumShards("test").numPrimaries, testFieldSettings); TestDoc[] testDocs = generateTestDocs(getNumShards("test").numPrimaries, testFieldSettings);

View File

@ -20,21 +20,29 @@ package org.elasticsearch.bwcompat;
import com.carrotsearch.randomizedtesting.LifecycleScope; import com.carrotsearch.randomizedtesting.LifecycleScope;
import com.carrotsearch.randomizedtesting.generators.RandomPicks; import com.carrotsearch.randomizedtesting.generators.RandomPicks;
import org.apache.lucene.index.Fields;
import org.apache.lucene.util.English; import org.apache.lucene.util.English;
import org.elasticsearch.ElasticsearchIllegalArgumentException; import org.elasticsearch.ElasticsearchIllegalArgumentException;
import org.elasticsearch.Version; import org.elasticsearch.Version;
import org.elasticsearch.action.admin.cluster.health.ClusterHealthStatus; import org.elasticsearch.action.admin.cluster.health.ClusterHealthStatus;
import org.elasticsearch.action.admin.cluster.snapshots.create.CreateSnapshotResponse; import org.elasticsearch.action.admin.cluster.snapshots.create.CreateSnapshotResponse;
import org.elasticsearch.action.admin.cluster.snapshots.restore.RestoreSnapshotResponse; import org.elasticsearch.action.admin.cluster.snapshots.restore.RestoreSnapshotResponse;
import org.elasticsearch.action.admin.indices.alias.Alias;
import org.elasticsearch.action.admin.indices.analyze.AnalyzeResponse;
import org.elasticsearch.action.admin.indices.settings.get.GetSettingsResponse; import org.elasticsearch.action.admin.indices.settings.get.GetSettingsResponse;
import org.elasticsearch.action.count.CountResponse; import org.elasticsearch.action.count.CountResponse;
import org.elasticsearch.action.delete.DeleteResponse; import org.elasticsearch.action.delete.DeleteResponse;
import org.elasticsearch.action.deletebyquery.DeleteByQueryResponse; import org.elasticsearch.action.deletebyquery.DeleteByQueryResponse;
import org.elasticsearch.action.deletebyquery.IndexDeleteByQueryResponse; import org.elasticsearch.action.deletebyquery.IndexDeleteByQueryResponse;
import org.elasticsearch.action.explain.ExplainResponse;
import org.elasticsearch.action.get.GetResponse; import org.elasticsearch.action.get.GetResponse;
import org.elasticsearch.action.index.IndexRequest; import org.elasticsearch.action.index.IndexRequest;
import org.elasticsearch.action.index.IndexRequestBuilder; import org.elasticsearch.action.index.IndexRequestBuilder;
import org.elasticsearch.action.index.IndexResponse;
import org.elasticsearch.action.search.SearchResponse; import org.elasticsearch.action.search.SearchResponse;
import org.elasticsearch.action.termvector.TermVectorResponse;
import org.elasticsearch.action.update.UpdateRequestBuilder;
import org.elasticsearch.action.update.UpdateResponse;
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.routing.IndexRoutingTable; import org.elasticsearch.cluster.routing.IndexRoutingTable;
@ -128,12 +136,12 @@ public class BasicBackwardsCompatibilityTest extends ElasticsearchBackwardsCompa
*/ */
@Test @Test
public void testIndexAndSearch() throws Exception { public void testIndexAndSearch() throws Exception {
createIndex("test"); assertAcked(prepareCreate("test").addAlias(new Alias("alias")));
int numDocs = randomIntBetween(10, 20); int numDocs = randomIntBetween(10, 20);
List<IndexRequestBuilder> builder = new ArrayList<>(); List<IndexRequestBuilder> builder = new ArrayList<>();
for (int i = 0; i < numDocs; i++) { for (int i = 0; i < numDocs; i++) {
String id = Integer.toString(i); String id = Integer.toString(i);
builder.add(client().prepareIndex("test", "type1", id).setSource("field1", English.intToEnglish(i), "the_id", id)); builder.add(client().prepareIndex(indexOrAlias(), "type1", id).setSource("field1", English.intToEnglish(i), "the_id", id));
} }
indexRandom(true, builder); indexRandom(true, builder);
for (int i = 0; i < numDocs; i++) { for (int i = 0; i < numDocs; i++) {
@ -290,8 +298,8 @@ public class BasicBackwardsCompatibilityTest extends ElasticsearchBackwardsCompa
docs[i] = client().prepareIndex(indexForDoc[i] = RandomPicks.randomFrom(getRandom(), indices), "type1", String.valueOf(i)).setSource("field1", English.intToEnglish(i), "num_int", randomInt(), "num_double", randomDouble()); docs[i] = client().prepareIndex(indexForDoc[i] = RandomPicks.randomFrom(getRandom(), indices), "type1", String.valueOf(i)).setSource("field1", English.intToEnglish(i), "num_int", randomInt(), "num_double", randomDouble());
} }
indexRandom(true, docs); indexRandom(true, docs);
for (int i = 0; i < indices.length; i++) { for (String index : indices) {
assertAllShardsOnNodes(indices[i], backwardsCluster().backwardsNodePattern()); assertAllShardsOnNodes(index, backwardsCluster().backwardsNodePattern());
} }
client().admin().indices().prepareUpdateSettings(indices).setSettings(ImmutableSettings.builder().put(EnableAllocationDecider.INDEX_ROUTING_ALLOCATION_ENABLE, "none")).get(); client().admin().indices().prepareUpdateSettings(indices).setSettings(ImmutableSettings.builder().put(EnableAllocationDecider.INDEX_ROUTING_ALLOCATION_ENABLE, "none")).get();
backwardsCluster().allowOnAllNodes(indices); backwardsCluster().allowOnAllNodes(indices);
@ -326,11 +334,11 @@ public class BasicBackwardsCompatibilityTest extends ElasticsearchBackwardsCompa
assertVersionCreated(compatibilityVersion(), indices); assertVersionCreated(compatibilityVersion(), indices);
} }
public void assertVersionCreated(Version version, String... index) { public void assertVersionCreated(Version version, String... indices) {
GetSettingsResponse getSettingsResponse = client().admin().indices().prepareGetSettings(index).get(); GetSettingsResponse getSettingsResponse = client().admin().indices().prepareGetSettings(indices).get();
ImmutableOpenMap<String,Settings> indexToSettings = getSettingsResponse.getIndexToSettings(); ImmutableOpenMap<String,Settings> indexToSettings = getSettingsResponse.getIndexToSettings();
for (int i = 0; i < index.length; i++) { for (String index : indices) {
Settings settings = indexToSettings.get(index[i]); Settings settings = indexToSettings.get(index);
assertThat(settings.getAsVersion(IndexMetaData.SETTING_VERSION_CREATED, null), notNullValue()); assertThat(settings.getAsVersion(IndexMetaData.SETTING_VERSION_CREATED, null), notNullValue());
assertThat(settings.getAsVersion(IndexMetaData.SETTING_VERSION_CREATED, null), equalTo(version)); assertThat(settings.getAsVersion(IndexMetaData.SETTING_VERSION_CREATED, null), equalTo(version));
} }
@ -567,7 +575,7 @@ public class BasicBackwardsCompatibilityTest extends ElasticsearchBackwardsCompa
int numDocs = iterations(10, 50); int numDocs = iterations(10, 50);
IndexRequestBuilder[] indexRequestBuilders = new IndexRequestBuilder[numDocs]; IndexRequestBuilder[] indexRequestBuilders = new IndexRequestBuilder[numDocs];
for (int i = 0; i < numDocs - 2; i++) { for (int i = 0; i < numDocs - 2; i++) {
indexRequestBuilders[i] = client().prepareIndex("test", "test", Integer.toString(i)) indexRequestBuilders[i] = client().prepareIndex("test", "test", Integer.toString(i))
.setRouting(randomAsciiOfLength(randomIntBetween(1, 10))).setSource("field", "value"); .setRouting(randomAsciiOfLength(randomIntBetween(1, 10))).setSource("field", "value");
} }
String firstDocId = Integer.toString(numDocs - 2); String firstDocId = Integer.toString(numDocs - 2);
@ -581,7 +589,7 @@ public class BasicBackwardsCompatibilityTest extends ElasticsearchBackwardsCompa
indexRandom(true, indexRequestBuilders); indexRandom(true, indexRequestBuilders);
SearchResponse searchResponse = client().prepareSearch("test").get(); SearchResponse searchResponse = client().prepareSearch("test").get();
assertThat(searchResponse.getHits().totalHits(), equalTo((long)numDocs)); assertThat(searchResponse.getHits().totalHits(), equalTo((long) numDocs));
//use routing //use routing
DeleteResponse deleteResponse = client().prepareDelete("test", "test", firstDocId).setRouting("routing").get(); DeleteResponse deleteResponse = client().prepareDelete("test", "test", firstDocId).setRouting("routing").get();
@ -590,7 +598,7 @@ public class BasicBackwardsCompatibilityTest extends ElasticsearchBackwardsCompa
assertThat(getResponse.isExists(), equalTo(false)); assertThat(getResponse.isExists(), equalTo(false));
refresh(); refresh();
searchResponse = client().prepareSearch("test").get(); searchResponse = client().prepareSearch("test").get();
assertThat(searchResponse.getHits().totalHits(), equalTo((long)numDocs - 1)); assertThat(searchResponse.getHits().totalHits(), equalTo((long) numDocs - 1));
//don't use routing and trigger a broadcast delete //don't use routing and trigger a broadcast delete
deleteResponse = client().prepareDelete("test", "test", secondDocId).get(); deleteResponse = client().prepareDelete("test", "test", secondDocId).get();
@ -599,6 +607,123 @@ public class BasicBackwardsCompatibilityTest extends ElasticsearchBackwardsCompa
assertThat(getResponse.isExists(), equalTo(false)); assertThat(getResponse.isExists(), equalTo(false));
refresh(); refresh();
searchResponse = client().prepareSearch("test").get(); searchResponse = client().prepareSearch("test").get();
assertThat(searchResponse.getHits().totalHits(), equalTo((long)numDocs - 2)); assertThat(searchResponse.getHits().totalHits(), equalTo((long) numDocs - 2));
}
@Test
public void testIndexGetAndDelete() throws ExecutionException, InterruptedException {
assertAcked(prepareCreate("test").addAlias(new Alias("alias")));
ensureYellow("test");
int numDocs = iterations(10, 50);
for (int i = 0; i < numDocs; i++) {
IndexResponse indexResponse = client().prepareIndex(indexOrAlias(), "type", Integer.toString(i)).setSource("field", "value-" + i).get();
assertThat(indexResponse.isCreated(), equalTo(true));
assertThat(indexResponse.getIndex(), equalTo("test"));
assertThat(indexResponse.getType(), equalTo("type"));
assertThat(indexResponse.getId(), equalTo(Integer.toString(i)));
}
refresh();
String docId = Integer.toString(randomIntBetween(0, numDocs - 1));
GetResponse getResponse = client().prepareGet(indexOrAlias(), "type", docId).get();
assertThat(getResponse.isExists(), equalTo(true));
assertThat(getResponse.getIndex(), equalTo("test"));
assertThat(getResponse.getType(), equalTo("type"));
assertThat(getResponse.getId(), equalTo(docId));
DeleteResponse deleteResponse = client().prepareDelete(indexOrAlias(), "type", docId).get();
assertThat(deleteResponse.isFound(), equalTo(true));
assertThat(deleteResponse.getIndex(), equalTo("test"));
assertThat(deleteResponse.getType(), equalTo("type"));
assertThat(deleteResponse.getId(), equalTo(docId));
getResponse = client().prepareGet(indexOrAlias(), "type", docId).get();
assertThat(getResponse.isExists(), equalTo(false));
refresh();
SearchResponse searchResponse = client().prepareSearch(indexOrAlias()).get();
assertThat(searchResponse.getHits().totalHits(), equalTo((long)numDocs - 1));
}
@Test
public void testUpdate() {
assertAcked(prepareCreate("test").addAlias(new Alias("alias")));
ensureYellow("test");
UpdateRequestBuilder updateRequestBuilder = client().prepareUpdate(indexOrAlias(), "type1", "1")
.setUpsert("field1", "value1").setDoc("field2", "value2");
UpdateResponse updateResponse = updateRequestBuilder.get();
assertThat(updateResponse.getIndex(), equalTo("test"));
assertThat(updateResponse.getType(), equalTo("type1"));
assertThat(updateResponse.getId(), equalTo("1"));
assertThat(updateResponse.isCreated(), equalTo(true));
GetResponse getResponse = client().prepareGet("test", "type1", "1").get();
assertThat(getResponse.isExists(), equalTo(true));
assertThat(getResponse.getSourceAsMap().containsKey("field1"), equalTo(true));
assertThat(getResponse.getSourceAsMap().containsKey("field2"), equalTo(false));
updateResponse = updateRequestBuilder.get();
assertThat(updateResponse.getIndex(), equalTo("test"));
assertThat(updateResponse.getType(), equalTo("type1"));
assertThat(updateResponse.getId(), equalTo("1"));
assertThat(updateResponse.isCreated(), equalTo(false));
getResponse = client().prepareGet("test", "type1", "1").get();
assertThat(getResponse.isExists(), equalTo(true));
assertThat(getResponse.getSourceAsMap().containsKey("field1"), equalTo(true));
assertThat(getResponse.getSourceAsMap().containsKey("field2"), equalTo(true));
}
@Test
public void testAnalyze() {
assertAcked(prepareCreate("test").addAlias(new Alias("alias"))
.addMapping("test", "field", "type=string,analyzer=keyword"));
ensureYellow("test");
AnalyzeResponse analyzeResponse = client().admin().indices().prepareAnalyze("this is a test").setIndex(indexOrAlias()).setField("field").get();
assertThat(analyzeResponse.getTokens().size(), equalTo(1));
assertThat(analyzeResponse.getTokens().get(0).getTerm(), equalTo("this is a test"));
}
@Test
public void testExplain() {
assertAcked(prepareCreate("test").addAlias(new Alias("alias")));
ensureYellow("test");
client().prepareIndex(indexOrAlias(), "test", "1").setSource("field", "value1").get();
refresh();
ExplainResponse response = client().prepareExplain(indexOrAlias(), "test", "1")
.setQuery(QueryBuilders.termQuery("field", "value1")).get();
assertThat(response.isExists(), equalTo(true));
assertThat(response.isMatch(), equalTo(true));
assertThat(response.getExplanation(), notNullValue());
assertThat(response.getExplanation().isMatch(), equalTo(true));
assertThat(response.getExplanation().getDetails().length, equalTo(1));
}
@Test
public void testGetTermVector() throws IOException {
assertAcked(prepareCreate("test").addAlias(new Alias("alias"))
.addMapping("type1", "field", "type=string,term_vector=with_positions_offsets_payloads"));
ensureYellow("test");
client().prepareIndex(indexOrAlias(), "type1", "1")
.setSource("field", "the quick brown fox jumps over the lazy dog").get();
refresh();
TermVectorResponse termVectorResponse = client().prepareTermVector(indexOrAlias(), "type1", "1").get();
assertThat(termVectorResponse.getIndex(), equalTo("test"));
assertThat(termVectorResponse.isExists(), equalTo(true));
Fields fields = termVectorResponse.getFields();
assertThat(fields.size(), equalTo(1));
assertThat(fields.terms("field").size(), equalTo(8l));
}
private static String indexOrAlias() {
return randomBoolean() ? "test" : "alias";
} }
} }

View File

@ -20,6 +20,7 @@
package org.elasticsearch.deleteByQuery; package org.elasticsearch.deleteByQuery;
import org.elasticsearch.action.ShardOperationFailedException; import org.elasticsearch.action.ShardOperationFailedException;
import org.elasticsearch.action.admin.indices.alias.Alias;
import org.elasticsearch.action.deletebyquery.DeleteByQueryRequestBuilder; import org.elasticsearch.action.deletebyquery.DeleteByQueryRequestBuilder;
import org.elasticsearch.action.deletebyquery.DeleteByQueryResponse; import org.elasticsearch.action.deletebyquery.DeleteByQueryResponse;
import org.elasticsearch.action.search.SearchResponse; import org.elasticsearch.action.search.SearchResponse;
@ -30,6 +31,7 @@ import org.elasticsearch.rest.RestStatus;
import org.elasticsearch.test.ElasticsearchIntegrationTest; import org.elasticsearch.test.ElasticsearchIntegrationTest;
import org.junit.Test; import org.junit.Test;
import static org.elasticsearch.test.hamcrest.ElasticsearchAssertions.assertAcked;
import static org.elasticsearch.test.hamcrest.ElasticsearchAssertions.assertHitCount; import static org.elasticsearch.test.hamcrest.ElasticsearchAssertions.assertHitCount;
import static org.hamcrest.Matchers.*; import static org.hamcrest.Matchers.*;
@ -101,22 +103,22 @@ public class DeleteByQueryTests extends ElasticsearchIntegrationTest {
@Test @Test
public void testFailure() throws Exception { public void testFailure() throws Exception {
client().admin().indices().prepareCreate("twitter").execute().actionGet(); assertAcked(prepareCreate("test").addAlias(new Alias("alias")));
DeleteByQueryResponse response = client().prepareDeleteByQuery("twitter") DeleteByQueryResponse response = client().prepareDeleteByQuery(indexOrAlias())
.setQuery(QueryBuilders.hasChildQuery("type", QueryBuilders.matchAllQuery())) .setQuery(QueryBuilders.hasChildQuery("type", QueryBuilders.matchAllQuery()))
.execute().actionGet(); .execute().actionGet();
NumShards twitter = getNumShards("twitter"); NumShards twitter = getNumShards("test");
assertThat(response.status(), equalTo(RestStatus.BAD_REQUEST)); assertThat(response.status(), equalTo(RestStatus.BAD_REQUEST));
assertThat(response.getIndex("twitter").getSuccessfulShards(), equalTo(0)); assertThat(response.getIndex("test").getSuccessfulShards(), equalTo(0));
assertThat(response.getIndex("twitter").getFailedShards(), equalTo(twitter.numPrimaries)); assertThat(response.getIndex("test").getFailedShards(), equalTo(twitter.numPrimaries));
assertThat(response.getIndices().size(), equalTo(1)); assertThat(response.getIndices().size(), equalTo(1));
assertThat(response.getIndices().get("twitter").getFailedShards(), equalTo(twitter.numPrimaries)); assertThat(response.getIndices().get("test").getFailedShards(), equalTo(twitter.numPrimaries));
assertThat(response.getIndices().get("twitter").getFailures().length, equalTo(twitter.numPrimaries)); assertThat(response.getIndices().get("test").getFailures().length, equalTo(twitter.numPrimaries));
for (ShardOperationFailedException failure : response.getIndices().get("twitter").getFailures()) { for (ShardOperationFailedException failure : response.getIndices().get("test").getFailures()) {
assertThat(failure.reason(), containsString("[twitter] [has_child] unsupported in delete_by_query api")); assertThat(failure.reason(), containsString("[test] [has_child] unsupported in delete_by_query api"));
assertThat(failure.status(), equalTo(RestStatus.BAD_REQUEST)); assertThat(failure.status(), equalTo(RestStatus.BAD_REQUEST));
assertThat(failure.shardId(), greaterThan(-1)); assertThat(failure.shardId(), greaterThan(-1));
} }
@ -124,7 +126,7 @@ public class DeleteByQueryTests extends ElasticsearchIntegrationTest {
@Test @Test
public void testDeleteByFieldQuery() throws Exception { public void testDeleteByFieldQuery() throws Exception {
client().admin().indices().prepareCreate("test").execute().actionGet(); assertAcked(prepareCreate("test").addAlias(new Alias("alias")));
int numDocs = scaledRandomIntBetween(10, 100); int numDocs = scaledRandomIntBetween(10, 100);
for (int i = 0; i < numDocs; i++) { for (int i = 0; i < numDocs; i++) {
client().prepareIndex("test", "test", Integer.toString(i)) client().prepareIndex("test", "test", Integer.toString(i))
@ -134,12 +136,13 @@ public class DeleteByQueryTests extends ElasticsearchIntegrationTest {
refresh(); refresh();
assertHitCount(client().prepareCount("test").setQuery(QueryBuilders.matchQuery("_id", Integer.toString(between(0, numDocs - 1)))).get(), 1); assertHitCount(client().prepareCount("test").setQuery(QueryBuilders.matchQuery("_id", Integer.toString(between(0, numDocs - 1)))).get(), 1);
assertHitCount(client().prepareCount("test").setQuery(QueryBuilders.matchAllQuery()).get(), numDocs); assertHitCount(client().prepareCount("test").setQuery(QueryBuilders.matchAllQuery()).get(), numDocs);
client().prepareDeleteByQuery("test") DeleteByQueryResponse deleteByQueryResponse = client().prepareDeleteByQuery(indexOrAlias())
.setQuery(QueryBuilders.matchQuery("_id", Integer.toString(between(0, numDocs - 1)))) .setQuery(QueryBuilders.matchQuery("_id", Integer.toString(between(0, numDocs - 1)))).get();
.execute().actionGet(); assertThat(deleteByQueryResponse.getIndices().size(), equalTo(1));
assertThat(deleteByQueryResponse.getIndex("test"), notNullValue());
refresh(); refresh();
assertHitCount(client().prepareCount("test").setQuery(QueryBuilders.matchAllQuery()).get(), numDocs - 1); assertHitCount(client().prepareCount("test").setQuery(QueryBuilders.matchAllQuery()).get(), numDocs - 1);
} }
@Test @Test
@ -153,5 +156,7 @@ public class DeleteByQueryTests extends ElasticsearchIntegrationTest {
assertHitCount(client().prepareCount("test").get(), 0); assertHitCount(client().prepareCount("test").get(), 0);
} }
private static String indexOrAlias() {
return randomBoolean() ? "test" : "alias";
}
} }

View File

@ -20,6 +20,8 @@
package org.elasticsearch.document; package org.elasticsearch.document;
import com.google.common.base.Charsets; import com.google.common.base.Charsets;
import org.elasticsearch.action.admin.indices.alias.Alias;
import org.elasticsearch.action.bulk.BulkItemResponse;
import org.elasticsearch.action.bulk.BulkRequestBuilder; import org.elasticsearch.action.bulk.BulkRequestBuilder;
import org.elasticsearch.action.bulk.BulkResponse; import org.elasticsearch.action.bulk.BulkResponse;
import org.elasticsearch.action.count.CountResponse; import org.elasticsearch.action.count.CountResponse;
@ -51,30 +53,36 @@ public class BulkTests extends ElasticsearchIntegrationTest {
@Test @Test
public void testBulkUpdate_simple() throws Exception { public void testBulkUpdate_simple() throws Exception {
createIndex("test"); assertAcked(prepareCreate("test").addAlias(new Alias("alias")));
ensureGreen(); ensureGreen();
BulkResponse bulkResponse = client().prepareBulk() BulkResponse bulkResponse = client().prepareBulk()
.add(client().prepareIndex().setIndex("test").setType("type1").setId("1").setSource("field", 1)) .add(client().prepareIndex().setIndex(indexOrAlias()).setType("type1").setId("1").setSource("field", 1))
.add(client().prepareIndex().setIndex("test").setType("type1").setId("2").setSource("field", 2).setCreate(true)) .add(client().prepareIndex().setIndex(indexOrAlias()).setType("type1").setId("2").setSource("field", 2).setCreate(true))
.add(client().prepareIndex().setIndex("test").setType("type1").setId("3").setSource("field", 3)) .add(client().prepareIndex().setIndex(indexOrAlias()).setType("type1").setId("3").setSource("field", 3))
.add(client().prepareIndex().setIndex("test").setType("type1").setId("4").setSource("field", 4)) .add(client().prepareIndex().setIndex(indexOrAlias()).setType("type1").setId("4").setSource("field", 4))
.add(client().prepareIndex().setIndex("test").setType("type1").setId("5").setSource("field", 5)) .add(client().prepareIndex().setIndex(indexOrAlias()).setType("type1").setId("5").setSource("field", 5))
.execute().actionGet(); .execute().actionGet();
assertThat(bulkResponse.hasFailures(), equalTo(false)); assertThat(bulkResponse.hasFailures(), equalTo(false));
assertThat(bulkResponse.getItems().length, equalTo(5)); assertThat(bulkResponse.getItems().length, equalTo(5));
for (BulkItemResponse bulkItemResponse : bulkResponse) {
assertThat(bulkItemResponse.getIndex(), equalTo("test"));
}
bulkResponse = client().prepareBulk() bulkResponse = client().prepareBulk()
.add(client().prepareUpdate().setIndex("test").setType("type1").setId("1") .add(client().prepareUpdate().setIndex(indexOrAlias()).setType("type1").setId("1")
.setScript("ctx._source.field += 1", ScriptService.ScriptType.INLINE)) .setScript("ctx._source.field += 1", ScriptService.ScriptType.INLINE))
.add(client().prepareUpdate().setIndex("test").setType("type1").setId("2") .add(client().prepareUpdate().setIndex(indexOrAlias()).setType("type1").setId("2")
.setScript("ctx._source.field += 1", ScriptService.ScriptType.INLINE).setRetryOnConflict(3)) .setScript("ctx._source.field += 1", ScriptService.ScriptType.INLINE).setRetryOnConflict(3))
.add(client().prepareUpdate().setIndex("test").setType("type1").setId("3").setDoc(jsonBuilder().startObject().field("field1", "test").endObject())) .add(client().prepareUpdate().setIndex(indexOrAlias()).setType("type1").setId("3").setDoc(jsonBuilder().startObject().field("field1", "test").endObject()))
.execute().actionGet(); .execute().actionGet();
assertThat(bulkResponse.hasFailures(), equalTo(false)); assertThat(bulkResponse.hasFailures(), equalTo(false));
assertThat(bulkResponse.getItems().length, equalTo(3)); assertThat(bulkResponse.getItems().length, equalTo(3));
for (BulkItemResponse bulkItemResponse : bulkResponse) {
assertThat(bulkItemResponse.getIndex(), equalTo("test"));
}
assertThat(((UpdateResponse) bulkResponse.getItems()[0].getResponse()).getId(), equalTo("1")); assertThat(((UpdateResponse) bulkResponse.getItems()[0].getResponse()).getId(), equalTo("1"));
assertThat(((UpdateResponse) bulkResponse.getItems()[0].getResponse()).getVersion(), equalTo(2l)); assertThat(((UpdateResponse) bulkResponse.getItems()[0].getResponse()).getVersion(), equalTo(2l));
assertThat(((UpdateResponse) bulkResponse.getItems()[1].getResponse()).getId(), equalTo("2")); assertThat(((UpdateResponse) bulkResponse.getItems()[1].getResponse()).getId(), equalTo("2"));
@ -98,12 +106,12 @@ public class BulkTests extends ElasticsearchIntegrationTest {
assertThat(getResponse.getField("field1").getValue().toString(), equalTo("test")); assertThat(getResponse.getField("field1").getValue().toString(), equalTo("test"));
bulkResponse = client().prepareBulk() bulkResponse = client().prepareBulk()
.add(client().prepareUpdate().setIndex("test").setType("type1").setId("6") .add(client().prepareUpdate().setIndex(indexOrAlias()).setType("type1").setId("6")
.setScript("ctx._source.field += 1", ScriptService.ScriptType.INLINE) .setScript("ctx._source.field += 1", ScriptService.ScriptType.INLINE)
.setUpsert(jsonBuilder().startObject().field("field", 0).endObject())) .setUpsert(jsonBuilder().startObject().field("field", 0).endObject()))
.add(client().prepareUpdate().setIndex("test").setType("type1").setId("7") .add(client().prepareUpdate().setIndex(indexOrAlias()).setType("type1").setId("7")
.setScript("ctx._source.field += 1", ScriptService.ScriptType.INLINE)) .setScript("ctx._source.field += 1", ScriptService.ScriptType.INLINE))
.add(client().prepareUpdate().setIndex("test").setType("type1").setId("2") .add(client().prepareUpdate().setIndex(indexOrAlias()).setType("type1").setId("2")
.setScript("ctx._source.field += 1", ScriptService.ScriptType.INLINE)) .setScript("ctx._source.field += 1", ScriptService.ScriptType.INLINE))
.execute().actionGet(); .execute().actionGet();
@ -112,9 +120,11 @@ public class BulkTests extends ElasticsearchIntegrationTest {
assertThat(((UpdateResponse) bulkResponse.getItems()[0].getResponse()).getId(), equalTo("6")); assertThat(((UpdateResponse) bulkResponse.getItems()[0].getResponse()).getId(), equalTo("6"));
assertThat(((UpdateResponse) bulkResponse.getItems()[0].getResponse()).getVersion(), equalTo(1l)); assertThat(((UpdateResponse) bulkResponse.getItems()[0].getResponse()).getVersion(), equalTo(1l));
assertThat(bulkResponse.getItems()[1].getResponse(), nullValue()); assertThat(bulkResponse.getItems()[1].getResponse(), nullValue());
assertThat(bulkResponse.getItems()[1].getFailure().getIndex(), equalTo("test"));
assertThat(bulkResponse.getItems()[1].getFailure().getId(), equalTo("7")); assertThat(bulkResponse.getItems()[1].getFailure().getId(), equalTo("7"));
assertThat(bulkResponse.getItems()[1].getFailure().getMessage(), containsString("DocumentMissingException")); assertThat(bulkResponse.getItems()[1].getFailure().getMessage(), containsString("DocumentMissingException"));
assertThat(((UpdateResponse) bulkResponse.getItems()[2].getResponse()).getId(), equalTo("2")); assertThat(((UpdateResponse) bulkResponse.getItems()[2].getResponse()).getId(), equalTo("2"));
assertThat(((UpdateResponse) bulkResponse.getItems()[2].getResponse()).getIndex(), equalTo("test"));
assertThat(((UpdateResponse) bulkResponse.getItems()[2].getResponse()).getVersion(), equalTo(3l)); assertThat(((UpdateResponse) bulkResponse.getItems()[2].getResponse()).getVersion(), equalTo(3l));
getResponse = client().prepareGet().setIndex("test").setType("type1").setId("6").setFields("field").execute().actionGet(); getResponse = client().prepareGet().setIndex("test").setType("type1").setId("6").setFields("field").execute().actionGet();
@ -640,5 +650,9 @@ public class BulkTests extends ElasticsearchIntegrationTest {
assertThat(bulkItemResponse.getItems()[4].getOpType(), is("delete")); assertThat(bulkItemResponse.getItems()[4].getOpType(), is("delete"));
assertThat(bulkItemResponse.getItems()[5].getOpType(), is("delete")); assertThat(bulkItemResponse.getItems()[5].getOpType(), is("delete"));
} }
private static String indexOrAlias() {
return randomBoolean() ? "test" : "alias";
}
} }

View File

@ -19,6 +19,7 @@
package org.elasticsearch.explain; package org.elasticsearch.explain;
import org.elasticsearch.action.admin.indices.alias.Alias;
import org.elasticsearch.action.explain.ExplainResponse; import org.elasticsearch.action.explain.ExplainResponse;
import org.elasticsearch.common.settings.ImmutableSettings; import org.elasticsearch.common.settings.ImmutableSettings;
import org.elasticsearch.index.query.FilterBuilders; import org.elasticsearch.index.query.FilterBuilders;
@ -33,7 +34,9 @@ import java.util.Map;
import static org.elasticsearch.common.xcontent.XContentFactory.jsonBuilder; import static org.elasticsearch.common.xcontent.XContentFactory.jsonBuilder;
import static org.elasticsearch.index.query.QueryBuilders.queryString; import static org.elasticsearch.index.query.QueryBuilders.queryString;
import static org.elasticsearch.test.hamcrest.ElasticsearchAssertions.assertAcked;
import static org.hamcrest.Matchers.equalTo; import static org.hamcrest.Matchers.equalTo;
import static org.hamcrest.Matchers.notNullValue;
/** /**
*/ */
@ -41,18 +44,15 @@ public class ExplainActionTests extends ElasticsearchIntegrationTest {
@Test @Test
public void testSimple() throws Exception { public void testSimple() throws Exception {
client().admin().indices().prepareCreate("test").setSettings( assertAcked(prepareCreate("test")
ImmutableSettings.settingsBuilder().put("index.refresh_interval", -1) .addAlias(new Alias("alias"))
).execute().actionGet(); .setSettings(ImmutableSettings.settingsBuilder().put("index.refresh_interval", -1)));
client().admin().cluster().prepareHealth("test").setWaitForGreenStatus().execute().actionGet(); ensureGreen("test");
client().prepareIndex("test", "test", "1") client().prepareIndex("test", "test", "1").setSource("field", "value1").get();
.setSource("field", "value1")
.execute().actionGet(); ExplainResponse response = client().prepareExplain(indexOrAlias(), "test", "1")
.setQuery(QueryBuilders.matchAllQuery()).get();
ExplainResponse response = client().prepareExplain("test", "test", "1")
.setQuery(QueryBuilders.matchAllQuery())
.execute().actionGet();
assertNotNull(response); assertNotNull(response);
assertFalse(response.isExists()); // not a match b/c not realtime assertFalse(response.isExists()); // not a match b/c not realtime
assertThat(response.getIndex(), equalTo("test")); assertThat(response.getIndex(), equalTo("test"));
@ -60,10 +60,9 @@ public class ExplainActionTests extends ElasticsearchIntegrationTest {
assertThat(response.getId(), equalTo("1")); assertThat(response.getId(), equalTo("1"));
assertFalse(response.isMatch()); // not a match b/c not realtime assertFalse(response.isMatch()); // not a match b/c not realtime
client().admin().indices().prepareRefresh("test").execute().actionGet(); refresh();
response = client().prepareExplain("test", "test", "1") response = client().prepareExplain(indexOrAlias(), "test", "1")
.setQuery(QueryBuilders.matchAllQuery()) .setQuery(QueryBuilders.matchAllQuery()).get();
.execute().actionGet();
assertNotNull(response); assertNotNull(response);
assertTrue(response.isMatch()); assertTrue(response.isMatch());
assertNotNull(response.getExplanation()); assertNotNull(response.getExplanation());
@ -73,10 +72,8 @@ public class ExplainActionTests extends ElasticsearchIntegrationTest {
assertThat(response.getId(), equalTo("1")); assertThat(response.getId(), equalTo("1"));
assertThat(response.getExplanation().getValue(), equalTo(1.0f)); assertThat(response.getExplanation().getValue(), equalTo(1.0f));
client().admin().indices().prepareRefresh("test").execute().actionGet(); response = client().prepareExplain(indexOrAlias(), "test", "1")
response = client().prepareExplain("test", "test", "1") .setQuery(QueryBuilders.termQuery("field", "value2")).get();
.setQuery(QueryBuilders.termQuery("field", "value2"))
.execute().actionGet();
assertNotNull(response); assertNotNull(response);
assertTrue(response.isExists()); assertTrue(response.isExists());
assertFalse(response.isMatch()); assertFalse(response.isMatch());
@ -86,13 +83,10 @@ public class ExplainActionTests extends ElasticsearchIntegrationTest {
assertNotNull(response.getExplanation()); assertNotNull(response.getExplanation());
assertFalse(response.getExplanation().isMatch()); assertFalse(response.getExplanation().isMatch());
client().admin().indices().prepareRefresh("test").execute().actionGet(); response = client().prepareExplain(indexOrAlias(), "test", "1")
response = client().prepareExplain("test", "test", "1")
.setQuery(QueryBuilders.boolQuery() .setQuery(QueryBuilders.boolQuery()
.must(QueryBuilders.termQuery("field", "value1")) .must(QueryBuilders.termQuery("field", "value1"))
.must(QueryBuilders.termQuery("field", "value2")) .must(QueryBuilders.termQuery("field", "value2"))).get();
)
.execute().actionGet();
assertNotNull(response); assertNotNull(response);
assertTrue(response.isExists()); assertTrue(response.isExists());
assertFalse(response.isMatch()); assertFalse(response.isMatch());
@ -103,9 +97,8 @@ public class ExplainActionTests extends ElasticsearchIntegrationTest {
assertFalse(response.getExplanation().isMatch()); assertFalse(response.getExplanation().isMatch());
assertThat(response.getExplanation().getDetails().length, equalTo(2)); assertThat(response.getExplanation().getDetails().length, equalTo(2));
response = client().prepareExplain("test", "test", "2") response = client().prepareExplain(indexOrAlias(), "test", "2")
.setQuery(QueryBuilders.matchAllQuery()) .setQuery(QueryBuilders.matchAllQuery()).get();
.execute().actionGet();
assertNotNull(response); assertNotNull(response);
assertFalse(response.isExists()); assertFalse(response.isExists());
assertFalse(response.isMatch()); assertFalse(response.isMatch());
@ -117,8 +110,8 @@ public class ExplainActionTests extends ElasticsearchIntegrationTest {
@SuppressWarnings("unchecked") @SuppressWarnings("unchecked")
@Test @Test
public void testExplainWithFields() throws Exception { public void testExplainWithFields() throws Exception {
createIndex("test"); assertAcked(prepareCreate("test").addAlias(new Alias("alias")));
client().admin().cluster().prepareHealth("test").setWaitForGreenStatus().execute().actionGet(); ensureGreen("test");
client().prepareIndex("test", "test", "1") client().prepareIndex("test", "test", "1")
.setSource( .setSource(
@ -127,14 +120,12 @@ public class ExplainActionTests extends ElasticsearchIntegrationTest {
.field("field1", "value1") .field("field1", "value1")
.field("field2", "value2") .field("field2", "value2")
.endObject() .endObject()
.endObject() .endObject()).get();
).execute().actionGet();
client().admin().indices().prepareRefresh("test").execute().actionGet(); refresh();
ExplainResponse response = client().prepareExplain("test", "test", "1") ExplainResponse response = client().prepareExplain(indexOrAlias(), "test", "1")
.setQuery(QueryBuilders.matchAllQuery()) .setQuery(QueryBuilders.matchAllQuery())
.setFields("obj1.field1") .setFields("obj1.field1").get();
.execute().actionGet();
assertNotNull(response); assertNotNull(response);
assertTrue(response.isMatch()); assertTrue(response.isMatch());
assertNotNull(response.getExplanation()); assertNotNull(response.getExplanation());
@ -146,12 +137,10 @@ public class ExplainActionTests extends ElasticsearchIntegrationTest {
assertThat(response.getGetResult().getFields().get("obj1.field1").getValue().toString(), equalTo("value1")); assertThat(response.getGetResult().getFields().get("obj1.field1").getValue().toString(), equalTo("value1"));
assertThat(response.getGetResult().isSourceEmpty(), equalTo(true)); assertThat(response.getGetResult().isSourceEmpty(), equalTo(true));
client().admin().indices().prepareRefresh("test").execute().actionGet(); refresh();
response = client().prepareExplain("test", "test", "1") response = client().prepareExplain(indexOrAlias(), "test", "1")
.setQuery(QueryBuilders.matchAllQuery()) .setQuery(QueryBuilders.matchAllQuery())
.setFields("obj1.field1") .setFields("obj1.field1").setFetchSource(true).get();
.setFetchSource(true)
.get();
assertNotNull(response); assertNotNull(response);
assertTrue(response.isMatch()); assertTrue(response.isMatch());
assertNotNull(response.getExplanation()); assertNotNull(response.getExplanation());
@ -163,10 +152,9 @@ public class ExplainActionTests extends ElasticsearchIntegrationTest {
assertThat(response.getGetResult().getFields().get("obj1.field1").getValue().toString(), equalTo("value1")); assertThat(response.getGetResult().getFields().get("obj1.field1").getValue().toString(), equalTo("value1"));
assertThat(response.getGetResult().isSourceEmpty(), equalTo(false)); assertThat(response.getGetResult().isSourceEmpty(), equalTo(false));
response = client().prepareExplain("test", "test", "1") response = client().prepareExplain(indexOrAlias(), "test", "1")
.setQuery(QueryBuilders.matchAllQuery()) .setQuery(QueryBuilders.matchAllQuery())
.setFields("obj1.field1", "obj1.field2") .setFields("obj1.field1", "obj1.field2").get();
.execute().actionGet();
assertNotNull(response); assertNotNull(response);
assertTrue(response.isMatch()); assertTrue(response.isMatch());
String v1 = (String) response.getGetResult().field("obj1.field1").getValue(); String v1 = (String) response.getGetResult().field("obj1.field1").getValue();
@ -178,8 +166,8 @@ public class ExplainActionTests extends ElasticsearchIntegrationTest {
@SuppressWarnings("unchecked") @SuppressWarnings("unchecked")
@Test @Test
public void testExplainWitSource() throws Exception { public void testExplainWitSource() throws Exception {
client().admin().indices().prepareCreate("test").execute().actionGet(); assertAcked(prepareCreate("test").addAlias(new Alias("alias")));
client().admin().cluster().prepareHealth("test").setWaitForGreenStatus().execute().actionGet(); ensureGreen("test");
client().prepareIndex("test", "test", "1") client().prepareIndex("test", "test", "1")
.setSource( .setSource(
@ -188,14 +176,12 @@ public class ExplainActionTests extends ElasticsearchIntegrationTest {
.field("field1", "value1") .field("field1", "value1")
.field("field2", "value2") .field("field2", "value2")
.endObject() .endObject()
.endObject() .endObject()).get();
).execute().actionGet();
client().admin().indices().prepareRefresh("test").execute().actionGet(); refresh();
ExplainResponse response = client().prepareExplain("test", "test", "1") ExplainResponse response = client().prepareExplain(indexOrAlias(), "test", "1")
.setQuery(QueryBuilders.matchAllQuery()) .setQuery(QueryBuilders.matchAllQuery())
.setFetchSource("obj1.field1", null) .setFetchSource("obj1.field1", null).get();
.get();
assertNotNull(response); assertNotNull(response);
assertTrue(response.isMatch()); assertTrue(response.isMatch());
assertNotNull(response.getExplanation()); assertNotNull(response.getExplanation());
@ -206,35 +192,53 @@ public class ExplainActionTests extends ElasticsearchIntegrationTest {
assertThat(response.getGetResult().getSource().size(), equalTo(1)); assertThat(response.getGetResult().getSource().size(), equalTo(1));
assertThat(((Map<String, Object>) response.getGetResult().getSource().get("obj1")).get("field1").toString(), equalTo("value1")); assertThat(((Map<String, Object>) response.getGetResult().getSource().get("obj1")).get("field1").toString(), equalTo("value1"));
response = client().prepareExplain("test", "test", "1") response = client().prepareExplain(indexOrAlias(), "test", "1")
.setQuery(QueryBuilders.matchAllQuery()) .setQuery(QueryBuilders.matchAllQuery())
.setFetchSource(null, "obj1.field2") .setFetchSource(null, "obj1.field2").get();
.execute().actionGet();
assertNotNull(response); assertNotNull(response);
assertTrue(response.isMatch()); assertTrue(response.isMatch());
assertThat(((Map<String, Object>) response.getGetResult().getSource().get("obj1")).get("field1").toString(), equalTo("value1")); assertThat(((Map<String, Object>) response.getGetResult().getSource().get("obj1")).get("field1").toString(), equalTo("value1"));
} }
@Test @Test
public void testExplainWithAlias() throws Exception { public void testExplainWithFilteredAlias() throws Exception {
client().admin().indices().prepareCreate("test") assertAcked(prepareCreate("test").addAlias(new Alias("alias1").filter(FilterBuilders.termFilter("field2", "value2"))));
.execute().actionGet(); ensureGreen("test");
client().admin().cluster().prepareHealth("test").setWaitForGreenStatus().execute().actionGet();
client().admin().indices().prepareAliases().addAlias("test", "alias1", FilterBuilders.termFilter("field2", "value2")) client().prepareIndex("test", "test", "1").setSource("field1", "value1", "field2", "value1").get();
.execute().actionGet(); refresh();
client().prepareIndex("test", "test", "1").setSource("field1", "value1", "field2", "value1").execute().actionGet();
client().admin().indices().prepareRefresh("test").execute().actionGet();
ExplainResponse response = client().prepareExplain("alias1", "test", "1") ExplainResponse response = client().prepareExplain("alias1", "test", "1")
.setQuery(QueryBuilders.matchAllQuery()) .setQuery(QueryBuilders.matchAllQuery()).get();
.execute().actionGet(); assertNotNull(response);
assertTrue(response.isExists());
assertFalse(response.isMatch());
}
@Test
public void testExplainWithFilteredAliasFetchSource() throws Exception {
assertAcked(client().admin().indices().prepareCreate("test")
.addAlias(new Alias("alias1").filter(FilterBuilders.termFilter("field2", "value2"))));
ensureGreen("test");
client().prepareIndex("test", "test", "1").setSource("field1", "value1", "field2", "value1").get();
refresh();
ExplainResponse response = client().prepareExplain("alias1", "test", "1")
.setQuery(QueryBuilders.matchAllQuery()).setFetchSource(true).get();
assertNotNull(response); assertNotNull(response);
assertTrue(response.isExists()); assertTrue(response.isExists());
assertFalse(response.isMatch()); assertFalse(response.isMatch());
assertThat(response.getIndex(), equalTo("test")); assertThat(response.getIndex(), equalTo("test"));
assertThat(response.getType(), equalTo("test")); assertThat(response.getType(), equalTo("test"));
assertThat(response.getId(), equalTo("1")); assertThat(response.getId(), equalTo("1"));
assertThat(response.getGetResult(), notNullValue());
assertThat(response.getGetResult().getIndex(), equalTo("test"));
assertThat(response.getGetResult().getType(), equalTo("test"));
assertThat(response.getGetResult().getId(), equalTo("1"));
assertThat(response.getGetResult().getSource(), notNullValue());
assertThat((String)response.getGetResult().getSource().get("field1"), equalTo("value1"));
} }
@Test @Test
@ -252,4 +256,8 @@ public class ExplainActionTests extends ElasticsearchIntegrationTest {
assertThat(explainResponse.isExists(), equalTo(true)); assertThat(explainResponse.isExists(), equalTo(true));
assertThat(explainResponse.isMatch(), equalTo(true)); assertThat(explainResponse.isMatch(), equalTo(true));
} }
private static String indexOrAlias() {
return randomBoolean() ? "test" : "alias";
}
} }

File diff suppressed because it is too large Load Diff

View File

@ -183,7 +183,7 @@ public class TimestampMappingTests extends ElasticsearchSingleNodeTest {
MappingMetaData mappingMetaData = new MappingMetaData(docMapper); MappingMetaData mappingMetaData = new MappingMetaData(docMapper);
IndexRequest request = new IndexRequest("test", "type", "1").source(doc); IndexRequest request = new IndexRequest("test", "type", "1").source(doc);
request.process(metaData, null, mappingMetaData, true); request.process(metaData, mappingMetaData, true, "test");
assertThat(request.timestamp(), notNullValue()); assertThat(request.timestamp(), notNullValue());
// We should have less than one minute (probably some ms) // We should have less than one minute (probably some ms)
@ -209,7 +209,7 @@ public class TimestampMappingTests extends ElasticsearchSingleNodeTest {
MappingMetaData mappingMetaData = new MappingMetaData(docMapper); MappingMetaData mappingMetaData = new MappingMetaData(docMapper);
IndexRequest request = new IndexRequest("test", "type", "1").source(doc); IndexRequest request = new IndexRequest("test", "type", "1").source(doc);
request.process(metaData, null, mappingMetaData, true); request.process(metaData, mappingMetaData, true, "test");
assertThat(request.timestamp(), notNullValue()); assertThat(request.timestamp(), notNullValue());
// We should have less than one minute (probably some ms) // We should have less than one minute (probably some ms)
@ -238,7 +238,7 @@ public class TimestampMappingTests extends ElasticsearchSingleNodeTest {
MappingMetaData mappingMetaData = new MappingMetaData(docMapper); MappingMetaData mappingMetaData = new MappingMetaData(docMapper);
IndexRequest request = new IndexRequest("test", "type", "1").source(doc); IndexRequest request = new IndexRequest("test", "type", "1").source(doc);
request.process(metaData, null, mappingMetaData, true); request.process(metaData, mappingMetaData, true, "test");
assertThat(request.timestamp(), notNullValue()); assertThat(request.timestamp(), notNullValue());
assertThat(request.timestamp(), is(MappingMetaData.Timestamp.parseStringTimestamp("1970-01-01", Joda.forPattern("YYYY-MM-dd")))); assertThat(request.timestamp(), is(MappingMetaData.Timestamp.parseStringTimestamp("1970-01-01", Joda.forPattern("YYYY-MM-dd"))));
} }
@ -263,7 +263,7 @@ public class TimestampMappingTests extends ElasticsearchSingleNodeTest {
MappingMetaData mappingMetaData = new MappingMetaData(docMapper); MappingMetaData mappingMetaData = new MappingMetaData(docMapper);
IndexRequest request = new IndexRequest("test", "type", "1").source(doc); IndexRequest request = new IndexRequest("test", "type", "1").source(doc);
request.process(metaData, null, mappingMetaData, true); request.process(metaData, mappingMetaData, true, "test");
assertThat(request.timestamp(), notNullValue()); assertThat(request.timestamp(), notNullValue());
assertThat(request.timestamp(), is(MappingMetaData.Timestamp.parseStringTimestamp("1970-01-01", Joda.forPattern("YYYY-MM-dd")))); assertThat(request.timestamp(), is(MappingMetaData.Timestamp.parseStringTimestamp("1970-01-01", Joda.forPattern("YYYY-MM-dd"))));
} }
@ -289,7 +289,7 @@ public class TimestampMappingTests extends ElasticsearchSingleNodeTest {
MappingMetaData mappingMetaData = new MappingMetaData(docMapper); MappingMetaData mappingMetaData = new MappingMetaData(docMapper);
IndexRequest request = new IndexRequest("test", "type", "1").source(doc); IndexRequest request = new IndexRequest("test", "type", "1").source(doc);
request.process(metaData, null, mappingMetaData, true); request.process(metaData, mappingMetaData, true, "test");
assertThat(request.timestamp(), notNullValue()); assertThat(request.timestamp(), notNullValue());
// We should have less than one minute (probably some ms) // We should have less than one minute (probably some ms)
@ -317,7 +317,7 @@ public class TimestampMappingTests extends ElasticsearchSingleNodeTest {
MappingMetaData mappingMetaData = new MappingMetaData(docMapper); MappingMetaData mappingMetaData = new MappingMetaData(docMapper);
IndexRequest request = new IndexRequest("test", "type", "1").source(doc); IndexRequest request = new IndexRequest("test", "type", "1").source(doc);
request.process(metaData, null, mappingMetaData, true); request.process(metaData, mappingMetaData, true, "test");
assertThat(request.timestamp(), notNullValue()); assertThat(request.timestamp(), notNullValue());
// We should have less than one minute (probably some ms) // We should have less than one minute (probably some ms)
@ -345,7 +345,7 @@ public class TimestampMappingTests extends ElasticsearchSingleNodeTest {
MappingMetaData mappingMetaData = new MappingMetaData(docMapper); MappingMetaData mappingMetaData = new MappingMetaData(docMapper);
IndexRequest request = new IndexRequest("test", "type", "1").source(doc); IndexRequest request = new IndexRequest("test", "type", "1").source(doc);
request.process(metaData, null, mappingMetaData, true); request.process(metaData, mappingMetaData, true, "test");
} }
@Test(expected = TimestampParsingException.class) // Issue 4718: was throwing a TimestampParsingException: failed to parse timestamp [null] @Test(expected = TimestampParsingException.class) // Issue 4718: was throwing a TimestampParsingException: failed to parse timestamp [null]
@ -367,9 +367,10 @@ public class TimestampMappingTests extends ElasticsearchSingleNodeTest {
MappingMetaData mappingMetaData = new MappingMetaData(docMapper); MappingMetaData mappingMetaData = new MappingMetaData(docMapper);
IndexRequest request = new IndexRequest("test", "type", "1").source(doc); IndexRequest request = new IndexRequest("test", "type", "1").source(doc);
request.process(metaData, null, mappingMetaData, true); request.process(metaData, mappingMetaData, true, "test");
} }
@Test
public void testDefaultTimestampStream() throws IOException { public void testDefaultTimestampStream() throws IOException {
// Testing null value for default timestamp // Testing null value for default timestamp
{ {

View File

@ -20,9 +20,9 @@ package org.elasticsearch.indices.analyze;
import org.elasticsearch.ElasticsearchException; import org.elasticsearch.ElasticsearchException;
import org.elasticsearch.ElasticsearchIllegalArgumentException; import org.elasticsearch.ElasticsearchIllegalArgumentException;
import org.elasticsearch.action.admin.indices.alias.Alias;
import org.elasticsearch.action.admin.indices.analyze.AnalyzeRequestBuilder; import org.elasticsearch.action.admin.indices.analyze.AnalyzeRequestBuilder;
import org.elasticsearch.action.admin.indices.analyze.AnalyzeResponse; import org.elasticsearch.action.admin.indices.analyze.AnalyzeResponse;
import org.elasticsearch.common.xcontent.XContentFactory;
import org.elasticsearch.test.ElasticsearchIntegrationTest; import org.elasticsearch.test.ElasticsearchIntegrationTest;
import org.junit.Test; import org.junit.Test;
@ -30,9 +30,7 @@ import java.io.IOException;
import static org.elasticsearch.common.settings.ImmutableSettings.settingsBuilder; import static org.elasticsearch.common.settings.ImmutableSettings.settingsBuilder;
import static org.elasticsearch.test.hamcrest.ElasticsearchAssertions.assertAcked; import static org.elasticsearch.test.hamcrest.ElasticsearchAssertions.assertAcked;
import static org.hamcrest.Matchers.equalTo; import static org.hamcrest.Matchers.*;
import static org.hamcrest.Matchers.hasSize;
import static org.hamcrest.Matchers.is;
/** /**
* *
@ -41,11 +39,11 @@ public class AnalyzeActionTests extends ElasticsearchIntegrationTest {
@Test @Test
public void simpleAnalyzerTests() throws Exception { public void simpleAnalyzerTests() throws Exception {
createIndex("test"); assertAcked(prepareCreate("test").addAlias(new Alias("alias")));
ensureGreen(); ensureGreen();
for (int i = 0; i < 10; i++) { for (int i = 0; i < 10; i++) {
AnalyzeResponse analyzeResponse = client().admin().indices().prepareAnalyze("test", "this is a test").execute().actionGet(); AnalyzeResponse analyzeResponse = client().admin().indices().prepareAnalyze(indexOrAlias(), "this is a test").get();
assertThat(analyzeResponse.getTokens().size(), equalTo(4)); assertThat(analyzeResponse.getTokens().size(), equalTo(4));
AnalyzeResponse.AnalyzeToken token = analyzeResponse.getTokens().get(0); AnalyzeResponse.AnalyzeToken token = analyzeResponse.getTokens().get(0);
assertThat(token.getTerm(), equalTo("this")); assertThat(token.getTerm(), equalTo("this"));
@ -68,38 +66,34 @@ public class AnalyzeActionTests extends ElasticsearchIntegrationTest {
@Test @Test
public void analyzeNumericField() throws ElasticsearchException, IOException { public void analyzeNumericField() throws ElasticsearchException, IOException {
createIndex("test"); assertAcked(prepareCreate("test").addAlias(new Alias("alias")).addMapping("test", "long", "type=long", "double", "type=double"));
ensureGreen(); ensureGreen("test");
client().prepareIndex("test", "test", "1")
.setSource(XContentFactory.jsonBuilder()
.startObject()
.field("long", 1l)
.field("double", 1.0d)
.endObject())
.setRefresh(true).execute().actionGet();
try { try {
client().admin().indices().prepareAnalyze("test", "123").setField("long").execute().actionGet(); client().admin().indices().prepareAnalyze(indexOrAlias(), "123").setField("long").get();
fail("shouldn't get here");
} catch (ElasticsearchIllegalArgumentException ex) { } catch (ElasticsearchIllegalArgumentException ex) {
//all good
} }
try { try {
client().admin().indices().prepareAnalyze("test", "123.0").setField("double").execute().actionGet(); client().admin().indices().prepareAnalyze(indexOrAlias(), "123.0").setField("double").get();
fail("shouldn't get here");
} catch (ElasticsearchIllegalArgumentException ex) { } catch (ElasticsearchIllegalArgumentException ex) {
//all good
} }
} }
@Test @Test
public void analyzeWithNoIndex() throws Exception { public void analyzeWithNoIndex() throws Exception {
AnalyzeResponse analyzeResponse = client().admin().indices().prepareAnalyze("THIS IS A TEST").setAnalyzer("simple").execute().actionGet(); AnalyzeResponse analyzeResponse = client().admin().indices().prepareAnalyze("THIS IS A TEST").setAnalyzer("simple").get();
assertThat(analyzeResponse.getTokens().size(), equalTo(4)); assertThat(analyzeResponse.getTokens().size(), equalTo(4));
analyzeResponse = client().admin().indices().prepareAnalyze("THIS IS A TEST").setTokenizer("keyword").setTokenFilters("lowercase").execute().actionGet(); analyzeResponse = client().admin().indices().prepareAnalyze("THIS IS A TEST").setTokenizer("keyword").setTokenFilters("lowercase").get();
assertThat(analyzeResponse.getTokens().size(), equalTo(1)); assertThat(analyzeResponse.getTokens().size(), equalTo(1));
assertThat(analyzeResponse.getTokens().get(0).getTerm(), equalTo("this is a test")); assertThat(analyzeResponse.getTokens().get(0).getTerm(), equalTo("this is a test"));
analyzeResponse = client().admin().indices().prepareAnalyze("THIS IS A TEST").setTokenizer("standard").setTokenFilters("lowercase", "reverse").execute().actionGet(); analyzeResponse = client().admin().indices().prepareAnalyze("THIS IS A TEST").setTokenizer("standard").setTokenFilters("lowercase", "reverse").get();
assertThat(analyzeResponse.getTokens().size(), equalTo(4)); assertThat(analyzeResponse.getTokens().size(), equalTo(4));
AnalyzeResponse.AnalyzeToken token = analyzeResponse.getTokens().get(0); AnalyzeResponse.AnalyzeToken token = analyzeResponse.getTokens().get(0);
assertThat(token.getTerm(), equalTo("siht")); assertThat(token.getTerm(), equalTo("siht"));
@ -114,26 +108,26 @@ public class AnalyzeActionTests extends ElasticsearchIntegrationTest {
@Test @Test
public void analyzeWithCharFilters() throws Exception { public void analyzeWithCharFilters() throws Exception {
assertAcked(prepareCreate("test").setSettings(settingsBuilder() assertAcked(prepareCreate("test").addAlias(new Alias("alias"))
.put(indexSettings()) .setSettings(settingsBuilder().put(indexSettings())
.put("index.analysis.char_filter.custom_mapping.type", "mapping") .put("index.analysis.char_filter.custom_mapping.type", "mapping")
.putArray("index.analysis.char_filter.custom_mapping.mappings", "ph=>f", "qu=>q") .putArray("index.analysis.char_filter.custom_mapping.mappings", "ph=>f", "qu=>q")
.put("index.analysis.analyzer.custom_with_char_filter.tokenizer", "standard") .put("index.analysis.analyzer.custom_with_char_filter.tokenizer", "standard")
.putArray("index.analysis.analyzer.custom_with_char_filter.char_filter", "custom_mapping"))); .putArray("index.analysis.analyzer.custom_with_char_filter.char_filter", "custom_mapping")));
ensureGreen(); ensureGreen();
AnalyzeResponse analyzeResponse = client().admin().indices().prepareAnalyze("<h2><b>THIS</b> IS A</h2> <a href=\"#\">TEST</a>").setTokenizer("standard").setCharFilters("html_strip").execute().actionGet(); AnalyzeResponse analyzeResponse = client().admin().indices().prepareAnalyze("<h2><b>THIS</b> IS A</h2> <a href=\"#\">TEST</a>").setTokenizer("standard").setCharFilters("html_strip").get();
assertThat(analyzeResponse.getTokens().size(), equalTo(4)); assertThat(analyzeResponse.getTokens().size(), equalTo(4));
analyzeResponse = client().admin().indices().prepareAnalyze("THIS IS A <b>TEST</b>").setTokenizer("keyword").setTokenFilters("lowercase").setCharFilters("html_strip").execute().actionGet(); analyzeResponse = client().admin().indices().prepareAnalyze("THIS IS A <b>TEST</b>").setTokenizer("keyword").setTokenFilters("lowercase").setCharFilters("html_strip").get();
assertThat(analyzeResponse.getTokens().size(), equalTo(1)); assertThat(analyzeResponse.getTokens().size(), equalTo(1));
assertThat(analyzeResponse.getTokens().get(0).getTerm(), equalTo("this is a test")); assertThat(analyzeResponse.getTokens().get(0).getTerm(), equalTo("this is a test"));
analyzeResponse = client().admin().indices().prepareAnalyze("test", "jeff quit phish").setTokenizer("keyword").setTokenFilters("lowercase").setCharFilters("custom_mapping").execute().actionGet(); analyzeResponse = client().admin().indices().prepareAnalyze(indexOrAlias(), "jeff quit phish").setTokenizer("keyword").setTokenFilters("lowercase").setCharFilters("custom_mapping").get();
assertThat(analyzeResponse.getTokens().size(), equalTo(1)); assertThat(analyzeResponse.getTokens().size(), equalTo(1));
assertThat(analyzeResponse.getTokens().get(0).getTerm(), equalTo("jeff qit fish")); assertThat(analyzeResponse.getTokens().get(0).getTerm(), equalTo("jeff qit fish"));
analyzeResponse = client().admin().indices().prepareAnalyze("test", "<a href=\"#\">jeff quit fish</a>").setTokenizer("standard").setCharFilters("html_strip", "custom_mapping").execute().actionGet(); analyzeResponse = client().admin().indices().prepareAnalyze(indexOrAlias(), "<a href=\"#\">jeff quit fish</a>").setTokenizer("standard").setCharFilters("html_strip", "custom_mapping").get();
assertThat(analyzeResponse.getTokens().size(), equalTo(3)); assertThat(analyzeResponse.getTokens().size(), equalTo(3));
AnalyzeResponse.AnalyzeToken token = analyzeResponse.getTokens().get(0); AnalyzeResponse.AnalyzeToken token = analyzeResponse.getTokens().get(0);
assertThat(token.getTerm(), equalTo("jeff")); assertThat(token.getTerm(), equalTo("jeff"));
@ -145,8 +139,7 @@ public class AnalyzeActionTests extends ElasticsearchIntegrationTest {
@Test @Test
public void analyzerWithFieldOrTypeTests() throws Exception { public void analyzerWithFieldOrTypeTests() throws Exception {
assertAcked(prepareCreate("test").addAlias(new Alias("alias")));
createIndex("test");
ensureGreen(); ensureGreen();
client().admin().indices().preparePutMapping("test") client().admin().indices().preparePutMapping("test")
@ -161,12 +154,13 @@ public class AnalyzeActionTests extends ElasticsearchIntegrationTest {
" }\n" + " }\n" +
" }\n" + " }\n" +
"}" "}"
).execute().actionGet(); ).get();
for (int i = 0; i < 10; i++) { for (int i = 0; i < 10; i++) {
final AnalyzeRequestBuilder requestBuilder = client().admin().indices().prepareAnalyze("test", "THIS IS A TEST"); final AnalyzeRequestBuilder requestBuilder = client().admin().indices().prepareAnalyze("THIS IS A TEST");
requestBuilder.setIndex(indexOrAlias());
requestBuilder.setField("document.simple"); requestBuilder.setField("document.simple");
AnalyzeResponse analyzeResponse = requestBuilder.execute().actionGet(); AnalyzeResponse analyzeResponse = requestBuilder.get();
assertThat(analyzeResponse.getTokens().size(), equalTo(4)); assertThat(analyzeResponse.getTokens().size(), equalTo(4));
AnalyzeResponse.AnalyzeToken token = analyzeResponse.getTokens().get(3); AnalyzeResponse.AnalyzeToken token = analyzeResponse.getTokens().get(3);
assertThat(token.getTerm(), equalTo("test")); assertThat(token.getTerm(), equalTo("test"));
@ -193,4 +187,8 @@ public class AnalyzeActionTests extends ElasticsearchIntegrationTest {
assertThat(response.getTokens().get(i).getTerm(), is(tokens[i])); assertThat(response.getTokens().get(i).getTerm(), is(tokens[i]));
} }
} }
private static String indexOrAlias() {
return randomBoolean() ? "test" : "alias";
}
} }

View File

@ -18,6 +18,7 @@
*/ */
package org.elasticsearch.mget; package org.elasticsearch.mget;
import org.elasticsearch.action.admin.indices.alias.Alias;
import org.elasticsearch.action.get.MultiGetItemResponse; import org.elasticsearch.action.get.MultiGetItemResponse;
import org.elasticsearch.action.get.MultiGetRequest; import org.elasticsearch.action.get.MultiGetRequest;
import org.elasticsearch.action.get.MultiGetRequestBuilder; import org.elasticsearch.action.get.MultiGetRequestBuilder;
@ -71,25 +72,24 @@ public class SimpleMgetTests extends ElasticsearchIntegrationTest {
@Test @Test
public void testThatParentPerDocumentIsSupported() throws Exception { public void testThatParentPerDocumentIsSupported() throws Exception {
createIndex("test"); assertAcked(prepareCreate("test").addAlias(new Alias("alias"))
.addMapping("test", jsonBuilder()
.startObject()
.startObject("test")
.startObject("_parent")
.field("type", "foo")
.endObject()
.endObject()
.endObject()));
ensureYellow(); ensureYellow();
client().admin().indices().preparePutMapping("test").setType("test").setSource(jsonBuilder()
.startObject()
.startObject("test")
.startObject("_parent")
.field("type", "foo")
.endObject()
.endObject()
.endObject()
).get();
client().prepareIndex("test", "test", "1").setParent("4").setRefresh(true) client().prepareIndex("test", "test", "1").setParent("4").setRefresh(true)
.setSource(jsonBuilder().startObject().field("foo", "bar").endObject()) .setSource(jsonBuilder().startObject().field("foo", "bar").endObject())
.execute().actionGet(); .execute().actionGet();
MultiGetResponse mgetResponse = client().prepareMultiGet() MultiGetResponse mgetResponse = client().prepareMultiGet()
.add(new MultiGetRequest.Item("test", "test", "1").parent("4")) .add(new MultiGetRequest.Item(indexOrAlias(), "test", "1").parent("4"))
.add(new MultiGetRequest.Item("test", "test", "1")) .add(new MultiGetRequest.Item(indexOrAlias(), "test", "1"))
.execute().actionGet(); .execute().actionGet();
assertThat(mgetResponse.getResponses().length, is(2)); assertThat(mgetResponse.getResponses().length, is(2));
@ -104,7 +104,7 @@ public class SimpleMgetTests extends ElasticsearchIntegrationTest {
@SuppressWarnings("unchecked") @SuppressWarnings("unchecked")
@Test @Test
public void testThatSourceFilteringIsSupported() throws Exception { public void testThatSourceFilteringIsSupported() throws Exception {
createIndex("test"); assertAcked(prepareCreate("test").addAlias(new Alias("alias")));
ensureYellow(); ensureYellow();
BytesReference sourceBytesRef = jsonBuilder().startObject() BytesReference sourceBytesRef = jsonBuilder().startObject()
.field("field", "1", "2") .field("field", "1", "2")
@ -118,9 +118,9 @@ public class SimpleMgetTests extends ElasticsearchIntegrationTest {
MultiGetRequestBuilder request = client().prepareMultiGet(); MultiGetRequestBuilder request = client().prepareMultiGet();
for (int i = 0; i < 100; i++) { for (int i = 0; i < 100; i++) {
if (i % 2 == 0) { if (i % 2 == 0) {
request.add(new MultiGetRequest.Item("test", "type", Integer.toString(i)).fetchSourceContext(new FetchSourceContext("included", "*.hidden_field"))); request.add(new MultiGetRequest.Item(indexOrAlias(), "type", Integer.toString(i)).fetchSourceContext(new FetchSourceContext("included", "*.hidden_field")));
} else { } else {
request.add(new MultiGetRequest.Item("test", "type", Integer.toString(i)).fetchSourceContext(new FetchSourceContext(false))); request.add(new MultiGetRequest.Item(indexOrAlias(), "type", Integer.toString(i)).fetchSourceContext(new FetchSourceContext(false)));
} }
} }
@ -129,6 +129,7 @@ public class SimpleMgetTests extends ElasticsearchIntegrationTest {
assertThat(response.getResponses().length, equalTo(100)); assertThat(response.getResponses().length, equalTo(100));
for (int i = 0; i < 100; i++) { for (int i = 0; i < 100; i++) {
MultiGetItemResponse responseItem = response.getResponses()[i]; MultiGetItemResponse responseItem = response.getResponses()[i];
assertThat(responseItem.getIndex(), equalTo("test"));
if (i % 2 == 0) { if (i % 2 == 0) {
Map<String, Object> source = responseItem.getResponse().getSourceAsMap(); Map<String, Object> source = responseItem.getResponse().getSourceAsMap();
assertThat(source.size(), equalTo(1)); assertThat(source.size(), equalTo(1));
@ -139,15 +140,14 @@ public class SimpleMgetTests extends ElasticsearchIntegrationTest {
assertThat(responseItem.getResponse().getSourceAsBytes(), nullValue()); assertThat(responseItem.getResponse().getSourceAsBytes(), nullValue());
} }
} }
} }
@Test @Test
public void testThatRoutingPerDocumentIsSupported() throws Exception { public void testThatRoutingPerDocumentIsSupported() throws Exception {
assertAcked(prepareCreate("test").setSettings(ImmutableSettings.builder() assertAcked(prepareCreate("test").addAlias(new Alias("alias"))
.put(indexSettings()) .setSettings(ImmutableSettings.builder()
.put(IndexMetaData.SETTING_NUMBER_OF_SHARDS, between(2, DEFAULT_MAX_NUM_SHARDS)))); .put(indexSettings())
.put(IndexMetaData.SETTING_NUMBER_OF_SHARDS, between(2, DEFAULT_MAX_NUM_SHARDS))));
ensureYellow(); ensureYellow();
client().prepareIndex("test", "test", "1").setRefresh(true).setRouting("2") client().prepareIndex("test", "test", "1").setRefresh(true).setRouting("2")
@ -155,15 +155,21 @@ public class SimpleMgetTests extends ElasticsearchIntegrationTest {
.execute().actionGet(); .execute().actionGet();
MultiGetResponse mgetResponse = client().prepareMultiGet() MultiGetResponse mgetResponse = client().prepareMultiGet()
.add(new MultiGetRequest.Item("test", "test", "1").routing("2")) .add(new MultiGetRequest.Item(indexOrAlias(), "test", "1").routing("2"))
.add(new MultiGetRequest.Item("test", "test", "1")) .add(new MultiGetRequest.Item(indexOrAlias(), "test", "1"))
.execute().actionGet(); .execute().actionGet();
assertThat(mgetResponse.getResponses().length, is(2)); assertThat(mgetResponse.getResponses().length, is(2));
assertThat(mgetResponse.getResponses()[0].isFailed(), is(false)); assertThat(mgetResponse.getResponses()[0].isFailed(), is(false));
assertThat(mgetResponse.getResponses()[0].getResponse().isExists(), is(true)); assertThat(mgetResponse.getResponses()[0].getResponse().isExists(), is(true));
assertThat(mgetResponse.getResponses()[0].getResponse().getIndex(), is("test"));
assertThat(mgetResponse.getResponses()[1].isFailed(), is(false)); assertThat(mgetResponse.getResponses()[1].isFailed(), is(false));
assertThat(mgetResponse.getResponses()[1].getResponse().isExists(), is(false)); assertThat(mgetResponse.getResponses()[1].getResponse().isExists(), is(false));
assertThat(mgetResponse.getResponses()[1].getResponse().getIndex(), is("test"));
}
private static String indexOrAlias() {
return randomBoolean() ? "test" : "alias";
} }
} }

View File

@ -26,6 +26,7 @@ import org.elasticsearch.ElasticsearchException;
import org.elasticsearch.ElasticsearchTimeoutException; import org.elasticsearch.ElasticsearchTimeoutException;
import org.elasticsearch.action.ActionListener; import org.elasticsearch.action.ActionListener;
import org.elasticsearch.action.ActionRequestValidationException; import org.elasticsearch.action.ActionRequestValidationException;
import org.elasticsearch.action.admin.indices.alias.Alias;
import org.elasticsearch.action.delete.DeleteRequest; import org.elasticsearch.action.delete.DeleteRequest;
import org.elasticsearch.action.delete.DeleteResponse; import org.elasticsearch.action.delete.DeleteResponse;
import org.elasticsearch.action.get.GetResponse; import org.elasticsearch.action.get.GetResponse;
@ -57,48 +58,48 @@ import java.util.concurrent.Semaphore;
import java.util.concurrent.TimeUnit; import java.util.concurrent.TimeUnit;
import static org.elasticsearch.common.xcontent.XContentFactory.jsonBuilder; import static org.elasticsearch.common.xcontent.XContentFactory.jsonBuilder;
import static org.elasticsearch.test.hamcrest.ElasticsearchAssertions.assertAcked;
import static org.elasticsearch.test.hamcrest.ElasticsearchAssertions.assertThrows; import static org.elasticsearch.test.hamcrest.ElasticsearchAssertions.assertThrows;
import static org.hamcrest.Matchers.*; import static org.hamcrest.Matchers.*;
public class UpdateTests extends ElasticsearchIntegrationTest { public class UpdateTests extends ElasticsearchIntegrationTest {
private void createTestIndex() throws Exception {
protected void createIndex() throws Exception {
logger.info("--> creating index test"); logger.info("--> creating index test");
client().admin().indices().prepareCreate("test") assertAcked(prepareCreate("test").addAlias(new Alias("alias"))
.addMapping("type1", XContentFactory.jsonBuilder() .addMapping("type1", XContentFactory.jsonBuilder()
.startObject() .startObject()
.startObject("type1") .startObject("type1")
.startObject("_timestamp").field("enabled", true).field("store", "yes").endObject() .startObject("_timestamp").field("enabled", true).field("store", "yes").endObject()
.startObject("_ttl").field("enabled", true).field("store", "yes").endObject() .startObject("_ttl").field("enabled", true).field("store", "yes").endObject()
.endObject() .endObject()
.endObject()) .endObject()));
.execute().actionGet();
} }
@Test @Test
public void testUpsert() throws Exception { public void testUpsert() throws Exception {
createIndex(); createTestIndex();
ensureGreen(); ensureGreen();
UpdateResponse updateResponse = client().prepareUpdate("test", "type1", "1") UpdateResponse updateResponse = client().prepareUpdate(indexOrAlias(), "type1", "1")
.setUpsert(XContentFactory.jsonBuilder().startObject().field("field", 1).endObject()) .setUpsert(XContentFactory.jsonBuilder().startObject().field("field", 1).endObject())
.setScript("ctx._source.field += 1", ScriptService.ScriptType.INLINE) .setScript("ctx._source.field += 1", ScriptService.ScriptType.INLINE)
.execute().actionGet(); .execute().actionGet();
assertTrue(updateResponse.isCreated()); assertTrue(updateResponse.isCreated());
assertThat(updateResponse.getIndex(), equalTo("test"));
for (int i = 0; i < 5; i++) { for (int i = 0; i < 5; i++) {
GetResponse getResponse = client().prepareGet("test", "type1", "1").execute().actionGet(); GetResponse getResponse = client().prepareGet("test", "type1", "1").execute().actionGet();
assertThat(getResponse.getSourceAsMap().get("field").toString(), equalTo("1")); assertThat(getResponse.getSourceAsMap().get("field").toString(), equalTo("1"));
} }
updateResponse = client().prepareUpdate("test", "type1", "1") updateResponse = client().prepareUpdate(indexOrAlias(), "type1", "1")
.setUpsert(XContentFactory.jsonBuilder().startObject().field("field", 1).endObject()) .setUpsert(XContentFactory.jsonBuilder().startObject().field("field", 1).endObject())
.setScript("ctx._source.field += 1", ScriptService.ScriptType.INLINE) .setScript("ctx._source.field += 1", ScriptService.ScriptType.INLINE)
.execute().actionGet(); .execute().actionGet();
assertFalse(updateResponse.isCreated()); assertFalse(updateResponse.isCreated());
assertThat(updateResponse.getIndex(), equalTo("test"));
for (int i = 0; i < 5; i++) { for (int i = 0; i < 5; i++) {
GetResponse getResponse = client().prepareGet("test", "type1", "1").execute().actionGet(); GetResponse getResponse = client().prepareGet("test", "type1", "1").execute().actionGet();
@ -108,7 +109,7 @@ public class UpdateTests extends ElasticsearchIntegrationTest {
@Test @Test
public void testScriptedUpsert() throws Exception { public void testScriptedUpsert() throws Exception {
createIndex(); createTestIndex();
ensureGreen(); ensureGreen();
// Script logic is // Script logic is
@ -122,13 +123,14 @@ public class UpdateTests extends ElasticsearchIntegrationTest {
// Pay money from what will be a new account and opening balance comes from upsert doc // Pay money from what will be a new account and opening balance comes from upsert doc
// provided by client // provided by client
UpdateResponse updateResponse = client().prepareUpdate("test", "type1", "1") UpdateResponse updateResponse = client().prepareUpdate(indexOrAlias(), "type1", "1")
.setUpsert(XContentFactory.jsonBuilder().startObject().field("balance", openingBalance).endObject()) .setUpsert(XContentFactory.jsonBuilder().startObject().field("balance", openingBalance).endObject())
.setScriptedUpsert(true) .setScriptedUpsert(true)
.addScriptParam("payment", 2) .addScriptParam("payment", 2)
.setScript(script, ScriptService.ScriptType.INLINE) .setScript(script, ScriptService.ScriptType.INLINE)
.execute().actionGet(); .execute().actionGet();
assertTrue(updateResponse.isCreated()); assertTrue(updateResponse.isCreated());
assertThat(updateResponse.getIndex(), equalTo("test"));
for (int i = 0; i < 5; i++) { for (int i = 0; i < 5; i++) {
GetResponse getResponse = client().prepareGet("test", "type1", "1").execute().actionGet(); GetResponse getResponse = client().prepareGet("test", "type1", "1").execute().actionGet();
@ -136,42 +138,44 @@ public class UpdateTests extends ElasticsearchIntegrationTest {
} }
// Now pay money for an existing account where balance is stored in es // Now pay money for an existing account where balance is stored in es
updateResponse = client().prepareUpdate("test", "type1", "1") updateResponse = client().prepareUpdate(indexOrAlias(), "type1", "1")
.setUpsert(XContentFactory.jsonBuilder().startObject().field("balance", openingBalance).endObject()) .setUpsert(XContentFactory.jsonBuilder().startObject().field("balance", openingBalance).endObject())
.setScriptedUpsert(true) .setScriptedUpsert(true)
.addScriptParam("payment", 2) .addScriptParam("payment", 2)
.setScript(script, ScriptService.ScriptType.INLINE) .setScript(script, ScriptService.ScriptType.INLINE)
.execute().actionGet(); .execute().actionGet();
assertFalse(updateResponse.isCreated()); assertFalse(updateResponse.isCreated());
assertThat(updateResponse.getIndex(), equalTo("test"));
for (int i = 0; i < 5; i++) { for (int i = 0; i < 5; i++) {
GetResponse getResponse = client().prepareGet("test", "type1", "1").execute().actionGet(); GetResponse getResponse = client().prepareGet("test", "type1", "1").execute().actionGet();
assertThat(getResponse.getSourceAsMap().get("balance").toString(), equalTo("7")); assertThat(getResponse.getSourceAsMap().get("balance").toString(), equalTo("7"));
} }
} }
@Test @Test
public void testUpsertDoc() throws Exception { public void testUpsertDoc() throws Exception {
createIndex(); createTestIndex();
ensureGreen(); ensureGreen();
UpdateResponse updateResponse = client().prepareUpdate("test", "type1", "1") UpdateResponse updateResponse = client().prepareUpdate(indexOrAlias(), "type1", "1")
.setDoc(XContentFactory.jsonBuilder().startObject().field("bar", "baz").endObject()) .setDoc(XContentFactory.jsonBuilder().startObject().field("bar", "baz").endObject())
.setDocAsUpsert(true) .setDocAsUpsert(true)
.setFields("_source") .setFields("_source")
.execute().actionGet(); .execute().actionGet();
assertThat(updateResponse.getIndex(), equalTo("test"));
assertThat(updateResponse.getGetResult(), notNullValue()); assertThat(updateResponse.getGetResult(), notNullValue());
assertThat(updateResponse.getGetResult().getIndex(), equalTo("test"));
assertThat(updateResponse.getGetResult().sourceAsMap().get("bar").toString(), equalTo("baz")); assertThat(updateResponse.getGetResult().sourceAsMap().get("bar").toString(), equalTo("baz"));
} }
@Test @Test
// See: https://github.com/elasticsearch/elasticsearch/issues/3265 // See: https://github.com/elasticsearch/elasticsearch/issues/3265
public void testNotUpsertDoc() throws Exception { public void testNotUpsertDoc() throws Exception {
createIndex(); createTestIndex();
ensureGreen(); ensureGreen();
assertThrows(client().prepareUpdate("test", "type1", "1") assertThrows(client().prepareUpdate(indexOrAlias(), "type1", "1")
.setDoc(XContentFactory.jsonBuilder().startObject().field("bar", "baz").endObject()) .setDoc(XContentFactory.jsonBuilder().startObject().field("bar", "baz").endObject())
.setDocAsUpsert(false) .setDocAsUpsert(false)
.setFields("_source") .setFields("_source")
@ -180,47 +184,51 @@ public class UpdateTests extends ElasticsearchIntegrationTest {
@Test @Test
public void testUpsertFields() throws Exception { public void testUpsertFields() throws Exception {
createIndex(); createTestIndex();
ensureGreen(); ensureGreen();
UpdateResponse updateResponse = client().prepareUpdate("test", "type1", "1") UpdateResponse updateResponse = client().prepareUpdate(indexOrAlias(), "type1", "1")
.setUpsert(XContentFactory.jsonBuilder().startObject().field("bar", "baz").endObject()) .setUpsert(XContentFactory.jsonBuilder().startObject().field("bar", "baz").endObject())
.setScript("ctx._source.extra = \"foo\"", ScriptService.ScriptType.INLINE) .setScript("ctx._source.extra = \"foo\"", ScriptService.ScriptType.INLINE)
.setFields("_source") .setFields("_source")
.execute().actionGet(); .execute().actionGet();
assertThat(updateResponse.getIndex(), equalTo("test"));
assertThat(updateResponse.getGetResult(), notNullValue()); assertThat(updateResponse.getGetResult(), notNullValue());
assertThat(updateResponse.getGetResult().getIndex(), equalTo("test"));
assertThat(updateResponse.getGetResult().sourceAsMap().get("bar").toString(), equalTo("baz")); assertThat(updateResponse.getGetResult().sourceAsMap().get("bar").toString(), equalTo("baz"));
assertThat(updateResponse.getGetResult().sourceAsMap().get("extra"), nullValue()); assertThat(updateResponse.getGetResult().sourceAsMap().get("extra"), nullValue());
updateResponse = client().prepareUpdate("test", "type1", "1") updateResponse = client().prepareUpdate(indexOrAlias(), "type1", "1")
.setUpsert(XContentFactory.jsonBuilder().startObject().field("bar", "baz").endObject()) .setUpsert(XContentFactory.jsonBuilder().startObject().field("bar", "baz").endObject())
.setScript("ctx._source.extra = \"foo\"", ScriptService.ScriptType.INLINE) .setScript("ctx._source.extra = \"foo\"", ScriptService.ScriptType.INLINE)
.setFields("_source") .setFields("_source")
.execute().actionGet(); .execute().actionGet();
assertThat(updateResponse.getIndex(), equalTo("test"));
assertThat(updateResponse.getGetResult(), notNullValue()); assertThat(updateResponse.getGetResult(), notNullValue());
assertThat(updateResponse.getGetResult().getIndex(), equalTo("test"));
assertThat(updateResponse.getGetResult().sourceAsMap().get("bar").toString(), equalTo("baz")); assertThat(updateResponse.getGetResult().sourceAsMap().get("bar").toString(), equalTo("baz"));
assertThat(updateResponse.getGetResult().sourceAsMap().get("extra").toString(), equalTo("foo")); assertThat(updateResponse.getGetResult().sourceAsMap().get("extra").toString(), equalTo("foo"));
} }
@Test @Test
public void testVersionedUpdate() throws Exception { public void testVersionedUpdate() throws Exception {
createIndex("test"); assertAcked(prepareCreate("test").addAlias(new Alias("alias")));
ensureGreen(); ensureGreen();
index("test", "type", "1", "text", "value"); // version is now 1 index("test", "type", "1", "text", "value"); // version is now 1
assertThrows(client().prepareUpdate("test", "type", "1") assertThrows(client().prepareUpdate(indexOrAlias(), "type", "1")
.setScript("ctx._source.text = 'v2'", ScriptService.ScriptType.INLINE).setVersion(2).execute(), .setScript("ctx._source.text = 'v2'", ScriptService.ScriptType.INLINE).setVersion(2).execute(),
VersionConflictEngineException.class); VersionConflictEngineException.class);
client().prepareUpdate("test", "type", "1") client().prepareUpdate(indexOrAlias(), "type", "1")
.setScript("ctx._source.text = 'v2'", ScriptService.ScriptType.INLINE).setVersion(1).get(); .setScript("ctx._source.text = 'v2'", ScriptService.ScriptType.INLINE).setVersion(1).get();
assertThat(client().prepareGet("test", "type", "1").get().getVersion(), equalTo(2l)); assertThat(client().prepareGet("test", "type", "1").get().getVersion(), equalTo(2l));
// and again with a higher version.. // and again with a higher version..
client().prepareUpdate("test", "type", "1") client().prepareUpdate(indexOrAlias(), "type", "1")
.setScript("ctx._source.text = 'v3'", ScriptService.ScriptType.INLINE).setVersion(2).get(); .setScript("ctx._source.text = 'v3'", ScriptService.ScriptType.INLINE).setVersion(2).get();
assertThat(client().prepareGet("test", "type", "1").get().getVersion(), equalTo(3l)); assertThat(client().prepareGet("test", "type", "1").get().getVersion(), equalTo(3l));
@ -234,7 +242,7 @@ public class UpdateTests extends ElasticsearchIntegrationTest {
// external versioning // external versioning
client().prepareIndex("test", "type", "2").setSource("text", "value").setVersion(10).setVersionType(VersionType.EXTERNAL).get(); client().prepareIndex("test", "type", "2").setSource("text", "value").setVersion(10).setVersionType(VersionType.EXTERNAL).get();
assertThrows(client().prepareUpdate("test", "type", "2") assertThrows(client().prepareUpdate(indexOrAlias(), "type", "2")
.setScript("ctx._source.text = 'v2'", ScriptService.ScriptType.INLINE).setVersion(2) .setScript("ctx._source.text = 'v2'", ScriptService.ScriptType.INLINE).setVersion(2)
.setVersionType(VersionType.EXTERNAL).execute(), .setVersionType(VersionType.EXTERNAL).execute(),
ActionRequestValidationException.class); ActionRequestValidationException.class);
@ -242,14 +250,14 @@ public class UpdateTests extends ElasticsearchIntegrationTest {
// upserts - the combination with versions is a bit weird. Test are here to ensure we do not change our behavior unintentionally // upserts - the combination with versions is a bit weird. Test are here to ensure we do not change our behavior unintentionally
// With internal versions, tt means "if object is there with version X, update it or explode. If it is not there, index. // With internal versions, tt means "if object is there with version X, update it or explode. If it is not there, index.
client().prepareUpdate("test", "type", "3").setScript("ctx._source.text = 'v2'", ScriptService.ScriptType.INLINE) client().prepareUpdate(indexOrAlias(), "type", "3").setScript("ctx._source.text = 'v2'", ScriptService.ScriptType.INLINE)
.setVersion(10).setUpsert("{ \"text\": \"v0\" }").get(); .setVersion(10).setUpsert("{ \"text\": \"v0\" }").get();
GetResponse get = get("test", "type", "3"); GetResponse get = get("test", "type", "3");
assertThat(get.getVersion(), equalTo(1l)); assertThat(get.getVersion(), equalTo(1l));
assertThat((String) get.getSource().get("text"), equalTo("v0")); assertThat((String) get.getSource().get("text"), equalTo("v0"));
// With force version // With force version
client().prepareUpdate("test", "type", "4").setScript("ctx._source.text = 'v2'", ScriptService.ScriptType.INLINE) client().prepareUpdate(indexOrAlias(), "type", "4").setScript("ctx._source.text = 'v2'", ScriptService.ScriptType.INLINE)
.setVersion(10).setVersionType(VersionType.FORCE).setUpsert("{ \"text\": \"v0\" }").get(); .setVersion(10).setVersionType(VersionType.FORCE).setUpsert("{ \"text\": \"v0\" }").get();
get = get("test", "type", "4"); get = get("test", "type", "4");
@ -258,9 +266,7 @@ public class UpdateTests extends ElasticsearchIntegrationTest {
// retry on conflict is rejected: // retry on conflict is rejected:
assertThrows(client().prepareUpdate(indexOrAlias(), "type", "1").setVersion(10).setRetryOnConflict(5), ActionRequestValidationException.class);
assertThrows(client().prepareUpdate("test", "type", "1").setVersion(10).setRetryOnConflict(5), ActionRequestValidationException.class);
} }
@Test @Test
@ -271,18 +277,20 @@ public class UpdateTests extends ElasticsearchIntegrationTest {
.setFields("_source") .setFields("_source")
.execute().actionGet(); .execute().actionGet();
assertThat(updateResponse.getIndex(), equalTo("test"));
assertThat(updateResponse.getGetResult(), notNullValue()); assertThat(updateResponse.getGetResult(), notNullValue());
assertThat(updateResponse.getGetResult().getIndex(), equalTo("test"));
assertThat(updateResponse.getGetResult().sourceAsMap().get("bar").toString(), equalTo("baz")); assertThat(updateResponse.getGetResult().sourceAsMap().get("bar").toString(), equalTo("baz"));
assertThat(updateResponse.getGetResult().sourceAsMap().get("extra"), nullValue()); assertThat(updateResponse.getGetResult().sourceAsMap().get("extra"), nullValue());
} }
@Test @Test
public void testUpdate() throws Exception { public void testUpdate() throws Exception {
createIndex(); createTestIndex();
ensureGreen(); ensureGreen();
try { try {
client().prepareUpdate("test", "type1", "1") client().prepareUpdate(indexOrAlias(), "type1", "1")
.setScript("ctx._source.field++", ScriptService.ScriptType.INLINE).execute().actionGet(); .setScript("ctx._source.field++", ScriptService.ScriptType.INLINE).execute().actionGet();
fail(); fail();
} catch (DocumentMissingException e) { } catch (DocumentMissingException e) {
@ -291,21 +299,23 @@ public class UpdateTests extends ElasticsearchIntegrationTest {
client().prepareIndex("test", "type1", "1").setSource("field", 1).execute().actionGet(); client().prepareIndex("test", "type1", "1").setSource("field", 1).execute().actionGet();
UpdateResponse updateResponse = client().prepareUpdate("test", "type1", "1") UpdateResponse updateResponse = client().prepareUpdate(indexOrAlias(), "type1", "1")
.setScript("ctx._source.field += 1", ScriptService.ScriptType.INLINE).execute().actionGet(); .setScript("ctx._source.field += 1", ScriptService.ScriptType.INLINE).execute().actionGet();
assertThat(updateResponse.getVersion(), equalTo(2L)); assertThat(updateResponse.getVersion(), equalTo(2L));
assertFalse(updateResponse.isCreated()); assertFalse(updateResponse.isCreated());
assertThat(updateResponse.getIndex(), equalTo("test"));
for (int i = 0; i < 5; i++) { for (int i = 0; i < 5; i++) {
GetResponse getResponse = client().prepareGet("test", "type1", "1").execute().actionGet(); GetResponse getResponse = client().prepareGet("test", "type1", "1").execute().actionGet();
assertThat(getResponse.getSourceAsMap().get("field").toString(), equalTo("2")); assertThat(getResponse.getSourceAsMap().get("field").toString(), equalTo("2"));
} }
updateResponse = client().prepareUpdate("test", "type1", "1") updateResponse = client().prepareUpdate(indexOrAlias(), "type1", "1")
.setScript("ctx._source.field += count", ScriptService.ScriptType.INLINE) .setScript("ctx._source.field += count", ScriptService.ScriptType.INLINE)
.addScriptParam("count", 3).execute().actionGet(); .addScriptParam("count", 3).execute().actionGet();
assertThat(updateResponse.getVersion(), equalTo(3L)); assertThat(updateResponse.getVersion(), equalTo(3L));
assertFalse(updateResponse.isCreated()); assertFalse(updateResponse.isCreated());
assertThat(updateResponse.getIndex(), equalTo("test"));
for (int i = 0; i < 5; i++) { for (int i = 0; i < 5; i++) {
GetResponse getResponse = client().prepareGet("test", "type1", "1").execute().actionGet(); GetResponse getResponse = client().prepareGet("test", "type1", "1").execute().actionGet();
@ -313,10 +323,11 @@ public class UpdateTests extends ElasticsearchIntegrationTest {
} }
// check noop // check noop
updateResponse = client().prepareUpdate("test", "type1", "1") updateResponse = client().prepareUpdate(indexOrAlias(), "type1", "1")
.setScript("ctx.op = 'none'", ScriptService.ScriptType.INLINE).execute().actionGet(); .setScript("ctx.op = 'none'", ScriptService.ScriptType.INLINE).execute().actionGet();
assertThat(updateResponse.getVersion(), equalTo(3L)); assertThat(updateResponse.getVersion(), equalTo(3L));
assertFalse(updateResponse.isCreated()); assertFalse(updateResponse.isCreated());
assertThat(updateResponse.getIndex(), equalTo("test"));
for (int i = 0; i < 5; i++) { for (int i = 0; i < 5; i++) {
GetResponse getResponse = client().prepareGet("test", "type1", "1").execute().actionGet(); GetResponse getResponse = client().prepareGet("test", "type1", "1").execute().actionGet();
@ -324,10 +335,11 @@ public class UpdateTests extends ElasticsearchIntegrationTest {
} }
// check delete // check delete
updateResponse = client().prepareUpdate("test", "type1", "1") updateResponse = client().prepareUpdate(indexOrAlias(), "type1", "1")
.setScript("ctx.op = 'delete'", ScriptService.ScriptType.INLINE).execute().actionGet(); .setScript("ctx.op = 'delete'", ScriptService.ScriptType.INLINE).execute().actionGet();
assertThat(updateResponse.getVersion(), equalTo(4L)); assertThat(updateResponse.getVersion(), equalTo(4L));
assertFalse(updateResponse.isCreated()); assertFalse(updateResponse.isCreated());
assertThat(updateResponse.getIndex(), equalTo("test"));
for (int i = 0; i < 5; i++) { for (int i = 0; i < 5; i++) {
GetResponse getResponse = client().prepareGet("test", "type1", "1").execute().actionGet(); GetResponse getResponse = client().prepareGet("test", "type1", "1").execute().actionGet();
@ -339,13 +351,13 @@ public class UpdateTests extends ElasticsearchIntegrationTest {
GetResponse getResponse = client().prepareGet("test", "type1", "2").setFields("_ttl").execute().actionGet(); GetResponse getResponse = client().prepareGet("test", "type1", "2").setFields("_ttl").execute().actionGet();
long ttl = ((Number) getResponse.getField("_ttl").getValue()).longValue(); long ttl = ((Number) getResponse.getField("_ttl").getValue()).longValue();
assertThat(ttl, greaterThan(0L)); assertThat(ttl, greaterThan(0L));
client().prepareUpdate("test", "type1", "2").setScript("ctx._source.field += 1", ScriptService.ScriptType.INLINE).execute().actionGet(); client().prepareUpdate(indexOrAlias(), "type1", "2").setScript("ctx._source.field += 1", ScriptService.ScriptType.INLINE).execute().actionGet();
getResponse = client().prepareGet("test", "type1", "2").setFields("_ttl").execute().actionGet(); getResponse = client().prepareGet("test", "type1", "2").setFields("_ttl").execute().actionGet();
ttl = ((Number) getResponse.getField("_ttl").getValue()).longValue(); ttl = ((Number) getResponse.getField("_ttl").getValue()).longValue();
assertThat(ttl, greaterThan(0L)); assertThat(ttl, greaterThan(0L));
// check TTL update // check TTL update
client().prepareUpdate("test", "type1", "2").setScript("ctx._ttl = 3600000", ScriptService.ScriptType.INLINE).execute().actionGet(); client().prepareUpdate(indexOrAlias(), "type1", "2").setScript("ctx._ttl = 3600000", ScriptService.ScriptType.INLINE).execute().actionGet();
getResponse = client().prepareGet("test", "type1", "2").setFields("_ttl").execute().actionGet(); getResponse = client().prepareGet("test", "type1", "2").setFields("_ttl").execute().actionGet();
ttl = ((Number) getResponse.getField("_ttl").getValue()).longValue(); ttl = ((Number) getResponse.getField("_ttl").getValue()).longValue();
assertThat(ttl, greaterThan(0L)); assertThat(ttl, greaterThan(0L));
@ -353,7 +365,7 @@ public class UpdateTests extends ElasticsearchIntegrationTest {
// check timestamp update // check timestamp update
client().prepareIndex("test", "type1", "3").setSource("field", 1).setRefresh(true).execute().actionGet(); client().prepareIndex("test", "type1", "3").setSource("field", 1).setRefresh(true).execute().actionGet();
client().prepareUpdate("test", "type1", "3") client().prepareUpdate(indexOrAlias(), "type1", "3")
.setScript("ctx._timestamp = \"2009-11-15T14:12:12\"", ScriptService.ScriptType.INLINE).execute().actionGet(); .setScript("ctx._timestamp = \"2009-11-15T14:12:12\"", ScriptService.ScriptType.INLINE).execute().actionGet();
getResponse = client().prepareGet("test", "type1", "3").setFields("_timestamp").execute().actionGet(); getResponse = client().prepareGet("test", "type1", "3").setFields("_timestamp").execute().actionGet();
long timestamp = ((Number) getResponse.getField("_timestamp").getValue()).longValue(); long timestamp = ((Number) getResponse.getField("_timestamp").getValue()).longValue();
@ -361,16 +373,18 @@ public class UpdateTests extends ElasticsearchIntegrationTest {
// check fields parameter // check fields parameter
client().prepareIndex("test", "type1", "1").setSource("field", 1).execute().actionGet(); client().prepareIndex("test", "type1", "1").setSource("field", 1).execute().actionGet();
updateResponse = client().prepareUpdate("test", "type1", "1") updateResponse = client().prepareUpdate(indexOrAlias(), "type1", "1")
.setScript("ctx._source.field += 1", ScriptService.ScriptType.INLINE).setFields("_source", "field").execute().actionGet(); .setScript("ctx._source.field += 1", ScriptService.ScriptType.INLINE).setFields("_source", "field").execute().actionGet();
assertThat(updateResponse.getIndex(), equalTo("test"));
assertThat(updateResponse.getGetResult(), notNullValue()); assertThat(updateResponse.getGetResult(), notNullValue());
assertThat(updateResponse.getGetResult().getIndex(), equalTo("test"));
assertThat(updateResponse.getGetResult().sourceRef(), notNullValue()); assertThat(updateResponse.getGetResult().sourceRef(), notNullValue());
assertThat(updateResponse.getGetResult().field("field").getValue(), notNullValue()); assertThat(updateResponse.getGetResult().field("field").getValue(), notNullValue());
// check updates without script // check updates without script
// add new field // add new field
client().prepareIndex("test", "type1", "1").setSource("field", 1).execute().actionGet(); client().prepareIndex("test", "type1", "1").setSource("field", 1).execute().actionGet();
updateResponse = client().prepareUpdate("test", "type1", "1").setDoc(XContentFactory.jsonBuilder().startObject().field("field2", 2).endObject()).execute().actionGet(); updateResponse = client().prepareUpdate(indexOrAlias(), "type1", "1").setDoc(XContentFactory.jsonBuilder().startObject().field("field2", 2).endObject()).execute().actionGet();
for (int i = 0; i < 5; i++) { for (int i = 0; i < 5; i++) {
getResponse = client().prepareGet("test", "type1", "1").execute().actionGet(); getResponse = client().prepareGet("test", "type1", "1").execute().actionGet();
assertThat(getResponse.getSourceAsMap().get("field").toString(), equalTo("1")); assertThat(getResponse.getSourceAsMap().get("field").toString(), equalTo("1"));
@ -378,7 +392,7 @@ public class UpdateTests extends ElasticsearchIntegrationTest {
} }
// change existing field // change existing field
updateResponse = client().prepareUpdate("test", "type1", "1").setDoc(XContentFactory.jsonBuilder().startObject().field("field", 3).endObject()).execute().actionGet(); updateResponse = client().prepareUpdate(indexOrAlias(), "type1", "1").setDoc(XContentFactory.jsonBuilder().startObject().field("field", 3).endObject()).execute().actionGet();
for (int i = 0; i < 5; i++) { for (int i = 0; i < 5; i++) {
getResponse = client().prepareGet("test", "type1", "1").execute().actionGet(); getResponse = client().prepareGet("test", "type1", "1").execute().actionGet();
assertThat(getResponse.getSourceAsMap().get("field").toString(), equalTo("3")); assertThat(getResponse.getSourceAsMap().get("field").toString(), equalTo("3"));
@ -396,7 +410,7 @@ public class UpdateTests extends ElasticsearchIntegrationTest {
testMap.put("map1", 8); testMap.put("map1", 8);
client().prepareIndex("test", "type1", "1").setSource("map", testMap).execute().actionGet(); client().prepareIndex("test", "type1", "1").setSource("map", testMap).execute().actionGet();
updateResponse = client().prepareUpdate("test", "type1", "1").setDoc(XContentFactory.jsonBuilder().startObject().field("map", testMap3).endObject()).execute().actionGet(); updateResponse = client().prepareUpdate(indexOrAlias(), "type1", "1").setDoc(XContentFactory.jsonBuilder().startObject().field("map", testMap3).endObject()).execute().actionGet();
for (int i = 0; i < 5; i++) { for (int i = 0; i < 5; i++) {
getResponse = client().prepareGet("test", "type1", "1").execute().actionGet(); getResponse = client().prepareGet("test", "type1", "1").execute().actionGet();
Map map1 = (Map) getResponse.getSourceAsMap().get("map"); Map map1 = (Map) getResponse.getSourceAsMap().get("map");
@ -414,11 +428,11 @@ public class UpdateTests extends ElasticsearchIntegrationTest {
@Test @Test
public void testUpdateRequestWithBothScriptAndDoc() throws Exception { public void testUpdateRequestWithBothScriptAndDoc() throws Exception {
createIndex(); createTestIndex();
ensureGreen(); ensureGreen();
try { try {
client().prepareUpdate("test", "type1", "1") client().prepareUpdate(indexOrAlias(), "type1", "1")
.setDoc(XContentFactory.jsonBuilder().startObject().field("field", 1).endObject()) .setDoc(XContentFactory.jsonBuilder().startObject().field("field", 1).endObject())
.setScript("ctx._source.field += 1", ScriptService.ScriptType.INLINE) .setScript("ctx._source.field += 1", ScriptService.ScriptType.INLINE)
.execute().actionGet(); .execute().actionGet();
@ -432,10 +446,10 @@ public class UpdateTests extends ElasticsearchIntegrationTest {
@Test @Test
public void testUpdateRequestWithScriptAndShouldUpsertDoc() throws Exception { public void testUpdateRequestWithScriptAndShouldUpsertDoc() throws Exception {
createIndex(); createTestIndex();
ensureGreen(); ensureGreen();
try { try {
client().prepareUpdate("test", "type1", "1") client().prepareUpdate(indexOrAlias(), "type1", "1")
.setScript("ctx._source.field += 1", ScriptService.ScriptType.INLINE) .setScript("ctx._source.field += 1", ScriptService.ScriptType.INLINE)
.setDocAsUpsert(true) .setDocAsUpsert(true)
.execute().actionGet(); .execute().actionGet();
@ -451,7 +465,7 @@ public class UpdateTests extends ElasticsearchIntegrationTest {
@Slow @Slow
public void testConcurrentUpdateWithRetryOnConflict() throws Exception { public void testConcurrentUpdateWithRetryOnConflict() throws Exception {
final boolean useBulkApi = randomBoolean(); final boolean useBulkApi = randomBoolean();
createIndex(); createTestIndex();
ensureGreen(); ensureGreen();
int numberOfThreads = scaledRandomIntBetween(2,5); int numberOfThreads = scaledRandomIntBetween(2,5);
@ -468,13 +482,13 @@ public class UpdateTests extends ElasticsearchIntegrationTest {
startLatch.await(); startLatch.await();
for (int i = 0; i < numberOfUpdatesPerThread; i++) { for (int i = 0; i < numberOfUpdatesPerThread; i++) {
if (useBulkApi) { if (useBulkApi) {
UpdateRequestBuilder updateRequestBuilder = client().prepareUpdate("test", "type1", Integer.toString(i)) UpdateRequestBuilder updateRequestBuilder = client().prepareUpdate(indexOrAlias(), "type1", Integer.toString(i))
.setScript("ctx._source.field += 1", ScriptService.ScriptType.INLINE) .setScript("ctx._source.field += 1", ScriptService.ScriptType.INLINE)
.setRetryOnConflict(Integer.MAX_VALUE) .setRetryOnConflict(Integer.MAX_VALUE)
.setUpsert(jsonBuilder().startObject().field("field", 1).endObject()); .setUpsert(jsonBuilder().startObject().field("field", 1).endObject());
client().prepareBulk().add(updateRequestBuilder).execute().actionGet(); client().prepareBulk().add(updateRequestBuilder).execute().actionGet();
} else { } else {
client().prepareUpdate("test", "type1", Integer.toString(i)) client().prepareUpdate(indexOrAlias(), "type1", Integer.toString(i))
.setScript("ctx._source.field += 1", ScriptService.ScriptType.INLINE) .setScript("ctx._source.field += 1", ScriptService.ScriptType.INLINE)
.setRetryOnConflict(Integer.MAX_VALUE) .setRetryOnConflict(Integer.MAX_VALUE)
.setUpsert(jsonBuilder().startObject().field("field", 1).endObject()) .setUpsert(jsonBuilder().startObject().field("field", 1).endObject())
@ -528,7 +542,7 @@ public class UpdateTests extends ElasticsearchIntegrationTest {
@Slow @Slow
public void stressUpdateDeleteConcurrency() throws Exception { public void stressUpdateDeleteConcurrency() throws Exception {
//We create an index with merging disabled so that deletes don't get merged away //We create an index with merging disabled so that deletes don't get merged away
client().admin().indices().prepareCreate("test") assertAcked(prepareCreate("test")
.addMapping("type1", XContentFactory.jsonBuilder() .addMapping("type1", XContentFactory.jsonBuilder()
.startObject() .startObject()
.startObject("type1") .startObject("type1")
@ -536,8 +550,7 @@ public class UpdateTests extends ElasticsearchIntegrationTest {
.startObject("_ttl").field("enabled", true).field("store", "yes").endObject() .startObject("_ttl").field("enabled", true).field("store", "yes").endObject()
.endObject() .endObject()
.endObject()) .endObject())
.setSettings(ImmutableSettings.builder().put(MergePolicyModule.MERGE_POLICY_TYPE_KEY, NoMergePolicyProvider.class)) .setSettings(ImmutableSettings.builder().put(MergePolicyModule.MERGE_POLICY_TYPE_KEY, NoMergePolicyProvider.class)));
.execute().actionGet();
ensureGreen(); ensureGreen();
final int numberOfThreads = scaledRandomIntBetween(3,5); final int numberOfThreads = scaledRandomIntBetween(3,5);
@ -748,4 +761,7 @@ public class UpdateTests extends ElasticsearchIntegrationTest {
} }
} }
private static String indexOrAlias() {
return randomBoolean() ? "test" : "alias";
}
} }