Skip to content

CASSSIDECAR-226 Adding endpoint for verifying files post data copy during live migration#309

Open
nvharikrishna wants to merge 8 commits intoapache:trunkfrom
nvharikrishna:226-lm-file-digests-trunk
Open

CASSSIDECAR-226 Adding endpoint for verifying files post data copy during live migration#309
nvharikrishna wants to merge 8 commits intoapache:trunkfrom
nvharikrishna:226-lm-file-digests-trunk

Conversation

@nvharikrishna
Copy link
Copy Markdown
Contributor

@nvharikrishna nvharikrishna commented Jan 25, 2026

CASSSIDECAR-226 Adding an endpoint for verifying files between source and destination post data copy.

This implementation uses a two-task approach (data copy + file verification) rather than inline digest verification during data copy (as originally proposed in CEP-40). This design choice is motivated by:

  1. Performance Efficiency: The data copy task executes multiple iterations internally. Even with successThreshold=1.0, the task might require at least two internal iterations (iteration 0: download → DOWNLOAD_COMPLETE, iteration 1: verify threshold → SUCCESS). Inline digest verification would calculate digests twice per file (once in each iteration), doubling the I/O cost. With separate tasks, digests are calculated once after data stabilizes.
  2. Code Simplicity: Separating digest verification from file copying provides clear separation of concerns, making each task easier to understand, test, and maintain.
  3. Operational Flexibility: Users can run verification independently, repeat it if needed, or skip it for non-critical migrations. Inline verification would make this mandatory overhead.

Here are the endpoint details:

Sample files verification task submission request:

curl -X POST http://dest-host.example.com:9043/api/v1/live-migration/files-verification-tasks \
  -H "Content-Type: application/json" \
  -d '{
    "maxConcurrency": 10,
    "digestAlgorithm": "MD5"
  }'

It supports XXHash32 algorithm too and seed as additional input in the payload.

Sample response:

{
  "taskId": "b8e4f3d2-5c6b-5d9e-0f2g-3b4c5d6e7f8g",
  "statusUrl": "/api/v1/live-migration/files-verification-tasks/b8e4f3d2-5c6b-5d9e-0f2g-3b4c5d6e7f8g"
}

Fetching files verification task status

curl -X GET http://dest-host.example.com:9043/api/v1/live-migration/files-verification-tasks/b8e4f3d2-5c6b-5d9e-0f2g-3b4c5d6e7f8g

Sample response:

{
  "id": "b8e4f3d2-5c6b-5d9e-0f2g-3b4c5d6e7f8g ",
  "digestAlgorithm": "md5",
  "seed": null,
  "state": "COMPLETED",
  "source": "localhost1",
  "port": 9043,
  "filesNotFoundAtSource": 0,
  "filesNotFoundAtDestination": 0,
  "metadataMatched": 379,
  "metadataMismatches": 0,
  "digestMismatches": 0,
  "digestVerificationFailures": 0,
  "filesMatched": 323
}

Also made additional changes to ensure that either data copy task or file verification task can be executed at any point of time.

Comment on lines +64 to +65
String fullURI = seed != null
? String.format("%s?%s=%s&%s=%d", requestURI, DIGEST_ALGORITHM_PARAM, digestAlgorithm, SEED_PARAM, seed)
Copy link
Copy Markdown
Contributor

@yifan-c yifan-c Feb 15, 2026

Choose a reason for hiding this comment

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

One learning from the RestoreJob work is that the custom seed does not provide benefit for data integrity validation, but only adds code complexity. I would just drop the support of custom seed support to simplify the implementation, and use the fixed seed 0, which also makes the client-server communication simpler.
Not strong on removing the seed support, but feel ideal to do so.

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 agree with the code complexity and simplifying code suggestions. I can remove the support for seed for live migration.

