Skip to content
Open
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension


Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
Original file line number Diff line number Diff line change
Expand Up @@ -428,6 +428,15 @@ CompletableFuture<Void> dropPartition(
CompletableFuture<KvSnapshotMetadata> getKvSnapshotMetadata(
TableBucket bucket, long snapshotId);

/**
* Creates a new KV snapshot lease with the given ID and duration.
*
* @param leaseId the unique identifier for the lease
* @param leaseDurationMs the lease duration in milliseconds
* @return a {@link KvSnapshotLease} instance representing the created lease
*/
KvSnapshotLease createKvSnapshotLease(String leaseId, long leaseDurationMs);

/**
* Get table lake snapshot info of the given table asynchronously.
*
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -400,6 +400,11 @@ public CompletableFuture<KvSnapshotMetadata> getKvSnapshotMetadata(
.thenApply(ClientRpcMessageUtils::toKvSnapshotMetadata);
}

@Override
public KvSnapshotLease createKvSnapshotLease(String leaseId, long leaseDurationMs) {
return new KvSnapshotLeaseImpl(leaseId, leaseDurationMs, gateway);
}

@Override
public CompletableFuture<LakeSnapshot> getLatestLakeSnapshot(TablePath tablePath) {
GetLatestLakeSnapshotRequest request = new GetLatestLakeSnapshotRequest();
Expand Down
Original file line number Diff line number Diff line change
@@ -0,0 +1,73 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/

package org.apache.fluss.client.admin;

import org.apache.fluss.annotation.PublicEvolving;
import org.apache.fluss.client.metadata.AcquireKvSnapshotLeaseResult;
import org.apache.fluss.metadata.TableBucket;

import java.util.Map;
import java.util.Set;
import java.util.concurrent.CompletableFuture;

/**
* Represents a lease for managing KV snapshots. A lease allows acquiring, renewing, releasing, and
* dropping snapshot references under a time-bound agreement.
*
* @since 0.9
*/
@PublicEvolving
public interface KvSnapshotLease {

/** Returns the unique identifier of this lease. */
String leaseId();

/** Returns the lease duration in milliseconds. */
long leaseDurationMs();

/**
* Acquires snapshots for the specified table buckets under this lease.
*
* @param snapshotIds mapping from table buckets to desired snapshot IDs
* @return a future containing the result of the acquisition
*/
CompletableFuture<AcquireKvSnapshotLeaseResult> acquireSnapshots(
Map<TableBucket, Long> snapshotIds);

/**
* Renews the lease, extending its validity period.
*
* @return a future that completes when the renewal is acknowledged
*/
CompletableFuture<Void> renew();

/**
* Releases snapshots for the specified table buckets, freeing associated resources.
*
* @param bucketsToRelease the set of table buckets whose snapshots should be released
* @return a future that completes when the release is processed
*/
CompletableFuture<Void> releaseSnapshots(Set<TableBucket> bucketsToRelease);

/**
* Drops the entire lease, releasing all held snapshots and invalidating the lease.
*
* @return a future that completes when the lease is fully dropped
*/
CompletableFuture<Void> dropLease();
}
Original file line number Diff line number Diff line change
@@ -0,0 +1,90 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/

package org.apache.fluss.client.admin;

import org.apache.fluss.client.metadata.AcquireKvSnapshotLeaseResult;
import org.apache.fluss.client.utils.ClientRpcMessageUtils;
import org.apache.fluss.metadata.TableBucket;
import org.apache.fluss.rpc.gateway.AdminGateway;
import org.apache.fluss.rpc.messages.AcquireKvSnapshotLeaseRequest;
import org.apache.fluss.rpc.messages.DropKvSnapshotLeaseRequest;

import java.util.Map;
import java.util.Set;
import java.util.concurrent.CompletableFuture;

import static org.apache.fluss.client.utils.ClientRpcMessageUtils.makeAcquireKvSnapshotLeaseRequest;
import static org.apache.fluss.client.utils.ClientRpcMessageUtils.makeReleaseKvSnapshotLeaseRequest;

/** The default implementation of KvSnapshotLease. */
public class KvSnapshotLeaseImpl implements KvSnapshotLease {
private final String leaseId;
private final long leaseDurationMs;
private final AdminGateway gateway;

public KvSnapshotLeaseImpl(String leaseId, long leaseDurationMs, AdminGateway gateway) {
this.leaseId = leaseId;
this.leaseDurationMs = leaseDurationMs;
this.gateway = gateway;
}

@Override
public String leaseId() {
return leaseId;
}

@Override
public long leaseDurationMs() {
return leaseDurationMs;
}

@Override
public CompletableFuture<AcquireKvSnapshotLeaseResult> acquireSnapshots(
Map<TableBucket, Long> snapshotIds) {
if (snapshotIds.isEmpty()) {
throw new IllegalArgumentException(
"The snapshotIds to acquire kv snapshot lease is empty");
}

return gateway.acquireKvSnapshotLease(
makeAcquireKvSnapshotLeaseRequest(leaseId, snapshotIds, leaseDurationMs))
.thenApply(ClientRpcMessageUtils::toAcquireKvSnapshotLeaseResult);
}

@Override
public CompletableFuture<Void> renew() {
AcquireKvSnapshotLeaseRequest request =
new AcquireKvSnapshotLeaseRequest()
.setLeaseId(leaseId)
.setLeaseDurationMs(leaseDurationMs);
return gateway.acquireKvSnapshotLease(request).thenApply(r -> null);
}

@Override
public CompletableFuture<Void> releaseSnapshots(Set<TableBucket> bucketsToRelease) {
return gateway.releaseKvSnapshotLease(
makeReleaseKvSnapshotLeaseRequest(leaseId, bucketsToRelease))
.thenApply(r -> null);
}

@Override
public CompletableFuture<Void> dropLease() {
DropKvSnapshotLeaseRequest request = new DropKvSnapshotLeaseRequest().setLeaseId(leaseId);
return gateway.dropKvSnapshotLease(request).thenApply(r -> null);
}
}
Original file line number Diff line number Diff line change
@@ -0,0 +1,55 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/

package org.apache.fluss.client.metadata;

import org.apache.fluss.annotation.PublicEvolving;
import org.apache.fluss.metadata.TableBucket;

import java.util.Map;
import java.util.Set;

/**
* A class to represent the result of acquire kv snapshot lease. It contains:
*
* <ul>
* <li>A map of unavailable snapshots. Such as the specify snapshotId is not exist for this table
* bucket.
* </ul>
*
* @since 0.9
*/
@PublicEvolving
public class AcquireKvSnapshotLeaseResult {
private final Map<TableBucket, Long> unavailableSnapshots;

public AcquireKvSnapshotLeaseResult(Map<TableBucket, Long> unavailableSnapshots) {
this.unavailableSnapshots = unavailableSnapshots;
}

/**
* Returns the set of buckets that could not be locked (e.g., snapshot ID doesn't exist or has
* already been GC'ed).
*/
public Map<TableBucket, Long> getUnavailableSnapshots() {
return unavailableSnapshots;
}

public Set<TableBucket> getUnavailableTableBucketSet() {
return unavailableSnapshots.keySet();
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -18,12 +18,14 @@
package org.apache.fluss.client.metadata;

import org.apache.fluss.annotation.PublicEvolving;
import org.apache.fluss.metadata.TableBucket;

import javax.annotation.Nullable;

import java.util.Map;
import java.util.OptionalLong;
import java.util.Set;
import java.util.stream.Collectors;

/**
* A class representing the kv snapshots of a table or a partition. It contains multiple snapshots
Expand Down Expand Up @@ -71,6 +73,12 @@ public Set<Integer> getBucketIds() {
return snapshotIds.keySet();
}

public Set<TableBucket> getTableBuckets() {
return snapshotIds.keySet().stream()
.map(bucketId -> new TableBucket(tableId, partitionId, bucketId))
.collect(Collectors.toSet());
}

/**
* Get the latest snapshot id for this kv tablet (bucket), or empty if there are no snapshots.
*/
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -21,6 +21,7 @@
import org.apache.fluss.client.admin.ProducerOffsetsResult;
import org.apache.fluss.client.lookup.LookupBatch;
import org.apache.fluss.client.lookup.PrefixLookupBatch;
import org.apache.fluss.client.metadata.AcquireKvSnapshotLeaseResult;
import org.apache.fluss.client.metadata.KvSnapshotMetadata;
import org.apache.fluss.client.metadata.KvSnapshots;
import org.apache.fluss.client.metadata.LakeSnapshot;
Expand All @@ -43,6 +44,8 @@
import org.apache.fluss.metadata.TableBucket;
import org.apache.fluss.metadata.TableChange;
import org.apache.fluss.metadata.TablePath;
import org.apache.fluss.rpc.messages.AcquireKvSnapshotLeaseRequest;
import org.apache.fluss.rpc.messages.AcquireKvSnapshotLeaseResponse;
import org.apache.fluss.rpc.messages.AlterTableRequest;
import org.apache.fluss.rpc.messages.CreatePartitionRequest;
import org.apache.fluss.rpc.messages.DropPartitionRequest;
Expand All @@ -65,6 +68,8 @@
import org.apache.fluss.rpc.messages.PbDropColumn;
import org.apache.fluss.rpc.messages.PbKeyValue;
import org.apache.fluss.rpc.messages.PbKvSnapshot;
import org.apache.fluss.rpc.messages.PbKvSnapshotLeaseForBucket;
import org.apache.fluss.rpc.messages.PbKvSnapshotLeaseForTable;
import org.apache.fluss.rpc.messages.PbLakeSnapshotForBucket;
import org.apache.fluss.rpc.messages.PbLookupReqForBucket;
import org.apache.fluss.rpc.messages.PbModifyColumn;
Expand All @@ -78,10 +83,12 @@
import org.apache.fluss.rpc.messages.PbRebalanceProgressForTable;
import org.apache.fluss.rpc.messages.PbRemotePathAndLocalFile;
import org.apache.fluss.rpc.messages.PbRenameColumn;
import org.apache.fluss.rpc.messages.PbTableBucket;
import org.apache.fluss.rpc.messages.PrefixLookupRequest;
import org.apache.fluss.rpc.messages.ProduceLogRequest;
import org.apache.fluss.rpc.messages.PutKvRequest;
import org.apache.fluss.rpc.messages.RegisterProducerOffsetsRequest;
import org.apache.fluss.rpc.messages.ReleaseKvSnapshotLeaseRequest;
import org.apache.fluss.rpc.protocol.MergeMode;
import org.apache.fluss.utils.json.DataTypeJsonSerde;
import org.apache.fluss.utils.json.JsonSerdeUtils;
Expand Down Expand Up @@ -411,6 +418,76 @@ public static AlterTableRequest makeAlterTableRequest(
return request;
}

public static AcquireKvSnapshotLeaseRequest makeAcquireKvSnapshotLeaseRequest(
String leaseId, Map<TableBucket, Long> snapshotIds, long leaseDuration) {
AcquireKvSnapshotLeaseRequest request = new AcquireKvSnapshotLeaseRequest();
request.setLeaseId(leaseId).setLeaseDurationMs(leaseDuration);

Map<Long, List<PbKvSnapshotLeaseForBucket>> pbLeaseForTables = new HashMap<>();
for (Map.Entry<TableBucket, Long> entry : snapshotIds.entrySet()) {
TableBucket tableBucket = entry.getKey();
Long snapshotId = entry.getValue();
PbKvSnapshotLeaseForBucket pbLeaseForBucket =
new PbKvSnapshotLeaseForBucket()
.setBucketId(tableBucket.getBucket())
.setSnapshotId(snapshotId);
if (tableBucket.getPartitionId() != null) {
pbLeaseForBucket.setPartitionId(tableBucket.getPartitionId());
}
pbLeaseForTables
.computeIfAbsent(tableBucket.getTableId(), k -> new ArrayList<>())
.add(pbLeaseForBucket);
}

for (Map.Entry<Long, List<PbKvSnapshotLeaseForBucket>> entry :
pbLeaseForTables.entrySet()) {
request.addSnapshotsToLease()
.setTableId(entry.getKey())
.addAllBucketSnapshots(entry.getValue());
}
return request;
}

public static AcquireKvSnapshotLeaseResult toAcquireKvSnapshotLeaseResult(
AcquireKvSnapshotLeaseResponse response) {
Map<TableBucket, Long> unavailableSnapshots = new HashMap<>();
for (PbKvSnapshotLeaseForTable unavailableSnapshot :
response.getUnavailableSnapshotsList()) {
long tableId = unavailableSnapshot.getTableId();
for (PbKvSnapshotLeaseForBucket leaseForBucket :
unavailableSnapshot.getBucketSnapshotsList()) {
TableBucket tableBucket =
new TableBucket(
tableId,
leaseForBucket.hasPartitionId()
? leaseForBucket.getPartitionId()
: null,
leaseForBucket.getBucketId());
unavailableSnapshots.put(tableBucket, leaseForBucket.getSnapshotId());
}
}
return new AcquireKvSnapshotLeaseResult(unavailableSnapshots);
}

public static ReleaseKvSnapshotLeaseRequest makeReleaseKvSnapshotLeaseRequest(
String leaseId, Set<TableBucket> bucketsToRelease) {
ReleaseKvSnapshotLeaseRequest request = new ReleaseKvSnapshotLeaseRequest();
request.setLeaseId(leaseId);

List<PbTableBucket> pbTableBuckets = new ArrayList<>();
for (TableBucket tb : bucketsToRelease) {
PbTableBucket pbBucket =
new PbTableBucket().setTableId(tb.getTableId()).setBucketId(tb.getBucket());
if (tb.getPartitionId() != null) {
pbBucket.setPartitionId(tb.getPartitionId());
}
pbTableBuckets.add(pbBucket);
}

request.addAllBucketsToReleases(pbTableBuckets);
return request;
}

public static Optional<RebalanceProgress> toRebalanceProgress(
ListRebalanceProgressResponse response) {
if (!response.hasRebalanceId()) {
Expand Down
Loading
Loading