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] Support index lookup for primary key table #222

Merged
merged 2 commits into from
Jan 3, 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
@@ -0,0 +1,47 @@
/*
* 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.Internal;
import com.alibaba.fluss.metadata.TableBucket;

import java.util.concurrent.CompletableFuture;

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

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

public AbstractLookup(TableBucket tableBucket, byte[] key) {
this.tableBucket = tableBucket;
this.key = key;
}

public byte[] key() {
return key;
}

public TableBucket tableBucket() {
return tableBucket;
}

public abstract LookupType lookupType();
swuferhong marked this conversation as resolved.
Show resolved Hide resolved

public abstract CompletableFuture<T> future();
}
Original file line number Diff line number Diff line change
@@ -0,0 +1,58 @@
/*
* 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.Internal;
import com.alibaba.fluss.metadata.TableBucket;

import java.util.ArrayList;
import java.util.List;

/** An abstract lookup batch. */
@Internal
public abstract class AbstractLookupBatch<T> {

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

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

/** Complete the lookup operations using given values . */
public abstract void complete(List<T> values);

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

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

public TableBucket tableBucket() {
return tableBucket;
}

/** Complete the get operations with given exception. */
public void completeExceptionally(Exception exception) {
for (AbstractLookup<T> lookup : lookups) {
lookup.future().completeExceptionally(exception);
}
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -26,26 +26,21 @@
* from, the bytes of the key, and a future for the lookup operation.
*/
@Internal
public class Lookup {
public class Lookup extends AbstractLookup<byte[]> {

private final TableBucket tableBucket;
private final byte[] key;
private final CompletableFuture<byte[]> future;

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

public TableBucket tableBucket() {
return tableBucket;
}

public byte[] key() {
return key;
@Override
public LookupType lookupType() {
return LookupType.LOOKUP;
}

@Override
public CompletableFuture<byte[]> future() {
return future;
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -19,7 +19,6 @@
import com.alibaba.fluss.annotation.Internal;
import com.alibaba.fluss.exception.FlussRuntimeException;
import com.alibaba.fluss.metadata.TableBucket;
import com.alibaba.fluss.rpc.messages.PbValue;

import java.util.ArrayList;
import java.util.List;
Expand Down Expand Up @@ -51,21 +50,21 @@ public TableBucket tableBucket() {
}

/** Complete the lookup operations using given values . */
public void complete(List<PbValue> pbValues) {
public void complete(List<byte[]> values) {
// if the size of return values of lookup operation are not equal to the number of lookups,
// should complete an exception.
if (pbValues.size() != lookups.size()) {
if (values.size() != lookups.size()) {
completeExceptionally(
new FlussRuntimeException(
String.format(
"The number of return values of lookup operation is not equal to the number of "
+ "lookups. Return %d values, but expected %d.",
pbValues.size(), lookups.size())));
values.size(), lookups.size())));
} else {
for (int i = 0; i < pbValues.size(); i++) {
Lookup lookup = lookups.get(i);
PbValue pbValue = pbValues.get(i);
lookup.future().complete(pbValue.hasValues() ? pbValue.getValues() : null);
for (int i = 0; i < values.size(); i++) {
AbstractLookup<byte[]> lookup = lookups.get(i);
// single value.
lookup.future().complete(values.get(i));
}
}
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -29,6 +29,7 @@
import javax.annotation.concurrent.ThreadSafe;

import java.time.Duration;
import java.util.List;
import java.util.concurrent.CompletableFuture;
import java.util.concurrent.ExecutorService;
import java.util.concurrent.Executors;
Expand Down Expand Up @@ -81,6 +82,14 @@ public CompletableFuture<byte[]> lookup(TableBucket tableBucket, byte[] keyBytes
return lookup.future();
}

public CompletableFuture<List<byte[]>> prefixLookup(
long tableId, int bucketId, byte[] keyBytes) {
// TODO prefix lookup support partition table (#266)
PrefixLookup prefixLookup = new PrefixLookup(new TableBucket(tableId, bucketId), keyBytes);
lookupQueue.appendLookup(prefixLookup);
return prefixLookup.future();
}

public void close(Duration timeout) {
LOG.info("Closing lookup client and lookup sender.");

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -36,21 +36,25 @@
class LookupQueue {

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

LookupQueue(Configuration conf) {
this.lookupQueue =
new ArrayBlockingQueue<>(conf.get(ConfigOptions.CLIENT_LOOKUP_QUEUE_SIZE));
this.maxBatchSize = conf.get(ConfigOptions.CLIENT_LOOKUP_MAX_BATCH_SIZE);
this.batchTimeoutNanos = conf.get(ConfigOptions.CLIENT_LOOKUP_BATCH_TIMEOUT).toNanos();
this.closed = false;
}

void appendLookup(Lookup lookup) {
void appendLookup(AbstractLookup<?> lookup) {
if (closed) {
throw new IllegalStateException(
"Can not append lookup operation since the LookupQueue is closed.");
}

try {
lookupQueue.put(lookup);
} catch (InterruptedException e) {
Expand All @@ -63,21 +67,36 @@ boolean hasUnDrained() {
}

/** Drain a batch of {@link Lookup}s from the lookup queue. */
List<Lookup> drain() throws Exception {
List<Lookup> lookups = new ArrayList<>(maxBatchSize);
Lookup firstLookup = lookupQueue.poll(300, TimeUnit.MILLISECONDS);
if (firstLookup != null) {
lookups.add(firstLookup);
lookupQueue.drainTo(lookups, maxBatchSize - 1);
List<AbstractLookup<?>> drain() throws Exception {
final long startNanos = System.nanoTime();
List<AbstractLookup<?>> 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);
if (lookup == null) {
break;
}
lookupOperations.add(lookup);
count++;
int transferred = lookupQueue.drainTo(lookupOperations, maxBatchSize - count);
count += transferred;
if (count >= maxBatchSize) {
break;
}
}
return lookups;
return lookupOperations;
}

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

public void close() {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -14,9 +14,10 @@
* limitations under the License.
*/

package com.alibaba.fluss.client.table;
package com.alibaba.fluss.client.lookup;

import com.alibaba.fluss.annotation.PublicEvolving;
import com.alibaba.fluss.client.table.Table;
import com.alibaba.fluss.row.InternalRow;

import javax.annotation.Nullable;
Expand Down
Loading