Optimize dynamic mapping updates on master by processing latest one per index/node

Instead of processing all the bulk of update mappings we have per index/node, we can only update the last ordered one out of those (cause they are incremented on the node/index level). This will improve the processing time of an index that have large updates of mappings.
closes #4373
This commit is contained in:
Shay Banon 2013-12-07 19:20:16 +01:00
parent dd95895977
commit 4a3978d1b7
5 changed files with 194 additions and 44 deletions

View File

@ -42,6 +42,7 @@ import org.elasticsearch.cluster.block.ClusterBlockException;
import org.elasticsearch.cluster.block.ClusterBlockLevel;
import org.elasticsearch.cluster.metadata.IndexMetaData;
import org.elasticsearch.cluster.metadata.MappingMetaData;
import org.elasticsearch.cluster.node.DiscoveryNode;
import org.elasticsearch.cluster.routing.ShardIterator;
import org.elasticsearch.common.bytes.BytesReference;
import org.elasticsearch.common.collect.Tuple;
@ -567,11 +568,18 @@ public class TransportShardBulkAction extends TransportShardReplicationOperation
if (documentMapper == null) { // should not happen
return;
}
IndexMetaData metaData = clusterService.state().metaData().index(index);
if (metaData == null) {
return;
}
// we generate the order id before we get the mapping to send and refresh the source, so
// if 2 happen concurrently, we know that the later order will include the previous one
long orderId = mappingUpdatedAction.generateNextMappingUpdateOrder();
documentMapper.refreshSource();
IndexMetaData metaData = clusterService.state().metaData().index(index);
final MappingUpdatedAction.MappingUpdatedRequest request = new MappingUpdatedAction.MappingUpdatedRequest(index, metaData.uuid(), type, documentMapper.mappingSource());
DiscoveryNode node = clusterService.localNode();
final MappingUpdatedAction.MappingUpdatedRequest request = new MappingUpdatedAction.MappingUpdatedRequest(index, metaData.uuid(), type, documentMapper.mappingSource(), orderId, node != null ? node.id() : null);
mappingUpdatedAction.execute(request, new ActionListener<MappingUpdatedAction.MappingUpdatedResponse>() {
@Override
public void onResponse(MappingUpdatedAction.MappingUpdatedResponse mappingUpdatedResponse) {

View File

@ -36,6 +36,7 @@ import org.elasticsearch.cluster.block.ClusterBlockLevel;
import org.elasticsearch.cluster.metadata.IndexMetaData;
import org.elasticsearch.cluster.metadata.MappingMetaData;
import org.elasticsearch.cluster.metadata.MetaData;
import org.elasticsearch.cluster.node.DiscoveryNode;
import org.elasticsearch.cluster.routing.ShardIterator;
import org.elasticsearch.common.inject.Inject;
import org.elasticsearch.common.settings.Settings;
@ -272,9 +273,13 @@ public class TransportIndexAction extends TransportShardReplicationOperationActi
if (documentMapper == null) { // should not happen
return;
}
// we generate the order id before we get the mapping to send and refresh the source, so
// if 2 happen concurrently, we know that the later order will include the previous one
long orderId = mappingUpdatedAction.generateNextMappingUpdateOrder();
documentMapper.refreshSource();
DiscoveryNode node = clusterService.localNode();
final MappingUpdatedAction.MappingUpdatedRequest mappingRequest =
new MappingUpdatedAction.MappingUpdatedRequest(request.index(), indexMetaData.uuid(), request.type(), documentMapper.mappingSource());
new MappingUpdatedAction.MappingUpdatedRequest(request.index(), indexMetaData.uuid(), request.type(), documentMapper.mappingSource(), orderId, node != null ? node.id() : null);
logger.trace("Sending mapping updated to master: {}", mappingRequest);
mappingUpdatedAction.execute(mappingRequest, new ActionListener<MappingUpdatedAction.MappingUpdatedResponse>() {
@Override

View File

@ -41,6 +41,7 @@ import org.elasticsearch.threadpool.ThreadPool;
import org.elasticsearch.transport.TransportService;
import java.io.IOException;
import java.util.concurrent.atomic.AtomicLong;
/**
* Called by shards in the cluster when their mapping was dynamically updated and it needs to be updated
@ -48,6 +49,7 @@ import java.io.IOException;
*/
public class MappingUpdatedAction extends TransportMasterNodeOperationAction<MappingUpdatedAction.MappingUpdatedRequest, MappingUpdatedAction.MappingUpdatedResponse> {
private final AtomicLong mappingUpdateOrderGen = new AtomicLong();
private final MetaDataMappingService metaDataMappingService;
@Inject
@ -57,6 +59,10 @@ public class MappingUpdatedAction extends TransportMasterNodeOperationAction<Map
this.metaDataMappingService = metaDataMappingService;
}
public long generateNextMappingUpdateOrder() {
return mappingUpdateOrderGen.incrementAndGet();
}
@Override
protected String transportAction() {
return "cluster/mappingUpdated";
@ -80,7 +86,7 @@ public class MappingUpdatedAction extends TransportMasterNodeOperationAction<Map
@Override
protected void masterOperation(final MappingUpdatedRequest request, final ClusterState state, final ActionListener<MappingUpdatedResponse> listener) throws ElasticSearchException {
metaDataMappingService.updateMapping(request.index(), request.indexUUID(), request.type(), request.mappingSource(), new ClusterStateUpdateListener() {
metaDataMappingService.updateMapping(request.index(), request.indexUUID(), request.type(), request.mappingSource(), request.order, request.nodeId, new ClusterStateUpdateListener() {
@Override
public void onResponse(ClusterStateUpdateResponse response) {
listener.onResponse(new MappingUpdatedResponse());
@ -112,15 +118,19 @@ public class MappingUpdatedAction extends TransportMasterNodeOperationAction<Map
private String indexUUID = IndexMetaData.INDEX_UUID_NA_VALUE;
private String type;
private CompressedString mappingSource;
private long order = -1; // -1 means not set...
private String nodeId = null; // null means not set
MappingUpdatedRequest() {
}
public MappingUpdatedRequest(String index, String indexUUID, String type, CompressedString mappingSource) {
public MappingUpdatedRequest(String index, String indexUUID, String type, CompressedString mappingSource, long order, String nodeId) {
this.index = index;
this.indexUUID = indexUUID;
this.type = type;
this.mappingSource = mappingSource;
this.order = order;
this.nodeId = nodeId;
}
public String index() {
@ -139,6 +149,20 @@ public class MappingUpdatedAction extends TransportMasterNodeOperationAction<Map
return mappingSource;
}
/**
* Returns -1 if not set...
*/
public long order() {
return this.order;
}
/**
* Returns null for not set.
*/
public String nodeId() {
return this.nodeId;
}
@Override
public ActionRequestValidationException validate() {
return null;
@ -153,6 +177,10 @@ public class MappingUpdatedAction extends TransportMasterNodeOperationAction<Map
if (in.getVersion().onOrAfter(Version.V_0_90_6)) {
indexUUID = in.readString();
}
if (in.getVersion().after(Version.V_0_90_7)) {
order = in.readLong();
nodeId = in.readOptionalString();
}
}
@Override
@ -164,6 +192,10 @@ public class MappingUpdatedAction extends TransportMasterNodeOperationAction<Map
if (out.getVersion().onOrAfter(Version.V_0_90_6)) {
out.writeString(indexUUID);
}
if (out.getVersion().after(Version.V_0_90_7)) {
out.writeLong(order);
out.writeOptionalString(nodeId);
}
}
@Override

View File

@ -38,7 +38,6 @@ import org.elasticsearch.common.compress.CompressedString;
import org.elasticsearch.common.inject.Inject;
import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.common.unit.TimeValue;
import org.elasticsearch.common.util.concurrent.ConcurrentCollections;
import org.elasticsearch.index.Index;
import org.elasticsearch.index.mapper.DocumentMapper;
import org.elasticsearch.index.mapper.MapperService;
@ -49,9 +48,9 @@ import org.elasticsearch.indices.IndicesService;
import org.elasticsearch.indices.InvalidTypeNameException;
import org.elasticsearch.indices.TypeMissingException;
import org.elasticsearch.percolator.PercolatorService;
import org.elasticsearch.threadpool.ThreadPool;
import java.util.*;
import java.util.concurrent.BlockingQueue;
import static com.google.common.collect.Maps.newHashMap;
import static org.elasticsearch.index.mapper.DocumentMapper.MergeFlags.mergeFlags;
@ -61,15 +60,20 @@ import static org.elasticsearch.index.mapper.DocumentMapper.MergeFlags.mergeFlag
*/
public class MetaDataMappingService extends AbstractComponent {
private final ThreadPool threadPool;
private final ClusterService clusterService;
private final IndicesService indicesService;
private final BlockingQueue<MappingTask> refreshOrUpdateQueue = ConcurrentCollections.newBlockingQueue();
// the mutex protect all the refreshOrUpdate variables!
private final Object refreshOrUpdateMutex = new Object();
private final List<MappingTask> refreshOrUpdateQueue = new ArrayList<MappingTask>();
private long refreshOrUpdateInsertOrder;
private long refreshOrUpdateProcessedInsertOrder;
@Inject
public MetaDataMappingService(Settings settings, ClusterService clusterService, IndicesService indicesService) {
public MetaDataMappingService(Settings settings, ThreadPool threadPool, ClusterService clusterService, IndicesService indicesService) {
super(settings);
this.threadPool = threadPool;
this.clusterService = clusterService;
this.indicesService = indicesService;
}
@ -96,12 +100,16 @@ public class MetaDataMappingService extends AbstractComponent {
static class UpdateTask extends MappingTask {
final String type;
final CompressedString mappingSource;
final long order; // -1 for unknown
final String nodeId; // null fr unknown
final ClusterStateUpdateListener listener;
UpdateTask(String index, String indexUUID, String type, CompressedString mappingSource, ClusterStateUpdateListener listener) {
UpdateTask(String index, String indexUUID, String type, CompressedString mappingSource, long order, String nodeId, ClusterStateUpdateListener listener) {
super(index, indexUUID);
this.type = type;
this.mappingSource = mappingSource;
this.order = order;
this.nodeId = nodeId;
this.listener = listener;
}
}
@ -111,9 +119,26 @@ public class MetaDataMappingService extends AbstractComponent {
* as possible so we won't create the same index all the time for example for the updates on the same mapping
* and generate a single cluster change event out of all of those.
*/
ClusterState executeRefreshOrUpdate(final ClusterState currentState) throws Exception {
List<MappingTask> allTasks = new ArrayList<MappingTask>();
refreshOrUpdateQueue.drainTo(allTasks);
ClusterState executeRefreshOrUpdate(final ClusterState currentState, final long insertionOrder) throws Exception {
final List<MappingTask> allTasks = new ArrayList<MappingTask>();
synchronized (refreshOrUpdateMutex) {
if (refreshOrUpdateQueue.isEmpty()) {
return currentState;
}
// we already processed this task in a bulk manner in a previous cluster event, simply ignore
// it so we will let other tasks get in and processed ones, we will handle the queued ones
// later on in a subsequent cluster state event
if (insertionOrder < refreshOrUpdateProcessedInsertOrder) {
return currentState;
}
allTasks.addAll(refreshOrUpdateQueue);
refreshOrUpdateQueue.clear();
refreshOrUpdateProcessedInsertOrder = refreshOrUpdateInsertOrder;
}
if (allTasks.isEmpty()) {
return currentState;
@ -132,32 +157,61 @@ public class MetaDataMappingService extends AbstractComponent {
tasksPerIndex.put(task.index, indexTasks);
}
indexTasks.add(task);
}
boolean dirty = false;
MetaData.Builder mdBuilder = MetaData.builder(currentState.metaData());
for (Map.Entry<String, List<MappingTask>> entry : tasksPerIndex.entrySet()) {
String index = entry.getKey();
List<MappingTask> tasks = entry.getValue();
final IndexMetaData indexMetaData = mdBuilder.get(index);
if (indexMetaData == null) {
// index got deleted on us, ignore...
logger.debug("[{}] ignoring tasks - index meta data doesn't exist", index);
continue;
}
// the tasks lists to iterate over, filled with the list of mapping tasks, trying to keep
// the latest (based on order) update mapping one per node
List<MappingTask> allIndexTasks = entry.getValue();
List<MappingTask> tasks = new ArrayList<MappingTask>();
for (MappingTask task : allIndexTasks) {
if (!indexMetaData.isSameUUID(task.indexUUID)) {
logger.debug("[{}] ignoring task [{}] - index meta data doesn't match task uuid", index, task);
continue;
}
boolean add = true;
// if its an update task, make sure we only process the latest ordered one per node
if (task instanceof UpdateTask) {
UpdateTask uTask = (UpdateTask) task;
// we can only do something to compare if we have the order && node
if (uTask.order != -1 && uTask.nodeId != null) {
for (int i = 0; i < tasks.size(); i++) {
MappingTask existing = tasks.get(i);
if (existing instanceof UpdateTask) {
UpdateTask eTask = (UpdateTask) existing;
// if we have the order, and the node id, then we can compare, and replace if applicable
if (eTask.order != -1 && eTask.nodeId != null) {
if (eTask.nodeId.equals(uTask.nodeId) && uTask.order > eTask.order) {
// a newer update task, we can replace so we execute it one!
tasks.set(i, uTask);
add = false;
break;
}
}
}
}
}
}
if (add) {
tasks.add(task);
}
}
boolean removeIndex = false;
// keep track of what we already refreshed, no need to refresh it again...
Set<String> processedRefreshes = Sets.newHashSet();
try {
for (MappingTask task : tasks) {
final IndexMetaData indexMetaData = mdBuilder.get(index);
if (indexMetaData == null) {
// index got deleted on us, ignore...
logger.debug("[{}] ignoring task [{}] - index meta data doesn't exist", index, task);
continue;
}
if (!indexMetaData.isSameUUID(task.indexUUID)) {
// index got deleted on us, ignore...
logger.debug("[{}] ignoring task [{}] - index meta data doesn't match task uuid", index, task);
continue;
}
if (task instanceof RefreshTask) {
RefreshTask refreshTask = (RefreshTask) task;
try {
@ -249,13 +303,24 @@ public class MetaDataMappingService extends AbstractComponent {
if (removeIndex) {
indicesService.removeIndex(index, "created for mapping processing");
}
for (Object task : tasks) {
}
}
// fork sending back updates, so we won't wait to send them back on the cluster state, there
// might be a few of those...
threadPool.generic().execute(new Runnable() {
@Override
public void run() {
for (Object task : allTasks) {
if (task instanceof UpdateTask) {
((UpdateTask) task).listener.onResponse(new ClusterStateUpdateResponse(true));
UpdateTask uTask = (UpdateTask) task;
ClusterStateUpdateResponse response = new ClusterStateUpdateResponse(true);
uTask.listener.onResponse(response);
}
}
}
}
});
if (!dirty) {
return currentState;
@ -267,7 +332,11 @@ public class MetaDataMappingService extends AbstractComponent {
* Refreshes mappings if they are not the same between original and parsed version
*/
public void refreshMapping(final String index, final String indexUUID, final String... types) {
refreshOrUpdateQueue.add(new RefreshTask(index, indexUUID, types));
final long insertOrder;
synchronized (refreshOrUpdateMutex) {
insertOrder = ++refreshOrUpdateInsertOrder;
refreshOrUpdateQueue.add(new RefreshTask(index, indexUUID, types));
}
clusterService.submitStateUpdateTask("refresh-mapping [" + index + "][" + Arrays.toString(types) + "]", Priority.HIGH, new ClusterStateUpdateTask() {
@Override
public void onFailure(String source, Throwable t) {
@ -276,13 +345,17 @@ public class MetaDataMappingService extends AbstractComponent {
@Override
public ClusterState execute(ClusterState currentState) throws Exception {
return executeRefreshOrUpdate(currentState);
return executeRefreshOrUpdate(currentState, insertOrder);
}
});
}
public void updateMapping(final String index, final String indexUUID, final String type, final CompressedString mappingSource, final ClusterStateUpdateListener listener) {
refreshOrUpdateQueue.add(new UpdateTask(index, indexUUID, type, mappingSource, listener));
public void updateMapping(final String index, final String indexUUID, final String type, final CompressedString mappingSource, final long order, final String nodeId, final ClusterStateUpdateListener listener) {
final long insertOrder;
synchronized (refreshOrUpdateMutex) {
insertOrder = ++refreshOrUpdateInsertOrder;
refreshOrUpdateQueue.add(new UpdateTask(index, indexUUID, type, mappingSource, order, nodeId, listener));
}
clusterService.submitStateUpdateTask("update-mapping [" + index + "][" + type + "]", Priority.HIGH, new ClusterStateUpdateTask() {
@Override
public void onFailure(String source, Throwable t) {
@ -291,7 +364,7 @@ public class MetaDataMappingService extends AbstractComponent {
@Override
public ClusterState execute(final ClusterState currentState) throws Exception {
return executeRefreshOrUpdate(currentState);
return executeRefreshOrUpdate(currentState, insertOrder);
}
});
}

View File

@ -1,10 +1,14 @@
package org.elasticsearch.indices.mapping;
import com.google.common.base.Predicate;
import org.elasticsearch.action.ActionListener;
import org.elasticsearch.action.admin.cluster.tasks.PendingClusterTasksResponse;
import org.elasticsearch.action.admin.indices.mapping.get.GetMappingsResponse;
import org.elasticsearch.action.admin.indices.mapping.put.PutMappingResponse;
import org.elasticsearch.action.admin.indices.refresh.RefreshResponse;
import org.elasticsearch.action.count.CountResponse;
import org.elasticsearch.action.get.GetResponse;
import org.elasticsearch.action.index.IndexResponse;
import org.elasticsearch.client.Client;
import org.elasticsearch.cluster.metadata.MappingMetaData;
import org.elasticsearch.common.Priority;
@ -20,6 +24,7 @@ import org.junit.Test;
import java.util.ArrayList;
import java.util.Map;
import java.util.concurrent.CountDownLatch;
import java.util.concurrent.CyclicBarrier;
import java.util.concurrent.atomic.AtomicBoolean;
@ -32,24 +37,51 @@ public class UpdateMappingTests extends ElasticsearchIntegrationTest {
@Test
public void dynamicUpdates() throws Exception {
client().admin().indices().prepareCreate("test")
.setSettings(
ImmutableSettings.settingsBuilder()
.put("index.number_of_shards", 2)
.put("index.number_of_shards", 1)
.put("index.number_of_replicas", 0)
).execute().actionGet();
client().admin().cluster().prepareHealth().setWaitForEvents(Priority.LANGUID).setWaitForGreenStatus().execute().actionGet();
long recCount = 20;
int recCount = 200;
final CountDownLatch latch = new CountDownLatch(recCount);
for (int rec = 0; rec < recCount; rec++) {
client().prepareIndex("test", "type", "rec" + rec).setSource("field" + rec, "some_value").execute().actionGet();
client().prepareIndex("test", "type", "rec" + rec).setSource("field" + rec, "some_value").execute(new ActionListener<IndexResponse>() {
@Override
public void onResponse(IndexResponse indexResponse) {
latch.countDown();
}
@Override
public void onFailure(Throwable e) {
logger.error("failed to index in test", e);
latch.countDown();
}
});
}
latch.await();
logger.info("wait till the mappings have been processed...");
awaitBusy(new Predicate<Object>() {
@Override
public boolean apply(Object input) {
PendingClusterTasksResponse pendingTasks = client().admin().cluster().preparePendingClusterTasks().get();
return pendingTasks.pendingTasks().isEmpty();
}
});
logger.info("checking all the documents are there");
RefreshResponse refreshResponse = client().admin().indices().prepareRefresh().execute().actionGet();
assertThat(refreshResponse.getFailedShards(), equalTo(0));
logger.info("Searching");
CountResponse response = client().prepareCount("test").execute().actionGet();
assertThat(response.getCount(), equalTo(recCount));
assertThat(response.getCount(), equalTo((long) recCount));
logger.info("checking all the fields are in the mappings");
String source = client().admin().indices().prepareGetMappings("test").setTypes("type").get().getMappings().get("test").get("type").source().string();
for (int rec = 0; rec < recCount; rec++) {
assertThat(source, containsString("\"field" + rec + "\""));
}
}
@Test(expected = MergeMappingException.class)