Skip to content

SOLR-17208: Parallelize backup and restore file transfers#4023

Merged
dsmiley merged 19 commits into
apache:mainfrom
elangelo:parallelizebackups
Jun 10, 2026
Merged

SOLR-17208: Parallelize backup and restore file transfers#4023
dsmiley merged 19 commits into
apache:mainfrom
elangelo:parallelizebackups

Conversation

@elangelo

@elangelo elangelo commented Jan 6, 2026

Copy link
Copy Markdown
Contributor

Description

This PR ensures multiple threads are used to create backups and to restore backups. This ensures a considerate speedup when using cloud storage such as S3.
For comparison a backup to s3 of 1.8TiB takes roughly 16 minutes with this code. a 340GiB collection on the old code takes roughly 50 minutes.
Restoring the same collection took 7 minutes instead of 1 hour and 20 minutes (on a 6 node cluster)

Solution

As the previous implementation already had a loop over all files that needed to be backed up to the backup repository I simply wrapped that in a ThreadPool Executor

Tests

I have ran this code locally on a solrcloud cluster

Checklist

Please review the following and check all that apply:

  • I have reviewed the guidelines for How to Contribute and my code conforms to the standards described there to the best of my ability.
  • I have created a Jira issue and added the issue ID to my pull request title.
  • I have given Solr maintainers access to contribute to my PR branch. (optional but recommended, not available for branches on forks living under an organisation)
  • I have developed this patch against the main branch.
  • I have run ./gradlew check.
  • I have added tests for my changes.
  • The current tests cover the improvement
  • I have added documentation for the Reference Guide
  • I have added a changelog entry for my change

@elangelo elangelo marked this pull request as draft January 6, 2026 16:20
…to 1 thread, allow overriding with system properties
@elangelo elangelo marked this pull request as ready for review January 6, 2026 17:31

@epugh epugh left a comment

Copy link
Copy Markdown
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I don't have the multi thread chops to approve this, but reaidn through it looks good. I wanted a change to the variable name. Do we need any new tests for this capablity, or do the existing ones cover it well enough?

Comment thread solr/core/src/java/org/apache/solr/handler/IncrementalShardBackup.java Outdated
Comment thread solr/solr-ref-guide/modules/deployment-guide/pages/backup-restore.adoc Outdated
@elangelo

elangelo commented Jan 7, 2026

Copy link
Copy Markdown
Contributor Author

I don't have the multi thread chops to approve this, but reaidn through it looks good. I wanted a change to the variable name. Do we need any new tests for this capablity, or do the existing ones cover it well enough?

I think the current tests actually cover everything already. Mind that I did change the gcsrepository and s3repository tests to have some parallelism. Unfortunately I was limited to only 2 threads as with more I got an OutOfMemoryException. But I think it still covers what needs covering.

@epugh epugh left a comment

Copy link
Copy Markdown
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

LGTM. I'd love another committer who is more comfortable with this code base and especially the multithreaded nature of it to review as well.

…was referred to by the non-canonical name `ExecutorUtil.MDCAwareThreadPoolExecutor.CallerRunsPolicy`
@github-actions

Copy link
Copy Markdown

This PR has had no activity for 60 days and is now labeled as stale. Any new activity will remove the stale label. To attract more reviewers, please tag people who might be familiar with the code area and/or notify the dev@solr.apache.org mailing list. To exempt this PR from being marked as stale, make it a draft PR or add the label "exempt-stale". If left unattended, this PR will be closed after another 60 days of inactivity. Thank you for your contribution!

@github-actions github-actions Bot added the stale PR not updated in 60 days label Mar 10, 2026
@janhoy janhoy requested a review from Copilot March 10, 2026 01:34

Copilot AI left a comment

Copy link
Copy Markdown
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Pull request overview

This PR adds configurable parallelism to Solr’s backup (incremental shard backup) and restore (core restore) file-transfer loops to improve throughput, especially for higher-latency cloud repositories (e.g., S3/GCS).

