-
Notifications
You must be signed in to change notification settings - Fork 25.1k
New issue
Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.
By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.
Already on GitHub? Sign in to your account
Implement follower rate limiting for file restore #37449
Changes from 3 commits
adae8e8
fc4be44
f26e0e6
425fa92
eba3449
1c135ec
c98b635
b529aba
fd4632d
File filter
Filter by extension
Conversations
Jump to
Diff view
Diff view
There are no files selected for viewing
Original file line number | Diff line number | Diff line change |
---|---|---|
|
@@ -7,6 +7,7 @@ | |
package org.elasticsearch.xpack.ccr.repository; | ||
|
||
import org.apache.lucene.index.IndexCommit; | ||
import org.apache.lucene.store.RateLimiter; | ||
import org.elasticsearch.Version; | ||
import org.elasticsearch.action.admin.cluster.state.ClusterStateRequest; | ||
import org.elasticsearch.action.admin.cluster.state.ClusterStateResponse; | ||
|
@@ -24,6 +25,7 @@ | |
import org.elasticsearch.common.collect.ImmutableOpenMap; | ||
import org.elasticsearch.common.component.AbstractLifecycleComponent; | ||
import org.elasticsearch.common.io.stream.StreamInput; | ||
import org.elasticsearch.common.metrics.CounterMetric; | ||
import org.elasticsearch.common.settings.Settings; | ||
import org.elasticsearch.common.unit.ByteSizeValue; | ||
import org.elasticsearch.index.Index; | ||
|
@@ -66,6 +68,9 @@ | |
import java.util.List; | ||
import java.util.Map; | ||
import java.util.Set; | ||
import java.util.concurrent.atomic.AtomicLong; | ||
|
||
import static org.elasticsearch.xpack.ccr.CcrSettings.FOLLOWER_RECOVERY_MAX_BYTES_READ_PER_SECOND; | ||
|
||
/** | ||
* This repository relies on a remote cluster for Ccr restores. It is read-only so it can only be used to | ||
|
@@ -83,13 +88,18 @@ public class CcrRepository extends AbstractLifecycleComponent implements Reposit | |
private final Client client; | ||
private final CcrLicenseChecker ccrLicenseChecker; | ||
|
||
private final RateLimiter.SimpleRateLimiter rateLimiter; | ||
private final CounterMetric throttledTime = new CounterMetric(); | ||
private final AtomicLong bytesSinceLastPause = new AtomicLong(); | ||
|
||
public CcrRepository(RepositoryMetaData metadata, Client client, CcrLicenseChecker ccrLicenseChecker, Settings settings) { | ||
super(settings); | ||
this.metadata = metadata; | ||
assert metadata.name().startsWith(NAME_PREFIX) : "CcrRepository metadata.name() must start with: " + NAME_PREFIX; | ||
this.remoteClusterAlias = Strings.split(metadata.name(), NAME_PREFIX)[1]; | ||
this.ccrLicenseChecker = ccrLicenseChecker; | ||
this.client = client; | ||
this.rateLimiter = new RateLimiter.SimpleRateLimiter(FOLLOWER_RECOVERY_MAX_BYTES_READ_PER_SECOND.get(settings).getMbFrac()); | ||
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. This setting is dynamic, so let's allow to dynamically adjust it by registering a settings update consumer (see the There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. I registered a listener for the setting change. |
||
} | ||
|
||
@Override | ||
|
@@ -207,7 +217,7 @@ public long getSnapshotThrottleTimeInNanos() { | |
|
||
@Override | ||
public long getRestoreThrottleTimeInNanos() { | ||
return 0; | ||
return throttledTime.count(); | ||
} | ||
|
||
@Override | ||
|
@@ -258,7 +268,7 @@ public void restoreShard(IndexShard indexShard, SnapshotId snapshotId, Version v | |
// TODO: There should be some local timeout. And if the remote cluster returns an unknown session | ||
// response, we should be able to retry by creating a new session. | ||
String name = metadata.name(); | ||
try (RestoreSession restoreSession = RestoreSession.openSession(name, remoteClient, leaderShardId, indexShard, recoveryState)) { | ||
try (RestoreSession restoreSession = openSession(name, remoteClient, leaderShardId, indexShard, recoveryState)) { | ||
restoreSession.restoreFiles(); | ||
} catch (Exception e) { | ||
throw new IndexShardRestoreFailedException(indexShard.shardId(), "failed to restore snapshot [" + snapshotId + "]", e); | ||
|
@@ -286,7 +296,16 @@ private void maybeUpdateMappings(Client localClient, Client remoteClient, Index | |
} | ||
} | ||
|
||
private static class RestoreSession extends FileRestoreContext implements Closeable { | ||
private RestoreSession openSession(String repositoryName, Client remoteClient, ShardId leaderShardId, IndexShard indexShard, | ||
RecoveryState recoveryState) { | ||
String sessionUUID = UUIDs.randomBase64UUID(); | ||
PutCcrRestoreSessionAction.PutCcrRestoreSessionResponse response = remoteClient.execute(PutCcrRestoreSessionAction.INSTANCE, | ||
new PutCcrRestoreSessionRequest(sessionUUID, leaderShardId)).actionGet(); | ||
return new RestoreSession(repositoryName, remoteClient, sessionUUID, response.getNode(), indexShard, recoveryState, | ||
response.getStoreFileMetaData()); | ||
} | ||
|
||
private class RestoreSession extends FileRestoreContext implements Closeable { | ||
|
||
private static final int BUFFER_SIZE = 1 << 16; | ||
|
||
|
@@ -304,15 +323,6 @@ private static class RestoreSession extends FileRestoreContext implements Closea | |
this.sourceMetaData = sourceMetaData; | ||
} | ||
|
||
static RestoreSession openSession(String repositoryName, Client remoteClient, ShardId leaderShardId, IndexShard indexShard, | ||
RecoveryState recoveryState) { | ||
String sessionUUID = UUIDs.randomBase64UUID(); | ||
PutCcrRestoreSessionAction.PutCcrRestoreSessionResponse response = remoteClient.execute(PutCcrRestoreSessionAction.INSTANCE, | ||
new PutCcrRestoreSessionRequest(sessionUUID, leaderShardId)).actionGet(); | ||
return new RestoreSession(repositoryName, remoteClient, sessionUUID, response.getNode(), indexShard, recoveryState, | ||
response.getStoreFileMetaData()); | ||
} | ||
|
||
void restoreFiles() throws IOException { | ||
ArrayList<BlobStoreIndexShardSnapshot.FileInfo> fileInfos = new ArrayList<>(); | ||
for (StoreFileMetaData fileMetaData : sourceMetaData) { | ||
|
@@ -336,7 +346,7 @@ public void close() { | |
} | ||
} | ||
|
||
private static class RestoreFileInputStream extends InputStream { | ||
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. why remove the There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. In order to access the rate limiter field. We also need to add a |
||
private class RestoreFileInputStream extends InputStream { | ||
Tim-Brooks marked this conversation as resolved.
Show resolved
Hide resolved
|
||
|
||
private final Client remoteClient; | ||
private final String sessionUUID; | ||
|
@@ -366,6 +376,9 @@ public int read(byte[] bytes, int off, int len) throws IOException { | |
} | ||
|
||
int bytesRequested = (int) Math.min(remainingBytes, len); | ||
|
||
maybePause(bytesRequested); | ||
|
||
String fileName = fileToRecover.name(); | ||
GetCcrRestoreFileChunkRequest request = new GetCcrRestoreFileChunkRequest(node, sessionUUID, fileName, bytesRequested); | ||
GetCcrRestoreFileChunkAction.GetCcrRestoreFileChunkResponse response = | ||
|
@@ -389,5 +402,17 @@ public int read(byte[] bytes, int off, int len) throws IOException { | |
|
||
return bytesReceived; | ||
} | ||
|
||
private void maybePause(int bytesRequested) { | ||
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. just to make sure, this is the same code as in PeerRecoveryTargetService. Let's have a follow-up to refactor it into a class that is reusable, and use it everywhere (peer recovery source, target as well as snapshot + restore). There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. I did have to extract a There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. ok, sounds good. |
||
long bytesSincePause = bytesSinceLastPause.addAndGet(bytesRequested); | ||
if (bytesSincePause > rateLimiter.getMinPauseCheckBytes()) { | ||
// Time to pause | ||
bytesSinceLastPause.addAndGet(-bytesSincePause); | ||
long throttleTimeInNanos = rateLimiter.pause(bytesSincePause); | ||
if (throttleTimeInNanos > 0) { | ||
throttledTime.inc(throttleTimeInNanos); | ||
} | ||
} | ||
} | ||
} | ||
} |
Original file line number | Diff line number | Diff line change |
---|---|---|
|
@@ -21,6 +21,7 @@ | |
import org.elasticsearch.cluster.service.ClusterService; | ||
import org.elasticsearch.common.collect.ImmutableOpenMap; | ||
import org.elasticsearch.common.settings.Settings; | ||
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; | ||
|
@@ -38,6 +39,8 @@ | |
import org.elasticsearch.xpack.ccr.repository.CcrRestoreSourceService; | ||
|
||
import java.io.IOException; | ||
import java.util.ArrayList; | ||
import java.util.List; | ||
import java.util.Locale; | ||
import java.util.Map; | ||
import java.util.concurrent.TimeUnit; | ||
|
@@ -235,6 +238,60 @@ public void testDocsAreRecovered() throws Exception { | |
thread.join(); | ||
} | ||
|
||
public void testRateLimitingIsEmployed() throws Exception { | ||
restartClustersWithSettings(Settings.builder().put(CcrSettings.FOLLOWER_RECOVERY_MAX_BYTES_READ_PER_SECOND.getKey(), | ||
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. by making the setting truly dynamic, we won't need the restarts here, which will greatly speed up the tests. See SharedClusterSnapshotRestoreIT#testThrottling for how it's tested for snapshot /restore. There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. I did the same setup and assertion as that test. |
||
new ByteSizeValue(500)).build()); | ||
String leaderClusterRepoName = CcrRepository.NAME_PREFIX + "leader_cluster"; | ||
String leaderIndex = "index1"; | ||
String followerIndex = "index2"; | ||
|
||
final int numberOfPrimaryShards = randomIntBetween(1, 3); | ||
final String leaderIndexSettings = getIndexSettings(numberOfPrimaryShards, between(0, 1), | ||
singletonMap(IndexSettings.INDEX_SOFT_DELETES_SETTING.getKey(), "true")); | ||
assertAcked(leaderClient().admin().indices().prepareCreate(leaderIndex).setSource(leaderIndexSettings, XContentType.JSON)); | ||
ensureLeaderGreen(leaderIndex); | ||
|
||
final RestoreService restoreService = getFollowerCluster().getCurrentMasterNodeInstance(RestoreService.class); | ||
final ClusterService clusterService = getFollowerCluster().getCurrentMasterNodeInstance(ClusterService.class); | ||
|
||
List<CcrRepository> repositories = new ArrayList<>(); | ||
try { | ||
for (RepositoriesService repositoriesService : getFollowerCluster().getDataOrMasterNodeInstances(RepositoriesService.class)) { | ||
Repository repository = repositoriesService.repository(leaderClusterRepoName); | ||
repositories.add((CcrRepository) repository); | ||
} | ||
} catch (RepositoryMissingException e) { | ||
fail("need repository"); | ||
Tim-Brooks marked this conversation as resolved.
Show resolved
Hide resolved
|
||
} | ||
|
||
final int firstBatchNumDocs = 10; | ||
logger.info("Indexing [{}] docs as first batch", firstBatchNumDocs); | ||
for (int i = 0; i < firstBatchNumDocs; i++) { | ||
final String source = String.format(Locale.ROOT, "{\"f\":%d}", i); | ||
leaderClient().prepareIndex("index1", "doc", Integer.toString(i)).setSource(source, XContentType.JSON).get(); | ||
} | ||
|
||
leaderClient().admin().indices().prepareFlush(leaderIndex).setForce(true).setWaitIfOngoing(true).get(); | ||
|
||
try { | ||
Settings.Builder settingsBuilder = Settings.builder() | ||
.put(IndexMetaData.SETTING_INDEX_PROVIDED_NAME, followerIndex) | ||
.put(CcrSettings.CCR_FOLLOWING_INDEX_SETTING.getKey(), true); | ||
RestoreService.RestoreRequest restoreRequest = new RestoreService.RestoreRequest(leaderClusterRepoName, | ||
CcrRepository.LATEST, new String[]{leaderIndex}, indicesOptions, | ||
"^(.*)$", followerIndex, Settings.EMPTY, new TimeValue(1, TimeUnit.HOURS), false, | ||
false, true, settingsBuilder.build(), new String[0], | ||
"restore_snapshot[" + leaderClusterRepoName + ":" + leaderIndex + "]"); | ||
|
||
PlainActionFuture<RestoreInfo> future = PlainActionFuture.newFuture(); | ||
restoreService.restoreSnapshot(restoreRequest, waitForRestore(clusterService, future)); | ||
|
||
assertBusy(() -> assertTrue(repositories.stream().anyMatch(cr -> cr.getRestoreThrottleTimeInNanos() > 0))); | ||
} finally { | ||
restartClustersWithSettings(Settings.EMPTY); | ||
} | ||
} | ||
|
||
public void testFollowerMappingIsUpdated() throws IOException { | ||
String leaderClusterRepoName = CcrRepository.NAME_PREFIX + "leader_cluster"; | ||
String leaderIndex = "index1"; | ||
|
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
let's use the same setting for rate-limiting inbound as well as outbound CCR recovery trafic. This means removing the "follower" part from the setting name. Perhaps
ccr.indices.recovery.max_bytes_per_sec
.There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
let's also add a bullet point to the meta issue to document the newly introduced settings