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 @@ -214,6 +214,9 @@ public void close() throws Exception {
securityTokenManager.stop();
}

// Close metadata updater to release scheduler and pending requests
metadataUpdater.close();

clientMetricGroup.close();
rpcClient.close();
metricRegistry.closeAsync().get();
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -28,14 +28,16 @@
import org.apache.fluss.row.InternalRow;
import org.apache.fluss.row.encode.KeyEncoder;
import org.apache.fluss.types.RowType;
import org.apache.fluss.utils.ExceptionUtils;
import org.apache.fluss.utils.concurrent.FutureUtils;

import javax.annotation.Nullable;
import javax.annotation.concurrent.NotThreadSafe;

import java.util.Collections;
import java.util.concurrent.CompletableFuture;

import static org.apache.fluss.client.utils.ClientUtils.getPartitionId;
import static org.apache.fluss.client.utils.ClientUtils.getPartitionIdAsync;
import static org.apache.fluss.utils.Preconditions.checkArgument;

/** An implementation of {@link Lookuper} that lookups by primary key. */
Expand Down Expand Up @@ -89,44 +91,68 @@ public PrimaryKeyLookuper(

@Override
public CompletableFuture<LookupResult> lookup(InternalRow lookupKey) {
// encoding the key row using a compacted way consisted with how the key is encoded when put
// a row
byte[] pkBytes = primaryKeyEncoder.encodeKey(lookupKey);
byte[] bkBytes =
bucketKeyEncoder == primaryKeyEncoder
? pkBytes
: bucketKeyEncoder.encodeKey(lookupKey);
Long partitionId = null;
if (partitionGetter != null) {
try {
partitionId =
getPartitionId(
try {
// encoding the key row using a compacted way consisted with how the key is encoded when
// put a row
byte[] pkBytes = primaryKeyEncoder.encodeKey(lookupKey);
byte[] bkBytes =
bucketKeyEncoder == primaryKeyEncoder
? pkBytes
: bucketKeyEncoder.encodeKey(lookupKey);

// If partition getter is present, we need to get partition ID asynchronously
if (partitionGetter != null) {
// Use async version to avoid blocking Netty IO threads
return getPartitionIdAsync(
lookupKey,
partitionGetter,
tableInfo.getTablePath(),
metadataUpdater);
} catch (PartitionNotExistException e) {
return CompletableFuture.completedFuture(new LookupResult(Collections.emptyList()));
metadataUpdater)
.thenCompose(partitionId -> performLookup(partitionId, bkBytes, pkBytes))
.exceptionally(
throwable -> {
// Handle partition not exist exception by returning null result
if (ExceptionUtils.findThrowable(
throwable, PartitionNotExistException.class)
.isPresent()) {
return new LookupResult((InternalRow) null);
}
// Re-throw other exceptions
throw new RuntimeException(throwable);
});
} else {
// No partition, directly perform lookup
return performLookup(null, bkBytes, pkBytes);
}
} catch (Exception e) {
return FutureUtils.failedCompletableFuture(e);
}
}

/**
* Perform the actual lookup operation and process the result.
*
* @param partitionId the partition ID, or null if the table is not partitioned
* @param bkBytes the encoded bucket key bytes
* @param pkBytes the encoded primary key bytes
* @return a CompletableFuture containing the lookup result
*/
private CompletableFuture<LookupResult> performLookup(
@Nullable Long partitionId, byte[] bkBytes, byte[] pkBytes) {
int bucketId = bucketingFunction.bucketing(bkBytes, numBuckets);
TableBucket tableBucket = new TableBucket(tableInfo.getTableId(), partitionId, bucketId);
CompletableFuture<LookupResult> lookupFuture = new CompletableFuture<>();
lookupClient
return lookupClient
.lookup(tableInfo.getTablePath(), tableBucket, pkBytes)
.whenComplete(
(result, error) -> {
if (error != null) {
lookupFuture.completeExceptionally(error);
} else {
handleLookupResponse(
result == null
? Collections.emptyList()
: Collections.singletonList(result),
lookupFuture);
}
.thenCompose(
result -> {
CompletableFuture<LookupResult> resultFuture =
new CompletableFuture<>();
handleLookupResponse(
result == null
? Collections.emptyList()
: Collections.singletonList(result),
resultFuture);
return resultFuture;
});
return lookupFuture;
}
}
Original file line number Diff line number Diff line change
@@ -0,0 +1,56 @@
/*
* 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.cluster.Cluster;
import org.apache.fluss.metadata.PhysicalTablePath;
import org.apache.fluss.metadata.TablePath;
import org.apache.fluss.rpc.gateway.AdminReadOnlyGateway;

import javax.annotation.Nullable;

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

/**
* An abstraction for fetching metadata and rebuilding cluster. This interface allows dependency
* injection for testing purposes.
*/
@FunctionalInterface
public interface ClusterMetadataFetcher {

/**
* Fetch metadata and rebuild cluster asynchronously.
*
* @param gateway the gateway to send request
* @param partialUpdate whether to perform partial update (merge with existing cluster)
* @param originCluster the original cluster to merge with (if partial update)
* @param tablePaths tables to request metadata for
* @param tablePartitions partitions to request metadata for
* @param tablePartitionIds partition ids to request metadata for
* @return a future that completes with the new cluster
*/
CompletableFuture<Cluster> fetch(
AdminReadOnlyGateway gateway,
boolean partialUpdate,
Cluster originCluster,
@Nullable Set<TablePath> tablePaths,
@Nullable Collection<PhysicalTablePath> tablePartitions,
@Nullable Collection<Long> tablePartitionIds);
}
Loading