Skip to content
New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

[kv] Partition table support PrefixLookup #317

Merged
merged 2 commits into from
Jan 26, 2025
Merged
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 @@ -78,12 +78,12 @@ public Table getTable(TablePath tablePath) {
tablePath,
rpcClient,
metadataUpdater,
this::maybeCreateWriter,
this::maybeCreateLookupClient,
this::getOrCreateWriterClient,
this::getOrCreateLookupClient,
clientMetricGroup);
}

private WriterClient maybeCreateWriter() {
public WriterClient getOrCreateWriterClient() {
if (writerClient == null) {
synchronized (this) {
if (writerClient == null) {
Expand All @@ -94,7 +94,7 @@ private WriterClient maybeCreateWriter() {
return writerClient;
}

private LookupClient maybeCreateLookupClient() {
public LookupClient getOrCreateLookupClient() {
if (lookupClient == null) {
synchronized (this) {
if (lookupClient == null) {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -26,7 +26,7 @@
@Internal
public abstract class AbstractLookupBatch<T> {

protected final List<AbstractLookup<T>> lookups;
protected final List<AbstractLookupQuery<T>> lookups;
private final TableBucket tableBucket;

public AbstractLookupBatch(TableBucket tableBucket) {
Expand All @@ -37,11 +37,11 @@ public AbstractLookupBatch(TableBucket tableBucket) {
/** Complete the lookup operations using given values . */
public abstract void complete(List<T> values);

public void addLookup(AbstractLookup<T> lookup) {
public void addLookup(AbstractLookupQuery<T> lookup) {
lookups.add(lookup);
}

public List<AbstractLookup<T>> lookups() {
public List<AbstractLookupQuery<T>> lookups() {
return lookups;
}

Expand All @@ -51,7 +51,7 @@ public TableBucket tableBucket() {

/** Complete the get operations with given exception. */
public void completeExceptionally(Exception exception) {
for (AbstractLookup<T> lookup : lookups) {
for (AbstractLookupQuery<T> lookup : lookups) {
lookup.future().completeExceptionally(exception);
}
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -23,12 +23,12 @@

/** Abstract Class to represent a lookup operation. */
@Internal
public abstract class AbstractLookup<T> {
public abstract class AbstractLookupQuery<T> {

private final TableBucket tableBucket;
private final byte[] key;

public AbstractLookup(TableBucket tableBucket, byte[] key) {
public AbstractLookupQuery(TableBucket tableBucket, byte[] key) {
this.tableBucket = tableBucket;
this.key = key;
}
Expand Down
Original file line number Diff line number Diff line change
@@ -0,0 +1,152 @@
/*
* Copyright (c) 2024 Alibaba Group Holding Ltd.
*
* Licensed 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 com.alibaba.fluss.client.lookup;

import com.alibaba.fluss.annotation.PublicEvolving;
import com.alibaba.fluss.client.lakehouse.LakeTableBucketAssigner;
import com.alibaba.fluss.client.metadata.MetadataUpdater;
import com.alibaba.fluss.client.table.getter.PartitionGetter;
import com.alibaba.fluss.metadata.Schema;
import com.alibaba.fluss.metadata.TableBucket;
import com.alibaba.fluss.metadata.TableDescriptor;
import com.alibaba.fluss.metadata.TableInfo;
import com.alibaba.fluss.row.InternalRow;
import com.alibaba.fluss.row.encode.KeyEncoder;
import com.alibaba.fluss.row.encode.ValueDecoder;
import com.alibaba.fluss.types.RowType;

import javax.annotation.Nullable;

import java.util.concurrent.CompletableFuture;

import static com.alibaba.fluss.client.utils.ClientUtils.getBucketId;
import static com.alibaba.fluss.client.utils.ClientUtils.getPartitionId;
import static com.alibaba.fluss.utils.Preconditions.checkArgument;

/**
* The default impl of {@link Lookuper}.
*
* @since 0.6
*/
@PublicEvolving
public class FlussLookuper implements Lookuper {

private final TableInfo tableInfo;

private final MetadataUpdater metadataUpdater;

private final LookupClient lookupClient;

private final KeyEncoder primaryKeyEncoder;

/**
* Extract bucket key from lookup key row, use {@link #primaryKeyEncoder} if is default bucket
* key (bucket key = physical primary key).
*/
private final KeyEncoder bucketKeyEncoder;

private final boolean isDataLakeEnable;

private final int numBuckets;

private final LakeTableBucketAssigner lakeTableBucketAssigner;

/** a getter to extract partition from lookup key row, null when it's not a partitioned. */
private @Nullable final PartitionGetter partitionGetter;

/** Decode the lookup bytes to result row. */
private final ValueDecoder kvValueDecoder;

public FlussLookuper(
TableInfo tableInfo,
int numBuckets,
MetadataUpdater metadataUpdater,
LookupClient lookupClient,
ValueDecoder kvValueDecoder) {
checkArgument(
tableInfo.getTableDescriptor().hasPrimaryKey(),
"Log table %s doesn't support lookup",
tableInfo.getTablePath());
this.tableInfo = tableInfo;
this.numBuckets = numBuckets;
this.metadataUpdater = metadataUpdater;
this.lookupClient = lookupClient;

TableDescriptor tableDescriptor = tableInfo.getTableDescriptor();
Schema schema = tableDescriptor.getSchema();
RowType primaryKeyRowType = schema.toRowType().project(schema.getPrimaryKeyIndexes());
this.primaryKeyEncoder =
KeyEncoder.createKeyEncoder(
primaryKeyRowType,
primaryKeyRowType.getFieldNames(),
tableDescriptor.getPartitionKeys());
if (tableDescriptor.isDefaultBucketKey()) {
this.bucketKeyEncoder = primaryKeyEncoder;
} else {
// bucket key doesn't contain partition key, so no need exclude partition keys
this.bucketKeyEncoder =
new KeyEncoder(primaryKeyRowType, tableDescriptor.getBucketKeyIndexes());
}
this.isDataLakeEnable = tableInfo.getTableDescriptor().isDataLakeEnabled();
this.lakeTableBucketAssigner =
new LakeTableBucketAssigner(
primaryKeyRowType, tableDescriptor.getBucketKey(), numBuckets);
this.partitionGetter =
tableDescriptor.isPartitioned()
? new PartitionGetter(primaryKeyRowType, tableDescriptor.getPartitionKeys())
: null;
this.kvValueDecoder = kvValueDecoder;
}

@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.encode(lookupKey);
byte[] bkBytes =
bucketKeyEncoder == primaryKeyEncoder
? pkBytes
: bucketKeyEncoder.encode(lookupKey);
Long partitionId =
partitionGetter == null
? null
: getPartitionId(
lookupKey,
partitionGetter,
tableInfo.getTablePath(),
metadataUpdater);
int bucketId =
getBucketId(
bkBytes,
lookupKey,
lakeTableBucketAssigner,
isDataLakeEnable,
numBuckets,
metadataUpdater);
TableBucket tableBucket = new TableBucket(tableInfo.getTableId(), partitionId, bucketId);
return lookupClient
.lookup(tableBucket, pkBytes)
.thenApply(
valueBytes -> {
InternalRow row =
valueBytes == null
? null
: kvValueDecoder.decodeValue(valueBytes).row;
return new LookupResult(row);
});
}
}
Loading