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 0e8cd0f commit be9ec89
Show file tree
Hide file tree
Showing 26 changed files with 601 additions and 209 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<AbstractLookupOperation<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(AbstractLookupOperation<T> lookup) {
lookups.add(lookup);
}

public List<AbstractLookup<T>> lookups() {
public List<AbstractLookupOperation<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 (AbstractLookupOperation<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 AbstractLookupOperation<T> {

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

public AbstractLookup(TableBucket tableBucket, byte[] key) {
public AbstractLookupOperation(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,146 @@
/*
* 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.client.write.HashBucketAssigner;
import com.alibaba.fluss.metadata.PhysicalTablePath;
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 com.alibaba.fluss.types.RowType;
import com.alibaba.fluss.utils.Preconditions;

import javax.annotation.Nullable;

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

/**
* 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 RowType primaryKeyRowType;

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 @Nullable LakeTableBucketAssigner lakeTableBucketAssigner;

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

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

public FlussPrefixLookuper(
TableInfo tableInfo,
int numBuckets,
MetadataUpdater metadataUpdater,
LookupClient lookupClient,
RowType primaryKeyRowType,
KeyEncoder bucketKeyEncoder,
boolean isDataLakeEnable,
@Nullable LakeTableBucketAssigner lakeTableBucketAssigner,
@Nullable PartitionGetter prefixKeyRowPartitionGetter,
ValueDecoder kvValueDecoder) {
this.tableInfo = tableInfo;
this.numBuckets = numBuckets;
this.metadataUpdater = metadataUpdater;
this.lookupClient = lookupClient;
this.primaryKeyRowType = primaryKeyRowType;
this.isDataLakeEnable = isDataLakeEnable;
this.lakeTableBucketAssigner = lakeTableBucketAssigner;
this.bucketKeyEncoder = bucketKeyEncoder;
this.prefixKeyRowPartitionGetter = prefixKeyRowPartitionGetter;
this.kvValueDecoder = kvValueDecoder;
}

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

Long partitionId = null;
if (prefixKeyRowPartitionGetter != null) {
partitionId = getPartitionId(prefixKey);
}

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);
});
}

private Long getPartitionId(InternalRow row) {
Preconditions.checkNotNull(
prefixKeyRowPartitionGetter, "partitionGetter shouldn't be null.");
String partitionName = prefixKeyRowPartitionGetter.getPartition(row);
PhysicalTablePath physicalTablePath =
PhysicalTablePath.of(tableInfo.getTablePath(), partitionName);
metadataUpdater.checkAndUpdatePartitionMetadata(physicalTablePath);
return metadataUpdater.getCluster().getPartitionIdOrElseThrow(physicalTablePath);
}

private int getBucketId(byte[] keyBytes, InternalRow key) {
if (!isDataLakeEnable) {
return HashBucketAssigner.bucketForRowKey(keyBytes, numBuckets);
} else {
if (lakeTableBucketAssigner == null) {
lakeTableBucketAssigner =
new LakeTableBucketAssigner(
primaryKeyRowType,
tableInfo.getTableDescriptor().getBucketKey(),
numBuckets);
}
return lakeTableBucketAssigner.assignBucket(
keyBytes, key, metadataUpdater.getCluster());
}
}
}
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<LookupOperation> lookups;

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

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

public List<Lookup> lookups() {
public List<LookupOperation> 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);
AbstractLookupOperation<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 (LookupOperation 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);
LookupOperation lookup = new LookupOperation(tableBucket, keyBytes);
lookupQueue.appendLookup(lookup);
return lookup.future();
}

public CompletableFuture<List<byte[]>> prefixLookup(TableBucket tableBucket, byte[] keyBytes) {
PrefixLookup prefixLookup = new PrefixLookup(tableBucket, keyBytes);
PrefixLookupOperation prefixLookup = new PrefixLookupOperation(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 LookupOperation extends AbstractLookupOperation<byte[]> {

private final CompletableFuture<byte[]> future;

Lookup(TableBucket tableBucket, byte[] key) {
LookupOperation(TableBucket tableBucket, byte[] key) {
super(tableBucket, key);
this.future = new CompletableFuture<>();
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -28,16 +28,16 @@
import java.util.concurrent.TimeUnit;

/**
* A queue that buffers the pending lookup operations and provides a list of {@link Lookup} when
* call method {@link #drain()}.
* A queue that buffers the pending lookup operations and provides a list of {@link LookupOperation}
* when call method {@link #drain()}.
*/
@ThreadSafe
@Internal
class LookupQueue {

private volatile boolean closed;
// buffering both the Lookup and PrefixLookup.
private final ArrayBlockingQueue<AbstractLookup<?>> lookupQueue;
private final ArrayBlockingQueue<AbstractLookupOperation<?>> lookupQueue;
private final int maxBatchSize;
private final long batchTimeoutNanos;

Expand All @@ -49,7 +49,7 @@ class LookupQueue {
this.closed = false;
}

void appendLookup(AbstractLookup<?> lookup) {
void appendLookup(AbstractLookupOperation<?> lookup) {
if (closed) {
throw new IllegalStateException(
"Can not append lookup operation since the LookupQueue is closed.");
Expand All @@ -66,18 +66,18 @@ boolean hasUnDrained() {
return !lookupQueue.isEmpty();
}

/** Drain a batch of {@link Lookup}s from the lookup queue. */
List<AbstractLookup<?>> drain() throws Exception {
/** Drain a batch of {@link LookupOperation}s from the lookup queue. */
List<AbstractLookupOperation<?>> drain() throws Exception {
final long startNanos = System.nanoTime();
List<AbstractLookup<?>> lookupOperations = new ArrayList<>(maxBatchSize);
List<AbstractLookupOperation<?>> lookupOperations = new ArrayList<>(maxBatchSize);
int count = 0;
while (true) {
long waitNanos = batchTimeoutNanos - (System.nanoTime() - startNanos);
if (waitNanos <= 0) {
break;
}

AbstractLookup<?> lookup = lookupQueue.poll(waitNanos, TimeUnit.NANOSECONDS);
AbstractLookupOperation<?> lookup = lookupQueue.poll(waitNanos, TimeUnit.NANOSECONDS);
if (lookup == null) {
break;
}
Expand All @@ -92,9 +92,9 @@ List<AbstractLookup<?>> drain() throws Exception {
return lookupOperations;
}

/** Drain all the {@link Lookup}s from the lookup queue. */
List<AbstractLookup<?>> drainAll() {
List<AbstractLookup<?>> lookupOperations = new ArrayList<>(lookupQueue.size());
/** Drain all the {@link LookupOperation}s from the lookup queue. */
List<AbstractLookupOperation<?>> drainAll() {
List<AbstractLookupOperation<?>> lookupOperations = new ArrayList<>(lookupQueue.size());
lookupQueue.drainTo(lookupOperations);
return lookupOperations;
}
Expand Down
Loading

0 comments on commit be9ec89

Please sign in to comment.