create mapping operation should wait for the mappings to be created on all the nodes before it returns (with a new timeout value)

This commit is contained in:
kimchy 2010-02-18 02:10:15 +02:00
parent 778156787a
commit f135a93b70
7 changed files with 219 additions and 12 deletions

View File

@ -22,13 +22,16 @@ package org.elasticsearch.action.admin.indices.mapping.create;
import org.elasticsearch.action.ActionRequest;
import org.elasticsearch.action.ActionRequestValidationException;
import org.elasticsearch.util.Required;
import org.elasticsearch.util.TimeValue;
import org.elasticsearch.util.io.Streamable;
import java.io.DataInput;
import java.io.DataOutput;
import java.io.IOException;
import java.util.concurrent.TimeUnit;
import static org.elasticsearch.action.Actions.*;
import static org.elasticsearch.util.TimeValue.*;
/**
* @author kimchy (Shay Banon)
@ -41,6 +44,8 @@ public class CreateMappingRequest implements ActionRequest, Streamable {
private String mappingSource;
private TimeValue timeout = new TimeValue(10, TimeUnit.SECONDS);
CreateMappingRequest() {
}
@ -79,7 +84,7 @@ public class CreateMappingRequest implements ActionRequest, Streamable {
return indices;
}
String mappingType() {
String type() {
return mappingType;
}
@ -87,7 +92,7 @@ public class CreateMappingRequest implements ActionRequest, Streamable {
* The type of the mappings. Not required since it can be defined explicitly within the mapping source.
* If it is not defined within the mapping source, then it is required.
*/
public CreateMappingRequest mappingType(String mappingType) {
public CreateMappingRequest type(String mappingType) {
this.mappingType = mappingType;
return this;
}
@ -101,6 +106,15 @@ public class CreateMappingRequest implements ActionRequest, Streamable {
return this;
}
TimeValue timeout() {
return timeout;
}
public CreateMappingRequest timeout(TimeValue timeout) {
this.timeout = timeout;
return this;
}
@Override public void readFrom(DataInput in) throws IOException, ClassNotFoundException {
indices = new String[in.readInt()];
for (int i = 0; i < indices.length; i++) {
@ -110,6 +124,7 @@ public class CreateMappingRequest implements ActionRequest, Streamable {
mappingType = in.readUTF();
}
mappingSource = in.readUTF();
timeout = readTimeValue(in);
}
@Override public void writeTo(DataOutput out) throws IOException {
@ -128,5 +143,6 @@ public class CreateMappingRequest implements ActionRequest, Streamable {
out.writeUTF(mappingType);
}
out.writeUTF(mappingSource);
timeout.writeTo(out);
}
}

View File

@ -103,7 +103,7 @@ public class TransportCreateMappingAction extends BaseAction<CreateMappingReques
threadPool.execute(new Runnable() {
@Override public void run() {
try {
metaDataService.addMapping(indices, request.mappingType(), request.mappingSource());
metaDataService.addMapping(indices, request.type(), request.mappingSource(), request.timeout());
listener.onResponse(new CreateMappingResponse());
} catch (Exception e) {
listener.onFailure(e);
@ -135,7 +135,7 @@ public class TransportCreateMappingAction extends BaseAction<CreateMappingReques
String[] indices = Actions.processIndices(clusterService.state(), request.indices());
if (clusterService.state().nodes().localNodeMaster()) {
// handle the actual creation of a new index
metaDataService.addMapping(indices, request.mappingType(), request.mappingSource());
metaDataService.addMapping(indices, request.type(), request.mappingSource(), request.timeout());
channel.sendResponse(VoidStreamable.INSTANCE);
} else {
transportService.sendRequest(clusterService.state().nodes().masterNode(), TransportActions.Admin.Indices.Mapping.CREATE, request, new VoidTransportResponseHandler() {

View File

@ -22,6 +22,7 @@ package org.elasticsearch.cluster;
import com.google.inject.AbstractModule;
import org.elasticsearch.cluster.action.index.NodeIndexCreatedAction;
import org.elasticsearch.cluster.action.index.NodeIndexDeletedAction;
import org.elasticsearch.cluster.action.index.NodeMappingCreatedAction;
import org.elasticsearch.cluster.action.shard.ShardStateAction;
import org.elasticsearch.cluster.metadata.MetaDataService;
import org.elasticsearch.cluster.routing.RoutingService;
@ -53,5 +54,6 @@ public class ClusterModule extends AbstractModule {
bind(ShardStateAction.class).asEagerSingleton();
bind(NodeIndexCreatedAction.class).asEagerSingleton();
bind(NodeIndexDeletedAction.class).asEagerSingleton();
bind(NodeMappingCreatedAction.class).asEagerSingleton();
}
}

View File

@ -0,0 +1,151 @@
/*
* Licensed to Elastic Search and Shay Banon under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. Elastic Search licenses this
* file to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing,
* software distributed under the License is distributed on an
* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
* KIND, either express or implied. See the License for the
* specific language governing permissions and limitations
* under the License.
*/
package org.elasticsearch.cluster.action.index;
import com.google.inject.Inject;
import org.elasticsearch.ElasticSearchException;
import org.elasticsearch.cluster.ClusterService;
import org.elasticsearch.cluster.node.Nodes;
import org.elasticsearch.threadpool.ThreadPool;
import org.elasticsearch.transport.BaseTransportRequestHandler;
import org.elasticsearch.transport.TransportChannel;
import org.elasticsearch.transport.TransportService;
import org.elasticsearch.transport.VoidTransportResponseHandler;
import org.elasticsearch.util.component.AbstractComponent;
import org.elasticsearch.util.io.Streamable;
import org.elasticsearch.util.io.VoidStreamable;
import org.elasticsearch.util.settings.Settings;
import java.io.DataInput;
import java.io.DataOutput;
import java.io.IOException;
import java.util.List;
import java.util.concurrent.CopyOnWriteArrayList;
/**
* @author kimchy (Shay Banon)
*/
public class NodeMappingCreatedAction extends AbstractComponent {
private final ThreadPool threadPool;
private final TransportService transportService;
private final ClusterService clusterService;
private final List<Listener> listeners = new CopyOnWriteArrayList<Listener>();
@Inject public NodeMappingCreatedAction(Settings settings, ThreadPool threadPool, TransportService transportService, ClusterService clusterService) {
super(settings);
this.threadPool = threadPool;
this.transportService = transportService;
this.clusterService = clusterService;
transportService.registerHandler(NodeMappingCreatedTransportHandler.ACTION, new NodeMappingCreatedTransportHandler());
}
public void add(Listener listener) {
listeners.add(listener);
}
public void remove(Listener listener) {
listeners.remove(listener);
}
public void nodeMappingCreated(final NodeMappingCreatedResponse response) throws ElasticSearchException {
Nodes nodes = clusterService.state().nodes();
if (nodes.localNodeMaster()) {
threadPool.execute(new Runnable() {
@Override public void run() {
innerNodeIndexCreated(response);
}
});
} else {
transportService.sendRequest(clusterService.state().nodes().masterNode(),
NodeMappingCreatedTransportHandler.ACTION, response, VoidTransportResponseHandler.INSTANCE);
}
}
private void innerNodeIndexCreated(NodeMappingCreatedResponse response) {
for (Listener listener : listeners) {
listener.onNodeMappingCreated(response);
}
}
public static interface Listener {
void onNodeMappingCreated(NodeMappingCreatedResponse response);
}
private class NodeMappingCreatedTransportHandler extends BaseTransportRequestHandler<NodeMappingCreatedResponse> {
static final String ACTION = "cluster/nodeMappingCreated";
@Override public NodeMappingCreatedResponse newInstance() {
return new NodeMappingCreatedResponse();
}
@Override public void messageReceived(NodeMappingCreatedResponse response, TransportChannel channel) throws Exception {
innerNodeIndexCreated(response);
channel.sendResponse(VoidStreamable.INSTANCE);
}
}
public static class NodeMappingCreatedResponse implements Streamable {
private String index;
private String type;
private String nodeId;
private NodeMappingCreatedResponse() {
}
public NodeMappingCreatedResponse(String index, String type, String nodeId) {
this.index = index;
this.type = type;
this.nodeId = nodeId;
}
public String index() {
return index;
}
public String type() {
return type;
}
public String nodeId() {
return nodeId;
}
@Override public void writeTo(DataOutput out) throws IOException {
out.writeUTF(index);
out.writeUTF(type);
out.writeUTF(nodeId);
}
@Override public void readFrom(DataInput in) throws IOException, ClassNotFoundException {
index = in.readUTF();
type = in.readUTF();
nodeId = in.readUTF();
}
}
}

View File

@ -19,6 +19,7 @@
package org.elasticsearch.cluster.metadata;
import com.google.common.collect.Sets;
import com.google.inject.Inject;
import org.elasticsearch.ElasticSearchException;
import org.elasticsearch.cluster.ClusterService;
@ -26,6 +27,7 @@ import org.elasticsearch.cluster.ClusterState;
import org.elasticsearch.cluster.ClusterStateUpdateTask;
import org.elasticsearch.cluster.action.index.NodeIndexCreatedAction;
import org.elasticsearch.cluster.action.index.NodeIndexDeletedAction;
import org.elasticsearch.cluster.action.index.NodeMappingCreatedAction;
import org.elasticsearch.cluster.routing.IndexRoutingTable;
import org.elasticsearch.cluster.routing.RoutingTable;
import org.elasticsearch.cluster.routing.strategy.ShardsRoutingStrategy;
@ -44,6 +46,7 @@ import org.elasticsearch.util.settings.ImmutableSettings;
import org.elasticsearch.util.settings.Settings;
import java.util.Arrays;
import java.util.Set;
import java.util.concurrent.CountDownLatch;
import java.util.concurrent.TimeUnit;
@ -67,14 +70,18 @@ public class MetaDataService extends AbstractComponent {
private final NodeIndexDeletedAction nodeIndexDeletedAction;
private final NodeMappingCreatedAction nodeMappingCreatedAction;
@Inject public MetaDataService(Settings settings, ClusterService clusterService, IndicesService indicesService, ShardsRoutingStrategy shardsRoutingStrategy,
NodeIndexCreatedAction nodeIndexCreatedAction, NodeIndexDeletedAction nodeIndexDeletedAction) {
NodeIndexCreatedAction nodeIndexCreatedAction, NodeIndexDeletedAction nodeIndexDeletedAction,
NodeMappingCreatedAction nodeMappingCreatedAction) {
super(settings);
this.clusterService = clusterService;
this.indicesService = indicesService;
this.shardsRoutingStrategy = shardsRoutingStrategy;
this.nodeIndexCreatedAction = nodeIndexCreatedAction;
this.nodeIndexDeletedAction = nodeIndexDeletedAction;
this.nodeMappingCreatedAction = nodeMappingCreatedAction;
}
public synchronized boolean createIndex(final String index, final Settings indexSettings, TimeValue timeout) throws IndexAlreadyExistsException {
@ -193,7 +200,7 @@ public class MetaDataService extends AbstractComponent {
}
}
public void addMapping(final String[] indices, String mappingType, final String mappingSource) throws ElasticSearchException {
public boolean addMapping(final String[] indices, String mappingType, final String mappingSource, TimeValue timeout) throws ElasticSearchException {
ClusterState clusterState = clusterService.state();
for (String index : indices) {
IndexRoutingTable indexTable = clusterState.routingTable().indicesRouting().get(index);
@ -224,6 +231,18 @@ public class MetaDataService extends AbstractComponent {
logger.info("Indices [" + Arrays.toString(indices) + "]: Creating mapping [" + mappingType + "] with source [" + mappingSource + "]");
final CountDownLatch latch = new CountDownLatch(clusterService.state().nodes().size() * indices.length);
final Set<String> indicesSet = Sets.newHashSet(indices);
final String fMappingType = mappingType;
NodeMappingCreatedAction.Listener listener = new NodeMappingCreatedAction.Listener() {
@Override public void onNodeMappingCreated(NodeMappingCreatedAction.NodeMappingCreatedResponse response) {
if (indicesSet.contains(response.index()) && response.type().equals(fMappingType)) {
latch.countDown();
}
}
};
nodeMappingCreatedAction.add(listener);
final String mappingTypeP = mappingType;
clusterService.submitStateUpdateTask("create-mapping [" + mappingTypeP + "]", new ClusterStateUpdateTask() {
@Override public ClusterState execute(ClusterState currentState) {
@ -238,6 +257,14 @@ public class MetaDataService extends AbstractComponent {
return newClusterStateBuilder().state(currentState).metaData(builder).build();
}
});
try {
return latch.await(timeout.millis(), TimeUnit.MILLISECONDS);
} catch (InterruptedException e) {
return false;
} finally {
nodeMappingCreatedAction.remove(listener);
}
}
}

View File

@ -27,6 +27,7 @@ import org.elasticsearch.cluster.ClusterService;
import org.elasticsearch.cluster.ClusterStateListener;
import org.elasticsearch.cluster.action.index.NodeIndexCreatedAction;
import org.elasticsearch.cluster.action.index.NodeIndexDeletedAction;
import org.elasticsearch.cluster.action.index.NodeMappingCreatedAction;
import org.elasticsearch.cluster.action.shard.ShardStateAction;
import org.elasticsearch.cluster.metadata.IndexMetaData;
import org.elasticsearch.cluster.metadata.MetaData;
@ -78,9 +79,12 @@ public class IndicesClusterStateService extends AbstractComponent implements Clu
private final NodeIndexDeletedAction nodeIndexDeletedAction;
private final NodeMappingCreatedAction nodeMappingCreatedAction;
@Inject public IndicesClusterStateService(Settings settings, IndicesService indicesService, ClusterService clusterService,
ThreadPool threadPool, ShardStateAction shardStateAction,
NodeIndexCreatedAction nodeIndexCreatedAction, NodeIndexDeletedAction nodeIndexDeletedAction) {
NodeIndexCreatedAction nodeIndexCreatedAction, NodeIndexDeletedAction nodeIndexDeletedAction,
NodeMappingCreatedAction nodeMappingCreatedAction) {
super(settings);
this.indicesService = indicesService;
this.clusterService = clusterService;
@ -88,6 +92,7 @@ public class IndicesClusterStateService extends AbstractComponent implements Clu
this.shardStateAction = shardStateAction;
this.nodeIndexCreatedAction = nodeIndexCreatedAction;
this.nodeIndexDeletedAction = nodeIndexDeletedAction;
this.nodeMappingCreatedAction = nodeMappingCreatedAction;
}
@Override public Lifecycle.State lifecycleState() {
@ -167,6 +172,7 @@ public class IndicesClusterStateService extends AbstractComponent implements Clu
logger.debug("Index [" + index + "] Adding mapping [" + mappingType + "], source [" + mappingSource + "]");
}
mapperService.add(mappingType, mappingSource);
nodeMappingCreatedAction.nodeMappingCreated(new NodeMappingCreatedAction.NodeMappingCreatedResponse(index, mappingType, event.state().nodes().localNodeId()));
} else {
DocumentMapper existingMapper = mapperService.documentMapper(mappingType);
if (!mappingSource.equals(existingMapper.mappingSource())) {
@ -175,6 +181,7 @@ public class IndicesClusterStateService extends AbstractComponent implements Clu
logger.debug("Index [" + index + "] Updating mapping [" + mappingType + "], source [" + mappingSource + "]");
}
mapperService.add(mappingType, mappingSource);
nodeMappingCreatedAction.nodeMappingCreated(new NodeMappingCreatedAction.NodeMappingCreatedResponse(index, mappingType, event.state().nodes().localNodeId()));
}
}
} catch (Exception e) {

View File

@ -27,16 +27,19 @@ import org.elasticsearch.client.Client;
import org.elasticsearch.index.mapper.InvalidTypeNameException;
import org.elasticsearch.indices.IndexMissingException;
import org.elasticsearch.rest.*;
import org.elasticsearch.rest.action.support.RestActions;
import org.elasticsearch.rest.action.support.RestJsonBuilder;
import org.elasticsearch.util.TimeValue;
import org.elasticsearch.util.json.JsonBuilder;
import org.elasticsearch.util.settings.Settings;
import java.io.IOException;
import static org.elasticsearch.ExceptionsHelper.*;
import static org.elasticsearch.client.Requests.*;
import static org.elasticsearch.rest.RestRequest.Method.*;
import static org.elasticsearch.rest.RestResponse.Status.*;
import static org.elasticsearch.rest.action.support.RestActions.*;
import static org.elasticsearch.util.TimeValue.*;
/**
* @author kimchy (Shay Banon)
@ -50,10 +53,11 @@ public class RestCreateMappingAction extends BaseRestHandler {
}
@Override public void handleRequest(final RestRequest request, final RestChannel channel) {
String[] indices = RestActions.splitIndices(request.param("index"));
String mappingType = request.param("type");
String mappingSource = request.contentAsString();
client.admin().indices().execCreateMapping(new CreateMappingRequest(indices, mappingType, mappingSource), new ActionListener<CreateMappingResponse>() {
CreateMappingRequest createMappingRequest = createMappingRequest(splitIndices(request.param("index")));
createMappingRequest.type(request.param("type"));
createMappingRequest.mappingSource(request.contentAsString());
createMappingRequest.timeout(TimeValue.parseTimeValue(request.param("timeout"), timeValueSeconds(10)));
client.admin().indices().execCreateMapping(createMappingRequest, new ActionListener<CreateMappingResponse>() {
@Override public void onResponse(CreateMappingResponse result) {
try {
JsonBuilder builder = RestJsonBuilder.cached(request);