Don't update default mapping on phase2 recovery
the default mapping is not merged, but updated in place, and only put mapping API can change it, no need to make sure it has been properly updated on the master. This can cause conflicts when a put mapping for it happens at the same time. closes #6771
This commit is contained in:
parent
15b81c91f7
commit
807b2409ee
|
@ -43,6 +43,7 @@ import org.elasticsearch.common.unit.TimeValue;
|
|||
import org.elasticsearch.common.util.concurrent.ConcurrentCollections;
|
||||
import org.elasticsearch.common.util.concurrent.EsExecutors;
|
||||
import org.elasticsearch.index.mapper.DocumentMapper;
|
||||
import org.elasticsearch.index.mapper.MapperService;
|
||||
import org.elasticsearch.node.settings.NodeSettingsService;
|
||||
import org.elasticsearch.threadpool.ThreadPool;
|
||||
import org.elasticsearch.transport.TransportService;
|
||||
|
@ -108,6 +109,7 @@ public class MappingUpdatedAction extends TransportMasterNodeOperationAction<Map
|
|||
}
|
||||
|
||||
public void updateMappingOnMaster(String index, DocumentMapper documentMapper, String indexUUID, MappingUpdateListener listener) {
|
||||
assert !documentMapper.type().equals(MapperService.DEFAULT_MAPPING) : "_default_ mapping should not be updated";
|
||||
masterMappingUpdater.add(new MappingChange(documentMapper, index, indexUUID, listener));
|
||||
}
|
||||
|
||||
|
|
|
@ -40,6 +40,7 @@ import org.elasticsearch.common.unit.TimeValue;
|
|||
import org.elasticsearch.index.deletionpolicy.SnapshotIndexCommit;
|
||||
import org.elasticsearch.index.engine.Engine;
|
||||
import org.elasticsearch.index.mapper.DocumentMapper;
|
||||
import org.elasticsearch.index.mapper.MapperService;
|
||||
import org.elasticsearch.index.service.IndexService;
|
||||
import org.elasticsearch.index.shard.IllegalIndexShardStateException;
|
||||
import org.elasticsearch.index.shard.IndexShardClosedException;
|
||||
|
@ -279,8 +280,16 @@ public class RecoverySource extends AbstractComponent {
|
|||
}
|
||||
|
||||
private void updateMappingOnMaster() {
|
||||
List<DocumentMapper> documentMappersToUpdate = Lists.newArrayList(indexService.mapperService());
|
||||
if (documentMappersToUpdate.size() == 0) {
|
||||
List<DocumentMapper> documentMappersToUpdate = Lists.newArrayList();
|
||||
for (DocumentMapper documentMapper : indexService.mapperService()) {
|
||||
// default mapping should not be sent back, it can only be updated by put mapping API, and its
|
||||
// a full in place replace, we don't want to override a potential update coming it
|
||||
if (documentMapper.type().equals(MapperService.DEFAULT_MAPPING)) {
|
||||
continue;
|
||||
}
|
||||
documentMappersToUpdate.add(documentMapper);
|
||||
}
|
||||
if (documentMappersToUpdate.isEmpty()) {
|
||||
return;
|
||||
}
|
||||
final CountDownLatch countDownLatch = new CountDownLatch(documentMappersToUpdate.size());
|
||||
|
|
|
@ -19,9 +19,7 @@
|
|||
|
||||
package org.elasticsearch.indices.mapping;
|
||||
|
||||
import com.carrotsearch.hppc.cursors.ObjectObjectCursor;
|
||||
import com.google.common.collect.Lists;
|
||||
import com.google.common.collect.Maps;
|
||||
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;
|
||||
|
@ -29,7 +27,6 @@ import org.elasticsearch.action.count.CountResponse;
|
|||
import org.elasticsearch.action.get.GetResponse;
|
||||
import org.elasticsearch.action.index.IndexRequestBuilder;
|
||||
import org.elasticsearch.client.Client;
|
||||
import org.elasticsearch.cluster.ClusterState;
|
||||
import org.elasticsearch.cluster.metadata.MappingMetaData;
|
||||
import org.elasticsearch.common.Priority;
|
||||
import org.elasticsearch.common.collect.ImmutableOpenMap;
|
||||
|
|
Loading…
Reference in New Issue