Skip to content

Commit

Permalink
Allow some Repository Settings to be Updated Dynamically (#72543) (#7…
Browse files Browse the repository at this point in the history
…4052)

This commit serves two purposes. For one, we need the ability to dynamically
update a repository setting for the encrypted repository work.

Also, this allows dynamically updating repository rate limits while snapshots are
in progress. This has often been an issue in the past where a long running snapshot
made progress over a long period of time already but is going too slowly with the
current rate limit. This left no good options, either throw away the existing
partly done snapshot's work and recreate the repo with a higher rate limit to speed
things up or wait for a long time with the current rate limit.
With this change the rate limit can simply be increased while a snapshot or restore
is running and will take effect immediately.
  • Loading branch information
original-brownbear committed Jun 14, 2021
1 parent 70c9b74 commit 5483bf2
Show file tree
Hide file tree
Showing 9 changed files with 238 additions and 21 deletions.
Original file line number Diff line number Diff line change
@@ -0,0 +1,123 @@
/*
* Copyright Elasticsearch B.V. and/or licensed to Elasticsearch B.V. under one
* or more contributor license agreements. Licensed under the Elastic License
* 2.0 and the Server Side Public License, v 1; you may not use this file except
* in compliance with, at your election, the Elastic License 2.0 or the Server
* Side Public License, v 1.
*/
package org.elasticsearch.repositories.blobstore;

import org.elasticsearch.ExceptionsHelper;
import org.elasticsearch.action.ActionFuture;
import org.elasticsearch.action.admin.cluster.snapshots.create.CreateSnapshotResponse;
import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.common.unit.ByteSizeValue;
import org.elasticsearch.repositories.Repository;
import org.elasticsearch.snapshots.AbstractSnapshotIntegTestCase;
import org.elasticsearch.snapshots.mockstore.MockRepository;
import org.elasticsearch.test.ESIntegTestCase;

import static org.hamcrest.Matchers.instanceOf;