@nvharikrishna nvharikrishna force-pushed the 226-lm-file-digests-trunk branch from 9e9da17 to 39cb915 Compare February 20, 2026 19:16
Comment on lines +98 to +103
if (request.maxConcurrency() > liveMigrationConfiguration.maxConcurrentFileRequests())
{
throw new IllegalArgumentException("Invalid maxConcurrency " + request.maxConcurrency() +
". It cannot be greater than " +
liveMigrationConfiguration.maxConcurrentFileRequests());
}
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.

In FilesVerificationTaskManager, it handles maxConcurrency differently. Can you address the inconsistency or the duplication? There seems to be sufficient to have one validation only.

        if (request.maxConcurrency() > maxPossibleConcurrency)
        {
            return Future.failedFuture(
            new LiveMigrationInvalidRequestException("max concurrency can not be more than " + maxPossibleConcurrency));
        }

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.

De-duplicated the checks

{
LOGGER.error("Cannot start a new files verification task for host {} " +
"while another live migration task is in progress.", host);
context.fail(wrapHttpException(FORBIDDEN, throwable.getMessage(), throwable));
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.

Should the status code be 409 Conflict, instead of Forbidden?
Forbidden typically means no permission to perform an action, not the cause here.

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.

CONFLICT makes more sense. Updated it.

* executed asynchronously to validate file integrity between source and destination nodes.
*/
@Singleton
public class FilesVerificationTaskManager
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.

Should FilesVerificationTaskManager and DataCopyTaskManager have a common base class? There are several almost identical methods, e.g. getAllTasks(), getTask() and cancelTask().

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.

Started with inheriting LiveMigrationTaskManager, but had to switch to association. Since FilesVerificationTaskManager and DataCopyTaskManager are different types, Guice creates a separate instance for each, meaning each would get its own currentTasks map (instance to task map in LiveMigrationTaskManager). This breaks the invariant that only one task of any type can be active per instance at a time (line 44). Don't want to use a static mutable map for inheritance. So, used association.

{
return Collections.emptyList();
}
return Collections.singletonList(currentTasks.get(localInstance.id()));
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.

localInstance.id() could potentially be removed at this step due to race condition. Instead, let's get the value at line#99 and return based on whether value is null or not.

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.

Fetching task only once and returning the value.

Comment on lines +331 to +338
if (digestAlgorithm.equalsIgnoreCase(MD5Digest.MD5_ALGORITHM))
{
return Future.succeededFuture(new MD5Digest(digestResponse.digest));
}
else if (digestAlgorithm.equalsIgnoreCase(XXHash32Digest.XXHASH_32_ALGORITHM))
{
return Future.succeededFuture(new XXHash32Digest(digestResponse.digest));
}
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.

Should it be in DigestAlgorithmFactory?

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.

Felt both Digest and DigestResponse are not very relative to DigestAlgorithm. So, did not placed it DigestAlgorithmFactory. Moved it to DigestResponse.

Comment on lines +305 to +326
Future<String> verifyDigest(InstanceFileInfo fileInfo)
{
return getSourceFileDigest(fileInfo)
.compose(digest -> {
String path = localPath(fileInfo.fileUrl, instanceMetadata).toAbsolutePath().toString();
return digestVerifierFactory.verifier(MultiMap.caseInsensitiveMultiMap().addAll(digest.headers()))
.verify(path)
.compose(verified -> Future.succeededFuture(path))
.recover(cause -> Future.failedFuture(
new DigestMismatchException(path, fileInfo.fileUrl, cause)));
})
.onSuccess(filePath -> LOGGER.debug("{} Verified file {}", logPrefix, fileInfo.fileUrl))
.onFailure(cause -> LOGGER.error("{} Failed to verify file {}", logPrefix, fileInfo.fileUrl, cause));
}

private Future<Digest> getSourceFileDigest(InstanceFileInfo fileInfo)
{
return Future.fromCompletionStage(sidecarClient.liveMigrationFileDigestAsync(new SidecarInstanceImpl(source, port),
fileInfo.fileUrl,
request.digestAlgorithm()))
.compose(this::toDigest);
}
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.

It makes 1 http request to source per file to get the digest. According to LiveMigrationConcurrencyLimitHandler, TOO_MANY_REQUESTS can be thrown. There is no retry implemented to handle it, due to SingleInstanceSelectionPolicy + default retry policy. I think you want to add custom retry policy for the applicable requests.

Beside no retry and fail silently, 1 request per file seems to ensure slowness already. Maybe we should revisit the design decision later.

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.

After giving some more thought, I feel the 429 status code (TOO_MANY_REQUESTS) is not appropriate and the 503 (SERVICE UNAVAILABLE) is more appropriate as maxConcurrentFileRequests (used by LiveMigrationConcurrencyLimitHandler) is a server-side concurrency cap shared across all clients, and an individual client hitting this limit hasn't done anything wrong - the server is simply busy with other requests. 503 describes this situation.

SidecarclientProvider is using ExponentialBackoffRetryPolicy as the default retry policy, which does retries. If we suspect that the default retry policy can be overridden, then I can explicitly initiate an instance of ExponentialBackoffRetryPolicy and use it.

What do you think?

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.

5xx status code are considered server side error (unrelated to client activities).
4xx status codes are errors triggered by client.
IMO, 429 makes sense in this scenario. The error happens only when there are enough concurrent requests are issued by client.
The default retry policy does not react on 429 status code and no retries are performed.

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 concurrency limit used here (sidecarConfiguration.liveMigrationConfiguration().maxConcurrentFileRequests()) is global across all clients, not per-client. A client sending its very first request can get rejected because other clients have exhausted the pool.

429 means "the user has sent too many requests" — implying the specific client is at fault, which I feel misleading here. 503 could also mean "the server is unable to handle the request due to a temporary overload" - which I think describes the situation in this case.

If this were a per-client throttle, 429 would be correct. Since maxConcurrentFileRequests is a global limit protecting the source node, I feel 503 is more accurate.

Happy to change it to 429 if you feel differently though! Open to any other suggestions as well.

@nvharikrishna nvharikrishna force-pushed the 226-lm-file-digests-trunk branch from 39cb915 to 639c95a Compare March 5, 2026 04:03
@nvharikrishna nvharikrishna force-pushed the 226-lm-file-digests-trunk branch from 639c95a to 6f81758 Compare March 30, 2026 14:17
@nvharikrishna nvharikrishna force-pushed the 226-lm-file-digests-trunk branch from 6f81758 to cc62f94 Compare April 22, 2026 19:40
Copy link
Copy Markdown
Contributor

@frankgh frankgh left a comment

Choose a reason for hiding this comment

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

Looks good in general. I've added a few comments

Comment thread CHANGES.txt
Objects.requireNonNull(digestAlgorithm, "digestAlgorithm is required");
this.digest = digest;
this.digestAlgorithm = digestAlgorithm;
}
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.

