Skip to content
Draft
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 @@ -23,7 +23,9 @@
import org.apache.fluss.client.lookup.TableLookup;
import org.apache.fluss.client.metadata.ClientSchemaGetter;
import org.apache.fluss.client.table.scanner.Scan;
import org.apache.fluss.client.table.scanner.SnapshotQuery;
import org.apache.fluss.client.table.scanner.TableScan;
import org.apache.fluss.client.table.scanner.TableSnapshotQuery;
import org.apache.fluss.client.table.writer.Append;
import org.apache.fluss.client.table.writer.TableAppend;
import org.apache.fluss.client.table.writer.TableUpsert;
Expand Down Expand Up @@ -67,6 +69,15 @@ public Scan newScan() {
return new TableScan(conn, tableInfo, schemaGetter);
}

@Override
public SnapshotQuery newSnapshotQuery() {
checkState(
hasPrimaryKey,
"Table %s is not a Primary Key Table and doesn't support SnapshotQuery.",
tablePath);
return new TableSnapshotQuery(conn, tableInfo, schemaGetter);
}

@Override
public Lookup newLookup() {
return new TableLookup(
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -22,6 +22,7 @@
import org.apache.fluss.client.lookup.Lookup;
import org.apache.fluss.client.lookup.Lookuper;
import org.apache.fluss.client.table.scanner.Scan;
import org.apache.fluss.client.table.scanner.SnapshotQuery;
import org.apache.fluss.client.table.writer.Append;
import org.apache.fluss.client.table.writer.AppendWriter;
import org.apache.fluss.client.table.writer.Upsert;
Expand Down Expand Up @@ -55,6 +56,12 @@ public interface Table extends AutoCloseable {
*/
Scan newScan();

/**
* Creates a new {@link SnapshotQuery} for this table to configure and execute a snapshot query
* to read all current data in a table bucket (requires to be a Primary Key Table).
*/
SnapshotQuery newSnapshotQuery();

/**
* Creates a new {@link Lookup} for this table to configure and create a {@link Lookuper} to
* lookup data for this table by primary key or a prefix of primary key.
Expand Down
Original file line number Diff line number Diff line change
@@ -0,0 +1,47 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You 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 org.apache.fluss.client.table.scanner;

import org.apache.fluss.annotation.PublicEvolving;
import org.apache.fluss.metadata.TableBucket;
import org.apache.fluss.row.InternalRow;
import org.apache.fluss.utils.CloseableIterator;

/**
* Used to configure and execute a snapshot query to read all kv data of a primary key table.
*
* @since 0.9
*/
@PublicEvolving
public interface SnapshotQuery {
/**
* Executes the snapshot query to read all current data in the given table bucket.
*
* @param tableBucket the table bucket to read
* @return a closeable iterator of the rows in the table bucket
*/
CloseableIterator<InternalRow> execute(TableBucket tableBucket);

/**
* Executes the snapshot query to read all current data in the table. Everything around
* partitions and buckets will be taken care of from the client.
*
* @return a closeable iterator of the rows in the table
*/
CloseableIterator<InternalRow> execute();
}
Original file line number Diff line number Diff line change
Expand Up @@ -21,6 +21,7 @@
import org.apache.fluss.client.admin.Admin;
import org.apache.fluss.client.metadata.KvSnapshotMetadata;
import org.apache.fluss.client.table.scanner.batch.BatchScanner;
import org.apache.fluss.client.table.scanner.batch.KvBatchScanner;
import org.apache.fluss.client.table.scanner.batch.KvSnapshotBatchScanner;
import org.apache.fluss.client.table.scanner.batch.LimitBatchScanner;
import org.apache.fluss.client.table.scanner.log.LogScanner;
Expand Down Expand Up @@ -123,10 +124,19 @@ public <T> TypedLogScanner<T> createTypedLogScanner(Class<T> pojoClass) {

@Override
public BatchScanner createBatchScanner(TableBucket tableBucket) {
if (tableInfo.hasPrimaryKey()) {
return new KvBatchScanner(
tableInfo,
tableBucket,
schemaGetter,
conn.getMetadataUpdater(),
projectedColumns,
limit == null ? null : (long) limit);
}
if (limit == null) {
throw new UnsupportedOperationException(
String.format(
"Currently, BatchScanner is only available when limit is set. Table: %s, bucket: %s",
"Currently, for log table, BatchScanner is only available when limit is set. Table: %s, bucket: %s",
tableInfo.getTablePath(), tableBucket));
}
return new LimitBatchScanner(
Expand Down
Original file line number Diff line number Diff line change
@@ -0,0 +1,204 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You 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 org.apache.fluss.client.table.scanner;

import org.apache.fluss.client.FlussConnection;
import org.apache.fluss.client.table.scanner.batch.BatchScanner;
import org.apache.fluss.exception.FlussRuntimeException;
import org.apache.fluss.metadata.PartitionInfo;
import org.apache.fluss.metadata.SchemaGetter;
import org.apache.fluss.metadata.TableBucket;
import org.apache.fluss.metadata.TableInfo;
import org.apache.fluss.row.InternalRow;
import org.apache.fluss.utils.CloseableIterator;

import javax.annotation.Nullable;

import java.io.IOException;
import java.time.Duration;
import java.util.ArrayList;
import java.util.Iterator;
import java.util.List;
import java.util.NoSuchElementException;

/** Implementation of {@link SnapshotQuery}. */
public class TableSnapshotQuery implements SnapshotQuery {

private final FlussConnection conn;
private final TableInfo tableInfo;
private final SchemaGetter schemaGetter;

/** The projected fields to do projection. No projection if is null. */
@Nullable private final int[] projectedColumns;

public TableSnapshotQuery(
FlussConnection conn, TableInfo tableInfo, SchemaGetter schemaGetter) {
this(conn, tableInfo, schemaGetter, null);
}

private TableSnapshotQuery(
FlussConnection conn,
TableInfo tableInfo,
SchemaGetter schemaGetter,
@Nullable int[] projectedColumns) {
this.conn = conn;
this.tableInfo = tableInfo;
this.schemaGetter = schemaGetter;
this.projectedColumns = projectedColumns;
}

@Override
public CloseableIterator<InternalRow> execute(TableBucket tableBucket) {
Scan scan = new TableScan(conn, tableInfo, schemaGetter);
if (projectedColumns != null) {
scan = scan.project(projectedColumns);
}
BatchScanner batchScanner = scan.createBatchScanner(tableBucket);
return new BatchScannerIterator(batchScanner);
}

@Override
public CloseableIterator<InternalRow> execute() {
List<TableBucket> buckets = new ArrayList<>();
try {
if (tableInfo.isPartitioned()) {
List<PartitionInfo> partitions =
conn.getAdmin().listPartitionInfos(tableInfo.getTablePath()).get();
for (PartitionInfo partition : partitions) {
for (int i = 0; i < tableInfo.getNumBuckets(); i++) {
buckets.add(
new TableBucket(
tableInfo.getTableId(), partition.getPartitionId(), i));
}
}
} else {
for (int i = 0; i < tableInfo.getNumBuckets(); i++) {
buckets.add(new TableBucket(tableInfo.getTableId(), i));
}
}
} catch (Exception e) {
throw new FlussRuntimeException(
"Failed to list partitions for table " + tableInfo.getTablePath(), e);
}

return new MultiBucketBatchScannerIterator(buckets);
}

private class MultiBucketBatchScannerIterator implements CloseableIterator<InternalRow> {
private final Iterator<TableBucket> bucketIterator;
private CloseableIterator<InternalRow> currentScannerIterator;
private boolean isClosed = false;

private MultiBucketBatchScannerIterator(List<TableBucket> buckets) {
this.bucketIterator = buckets.iterator();
}

@Override
public boolean hasNext() {
if (isClosed) {
return false;
}
while (currentScannerIterator == null || !currentScannerIterator.hasNext()) {
if (currentScannerIterator != null) {
currentScannerIterator.close();
currentScannerIterator = null;
}
if (bucketIterator.hasNext()) {
currentScannerIterator = execute(bucketIterator.next());
} else {
return false;
}
}
return true;
}

@Override
public InternalRow next() {
if (!hasNext()) {
throw new NoSuchElementException();
}
return currentScannerIterator.next();
}

@Override
public void close() {
if (!isClosed) {
if (currentScannerIterator != null) {
currentScannerIterator.close();
}
isClosed = true;
}
}
}

private static class BatchScannerIterator implements CloseableIterator<InternalRow> {
private final BatchScanner scanner;
private Iterator<InternalRow> currentBatch;
private boolean isClosed = false;

private BatchScannerIterator(BatchScanner scanner) {
this.scanner = scanner;
}

@Override
public boolean hasNext() {
ensureBatch();
return currentBatch != null && currentBatch.hasNext();
}

@Override
public InternalRow next() {
if (!hasNext()) {
throw new NoSuchElementException();
}
return currentBatch.next();
}

private void ensureBatch() {
try {
while ((currentBatch == null || !currentBatch.hasNext()) && !isClosed) {
CloseableIterator<InternalRow> it =
scanner.pollBatch(Duration.ofMinutes(1)); // Use a large timeout
if (it == null) {
isClosed = true;
break;
}
if (it.hasNext()) {
currentBatch = it;
} else {
it.close();
}
}
} catch (IOException e) {
throw new RuntimeException("Error polling batch from scanner", e);
}
}

@Override
public void close() {
if (!isClosed) {
try {
scanner.close();
} catch (IOException e) {
throw new RuntimeException("Error closing scanner", e);
}
isClosed = true;
}
}
}
}
Loading