Changes:

  • Add parallel upload/download execution for index file transfers during backup and restore, gated by new sysprop/env settings.
  • Document the new parallel transfer settings in the ref guide.
  • Update S3/GCS incremental backup tests to enable parallelism and add an unreleased changelog entry.

Reviewed changes

Copilot reviewed 6 out of 6 changed files in this pull request and generated 9 comments.

Show a summary per file
File Description
solr/solr-ref-guide/modules/deployment-guide/pages/backup-restore.adoc Documents new parallel upload/download properties and tuning guidance.
solr/modules/s3-repository/src/test/org/apache/solr/s3/S3IncrementalBackupTest.java Enables parallel backup/restore via sysprops for S3 incremental backup tests.
solr/modules/gcs-repository/src/test/org/apache/solr/gcs/GCSIncrementalBackupTest.java Enables parallel backup/restore via sysprops for GCS incremental backup tests.
solr/core/src/java/org/apache/solr/handler/RestoreCore.java Parallelizes restore file copy/download work via an executor and aggregates errors.
solr/core/src/java/org/apache/solr/handler/IncrementalShardBackup.java Parallelizes incremental backup upload work, makes stats thread-safe, aggregates errors.
changelog/unreleased/parallelizebackups.yml Adds changelog entry for the feature.

💡 Add Copilot custom instructions for smarter, more guided reviews. Learn how to get started.

Comment on lines +4 to +6
authors:
- name: Samuel Verstraete
github: elangelo

Copilot AI Mar 10, 2026

Copy link

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

The changelog author metadata uses a github field, but this repository’s changelog format documentation uses nick (optionally with url) under authors. Using an unexpected key may fail changelog validation or omit author info; please switch github: elangelo to nick: elangelo (and add url if desired).

Copilot uses AI. Check for mistakes.

Copy link
Copy Markdown
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

fixed

Comment on lines +141 to +145
60L,
TimeUnit.SECONDS,
new SynchronousQueue<>(),
new SolrNamedThreadFactory("RestoreCore"),
new ThreadPoolExecutor.CallerRunsPolicy())

Copilot AI Mar 10, 2026

Copy link

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Using SynchronousQueue with CallerRunsPolicy means once all maxParallelDownloads threads are busy, additional downloads will execute on the calling thread. That can exceed the configured cap (up to maxParallelDownloads + 1 concurrent transfers) and also bypass the MDCAwareThreadPoolExecutor wrapping for those caller-run tasks. Consider a bounded queue/fixed pool or explicitly limiting in-flight submissions to enforce the configured parallelism.

Copilot uses AI. Check for mistakes.

Copy link
Copy Markdown
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

The CallerRunsPolicy fallback does mean the submitting thread can run a task when the pool is saturated, but the submitting thread is the Solr request thread — it already carries full MDC context, so there's no MDC loss here. MDCAwareThreadPoolExecutor exists to propagate MDC to new pool threads; the caller-runs case doesn't need that propagation. On the cap concern: the maxParallel* setting is a throughput knob, not a hard safety limit. An occasional N+1 concurrent transfer when the pool is fully busy is negligible for a backup/restore workload.

Copy link
Copy Markdown
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Not saying this is wrong and your argument makes sense. Seems like a all of our executors in Solr use a LinkedBlockingQueue which also seems if you do that, we can just set executor to 1 instead of null and avoid all these executor != null checks through this function.

My main problem I can tell from this executor is that it is being created inside this function as a local variable. If I am not mistaken, that means if I do 2 backup/restore calls, I just created 2 thread pools giving us 2x the size. Someone with many collections and backing up with many calls can cause a thread explosion. This should be a global/static executor somewhere else that shares this.

Copy link
Copy Markdown
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Actually now that I think of it more the static pool might make this worse since then if set to 1 then all callers fight for 1 thread. So we should definitely use what you are doing here and but don't create the pool in the call, but make it a static pool with ThreadPoolExecutor.CallerRunsPolicy(). Now if the global pool gets saturated, the calling thread takes over instead of creating executors per call.

Copy link
Copy Markdown
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