very small NIT, no need to change it

Suggested change
}
this.digest = Objects.requireNonNull(digest, "digest is required");
this.digestAlgorithm = Objects.requireNonNull(digestAlgorithm, "digestAlgorithm is required");

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.

Done

Comment on lines +62 to +66
Objects.requireNonNull(id, "id of files verification task must be specified");
Objects.requireNonNull(state, "state of files verification task must be specified");
this.id = id;
this.digestAlgorithm = digestAlgorithm;
this.state = state;
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.

Small NIT, feel free to ignore:

Suggested change
Objects.requireNonNull(id, "id of files verification task must be specified");
Objects.requireNonNull(state, "state of files verification task must be specified");
this.id = id;
this.digestAlgorithm = digestAlgorithm;
this.state = state;
this.id = Objects.requireNonNull(id, "id of files verification task must be specified");
this.digestAlgorithm = digestAlgorithm;
this.state = Objects.requireNonNull(state, "state of files verification task must be specified");

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.

done

Comment on lines +39 to +43
Objects.requireNonNull(taskId, "taskId cannot be null");
Objects.requireNonNull(statusUrl, "statusUrl cannot be null");

this.taskId = taskId;
this.statusUrl = statusUrl;
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.

Small NIT, feel free to ignore

Suggested change
Objects.requireNonNull(taskId, "taskId cannot be null");
Objects.requireNonNull(statusUrl, "statusUrl cannot be null");
this.taskId = taskId;
this.statusUrl = statusUrl;
this.taskId = Objects.requireNonNull(taskId, "taskId cannot be null");
this.statusUrl = Objects.requireNonNull(statusUrl, "statusUrl cannot be null");

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.

