CASSSIDECAR-226 Adding endpoint for verifying files post data copy during live migration#309
CASSSIDECAR-226 Adding endpoint for verifying files post data copy during live migration#309nvharikrishna wants to merge 3 commits intoapache:trunkfrom
Conversation
| String fullURI = seed != null | ||
| ? String.format("%s?%s=%s&%s=%d", requestURI, DIGEST_ALGORITHM_PARAM, digestAlgorithm, SEED_PARAM, seed) |
There was a problem hiding this comment.
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.
There was a problem hiding this comment.
I agree with the code complexity and simplifying code suggestions. I can remove the support for seed for live migration.
9e9da17 to
39cb915
Compare
| if (request.maxConcurrency() > liveMigrationConfiguration.maxConcurrentFileRequests()) | ||
| { | ||
| throw new IllegalArgumentException("Invalid maxConcurrency " + request.maxConcurrency() + | ||
| ". It cannot be greater than " + | ||
| liveMigrationConfiguration.maxConcurrentFileRequests()); | ||
| } |
There was a problem hiding this comment.
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));
}
| { | ||
| 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)); |
There was a problem hiding this comment.
Should the status code be 409 Conflict, instead of Forbidden?
Forbidden typically means no permission to perform an action, not the cause here.
| * executed asynchronously to validate file integrity between source and destination nodes. | ||
| */ | ||
| @Singleton | ||
| public class FilesVerificationTaskManager |
There was a problem hiding this comment.
Should FilesVerificationTaskManager and DataCopyTaskManager have a common base class? There are several almost identical methods, e.g. getAllTasks(), getTask() and cancelTask().
| { | ||
| return Collections.emptyList(); | ||
| } | ||
| return Collections.singletonList(currentTasks.get(localInstance.id())); |
There was a problem hiding this comment.
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.
| logPrefix, source, port, cause)); | ||
| } | ||
|
|
||
| private @NotNull Future<List<InstanceFileInfo>> compareFilesMeta(List<InstanceFileInfo> localFiles, |
There was a problem hiding this comment.
nit: prefer compareFilesMetadata
| * <p>The verification process consists of three stages: | ||
| * <ol> | ||
| * <li>Fetch file lists from both source and destination instances concurrently</li> | ||
| * <li>Compare file metadata (size, type, modification time) - fails fast on mismatches</li> |
There was a problem hiding this comment.
I do not think compareFilesMeta() (again, prefer compareFilesMetadata()) fail fast. It loops through all files and collects all the errors. Please update the java doc to reflect the actual implementation.
| 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)); | ||
| } |
There was a problem hiding this comment.
Should it be in DigestAlgorithmFactory?
| private Future<LiveMigrationTask<LiveMigrationFilesVerificationResponse>> createVerifier(LiveMigrationFilesVerificationRequest request, | ||
| String source, | ||
| InstanceMetadata localInstanceMetadata) | ||
| { | ||
| String timeUuid = UUIDs.timeBased().toString(); | ||
| return Future.succeededFuture(taskFactory.create(timeUuid, | ||
| source, | ||
| sidecarConfiguration.serviceConfiguration().port(), | ||
| request, | ||
| localInstanceMetadata)); | ||
| } |
There was a problem hiding this comment.
It is a private method and synchronous. The only reason is to chain compose. I think you can avoid wrapping Future, and just return LiveMigrationTask.
| else | ||
| { | ||
| return Future.failedFuture(new LiveMigrationTaskInProgressException( | ||
| "Another files digests verification is in progress for instance=" + localInstanceMetadata.id())); |
There was a problem hiding this comment.
| "Another files digests verification is in progress for instance=" + localInstanceMetadata.id())); | |
| "Another files digest verification is in progress for instance=" + localInstanceMetadata.id())); |
| 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); | ||
| } |
There was a problem hiding this comment.
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.
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:
Here are the endpoint details:
Sample files verification task submission request:
It supports XXHash32 algorithm too and
seedas 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
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.