Skip to content
Open
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 @@ -21,6 +21,7 @@
import org.apache.fluss.client.table.scanner.batch.BatchScanner;
import org.apache.fluss.client.table.scanner.log.LogScanner;
import org.apache.fluss.client.table.scanner.log.TypedLogScanner;
import org.apache.fluss.lake.source.LakeSource;
import org.apache.fluss.metadata.TableBucket;
import org.apache.fluss.predicate.Predicate;

Expand Down Expand Up @@ -72,6 +73,8 @@ public interface Scan {
*/
Scan filter(@Nullable Predicate predicate);

Scan lakeSource(@Nullable LakeSource lakeSource);

/**
* Creates a {@link LogScanner} to continuously read log data for this scan.
*
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -30,6 +30,7 @@
import org.apache.fluss.client.table.scanner.log.TypedLogScannerImpl;
import org.apache.fluss.config.ConfigOptions;
import org.apache.fluss.exception.FlussRuntimeException;
import org.apache.fluss.lake.source.LakeSource;
import org.apache.fluss.metadata.LogFormat;
import org.apache.fluss.metadata.PartitionInfo;
import org.apache.fluss.metadata.SchemaGetter;
Expand Down Expand Up @@ -60,8 +61,11 @@ public class TableScan implements Scan {
/** The record batch filter to apply. No filter if is null. */
@Nullable private final Predicate recordBatchFilter;

/** The lake source to do the fallback read. No lake source if is null. */
@Nullable private final LakeSource lakeSource;

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

private TableScan(
Expand All @@ -70,19 +74,27 @@ private TableScan(
SchemaGetter schemaGetter,
@Nullable int[] projectedColumns,
@Nullable Integer limit,
@Nullable Predicate recordBatchFilter) {
@Nullable Predicate recordBatchFilter,
@Nullable LakeSource lakeSource) {
this.conn = conn;
this.tableInfo = tableInfo;
this.projectedColumns = projectedColumns;
this.limit = limit;
this.schemaGetter = schemaGetter;
this.recordBatchFilter = recordBatchFilter;
this.lakeSource = lakeSource;
}

@Override
public Scan project(@Nullable int[] projectedColumns) {
return new TableScan(
conn, tableInfo, schemaGetter, projectedColumns, limit, recordBatchFilter);
conn,
tableInfo,
schemaGetter,
projectedColumns,
limit,
recordBatchFilter,
lakeSource);
}

@Override
Expand All @@ -102,18 +114,37 @@ public Scan project(List<String> projectedColumnNames) {
columnIndexes[i] = index;
}
return new TableScan(
conn, tableInfo, schemaGetter, columnIndexes, limit, recordBatchFilter);
conn, tableInfo, schemaGetter, columnIndexes, limit, recordBatchFilter, lakeSource);
}

@Override
public Scan limit(int rowNumber) {
return new TableScan(
conn, tableInfo, schemaGetter, projectedColumns, rowNumber, recordBatchFilter);
conn,
tableInfo,
schemaGetter,
projectedColumns,
rowNumber,
recordBatchFilter,
lakeSource);
}

@Override
public Scan filter(@Nullable Predicate predicate) {
return new TableScan(conn, tableInfo, schemaGetter, projectedColumns, limit, predicate);
return new TableScan(
conn, tableInfo, schemaGetter, projectedColumns, limit, predicate, lakeSource);
}

@Override
public Scan lakeSource(@Nullable LakeSource lakeSource) {
return new TableScan(
conn,
tableInfo,
schemaGetter,
projectedColumns,
limit,
recordBatchFilter,
lakeSource);
}

@Override
Expand All @@ -140,6 +171,7 @@ public LogScanner createLogScanner() {
conn.getMetadataUpdater(),
conn.getClientMetricGroup(),
conn.getOrCreateRemoteFileDownloader(),
lakeSource,
projectedColumns,
schemaGetter,
recordBatchFilter);
Expand Down
Original file line number Diff line number Diff line change
@@ -0,0 +1,121 @@
/*
* 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.log;

import org.apache.fluss.client.table.scanner.ScanRecord;
import org.apache.fluss.exception.FetchException;
import org.apache.fluss.metadata.TableBucket;
import org.apache.fluss.record.LogRecord;
import org.apache.fluss.record.LogRecordReadContext;
import org.apache.fluss.rpc.protocol.ApiError;
import org.apache.fluss.utils.CloseableIterator;

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

/** A completed fetch backed by log records read from a lake snapshot. */
class LakeCompletedFetch extends CompletedFetch {

private final CloseableIterator<LogRecord> records;
private final long endOffset;

private long nextFetchOffset;
private boolean consumed;

LakeCompletedFetch(
TableBucket tableBucket,
CloseableIterator<LogRecord> records,
long fetchOffset,
long endOffset,
long highWatermark,
LogRecordReadContext readContext,
LogScannerStatus logScannerStatus) {
super(
tableBucket,
ApiError.NONE,
0,
highWatermark,
Collections.emptyIterator(),
readContext,
logScannerStatus,
false,
fetchOffset,
NO_FILTERED_END_OFFSET);
this.records = records;
this.nextFetchOffset = fetchOffset;
this.endOffset = endOffset;
}

@Override
boolean isConsumed() {
return consumed;
}

@Override
long nextFetchOffset() {
return nextFetchOffset;
}

@Override
void drain() {
if (!consumed) {
records.close();
consumed = true;
}
super.drain();
}

@Override
public List<ScanRecord> fetchRecords(int maxRecords) {
if (consumed) {
return Collections.emptyList();
}

List<ScanRecord> scanRecords = new ArrayList<>();
try {
for (int i = 0; i < maxRecords && records.hasNext(); ) {
LogRecord record = records.next();
if (record.logOffset() < nextFetchOffset) {
continue;
}
if (record.logOffset() >= endOffset) {
nextFetchOffset = Math.max(nextFetchOffset, endOffset);
drain();
break;
}

scanRecords.add(toScanRecord(record));
nextFetchOffset = record.logOffset() + 1;
i++;
}

if (!records.hasNext()) {
nextFetchOffset = Math.max(nextFetchOffset, endOffset);
drain();
}
} catch (Exception e) {
if (scanRecords.isEmpty()) {
throw new FetchException(
"Received exception when fetching lake records from " + tableBucket, e);
}
}

return scanRecords;
}
}
Loading