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 @@ -18,6 +18,7 @@
package org.apache.fluss.client.admin;

import org.apache.fluss.annotation.PublicEvolving;
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 Down Expand Up @@ -60,6 +61,8 @@

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

/**
Expand Down Expand Up @@ -400,6 +403,56 @@ CompletableFuture<Void> dropPartition(
CompletableFuture<KvSnapshotMetadata> getKvSnapshotMetadata(
TableBucket bucket, long snapshotId);

/**
* Acquires a lease for specific KV snapshots of the given tableBuckets asynchronously.
*
* <p>Once acquired, the specified KV snapshots will be protected from garbage collection for
* the duration of the {@code leaseDuration}. The client must call {@link
* #releaseKvSnapshotLease} to release the lock early when reading is finished.
*
* <p>If the lease expires (no renew received within duration), the server is free to delete the
* snapshot files.
*
* <p>The following exceptions can be anticipated when calling {@code get()} on returned future:
*
* <ul>
* <li>{@link TableNotExistException} if the table does not exist.
* <li>{@link PartitionNotExistException} if the partition does not exist.
* </ul>
*
* @param leaseId The unique ID for this lease session (usually a UUID generated by client).
* @param snapshotIds The snapshots to lease, a map from TableBucket to kvSnapshotId.
* @param leaseDuration The duration (in milliseconds) for which the snapshots should be kept.
* @return The result of the acquire operation, containing any buckets that failed to be locked.
*/
CompletableFuture<AcquireKvSnapshotLeaseResult> acquireKvSnapshotLease(
String leaseId, Map<TableBucket, Long> snapshotIds, long leaseDuration);

/**
* Releases the lease for specific tableBuckets asynchronously.
*
* <p>This is typically called when a client finishes reading a specific bucket (or a batch of
* buckets) but is still reading others under the same leaseId.
*
* <p>If {@code bucketsToRelease} contains all buckets under this leaseId, the lease itself will
* be removed.
*
* @param leaseId The lease id.
* @param bucketsToRelease The specific tableBuckets to release.
*/
CompletableFuture<Void> releaseKvSnapshotLease(
String leaseId, Set<TableBucket> bucketsToRelease);

/**
* Drops the entire lease asynchronously.
*
* <p>All snapshots locked under this {@code leaseId} will be released immediately. This is
* equivalent to calling {@link #releaseKvSnapshotLease} with all held buckets.
*
* @param leaseId The lease id to drop.
*/
CompletableFuture<Void> dropKvSnapshotLease(String leaseId);

