Skip to content

Commit

Permalink
[common] Use a unified way to encode keys and assign buckets for data…
Browse files Browse the repository at this point in the history
…lake enabled or not
  • Loading branch information
wuchong committed Feb 22, 2025
1 parent 7a4d71e commit 3bf243d
Show file tree
Hide file tree
Showing 43 changed files with 399 additions and 572 deletions.
Original file line number Diff line number Diff line change
Expand Up @@ -16,17 +16,14 @@

package com.alibaba.fluss.client.lookup;

import com.alibaba.fluss.bucketing.BucketingFunction;
import com.alibaba.fluss.client.metadata.MetadataUpdater;
import com.alibaba.fluss.client.table.getter.PartitionGetter;
import com.alibaba.fluss.lakehouse.DataLakeFormat;
import com.alibaba.fluss.lakehouse.LakeBucketAssigner;
import com.alibaba.fluss.lakehouse.LakeBucketAssignerFactory;
import com.alibaba.fluss.lakehouse.LakeKeyEncoderFactory;
import com.alibaba.fluss.metadata.DataLakeFormat;
import com.alibaba.fluss.metadata.TableBucket;
import com.alibaba.fluss.metadata.TableInfo;
import com.alibaba.fluss.row.InternalRow;
import com.alibaba.fluss.row.decode.RowDecoder;
import com.alibaba.fluss.row.encode.CompactedKeyEncoder;
import com.alibaba.fluss.row.encode.KeyEncoder;
import com.alibaba.fluss.row.encode.ValueDecoder;
import com.alibaba.fluss.types.DataType;
Expand All @@ -37,11 +34,9 @@
import java.util.ArrayList;
import java.util.HashSet;
import java.util.List;
import java.util.Optional;
import java.util.Set;
import java.util.concurrent.CompletableFuture;

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

/**
Expand All @@ -59,11 +54,9 @@ class PrefixKeyLookuper implements Lookuper {
/** Extract bucket key from prefix lookup key row. */
private final KeyEncoder bucketKeyEncoder;

private final BucketingFunction bucketingFunction;
private final int numBuckets;

// won't be null if the datalake format is set in the table
private @Nullable final LakeBucketAssigner lakeBucketAssigner;