if set to 1

Then don't do that :-) It's up to the operator. Perhaps 1 could be sensible if the operator wants to ensure that backups/restores minimally contend with real traffic.

I think most of Solr's thread pools (/executors) are basically singleton pools. Why should this one be different?

Copy link
Copy Markdown
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Fixed this. Can you review?

Copy link
Copy Markdown
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Thanks! I see you dropped CallerRunsPolicy. I kind of liked it so that the calling thread can handle some of the backpressure but I can see an argument for not keeping it as it is harder to control. Also when this gets released, since we set the default to 1, backup/restore will be slow if someone upgrades without knowing this change was implemented and the calling thread isn't doing the backup anymore. @dsmiley wdyt?

Copy link
Copy Markdown
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I agree on harder to control; lets drop it.

Copy link
Copy Markdown
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

i understand that i leave the code as it is now. correct me if i'm wrong.

Copy link
Copy Markdown
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Leave the code as it is. We don't want to use CallerRunsPolicy

Comment on lines +215 to +219
Throwable cause = e.getCause();
// Unwrap RuntimeExceptions that wrap the original IOException
if (cause instanceof RuntimeException && cause.getCause() != null) {
firstError = cause.getCause();
} else {

Copilot AI Mar 10, 2026

Copy link

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

In the parallel path, the RuntimeException unwrapping (firstError = cause.getCause()) can discard the wrapper message that contains the filename. Prefer preserving that message (or re-wrapping the cause with filename context) when propagating the first failure.

Copilot uses AI. Check for mistakes.

Copy link
Copy Markdown
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

fixed

Comment on lines +224 to +228
60L,
TimeUnit.SECONDS,
new SynchronousQueue<>(),
new SolrNamedThreadFactory("IncrementalBackup"),
new ThreadPoolExecutor.CallerRunsPolicy())

Copilot AI Mar 10, 2026

Copy link

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

With SynchronousQueue + CallerRunsPolicy, once maxParallelUploads threads are busy, additional uploads will run on the submitting thread. That can exceed the configured cap (up to maxParallelUploads + 1 concurrent uploads) and bypass MDCAwareThreadPoolExecutor wrapping for caller-run tasks. Consider a bounded queue/fixed pool or explicitly bounding in-flight submissions to enforce the configured limit.

Copilot uses AI. Check for mistakes.

Copy link
Copy Markdown
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

see above

Comment on lines +231 to +233
List<Future<?>> uploadFutures = new ArrayList<>();