/**
* Get table lake snapshot info of the given table asynchronously.
*
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -17,6 +17,7 @@

package org.apache.fluss.client.admin;

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 Down Expand Up @@ -54,6 +55,7 @@
import org.apache.fluss.rpc.messages.DescribeClusterConfigsRequest;
import org.apache.fluss.rpc.messages.DropAclsRequest;
import org.apache.fluss.rpc.messages.DropDatabaseRequest;
import org.apache.fluss.rpc.messages.DropKvSnapshotLeaseRequest;
import org.apache.fluss.rpc.messages.DropTableRequest;
import org.apache.fluss.rpc.messages.GetDatabaseInfoRequest;
import org.apache.fluss.rpc.messages.GetKvSnapshotMetadataRequest;
Expand Down Expand Up @@ -87,13 +89,16 @@
import java.util.HashMap;
import java.util.List;
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.makeAlterTableRequest;
import static org.apache.fluss.client.utils.ClientRpcMessageUtils.makeCreatePartitionRequest;
import static org.apache.fluss.client.utils.ClientRpcMessageUtils.makeDropPartitionRequest;
import static org.apache.fluss.client.utils.ClientRpcMessageUtils.makeListOffsetsRequest;
import static org.apache.fluss.client.utils.ClientRpcMessageUtils.makePbPartitionSpec;
import static org.apache.fluss.client.utils.ClientRpcMessageUtils.makeReleaseKvSnapshotLeaseRequest;
import static org.apache.fluss.client.utils.ClientRpcMessageUtils.toConfigEntries;
import static org.apache.fluss.client.utils.MetadataUtils.sendMetadataRequestAndRebuildCluster;
import static org.apache.fluss.rpc.util.CommonRpcMessageUtils.toAclBindings;
Expand Down Expand Up @@ -377,6 +382,33 @@ public CompletableFuture<KvSnapshotMetadata> getKvSnapshotMetadata(
.thenApply(ClientRpcMessageUtils::toKvSnapshotMetadata);
}

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

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

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

@Override
public CompletableFuture<Void> dropKvSnapshotLease(String leaseId) {
DropKvSnapshotLeaseRequest request = new DropKvSnapshotLeaseRequest().setLeaseId(leaseId);
return gateway.dropKvSnapshotLease(request).thenApply(r -> null);
}

@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,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 @@ -20,6 +20,7 @@
import org.apache.fluss.client.admin.OffsetSpec;
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 @@ -37,6 +38,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 @@ -50,10 +53,13 @@
import org.apache.fluss.rpc.messages.MetadataRequest;
import org.apache.fluss.rpc.messages.PbAddColumn;
import org.apache.fluss.rpc.messages.PbAlterConfig;
import org.apache.fluss.rpc.messages.PbBucket;
import org.apache.fluss.rpc.messages.PbDescribeConfig;
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 @@ -66,6 +72,7 @@
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.ReleaseKvSnapshotLeaseRequest;
import org.apache.fluss.utils.json.DataTypeJsonSerde;
import org.apache.fluss.utils.json.JsonSerdeUtils;

Expand Down Expand Up @@ -370,6 +377,75 @@ 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).setLeaseDuration(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.addTableLeaseReq()
.setTableId(entry.getKey())
.addAllBucketsReqs(entry.getValue());
}
return request;
}

public static AcquireKvSnapshotLeaseResult toAcquireKvSnapshotLeaseResult(
AcquireKvSnapshotLeaseResponse response) {
Map<TableBucket, Long> unavailableSnapshots = new HashMap<>();
for (PbKvSnapshotLeaseForTable leaseForTable : response.getTablesLeaseResList()) {
long tableId = leaseForTable.getTableId();
for (PbKvSnapshotLeaseForBucket leaseForBucket : leaseForTable.getBucketsReqsList()) {
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);

Map<Long, List<PbBucket>> pbLeasedTable = new HashMap<>();
for (TableBucket tb : bucketsToRelease) {
PbBucket pbBucket = new PbBucket().setBucketId(tb.getBucket());
if (tb.getPartitionId() != null) {
pbBucket.setPartitionId(tb.getPartitionId());
}
pbLeasedTable.computeIfAbsent(tb.getTableId(), k -> new ArrayList<>()).add(pbBucket);
}

for (Map.Entry<Long, List<PbBucket>> entry : pbLeasedTable.entrySet()) {
request.addReleaseTable().setTableId(entry.getKey()).addAllBuckets(entry.getValue());
}
return request;
}

public static List<PartitionInfo> toPartitionInfos(ListPartitionInfosResponse response) {
return response.getPartitionsInfosList().stream()
.map(
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -32,13 +32,13 @@ public class TestingClientSchemaGetter extends ClientSchemaGetter {
public TestingClientSchemaGetter(
TablePath tablePath,
SchemaInfo latestSchemaInfo,
TestingMetadataUpdater metadataUpdater) {
TestingMetadataUpdater metadataUpdater,
Configuration conf) {
super(
tablePath,
latestSchemaInfo,
new FlussAdmin(
RpcClient.create(
new Configuration(), TestingClientMetricGroup.newInstance(), false),
RpcClient.create(conf, TestingClientMetricGroup.newInstance(), false),
metadataUpdater));
}

Expand Down
Loading
Loading