Skip to content

Commit

Permalink
address jark's comments
Browse files Browse the repository at this point in the history
  • Loading branch information
swuferhong committed Jan 20, 2025
1 parent 7566cb3 commit 86e1285
Show file tree
Hide file tree
Showing 31 changed files with 886 additions and 338 deletions.
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,123 @@
/*
* 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.TableBucket;
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 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;

/**
* 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. */
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,
KeyEncoder primaryKeyEncoder,
KeyEncoder bucketKeyEncoder,
LakeTableBucketAssigner lakeTableBucketAssigner,
@Nullable PartitionGetter partitionGetter,
ValueDecoder kvValueDecoder) {
this.tableInfo = tableInfo;
this.numBuckets = numBuckets;
this.metadataUpdater = metadataUpdater;
this.lookupClient = lookupClient;
this.primaryKeyEncoder = primaryKeyEncoder;
this.bucketKeyEncoder = bucketKeyEncoder;
this.isDataLakeEnable = tableInfo.getTableDescriptor().isDataLakeEnabled();
this.lakeTableBucketAssigner = lakeTableBucketAssigner;
this.partitionGetter = partitionGetter;
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.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);
});
}
}
Original file line number Diff line number Diff line change
@@ -0,0 +1,123 @@
/*
* 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.TableBucket;
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 javax.annotation.Nullable;

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

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

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

private final TableInfo tableInfo;

private final MetadataUpdater metadataUpdater;

private final LookupClient lookupClient;

/** Extract bucket key from prefix lookup key row. */
private final KeyEncoder bucketKeyEncoder;

private final boolean isDataLakeEnable;

private final int numBuckets;

private final LakeTableBucketAssigner lakeTableBucketAssigner;

/**
* a getter to extract partition from prefix 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 FlussPrefixLookuper(
TableInfo tableInfo,
int numBuckets,
MetadataUpdater metadataUpdater,
LookupClient lookupClient,
KeyEncoder bucketKeyEncoder,
LakeTableBucketAssigner lakeTableBucketAssigner,
@Nullable PartitionGetter partitionGetter,
ValueDecoder kvValueDecoder) {
this.tableInfo = tableInfo;
this.numBuckets = numBuckets;
this.metadataUpdater = metadataUpdater;
this.lookupClient = lookupClient;
this.isDataLakeEnable = tableInfo.getTableDescriptor().isDataLakeEnabled();
this.lakeTableBucketAssigner = lakeTableBucketAssigner;
this.bucketKeyEncoder = bucketKeyEncoder;
this.partitionGetter = partitionGetter;
this.kvValueDecoder = kvValueDecoder;
}

@Override
public CompletableFuture<PrefixLookupResult> prefixLookup(InternalRow prefixKey) {
byte[] prefixKeyBytes = bucketKeyEncoder.encode(prefixKey);
int bucketId =
getBucketId(
prefixKeyBytes,
prefixKey,
lakeTableBucketAssigner,
isDataLakeEnable,
numBuckets,
metadataUpdater);

Long partitionId = null;
if (partitionGetter != null) {
partitionId =
getPartitionId(
prefixKey, partitionGetter, tableInfo.getTablePath(), metadataUpdater);
}

TableBucket tableBucket = new TableBucket(tableInfo.getTableId(), partitionId, bucketId);
return lookupClient
.prefixLookup(tableBucket, prefixKeyBytes)
.thenApply(
result -> {
List<InternalRow> rowList = new ArrayList<>();
for (byte[] valueBytes : result) {
rowList.add(
valueBytes == null
? null
: kvValueDecoder.decodeValue(valueBytes).row);
}
return new PrefixLookupResult(rowList);
});
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -30,18 +30,18 @@ public class LookupBatch {
/** The table bucket that the lookup operations should fall into. */
private final TableBucket tableBucket;

private final List<Lookup> lookups;
private final List<LookupQuery> lookups;

public LookupBatch(TableBucket tableBucket) {
this.tableBucket = tableBucket;
this.lookups = new ArrayList<>();
}

public void addLookup(Lookup lookup) {
public void addLookup(LookupQuery lookup) {
lookups.add(lookup);
}

public List<Lookup> lookups() {
public List<LookupQuery> lookups() {
return lookups;
}

Expand All @@ -62,7 +62,7 @@ public void complete(List<byte[]> values) {
values.size(), lookups.size())));
} else {
for (int i = 0; i < values.size(); i++) {
AbstractLookup<byte[]> lookup = lookups.get(i);
AbstractLookupQuery<byte[]> lookup = lookups.get(i);
// single value.
lookup.future().complete(values.get(i));
}
Expand All @@ -71,7 +71,7 @@ public void complete(List<byte[]> values) {

/** Complete the lookup operations with given exception. */
public void completeExceptionally(Exception exception) {
for (Lookup lookup : lookups) {
for (LookupQuery lookup : lookups) {
lookup.future().completeExceptionally(exception);
}
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -77,13 +77,13 @@ private ExecutorService createThreadPool() {
}

public CompletableFuture<byte[]> lookup(TableBucket tableBucket, byte[] keyBytes) {
Lookup lookup = new Lookup(tableBucket, keyBytes);
LookupQuery lookup = new LookupQuery(tableBucket, keyBytes);
lookupQueue.appendLookup(lookup);
return lookup.future();
}

public CompletableFuture<List<byte[]>> prefixLookup(TableBucket tableBucket, byte[] keyBytes) {
PrefixLookup prefixLookup = new PrefixLookup(tableBucket, keyBytes);
PrefixLookupQuery prefixLookup = new PrefixLookupQuery(tableBucket, keyBytes);
lookupQueue.appendLookup(prefixLookup);
return prefixLookup.future();
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -26,11 +26,11 @@
* from, the bytes of the key, and a future for the lookup operation.
*/
@Internal
public class Lookup extends AbstractLookup<byte[]> {
public class LookupQuery extends AbstractLookupQuery<byte[]> {

private final CompletableFuture<byte[]> future;

Lookup(TableBucket tableBucket, byte[] key) {
LookupQuery(TableBucket tableBucket, byte[] key) {
super(tableBucket, key);
this.future = new CompletableFuture<>();
}
Expand Down
Loading

0 comments on commit 86e1285

Please sign in to comment.