2020import org .elasticsearch .common .settings .Settings ;
2121import org .elasticsearch .common .unit .ByteSizeUnit ;
2222import org .elasticsearch .common .unit .ByteSizeValue ;
23+ import org .elasticsearch .common .util .concurrent .AdjustableSemaphore ;
24+ import org .elasticsearch .core .Nullable ;
25+ import org .elasticsearch .core .Releasable ;
26+ import org .elasticsearch .core .Releasables ;
2327import org .elasticsearch .core .TimeValue ;
2428import org .elasticsearch .jdk .JavaVersion ;
2529import org .elasticsearch .monitor .os .OsProbe ;
2630import org .elasticsearch .node .NodeRoleSettings ;
2731
32+ import java .util .Collection ;
33+ import java .util .Collections ;
34+ import java .util .Iterator ;
2835import java .util .List ;
36+ import java .util .Locale ;
37+ import java .util .Map ;
2938import java .util .stream .Collectors ;
3039
40+ import static org .elasticsearch .common .settings .Setting .parseInt ;
41+
3142public class RecoverySettings {
3243 public static final Version SNAPSHOT_RECOVERIES_SUPPORTED_VERSION = Version .V_7_15_0 ;
3344 public static final Version SEQ_NO_SNAPSHOT_RECOVERIES_SUPPORTED_VERSION = Version .V_7_16_0 ;
@@ -135,7 +146,7 @@ public class RecoverySettings {
135146
136147 /**
137148 * recoveries would try to use files from available snapshots instead of sending them from the source node.
138- * defaults to `false `
149+ * defaults to `true `
139150 */
140151 public static final Setting <Boolean > INDICES_RECOVERY_USE_SNAPSHOTS_SETTING =
141152 Setting .boolSetting ("indices.recovery.use_snapshots" , true , Property .Dynamic , Property .NodeScope );
@@ -149,6 +160,43 @@ public class RecoverySettings {
149160 Property .NodeScope
150161 );
151162
163+ public static final Setting <Integer > INDICES_RECOVERY_MAX_CONCURRENT_SNAPSHOT_FILE_DOWNLOADS_PER_NODE = new Setting <>(
164+ "indices.recovery.max_concurrent_snapshot_file_downloads_per_node" ,
165+ "25" ,
166+ (s ) -> parseInt (s , 1 , 25 , "indices.recovery.max_concurrent_snapshot_file_downloads_per_node" , false ),
167+ new Setting .Validator <>() {
168+ private final Collection <Setting <?>> dependencies =
169+ Collections .singletonList (INDICES_RECOVERY_MAX_CONCURRENT_SNAPSHOT_FILE_DOWNLOADS );
170+ @ Override
171+ public void validate (Integer value ) {
172+ // ignore
173+ }
174+
175+ @ Override
176+ public void validate (Integer maxConcurrentSnapshotFileDownloadsPerNode , Map <Setting <?>, Object > settings ) {
177+ int maxConcurrentSnapshotFileDownloads = (int ) settings .get (INDICES_RECOVERY_MAX_CONCURRENT_SNAPSHOT_FILE_DOWNLOADS );
178+ if (maxConcurrentSnapshotFileDownloadsPerNode < maxConcurrentSnapshotFileDownloads ) {
179+ throw new IllegalArgumentException (
180+ String .format (Locale .ROOT ,
181+ "[%s]=%d is less than [%s]=%d" ,
182+ INDICES_RECOVERY_MAX_CONCURRENT_SNAPSHOT_FILE_DOWNLOADS_PER_NODE .getKey (),
183+ maxConcurrentSnapshotFileDownloadsPerNode ,
184+ INDICES_RECOVERY_MAX_CONCURRENT_SNAPSHOT_FILE_DOWNLOADS .getKey (),
185+ maxConcurrentSnapshotFileDownloads
186+ )
187+ );
188+ }
189+ }
190+
191+ @ Override
192+ public Iterator <Setting <?>> settings () {
193+ return dependencies .iterator ();
194+ }
195+ },
196+ Setting .Property .Dynamic ,
197+ Setting .Property .NodeScope
198+ );
199+
152200 public static final ByteSizeValue DEFAULT_CHUNK_SIZE = new ByteSizeValue (512 , ByteSizeUnit .KB );
153201
154202 private volatile ByteSizeValue maxBytesPerSec ;
@@ -162,7 +210,9 @@ public class RecoverySettings {
162210 private volatile TimeValue internalActionRetryTimeout ;
163211 private volatile TimeValue internalActionLongTimeout ;
164212 private volatile boolean useSnapshotsDuringRecovery ;
165- private volatile int getMaxConcurrentSnapshotFileDownloads ;
213+ private volatile int maxConcurrentSnapshotFileDownloads ;
214+
215+ private final AdjustableSemaphore maxSnapshotFileDownloadsPerNodeSemaphore ;
166216
167217 private volatile ByteSizeValue chunkSize = DEFAULT_CHUNK_SIZE ;
168218
@@ -186,7 +236,7 @@ public RecoverySettings(Settings settings, ClusterSettings clusterSettings) {
186236 rateLimiter = new SimpleRateLimiter (maxBytesPerSec .getMbFrac ());
187237 }
188238 this .useSnapshotsDuringRecovery = INDICES_RECOVERY_USE_SNAPSHOTS_SETTING .get (settings );
189- this .getMaxConcurrentSnapshotFileDownloads = INDICES_RECOVERY_MAX_CONCURRENT_SNAPSHOT_FILE_DOWNLOADS .get (settings );
239+ this .maxConcurrentSnapshotFileDownloads = INDICES_RECOVERY_MAX_CONCURRENT_SNAPSHOT_FILE_DOWNLOADS .get (settings );
190240
191241 logger .debug ("using max_bytes_per_sec[{}]" , maxBytesPerSec );
192242
@@ -202,7 +252,12 @@ public RecoverySettings(Settings settings, ClusterSettings clusterSettings) {
202252 clusterSettings .addSettingsUpdateConsumer (INDICES_RECOVERY_ACTIVITY_TIMEOUT_SETTING , this ::setActivityTimeout );
203253 clusterSettings .addSettingsUpdateConsumer (INDICES_RECOVERY_USE_SNAPSHOTS_SETTING , this ::setUseSnapshotsDuringRecovery );
204254 clusterSettings .addSettingsUpdateConsumer (INDICES_RECOVERY_MAX_CONCURRENT_SNAPSHOT_FILE_DOWNLOADS ,
205- this ::setGetMaxConcurrentSnapshotFileDownloads );
255+ this ::setMaxConcurrentSnapshotFileDownloads );
256+
257+ int maxSnapshotFileDownloadsPerNode = INDICES_RECOVERY_MAX_CONCURRENT_SNAPSHOT_FILE_DOWNLOADS_PER_NODE .get (settings );
258+ this .maxSnapshotFileDownloadsPerNodeSemaphore = new AdjustableSemaphore (maxSnapshotFileDownloadsPerNode , true );
259+ clusterSettings .addSettingsUpdateConsumer (INDICES_RECOVERY_MAX_CONCURRENT_SNAPSHOT_FILE_DOWNLOADS_PER_NODE ,
260+ this ::setMaxConcurrentSnapshotFileDownloadsPerNode );
206261 }
207262
208263 public RateLimiter rateLimiter () {
@@ -298,10 +353,25 @@ private void setUseSnapshotsDuringRecovery(boolean useSnapshotsDuringRecovery) {
298353 }
299354
300355 public int getMaxConcurrentSnapshotFileDownloads () {
301- return getMaxConcurrentSnapshotFileDownloads ;
356+ return maxConcurrentSnapshotFileDownloads ;
302357 }
303358
304- public void setGetMaxConcurrentSnapshotFileDownloads (int getMaxConcurrentSnapshotFileDownloads ) {
305- this .getMaxConcurrentSnapshotFileDownloads = getMaxConcurrentSnapshotFileDownloads ;
359+ public void setMaxConcurrentSnapshotFileDownloads (int getMaxConcurrentSnapshotFileDownloads ) {
360+ this .maxConcurrentSnapshotFileDownloads = getMaxConcurrentSnapshotFileDownloads ;
361+ }
362+
363+ private void setMaxConcurrentSnapshotFileDownloadsPerNode (int maxConcurrentSnapshotFileDownloadsPerNode ) {
364+ maxSnapshotFileDownloadsPerNodeSemaphore .setMaxPermits (maxConcurrentSnapshotFileDownloadsPerNode );
365+ }
366+
367+ @ Nullable
368+ Releasable tryAcquireSnapshotDownloadPermits () {
369+ final int maxConcurrentSnapshotFileDownloads = getMaxConcurrentSnapshotFileDownloads ();
370+ if (getUseSnapshotsDuringRecovery () == false ||
371+ maxSnapshotFileDownloadsPerNodeSemaphore .tryAcquire (maxConcurrentSnapshotFileDownloads ) == false ) {
372+ return null ;
373+ }
374+
375+ return Releasables .releaseOnce (() -> maxSnapshotFileDownloadsPerNodeSemaphore .release (maxConcurrentSnapshotFileDownloads ));
306376 }
307377}
0 commit comments