try {

Copilot AI Mar 10, 2026

Copy link

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

This implementation queues a Future for every index file and holds them in uploadFutures until the end. For large indexes this can create significant memory overhead and delays error reporting. Consider processing completed tasks as they finish (e.g., ExecutorCompletionService) and/or limiting in-flight submissions to maxParallelUploads.

Copilot uses AI. Check for mistakes.

Copy link
Copy Markdown
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

this really doesn't hold. we need to wait for all futures anyway, so storing them in a list is what we need to do here

Comment on lines +277 to +281
} catch (RuntimeException e) {
if (e.getCause() instanceof IOException) {
throw (IOException) e.getCause();
}
throw e;

Copilot AI Mar 10, 2026

Copy link

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

In the synchronous path, rethrowing only e.getCause() (when it’s an IOException) loses the wrapper message that includes the filename ("Failed to process file: ..."). Preserve that per-file context when propagating errors so backup failures are diagnosable.

Copilot uses AI. Check for mistakes.

Copy link
Copy Markdown
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

fixed

Comment on lines +197 to +201
} catch (RuntimeException e) {
if (e.getCause() instanceof IOException) {
throw (IOException) e.getCause();
}
throw e;

Copilot AI Mar 10, 2026

Copy link

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

In the synchronous path, unwrapping and rethrowing only e.getCause() drops the wrapper message that includes per-file context (filename). Preserve the wrapper message (or re-wrap the cause including the filename) so restore failures remain actionable.

Copilot uses AI. Check for mistakes.

Copy link
Copy Markdown
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

fixed

Comment on lines +148 to +149
List<Future<?>> downloadFutures = new ArrayList<>();

Copilot AI Mar 10, 2026

Copy link

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

This submits one task per index file and retains every Future in downloadFutures until the end. For large collections with many segment files, that can add substantial memory/GC overhead and delays surfacing failures until all tasks are submitted. Consider processing completions incrementally (e.g., ExecutorCompletionService) and/or bounding the number of in-flight tasks to maxParallelDownloads.

Copilot uses AI. Check for mistakes.

Copy link
Copy Markdown
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

see above

Comment on lines +296 to +300
Throwable cause = e.getCause();
// Unwrap RuntimeExceptions that wrap the original IOException
if (cause instanceof RuntimeException && cause.getCause() != null) {
firstError = cause.getCause();
} else {

Copilot AI Mar 10, 2026

Copy link

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

In the parallel join logic, unwrapping RuntimeException to cause.getCause() can discard the wrapper message that includes the filename. Preserve the wrapper message (or re-wrap the underlying IOException with file context) when surfacing the first failure from future.get().

Copilot uses AI. Check for mistakes.

Copy link
Copy Markdown
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

fixed

@github-actions github-actions Bot removed the stale PR not updated in 60 days label Mar 11, 2026
- Replace unsafe IOException cast with `new IOException(msg, cause)` to
  preserve the original cause chain in IncrementalShardBackup and RestoreCore
- Simplify ExecutionException handling by removing unnecessary RuntimeException
  unwrapping; directly assign `e.getCause()` as the first error
- Fix changelog entry: rename `github` field to `nick` for author metadata
@elangelo

Copy link
Copy Markdown
Contributor Author

As far as I can see that test failure is not related to these changes

@dsmiley dsmiley self-requested a review April 14, 2026 18:08

@mlbiscoc mlbiscoc left a comment

Copy link
Copy Markdown
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I pulled this in to test and saw a significant improvement myself. Thanks @elangelo! For a 400GB index that took 20 minutes, with the parallelization, finished in 6 minutes for me. We should look to get this merged.

This PR here looks to be solving https://issues.apache.org/jira/browse/SOLR-17208. Can you update the PR to use this JIRA? CC @gerlowskija

+
Maximum number of index files to upload in parallel during backup operations.
Can also be set via the `SOLR_BACKUP_MAXPARALLELUPLOADS` environment variable.
For cloud storage repositories (S3, GCS), consider setting this to `8` or higher to improve backup performance.

Copy link
Copy Markdown
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

How did you come up with the number 8? Was it just the number you set? Just curious. I would probably not place a number here to recommend as this depends on a persons hardware.

@elangelo elangelo Apr 29, 2026

Copy link
Copy Markdown
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I landed on 8 as that was where I had the biggest throughput in my tests. Higher and I would see too many retries to the S3 buckets, lower and I was not using the full capacity of target solrcloud cluster. Mind you the 8 is really great for disaster recovery scenario's. If you want to restore to a live solrcloud cluster I would not use 8, as 8 does cause a significant load on the solrcloud cluster. (especially IOPS and bandwidth on the disks).

So I can document all of this of course but I never saw any kind of 'operating sanity defaults' anywhere else in the solrcloud documentation, so I didn't.

Copy link
Copy Markdown
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Got it, glad the 8 worked but I still personally don't think we should put 8 here. You also just said don't use 8 on a live cluster as 8 does cause a significant load which is counters your # recommendation. It is a tuning parameter so the the user should figure out for their cloud and hardware not the number we placed on a ref-guide without context of their setup. I think if you want to place a recommendation, it is completely fair to say something along the lines of "Increasing this value can improve backup/restore throughput but too high can result in xyz problems. Recommend to start small and increase based on your Solr clouds observed throughput."

Copy link
Copy Markdown
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

We could set the thread priority of this pool lower so as to reduce its CPU impact and indirectly then IO. This could be done by adding such an option to SolrNamedThreadFactory in another constructor.

Copy link
Copy Markdown
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

@mlbiscoc fixed this in the docs.

Comment thread solr/core/src/java/org/apache/solr/handler/IncrementalShardBackup.java Outdated
Comment thread solr/core/src/java/org/apache/solr/handler/IncrementalShardBackup.java Outdated
Comment on lines +141 to +145
60L,
TimeUnit.SECONDS,
new SynchronousQueue<>(),
new SolrNamedThreadFactory("RestoreCore"),
new ThreadPoolExecutor.CallerRunsPolicy())

Copy link
Copy Markdown
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Not saying this is wrong and your argument makes sense. Seems like a all of our executors in Solr use a LinkedBlockingQueue which also seems if you do that, we can just set executor to 1 instead of null and avoid all these executor != null checks through this function.

My main problem I can tell from this executor is that it is being created inside this function as a local variable. If I am not mistaken, that means if I do 2 backup/restore calls, I just created 2 thread pools giving us 2x the size. Someone with many collections and backing up with many calls can cause a thread explosion. This should be a global/static executor somewhere else that shares this.

@dsmiley dsmiley changed the title Parallelize backups and restore file operations SOLR-17208: Parallelize backup and restore file transfers Apr 29, 2026
@dsmiley dsmiley requested review from gerlowskija and removed request for dsmiley April 29, 2026 03:35
Refactor IncrementalShardBackup and RestoreCore to use shared static
ExecutorService instances instead of creating new executor per operation.
Simplifies conditional logic for parallel vs sequential operations and
avoids thread pool creation overhead. Move executor lifecycle management
outside the try-finally block since shared pools should not be shut down.
Revise recommendations for backup and restore parallelism settings to
emphasize resource constraints and iterative tuning. Update tip to clarify
that global thread pools are shared across concurrent operations on a node.
Add thread name filters for IncrementalBackupExecutor and RestoreCoreExecutor
to SolrIgnoredThreadsFilter so that static pool threads are properly excluded
from test leak detection.
Comment thread solr/core/src/java/org/apache/solr/handler/IncrementalShardBackup.java Outdated
Comment thread solr/core/src/java/org/apache/solr/handler/IncrementalShardBackup.java Outdated
Comment thread solr/core/src/java/org/apache/solr/handler/IncrementalShardBackup.java Outdated
backupStats.skippedUploadingFile(existedFileCS);
continue;
}
ExecutorService executor = BACKUP_EXECUTOR;