/**
* a getter to extract partition from prefix lookup key row, null when it's not a partitioned.
*/
Expand All @@ -86,21 +79,10 @@ public PrefixKeyLookuper(
this.lookupClient = lookupClient;
// the row type of the input lookup row
RowType lookupRowType = tableInfo.getRowType().project(lookupColumnNames);
Optional<DataLakeFormat> optDataLakeFormat = tableInfo.getTableConfig().getDataLakeFormat();

if (optDataLakeFormat.isPresent()) {
DataLakeFormat dataLakeFormat = optDataLakeFormat.get();
this.bucketKeyEncoder =
LakeKeyEncoderFactory.createKeyEncoder(
dataLakeFormat, lookupRowType, tableInfo.getBucketKeys());
this.lakeBucketAssigner =
LakeBucketAssignerFactory.createLakeBucketAssigner(dataLakeFormat, numBuckets);
} else {
this.bucketKeyEncoder =
CompactedKeyEncoder.createKeyEncoder(lookupRowType, tableInfo.getBucketKeys());
this.lakeBucketAssigner = null;
}
DataLakeFormat lakeFormat = tableInfo.getTableConfig().getDataLakeFormat().orElse(null);

this.bucketKeyEncoder = KeyEncoder.of(lookupRowType, tableInfo.getBucketKeys(), lakeFormat);
this.bucketingFunction = BucketingFunction.of(lakeFormat);
this.partitionGetter =
tableInfo.isPartitioned()
? new PartitionGetter(lookupRowType, tableInfo.getPartitionKeys())
Expand Down Expand Up @@ -163,7 +145,7 @@ private void validatePrefixLookup(TableInfo tableInfo, List<String> lookupColumn
@Override
public CompletableFuture<LookupResult> lookup(InternalRow prefixKey) {
byte[] bucketKeyBytes = bucketKeyEncoder.encodeKey(prefixKey);
int bucketId = getBucketId(bucketKeyBytes, lakeBucketAssigner, numBuckets);
int bucketId = bucketingFunction.bucketing(bucketKeyBytes, numBuckets);

Long partitionId = null;
if (partitionGetter != null) {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -16,28 +16,23 @@

package com.alibaba.fluss.client.lookup;

import com.alibaba.fluss.bucketing.BucketingFunction;
import com.alibaba.fluss.client.metadata.MetadataUpdater;
import com.alibaba.fluss.client.table.getter.PartitionGetter;
import com.alibaba.fluss.lakehouse.DataLakeFormat;
import com.alibaba.fluss.lakehouse.LakeBucketAssigner;
import com.alibaba.fluss.lakehouse.LakeBucketAssignerFactory;
import com.alibaba.fluss.lakehouse.LakeKeyEncoderFactory;
import com.alibaba.fluss.metadata.DataLakeFormat;
import com.alibaba.fluss.metadata.TableBucket;
import com.alibaba.fluss.metadata.TableInfo;
import com.alibaba.fluss.row.InternalRow;
import com.alibaba.fluss.row.decode.RowDecoder;
import com.alibaba.fluss.row.encode.CompactedKeyEncoder;
import com.alibaba.fluss.row.encode.KeyEncoder;
import com.alibaba.fluss.row.encode.ValueDecoder;
import com.alibaba.fluss.types.DataType;
import com.alibaba.fluss.types.RowType;

import javax.annotation.Nullable;

import java.util.Optional;
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;

Expand All @@ -58,9 +53,7 @@ class PrimaryKeyLookuper implements Lookuper {
*/
private final KeyEncoder bucketKeyEncoder;

// won't be null if the datalake type is set in the table
private @Nullable final LakeBucketAssigner lakeBucketAssigner;

private final BucketingFunction bucketingFunction;
private final int numBuckets;

/** a getter to extract partition from lookup key row, null when it's not a partitioned. */
Expand All @@ -82,33 +75,16 @@ public PrimaryKeyLookuper(

// the row type of the input lookup row
RowType lookupRowType = tableInfo.getRowType().project(tableInfo.getPrimaryKeys());
Optional<DataLakeFormat> optDataLakeFormat = tableInfo.getTableConfig().getDataLakeFormat();

if (optDataLakeFormat.isPresent()) {
DataLakeFormat dataLakeFormat = optDataLakeFormat.get();
// the encoded primary key is the physical primary key
this.primaryKeyEncoder =
LakeKeyEncoderFactory.createKeyEncoder(
dataLakeFormat, lookupRowType, tableInfo.getPhysicalPrimaryKeys());
this.bucketKeyEncoder =
tableInfo.isDefaultBucketKey()
? primaryKeyEncoder
: LakeKeyEncoderFactory.createKeyEncoder(
dataLakeFormat, lookupRowType, tableInfo.getBucketKeys());
this.lakeBucketAssigner =
LakeBucketAssignerFactory.createLakeBucketAssigner(dataLakeFormat, numBuckets);
} else {
// the encoded primary key is the physical primary key
this.primaryKeyEncoder =
CompactedKeyEncoder.createKeyEncoder(
lookupRowType, tableInfo.getPhysicalPrimaryKeys());
this.bucketKeyEncoder =
tableInfo.isDefaultBucketKey()
? primaryKeyEncoder
: CompactedKeyEncoder.createKeyEncoder(
lookupRowType, tableInfo.getBucketKeys());
this.lakeBucketAssigner = null;
}
DataLakeFormat lakeFormat = tableInfo.getTableConfig().getDataLakeFormat().orElse(null);

// the encoded primary key is the physical primary key
this.primaryKeyEncoder =
KeyEncoder.of(lookupRowType, tableInfo.getPhysicalPrimaryKeys(), lakeFormat);
this.bucketKeyEncoder =
tableInfo.isDefaultBucketKey()
? primaryKeyEncoder
: KeyEncoder.of(lookupRowType, tableInfo.getBucketKeys(), lakeFormat);
this.bucketingFunction = BucketingFunction.of(lakeFormat);

this.partitionGetter =
tableInfo.isPartitioned()
Expand Down Expand Up @@ -138,7 +114,7 @@ public CompletableFuture<LookupResult> lookup(InternalRow lookupKey) {
partitionGetter,
tableInfo.getTablePath(),
metadataUpdater);
int bucketId = getBucketId(bkBytes, lakeBucketAssigner, numBuckets);
int bucketId = bucketingFunction.bucketing(bkBytes, numBuckets);
TableBucket tableBucket = new TableBucket(tableInfo.getTableId(), partitionId, bucketId);
return lookupClient
.lookup(tableBucket, pkBytes)
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -19,15 +19,13 @@
import com.alibaba.fluss.client.metadata.MetadataUpdater;
import com.alibaba.fluss.client.write.WriteRecord;
import com.alibaba.fluss.client.write.WriterClient;
import com.alibaba.fluss.lakehouse.DataLakeFormat;
import com.alibaba.fluss.lakehouse.LakeKeyEncoderFactory;
import com.alibaba.fluss.metadata.DataLakeFormat;
import com.alibaba.fluss.metadata.LogFormat;
import com.alibaba.fluss.metadata.PhysicalTablePath;
import com.alibaba.fluss.metadata.TableInfo;
import com.alibaba.fluss.metadata.TablePath;
import com.alibaba.fluss.row.InternalRow;
import com.alibaba.fluss.row.InternalRow.FieldGetter;
import com.alibaba.fluss.row.encode.CompactedKeyEncoder;
import com.alibaba.fluss.row.encode.IndexedRowEncoder;
import com.alibaba.fluss.row.encode.KeyEncoder;
import com.alibaba.fluss.row.indexed.IndexedRow;
Expand All @@ -36,7 +34,6 @@
import javax.annotation.Nullable;

import java.util.List;
import java.util.Optional;
import java.util.concurrent.CompletableFuture;

/** The writer to write data to the log table. */
Expand All @@ -60,18 +57,8 @@ class AppendWriterImpl extends AbstractTableWriter implements AppendWriter {
this.bucketKeyEncoder = null;
} else {
RowType rowType = tableInfo.getSchema().getRowType();
Optional<DataLakeFormat> optDataLakeFormat =
tableInfo.getTableConfig().getDataLakeFormat();
this.bucketKeyEncoder =
optDataLakeFormat
.map(
dataLakeFormat ->
LakeKeyEncoderFactory.createKeyEncoder(
dataLakeFormat, rowType, bucketKeys))
.orElseGet(
() ->
CompactedKeyEncoder.createKeyEncoder(
rowType, bucketKeys));
DataLakeFormat lakeFormat = tableInfo.getTableConfig().getDataLakeFormat().orElse(null);
this.bucketKeyEncoder = KeyEncoder.of(rowType, bucketKeys, lakeFormat);
}

this.logFormat = tableInfo.getTableConfig().getLogFormat();
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -19,16 +19,14 @@
import com.alibaba.fluss.client.metadata.MetadataUpdater;
import com.alibaba.fluss.client.write.WriteRecord;
import com.alibaba.fluss.client.write.WriterClient;
import com.alibaba.fluss.lakehouse.DataLakeFormat;
import com.alibaba.fluss.lakehouse.LakeKeyEncoderFactory;
import com.alibaba.fluss.metadata.DataLakeFormat;
import com.alibaba.fluss.metadata.KvFormat;
import com.alibaba.fluss.metadata.TableInfo;
import com.alibaba.fluss.metadata.TablePath;
import com.alibaba.fluss.row.BinaryRow;
import com.alibaba.fluss.row.InternalRow;
import com.alibaba.fluss.row.InternalRow.FieldGetter;
import com.alibaba.fluss.row.compacted.CompactedRow;
import com.alibaba.fluss.row.encode.CompactedKeyEncoder;
import com.alibaba.fluss.row.encode.KeyEncoder;
import com.alibaba.fluss.row.encode.RowEncoder;
import com.alibaba.fluss.row.indexed.IndexedRow;
Expand All @@ -38,7 +36,6 @@

import java.util.BitSet;
import java.util.List;
import java.util.Optional;
import java.util.concurrent.CompletableFuture;

/** The writer to write data to the primary key table. */
Expand Down Expand Up @@ -67,30 +64,14 @@ class UpsertWriterImpl extends AbstractTableWriter implements UpsertWriter {
sanityCheck(rowType, tableInfo.getPrimaryKeys(), partialUpdateColumns);

this.targetColumns = partialUpdateColumns;
Optional<DataLakeFormat> optDataLakeFormat = tableInfo.getTableConfig().getDataLakeFormat();

if (optDataLakeFormat.isPresent()) {
DataLakeFormat dataLakeFormat = optDataLakeFormat.get();
// encode primary key using physical primary key
this.primaryKeyEncoder =
LakeKeyEncoderFactory.createKeyEncoder(
dataLakeFormat, rowType, tableInfo.getPhysicalPrimaryKeys());
this.bucketKeyEncoder =
tableInfo.isDefaultBucketKey()
? primaryKeyEncoder
: LakeKeyEncoderFactory.createKeyEncoder(
dataLakeFormat, rowType, tableInfo.getBucketKeys());
} else {
// encode primary key using physical primary key
this.primaryKeyEncoder =
CompactedKeyEncoder.createKeyEncoder(
rowType, tableInfo.getPhysicalPrimaryKeys());
this.bucketKeyEncoder =
tableInfo.isDefaultBucketKey()
? primaryKeyEncoder
: CompactedKeyEncoder.createKeyEncoder(
rowType, tableInfo.getBucketKeys());
}
DataLakeFormat lakeFormat = tableInfo.getTableConfig().getDataLakeFormat().orElse(null);
// encode primary key using physical primary key
this.primaryKeyEncoder =
KeyEncoder.of(rowType, tableInfo.getPhysicalPrimaryKeys(), lakeFormat);
this.bucketKeyEncoder =
tableInfo.isDefaultBucketKey()
? primaryKeyEncoder
: KeyEncoder.of(rowType, tableInfo.getBucketKeys(), lakeFormat);

this.kvFormat = tableInfo.getTableConfig().getKvFormat();
this.rowEncoder = RowEncoder.create(kvFormat, rowType);
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -18,20 +18,16 @@

import com.alibaba.fluss.client.metadata.MetadataUpdater;
import com.alibaba.fluss.client.table.getter.PartitionGetter;
import com.alibaba.fluss.client.write.HashBucketAssigner;
import com.alibaba.fluss.config.ConfigOptions;
import com.alibaba.fluss.exception.IllegalConfigurationException;
import com.alibaba.fluss.exception.PartitionNotExistException;
import com.alibaba.fluss.lakehouse.LakeBucketAssigner;
import com.alibaba.fluss.metadata.PhysicalTablePath;
import com.alibaba.fluss.metadata.TablePath;
import com.alibaba.fluss.row.InternalRow;

import org.slf4j.Logger;
import org.slf4j.LoggerFactory;

import javax.annotation.Nullable;

import java.net.InetSocketAddress;
import java.util.ArrayList;
import java.util.List;
Expand Down Expand Up @@ -134,15 +130,4 @@ public static Long getPartitionId(
metadataUpdater.checkAndUpdatePartitionMetadata(physicalTablePath);
return metadataUpdater.getCluster().getPartitionIdOrElseThrow(physicalTablePath);
}

public static int getBucketId(
byte[] bucketKeyBytes,
@Nullable LakeBucketAssigner lakeBucketAssigner,
int numBuckets) {
if (lakeBucketAssigner == null) {
return HashBucketAssigner.bucketForRowKey(bucketKeyBytes, numBuckets);
} else {
return lakeBucketAssigner.assignBucket(bucketKeyBytes);
}
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -17,33 +17,26 @@
package com.alibaba.fluss.client.write;

import com.alibaba.fluss.annotation.Internal;
import com.alibaba.fluss.utils.MathUtils;
import com.alibaba.fluss.utils.MurmurHashUtils;
import com.alibaba.fluss.utils.Preconditions;

import static com.alibaba.fluss.utils.UnsafeUtils.BYTE_ARRAY_BASE_OFFSET;
import com.alibaba.fluss.bucketing.BucketingFunction;
import com.alibaba.fluss.bucketing.FlussBucketingFunction;

/** Hash bucket assigner. */
@Internal
public class HashBucketAssigner extends StaticBucketAssigner {

private final int numBuckets;
private final BucketingFunction function;

public HashBucketAssigner(int numBuckets) {
this.numBuckets = numBuckets;
this(numBuckets, new FlussBucketingFunction());
}

public int assignBucket(byte[] bucketKeys) {
return bucketForRowKey(bucketKeys, numBuckets);
public HashBucketAssigner(int numBuckets, BucketingFunction function) {
this.numBuckets = numBuckets;
this.function = function;
}

/**
* If the table contains primary key, the default hashing function to choose a bucket from the
* serialized key bytes.
*/
public static int bucketForRowKey(final byte[] key, final int numBuckets) {
Preconditions.checkArgument(key.length != 0, "Assigned key must not be empty!");
int keyHash = MurmurHashUtils.hashUnsafeBytes(key, BYTE_ARRAY_BASE_OFFSET, key.length);
return MathUtils.murmurHash(keyHash) % numBuckets;
public int assignBucket(byte[] bucketKeys) {
return function.bucketing(bucketKeys, numBuckets);
}
}
Loading

0 comments on commit 3bf243d

Please sign in to comment.