done

Comment on lines +57 to +70
// Verify all fields match
assertThat(deserialized.id()).isEqualTo(original.id());
assertThat(deserialized.digestAlgorithm()).isEqualTo(original.digestAlgorithm());
assertThat(deserialized.state()).isEqualTo(original.state());
assertThat(deserialized.source()).isEqualTo(original.source());
assertThat(deserialized.port()).isEqualTo(original.port());
assertThat(deserialized.filesNotFoundAtSource()).isEqualTo(original.filesNotFoundAtSource());
assertThat(deserialized.filesNotFoundAtDestination()).isEqualTo(original.filesNotFoundAtDestination());
assertThat(deserialized.metadataMatched()).isEqualTo(original.metadataMatched());
assertThat(deserialized.metadataMismatches()).isEqualTo(original.metadataMismatches());
assertThat(deserialized.digestMismatches()).isEqualTo(original.digestMismatches());
assertThat(deserialized.digestVerificationFailures()).isEqualTo(original.digestVerificationFailures());
assertThat(deserialized.filesMatched()).isEqualTo(original.filesMatched());
assertThat(deserialized.isVerificationSuccessful()).isEqualTo(original.isVerificationSuccessful());
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.

Let's move to a helper method and reuse for other tests.

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.

Now it appears only once after using parameterised tests.

try
{
LiveMigrationTask<LiveMigrationFilesVerificationResponse> task = taskManager.getTask(taskId, host);
LOGGER.info("Found live migration task with taskId={} on host={}", taskId, host);
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.

seems too much, can we lower to debug?

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.

done

.port(port)
.vertx(vertx)
.executorPools(executorPools)
.sidecarClient(sidecarClientProvider.get())
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.

What's the side-effect of calling sidecarClientProvider.get()? I don't remember off the top of my head. Can you ensure that calling sidecarClientProvider.get() does not create a new client? We should reuse a single client throughout the application.

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.

SidecarClientProvider is Singleton and .get() is returning its internal initialized client. Only one client instance is used.

*/
public boolean submitTask(int instanceId, LiveMigrationTask<?> newTask)
{
return currentTasks.compute(instanceId, (integer, taskInMap) -> {
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.

NIT

Suggested change
return currentTasks.compute(instanceId, (integer, taskInMap) -> {
return currentTasks.compute(instanceId, (ignored, taskInMap) -> {

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.

done

{
throw new IllegalStateException("No instance found for host: " + currentHost);
}

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 check is not required. instanceFromHost returns non-null results

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.

removed

Comment on lines +141 to +144
if (localInstance == null)
{
throw new IllegalStateException("No instance found for host: " + currentHost);
}
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 necessary to do a null check here

Suggested change
if (localInstance == null)
{
throw new IllegalStateException("No instance found for host: " + currentHost);
}

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.

removed

@nvharikrishna nvharikrishna force-pushed the 226-lm-file-digests-trunk branch from 865a886 to 5147638 Compare May 4, 2026 11:24
Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment

Labels

None yet

Projects

None yet

Development

Successfully merging this pull request may close these issues.

3 participants