@ESIntegTestCase.ClusterScope(scope = ESIntegTestCase.Scope.TEST, numDataNodes = 0)
public class BlobStoreDynamicSettingsIT extends AbstractSnapshotIntegTestCase {

public void testUpdateRateLimitsDynamically() throws Exception {
final String masterNode = internalCluster().startMasterOnlyNode();

final boolean largeSnapshotPool = randomBoolean();
final String dataNode;
if (largeSnapshotPool) {
dataNode = startDataNodeWithLargeSnapshotPool();
} else {
dataNode = internalCluster().startDataOnlyNode();
}

final String repoName = "test-repo";
// use a small chunk size so the rate limiter does not overshoot to far and get blocked a very long time below
createRepository(repoName, "mock", randomRepositorySettings().put("chunk_size", "100b"));

if (randomBoolean()) {
createFullSnapshot(repoName, "snapshot-1");
}

final String indexName = "test-idx";
createIndexWithContent(indexName);

final Repository repoOnMaster = getRepositoryOnNode(repoName, masterNode);
final Repository repoOnDataNode = getRepositoryOnNode(repoName, dataNode);

final Settings currentSettings = repoOnMaster.getMetadata().settings();
assertNull(currentSettings.get(BlobStoreRepository.MAX_SNAPSHOT_BYTES_PER_SEC.getKey()));
assertNull(currentSettings.get(BlobStoreRepository.MAX_RESTORE_BYTES_PER_SEC.getKey()));

createRepository(
repoName,
"mock",
Settings.builder().put(currentSettings).put(BlobStoreRepository.MAX_SNAPSHOT_BYTES_PER_SEC.getKey(), "1b"),
randomBoolean()
);

assertSame(repoOnMaster, getRepositoryOnNode(repoName, masterNode));
assertSame(repoOnDataNode, getRepositoryOnNode(repoName, dataNode));

final Settings updatedSettings = repoOnMaster.getMetadata().settings();
assertEquals(
ByteSizeValue.ofBytes(1L),
updatedSettings.getAsBytesSize(BlobStoreRepository.MAX_SNAPSHOT_BYTES_PER_SEC.getKey(), ByteSizeValue.ZERO)
);
assertNull(currentSettings.get(BlobStoreRepository.MAX_RESTORE_BYTES_PER_SEC.getKey()));

final ActionFuture<CreateSnapshotResponse> snapshot1 = startFullSnapshotBlockedOnDataNode("snapshot-2", repoName, dataNode);

// we only run concurrent verification when we have a large SNAPSHOT pool on the data node because otherwise the verification would
// deadlock since the small pool is already blocked by the snapshot on the data node
createRepository(
repoName,
"mock",
Settings.builder().put(updatedSettings).put(BlobStoreRepository.MAX_SNAPSHOT_BYTES_PER_SEC.getKey(), "1024b"),
largeSnapshotPool && randomBoolean()
);
assertSame(repoOnMaster, getRepositoryOnNode(repoName, masterNode));
assertSame(repoOnDataNode, getRepositoryOnNode(repoName, dataNode));

logger.info("--> verify that we can't update [location] dynamically");
try {
// this setting update will fail so we can set the verification parameter randomly even if the SNAPSHOT pool is already blocked
// since we will never actually get to the verification step
createRepository(
repoName,
"mock",
Settings.builder().put(repoOnMaster.getMetadata().settings()).put("location", randomRepoPath()),
randomBoolean()
);
} catch (Exception e) {
final Throwable ise = ExceptionsHelper.unwrap(e, IllegalStateException.class);
assertThat(ise, instanceOf(IllegalStateException.class));
assertEquals(
ise.getMessage(),
"trying to modify or unregister repository [test-repo] that is currently used (snapshot is in progress)"
);
}

logger.info("--> verify that we can update [{}] dynamically", MockRepository.DUMMY_UPDATABLE_SETTING_NAME);
final String dummySettingValue = randomUnicodeOfCodepointLength(10);
// we only run concurrent verification when we have a large SNAPSHOT pool on the data node because otherwise the verification would
// deadlock since the small pool is already blocked by the snapshot on the data node
createRepository(
repoName,
"mock",
Settings.builder()
.put(repoOnMaster.getMetadata().settings())
.put(MockRepository.DUMMY_UPDATABLE_SETTING_NAME, dummySettingValue),
largeSnapshotPool && randomBoolean()
);
final Repository newRepoOnMaster = getRepositoryOnNode(repoName, masterNode);
assertSame(repoOnMaster, newRepoOnMaster);
assertSame(repoOnDataNode, getRepositoryOnNode(repoName, dataNode));
assertEquals(dummySettingValue, newRepoOnMaster.getMetadata().settings().get(MockRepository.DUMMY_UPDATABLE_SETTING_NAME));

unblockNode(repoName, dataNode);
assertSuccessful(snapshot1);
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -1365,10 +1365,6 @@ public void testStartWithSuccessfulShardSnapshotPendingFinalization() throws Exc
assertSuccessful(otherSnapshot);
}

private static String startDataNodeWithLargeSnapshotPool() {
return internalCluster().startDataOnlyNode(LARGE_SNAPSHOT_POOL_SETTINGS);
}

private static void assertSnapshotStatusCountOnRepo(String otherBlockedRepoName, int count) {
final SnapshotsStatusResponse snapshotsStatusResponse = client().admin()
.cluster()
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -206,4 +206,8 @@ public String toString() {
public RepositoryMetadata withUuid(String uuid) {
return new RepositoryMetadata(name, uuid, type, settings, generation, pendingGeneration);
}

public RepositoryMetadata withSettings(Settings settings) {
return new RepositoryMetadata(name, uuid, type, settings, generation, pendingGeneration);
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -17,6 +17,7 @@
import org.elasticsearch.cluster.node.DiscoveryNode;
import org.elasticsearch.common.component.Lifecycle;
import org.elasticsearch.common.component.LifecycleListener;
import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.index.shard.ShardId;
import org.elasticsearch.index.snapshots.IndexShardSnapshotStatus;
import org.elasticsearch.index.store.Store;
Expand All @@ -27,6 +28,7 @@
import java.io.IOException;
import java.util.Collection;
import java.util.List;
import java.util.Set;
import java.util.function.Consumer;
import java.util.function.Function;

Expand Down Expand Up @@ -155,6 +157,11 @@ public IndexShardSnapshotStatus getShardSnapshotStatus(SnapshotId snapshotId, In
return in.getShardSnapshotStatus(snapshotId, indexId, shardId);
}

@Override
public boolean canUpdateInPlace(Settings updatedSettings, Set<String> ignoredSettings) {
return in.canUpdateInPlace(updatedSettings, ignoredSettings);
}

@Override
public void updateState(ClusterState state) {
in.updateState(state);
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -196,7 +196,6 @@ public void registerRepository(final PutRepositoryRequest request, final ActionL

@Override
public ClusterState execute(ClusterState currentState) {
ensureRepositoryNotInUse(currentState, request.name());
Metadata metadata = currentState.metadata();
Metadata.Builder mdBuilder = Metadata.builder(currentState.metadata());
RepositoriesMetadata repositories = metadata.custom(RepositoriesMetadata.TYPE, RepositoriesMetadata.EMPTY);
Expand All @@ -207,8 +206,22 @@ public ClusterState execute(ClusterState currentState) {
// Previous version is the same as this one no update is needed.
return currentState;
}
Repository existing = RepositoriesService.this.repositories.get(request.name());
if (existing == null) {
existing = RepositoriesService.this.internalRepositories.get(request.name());
}
assert existing != null : "repository [" + newRepositoryMetadata.name() + "] must exist";
assert existing.getMetadata() == repositoryMetadata;
final RepositoryMetadata updatedMetadata;
if (canUpdateInPlace(newRepositoryMetadata, existing)) {
// we're updating in place so the updated metadata must point at the same uuid and generations
updatedMetadata = repositoryMetadata.withSettings(newRepositoryMetadata.settings());
} else {
ensureRepositoryNotInUse(currentState, request.name());
updatedMetadata = newRepositoryMetadata;
}
found = true;
repositoriesMetadata.add(newRepositoryMetadata);
repositoriesMetadata.add(updatedMetadata);
} else {
repositoriesMetadata.add(repositoryMetadata);
}
Expand Down Expand Up @@ -471,9 +484,7 @@ public void applyClusterState(ClusterChangedEvent event) {
Repository repository = survivors.get(repositoryMetadata.name());
if (repository != null) {
// Found previous version of this repository
RepositoryMetadata previousMetadata = repository.getMetadata();
if (previousMetadata.type().equals(repositoryMetadata.type()) == false
|| previousMetadata.settings().equals(repositoryMetadata.settings()) == false) {
if (canUpdateInPlace(repositoryMetadata, repository) == false) {
// Previous version is different from the version in settings
logger.debug("updating repository [{}]", repositoryMetadata.name());
closeRepository(repository);
Expand Down Expand Up @@ -509,6 +520,12 @@ public void applyClusterState(ClusterChangedEvent event) {
}
}

private boolean canUpdateInPlace(RepositoryMetadata updatedMetadata, Repository repository) {
assert updatedMetadata.name().equals(repository.getMetadata().name());
return repository.getMetadata().type().equals(updatedMetadata.type())
&& repository.canUpdateInPlace(updatedMetadata.settings(), Collections.emptySet());
}

/**
* Gets the {@link RepositoryData} for the given repository.
*
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -18,6 +18,7 @@
import org.elasticsearch.cluster.node.DiscoveryNode;
import org.elasticsearch.core.Nullable;
import org.elasticsearch.common.component.LifecycleComponent;
import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.index.shard.ShardId;
import org.elasticsearch.index.snapshots.IndexShardSnapshotStatus;
import org.elasticsearch.index.store.Store;
Expand All @@ -29,6 +30,7 @@
import java.util.Collection;
import java.util.List;
import java.util.Map;
import java.util.Set;
import java.util.function.Consumer;
import java.util.function.Function;

Expand Down Expand Up @@ -246,6 +248,17 @@ void restoreShard(
*/
IndexShardSnapshotStatus getShardSnapshotStatus(SnapshotId snapshotId, IndexId indexId, ShardId shardId);

/**
* Check if this instances {@link Settings} can be changed to the provided updated settings without recreating the repository.
*
* @param updatedSettings new repository settings
* @param ignoredSettings setting names to ignore even if changed
* @return true if the repository can be updated in place
*/
default boolean canUpdateInPlace(Settings updatedSettings, Set<String> ignoredSettings) {
return getMetadata().settings().equals(updatedSettings);
}

/**
* Update the repository with the incoming cluster state. This method is invoked from {@link RepositoriesService#applyClusterState} and
* thus the same semantics as with {@link org.elasticsearch.cluster.ClusterStateApplier#applyClusterState} apply for the
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -121,9 +121,11 @@
import java.util.ArrayList;
import java.util.Collection;
import java.util.Collections;
import java.util.HashSet;
import java.util.Iterator;
import java.util.List;
import java.util.Map;
import java.util.Objects;
import java.util.Optional;
import java.util.Set;
import java.util.concurrent.BlockingQueue;
Expand Down Expand Up @@ -249,9 +251,9 @@ public abstract class BlobStoreRepository extends AbstractLifecycleComponent imp

private final boolean cacheRepositoryData;

private final RateLimiter snapshotRateLimiter;
private volatile RateLimiter snapshotRateLimiter;

private final RateLimiter restoreRateLimiter;
private volatile RateLimiter restoreRateLimiter;

private final CounterMetric snapshotRateLimitingTimeInNanos = new CounterMetric();

Expand Down Expand Up @@ -289,6 +291,28 @@ public abstract class BlobStoreRepository extends AbstractLifecycleComponent imp
BlobStoreIndexShardSnapshots::fromXContent
);

public static final Setting<ByteSizeValue> MAX_SNAPSHOT_BYTES_PER_SEC = Setting.byteSizeSetting(
"max_snapshot_bytes_per_sec",
new ByteSizeValue(40, ByteSizeUnit.MB),
Setting.Property.Dynamic,
Setting.Property.NodeScope
);

public static final Setting<ByteSizeValue> MAX_RESTORE_BYTES_PER_SEC = Setting.byteSizeSetting(
"max_restore_bytes_per_sec",
ByteSizeValue.ZERO,
Setting.Property.Dynamic,
Setting.Property.NodeScope
);

/**
* Repository settings that can be updated dynamically without having to create a new repository.
*/
private static final Set<String> DYNAMIC_SETTING_NAMES = org.elasticsearch.core.Set.of(
MAX_SNAPSHOT_BYTES_PER_SEC.getKey(),
MAX_RESTORE_BYTES_PER_SEC.getKey()
);

private final boolean readOnly;

private final Object lock = new Object();
Expand Down Expand Up @@ -364,8 +388,8 @@ protected BlobStoreRepository(
this.bigArrays = bigArrays;
this.recoverySettings = recoverySettings;
this.supportURLRepo = SUPPORT_URL_REPO.get(metadata.settings());
snapshotRateLimiter = getRateLimiter(metadata.settings(), "max_snapshot_bytes_per_sec", new ByteSizeValue(40, ByteSizeUnit.MB));
restoreRateLimiter = getRateLimiter(metadata.settings(), "max_restore_bytes_per_sec", ByteSizeValue.ZERO);
snapshotRateLimiter = getRateLimiter(metadata.settings(), MAX_SNAPSHOT_BYTES_PER_SEC);
restoreRateLimiter = getRateLimiter(metadata.settings(), MAX_RESTORE_BYTES_PER_SEC);
readOnly = metadata.settings().getAsBoolean(READONLY_SETTING_KEY, false);
cacheRepositoryData = CACHE_REPOSITORY_DATA.get(metadata.settings());
bufferSize = Math.toIntExact(BUFFER_SIZE_SETTING.get(metadata.settings()).getBytes());
Expand Down Expand Up @@ -555,11 +579,32 @@ private static int getSegmentInfoFileCount(List<BlobStoreIndexShardSnapshot.File
return Math.toIntExact(Math.min(Integer.MAX_VALUE, indexFiles.stream().filter(fi -> fi.physicalName().endsWith(".si")).count()));
}

@Override
public boolean canUpdateInPlace(Settings updatedSettings, Set<String> ignoredSettings) {
final Settings current = metadata.settings();
if (current.equals(updatedSettings)) {
return true;
}
final Set<String> changedSettingNames = new HashSet<>(current.keySet());
changedSettingNames.addAll(updatedSettings.keySet());
changedSettingNames.removeAll(ignoredSettings);
changedSettingNames.removeIf(setting -> Objects.equals(current.get(setting), updatedSettings.get(setting)));
changedSettingNames.removeAll(DYNAMIC_SETTING_NAMES);
return changedSettingNames.isEmpty();
}

// Inspects all cluster state elements that contain a hint about what the current repository generation is and updates
// #latestKnownRepoGen if a newer than currently known generation is found
@Override
public void updateState(ClusterState state) {
final Settings previousSettings = metadata.settings();
metadata = getRepoMetadata(state);
final Settings updatedSettings = metadata.settings();
if (updatedSettings.equals(previousSettings) == false) {
snapshotRateLimiter = getRateLimiter(metadata.settings(), MAX_SNAPSHOT_BYTES_PER_SEC);
restoreRateLimiter = getRateLimiter(metadata.settings(), MAX_RESTORE_BYTES_PER_SEC);
}

uncleanStart = uncleanStart && metadata.generation() != metadata.pendingGeneration();
final boolean wasBestEffortConsistency = bestEffortConsistency;
bestEffortConsistency = uncleanStart
Expand Down Expand Up @@ -1509,11 +1554,10 @@ public BlobContainer shardContainer(IndexId indexId, int shardId) {
*
* @param repositorySettings repository settings
* @param setting setting to use to configure rate limiter
* @param defaultRate default limiting rate
* @return rate limiter or null of no throttling is needed
*/
private RateLimiter getRateLimiter(Settings repositorySettings, String setting, ByteSizeValue defaultRate) {
ByteSizeValue maxSnapshotBytesPerSec = repositorySettings.getAsBytesSize(setting, defaultRate);
private static RateLimiter getRateLimiter(Settings repositorySettings, Setting<ByteSizeValue> setting) {
ByteSizeValue maxSnapshotBytesPerSec = setting.get(repositorySettings);
if (maxSnapshotBytesPerSec.getBytes() <= 0) {
return null;
} else {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -191,6 +191,10 @@ protected void stopNode(final String node) throws IOException {
internalCluster().stopRandomNode(settings -> settings.get("node.name").equals(node));
}

protected static String startDataNodeWithLargeSnapshotPool() {
return internalCluster().startDataOnlyNode(LARGE_SNAPSHOT_POOL_SETTINGS);
}

public void waitForBlock(String node, String repository) throws Exception {
logger.info("--> waiting for [{}] to be blocked on node [{}]", repository, node);
MockRepository mockRepository = getRepositoryOnNode(repository, node);
Expand Down Expand Up @@ -303,11 +307,7 @@ protected void createRepository(String repoName, String type) {
}

protected void createRepositoryNoVerify(String repoName, String type) {
logger.info("--> creating repository [{}] [{}]", repoName, type);
assertAcked(clusterAdmin().preparePutRepository(repoName)
.setVerify(false)
.setType(type)
.setSettings(randomRepositorySettings()));
createRepository(repoName, type, randomRepositorySettings(), false);
}

public static void createRepository(Logger logger, String repoName, String type) {
Expand Down

0 comments on commit 5483bf2

Please sign in to comment.