Ensure changes requests return the latest mapping version (#37633)

Today we keep the mapping on the follower in sync with the leader's
using the mapping version from changes requests. There are two rare
cases where the mapping on the follower is not synced properly:

1. The returned mapping version (from ClusterService) is outdated than
the actual mapping. This happens because we expose the latest cluster
state in ClusterService after applying it to IndexService.

2. It's possible for the FollowTask to receive an outdated mapping than
the min_required_mapping. In that case, it should fetch the mapping
again; otherwise, the follower won't have the right mapping.

Relates to #31140
This commit is contained in:
Nhat Nguyen 2019-01-23 13:41:13 -05:00 committed by GitHub
parent d7fe4e57fe
commit 0096f1b2e4
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
13 changed files with 159 additions and 56 deletions

View File

@ -179,8 +179,7 @@ public class Ccr extends Plugin implements ActionPlugin, PersistentTaskPlugin, E
ThreadPool threadPool,
Client client,
SettingsModule settingsModule) {
IndexScopedSettings indexScopedSettings = settingsModule.getIndexScopedSettings();
return Collections.singletonList(new ShardFollowTasksExecutor(client, threadPool, clusterService, indexScopedSettings));
return Collections.singletonList(new ShardFollowTasksExecutor(client, threadPool, clusterService, settingsModule));
}
public List<ActionHandler<? extends ActionRequest, ? extends ActionResponse>> getActions() {

View File

@ -30,11 +30,17 @@ public final class CcrSettings {
Setting.boolSetting("index.xpack.ccr.following_index", false, Property.IndexScope, Property.InternalIndex);
/**
* Dynamic node setting for specifying the wait_for_timeout that the auto follow coordinator should be using.
* Dynamic node setting for specifying the wait_for_timeout that the auto follow coordinator and shard follow task should be using.
*/
public static final Setting<TimeValue> CCR_AUTO_FOLLOW_WAIT_FOR_METADATA_TIMEOUT = Setting.timeSetting(
"ccr.auto_follow.wait_for_metadata_timeout", TimeValue.timeValueSeconds(60), Property.NodeScope, Property.Dynamic);
public static final Setting<TimeValue> CCR_WAIT_FOR_METADATA_TIMEOUT = Setting.timeSetting(
"ccr.wait_for_metadata_timeout", TimeValue.timeValueSeconds(60), Property.NodeScope, Property.Dynamic);
/**
* Dynamic node setting for specifying the wait_for_timeout that the auto follow coordinator should be using.
* TODO: Deprecate and remove this setting
*/
private static final Setting<TimeValue> CCR_AUTO_FOLLOW_WAIT_FOR_METADATA_TIMEOUT = Setting.timeSetting(
"ccr.auto_follow.wait_for_metadata_timeout", CCR_WAIT_FOR_METADATA_TIMEOUT, Property.NodeScope, Property.Dynamic);
/**
* Max bytes a node can recover per second.
@ -62,7 +68,8 @@ public final class CcrSettings {
CCR_FOLLOWING_INDEX_SETTING,
RECOVERY_MAX_BYTES_PER_SECOND,
INDICES_RECOVERY_ACTIVITY_TIMEOUT_SETTING,
CCR_AUTO_FOLLOW_WAIT_FOR_METADATA_TIMEOUT);
CCR_AUTO_FOLLOW_WAIT_FOR_METADATA_TIMEOUT,
CCR_WAIT_FOR_METADATA_TIMEOUT);
}
private final CombinedRateLimiter ccrRateLimiter;

View File

@ -113,8 +113,8 @@ public class AutoFollowCoordinator implements ClusterStateListener {
waitForMetadataTimeOut = newWaitForTimeOut;
}
};
clusterService.getClusterSettings().addSettingsUpdateConsumer(CcrSettings.CCR_AUTO_FOLLOW_WAIT_FOR_METADATA_TIMEOUT, updater);
waitForMetadataTimeOut = CcrSettings.CCR_AUTO_FOLLOW_WAIT_FOR_METADATA_TIMEOUT.get(settings);
clusterService.getClusterSettings().addSettingsUpdateConsumer(CcrSettings.CCR_WAIT_FOR_METADATA_TIMEOUT, updater);
waitForMetadataTimeOut = CcrSettings.CCR_WAIT_FOR_METADATA_TIMEOUT.get(settings);
}
public synchronized AutoFollowStats getStats() {

View File

@ -335,10 +335,6 @@ public class ShardChangesAction extends Action<ShardChangesAction.Response> {
final IndexService indexService = indicesService.indexServiceSafe(request.getShard().getIndex());
final IndexShard indexShard = indexService.getShard(request.getShard().id());
final SeqNoStats seqNoStats = indexShard.seqNoStats();
final IndexMetaData indexMetaData = clusterService.state().metaData().index(shardId.getIndex());
final long mappingVersion = indexMetaData.getMappingVersion();
final long settingsVersion = indexMetaData.getSettingsVersion();
final Translog.Operation[] operations = getOperations(
indexShard,
seqNoStats.getGlobalCheckpoint(),
@ -346,8 +342,14 @@ public class ShardChangesAction extends Action<ShardChangesAction.Response> {
request.getMaxOperationCount(),
request.getExpectedHistoryUUID(),
request.getMaxBatchSize());
// must capture after after snapshotting operations to ensure this MUS is at least the highest MUS of any of these operations.
// must capture after snapshotting operations to ensure this MUS is at least the highest MUS of any of these operations.
final long maxSeqNoOfUpdatesOrDeletes = indexShard.getMaxSeqNoOfUpdatesOrDeletes();
// must capture IndexMetaData after snapshotting operations to ensure the returned mapping version is at least as up-to-date
// as the mapping version that these operations used. Here we must not use IndexMetaData from ClusterService for we expose
// a new cluster state to ClusterApplier(s) before exposing it in the ClusterService.
final IndexMetaData indexMetaData = indexService.getMetaData();
final long mappingVersion = indexMetaData.getMappingVersion();
final long settingsVersion = indexMetaData.getSettingsVersion();
return getResponse(
mappingVersion,
settingsVersion,

View File

@ -130,7 +130,7 @@ public abstract class ShardFollowNodeTask extends AllocatedPersistentTask {
}
// updates follower mapping, this gets us the leader mapping version and makes sure that leader and follower mapping are identical
updateMapping(followerMappingVersion -> {
updateMapping(0L, followerMappingVersion -> {
synchronized (ShardFollowNodeTask.this) {
currentMappingVersion = followerMappingVersion;
}
@ -370,7 +370,7 @@ public abstract class ShardFollowNodeTask extends AllocatedPersistentTask {
coordinateReads();
}
private synchronized void maybeUpdateMapping(Long minimumRequiredMappingVersion, Runnable task) {
private synchronized void maybeUpdateMapping(long minimumRequiredMappingVersion, Runnable task) {
if (currentMappingVersion >= minimumRequiredMappingVersion) {
LOGGER.trace("{} mapping version [{}] is higher or equal than minimum required mapping version [{}]",
params.getFollowShardId(), currentMappingVersion, minimumRequiredMappingVersion);
@ -378,7 +378,7 @@ public abstract class ShardFollowNodeTask extends AllocatedPersistentTask {
} else {
LOGGER.trace("{} updating mapping, mapping version [{}] is lower than minimum required mapping version [{}]",
params.getFollowShardId(), currentMappingVersion, minimumRequiredMappingVersion);
updateMapping(mappingVersion -> {
updateMapping(minimumRequiredMappingVersion, mappingVersion -> {
currentMappingVersion = mappingVersion;
task.run();
});
@ -400,12 +400,13 @@ public abstract class ShardFollowNodeTask extends AllocatedPersistentTask {
}
}
private void updateMapping(LongConsumer handler) {
updateMapping(handler, new AtomicInteger(0));
private void updateMapping(long minRequiredMappingVersion, LongConsumer handler) {
updateMapping(minRequiredMappingVersion, handler, new AtomicInteger(0));
}
private void updateMapping(LongConsumer handler, AtomicInteger retryCounter) {
innerUpdateMapping(handler, e -> handleFailure(e, retryCounter, () -> updateMapping(handler, retryCounter)));
private void updateMapping(long minRequiredMappingVersion, LongConsumer handler, AtomicInteger retryCounter) {
innerUpdateMapping(minRequiredMappingVersion, handler,
e -> handleFailure(e, retryCounter, () -> updateMapping(minRequiredMappingVersion, handler, retryCounter)));
}
private void updateSettings(final LongConsumer handler) {
@ -471,7 +472,7 @@ public abstract class ShardFollowNodeTask extends AllocatedPersistentTask {
}
// These methods are protected for testing purposes:
protected abstract void innerUpdateMapping(LongConsumer handler, Consumer<Exception> errorHandler);
protected abstract void innerUpdateMapping(long minRequiredMappingVersion, LongConsumer handler, Consumer<Exception> errorHandler);
protected abstract void innerUpdateSettings(LongConsumer handler, Consumer<Exception> errorHandler);

View File

@ -24,11 +24,13 @@ import org.elasticsearch.client.Client;
import org.elasticsearch.cluster.ClusterState;
import org.elasticsearch.cluster.metadata.IndexMetaData;
import org.elasticsearch.cluster.metadata.MappingMetaData;
import org.elasticsearch.cluster.metadata.MetaData;
import org.elasticsearch.cluster.routing.IndexRoutingTable;
import org.elasticsearch.cluster.service.ClusterService;
import org.elasticsearch.common.CheckedConsumer;
import org.elasticsearch.common.settings.IndexScopedSettings;
import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.common.settings.SettingsModule;
import org.elasticsearch.common.unit.TimeValue;
import org.elasticsearch.common.util.concurrent.EsRejectedExecutionException;
import org.elasticsearch.index.Index;
@ -46,6 +48,7 @@ import org.elasticsearch.persistent.PersistentTasksExecutor;
import org.elasticsearch.tasks.TaskId;
import org.elasticsearch.threadpool.ThreadPool;
import org.elasticsearch.xpack.ccr.Ccr;
import org.elasticsearch.xpack.ccr.CcrSettings;
import org.elasticsearch.xpack.ccr.action.bulk.BulkShardOperationsAction;
import org.elasticsearch.xpack.ccr.action.bulk.BulkShardOperationsRequest;
import org.elasticsearch.xpack.ccr.action.bulk.BulkShardOperationsResponse;
@ -69,16 +72,20 @@ public class ShardFollowTasksExecutor extends PersistentTasksExecutor<ShardFollo
private final ThreadPool threadPool;
private final ClusterService clusterService;
private final IndexScopedSettings indexScopedSettings;
private volatile TimeValue waitForMetadataTimeOut;
public ShardFollowTasksExecutor(Client client,
ThreadPool threadPool,
ClusterService clusterService,
IndexScopedSettings indexScopedSettings) {
SettingsModule settingsModule) {
super(ShardFollowTask.NAME, Ccr.CCR_THREAD_POOL_NAME);
this.client = client;
this.threadPool = threadPool;
this.clusterService = clusterService;
this.indexScopedSettings = indexScopedSettings;
this.indexScopedSettings = settingsModule.getIndexScopedSettings();
this.waitForMetadataTimeOut = CcrSettings.CCR_WAIT_FOR_METADATA_TIMEOUT.get(settingsModule.getSettings());
clusterService.getClusterSettings().addSettingsUpdateConsumer(CcrSettings.CCR_WAIT_FOR_METADATA_TIMEOUT,
newVal -> this.waitForMetadataTimeOut = newVal);
}
@Override
@ -112,33 +119,25 @@ public class ShardFollowTasksExecutor extends PersistentTasksExecutor<ShardFollo
scheduler, System::nanoTime) {
@Override
protected void innerUpdateMapping(LongConsumer handler, Consumer<Exception> errorHandler) {
Index leaderIndex = params.getLeaderShardId().getIndex();
Index followIndex = params.getFollowShardId().getIndex();
ClusterStateRequest clusterStateRequest = CcrRequests.metaDataRequest(leaderIndex.getName());
CheckedConsumer<ClusterStateResponse, Exception> onResponse = clusterStateResponse -> {
IndexMetaData indexMetaData = clusterStateResponse.getState().metaData().getIndexSafe(leaderIndex);
if (indexMetaData.getMappings().isEmpty()) {
assert indexMetaData.getMappingVersion() == 1;
handler.accept(indexMetaData.getMappingVersion());
return;
}
assert indexMetaData.getMappings().size() == 1 : "expected exactly one mapping, but got [" +
indexMetaData.getMappings().size() + "]";
MappingMetaData mappingMetaData = indexMetaData.getMappings().iterator().next().value;
PutMappingRequest putMappingRequest = CcrRequests.putMappingRequest(followIndex.getName(), mappingMetaData);
followerClient.admin().indices().putMapping(putMappingRequest, ActionListener.wrap(
putMappingResponse -> handler.accept(indexMetaData.getMappingVersion()),
errorHandler));
};
try {
remoteClient(params).admin().cluster().state(clusterStateRequest, ActionListener.wrap(onResponse, errorHandler));
} catch (Exception e) {
errorHandler.accept(e);
}
protected void innerUpdateMapping(long minRequiredMappingVersion, LongConsumer handler, Consumer<Exception> errorHandler) {
final Index followerIndex = params.getFollowShardId().getIndex();
getIndexMetadata(minRequiredMappingVersion, 0L, params, ActionListener.wrap(
indexMetaData -> {
if (indexMetaData.getMappings().isEmpty()) {
assert indexMetaData.getMappingVersion() == 1;
handler.accept(indexMetaData.getMappingVersion());
return;
}
assert indexMetaData.getMappings().size() == 1 : "expected exactly one mapping, but got [" +
indexMetaData.getMappings().size() + "]";
MappingMetaData mappingMetaData = indexMetaData.getMappings().iterator().next().value;
PutMappingRequest putMappingRequest = CcrRequests.putMappingRequest(followerIndex.getName(), mappingMetaData);
followerClient.admin().indices().putMapping(putMappingRequest, ActionListener.wrap(
putMappingResponse -> handler.accept(indexMetaData.getMappingVersion()),
errorHandler));
},
errorHandler
));
}
@Override
@ -257,6 +256,39 @@ public class ShardFollowTasksExecutor extends PersistentTasksExecutor<ShardFollo
return wrapClient(client.getRemoteClusterClient(params.getRemoteCluster()), params.getHeaders());
}
private void getIndexMetadata(long minRequiredMappingVersion, long minRequiredMetadataVersion,
ShardFollowTask params, ActionListener<IndexMetaData> listener) {
final Index leaderIndex = params.getLeaderShardId().getIndex();
final ClusterStateRequest clusterStateRequest = CcrRequests.metaDataRequest(leaderIndex.getName());
if (minRequiredMetadataVersion > 0) {
clusterStateRequest.waitForMetaDataVersion(minRequiredMetadataVersion).waitForTimeout(waitForMetadataTimeOut);
}
try {
remoteClient(params).admin().cluster().state(clusterStateRequest, ActionListener.wrap(
r -> {
// if wait_for_metadata_version timeout, the response is empty
if (r.getState() == null) {
assert minRequiredMetadataVersion > 0;
getIndexMetadata(minRequiredMappingVersion, minRequiredMetadataVersion, params, listener);
return;
}
final MetaData metaData = r.getState().metaData();
final IndexMetaData indexMetaData = metaData.getIndexSafe(leaderIndex);
if (indexMetaData.getMappingVersion() < minRequiredMappingVersion) {
// ask for the next version.
getIndexMetadata(minRequiredMappingVersion, metaData.version() + 1, params, listener);
} else {
assert metaData.version() >= minRequiredMetadataVersion : metaData.version() + " < " + minRequiredMetadataVersion;
listener.onResponse(indexMetaData);
}
},
listener::onFailure
));
} catch (Exception e) {
listener.onFailure(e);
}
}
interface FollowerStatsInfoHandler {
void accept(String followerHistoryUUID, long globalCheckpoint, long maxSeqNo);
}

View File

@ -201,7 +201,7 @@ public abstract class CcrIntegTestCase extends ESTestCase {
builder.put(XPackSettings.LOGSTASH_ENABLED.getKey(), false);
builder.put(LicenseService.SELF_GENERATED_LICENSE_TYPE.getKey(), "trial");
// Let cluster state api return quickly in order to speed up auto follow tests:
builder.put(CcrSettings.CCR_AUTO_FOLLOW_WAIT_FOR_METADATA_TIMEOUT.getKey(), TimeValue.timeValueMillis(100));
builder.put(CcrSettings.CCR_WAIT_FOR_METADATA_TIMEOUT.getKey(), TimeValue.timeValueMillis(100));
if (configureRemoteClusterViaNodeSettings() && leaderSeedAddress != null) {
builder.put("cluster.remote.leader_cluster.seeds", leaderSeedAddress);
}

View File

@ -46,7 +46,7 @@ public abstract class CcrSingleNodeTestCase extends ESSingleNodeTestCase {
builder.put(XPackSettings.LOGSTASH_ENABLED.getKey(), false);
builder.put(LicenseService.SELF_GENERATED_LICENSE_TYPE.getKey(), "trial");
// Let cluster state api return quickly in order to speed up auto follow tests:
builder.put(CcrSettings.CCR_AUTO_FOLLOW_WAIT_FOR_METADATA_TIMEOUT.getKey(), TimeValue.timeValueMillis(100));
builder.put(CcrSettings.CCR_WAIT_FOR_METADATA_TIMEOUT.getKey(), TimeValue.timeValueMillis(100));
return builder.build();
}

View File

@ -6,25 +6,34 @@
package org.elasticsearch.xpack.ccr;
import org.elasticsearch.action.DocWriteResponse;
import org.elasticsearch.action.delete.DeleteResponse;
import org.elasticsearch.action.index.IndexResponse;
import org.elasticsearch.cluster.ClusterState;
import org.elasticsearch.cluster.metadata.IndexMetaData;
import org.elasticsearch.cluster.node.DiscoveryNode;
import org.elasticsearch.cluster.routing.ShardRouting;
import org.elasticsearch.cluster.service.ClusterService;
import org.elasticsearch.common.Strings;
import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.common.unit.ByteSizeUnit;
import org.elasticsearch.common.unit.ByteSizeValue;
import org.elasticsearch.common.unit.TimeValue;
import org.elasticsearch.common.xcontent.XContentType;
import org.elasticsearch.common.xcontent.support.XContentMapValues;
import org.elasticsearch.index.IndexSettings;
import org.elasticsearch.index.shard.IndexShard;
import org.elasticsearch.index.shard.ShardId;
import org.elasticsearch.indices.IndicesService;
import org.elasticsearch.test.InternalTestCluster;
import org.elasticsearch.xpack.CcrIntegTestCase;
import org.elasticsearch.xpack.core.ccr.action.FollowStatsAction;
import org.elasticsearch.xpack.core.ccr.action.PutFollowAction;
import org.elasticsearch.xpack.core.ccr.client.CcrClient;
import org.hamcrest.Matchers;
import java.util.Locale;
import java.util.concurrent.CountDownLatch;
import java.util.concurrent.Semaphore;
import java.util.concurrent.TimeUnit;
import java.util.concurrent.atomic.AtomicBoolean;
@ -32,6 +41,7 @@ import java.util.concurrent.atomic.AtomicInteger;
import static java.util.Collections.singletonMap;
import static org.elasticsearch.test.hamcrest.ElasticsearchAssertions.assertAcked;
import static org.hamcrest.Matchers.equalTo;
public class FollowerFailOverIT extends CcrIntegTestCase {
@ -220,4 +230,56 @@ public class FollowerFailOverIT extends CcrIntegTestCase {
pauseFollow("follower-index");
}
public void testReadRequestsReturnsLatestMappingVersion() throws Exception {
InternalTestCluster leaderCluster = getLeaderCluster();
Settings nodeAttributes = Settings.builder().put("node.attr.box", "large").build();
String dataNode = leaderCluster.startDataOnlyNode(nodeAttributes);
assertAcked(
leaderClient().admin().indices().prepareCreate("leader-index")
.setSettings(Settings.builder()
.put(IndexMetaData.SETTING_NUMBER_OF_SHARDS, 1)
.put(IndexMetaData.SETTING_NUMBER_OF_REPLICAS, 0)
.put(IndexSettings.INDEX_SOFT_DELETES_SETTING.getKey(), "true")
.put("index.routing.allocation.require.box", "large"))
.get()
);
ClusterService clusterService = leaderCluster.clusterService(dataNode);
ShardId shardId = clusterService.state().routingTable().index("leader-index").shard(0).shardId();
IndicesService indicesService = leaderCluster.getInstance(IndicesService.class, dataNode);
IndexShard indexShard = indicesService.getShardOrNull(shardId);
// Block the ClusterService from exposing the cluster state with the mapping change. This makes the ClusterService
// have an older mapping version than the actual mapping version that IndexService will use to index "doc1".
final CountDownLatch latch = new CountDownLatch(1);
clusterService.addLowPriorityApplier(event -> {
IndexMetaData imd = event.state().metaData().index("leader-index");
if (imd != null && imd.mapping() != null &&
XContentMapValues.extractValue("properties.balance.type", imd.mapping().sourceAsMap()) != null) {
try {
logger.info("--> block ClusterService from exposing new mapping version");
latch.await();
} catch (Exception e) {
throw new AssertionError(e);
}
}
});
leaderCluster.client().admin().indices().preparePutMapping().setType("doc")
.setSource("balance", "type=long").setTimeout(TimeValue.ZERO).get();
IndexResponse indexResp = leaderCluster.client(dataNode).prepareIndex("leader-index", "doc", "1")
.setSource("{\"balance\": 100}", XContentType.JSON).setTimeout(TimeValue.ZERO).get();
assertThat(indexResp.getResult(), equalTo(DocWriteResponse.Result.CREATED));
assertThat(indexShard.getGlobalCheckpoint(), equalTo(0L));
getFollowerCluster().startDataOnlyNode(nodeAttributes);
followerClient().execute(PutFollowAction.INSTANCE, putFollow("leader-index", "follower-index")).get();
ensureFollowerGreen("follower-index");
// Make sure at least one read-request which requires mapping sync is completed.
assertBusy(() -> {
CcrClient ccrClient = new CcrClient(followerClient());
FollowStatsAction.StatsResponses responses = ccrClient.followStats(new FollowStatsAction.StatsRequest()).actionGet();
long bytesRead = responses.getStatsResponses().stream().mapToLong(r -> r.status().bytesRead()).sum();
assertThat(bytesRead, Matchers.greaterThan(0L));
}, 60, TimeUnit.SECONDS);
latch.countDown();
assertIndexFullyReplicatedToFollower("leader-index", "follower-index");
pauseFollow("follower-index");
}
}

View File

@ -1016,7 +1016,7 @@ public class AutoFollowCoordinatorTests extends ESTestCase {
private ClusterService mockClusterService() {
ClusterService clusterService = mock(ClusterService.class);
ClusterSettings clusterSettings =
new ClusterSettings(Settings.EMPTY, Collections.singleton(CcrSettings.CCR_AUTO_FOLLOW_WAIT_FOR_METADATA_TIMEOUT));
new ClusterSettings(Settings.EMPTY, Collections.singleton(CcrSettings.CCR_WAIT_FOR_METADATA_TIMEOUT));
when(clusterService.getClusterSettings()).thenReturn(clusterSettings);
return clusterService;
}

View File

@ -111,7 +111,7 @@ public class ShardFollowNodeTaskRandomTests extends ESTestCase {
private final Map<Long, Integer> fromToSlot = new HashMap<>();
@Override
protected void innerUpdateMapping(LongConsumer handler, Consumer<Exception> errorHandler) {
protected void innerUpdateMapping(long minRequiredMappingVersion, LongConsumer handler, Consumer<Exception> errorHandler) {
handler.accept(mappingVersion);
}

View File

@ -988,7 +988,7 @@ public class ShardFollowNodeTaskTests extends ESTestCase {
1L, "type", ShardFollowTask.NAME, "description", null, Collections.emptyMap(), followTask, scheduler, System::nanoTime) {
@Override
protected void innerUpdateMapping(LongConsumer handler, Consumer<Exception> errorHandler) {
protected void innerUpdateMapping(long minRequiredMappingVersion, LongConsumer handler, Consumer<Exception> errorHandler) {
Exception failure = mappingUpdateFailures.poll();
if (failure != null) {
errorHandler.accept(failure);

View File

@ -396,7 +396,7 @@ public class ShardFollowTaskReplicationTests extends ESIndexLevelReplicationTest
}
@Override
protected void innerUpdateMapping(LongConsumer handler, Consumer<Exception> errorHandler) {
protected void innerUpdateMapping(long minRequiredMappingVersion, LongConsumer handler, Consumer<Exception> errorHandler) {
// noop, as mapping updates are not tested
handler.accept(1L);
}