Copy link
Copy Markdown
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Just use BACKUP_EXECUTOR instead of doing this.

Copy link
Copy Markdown
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

fixed

* property {@code solr.backup.maxparalleluploads} or environment variable {@code
* SOLR_BACKUP_MAXPARALLELUPLOADS}.
*/
private static final int DEFAULT_MAX_PARALLEL_UPLOADS =

Copy link
Copy Markdown
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

This should just be MAX_PARALLEL_UPLOADS and drop the default prefix

Copy link
Copy Markdown
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

fixed

Comment on lines +291 to +293
throw new IOException("Backup interrupted", firstError);
} else {
throw new IOException("Error during parallel backup upload", firstError);

Copy link
Copy Markdown
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

You threw a SolrException in RestoreCore but throw IOException here.

Copy link
Copy Markdown
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

At most API layers in Solr, SolrException is best.

Copy link
Copy Markdown
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

fixed

Increasing this value can significantly improve restore throughput when using cloud storage (S3, GCS), but too high a value will increase IOPS and bandwidth pressure on your cluster.
Start small and increase based on observed throughput and available resources.

TIP: Both settings share a single global thread pool per property, so the configured limit applies across all concurrent backup or restore operations on the node.

Copy link
Copy Markdown
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I misread this the first time as both backup restore share a single pool which contradicted the settings. Can you rewrite this to be clearly state that these are 2 separate thread pools

Copy link
Copy Markdown
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

fixed

Comment on lines +141 to +145
60L,
TimeUnit.SECONDS,
new SynchronousQueue<>(),
new SolrNamedThreadFactory("RestoreCore"),
new ThreadPoolExecutor.CallerRunsPolicy())

