|
18 | 18 | package org.apache.fluss.client.admin; |
19 | 19 |
|
20 | 20 | import org.apache.fluss.annotation.PublicEvolving; |
| 21 | +import org.apache.fluss.client.metadata.AcquireKvSnapshotLeaseResult; |
21 | 22 | import org.apache.fluss.client.metadata.KvSnapshotMetadata; |
22 | 23 | import org.apache.fluss.client.metadata.KvSnapshots; |
23 | 24 | import org.apache.fluss.client.metadata.LakeSnapshot; |
24 | | -import org.apache.fluss.client.metadata.RegisterKvSnapshotResult; |
25 | 25 | import org.apache.fluss.cluster.ServerNode; |
26 | 26 | import org.apache.fluss.config.ConfigOptions; |
27 | 27 | import org.apache.fluss.config.cluster.AlterConfig; |
|
34 | 34 | import org.apache.fluss.exception.InvalidPartitionException; |
35 | 35 | import org.apache.fluss.exception.InvalidReplicationFactorException; |
36 | 36 | import org.apache.fluss.exception.InvalidTableException; |
37 | | -import org.apache.fluss.exception.KvSnapshotConsumerNotExistException; |
| 37 | +import org.apache.fluss.exception.KvSnapshotLeaseNotExistException; |
38 | 38 | import org.apache.fluss.exception.KvSnapshotNotExistException; |
39 | 39 | import org.apache.fluss.exception.LakeTableSnapshotNotExistException; |
40 | 40 | import org.apache.fluss.exception.NonPrimaryKeyTableException; |
@@ -405,60 +405,73 @@ CompletableFuture<KvSnapshotMetadata> getKvSnapshotMetadata( |
405 | 405 | TableBucket bucket, long snapshotId); |
406 | 406 |
|
407 | 407 | /** |
408 | | - * Register the kv snapshot consumer of the given tableBucket set asynchronously. After |
409 | | - * registered, the kv snapshot will not be deleted until unregistered or the consumer expired. |
| 408 | + * Acquires a lease for specific KV snapshots of the given tableBuckets asynchronously. |
410 | 409 | * |
411 | | - * <p>For the details bucket to consume, you can call {@link #getLatestKvSnapshots(TablePath)} |
412 | | - * or {@link #getLatestKvSnapshots(TablePath, String)} first. |
| 410 | + * <p>Once acquired, the specified KV snapshots will be protected from garbage collection for |
| 411 | + * the duration of the {@code leaseDuration}. The client must call {@link #renewKvSnapshotLease} |
| 412 | + * periodically to keep the lease alive, or call {@link #releaseKvSnapshotLease} to release the |
| 413 | + * lock early when reading is finished. |
413 | 414 | * |
414 | | - * <p>The following exceptions can be anticipated when calling {@code get()} on returned future. |
| 415 | + * <p>If the lease expires (no renew received within duration), the server is free to delete the |
| 416 | + * snapshot files. |
| 417 | + * |
| 418 | + * <p>The following exceptions can be anticipated when calling {@code get()} on returned future: |
415 | 419 | * |
416 | 420 | * <ul> |
417 | 421 | * <li>{@link TableNotExistException} if the table does not exist. |
418 | | - * <li>{@link PartitionNotExistException} if the partition does not exist |
| 422 | + * <li>{@link PartitionNotExistException} if the partition does not exist. |
419 | 423 | * </ul> |
420 | 424 | * |
421 | | - * @param consumerId the consumer id. |
422 | | - * @param consumeBuckets the tableBuckets to consume, a map from TableBucket to kvSnapshotId. |
423 | | - * @return the result of registering kv snapshot consumer. |
| 425 | + * @param leaseId The unique ID for this lease session (usually a UUID generated by client). |
| 426 | + * @param snapshotIds The snapshots to consume, a map from TableBucket to kvSnapshotId. |
| 427 | + * @param leaseDuration The duration (in milliseconds) for which the snapshots should be kept. |
| 428 | + * @return The result of the acquire operation, containing any buckets that failed to be locked. |
424 | 429 | */ |
425 | | - CompletableFuture<RegisterKvSnapshotResult> registerKvSnapshotConsumer( |
426 | | - String consumerId, Map<TableBucket, Long> consumeBuckets); |
| 430 | + CompletableFuture<AcquireKvSnapshotLeaseResult> acquireKvSnapshotLease( |
| 431 | + String leaseId, Map<TableBucket, Long> snapshotIds, long leaseDuration); |
427 | 432 |
|
428 | 433 | /** |
429 | | - * Unregister the kv snapshot consumer of the given tableBucket set asynchronously. |
| 434 | + * Renews the lease for the given leaseId asynchronously. |
| 435 | + * |
| 436 | + * <p>This acts as a heartbeat. It resets the expiration time for ALL tableBuckets currently |
| 437 | + * associated with this {@code leaseId}. |
430 | 438 | * |
431 | | - * <p>As the following situation happens, unregister operation will be partially ignored or |
432 | | - * completely ignored |
| 439 | + * <p>The following exceptions can be anticipated: |
433 | 440 | * |
434 | 441 | * <ul> |
435 | | - * <li>If the table/partition not exists, the unregister operation will be ignored for this |
436 | | - * table/partition, others will success. |
437 | | - * <li>If the partition bucket/kvSnapshotId not exists, the unregister operation will be |
438 | | - * ignored for this bucket/kvSnapshotId, others will success. |
439 | | - * <li>If the consumerId not exists, the unregister operation will be ignored. |
| 442 | + * <li>{@link KvSnapshotLeaseNotExistException} if the leaseId does not exist or has already |
| 443 | + * expired. |
440 | 444 | * </ul> |
441 | 445 | * |
442 | | - * @param consumerId the consumer id. |
443 | | - * @param bucketsToUnregister the tableBucket to unregister. |
| 446 | + * @param leaseId The lease id to renew. |
| 447 | + * @param leaseDuration The new duration (in milliseconds) from now. |
444 | 448 | */ |
445 | | - CompletableFuture<Void> unregisterKvSnapshotConsumer( |
446 | | - String consumerId, Set<TableBucket> bucketsToUnregister); |
| 449 | + CompletableFuture<Void> renewKvSnapshotLease(String leaseId, long leaseDuration); |
447 | 450 |
|
448 | 451 | /** |
449 | | - * Clear the kv snapshot consumer asynchronously. After cleared, all the registered kv snapshot |
450 | | - * consume info for this consumer will be cleared. |
| 452 | + * Releases the lease for specific tableBuckets asynchronously. |
451 | 453 | * |
452 | | - * <p>The following exceptions can be anticipated when calling {@code get()} on returned future. |
| 454 | + * <p>This is typically called when a client finishes reading a specific bucket (or a batch of |
| 455 | + * buckets) but is still reading others under the same leaseId. |
453 | 456 | * |
454 | | - * <ul> |
455 | | - * <li>{@link TableNotExistException} if the table does not exist. |
456 | | - * <li>{@link KvSnapshotConsumerNotExistException} if the kv snapshot consumer does not exist. |
457 | | - * </ul> |
| 457 | + * <p>If {@code bucketsToRelease} contains all buckets under this leaseId, the lease itself will |
| 458 | + * be removed. |
| 459 | + * |
| 460 | + * @param leaseId The lease id. |
| 461 | + * @param bucketsToRelease The specific tableBuckets to release. |
| 462 | + */ |
| 463 | + CompletableFuture<Void> releaseKvSnapshotLease( |
| 464 | + String leaseId, Set<TableBucket> bucketsToRelease); |
| 465 | + |
| 466 | + /** |
| 467 | + * Drops the entire lease asynchronously. |
| 468 | + * |
| 469 | + * <p>All snapshots locked under this {@code leaseId} will be released immediately. This is |
| 470 | + * equivalent to calling {@link #releaseKvSnapshotLease} with all held buckets. |
458 | 471 | * |
459 | | - * @param consumerId the consumer id. |
| 472 | + * @param leaseId The lease id to drop. |
460 | 473 | */ |
461 | | - CompletableFuture<Void> clearKvSnapshotConsumer(String consumerId); |
| 474 | + CompletableFuture<Void> dropKvSnapshotLease(String leaseId); |
462 | 475 |
|
463 | 476 | /** |
464 | 477 | * Get table lake snapshot info of the given table asynchronously. |
|
0 commit comments