Copy link
Copy Markdown
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Thanks! I see you dropped CallerRunsPolicy. I kind of liked it so that the calling thread can handle some of the backpressure but I can see an argument for not keeping it as it is harder to control. Also when this gets released, since we set the default to 1, backup/restore will be slow if someone upgrades without knowing this change was implemented and the calling thread isn't doing the backup anymore. @dsmiley wdyt?

Comment thread solr/solr-ref-guide/modules/deployment-guide/pages/backup-restore.adoc Outdated
Comment thread solr/core/src/java/org/apache/solr/handler/IncrementalShardBackup.java Outdated
Comment thread solr/core/src/java/org/apache/solr/handler/IncrementalShardBackup.java Outdated
Comment thread solr/core/src/java/org/apache/solr/handler/IncrementalShardBackup.java Outdated
Comment thread solr/core/src/java/org/apache/solr/handler/IncrementalShardBackup.java Outdated
Comment thread solr/core/src/java/org/apache/solr/handler/RestoreCore.java
Comment on lines +291 to +293
throw new IOException("Backup interrupted", firstError);
} else {
throw new IOException("Error during parallel backup upload", firstError);

Copy link
Copy Markdown
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

At most API layers in Solr, SolrException is best.

Comment thread solr/test-framework/src/java/org/apache/solr/SolrIgnoredThreadsFilter.java Outdated
Comment on lines +141 to +145
60L,
TimeUnit.SECONDS,
new SynchronousQueue<>(),
new SolrNamedThreadFactory("RestoreCore"),
new ThreadPoolExecutor.CallerRunsPolicy())

Copy link
Copy Markdown
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I agree on harder to control; lets drop it.

- Rename BACKUP_EXECUTOR -> BACKUP_UPLOAD_EXECUTOR (and thread name
  "BackupUploadExecutor") to reflect that it handles file uploads,
  not the backup process itself
- Rename RESTORE_EXECUTOR -> RESTORE_DOWNLOAD_EXECUTOR (and thread name
  "RestoreDownloadExecutor") for the same reason
- Drop DEFAULT_ prefix from MAX_PARALLEL_DOWNLOADS to match the
  uploads field naming
- Remove Math.max(1, ...) guard from both pool constructors; the
  operator is responsible for providing a valid positive value
- Update SolrIgnoredThreadsFilter to match the renamed thread prefixes
Per dsmiley's feedback: wrap the for loop in a single try/catch
instead of catching inside each iteration. This naturally aborts on
the first error without needing a firstError accumulator, and avoids
continuing to call future.get() on remaining futures after a failure.

Also drop Thread.currentThread().interrupt() on InterruptedException
since we're reporting the error and unwinding immediately anyway.
@dsmiley

dsmiley commented May 7, 2026

Copy link
Copy Markdown
Contributor

Looking good!

@dsmiley

dsmiley commented May 7, 2026

Copy link
Copy Markdown
Contributor

Maybe the his concurrency pattern could be codified into a utility method on ExecutorUtil taking an Executor, a list of Runnables, and throws Exception.

Comment thread changelog/unreleased/SOLR-1092-parallelizebackups.yml
@dsmiley

dsmiley commented Jun 9, 2026

Copy link
Copy Markdown
Contributor

I addressed the important executor lifecycle matter. I suppose this is ready to merge?

@elangelo

elangelo commented Jun 9, 2026

Copy link
Copy Markdown
Contributor Author

Thank you David for picking this up. I honestly haven't had time to look at this again in the last couple of weeks

.getCoreContainer()
.getObjectCache()
.computeIfAbsent(
"RestoreDownloadExecutor",

Copy link
Copy Markdown
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

should this not be the BackupUploadExecutor?

Copy link
Copy Markdown
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

thanks :-)

@dsmiley dsmiley merged commit 74974b1 into apache:main Jun 10, 2026
1 check passed
Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment

Projects

None yet

Development

Successfully merging this pull request may close these issues.

6 participants