From 95026495f8ad8a37f20fcc334b6f4b08bcb116b9 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?=E7=99=BD=E9=B5=BA?= Date: Sat, 28 Mar 2026 21:26:24 +0800 Subject: [PATCH 1/2] [server] Add server-side filter execution and client-side integration --- .../fluss/client/table/scanner/Scan.java | 14 + .../fluss/client/table/scanner/TableScan.java | 28 +- .../table/scanner/log/CompletedFetch.java | 28 +- .../scanner/log/DefaultCompletedFetch.java | 3 +- .../table/scanner/log/LogFetchCollector.java | 13 +- .../client/table/scanner/log/LogFetcher.java | 32 +- .../table/scanner/log/LogScannerImpl.java | 5 +- .../scanner/log/RemoteCompletedFetch.java | 3 +- .../scanner/log/LogFetcherFilterTest.java | 344 +++++++++++ .../table/scanner/log/LogFetcherITCase.java | 4 + .../table/scanner/log/LogFetcherTest.java | 24 +- .../fluss/record/FileLogProjection.java | 115 ++++ .../apache/fluss/record/FileLogRecords.java | 17 +- .../fluss/record/LogRecordReadContext.java | 41 +- .../record/bytesview/MultiBytesView.java | 24 + .../fluss/record/FileLogInputStreamTest.java | 182 ++++++ .../fluss/record/FileLogProjectionTest.java | 506 ++++++++++++++++ .../record/LogRecordBatchIteratorTest.java | 82 +++ .../fluss/record/RecordBatchFilterTest.java | 463 +++++++++++++++ .../rpc/entity/FetchLogResultForBucket.java | 29 +- .../fluss/rpc/util/CommonRpcMessageUtils.java | 14 +- fluss-rpc/src/main/proto/FlussApi.proto | 6 + .../fluss/server/kv/KvRecoverHelper.java | 3 + .../fluss/server/log/FetchDataInfo.java | 41 ++ .../apache/fluss/server/log/FetchParams.java | 30 +- .../fluss/server/log/FetchParamsBuilder.java | 73 +++ .../apache/fluss/server/log/FilterInfo.java | 39 ++ .../org/apache/fluss/server/log/LocalLog.java | 76 ++- .../apache/fluss/server/log/LogSegment.java | 262 ++++++++- .../apache/fluss/server/log/LogTablet.java | 17 +- .../server/log/PredicateSchemaResolver.java | 136 +++++ .../apache/fluss/server/replica/Replica.java | 82 ++- .../fluss/server/replica/ReplicaManager.java | 18 +- .../fluss/server/tablet/TabletService.java | 25 +- .../server/utils/ServerRpcMessageUtils.java | 27 + .../server/kv/KvTabletMergeModeTest.java | 10 +- .../apache/fluss/server/kv/KvTabletTest.java | 10 +- .../fluss/server/log/FetchParamsTest.java | 10 + .../apache/fluss/server/log/LocalLogTest.java | 347 ++++++++++- .../fluss/server/log/LogManagerTest.java | 2 +- .../fluss/server/log/LogSegmentTest.java | 543 ++++++++++++++++++ .../fluss/server/log/LogTabletTest.java | 2 +- .../log/PredicateSchemaResolverTest.java | 231 ++++++++ .../replica/fetcher/ReplicaFetcherITCase.java | 4 +- 44 files changed, 3869 insertions(+), 96 deletions(-) create mode 100644 fluss-client/src/test/java/org/apache/fluss/client/table/scanner/log/LogFetcherFilterTest.java create mode 100644 fluss-common/src/test/java/org/apache/fluss/record/LogRecordBatchIteratorTest.java create mode 100644 fluss-common/src/test/java/org/apache/fluss/record/RecordBatchFilterTest.java create mode 100644 fluss-server/src/main/java/org/apache/fluss/server/log/FetchParamsBuilder.java create mode 100644 fluss-server/src/main/java/org/apache/fluss/server/log/FilterInfo.java create mode 100644 fluss-server/src/main/java/org/apache/fluss/server/log/PredicateSchemaResolver.java create mode 100644 fluss-server/src/test/java/org/apache/fluss/server/log/PredicateSchemaResolverTest.java diff --git a/fluss-client/src/main/java/org/apache/fluss/client/table/scanner/Scan.java b/fluss-client/src/main/java/org/apache/fluss/client/table/scanner/Scan.java index 2b7a5e7329..430ee6b8aa 100644 --- a/fluss-client/src/main/java/org/apache/fluss/client/table/scanner/Scan.java +++ b/fluss-client/src/main/java/org/apache/fluss/client/table/scanner/Scan.java @@ -22,6 +22,7 @@ import org.apache.fluss.client.table.scanner.log.LogScanner; import org.apache.fluss.client.table.scanner.log.TypedLogScanner; import org.apache.fluss.metadata.TableBucket; +import org.apache.fluss.predicate.Predicate; import javax.annotation.Nullable; @@ -60,6 +61,19 @@ public interface Scan { */ Scan limit(int rowNumber); + /** + * Returns a new scan from this that will apply the given predicate filter. + * + *

Note: the filter currently only supports record batch level filtering for log scanners, + * not row level filtering. The computing engine still needs to perform secondary filtering on + * the results. Batch scanners do not support filter pushdown. + * + * @param predicate the predicate to apply for record batch level filtering + */ + default Scan filter(@Nullable Predicate predicate) { + throw new UnsupportedOperationException("Filter pushdown is not supported by this Scan."); + } + /** * Creates a {@link LogScanner} to continuously read log data for this scan. * diff --git a/fluss-client/src/main/java/org/apache/fluss/client/table/scanner/TableScan.java b/fluss-client/src/main/java/org/apache/fluss/client/table/scanner/TableScan.java index 640f2541a3..5b7718c7ad 100644 --- a/fluss-client/src/main/java/org/apache/fluss/client/table/scanner/TableScan.java +++ b/fluss-client/src/main/java/org/apache/fluss/client/table/scanner/TableScan.java @@ -34,6 +34,7 @@ import org.apache.fluss.metadata.SchemaGetter; import org.apache.fluss.metadata.TableBucket; import org.apache.fluss.metadata.TableInfo; +import org.apache.fluss.predicate.Predicate; import org.apache.fluss.types.RowType; import javax.annotation.Nullable; @@ -51,11 +52,15 @@ public class TableScan implements Scan { /** The projected fields to do projection. No projection if is null. */ @Nullable private final int[] projectedColumns; + /** The limited row number to read. No limit if is null. */ @Nullable private final Integer limit; + /** The record batch filter to apply. No filter if is null. */ + @Nullable private final Predicate recordBatchFilter; + public TableScan(FlussConnection conn, TableInfo tableInfo, SchemaGetter schemaGetter) { - this(conn, tableInfo, schemaGetter, null, null); + this(conn, tableInfo, schemaGetter, null, null, null); } private TableScan( @@ -63,17 +68,20 @@ private TableScan( TableInfo tableInfo, SchemaGetter schemaGetter, @Nullable int[] projectedColumns, - @Nullable Integer limit) { + @Nullable Integer limit, + @Nullable Predicate recordBatchFilter) { this.conn = conn; this.tableInfo = tableInfo; this.projectedColumns = projectedColumns; this.limit = limit; this.schemaGetter = schemaGetter; + this.recordBatchFilter = recordBatchFilter; } @Override public Scan project(@Nullable int[] projectedColumns) { - return new TableScan(conn, tableInfo, schemaGetter, projectedColumns, limit); + return new TableScan( + conn, tableInfo, schemaGetter, projectedColumns, limit, recordBatchFilter); } @Override @@ -92,12 +100,19 @@ public Scan project(List projectedColumnNames) { } columnIndexes[i] = index; } - return new TableScan(conn, tableInfo, schemaGetter, columnIndexes, limit); + return new TableScan( + conn, tableInfo, schemaGetter, columnIndexes, limit, recordBatchFilter); } @Override public Scan limit(int rowNumber) { - return new TableScan(conn, tableInfo, schemaGetter, projectedColumns, rowNumber); + return new TableScan( + conn, tableInfo, schemaGetter, projectedColumns, rowNumber, recordBatchFilter); + } + + @Override + public Scan filter(@Nullable Predicate predicate) { + return new TableScan(conn, tableInfo, schemaGetter, projectedColumns, limit, predicate); } @Override @@ -116,7 +131,8 @@ public LogScanner createLogScanner() { conn.getClientMetricGroup(), conn.getOrCreateRemoteFileDownloader(), projectedColumns, - schemaGetter); + schemaGetter, + recordBatchFilter); } @Override diff --git a/fluss-client/src/main/java/org/apache/fluss/client/table/scanner/log/CompletedFetch.java b/fluss-client/src/main/java/org/apache/fluss/client/table/scanner/log/CompletedFetch.java index 8f29f3ef34..fe18a457e6 100644 --- a/fluss-client/src/main/java/org/apache/fluss/client/table/scanner/log/CompletedFetch.java +++ b/fluss-client/src/main/java/org/apache/fluss/client/table/scanner/log/CompletedFetch.java @@ -27,7 +27,6 @@ import org.apache.fluss.record.LogRecordReadContext; import org.apache.fluss.row.GenericRow; import org.apache.fluss.row.InternalRow; -import org.apache.fluss.rpc.messages.FetchLogRequest; import org.apache.fluss.rpc.protocol.ApiError; import org.apache.fluss.utils.CloseableIterator; @@ -40,6 +39,8 @@ import java.util.Iterator; import java.util.List; +import static org.apache.fluss.utils.Preconditions.checkArgument; + /** * {@link CompletedFetch} represents the result that was returned from the tablet server via a * {@link FetchLogRequest}, which can be a {@link LogRecordBatch} or remote log segments path. It @@ -53,6 +54,7 @@ abstract class CompletedFetch { final ApiError error; final int sizeInBytes; final long highWatermark; + private final long fetchOffset; private final boolean isCheckCrcs; private final Iterator batches; @@ -79,7 +81,8 @@ public CompletedFetch( LogRecordReadContext readContext, LogScannerStatus logScannerStatus, boolean isCheckCrcs, - long fetchOffset) { + long fetchOffset, + long nextFetchOffset) { this.tableBucket = tableBucket; this.error = error; this.sizeInBytes = sizeInBytes; @@ -88,8 +91,21 @@ public CompletedFetch( this.readContext = readContext; this.isCheckCrcs = isCheckCrcs; this.logScannerStatus = logScannerStatus; - this.nextFetchOffset = fetchOffset; this.selectedFieldGetters = readContext.getSelectedFieldGetters(); + this.fetchOffset = fetchOffset; + checkArgument( + nextFetchOffset == -1 || nextFetchOffset >= fetchOffset, + "nextFetchOffset (%s) must be -1 or >= fetchOffset (%s) for bucket %s.", + nextFetchOffset, + fetchOffset, + tableBucket); + checkArgument( + nextFetchOffset == -1 || sizeInBytes == 0, + "When nextFetchOffset is set (%s), records must be empty (sizeInBytes=%s) for bucket %s.", + nextFetchOffset, + sizeInBytes, + tableBucket); + this.nextFetchOffset = nextFetchOffset != -1 ? nextFetchOffset : fetchOffset; } // TODO: optimize this to avoid deep copying the record. @@ -113,6 +129,10 @@ boolean isInitialized() { return initialized; } + long fetchOffset() { + return fetchOffset; + } + long nextFetchOffset() { return nextFetchOffset; } @@ -178,6 +198,8 @@ public List fetchRecords(int maxRecords) { ScanRecord record = toScanRecord(lastRecord); scanRecords.add(record); recordsRead++; + // Update nextFetchOffset based on the current record + // This will be overridden by batch-level nextLogOffset when batch is complete nextFetchOffset = lastRecord.logOffset() + 1; cachedRecordException = null; } diff --git a/fluss-client/src/main/java/org/apache/fluss/client/table/scanner/log/DefaultCompletedFetch.java b/fluss-client/src/main/java/org/apache/fluss/client/table/scanner/log/DefaultCompletedFetch.java index 95078bf36e..10c0298bad 100644 --- a/fluss-client/src/main/java/org/apache/fluss/client/table/scanner/log/DefaultCompletedFetch.java +++ b/fluss-client/src/main/java/org/apache/fluss/client/table/scanner/log/DefaultCompletedFetch.java @@ -46,6 +46,7 @@ public DefaultCompletedFetch( readContext, logScannerStatus, isCheckCrc, - fetchOffset); + fetchOffset, + fetchLogResultForBucket.getFilteredEndOffset()); } } diff --git a/fluss-client/src/main/java/org/apache/fluss/client/table/scanner/log/LogFetchCollector.java b/fluss-client/src/main/java/org/apache/fluss/client/table/scanner/log/LogFetchCollector.java index 34a3ec86e6..69791373a6 100644 --- a/fluss-client/src/main/java/org/apache/fluss/client/table/scanner/log/LogFetchCollector.java +++ b/fluss-client/src/main/java/org/apache/fluss/client/table/scanner/log/LogFetchCollector.java @@ -154,9 +154,12 @@ private List fetchRecords(CompletedFetch nextInLineFetch, int maxRec "Ignoring fetched records for {} at offset {} since the current offset is null which means the " + "bucket has been unsubscribe.", tb, - nextInLineFetch.nextFetchOffset()); + nextInLineFetch.fetchOffset()); } else { - if (nextInLineFetch.nextFetchOffset() == offset) { + // When server-side filtering is enabled, the CompletedFetch's nextFetchOffset + // may have been advanced by filteredEndOffset (all batches filtered out). + // The fetchOffset still matches the original request offset. + if (nextInLineFetch.fetchOffset() == offset) { List records = nextInLineFetch.fetchRecords(maxRecords); LOG.trace( "Returning {} fetched records at offset {} for assigned bucket {}.", @@ -180,7 +183,7 @@ private List fetchRecords(CompletedFetch nextInLineFetch, int maxRec LOG.warn( "Ignoring fetched records for {} at offset {} since the current offset is {}", nextInLineFetch.tableBucket, - nextInLineFetch.nextFetchOffset(), + nextInLineFetch.fetchOffset(), offset); } } @@ -215,7 +218,7 @@ private List fetchRecords(CompletedFetch nextInLineFetch, int maxRec private @Nullable CompletedFetch handleInitializeSuccess(CompletedFetch completedFetch) { TableBucket tb = completedFetch.tableBucket; - long fetchOffset = completedFetch.nextFetchOffset(); + long fetchOffset = completedFetch.fetchOffset(); // we are interested in this fetch only if the beginning offset matches the // current consumed position. @@ -249,7 +252,7 @@ private List fetchRecords(CompletedFetch nextInLineFetch, int maxRec private void handleInitializeErrors( CompletedFetch completedFetch, Errors error, String errorMessage) { TableBucket tb = completedFetch.tableBucket; - long fetchOffset = completedFetch.nextFetchOffset(); + long fetchOffset = completedFetch.fetchOffset(); if (error == Errors.NOT_LEADER_OR_FOLLOWER || error == Errors.LOG_STORAGE_EXCEPTION || error == Errors.KV_STORAGE_EXCEPTION diff --git a/fluss-client/src/main/java/org/apache/fluss/client/table/scanner/log/LogFetcher.java b/fluss-client/src/main/java/org/apache/fluss/client/table/scanner/log/LogFetcher.java index b5a03a1caf..92f6802f87 100644 --- a/fluss-client/src/main/java/org/apache/fluss/client/table/scanner/log/LogFetcher.java +++ b/fluss-client/src/main/java/org/apache/fluss/client/table/scanner/log/LogFetcher.java @@ -37,6 +37,7 @@ import org.apache.fluss.metadata.TableInfo; import org.apache.fluss.metadata.TablePartition; import org.apache.fluss.metadata.TablePath; +import org.apache.fluss.predicate.Predicate; import org.apache.fluss.record.LogRecordReadContext; import org.apache.fluss.record.LogRecords; import org.apache.fluss.record.MemoryLogRecords; @@ -52,6 +53,8 @@ import org.apache.fluss.rpc.messages.PbFetchLogRespForTable; import org.apache.fluss.rpc.protocol.ApiError; import org.apache.fluss.rpc.protocol.Errors; +import org.apache.fluss.rpc.util.PredicateMessageUtils; +import org.apache.fluss.types.RowType; import org.apache.fluss.utils.IOUtils; import org.apache.fluss.utils.Projection; @@ -93,6 +96,10 @@ public class LogFetcher implements Closeable { // bytes from remote file. private final LogRecordReadContext remoteReadContext; @Nullable private final Projection projection; + @Nullable private final Predicate recordBatchFilter; + @Nullable private final org.apache.fluss.rpc.messages.PbPredicate cachedPbPredicate; + private final RowType fullRowType; + private final int schemaId; private final int maxFetchBytes; private final int maxBucketFetchBytes; private final int minFetchBytes; @@ -115,6 +122,7 @@ public class LogFetcher implements Closeable { public LogFetcher( TableInfo tableInfo, @Nullable Projection projection, + @Nullable Predicate recordBatchFilter, LogScannerStatus logScannerStatus, Configuration conf, MetadataUpdater metadataUpdater, @@ -128,6 +136,14 @@ public LogFetcher( this.remoteReadContext = LogRecordReadContext.createReadContext(tableInfo, true, projection, schemaGetter); this.projection = projection; + this.recordBatchFilter = recordBatchFilter; + this.cachedPbPredicate = + recordBatchFilter != null + ? PredicateMessageUtils.toPbPredicate( + recordBatchFilter, tableInfo.getRowType()) + : null; + this.fullRowType = tableInfo.getRowType(); + this.schemaId = tableInfo.getSchemaId(); this.logScannerStatus = logScannerStatus; this.maxFetchBytes = (int) conf.get(ConfigOptions.CLIENT_SCANNER_LOG_FETCH_MAX_BYTES).getBytes(); @@ -386,9 +402,10 @@ private synchronized void handleFetchLogResponse( } else { LogRecords logRecords = fetchResultForBucket.recordsOrEmpty(); if (!MemoryLogRecords.EMPTY.equals(logRecords) - || fetchResultForBucket.getErrorCode() != Errors.NONE.code()) { - // In oder to not signal notEmptyCondition, add completed - // fetch to buffer until log records is not empty. + || fetchResultForBucket.getErrorCode() != Errors.NONE.code() + || fetchResultForBucket.hasFilteredEndOffset()) { + // In oder to not signal notEmptyCondition, add completed fetch to + // buffer until log records is not empty. DefaultCompletedFetch completedFetch = new DefaultCompletedFetch( tb, @@ -534,6 +551,10 @@ Map prepareFetchLogRequests() { } else { reqForTable.setProjectionPushdownEnabled(false); } + if (null != cachedPbPredicate) { + reqForTable.setFilterPredicate(cachedPbPredicate); + reqForTable.setFilterSchemaId(schemaId); + } reqForTable.addAllBucketsReqs(reqForBuckets); fetchLogRequest.addAllTablesReqs(Collections.singletonList(reqForTable)); fetchLogRequests.put(leaderId, fetchLogRequest); @@ -577,6 +598,11 @@ public synchronized void close() throws IOException { } } + @VisibleForTesting + LogScannerStatus getLogScannerStatus() { + return logScannerStatus; + } + @VisibleForTesting int getCompletedFetchesSize() { return logFetchBuffer.bufferedBuckets().size(); diff --git a/fluss-client/src/main/java/org/apache/fluss/client/table/scanner/log/LogScannerImpl.java b/fluss-client/src/main/java/org/apache/fluss/client/table/scanner/log/LogScannerImpl.java index 33181bd7ae..48953da5c2 100644 --- a/fluss-client/src/main/java/org/apache/fluss/client/table/scanner/log/LogScannerImpl.java +++ b/fluss-client/src/main/java/org/apache/fluss/client/table/scanner/log/LogScannerImpl.java @@ -28,6 +28,7 @@ import org.apache.fluss.metadata.TableBucket; import org.apache.fluss.metadata.TableInfo; import org.apache.fluss.metadata.TablePath; +import org.apache.fluss.predicate.Predicate; import org.apache.fluss.rpc.metrics.ClientMetricGroup; import org.apache.fluss.types.RowType; import org.apache.fluss.utils.Projection; @@ -84,7 +85,8 @@ public LogScannerImpl( ClientMetricGroup clientMetricGroup, RemoteFileDownloader remoteFileDownloader, @Nullable int[] projectedFields, - SchemaGetter schemaGetter) { + SchemaGetter schemaGetter, + @Nullable Predicate recordBatchFilter) { this.tablePath = tableInfo.getTablePath(); this.tableId = tableInfo.getTableId(); this.isPartitionedTable = tableInfo.isPartitioned(); @@ -98,6 +100,7 @@ public LogScannerImpl( new LogFetcher( tableInfo, projection, + recordBatchFilter, logScannerStatus, conf, metadataUpdater, diff --git a/fluss-client/src/main/java/org/apache/fluss/client/table/scanner/log/RemoteCompletedFetch.java b/fluss-client/src/main/java/org/apache/fluss/client/table/scanner/log/RemoteCompletedFetch.java index ef867a8f44..00ff712aa9 100644 --- a/fluss-client/src/main/java/org/apache/fluss/client/table/scanner/log/RemoteCompletedFetch.java +++ b/fluss-client/src/main/java/org/apache/fluss/client/table/scanner/log/RemoteCompletedFetch.java @@ -55,7 +55,8 @@ class RemoteCompletedFetch extends CompletedFetch { readContext, logScannerStatus, isCheckCrc, - fetchOffset); + fetchOffset, + -1); this.fileLogRecords = fileLogRecords; this.recycleCallback = recycleCallback; } diff --git a/fluss-client/src/test/java/org/apache/fluss/client/table/scanner/log/LogFetcherFilterTest.java b/fluss-client/src/test/java/org/apache/fluss/client/table/scanner/log/LogFetcherFilterTest.java new file mode 100644 index 0000000000..962b131aec --- /dev/null +++ b/fluss-client/src/test/java/org/apache/fluss/client/table/scanner/log/LogFetcherFilterTest.java @@ -0,0 +1,344 @@ +/* + * 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.admin.ClientToServerITCaseBase; +import org.apache.fluss.client.metadata.MetadataUpdater; +import org.apache.fluss.client.metrics.TestingScannerMetricGroup; +import org.apache.fluss.client.table.scanner.RemoteFileDownloader; +import org.apache.fluss.client.table.scanner.ScanRecord; +import org.apache.fluss.metadata.TableBucket; +import org.apache.fluss.predicate.Predicate; +import org.apache.fluss.predicate.PredicateBuilder; +import org.apache.fluss.record.LogRecordBatchStatisticsTestUtils; +import org.apache.fluss.record.MemoryLogRecords; +import org.apache.fluss.rpc.RpcClient; +import org.apache.fluss.rpc.gateway.TabletServerGateway; +import org.apache.fluss.types.RowType; + +import org.junit.jupiter.api.AfterEach; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; + +import java.time.Duration; +import java.util.Arrays; +import java.util.Collections; +import java.util.HashMap; +import java.util.List; +import java.util.Map; + +import static org.apache.fluss.record.TestData.DATA1_ROW_TYPE; +import static org.apache.fluss.record.TestData.DATA1_TABLE_DESCRIPTOR; +import static org.apache.fluss.record.TestData.DATA1_TABLE_INFO; +import static org.apache.fluss.record.TestData.DATA1_TABLE_PATH; +import static org.apache.fluss.record.TestData.DEFAULT_SCHEMA_ID; +import static org.apache.fluss.record.TestData.TEST_SCHEMA_GETTER; +import static org.apache.fluss.server.testutils.RpcMessageTestUtils.newProduceLogRequest; +import static org.apache.fluss.testutils.common.CommonTestUtils.retry; +import static org.assertj.core.api.Assertions.assertThat; + +/** Test for {@link LogFetcher} with recordBatchFilter pushdown scenarios. */ +public class LogFetcherFilterTest extends ClientToServerITCaseBase { + private LogFetcher logFetcher; + private long tableId; + private final int bucketId0 = 0; + private final int bucketId1 = 1; + + // Use the same row type as DATA1 to avoid Arrow compatibility issues + private static final RowType FILTER_TEST_ROW_TYPE = DATA1_ROW_TYPE; + + // Data that should match filter (a > 5) - using DATA1 compatible structure + private static final List MATCHING_DATA = + Arrays.asList( + new Object[] {6, "alice"}, + new Object[] {7, "bob"}, + new Object[] {8, "charlie"}, + new Object[] {9, "david"}, + new Object[] {10, "eve"}); + + // Data that should NOT match filter (a <= 5) + private static final List NON_MATCHING_DATA = + Arrays.asList( + new Object[] {1, "anna"}, + new Object[] {2, "brian"}, + new Object[] {3, "cindy"}, + new Object[] {4, "derek"}, + new Object[] {5, "fiona"}); + + @BeforeEach + protected void setup() throws Exception { + super.setup(); + + // Create table for filter testing + tableId = createTable(DATA1_TABLE_PATH, DATA1_TABLE_DESCRIPTOR, false); + FLUSS_CLUSTER_EXTENSION.waitUntilTableReady(tableId); + + RpcClient rpcClient = FLUSS_CLUSTER_EXTENSION.getRpcClient(); + MetadataUpdater metadataUpdater = new MetadataUpdater(clientConf, rpcClient); + metadataUpdater.checkAndUpdateTableMetadata(Collections.singleton(DATA1_TABLE_PATH)); + + Map scanBuckets = new HashMap<>(); + // Add bucket 0 and bucket 1 to log scanner status + scanBuckets.put(new TableBucket(tableId, bucketId0), 0L); + scanBuckets.put(new TableBucket(tableId, bucketId1), 0L); + LogScannerStatus logScannerStatus = new LogScannerStatus(); + logScannerStatus.assignScanBuckets(scanBuckets); + + // Create predicate for filter testing: field 'a' > 5 (first field in DATA1 structure) + PredicateBuilder builder = new PredicateBuilder(FILTER_TEST_ROW_TYPE); + Predicate recordBatchFilter = builder.greaterThan(0, 5); // a > 5 + + TestingScannerMetricGroup scannerMetricGroup = TestingScannerMetricGroup.newInstance(); + logFetcher = + new LogFetcher( + DATA1_TABLE_INFO, + null, // projection + recordBatchFilter, // recordBatchFilter + logScannerStatus, + clientConf, + metadataUpdater, + scannerMetricGroup, + new RemoteFileDownloader(1), + TEST_SCHEMA_GETTER); + } + + @AfterEach + void tearDown() throws Exception { + if (logFetcher != null) { + logFetcher.close(); + } + } + + @Test + void testFetchWithRecordBatchFilter() throws Exception { + TableBucket tb0 = new TableBucket(tableId, bucketId0); + TableBucket tb1 = new TableBucket(tableId, bucketId1); + + // Add matching data to bucket 0 - should be included in results + MemoryLogRecords matchingRecords = + LogRecordBatchStatisticsTestUtils.createLogRecordsWithStatistics( + MATCHING_DATA, FILTER_TEST_ROW_TYPE, 0L, DEFAULT_SCHEMA_ID); + addRecordsToBucket(tb0, matchingRecords); + + // Add non-matching data to bucket 1 - should be filtered out + MemoryLogRecords nonMatchingRecords = + LogRecordBatchStatisticsTestUtils.createLogRecordsWithStatistics( + NON_MATCHING_DATA, FILTER_TEST_ROW_TYPE, 0L, DEFAULT_SCHEMA_ID); + addRecordsToBucket(tb1, nonMatchingRecords); + + assertThat(logFetcher.hasAvailableFetches()).isFalse(); + + // Send fetcher to fetch data + logFetcher.sendFetches(); + + // Wait for fetch results + retry( + Duration.ofMinutes(1), + () -> { + assertThat(logFetcher.hasAvailableFetches()).isTrue(); + }); + + Map> records = logFetcher.collectFetch(); + + // Verify that only matching data is returned + // tb0 should have records (matching data), tb1 should have no records (filtered out) + assertThat(records.containsKey(tb0)).isTrue(); + assertThat(records.get(tb0)).isNotEmpty(); + assertThat(records.get(tb0).size()).isEqualTo(MATCHING_DATA.size()); + + // Verify the content of returned records - all should match filter (a > 5) + List tb0Records = records.get(tb0); + for (ScanRecord scanRecord : tb0Records) { + assertThat(scanRecord).isNotNull(); + // Verify that the record's first field (a) is greater than 5 + int aValue = scanRecord.getRow().getInt(0); + assertThat(aValue).isGreaterThan(5); + } + + // Verify tb1 (non-matching data) does not surface user-visible records. + // The server may return a filteredEndOffset response for the bucket, but collectFetch() + // only materializes buckets with actual records. + assertThat(records.getOrDefault(tb1, Collections.emptyList())).isEmpty(); + + // After collect fetch, the fetcher should be empty + assertThat(logFetcher.hasAvailableFetches()).isFalse(); + assertThat(logFetcher.getCompletedFetchesSize()).isEqualTo(0); + } + + @Test + void testFetchWithMixedData() throws Exception { + TableBucket tb0 = new TableBucket(tableId, bucketId0); + + // Create mixed data: some matching, some not matching (using DATA1 structure) + List mixedData = + Arrays.asList( + new Object[] {1, "low1"}, // Should be filtered (a <= 5) + new Object[] {6, "high1"}, // Should pass (a > 5) + new Object[] {3, "low2"}, // Should be filtered (a <= 5) + new Object[] {8, "high2"}, // Should pass (a > 5) + new Object[] {2, "low3"} // Should be filtered (a <= 5) + ); + + MemoryLogRecords mixedRecords = + LogRecordBatchStatisticsTestUtils.createLogRecordsWithStatistics( + mixedData, FILTER_TEST_ROW_TYPE, 0L, DEFAULT_SCHEMA_ID); + addRecordsToBucket(tb0, mixedRecords); + + logFetcher.sendFetches(); + + retry( + Duration.ofMinutes(1), + () -> { + assertThat(logFetcher.hasAvailableFetches()).isTrue(); + }); + + Map> records = logFetcher.collectFetch(); + + // With recordBatchFilter at batch level, the behavior depends on the batch statistics + // Since mixedData contains values 1,6,3,8,2 (min=1, max=8), and our filter is a > 5, + // the entire batch should be included because max value (8) > 5 + assertThat(records.containsKey(tb0)).isTrue(); + assertThat(records.get(tb0)).isNotEmpty(); + + List tb0Records = records.get(tb0); + // All records in the batch should be returned (including those that don't match + // individually) + // because recordBatchFilter works at batch level based on statistics + assertThat(tb0Records.size()).isEqualTo(mixedData.size()); + + // However, we can verify that the batch does contain some records that match the filter + boolean hasMatchingRecords = + tb0Records.stream() + .anyMatch( + record -> { + int aValue = record.getRow().getInt(0); + return aValue > 5; + }); + assertThat(hasMatchingRecords).isTrue(); + + // And some records that don't match (this proves batch-level filtering, not row-level) + boolean hasNonMatchingRecords = + tb0Records.stream() + .anyMatch( + record -> { + int aValue = record.getRow().getInt(0); + return aValue <= 5; + }); + assertThat(hasNonMatchingRecords).isTrue(); + } + + @Test + void testFilterCompletelyRejectsNonMatchingBatch() throws Exception { + TableBucket tb0 = new TableBucket(tableId, bucketId0); + + // Create a batch where ALL records don't match filter (all a <= 5) + List allNonMatchingData = + Arrays.asList( + new Object[] {1, "reject1"}, + new Object[] {2, "reject2"}, + new Object[] {3, "reject3"}, + new Object[] {4, "reject4"}, + new Object[] {5, "reject5"}); + + MemoryLogRecords nonMatchingRecords = + LogRecordBatchStatisticsTestUtils.createLogRecordsWithStatistics( + allNonMatchingData, FILTER_TEST_ROW_TYPE, 0L, DEFAULT_SCHEMA_ID); + addRecordsToBucket(tb0, nonMatchingRecords); + + logFetcher.sendFetches(); + + retry( + Duration.ofMinutes(1), + () -> { + // The fetch may complete even if all batches are filtered out + // depending on implementation + assertThat(logFetcher.hasAvailableFetches()).isTrue(); + }); + + Map> records = logFetcher.collectFetch(); + + // For a batch where max value = 5 and filter is a > 5, the entire batch should be + // filtered out at the server side. collectFetch() may omit the bucket entirely because + // there are no user-visible records to return. + if (records.containsKey(tb0)) { + assertThat(records.get(tb0)).isEmpty(); + } + } + + @Test + void testConsecutiveFetchesWithFilteredOffsetAdvancement() throws Exception { + // Verify that when all batches in a bucket are filtered out, the client's + // fetch offset advances via filteredEndOffset and doesn't get stuck polling. + // + // Write only non-matching data to bucket 0, matching data to bucket 1. + // Bucket 0 should be fully filtered — the client must advance its offset + // past the filtered data and not re-fetch the same offset indefinitely. + TableBucket tb0 = new TableBucket(tableId, bucketId0); + TableBucket tb1 = new TableBucket(tableId, bucketId1); + + // Bucket 0: non-matching data only (a <= 5, filtered by a > 5) + addRecordsToBucket( + tb0, + LogRecordBatchStatisticsTestUtils.createLogRecordsWithStatistics( + NON_MATCHING_DATA, FILTER_TEST_ROW_TYPE, 0L, DEFAULT_SCHEMA_ID)); + + // Bucket 1: matching data (a > 5) + addRecordsToBucket( + tb1, + LogRecordBatchStatisticsTestUtils.createLogRecordsWithStatistics( + MATCHING_DATA, FILTER_TEST_ROW_TYPE, 0L, DEFAULT_SCHEMA_ID)); + + // Use retry loop to handle the case where bucket 0's HW hasn't advanced yet + // on the first fetch cycle. The filtered-empty response (with filteredEndOffset) + // is only returned when the server has data up to HW to scan. + retry( + Duration.ofMinutes(1), + () -> { + logFetcher.sendFetches(); + retry( + Duration.ofSeconds(30), + () -> assertThat(logFetcher.hasAvailableFetches()).isTrue()); + logFetcher.collectFetch(); + + // Verify that bucket 0's fetch offset has advanced past 0. + Long tb0Offset = logFetcher.getLogScannerStatus().getBucketOffset(tb0); + assertThat(tb0Offset) + .as("Bucket 0 fetch offset should advance past filtered data") + .isGreaterThan(0L); + }); + + // After offset advancement, fetcher should be clean + assertThat(logFetcher.getCompletedFetchesSize()).isEqualTo(0); + } + + private void addRecordsToBucket(TableBucket tableBucket, MemoryLogRecords logRecords) + throws Exception { + int leader = FLUSS_CLUSTER_EXTENSION.waitAndGetLeader(tableBucket); + TabletServerGateway leaderGateWay = + FLUSS_CLUSTER_EXTENSION.newTabletServerClientForNode(leader); + // Simplify to avoid the missing ProduceLogResponse import issue + leaderGateWay + .produceLog( + newProduceLogRequest( + tableBucket.getTableId(), + tableBucket.getBucket(), + -1, // need ack + logRecords)) + .get(); + } +} diff --git a/fluss-client/src/test/java/org/apache/fluss/client/table/scanner/log/LogFetcherITCase.java b/fluss-client/src/test/java/org/apache/fluss/client/table/scanner/log/LogFetcherITCase.java index 73c9b40fb6..d1919ab8d6 100644 --- a/fluss-client/src/test/java/org/apache/fluss/client/table/scanner/log/LogFetcherITCase.java +++ b/fluss-client/src/test/java/org/apache/fluss/client/table/scanner/log/LogFetcherITCase.java @@ -103,6 +103,7 @@ protected void setup() throws Exception { new LogFetcher( DATA1_TABLE_INFO, null, + null, logScannerStatus, clientConf, metadataUpdater, @@ -177,6 +178,7 @@ void testFetchWithSchemaChange() throws Exception { DATA1_TABLE_INFO.getCreatedTime(), DATA1_TABLE_INFO.getModifiedTime()), null, + null, logScannerStatus, clientConf, metadataUpdater, @@ -275,6 +277,7 @@ void testFetchWhenDestinationIsNullInMetadata() throws Exception { new LogFetcher( DATA1_TABLE_INFO, null, + null, logScannerStatus, clientConf, metadataUpdater, @@ -315,6 +318,7 @@ void testFetchWithInvalidTableOrPartitions() throws Exception { new LogFetcher( DATA1_TABLE_INFO, null, + null, logScannerStatus, clientConf, metadataUpdater1, diff --git a/fluss-client/src/test/java/org/apache/fluss/client/table/scanner/log/LogFetcherTest.java b/fluss-client/src/test/java/org/apache/fluss/client/table/scanner/log/LogFetcherTest.java index a01809c58c..ef691865ed 100644 --- a/fluss-client/src/test/java/org/apache/fluss/client/table/scanner/log/LogFetcherTest.java +++ b/fluss-client/src/test/java/org/apache/fluss/client/table/scanner/log/LogFetcherTest.java @@ -79,6 +79,7 @@ public void setup() { new LogFetcher( DATA1_TABLE_INFO, null, + null, logScannerStatus, new Configuration(), metadataUpdater, @@ -89,7 +90,8 @@ public void setup() { @Test void sendFetchRequestWithNotLeaderOrFollowerException() { - Map requestMap = logFetcher.prepareFetchLogRequests(); + Map requestMap = + logFetcher.prepareFetchLogRequests(); Set serverSet = requestMap.keySet(); assertThat(serverSet).containsExactlyInAnyOrder(1); @@ -110,16 +112,6 @@ void sendFetchRequestWithNotLeaderOrFollowerException() { assertThat(metadataUpdater.getBucketLocation(tb1)).isNotPresent(); } - private TestingMetadataUpdater initializeMetadataUpdater() { - - return new TestingMetadataUpdater( - TestingMetadataUpdater.COORDINATOR, - Arrays.asList(NODE1, NODE2, NODE3), - Collections.singletonMap(DATA1_TABLE_PATH, DATA1_TABLE_INFO), - Collections.singletonMap(1, new TestingTabletServerGateway()), - new Configuration()); - } - private LogScannerStatus initializeLogScannerStatus() { Map scanBucketAndOffsets = new HashMap<>(); scanBucketAndOffsets.put(tb1, 0L); @@ -152,4 +144,14 @@ public CompletableFuture fetchLog(FetchLogRequest request) { return CompletableFuture.completedFuture(makeFetchLogResponse(resultForBucketMap)); } } + + private TestingMetadataUpdater initializeMetadataUpdater() { + + return new TestingMetadataUpdater( + TestingMetadataUpdater.COORDINATOR, + Arrays.asList(NODE1, NODE2, NODE3), + Collections.singletonMap(DATA1_TABLE_PATH, DATA1_TABLE_INFO), + Collections.singletonMap(1, new TestingTabletServerGateway()), + new Configuration()); + } } diff --git a/fluss-common/src/main/java/org/apache/fluss/record/FileLogProjection.java b/fluss-common/src/main/java/org/apache/fluss/record/FileLogProjection.java index 1ba9592157..87e52731f2 100644 --- a/fluss-common/src/main/java/org/apache/fluss/record/FileLogProjection.java +++ b/fluss-common/src/main/java/org/apache/fluss/record/FileLogProjection.java @@ -21,6 +21,8 @@ import org.apache.fluss.compression.ArrowCompressionInfo; import org.apache.fluss.exception.InvalidColumnProjectionException; import org.apache.fluss.metadata.SchemaGetter; +import org.apache.fluss.record.FileLogInputStream.FileChannelLogRecordBatch; +import org.apache.fluss.record.bytesview.BytesView; import org.apache.fluss.record.bytesview.MultiBytesView; import org.apache.fluss.shaded.arrow.com.google.flatbuffers.FlatBufferBuilder; import org.apache.fluss.shaded.arrow.org.apache.arrow.flatbuf.Buffer; @@ -70,6 +72,7 @@ import static org.apache.fluss.record.LogRecordBatchFormat.statisticsLengthOffset; import static org.apache.fluss.utils.FileUtils.readFully; import static org.apache.fluss.utils.FileUtils.readFullyOrFail; +import static org.apache.fluss.utils.Preconditions.checkNotNull; import static org.apache.fluss.utils.Preconditions.checkState; /** Column projection util on Arrow format {@link FileLogRecords}. */ @@ -124,6 +127,118 @@ public void setCurrentProjection( this.selectedFieldPositions = selectedFieldPositions; } + /** + * Project a single record batch to a subset of fields. This is used by the filter path where + * batches are iterated individually rather than as a contiguous file region. + * + * @param batch the file channel log record batch to project + * @return the projected bytes view + */ + public BytesView projectRecordBatch(FileChannelLogRecordBatch batch) throws IOException { + FileChannel channel = batch.fileRecords.channel(); + int position = batch.position(); + + // Read the log header to get schema ID + logHeaderBuffer.rewind(); + readLogHeaderFullyOrFail(channel, logHeaderBuffer, position); + logHeaderBuffer.rewind(); + byte magic = logHeaderBuffer.get(MAGIC_OFFSET); + int recordBatchHeaderSize = recordBatchHeaderSize(magic); + int batchSizeInBytes = LOG_OVERHEAD + logHeaderBuffer.getInt(LENGTH_OFFSET); + short schemaId = logHeaderBuffer.getShort(schemaIdOffset(magic)); + + ProjectionInfo currentProjection = getOrCreateProjectionInfo(schemaId); + checkNotNull(currentProjection, "There is no projection registered yet."); + + MultiBytesView.Builder builder = MultiBytesView.builder(); + + // Return empty if meets empty batch + if (batchSizeInBytes == recordBatchHeaderSize) { + return builder.build(); + } + + boolean isAppendOnly = + (logHeaderBuffer.get(attributeOffset(magic)) & APPEND_ONLY_FLAG_MASK) > 0; + + // For V1+, skip statistics data between header and records + int statisticsLength = 0; + if (magic >= LOG_MAGIC_VALUE_V1) { + statisticsLength = logHeaderBuffer.getInt(statisticsLengthOffset(magic)); + } + int recordsStartOffset = recordBatchHeaderSize + statisticsLength; + + final int changeTypeBytes; + final long arrowHeaderOffset; + if (isAppendOnly) { + changeTypeBytes = 0; + arrowHeaderOffset = position + recordsStartOffset; + } else { + changeTypeBytes = logHeaderBuffer.getInt(recordsCountOffset(magic)); + arrowHeaderOffset = position + recordsStartOffset + changeTypeBytes; + } + + // read arrow header + arrowHeaderBuffer.rewind(); + readFullyOrFail(channel, arrowHeaderBuffer, arrowHeaderOffset, "arrow header"); + arrowHeaderBuffer.position(ARROW_IPC_METADATA_SIZE_OFFSET); + int arrowMetadataSize = arrowHeaderBuffer.getInt(); + + resizeArrowMetadataBuffer(arrowMetadataSize); + arrowMetadataBuffer.rewind(); + readFullyOrFail( + channel, + arrowMetadataBuffer, + arrowHeaderOffset + ARROW_HEADER_SIZE, + "arrow metadata"); + + arrowMetadataBuffer.rewind(); + Message metadata = Message.getRootAsMessage(arrowMetadataBuffer); + ProjectedArrowBatch projectedArrowBatch = + projectArrowBatch( + metadata, + currentProjection.nodesProjection, + currentProjection.buffersProjection, + currentProjection.bufferCount); + long arrowBodyLength = projectedArrowBatch.bodyLength(); + + int newBatchSizeInBytes = + recordBatchHeaderSize + + changeTypeBytes + + currentProjection.arrowMetadataLength + + (int) arrowBodyLength; + + // create new arrow batch metadata which already projected + byte[] headerMetadata = + serializeArrowRecordBatchMetadata( + projectedArrowBatch, arrowBodyLength, currentProjection.bodyCompression); + checkState( + headerMetadata.length == currentProjection.arrowMetadataLength, + "Invalid metadata length"); + + // update and copy log batch header + logHeaderBuffer.position(LENGTH_OFFSET); + logHeaderBuffer.putInt(newBatchSizeInBytes - LOG_OVERHEAD); + + // For V1+ format, clear statistics information since projection removes statistics + LogRecordBatchFormat.clearStatisticsFromHeader(logHeaderBuffer, magic); + + logHeaderBuffer.rewind(); + byte[] logHeader = new byte[recordBatchHeaderSize]; + logHeaderBuffer.get(logHeader); + + // build log records + builder.addBytes(logHeader); + if (!isAppendOnly) { + builder.addBytes(channel, position + recordsStartOffset, changeTypeBytes); + } + builder.addBytes(headerMetadata); + final long bufferOffset = arrowHeaderOffset + ARROW_HEADER_SIZE + arrowMetadataSize; + projectedArrowBatch.buffers.forEach( + b -> builder.addBytes(channel, bufferOffset + b.getOffset(), (int) b.getSize())); + + return builder.build(); + } + /** * Project the log records to a subset of fields and the size of returned log records shouldn't * exceed maxBytes. diff --git a/fluss-common/src/main/java/org/apache/fluss/record/FileLogRecords.java b/fluss-common/src/main/java/org/apache/fluss/record/FileLogRecords.java index e29e672eb1..46db535fb4 100644 --- a/fluss-common/src/main/java/org/apache/fluss/record/FileLogRecords.java +++ b/fluss-common/src/main/java/org/apache/fluss/record/FileLogRecords.java @@ -309,8 +309,23 @@ public FileChannelChunk toChunk() { private AbstractIterator batchIterator(int start) throws IOException { + return batchIterator(start, -1); + } + + /** + * Get an iterator over the record batches in the file, starting at a specific position and + * ending at a specific position. + * + * @param start the start position in the file + * @param endPosition the end position in the file, or -1 to use the default end + * @return An iterator over batches starting from {@code start} + */ + public AbstractIterator batchIterator(int start, int endPosition) + throws IOException { final int end; - if (isSlice) { + if (endPosition >= 0) { + end = endPosition; + } else if (isSlice) { end = this.end; } else { end = this.sizeInBytes(); diff --git a/fluss-common/src/main/java/org/apache/fluss/record/LogRecordReadContext.java b/fluss-common/src/main/java/org/apache/fluss/record/LogRecordReadContext.java index 2b0f695ecf..644e8f4081 100644 --- a/fluss-common/src/main/java/org/apache/fluss/record/LogRecordReadContext.java +++ b/fluss-common/src/main/java/org/apache/fluss/record/LogRecordReadContext.java @@ -171,7 +171,14 @@ public static LogRecordReadContext createIndexedReadContext( public static LogRecordReadContext createCompactedRowReadContext( RowType rowType, int schemaId) { int[] selectedFields = IntStream.range(0, rowType.getFieldCount()).toArray(); - return createCompactedRowReadContext(rowType, schemaId, selectedFields); + return createCompactedRowReadContext(rowType, schemaId, selectedFields, null); + } + + /** Creates a LogRecordReadContext for COMPACTED log format with schema evolution support. */ + public static LogRecordReadContext createCompactedRowReadContext( + RowType rowType, int schemaId, SchemaGetter schemaGetter) { + int[] selectedFields = IntStream.range(0, rowType.getFieldCount()).toArray(); + return createCompactedRowReadContext(rowType, schemaId, selectedFields, schemaGetter); } /** @@ -199,10 +206,26 @@ public static LogRecordReadContext createIndexedReadContext( */ public static LogRecordReadContext createCompactedRowReadContext( RowType rowType, int schemaId, int[] selectedFields) { + return createCompactedRowReadContext(rowType, schemaId, selectedFields, null); + } + + /** + * Creates a LogRecordReadContext for COMPACTED log format. + * + * @param rowType the schema of the read data + * @param schemaId the schemaId of the table + * @param selectedFields the final selected fields of the read data + * @param schemaGetter the schema getter to resolve evolved batch schemas + */ + public static LogRecordReadContext createCompactedRowReadContext( + RowType rowType, + int schemaId, + int[] selectedFields, + @Nullable SchemaGetter schemaGetter) { FieldGetter[] fieldGetters = buildProjectedFieldGetters(rowType, selectedFields); // for COMPACTED log format, the projection is NEVER push downed to the server side return new LogRecordReadContext( - LogFormat.COMPACTED, rowType, schemaId, null, fieldGetters, false, null); + LogFormat.COMPACTED, rowType, schemaId, null, fieldGetters, false, schemaGetter); } private LogRecordReadContext( @@ -247,6 +270,17 @@ public boolean isProjectionPushDowned() { return projectionPushDowned; } + /** Get the target schema ID. */ + public int getSchemaId() { + return targetSchemaId; + } + + /** Get the schema getter. */ + @Nullable + public SchemaGetter getSchemaGetter() { + return schemaGetter; + } + @Override public VectorSchemaRoot getVectorSchemaRoot(int schemaId) { if (logFormat != LogFormat.ARROW) { @@ -277,6 +311,9 @@ public ProjectedRow getOutputProjectedRow(int schemaId) { if (isSameRowType(schemaId)) { return null; } + if (schemaGetter == null) { + return null; + } // TODO: should we cache the projection? Schema originSchema = schemaGetter.getSchema(schemaId); Schema expectedSchema = schemaGetter.getSchema(targetSchemaId); diff --git a/fluss-common/src/main/java/org/apache/fluss/record/bytesview/MultiBytesView.java b/fluss-common/src/main/java/org/apache/fluss/record/bytesview/MultiBytesView.java index 7462d5a109..4c7269f310 100644 --- a/fluss-common/src/main/java/org/apache/fluss/record/bytesview/MultiBytesView.java +++ b/fluss-common/src/main/java/org/apache/fluss/record/bytesview/MultiBytesView.java @@ -150,6 +150,30 @@ public boolean isEmpty() { return views.isEmpty(); } + /** + * Adds a {@link BytesView} directly. If the view is a {@link MultiBytesView}, its inner + * views are added individually. {@link FileRegionBytesView} instances are handled through + * {@link #addBytes(FileChannel, long, int)} to preserve file region merging. + * + * @param bytesView the bytes view to add + * @return this builder instance for method chaining + */ + public Builder addBytes(BytesView bytesView) { + if (bytesView instanceof MultiBytesView) { + MultiBytesView multi = (MultiBytesView) bytesView; + for (BytesView inner : multi.views) { + addBytes(inner); + } + } else if (bytesView instanceof FileRegionBytesView) { + FileRegionBytesView fileView = (FileRegionBytesView) bytesView; + addBytes(fileView.fileChannel, fileView.position, fileView.size); + } else { + views.add(bytesView); + lastFileRegionView = null; + } + return this; + } + /** Builds a {@link MultiBytesView}. */ public MultiBytesView build() { return new MultiBytesView(views.toArray(new BytesView[0])); diff --git a/fluss-common/src/test/java/org/apache/fluss/record/FileLogInputStreamTest.java b/fluss-common/src/test/java/org/apache/fluss/record/FileLogInputStreamTest.java index 1e6e51d55b..6afb255493 100644 --- a/fluss-common/src/test/java/org/apache/fluss/record/FileLogInputStreamTest.java +++ b/fluss-common/src/test/java/org/apache/fluss/record/FileLogInputStreamTest.java @@ -20,18 +20,22 @@ import org.apache.fluss.metadata.LogFormat; import org.apache.fluss.utils.CloseableIterator; +import org.junit.jupiter.api.Test; import org.junit.jupiter.api.io.TempDir; import org.junit.jupiter.params.ParameterizedTest; import org.junit.jupiter.params.provider.ValueSource; import java.io.File; import java.util.Collections; +import java.util.Optional; import static org.apache.fluss.record.LogRecordBatchFormat.LOG_MAGIC_VALUE_V0; import static org.apache.fluss.record.LogRecordBatchFormat.LOG_MAGIC_VALUE_V1; +import static org.apache.fluss.record.LogRecordBatchFormat.LOG_MAGIC_VALUE_V2; import static org.apache.fluss.record.TestData.DATA1_ROW_TYPE; import static org.apache.fluss.record.TestData.DATA1_SCHEMA; import static org.apache.fluss.record.TestData.DEFAULT_SCHEMA_ID; +import static org.apache.fluss.record.TestData.TEST_SCHEMA_GETTER; import static org.apache.fluss.testutils.DataTestUtils.createRecordsWithoutBaseLogOffset; import static org.assertj.core.api.Assertions.assertThat; @@ -75,4 +79,182 @@ void testWriteTo(byte recordBatchMagic) throws Exception { } } } + + @Test + void testV2FormatWithStatistics() throws Exception { + // Create test data with statistics using V2 format + MemoryLogRecords memoryLogRecords = + LogRecordBatchStatisticsTestUtils.createLogRecordsWithStatistics( + TestData.DATA1, DATA1_ROW_TYPE, 0L, DEFAULT_SCHEMA_ID); + + try (FileLogRecords fileLogRecords = + FileLogRecords.open(new File(tempDir, "test_v2.tmp"))) { + fileLogRecords.append(memoryLogRecords); + fileLogRecords.flush(); + + FileLogInputStream logInputStream = + new FileLogInputStream(fileLogRecords, 0, fileLogRecords.sizeInBytes()); + + FileLogInputStream.FileChannelLogRecordBatch batch = logInputStream.nextBatch(); + assertThat(batch).isNotNull(); + assertThat(batch.magic()).isEqualTo(LOG_MAGIC_VALUE_V2); + assertThat(batch.getRecordCount()).isEqualTo(TestData.DATA1.size()); + + // Test statistics reading with ReadContext + try (LogRecordReadContext readContext = + LogRecordReadContext.createArrowReadContext( + DATA1_ROW_TYPE, DEFAULT_SCHEMA_ID, TEST_SCHEMA_GETTER)) { + + // Test getStatistics method + Optional statisticsOpt = batch.getStatistics(readContext); + assertThat(statisticsOpt).isPresent(); + + LogRecordBatchStatistics statistics = statisticsOpt.get(); + + // Verify statistics content for DATA1 + assertThat(statistics.getMinValues().getInt(0)).isEqualTo(1); // min id + assertThat(statistics.getMaxValues().getInt(0)).isEqualTo(10); // max id + assertThat(statistics.getNullCounts()[0]).isEqualTo(0); // no nulls + + // Test that statistics are cached (lazy loading) + Optional statisticsOpt2 = + batch.getStatistics(readContext); + assertThat(statisticsOpt2).isPresent(); + assertThat(statisticsOpt2.get()).isSameAs(statisticsOpt.get()); + } + + // Test that records can still be read correctly + try (LogRecordReadContext readContext = + LogRecordReadContext.createArrowReadContext( + DATA1_ROW_TYPE, DEFAULT_SCHEMA_ID, TEST_SCHEMA_GETTER); + CloseableIterator iterator = batch.records(readContext)) { + assertThat(iterator.hasNext()).isTrue(); + int recordCount = 0; + while (iterator.hasNext()) { + LogRecord record = iterator.next(); + assertThat(record).isNotNull(); + recordCount++; + } + assertThat(recordCount).isEqualTo(TestData.DATA1.size()); + } + } + } + + @Test + void testV2FormatWithoutStatistics() throws Exception { + // Create test data without statistics using V1 format (which doesn't support statistics) + try (FileLogRecords fileLogRecords = + FileLogRecords.open(new File(tempDir, "test_v1_no_stats.tmp"))) { + fileLogRecords.append( + createRecordsWithoutBaseLogOffset( + DATA1_ROW_TYPE, + DEFAULT_SCHEMA_ID, + 0L, + -1L, + LOG_MAGIC_VALUE_V1, + Collections.singletonList(new Object[] {0, "abc"}), + LogFormat.ARROW)); + fileLogRecords.flush(); + + FileLogInputStream logInputStream = + new FileLogInputStream(fileLogRecords, 0, fileLogRecords.sizeInBytes()); + + FileLogInputStream.FileChannelLogRecordBatch batch = logInputStream.nextBatch(); + assertThat(batch).isNotNull(); + assertThat(batch.magic()).isEqualTo(LOG_MAGIC_VALUE_V1); + + // Test that getStatistics returns empty when magic version doesn't support statistics + try (LogRecordReadContext readContext = + LogRecordReadContext.createArrowReadContext( + DATA1_ROW_TYPE, DEFAULT_SCHEMA_ID, TEST_SCHEMA_GETTER)) { + Optional statisticsOpt = batch.getStatistics(readContext); + assertThat(statisticsOpt).isEmpty(); + } + } + } + + @Test + void testGetStatisticsWithNullContext() throws Exception { + // Create test data with statistics + MemoryLogRecords memoryLogRecords = + LogRecordBatchStatisticsTestUtils.createLogRecordsWithStatistics( + TestData.DATA1, DATA1_ROW_TYPE, 0L, DEFAULT_SCHEMA_ID); + + try (FileLogRecords fileLogRecords = + FileLogRecords.open(new File(tempDir, "test_null_context.tmp"))) { + fileLogRecords.append(memoryLogRecords); + fileLogRecords.flush(); + + FileLogInputStream logInputStream = + new FileLogInputStream(fileLogRecords, 0, fileLogRecords.sizeInBytes()); + + FileLogInputStream.FileChannelLogRecordBatch batch = logInputStream.nextBatch(); + assertThat(batch).isNotNull(); + + // Test that getStatistics returns empty when context is null + Optional statisticsOpt = batch.getStatistics(null); + assertThat(statisticsOpt).isEmpty(); + } + } + + @Test + void testGetStatisticsWithMissingSchemaInGetter() throws Exception { + // Create test data with a schemaId that is NOT registered in the schemaGetter + int unregisteredSchemaId = 999; + MemoryLogRecords memoryLogRecords = + LogRecordBatchStatisticsTestUtils.createLogRecordsWithStatistics( + TestData.DATA1, DATA1_ROW_TYPE, 0L, unregisteredSchemaId); + + try (FileLogRecords fileLogRecords = + FileLogRecords.open(new File(tempDir, "test_invalid_schema.tmp"))) { + fileLogRecords.append(memoryLogRecords); + fileLogRecords.flush(); + + FileLogInputStream logInputStream = + new FileLogInputStream(fileLogRecords, 0, fileLogRecords.sizeInBytes()); + + FileLogInputStream.FileChannelLogRecordBatch batch = logInputStream.nextBatch(); + assertThat(batch).isNotNull(); + + // When the schemaGetter cannot find the batch's schemaId, getStatistics should + // gracefully return empty (the exception is caught internally) + try (LogRecordReadContext readContext = + LogRecordReadContext.createArrowReadContext( + DATA1_ROW_TYPE, DEFAULT_SCHEMA_ID, TEST_SCHEMA_GETTER)) { + Optional statisticsOpt = batch.getStatistics(readContext); + // The batch's schemaId (999) is not in TEST_SCHEMA_GETTER, so it should return + // empty + assertThat(statisticsOpt).isEmpty(); + } + } + } + + @Test + void testStatisticsCreation() throws Exception { + // Create test data with statistics using V2 format + MemoryLogRecords memoryLogRecords = + LogRecordBatchStatisticsTestUtils.createLogRecordsWithStatistics( + TestData.DATA1, DATA1_ROW_TYPE, 0L, DEFAULT_SCHEMA_ID); + + // Get the batch + LogRecordBatch memoryBatch = memoryLogRecords.batches().iterator().next(); + assertThat(memoryBatch.magic()).isEqualTo(LOG_MAGIC_VALUE_V2); + + // Test that the memory batch has statistics + try (LogRecordReadContext readContext = + LogRecordReadContext.createArrowReadContext( + DATA1_ROW_TYPE, DEFAULT_SCHEMA_ID, TEST_SCHEMA_GETTER)) { + Optional memoryStatsOpt = + memoryBatch.getStatistics(readContext); + assertThat(memoryStatsOpt).isPresent(); + + LogRecordBatchStatistics memoryStats = memoryStatsOpt.get(); + + // Verify statistics content + assertThat(memoryStats.getMinValues().getInt(0)).isEqualTo(1); + assertThat(memoryStats.getMaxValues().getInt(0)).isEqualTo(10); + assertThat(memoryStats.getMinValues().getString(1).toString()).isEqualTo("a"); + assertThat(memoryStats.getMaxValues().getString(1).toString()).isEqualTo("j"); + } + } } diff --git a/fluss-common/src/test/java/org/apache/fluss/record/FileLogProjectionTest.java b/fluss-common/src/test/java/org/apache/fluss/record/FileLogProjectionTest.java index b85e4db32f..c2602dda62 100644 --- a/fluss-common/src/test/java/org/apache/fluss/record/FileLogProjectionTest.java +++ b/fluss-common/src/test/java/org/apache/fluss/record/FileLogProjectionTest.java @@ -21,6 +21,7 @@ import org.apache.fluss.exception.SchemaNotExistException; import org.apache.fluss.metadata.LogFormat; import org.apache.fluss.metadata.SchemaInfo; +import org.apache.fluss.record.bytesview.BytesView; import org.apache.fluss.row.InternalRow; import org.apache.fluss.types.RowType; import org.apache.fluss.utils.CloseableIterator; @@ -45,6 +46,7 @@ import static org.apache.fluss.compression.ArrowCompressionInfo.DEFAULT_COMPRESSION; import static org.apache.fluss.record.LogRecordBatchFormat.LOG_MAGIC_VALUE_V0; import static org.apache.fluss.record.LogRecordBatchFormat.LOG_MAGIC_VALUE_V1; +import static org.apache.fluss.record.LogRecordBatchFormat.LOG_MAGIC_VALUE_V2; import static org.apache.fluss.record.LogRecordBatchFormat.MAGIC_OFFSET; import static org.apache.fluss.record.LogRecordBatchFormat.V0_RECORD_BATCH_HEADER_SIZE; import static org.apache.fluss.record.LogRecordBatchFormat.V1_RECORD_BATCH_HEADER_SIZE; @@ -524,4 +526,508 @@ private static void assertEquals(List actual, List expected) assertThat(actual.get(i)).isEqualTo(expected.get(i)); } } + + @ParameterizedTest + @ValueSource(bytes = {LOG_MAGIC_VALUE_V0, LOG_MAGIC_VALUE_V1, LOG_MAGIC_VALUE_V2}) + void testProjectRecordBatch(byte recordBatchMagic) throws Exception { + // Use schemaId=2 which corresponds to DATA2_SCHEMA (3 columns) in testingSchemaGetter + int schemaIdForData2 = 2; + + // Create test data with multiple batches + FileLogRecords fileLogRecords = + createFileLogRecords( + schemaIdForData2, + recordBatchMagic, + TestData.DATA2_ROW_TYPE, + TestData.DATA2, + TestData.DATA2); // Use DATA2 which has 3 columns: a(int), b(string), + // c(string) + + FileLogProjection projection = new FileLogProjection(new ProjectionPushdownCache()); + projection.setCurrentProjection( + 1L, + testingSchemaGetter, + DEFAULT_COMPRESSION, + new int[] {0, 2}); // Project columns a and c + + // Get the first batch + FileLogInputStream.FileChannelLogRecordBatch batch = + fileLogRecords.batchIterator(0, -1).next(); + + // Perform projection + BytesView projectedBytes = projection.projectRecordBatch(batch); + + // Verify the projected bytes are not empty + assertThat(projectedBytes.getBytesLength()).isGreaterThan(0); + + // Verify the projected data by reading it back + LogRecords projectedRecords = new BytesViewLogRecords(projectedBytes); + RowType projectedType = TestData.DATA2_ROW_TYPE.project(new int[] {0, 2}); + + try (LogRecordReadContext context = + createArrowReadContext(projectedType, schemaIdForData2, testingSchemaGetter)) { + for (LogRecordBatch projectedBatch : projectedRecords.batches()) { + try (CloseableIterator records = projectedBatch.records(context)) { + int recordCount = 0; + while (records.hasNext()) { + LogRecord record = records.next(); + InternalRow row = record.getRow(); + + // Verify projected row has correct number of fields + assertThat(row.getFieldCount()).isEqualTo(2); + + // Verify the projected fields contain correct data + assertThat(row.getInt(0)) + .isEqualTo(TestData.DATA2.get(recordCount)[0]); // column a + assertThat(row.getString(1).toString()) + .isEqualTo(TestData.DATA2.get(recordCount)[2]); // column c + + recordCount++; + } + // Verify we got all records from the original batch + assertThat(recordCount).isEqualTo(TestData.DATA2.size()); + } + } + } + } + + @ParameterizedTest + @ValueSource(bytes = {LOG_MAGIC_VALUE_V0, LOG_MAGIC_VALUE_V1, LOG_MAGIC_VALUE_V2}) + void testProjectRecordBatchEmptyBatch(byte recordBatchMagic) throws Exception { + // Create an empty batch (this would be a CDC batch with no records) + FileLogRecords fileLogRecords = FileLogRecords.open(new File(tempDir, "empty.tmp")); + + // Create an empty memory log records + MemoryLogRecords emptyRecords = + createRecordsWithoutBaseLogOffset( + TestData.DATA1_ROW_TYPE, + DEFAULT_SCHEMA_ID, + 0L, + System.currentTimeMillis(), + recordBatchMagic, + new ArrayList<>(), // Empty data + LogFormat.ARROW); + + fileLogRecords.append(emptyRecords); + fileLogRecords.flush(); + + FileLogProjection projection = new FileLogProjection(new ProjectionPushdownCache()); + projection.setCurrentProjection( + 1L, testingSchemaGetter, DEFAULT_COMPRESSION, new int[] {0}); + + // Get the batch (should be empty) + FileLogInputStream.FileChannelLogRecordBatch batch = + fileLogRecords.batchIterator(0, -1).next(); + + // Perform projection on empty batch + BytesView projectedBytes = projection.projectRecordBatch(batch); + + // Should return empty bytes for empty batch + assertThat(projectedBytes.getBytesLength()).isEqualTo(0); + } + + @ParameterizedTest + @ValueSource(bytes = {LOG_MAGIC_VALUE_V0, LOG_MAGIC_VALUE_V1, LOG_MAGIC_VALUE_V2}) + void testProjectRecordBatchSingleColumn(byte recordBatchMagic) throws Exception { + // Use schemaId=2 which corresponds to DATA2_SCHEMA (3 columns) in testingSchemaGetter + int schemaIdForData2 = 2; + + // Test projection to single column + FileLogRecords fileLogRecords = + createFileLogRecords( + schemaIdForData2, + recordBatchMagic, + TestData.DATA2_ROW_TYPE, + TestData.DATA2); + + FileLogProjection projection = new FileLogProjection(new ProjectionPushdownCache()); + projection.setCurrentProjection( + 1L, + testingSchemaGetter, + DEFAULT_COMPRESSION, + new int[] {1}); // Project only column b + + FileLogInputStream.FileChannelLogRecordBatch batch = + fileLogRecords.batchIterator(0, -1).next(); + BytesView projectedBytes = projection.projectRecordBatch(batch); + + assertThat(projectedBytes.getBytesLength()).isGreaterThan(0); + + // Verify single column projection + LogRecords projectedRecords = new BytesViewLogRecords(projectedBytes); + RowType projectedType = TestData.DATA2_ROW_TYPE.project(new int[] {1}); + + try (LogRecordReadContext context = + createArrowReadContext(projectedType, schemaIdForData2, testingSchemaGetter)) { + for (LogRecordBatch projectedBatch : projectedRecords.batches()) { + try (CloseableIterator records = projectedBatch.records(context)) { + int recordCount = 0; + while (records.hasNext()) { + LogRecord record = records.next(); + InternalRow row = record.getRow(); + + assertThat(row.getFieldCount()).isEqualTo(1); + assertThat(row.getString(0).toString()) + .isEqualTo(TestData.DATA2.get(recordCount)[1]); // column b + + recordCount++; + } + assertThat(recordCount).isEqualTo(TestData.DATA2.size()); + } + } + } + } + + @ParameterizedTest + @ValueSource(bytes = {LOG_MAGIC_VALUE_V0, LOG_MAGIC_VALUE_V1, LOG_MAGIC_VALUE_V2}) + void testProjectRecordBatchAllColumns(byte recordBatchMagic) throws Exception { + // Use schemaId=2 which corresponds to DATA2_SCHEMA (3 columns) in testingSchemaGetter + int schemaIdForData2 = 2; + + // Test projection to all columns (should be equivalent to no projection) + FileLogRecords fileLogRecords = + createFileLogRecords( + schemaIdForData2, + recordBatchMagic, + TestData.DATA2_ROW_TYPE, + TestData.DATA2); + + FileLogProjection projection = new FileLogProjection(new ProjectionPushdownCache()); + projection.setCurrentProjection( + 1L, + testingSchemaGetter, + DEFAULT_COMPRESSION, + new int[] {0, 1, 2}); // Project all columns + + FileLogInputStream.FileChannelLogRecordBatch batch = + fileLogRecords.batchIterator(0, -1).next(); + BytesView projectedBytes = projection.projectRecordBatch(batch); + + assertThat(projectedBytes.getBytesLength()).isGreaterThan(0); + + // Verify all columns projection + LogRecords projectedRecords = new BytesViewLogRecords(projectedBytes); + RowType projectedType = TestData.DATA2_ROW_TYPE.project(new int[] {0, 1, 2}); + + try (LogRecordReadContext context = + createArrowReadContext(projectedType, schemaIdForData2, testingSchemaGetter)) { + for (LogRecordBatch projectedBatch : projectedRecords.batches()) { + try (CloseableIterator records = projectedBatch.records(context)) { + int recordCount = 0; + while (records.hasNext()) { + LogRecord record = records.next(); + InternalRow row = record.getRow(); + + assertThat(row.getFieldCount()).isEqualTo(3); + assertThat(row.getInt(0)) + .isEqualTo(TestData.DATA2.get(recordCount)[0]); // column a + assertThat(row.getString(1).toString()) + .isEqualTo(TestData.DATA2.get(recordCount)[1]); // column b + assertThat(row.getString(2).toString()) + .isEqualTo(TestData.DATA2.get(recordCount)[2]); // column c + + recordCount++; + } + assertThat(recordCount).isEqualTo(TestData.DATA2.size()); + } + } + } + } + + @Test + void testProjectRecordBatchNoProjectionSet() throws Exception { + FileLogProjection projection = new FileLogProjection(new ProjectionPushdownCache()); + + // Create a file with actual data so we can get a valid batch + FileLogRecords fileLogRecords = + createFileLogRecords(LOG_MAGIC_VALUE_V2, TestData.DATA2_ROW_TYPE, TestData.DATA2); + FileLogInputStream.FileChannelLogRecordBatch batch = + fileLogRecords.batchIterator(0, -1).next(); + + // Should throw exception when no projection is set + // getOrCreateProjectionInfo throws NPE because no projection has been registered + assertThatThrownBy(() -> projection.projectRecordBatch(batch)) + .isInstanceOf(NullPointerException.class); + } + + @ParameterizedTest + @ValueSource(bytes = {LOG_MAGIC_VALUE_V0, LOG_MAGIC_VALUE_V1, LOG_MAGIC_VALUE_V2}) + void testProjectRecordBatchMultipleBatches(byte recordBatchMagic) throws Exception { + // Test projection across multiple batches + FileLogRecords fileLogRecords = + createFileLogRecords( + recordBatchMagic, + TestData.DATA1_ROW_TYPE, + TestData.DATA1, + TestData.ANOTHER_DATA1); + + FileLogProjection projection = new FileLogProjection(new ProjectionPushdownCache()); + projection.setCurrentProjection( + 1L, + testingSchemaGetter, + DEFAULT_COMPRESSION, + new int[] {0}); // Project only column a + + // Test projection on first batch + FileLogInputStream.FileChannelLogRecordBatch firstBatch = + fileLogRecords.batchIterator(0, -1).next(); + BytesView firstProjectedBytes = projection.projectRecordBatch(firstBatch); + assertThat(firstProjectedBytes.getBytesLength()).isGreaterThan(0); + + // Test projection on second batch + FileLogInputStream.FileChannelLogRecordBatch secondBatch = + fileLogRecords + .batchIterator(firstBatch.position() + firstBatch.sizeInBytes(), -1) + .next(); + BytesView secondProjectedBytes = projection.projectRecordBatch(secondBatch); + assertThat(secondProjectedBytes.getBytesLength()).isGreaterThan(0); + + // Verify both projections work correctly + LogRecords firstProjectedRecords = new BytesViewLogRecords(firstProjectedBytes); + LogRecords secondProjectedRecords = new BytesViewLogRecords(secondProjectedBytes); + RowType projectedType = TestData.DATA1_ROW_TYPE.project(new int[] {0}); + + try (LogRecordReadContext context = + createArrowReadContext(projectedType, DEFAULT_SCHEMA_ID, testingSchemaGetter)) { + // Verify first batch + int firstBatchCount = 0; + for (LogRecordBatch projectedBatch : firstProjectedRecords.batches()) { + try (CloseableIterator records = projectedBatch.records(context)) { + while (records.hasNext()) { + LogRecord record = records.next(); + InternalRow row = record.getRow(); + assertThat(row.getInt(0)).isEqualTo(TestData.DATA1.get(firstBatchCount)[0]); + firstBatchCount++; + } + } + } + assertThat(firstBatchCount).isEqualTo(TestData.DATA1.size()); + + // Verify second batch + int secondBatchCount = 0; + for (LogRecordBatch projectedBatch : secondProjectedRecords.batches()) { + try (CloseableIterator records = projectedBatch.records(context)) { + while (records.hasNext()) { + LogRecord record = records.next(); + InternalRow row = record.getRow(); + assertThat(row.getInt(0)) + .isEqualTo(TestData.ANOTHER_DATA1.get(secondBatchCount)[0]); + secondBatchCount++; + } + } + } + assertThat(secondBatchCount).isEqualTo(TestData.ANOTHER_DATA1.size()); + } + } + + @ParameterizedTest + @ValueSource(bytes = {LOG_MAGIC_VALUE_V2}) + void testProjectRecordBatchStatisticsClearing(byte recordBatchMagic) throws Exception { + // Test that statistics are properly cleared during projection for V2+ versions + FileLogRecords fileLogRecords = + createFileLogRecords(recordBatchMagic, TestData.DATA2_ROW_TYPE, TestData.DATA2); + + FileLogProjection projection = new FileLogProjection(new ProjectionPushdownCache()); + projection.setCurrentProjection( + 1L, + testingSchemaGetter, + DEFAULT_COMPRESSION, + new int[] {0, 1}); // Project columns a and b + + FileLogInputStream.FileChannelLogRecordBatch batch = + fileLogRecords.batchIterator(0, -1).next(); + BytesView projectedBytes = projection.projectRecordBatch(batch); + + assertThat(projectedBytes.getBytesLength()).isGreaterThan(0); + + // Verify the projected batch has statistics cleared + LogRecords projectedRecords = new BytesViewLogRecords(projectedBytes); + RowType projectedType = TestData.DATA2_ROW_TYPE.project(new int[] {0, 1}); + + try (LogRecordReadContext context = + createArrowReadContext(projectedType, DEFAULT_SCHEMA_ID, testingSchemaGetter)) { + for (LogRecordBatch projectedBatch : projectedRecords.batches()) { + // Verify that statistics are not available in projected batch + assertThat(projectedBatch.getStatistics(context)).isEmpty(); + + // Verify the projected data is correct + try (CloseableIterator records = projectedBatch.records(context)) { + int recordCount = 0; + while (records.hasNext()) { + LogRecord record = records.next(); + InternalRow row = record.getRow(); + + assertThat(row.getFieldCount()).isEqualTo(2); + assertThat(row.getInt(0)) + .isEqualTo(TestData.DATA2.get(recordCount)[0]); // column a + assertThat(row.getString(1).toString()) + .isEqualTo(TestData.DATA2.get(recordCount)[1]); // column b + + recordCount++; + } + assertThat(recordCount).isEqualTo(TestData.DATA2.size()); + } + } + } + } + + @ParameterizedTest + @ValueSource(bytes = {LOG_MAGIC_VALUE_V0, LOG_MAGIC_VALUE_V1}) + void testProjectRecordBatchNoStatisticsClearing(byte recordBatchMagic) throws Exception { + // Test that statistics clearing only happens for V2+ versions + FileLogRecords fileLogRecords = + createFileLogRecords(recordBatchMagic, TestData.DATA2_ROW_TYPE, TestData.DATA2); + + FileLogProjection projection = new FileLogProjection(new ProjectionPushdownCache()); + projection.setCurrentProjection( + 1L, + testingSchemaGetter, + DEFAULT_COMPRESSION, + new int[] {0, 1}); // Project columns a and b + + FileLogInputStream.FileChannelLogRecordBatch batch = + fileLogRecords.batchIterator(0, -1).next(); + BytesView projectedBytes = projection.projectRecordBatch(batch); + + assertThat(projectedBytes.getBytesLength()).isGreaterThan(0); + + // Verify the projected batch for V0/V1 versions (no statistics to clear) + LogRecords projectedRecords = new BytesViewLogRecords(projectedBytes); + RowType projectedType = TestData.DATA2_ROW_TYPE.project(new int[] {0, 1}); + + try (LogRecordReadContext context = + createArrowReadContext(projectedType, DEFAULT_SCHEMA_ID, testingSchemaGetter)) { + for (LogRecordBatch projectedBatch : projectedRecords.batches()) { + // For V0/V1, statistics should be 0 (not supported) + assertThat(projectedBatch.getStatistics(context)).isEmpty(); + + // Verify the projected data is correct + try (CloseableIterator records = projectedBatch.records(context)) { + int recordCount = 0; + while (records.hasNext()) { + LogRecord record = records.next(); + InternalRow row = record.getRow(); + + assertThat(row.getFieldCount()).isEqualTo(2); + assertThat(row.getInt(0)) + .isEqualTo(TestData.DATA2.get(recordCount)[0]); // column a + assertThat(row.getString(1).toString()) + .isEqualTo(TestData.DATA2.get(recordCount)[1]); // column b + + recordCount++; + } + assertThat(recordCount).isEqualTo(TestData.DATA2.size()); + } + } + } + } + + @ParameterizedTest + @ValueSource(bytes = {LOG_MAGIC_VALUE_V2}) + void testProjectStatisticsClearing(byte recordBatchMagic) throws Exception { + // Use schemaId=2 which corresponds to DATA2_SCHEMA (3 columns) in testingSchemaGetter + int schemaIdForData2 = 2; + + // Test that statistics are properly cleared during project() method for V2+ versions + FileLogRecords fileLogRecords = + createFileLogRecords( + schemaIdForData2, + recordBatchMagic, + TestData.DATA2_ROW_TYPE, + TestData.DATA2, + TestData.DATA2); // Multiple batches + + FileLogProjection projection = new FileLogProjection(new ProjectionPushdownCache()); + projection.setCurrentProjection( + 1L, + testingSchemaGetter, + DEFAULT_COMPRESSION, + new int[] {0, 2}); // Project columns a and c + + // Use project() method instead of projectRecordBatch() + BytesViewLogRecords projectedRecords = + projection.project( + fileLogRecords.channel(), + 0, + fileLogRecords.sizeInBytes(), + Integer.MAX_VALUE); + + assertThat(projectedRecords.sizeInBytes()).isGreaterThan(0); + + // Verify all projected batches have statistics cleared + RowType projectedType = TestData.DATA2_ROW_TYPE.project(new int[] {0, 2}); + + try (LogRecordReadContext context = + createArrowReadContext(projectedType, schemaIdForData2, testingSchemaGetter)) { + for (LogRecordBatch projectedBatch : projectedRecords.batches()) { + // Verify that statistics are not available in projected batch + assertThat(projectedBatch.getStatistics(context)).isEmpty(); + + // Verify the projected data is correct + try (CloseableIterator records = projectedBatch.records(context)) { + int recordCount = 0; + while (records.hasNext()) { + LogRecord record = records.next(); + InternalRow row = record.getRow(); + + assertThat(row.getFieldCount()).isEqualTo(2); + // Verify projected columns contain correct data (columns a and c) + assertThat(row.getInt(0)).isEqualTo(TestData.DATA2.get(recordCount)[0]); + assertThat(row.getString(1).toString()) + .isEqualTo(TestData.DATA2.get(recordCount)[2]); + recordCount++; + } + } + } + } + } + + @ParameterizedTest + @ValueSource(bytes = {LOG_MAGIC_VALUE_V2}) + void testProjectRecordBatchWithStatisticsFlag(byte recordBatchMagic) throws Exception { + // Test projection when original batch has statistics flag set + FileLogRecords fileLogRecords = + createFileLogRecords(recordBatchMagic, TestData.DATA2_ROW_TYPE, TestData.DATA2); + + FileLogProjection projection = new FileLogProjection(new ProjectionPushdownCache()); + projection.setCurrentProjection( + 1L, + testingSchemaGetter, + DEFAULT_COMPRESSION, + new int[] {1}); // Project only column b + + FileLogInputStream.FileChannelLogRecordBatch batch = + fileLogRecords.batchIterator(0, -1).next(); + BytesView projectedBytes = projection.projectRecordBatch(batch); + + assertThat(projectedBytes.getBytesLength()).isGreaterThan(0); + + // Verify the projected batch has statistics cleared even if original had statistics + LogRecords projectedRecords = new BytesViewLogRecords(projectedBytes); + RowType projectedType = TestData.DATA2_ROW_TYPE.project(new int[] {1}); + + try (LogRecordReadContext context = + createArrowReadContext(projectedType, DEFAULT_SCHEMA_ID, testingSchemaGetter)) { + for (LogRecordBatch projectedBatch : projectedRecords.batches()) { + // Verify that statistics are cleared in projected batch + assertThat(projectedBatch.getStatistics(context)).isEmpty(); + + // Verify the projected data is correct + try (CloseableIterator records = projectedBatch.records(context)) { + int recordCount = 0; + while (records.hasNext()) { + LogRecord record = records.next(); + InternalRow row = record.getRow(); + + assertThat(row.getFieldCount()).isEqualTo(1); + assertThat(row.getString(0).toString()) + .isEqualTo(TestData.DATA2.get(recordCount)[1]); // column b + + recordCount++; + } + assertThat(recordCount).isEqualTo(TestData.DATA2.size()); + } + } + } + } } diff --git a/fluss-common/src/test/java/org/apache/fluss/record/LogRecordBatchIteratorTest.java b/fluss-common/src/test/java/org/apache/fluss/record/LogRecordBatchIteratorTest.java new file mode 100644 index 0000000000..d2ac7c674e --- /dev/null +++ b/fluss-common/src/test/java/org/apache/fluss/record/LogRecordBatchIteratorTest.java @@ -0,0 +1,82 @@ +/* + * 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.record; + +import org.apache.fluss.testutils.DataTestUtils; + +import org.junit.jupiter.api.Test; + +import static org.apache.fluss.record.TestData.DATA1; +import static org.apache.fluss.record.TestData.DATA1_ROW_TYPE; +import static org.apache.fluss.record.TestData.DEFAULT_SCHEMA_ID; +import static org.apache.fluss.record.TestData.TEST_SCHEMA_GETTER; +import static org.assertj.core.api.Assertions.assertThat; + +/** Test for {@link LogRecordBatchIterator}. */ +public class LogRecordBatchIteratorTest { + + @Test + void testBasicLogRecordBatchIterator() throws Exception { + MemoryLogRecords memoryLogRecords = + LogRecordBatchStatisticsTestUtils.createLogRecordsWithStatistics( + DATA1, DATA1_ROW_TYPE, 0L, DEFAULT_SCHEMA_ID); + + LogRecordBatchIterator iterator = + new LogRecordBatchIterator<>( + new MemorySegmentLogInputStream( + memoryLogRecords.getMemorySegment(), + memoryLogRecords.getPosition(), + memoryLogRecords.sizeInBytes())); + + assertThat(iterator.hasNext()).isTrue(); + LogRecordBatch batch = iterator.next(); + assertThat(batch).isNotNull(); + assertThat(batch.getRecordCount()).isEqualTo(DATA1.size()); + assertThat(batch.baseLogOffset()).isEqualTo(0L); + + try (LogRecordReadContext readContext = + LogRecordReadContext.createArrowReadContext( + DATA1_ROW_TYPE, DEFAULT_SCHEMA_ID, TEST_SCHEMA_GETTER)) { + assertThat(batch.getStatistics(readContext)).isPresent(); + } + assertThat(iterator.hasNext()).isFalse(); + } + + @Test + void testGetStatisticsWithNullReadContext() throws Exception { + MemoryLogRecords memoryLogRecords = + LogRecordBatchStatisticsTestUtils.createLogRecordsWithStatistics( + DATA1, DATA1_ROW_TYPE, 0L, DEFAULT_SCHEMA_ID); + + LogRecordBatch batch = memoryLogRecords.batches().iterator().next(); + assertThat(batch.getStatistics(null)).isEmpty(); + } + + @Test + void testGetStatisticsWithV0Magic() throws Exception { + MemoryLogRecords memoryLogRecords = + DataTestUtils.genMemoryLogRecordsByObject((byte) 0, DATA1); + + LogRecordBatch batch = memoryLogRecords.batches().iterator().next(); + try (LogRecordReadContext readContext = + LogRecordReadContext.createArrowReadContext( + DATA1_ROW_TYPE, DEFAULT_SCHEMA_ID, TEST_SCHEMA_GETTER)) { + assertThat(batch.getStatistics(readContext)).isEmpty(); + } + } +} diff --git a/fluss-common/src/test/java/org/apache/fluss/record/RecordBatchFilterTest.java b/fluss-common/src/test/java/org/apache/fluss/record/RecordBatchFilterTest.java new file mode 100644 index 0000000000..df5b2aaf9d --- /dev/null +++ b/fluss-common/src/test/java/org/apache/fluss/record/RecordBatchFilterTest.java @@ -0,0 +1,463 @@ +/* + * 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.record; + +import org.apache.fluss.memory.MemorySegment; +import org.apache.fluss.memory.MemorySegmentOutputView; +import org.apache.fluss.metadata.Schema; +import org.apache.fluss.metadata.SchemaGetter; +import org.apache.fluss.metadata.SchemaInfo; +import org.apache.fluss.predicate.Predicate; +import org.apache.fluss.predicate.PredicateBuilder; +import org.apache.fluss.row.BinaryString; +import org.apache.fluss.row.GenericRow; +import org.apache.fluss.types.DataTypes; +import org.apache.fluss.types.RowType; +import org.apache.fluss.utils.SchemaUtil; + +import org.junit.jupiter.api.Test; + +import java.io.IOException; +import java.util.Arrays; +import java.util.List; +import java.util.Optional; + +import static org.assertj.core.api.Assertions.assertThat; + +/** Tests for schema-aware record-batch predicate evaluation. */ +class RecordBatchFilterTest { + + private static final int TEST_SCHEMA_ID = 123; + + // Helper method to create test predicate + private Predicate createTestPredicate() { + PredicateBuilder builder = new PredicateBuilder(TestData.FILTER_TEST_ROW_TYPE); + return builder.greaterThan(0, 5L); // id > 5 + } + + // Helper method to create statistics for testing schema-aware behavior + private DefaultLogRecordBatchStatistics createTestStatistics(int schemaId) { + try { + int[] statsMapping = new int[] {0, 1, 2}; // map all columns + + // Create collector and simulate processing some data + LogRecordBatchStatisticsCollector collector = + new LogRecordBatchStatisticsCollector( + TestData.FILTER_TEST_ROW_TYPE, statsMapping); + + // Process some test rows to generate statistics + for (Object[] data : TestData.FILTER_TEST_DATA) { + GenericRow row = new GenericRow(3); + row.setField(0, data[0]); + row.setField(1, BinaryString.fromString((String) data[1])); + row.setField(2, data[2]); + collector.processRow(row); + } + + // Serialize the statistics to memory + MemorySegmentOutputView outputView = new MemorySegmentOutputView(1024); + + collector.writeStatistics(outputView); + + // Get the written data + MemorySegment segment = outputView.getMemorySegment(); + + // Parse the serialized statistics back + + return LogRecordBatchStatisticsParser.parseStatistics( + segment, 0, TestData.FILTER_TEST_ROW_TYPE, schemaId); + } catch (IOException e) { + throw new RuntimeException("Failed to create test statistics", e); + } + } + + @Test + void testSchemaAwareStatisticsWithMatchingSchemaId() { + Predicate testPredicate = createTestPredicate(); // id > 5 + DefaultLogRecordBatchStatistics testStats = createTestStatistics(TEST_SCHEMA_ID); + + boolean result = mayMatch(testPredicate, TEST_SCHEMA_ID, null, 100L, testStats); + + // The statistics have max id=10 > 5, so predicate should return true (possible match) + assertThat(result).isTrue(); + } + + @Test + void testSchemaAwareStatisticsWithMismatchedSchemaIdNoEvolution() { + Predicate testPredicate = createTestPredicate(); // id > 5 + DefaultLogRecordBatchStatistics mismatchedStats = + createTestStatistics(456); // Different schema ID + + // Filter without schema evolution support + boolean result = mayMatch(testPredicate, TEST_SCHEMA_ID, null, 100L, mismatchedStats); + + // With mismatched schema ID and no schema evolution support, should return true + assertThat(result).isTrue(); + } + + @Test + void testSchemaAwareStatisticsWithNullStats() { + Predicate testPredicate = createTestPredicate(); // id > 5 + + boolean result = mayMatch(testPredicate, TEST_SCHEMA_ID, null, 100L, null); + + // With null statistics, should return true (cannot filter) + assertThat(result).isTrue(); + } + + // ===================================================== + // Schema Evolution Tests + // ===================================================== + + // Schema V1: id(0), name(1), score(2) with columnIds 1, 2, 3 + private static final Schema SCHEMA_V1 = + Schema.newBuilder() + .fromColumns( + Arrays.asList( + new Schema.Column("id", DataTypes.BIGINT(), null, 1), + new Schema.Column("name", DataTypes.STRING(), null, 2), + new Schema.Column("score", DataTypes.DOUBLE(), null, 3))) + .build(); + + // Schema V2: id(0), name(1), age(2), score(3) - added 'age' column with columnId 4 + // Column order: id(columnId=1), name(columnId=2), age(columnId=4), score(columnId=3) + private static final Schema SCHEMA_V2 = + Schema.newBuilder() + .fromColumns( + Arrays.asList( + new Schema.Column("id", DataTypes.BIGINT(), null, 1), + new Schema.Column("name", DataTypes.STRING(), null, 2), + new Schema.Column( + "age", DataTypes.INT(), null, 4), // New column + new Schema.Column("score", DataTypes.DOUBLE(), null, 3))) + .build(); + + private static final int SCHEMA_ID_V1 = 1; + private static final int SCHEMA_ID_V2 = 2; + + /** Create a TestingSchemaGetter with multiple schemas. */ + private TestingSchemaGetter createSchemaGetter( + int latestSchemaId, Schema latestSchema, int otherSchemaId, Schema otherSchema) { + TestingSchemaGetter getter = + new TestingSchemaGetter(new SchemaInfo(latestSchema, latestSchemaId)); + getter.updateLatestSchemaInfo(new SchemaInfo(otherSchema, otherSchemaId)); + return getter; + } + + /** Create statistics for a given schema. */ + private DefaultLogRecordBatchStatistics createStatisticsForSchema( + Schema schema, int schemaId, List testData) { + try { + RowType rowType = schema.getRowType(); + int[] statsMapping = new int[rowType.getFieldCount()]; + for (int i = 0; i < statsMapping.length; i++) { + statsMapping[i] = i; + } + + LogRecordBatchStatisticsCollector collector = + new LogRecordBatchStatisticsCollector(rowType, statsMapping); + + for (Object[] data : testData) { + GenericRow row = new GenericRow(data.length); + for (int i = 0; i < data.length; i++) { + if (data[i] instanceof String) { + row.setField(i, BinaryString.fromString((String) data[i])); + } else { + row.setField(i, data[i]); + } + } + collector.processRow(row); + } + + MemorySegmentOutputView outputView = new MemorySegmentOutputView(1024); + collector.writeStatistics(outputView); + MemorySegment segment = outputView.getMemorySegment(); + + return LogRecordBatchStatisticsParser.parseStatistics(segment, 0, rowType, schemaId); + } catch (IOException e) { + throw new RuntimeException("Failed to create test statistics", e); + } + } + + @Test + void testSchemaEvolution_FilterWithSameSchema() { + // Create predicate using RowType (no columnId needed - all schema evolution handled by + // Filter) + PredicateBuilder builder = new PredicateBuilder(SCHEMA_V1.getRowType()); + Predicate predicate = builder.greaterThan(0, 5L); // id > 5 + + // Create statistics for schema V1 + List v1Data = + Arrays.asList( + new Object[] {1L, "a", 1.0}, new Object[] {10L, "z", 100.0}); // max id = 10 + DefaultLogRecordBatchStatistics stats = + createStatisticsForSchema(SCHEMA_V1, SCHEMA_ID_V1, v1Data); + + boolean result = mayMatch(predicate, SCHEMA_ID_V1, null, 100L, stats); + + // max(id)=10 > 5, should return true (possible match) + assertThat(result).isTrue(); + } + + @Test + void testSchemaEvolution_FilterOldDataWithNewSchema() { + // Create predicate using V2's RowType + // Filter on 'id' column (index 0 in V2) + PredicateBuilder builder = new PredicateBuilder(SCHEMA_V2.getRowType()); + Predicate predicate = builder.greaterThan(0, 5L); // id > 5 + + // Statistics from old data (Schema V1, schemaId = 1) + List v1Data = + Arrays.asList( + new Object[] {1L, "a", 1.0}, new Object[] {10L, "z", 100.0}); // max id = 10 + DefaultLogRecordBatchStatistics statsV1 = + createStatisticsForSchema(SCHEMA_V1, SCHEMA_ID_V1, v1Data); + + // Create SchemaGetter for schema evolution support + TestingSchemaGetter schemaGetter = + createSchemaGetter(SCHEMA_ID_V2, SCHEMA_V2, SCHEMA_ID_V1, SCHEMA_V1); + + boolean result = mayMatch(predicate, SCHEMA_ID_V2, schemaGetter, 100L, statsV1); + + // Schema evolution: using SchemaUtil.getIndexMapping(statsSchema=V1, filterSchema=V2) + // Filter field index 0 (id) maps to stats field index 0 + // max(id)=10 > 5 => true + assertThat(result).isTrue(); + } + + @Test + void testSchemaEvolution_FilterWithNewColumnPredicate() { + // Predicate on 'age' column (only exists in V2) + // 'age' is at index 2 in V2 + PredicateBuilder builder = new PredicateBuilder(SCHEMA_V2.getRowType()); + Predicate predicate = builder.greaterThan(2, 18); // age > 18 + + // Statistics from old data (Schema V1, doesn't have 'age') + List v1Data = + Arrays.asList(new Object[] {1L, "a", 1.0}, new Object[] {10L, "z", 100.0}); + DefaultLogRecordBatchStatistics statsV1 = + createStatisticsForSchema(SCHEMA_V1, SCHEMA_ID_V1, v1Data); + + // Create SchemaGetter for schema evolution support + TestingSchemaGetter schemaGetter = + createSchemaGetter(SCHEMA_ID_V2, SCHEMA_V2, SCHEMA_ID_V1, SCHEMA_V1); + + boolean result = mayMatch(predicate, SCHEMA_ID_V2, schemaGetter, 100L, statsV1); + + // 'age' (index 2 in V2, columnId=4) doesn't exist in V1 + // indexMapping[2] = UNEXIST_MAPPING + // Cannot adapt predicate, should return true (include batch, safe fallback) + assertThat(result).isTrue(); + } + + @Test + void testSchemaEvolution_FilterCanReject() { + // Predicate: id > 100 + PredicateBuilder builder = new PredicateBuilder(SCHEMA_V2.getRowType()); + Predicate predicate = builder.greaterThan(0, 100L); // id > 100 + + // Statistics with max id = 10 (all data id <= 10) + List v1Data = + Arrays.asList( + new Object[] {1L, "a", 1.0}, new Object[] {10L, "z", 100.0}); // max id = 10 + DefaultLogRecordBatchStatistics statsV1 = + createStatisticsForSchema(SCHEMA_V1, SCHEMA_ID_V1, v1Data); + + // Create SchemaGetter for schema evolution support + TestingSchemaGetter schemaGetter = + createSchemaGetter(SCHEMA_ID_V2, SCHEMA_V2, SCHEMA_ID_V1, SCHEMA_V1); + + boolean result = mayMatch(predicate, SCHEMA_ID_V2, schemaGetter, 100L, statsV1); + + // max(id)=10 < 100, predicate id > 100 can never be satisfied + // Should return false (reject batch) + assertThat(result).isFalse(); + } + + @Test + void testSchemaEvolution_CompoundPredicate() { + // Compound predicate: id > 5 AND score < 50 + PredicateBuilder builder = new PredicateBuilder(SCHEMA_V2.getRowType()); + Predicate idPredicate = builder.greaterThan(0, 5L); // id > 5 + Predicate scorePredicate = builder.lessThan(3, 50.0); // score < 50 (index 3 in V2) + Predicate compoundPredicate = PredicateBuilder.and(idPredicate, scorePredicate); + + // Statistics: id range [1, 10], score range [1.0, 100.0] + List v1Data = + Arrays.asList(new Object[] {1L, "a", 1.0}, new Object[] {10L, "z", 100.0}); + DefaultLogRecordBatchStatistics statsV1 = + createStatisticsForSchema(SCHEMA_V1, SCHEMA_ID_V1, v1Data); + + // Create SchemaGetter for schema evolution support + TestingSchemaGetter schemaGetter = + createSchemaGetter(SCHEMA_ID_V2, SCHEMA_V2, SCHEMA_ID_V1, SCHEMA_V1); + + boolean result = mayMatch(compoundPredicate, SCHEMA_ID_V2, schemaGetter, 100L, statsV1); + + // Both conditions can potentially be satisfied: + // - max(id)=10 > 5: true + // - min(score)=1.0 < 50: true + // Should return true (possible match) + assertThat(result).isTrue(); + } + + @Test + void testSchemaEvolution_OrCompoundPredicate() { + // OR predicate: id > 100 OR score < 50 + PredicateBuilder builder = new PredicateBuilder(SCHEMA_V2.getRowType()); + Predicate idPredicate = builder.greaterThan(0, 100L); // id > 100 + Predicate scorePredicate = builder.lessThan(3, 50.0); // score < 50 (index 3 in V2) + Predicate orPredicate = PredicateBuilder.or(idPredicate, scorePredicate); + + // Statistics: id range [1, 10], score range [1.0, 100.0] + List v1Data = + Arrays.asList(new Object[] {1L, "a", 1.0}, new Object[] {10L, "z", 100.0}); + DefaultLogRecordBatchStatistics statsV1 = + createStatisticsForSchema(SCHEMA_V1, SCHEMA_ID_V1, v1Data); + + TestingSchemaGetter schemaGetter = + createSchemaGetter(SCHEMA_ID_V2, SCHEMA_V2, SCHEMA_ID_V1, SCHEMA_V1); + + boolean result = mayMatch(orPredicate, SCHEMA_ID_V2, schemaGetter, 100L, statsV1); + + // id > 100: max(id)=10 < 100 → false + // score < 50: min(score)=1.0 < 50 → true + // OR: false OR true → true (batch should be included) + assertThat(result).isTrue(); + } + + @Test + void testSchemaEvolution_OrPredicateBothFalse() { + // OR predicate where both branches reject: id > 100 OR score > 200 + PredicateBuilder builder = new PredicateBuilder(SCHEMA_V2.getRowType()); + Predicate idPredicate = builder.greaterThan(0, 100L); // id > 100 + Predicate scorePredicate = builder.greaterThan(3, 200.0); // score > 200 (index 3 in V2) + Predicate orPredicate = PredicateBuilder.or(idPredicate, scorePredicate); + + // Statistics: id range [1, 10], score range [1.0, 100.0] + List v1Data = + Arrays.asList(new Object[] {1L, "a", 1.0}, new Object[] {10L, "z", 100.0}); + DefaultLogRecordBatchStatistics statsV1 = + createStatisticsForSchema(SCHEMA_V1, SCHEMA_ID_V1, v1Data); + + TestingSchemaGetter schemaGetter = + createSchemaGetter(SCHEMA_ID_V2, SCHEMA_V2, SCHEMA_ID_V1, SCHEMA_V1); + + boolean result = mayMatch(orPredicate, SCHEMA_ID_V2, schemaGetter, 100L, statsV1); + + // id > 100: max(id)=10 < 100 → false + // score > 200: max(score)=100.0 < 200 → false + // OR: false OR false → false (batch should be rejected) + assertThat(result).isFalse(); + } + + @Test + void testSchemaEvolution_OrPredicateWithUnmappedColumn() { + // OR predicate: id > 100 OR age > 5 + // 'age' exists in V2 but not in V1 + PredicateBuilder builder = new PredicateBuilder(SCHEMA_V2.getRowType()); + Predicate idPredicate = builder.greaterThan(0, 100L); // id > 100 + Predicate agePredicate = builder.greaterThan(2, 5); // age > 5 (index 2 in V2) + Predicate orPredicate = PredicateBuilder.or(idPredicate, agePredicate); + + // Statistics from V1 (no 'age' column) + List v1Data = + Arrays.asList(new Object[] {1L, "a", 1.0}, new Object[] {10L, "z", 100.0}); + DefaultLogRecordBatchStatistics statsV1 = + createStatisticsForSchema(SCHEMA_V1, SCHEMA_ID_V1, v1Data); + + TestingSchemaGetter schemaGetter = + createSchemaGetter(SCHEMA_ID_V2, SCHEMA_V2, SCHEMA_ID_V1, SCHEMA_V1); + + boolean result = mayMatch(orPredicate, SCHEMA_ID_V2, schemaGetter, 100L, statsV1); + + // 'age' cannot be mapped to V1, so the predicate adaptation should fail + // or return a safe fallback (include the batch) + assertThat(result).isTrue(); + } + + @Test + void testSchemaEvolution_WithoutSchemaGetter() { + // Predicate created with V2's RowType + PredicateBuilder builder = new PredicateBuilder(SCHEMA_V2.getRowType()); + Predicate predicate = builder.greaterThan(0, 5L); // id > 5 + + // Statistics from old data (Schema V1) + List v1Data = + Arrays.asList(new Object[] {1L, "a", 1.0}, new Object[] {10L, "z", 100.0}); + DefaultLogRecordBatchStatistics statsV1 = + createStatisticsForSchema(SCHEMA_V1, SCHEMA_ID_V1, v1Data); + + boolean result = mayMatch(predicate, SCHEMA_ID_V2, null, 100L, statsV1); + + // Without SchemaGetter, cannot perform schema evolution + // Should return true (safe fallback) + assertThat(result).isTrue(); + } + + private boolean mayMatch( + Predicate predicate, + int predicateSchemaId, + SchemaGetter schemaGetter, + long recordCount, + DefaultLogRecordBatchStatistics statistics) { + if (statistics == null) { + return true; + } + + Predicate effectivePredicate = + resolvePredicateForSchema(predicate, predicateSchemaId, schemaGetter, statistics); + if (effectivePredicate == null) { + return true; + } + + return effectivePredicate.test( + recordCount, + statistics.getMinValues(), + statistics.getMaxValues(), + statistics.getNullCounts()); + } + + private Predicate resolvePredicateForSchema( + Predicate predicate, + int predicateSchemaId, + SchemaGetter schemaGetter, + DefaultLogRecordBatchStatistics statistics) { + if (predicateSchemaId == statistics.getSchemaId() || predicateSchemaId < 0) { + return predicate; + } + if (schemaGetter == null) { + return null; + } + + try { + Schema predicateSchema = schemaGetter.getSchema(predicateSchemaId); + Schema batchSchema = schemaGetter.getSchema(statistics.getSchemaId()); + if (predicateSchema == null || batchSchema == null) { + return null; + } + + int[] indexMapping = SchemaUtil.getIndexMapping(batchSchema, predicateSchema); + Optional adapted = + PredicateBuilder.transformFieldMapping(predicate, indexMapping); + return adapted.orElse(null); + } catch (Exception e) { + return null; + } + } +} diff --git a/fluss-rpc/src/main/java/org/apache/fluss/rpc/entity/FetchLogResultForBucket.java b/fluss-rpc/src/main/java/org/apache/fluss/rpc/entity/FetchLogResultForBucket.java index e2cbfd322d..4c34fc31e8 100644 --- a/fluss-rpc/src/main/java/org/apache/fluss/rpc/entity/FetchLogResultForBucket.java +++ b/fluss-rpc/src/main/java/org/apache/fluss/rpc/entity/FetchLogResultForBucket.java @@ -35,6 +35,7 @@ public class FetchLogResultForBucket extends ResultForBucket { private final @Nullable RemoteLogFetchInfo remoteLogFetchInfo; private final @Nullable LogRecords records; private final long highWatermark; + private final long filteredEndOffset; public FetchLogResultForBucket( TableBucket tableBucket, LogRecords records, long highWatermark) { @@ -43,11 +44,12 @@ public FetchLogResultForBucket( null, checkNotNull(records, "records can not be null"), highWatermark, + -1L, ApiError.NONE); } public FetchLogResultForBucket(TableBucket tableBucket, ApiError error) { - this(tableBucket, null, null, -1L, error); + this(tableBucket, null, null, -1L, -1L, error); } public FetchLogResultForBucket( @@ -57,19 +59,32 @@ public FetchLogResultForBucket( checkNotNull(remoteLogFetchInfo, "remote log fetch info can not be null"), null, highWatermark, + -1L, ApiError.NONE); } + /** + * Create a filtered empty response with the correct next fetch offset. This is used when all + * batches are filtered out but we need to inform the client about the correct offset to + * continue fetching from. + */ + public FetchLogResultForBucket( + TableBucket tableBucket, long highWatermark, long filteredEndOffset) { + this(tableBucket, null, null, highWatermark, filteredEndOffset, ApiError.NONE); + } + private FetchLogResultForBucket( TableBucket tableBucket, @Nullable RemoteLogFetchInfo remoteLogFetchInfo, @Nullable LogRecords records, long highWatermark, + long filteredEndOffset, ApiError error) { super(tableBucket, error); this.remoteLogFetchInfo = remoteLogFetchInfo; this.records = records; this.highWatermark = highWatermark; + this.filteredEndOffset = filteredEndOffset; } /** @@ -102,4 +117,16 @@ public LogRecords recordsOrEmpty() { public long getHighWatermark() { return highWatermark; } + + /** + * Returns whether a filtered end offset is set, indicating that server-side filtering was + * applied and all batches were filtered out. + */ + public boolean hasFilteredEndOffset() { + return filteredEndOffset >= 0; + } + + public long getFilteredEndOffset() { + return filteredEndOffset; + } } diff --git a/fluss-rpc/src/main/java/org/apache/fluss/rpc/util/CommonRpcMessageUtils.java b/fluss-rpc/src/main/java/org/apache/fluss/rpc/util/CommonRpcMessageUtils.java index 3794c531e3..1a02c5d8bc 100644 --- a/fluss-rpc/src/main/java/org/apache/fluss/rpc/util/CommonRpcMessageUtils.java +++ b/fluss-rpc/src/main/java/org/apache/fluss/rpc/util/CommonRpcMessageUtils.java @@ -202,8 +202,18 @@ public static FetchLogResultForBucket getFetchLogResultForBucket( respForBucket.hasRecords() ? MemoryLogRecords.pointToByteBuffer(recordsBuffer) : MemoryLogRecords.EMPTY; - fetchLogResultForBucket = - new FetchLogResultForBucket(tb, records, respForBucket.getHighWatermark()); + if (respForBucket.hasFilteredEndOffset() + && respForBucket.getFilteredEndOffset() >= 0) { + fetchLogResultForBucket = + new FetchLogResultForBucket( + tb, + respForBucket.getHighWatermark(), + respForBucket.getFilteredEndOffset()); + } else { + fetchLogResultForBucket = + new FetchLogResultForBucket( + tb, records, respForBucket.getHighWatermark()); + } } } diff --git a/fluss-rpc/src/main/proto/FlussApi.proto b/fluss-rpc/src/main/proto/FlussApi.proto index d78503963b..c40a0ea41b 100644 --- a/fluss-rpc/src/main/proto/FlussApi.proto +++ b/fluss-rpc/src/main/proto/FlussApi.proto @@ -816,6 +816,8 @@ message PbFetchLogReqForTable { required bool projection_pushdown_enabled = 2; repeated int32 projected_fields = 3 [packed = true]; repeated PbFetchLogReqForBucket buckets_req = 4; + optional PbPredicate filter_predicate = 5; + optional int32 filter_schema_id = 6; } message PbFetchLogReqForBucket { @@ -839,6 +841,10 @@ message PbFetchLogRespForBucket { optional int64 log_start_offset = 6; // TODO now we don't introduce log start offset, but remain it in protobuf optional PbRemoteLogFetchInfo remote_log_fetch_info = 7; optional bytes records = 8; + // The next fetch offset when all batches in this bucket are filtered out by server-side + // filter. When set (>= 0), the records field MUST be empty — the two fields are mutually + // exclusive. The client should use this offset as the starting offset for the next fetch. + optional int64 filtered_end_offset = 9; } message PbPutKvReqForBucket { diff --git a/fluss-server/src/main/java/org/apache/fluss/server/kv/KvRecoverHelper.java b/fluss-server/src/main/java/org/apache/fluss/server/kv/KvRecoverHelper.java index e200f8d604..d8bd516552 100644 --- a/fluss-server/src/main/java/org/apache/fluss/server/kv/KvRecoverHelper.java +++ b/fluss-server/src/main/java/org/apache/fluss/server/kv/KvRecoverHelper.java @@ -205,6 +205,9 @@ private long readLogRecordsAndApply( recoverContext.maxFetchLogSizeInRecoverKv, fetchIsolation, true, + null, + null, + null, null) .getRecords(); if (logRecords == MemoryLogRecords.EMPTY) { diff --git a/fluss-server/src/main/java/org/apache/fluss/server/log/FetchDataInfo.java b/fluss-server/src/main/java/org/apache/fluss/server/log/FetchDataInfo.java index 4afecb6c6b..19396deb3e 100644 --- a/fluss-server/src/main/java/org/apache/fluss/server/log/FetchDataInfo.java +++ b/fluss-server/src/main/java/org/apache/fluss/server/log/FetchDataInfo.java @@ -19,12 +19,14 @@ import org.apache.fluss.annotation.Internal; import org.apache.fluss.record.LogRecords; +import org.apache.fluss.record.MemoryLogRecords; /** FetchDataInfo is used to store the data of a fetch request. */ @Internal public class FetchDataInfo { private final LogOffsetMetadata fetchOffsetMetadata; private final LogRecords records; + private final long filteredEndOffset; public LogRecords getRecords() { return records; @@ -39,7 +41,46 @@ public FetchDataInfo(LogRecords records) { } public FetchDataInfo(LogOffsetMetadata fetchOffsetMetadata, LogRecords records) { + this(fetchOffsetMetadata, records, -1L); + } + + public FetchDataInfo( + LogOffsetMetadata fetchOffsetMetadata, LogRecords records, long filteredEndOffset) { this.fetchOffsetMetadata = fetchOffsetMetadata; this.records = records; + this.filteredEndOffset = filteredEndOffset; + } + + /** + * Create a filtered empty response with the correct next fetch offset. This is used when all + * batches are filtered out but we need to inform the client about the correct offset to + * continue fetching from. + */ + public static FetchDataInfo createFilteredEmptyResponse( + LogOffsetMetadata fetchOffsetMetadata, long filteredEndOffset) { + return new FetchDataInfo(fetchOffsetMetadata, MemoryLogRecords.EMPTY, filteredEndOffset); + } + + /** + * Returns whether a filtered end offset is set, indicating that server-side filtering was + * applied and all batches were filtered out. + */ + public boolean hasFilteredEndOffset() { + return filteredEndOffset >= 0; + } + + /** + * Gets the next valid fetch offset that the client should use for subsequent fetch requests. + * + *

This method provides explicit offset information to address scenarios where the client + * cannot implicitly determine the next fetch offset from the lastOffset of received data. This + * is particularly important when using record batch filters (recordBatchFilter pushdown), where + * all batches might be filtered out, leaving no data for the client to derive the next offset + * from. + * + * @return the next valid offset for fetching, or -1 if not applicable + */ + public long getFilteredEndOffset() { + return filteredEndOffset; } } diff --git a/fluss-server/src/main/java/org/apache/fluss/server/log/FetchParams.java b/fluss-server/src/main/java/org/apache/fluss/server/log/FetchParams.java index 0ec42a0903..084529de2a 100644 --- a/fluss-server/src/main/java/org/apache/fluss/server/log/FetchParams.java +++ b/fluss-server/src/main/java/org/apache/fluss/server/log/FetchParams.java @@ -26,6 +26,7 @@ import javax.annotation.Nullable; +import java.util.Map; import java.util.Objects; /** Fetch data params. */ @@ -62,19 +63,20 @@ public final class FetchParams { private long fetchOffset; // whether column projection is enabled private boolean projectionEnabled = false; + // filter info per table (predicate + schema ID), null if no filters + @Nullable private final Map tableFilterInfoMap; // the lazily initialized projection util to read and project file logs @Nullable private FileLogProjection fileLogProjection; - private final int minFetchBytes; private final long maxWaitMs; // TODO: add more params like epoch etc. public FetchParams(int replicaId, int maxFetchBytes) { - this(replicaId, true, maxFetchBytes, DEFAULT_MIN_FETCH_BYTES, DEFAULT_MAX_WAIT_MS); + this(replicaId, true, maxFetchBytes, DEFAULT_MIN_FETCH_BYTES, DEFAULT_MAX_WAIT_MS, null); } public FetchParams(int replicaId, int maxFetchBytes, int minFetchBytes, long maxWaitMs) { - this(replicaId, true, maxFetchBytes, minFetchBytes, maxWaitMs); + this(replicaId, true, maxFetchBytes, minFetchBytes, maxWaitMs, null); } @VisibleForTesting @@ -83,7 +85,8 @@ public FetchParams( boolean fetchOnlyLeader, int maxFetchBytes, int minFetchBytes, - long maxWaitMs) { + long maxWaitMs, + @Nullable Map tableFilterInfoMap) { this.replicaId = replicaId; this.fetchOnlyLeader = fetchOnlyLeader; this.maxFetchBytes = maxFetchBytes; @@ -92,6 +95,7 @@ public FetchParams( this.fetchOffset = -1; this.minFetchBytes = minFetchBytes; this.maxWaitMs = maxWaitMs; + this.tableFilterInfoMap = tableFilterInfoMap; } public void setCurrentFetch( @@ -130,6 +134,15 @@ public FileLogProjection projection() { } } + /** Returns the filter info for the given table, or null if no filter is registered. */ + @Nullable + public FilterInfo getFilterInfo(long tableId) { + if (tableFilterInfoMap == null) { + return null; + } + return tableFilterInfoMap.get(tableId); + } + /** * Marks that at least one message has been read. This turns off the {@link #minOneMessage} * flag. @@ -190,18 +203,19 @@ public boolean equals(Object o) { return replicaId == that.replicaId && maxFetchBytes == that.maxFetchBytes && minFetchBytes == that.minFetchBytes - && maxWaitMs == that.maxWaitMs; + && maxWaitMs == that.maxWaitMs + && Objects.equals(tableFilterInfoMap, that.tableFilterInfoMap); } @Override public int hashCode() { - return Objects.hash(replicaId, maxFetchBytes, minFetchBytes, maxWaitMs); + return Objects.hash(replicaId, maxFetchBytes, minFetchBytes, maxWaitMs, tableFilterInfoMap); } @Override public String toString() { return "FetchParams(" - + ", replicaId=" + + "replicaId=" + replicaId + ", maxFetchBytes=" + maxFetchBytes @@ -209,6 +223,8 @@ public String toString() { + minFetchBytes + ", maxWaitMs=" + maxWaitMs + + ", tableFilterInfoMap=" + + tableFilterInfoMap + ')'; } } diff --git a/fluss-server/src/main/java/org/apache/fluss/server/log/FetchParamsBuilder.java b/fluss-server/src/main/java/org/apache/fluss/server/log/FetchParamsBuilder.java new file mode 100644 index 0000000000..30d7e80245 --- /dev/null +++ b/fluss-server/src/main/java/org/apache/fluss/server/log/FetchParamsBuilder.java @@ -0,0 +1,73 @@ +/* + * 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.server.log; + +import org.apache.fluss.annotation.VisibleForTesting; + +import java.util.Map; + +import static org.apache.fluss.server.log.FetchParams.DEFAULT_MAX_WAIT_MS; +import static org.apache.fluss.server.log.FetchParams.DEFAULT_MIN_FETCH_BYTES; + +/** Builder of FetchParams. */ +public final class FetchParamsBuilder { + private int replicaId; + private boolean fetchOnlyLeader = true; + private int maxFetchBytes; + private Map tableFilterInfoMap; + private int minFetchBytes; + private long maxWaitMs; + + public FetchParamsBuilder(int replicaId, int maxFetchBytes) { + this.replicaId = replicaId; + this.maxFetchBytes = maxFetchBytes; + this.minFetchBytes = DEFAULT_MIN_FETCH_BYTES; + this.maxWaitMs = DEFAULT_MAX_WAIT_MS; + } + + @VisibleForTesting + public FetchParamsBuilder withFetchOnlyLeader(boolean fetchOnlyLeader) { + this.fetchOnlyLeader = fetchOnlyLeader; + return this; + } + + public FetchParamsBuilder withTableFilterInfoMap(Map tableFilterInfoMap) { + this.tableFilterInfoMap = tableFilterInfoMap; + return this; + } + + public FetchParamsBuilder withMinFetchBytes(int minFetchBytes) { + this.minFetchBytes = minFetchBytes; + return this; + } + + public FetchParamsBuilder withMaxWaitMs(long maxWaitMs) { + this.maxWaitMs = maxWaitMs; + return this; + } + + public FetchParams build() { + return new FetchParams( + replicaId, + fetchOnlyLeader, + maxFetchBytes, + minFetchBytes, + maxWaitMs, + tableFilterInfoMap); + } +} diff --git a/fluss-server/src/main/java/org/apache/fluss/server/log/FilterInfo.java b/fluss-server/src/main/java/org/apache/fluss/server/log/FilterInfo.java new file mode 100644 index 0000000000..b020ec7ce2 --- /dev/null +++ b/fluss-server/src/main/java/org/apache/fluss/server/log/FilterInfo.java @@ -0,0 +1,39 @@ +/* + * 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.server.log; + +import org.apache.fluss.rpc.messages.PbPredicate; + +/** Holds the raw filter predicate and schema ID from a fetch request. */ +public final class FilterInfo { + private final PbPredicate pbPredicate; + private final int schemaId; + + public FilterInfo(PbPredicate pbPredicate, int schemaId) { + this.pbPredicate = pbPredicate; + this.schemaId = schemaId; + } + + public PbPredicate getPbPredicate() { + return pbPredicate; + } + + public int getSchemaId() { + return schemaId; + } +} diff --git a/fluss-server/src/main/java/org/apache/fluss/server/log/LocalLog.java b/fluss-server/src/main/java/org/apache/fluss/server/log/LocalLog.java index 130b3855d9..edfbd3abe8 100644 --- a/fluss-server/src/main/java/org/apache/fluss/server/log/LocalLog.java +++ b/fluss-server/src/main/java/org/apache/fluss/server/log/LocalLog.java @@ -26,7 +26,9 @@ import org.apache.fluss.metadata.TableBucket; import org.apache.fluss.metrics.Counter; import org.apache.fluss.metrics.Histogram; +import org.apache.fluss.predicate.Predicate; import org.apache.fluss.record.FileLogProjection; +import org.apache.fluss.record.LogRecordBatch; import org.apache.fluss.record.MemoryLogRecords; import org.apache.fluss.server.metrics.group.TabletServerMetricGroup; import org.apache.fluss.utils.FileUtils; @@ -69,6 +71,14 @@ public final class LocalLog { public static final long UNKNOWN_OFFSET = -1L; + /** + * Maximum number of segments to scan per fetch request when a record batch filter is active. + * This prevents a single fetch from scanning the entire log when the filter rejects all data, + * which could cause latency spikes. When the limit is reached, the client receives a filtered + * empty response with the current skip offset and retries from there. + */ + static final int MAX_FILTER_SCAN_SEGMENTS = 16; + private final Configuration config; private final LogSegments segments; private final TableBucket tableBucket; @@ -337,7 +347,8 @@ LogSegment createAndDeleteSegment( * offset is out of range, throw an OffsetOutOfRangeException. */ LogOffsetMetadata convertToOffsetMetadataOrThrow(long offset) throws IOException { - FetchDataInfo fetchDataInfo = read(offset, 1, false, nextOffsetMetadata, null); + FetchDataInfo fetchDataInfo = + read(offset, 1, false, nextOffsetMetadata, null, null, null, null); return fetchDataInfo.getFetchOffsetMetadata(); } @@ -349,6 +360,12 @@ LogOffsetMetadata convertToOffsetMetadataOrThrow(long offset) throws IOException * @param minOneMessage If this is true, the first message will be returned even if it exceeds * `maxLength` (if one exists) * @param maxOffsetMetadata The metadata of the maximum offset to be fetched + * @param projection The column projection to apply to the log records + * @param recordBatchFilter The filter to apply to the log records (must be null if readContext + * is null). Note: filter pushdown only takes effect for ARROW format tables whose batches + * include statistics (V1+ magic). For INDEXED/COMPACTED formats, all batches pass through. + * @param readContext The read context for batch filtering (must be null if recordBatchFilter is + * null) * @throws LogOffsetOutOfRangeException If startOffset is beyond the log start and end offset * @return The fetch data information including fetch starting offset metadata and messages * read. @@ -358,8 +375,17 @@ public FetchDataInfo read( int maxLength, boolean minOneMessage, LogOffsetMetadata maxOffsetMetadata, - @Nullable FileLogProjection projection) + @Nullable FileLogProjection projection, + @Nullable Predicate recordBatchFilter, + @Nullable LogRecordBatch.ReadContext readContext, + @Nullable PredicateSchemaResolver predicateResolver) throws IOException { + // Validate that recordBatchFilter and readContext are either both null or both non-null + if ((recordBatchFilter == null) != (readContext == null)) { + throw new IllegalArgumentException( + "recordBatchFilter and readContext must be either both null or both non-null"); + } + if (LOG.isTraceEnabled()) { LOG.trace( "Reading maximum {} bytes at offset {} from log with total length {} bytes for bucket {}", @@ -396,6 +422,14 @@ public FetchDataInfo read( // read from successive segments until we get some messages, or we reach the end of the // log FetchDataInfo fetchDataInfo = null; + // Track the latest filteredEndOffset across segments when filter is active. + // When a segment's batches are all filtered out, readWithFilter returns a non-null + // "filtered empty" FetchDataInfo (filteredEndOffset > 0, empty records). + // We must continue scanning subsequent segments instead of returning immediately. + long lastFilteredSkipOffset = -1L; + // Limit the number of segments scanned when filtering to prevent a single fetch + // request from scanning the entire log when the filter rejects all data. + int filteredSegmentsScanned = 0; while (fetchDataInfo == null && segmentOpt.isPresent()) { LogSegment segment = segmentOpt.get(); long baseOffset = segment.getBaseOffset(); @@ -406,9 +440,32 @@ public FetchDataInfo read( ? maxOffsetMetadata.getRelativePositionInSegment() : segment.getSizeInBytes(); fetchDataInfo = - segment.read(readOffset, maxLength, maxPosition, minOneMessage, projection); + segment.read( + readOffset, + maxLength, + maxPosition, + minOneMessage, + projection, + recordBatchFilter, + readContext, + predicateResolver); if (fetchDataInfo == null) { segmentOpt = segments.higherSegment(baseOffset); + } else if (recordBatchFilter != null + && fetchDataInfo.hasFilteredEndOffset() + && fetchDataInfo.getRecords().sizeInBytes() == 0) { + // All batches in this segment were filtered out. Record the skip offset + // and continue scanning the next segment for matching data. + lastFilteredSkipOffset = fetchDataInfo.getFilteredEndOffset(); + readOffset = lastFilteredSkipOffset; + fetchDataInfo = null; + filteredSegmentsScanned++; + if (filteredSegmentsScanned >= MAX_FILTER_SCAN_SEGMENTS) { + // Stop scanning to bound the I/O cost per fetch request. + // The client will retry from lastFilteredSkipOffset. + break; + } + segmentOpt = segments.higherSegment(baseOffset); } } if (fetchDataInfo != null) { @@ -418,7 +475,18 @@ public FetchDataInfo read( // start offset is in range, this can happen when all messages with offset larger // than start offsets have been deleted. In this case, we will return the empty set // with log end offset metadata - return new FetchDataInfo(nextOffsetMetadata, MemoryLogRecords.EMPTY); + if (lastFilteredSkipOffset >= 0) { + // All segments were scanned but every batch was filtered out. + // Return a filtered empty response with the final skip offset so the client + // advances past all the filtered data. + return FetchDataInfo.createFilteredEmptyResponse( + nextOffsetMetadata, lastFilteredSkipOffset); + } else if (recordBatchFilter != null) { + return FetchDataInfo.createFilteredEmptyResponse( + nextOffsetMetadata, nextOffsetMetadata.getMessageOffset()); + } else { + return new FetchDataInfo(nextOffsetMetadata, MemoryLogRecords.EMPTY); + } } } } diff --git a/fluss-server/src/main/java/org/apache/fluss/server/log/LogSegment.java b/fluss-server/src/main/java/org/apache/fluss/server/log/LogSegment.java index c45c25e5c9..c299a73412 100644 --- a/fluss-server/src/main/java/org/apache/fluss/server/log/LogSegment.java +++ b/fluss-server/src/main/java/org/apache/fluss/server/log/LogSegment.java @@ -23,16 +23,24 @@ import org.apache.fluss.exception.CorruptRecordException; import org.apache.fluss.exception.InvalidColumnProjectionException; import org.apache.fluss.exception.InvalidRecordException; +import org.apache.fluss.exception.LogOffsetOutOfRangeException; import org.apache.fluss.exception.LogSegmentOffsetOverflowException; import org.apache.fluss.metadata.LogFormat; +import org.apache.fluss.predicate.Predicate; +import org.apache.fluss.record.BytesViewLogRecords; import org.apache.fluss.record.FileChannelChunk; +import org.apache.fluss.record.FileLogInputStream.FileChannelLogRecordBatch; import org.apache.fluss.record.FileLogProjection; import org.apache.fluss.record.FileLogRecords; import org.apache.fluss.record.LogRecordBatch; +import org.apache.fluss.record.LogRecordBatchStatistics; import org.apache.fluss.record.LogRecords; import org.apache.fluss.record.MemoryLogRecords; import org.apache.fluss.record.TimestampAndOffset; +import org.apache.fluss.record.bytesview.BytesView; +import org.apache.fluss.record.bytesview.MultiBytesView; import org.apache.fluss.shaded.guava32.com.google.common.collect.Iterables; +import org.apache.fluss.utils.AbstractIterator; import org.apache.fluss.utils.FileUtils; import org.apache.fluss.utils.FlussPaths; @@ -406,7 +414,10 @@ public long readNextOffset() throws IOException { offsetIndex().lastOffset(), fileLogRecords.sizeInBytes(), fileLogRecords.sizeInBytes(), - false); + false, + null, + null, + null); if (fetchData == null) { return baseOffset; } else { @@ -476,7 +487,7 @@ private FileLogRecords.LogOffsetPosition translateOffset(long offset, int starti public FetchDataInfo read( long startOffset, int maxSize, long maxPosition, boolean minOneMessage) throws IOException { - return read(startOffset, maxSize, maxPosition, minOneMessage, null); + return read(startOffset, maxSize, maxPosition, minOneMessage, null, null, null); } /** @@ -490,11 +501,104 @@ public FetchDataInfo read( * @param minOneMessage If this is true, the first message will be returned even if it exceeds * `maxSize` (if one exists) * @param projection The column projection to apply to the log records - * @return The fetched data and the offset metadata of the first message whose offset is >= - * startOffset, or null if the startOffset is larger than the largest offset in this log + * @param recordBatchFilter The filter to apply to the log records (must be null if readContext + * is null) + * @param readContext The read context for batch filtering (must be null if recordBatchFilter is + * null) + * @throws LogOffsetOutOfRangeException If startOffset is beyond the log start and end offset + * @return The fetch data information including fetch starting offset metadata and messages + * read. */ - @Nullable public FetchDataInfo read( + long startOffset, + int maxSize, + long maxPosition, + boolean minOneMessage, + @Nullable FileLogProjection projection, + @Nullable Predicate recordBatchFilter, + @Nullable LogRecordBatch.ReadContext readContext) + throws IOException { + return read( + startOffset, + maxSize, + maxPosition, + minOneMessage, + projection, + recordBatchFilter, + readContext, + null); + } + + /** + * Read a message set from this segment beginning with the first offset >= startOffset. The + * message set will include no more than maxSize bytes and will end before maxOffset if a + * maxOffset is specified. + * + * @param startOffset A lower bound on the first offset to include in the message set we read + * @param maxSize The maximum number of bytes to include in the message set we read + * @param maxPosition The maximum position in the log segment that should be exposed for read + * @param minOneMessage If this is true, the first message will be returned even if it exceeds + * `maxSize` (if one exists) + * @param projection The column projection to apply to the log records + * @param recordBatchFilter The filter to apply to the log records (must be null if readContext + * is null) + * @param readContext The read context for batch statistics retrieval (must be null if + * recordBatchFilter is null) + * @param predicateResolver Resolves the effective predicate for a given batch schema ID, + * handling schema evolution transparently. If null when filter is active, the original + * predicate is used for all batches. + * @throws LogOffsetOutOfRangeException If startOffset is beyond the log start and end offset + * @return The fetch data information including fetch starting offset metadata and messages + * read. + */ + public FetchDataInfo read( + long startOffset, + int maxSize, + long maxPosition, + boolean minOneMessage, + @Nullable FileLogProjection projection, + @Nullable Predicate recordBatchFilter, + @Nullable LogRecordBatch.ReadContext readContext, + @Nullable PredicateSchemaResolver predicateResolver) + throws IOException { + // Validate that recordBatchFilter and readContext are either both null or both non-null + if ((recordBatchFilter == null) != (readContext == null)) { + throw new IllegalArgumentException( + "recordBatchFilter and readContext must be either both null or both non-null"); + } + + if (recordBatchFilter != null) { + return readWithFilter( + startOffset, + maxSize, + maxPosition, + minOneMessage, + projection, + recordBatchFilter, + readContext, + predicateResolver); + } else { + return readWithoutFilter(startOffset, maxSize, maxPosition, minOneMessage, projection); + } + } + + public void changeFileSuffixes(String oldSuffix, String newSuffix) throws IOException { + fileLogRecords.renameTo( + new File( + FileUtils.replaceSuffix( + fileLogRecords.file().getPath(), oldSuffix, newSuffix))); + lazyOffsetIndex.renameTo( + new File( + FileUtils.replaceSuffix( + lazyOffsetIndex.file().getPath(), oldSuffix, newSuffix))); + lazyTimeIndex.renameTo( + new File( + FileUtils.replaceSuffix( + lazyTimeIndex.file().getPath(), oldSuffix, newSuffix))); + } + + @Nullable + private FetchDataInfo readWithoutFilter( long startOffset, int maxSize, long maxPosition, @@ -505,6 +609,7 @@ public FetchDataInfo read( throw new IllegalArgumentException( "Invalid max size " + maxSize + " for log read from segment " + fileLogRecords); } + FileLogRecords.LogOffsetPosition startOffsetAndSize = translateOffset(startOffset, 0); if (startOffsetAndSize == null) { return null; @@ -540,19 +645,140 @@ public FetchDataInfo read( } } - public void changeFileSuffixes(String oldSuffix, String newSuffix) throws IOException { - fileLogRecords.renameTo( - new File( - FileUtils.replaceSuffix( - fileLogRecords.file().getPath(), oldSuffix, newSuffix))); - lazyOffsetIndex.renameTo( - new File( - FileUtils.replaceSuffix( - lazyOffsetIndex.file().getPath(), oldSuffix, newSuffix))); - lazyTimeIndex.renameTo( - new File( - FileUtils.replaceSuffix( - lazyTimeIndex.file().getPath(), oldSuffix, newSuffix))); + // NOTE: Server-side filter pushdown currently only works for ARROW format log tables. + // INDEXED and COMPACTED formats use V0 batch magic which does not include batch-level + // statistics (min/max/nullCount). Without statistics, the predicate cannot be evaluated + // against a batch, so all batches are included as a safe fallback. + @Nullable + private FetchDataInfo readWithFilter( + long startOffset, + int maxSize, + long maxPosition, + boolean minOneMessage, + @Nullable FileLogProjection projection, + Predicate recordBatchFilter, + LogRecordBatch.ReadContext readContext, + @Nullable PredicateSchemaResolver predicateResolver) + throws IOException { + + if (maxSize < 0) { + throw new IllegalArgumentException( + "Invalid max size " + maxSize + " for log read from segment " + fileLogRecords); + } + + // Use translateOffset to precisely locate the starting position, same as readWithoutFilter + FileLogRecords.LogOffsetPosition startOffsetAndSize = translateOffset(startOffset, 0); + if (startOffsetAndSize == null) { + return null; + } + int startPosition = startOffsetAndSize.getPosition(); + + // Iterate batches from the translated position. + // Note: AbstractIterator doesn't implement AutoCloseable, so no explicit cleanup needed. + // The iterator's internal state will be garbage collected when this method returns. + AbstractIterator iter = + fileLogRecords.batchIterator(startPosition, (int) maxPosition); + + MultiBytesView.Builder builder = MultiBytesView.builder(); + int accumulatedSize = 0; + FileChannelLogRecordBatch firstIncludedBatch = null; + FileChannelLogRecordBatch lastIncludedBatch = null; + FileChannelLogRecordBatch lastScannedBatch = null; + int adjustedMaxSize = maxSize; + + while (iter.hasNext()) { + FileChannelLogRecordBatch batch = iter.next(); + + if (batch.position() > maxPosition) { + break; + } + lastScannedBatch = batch; + + // Apply filter using statistics. On any failure, fall back to including + // the batch so a single corrupt batch cannot break the entire fetch. + boolean include = true; + try { + Optional statsOpt = batch.getStatistics(readContext); + if (statsOpt.isPresent()) { + LogRecordBatchStatistics stats = statsOpt.get(); + Predicate effectivePredicate = + predicateResolver != null + ? predicateResolver.resolve(stats.getSchemaId()) + : recordBatchFilter; + if (effectivePredicate != null) { + include = + effectivePredicate.test( + batch.getRecordCount(), + stats.getMinValues(), + stats.getMaxValues(), + stats.getNullCounts()); + } + // If effectivePredicate is null, cannot adapt -> include batch (safe + // fallback) + } + } catch (Exception e) { + LOG.warn( + "Failed to evaluate filter for batch at offset {} in segment {}, " + + "including batch as safe fallback.", + batch.baseLogOffset(), + fileLogRecords, + e); + include = true; + } + + if (!include) { + continue; + } + + int batchSize = batch.sizeInBytes(); + + if (projection == null) { + // No projection: use original batch size for limit check + if (firstIncludedBatch == null) { + firstIncludedBatch = batch; + adjustedMaxSize = minOneMessage ? Math.max(maxSize, batchSize) : maxSize; + } else if (accumulatedSize + batchSize > adjustedMaxSize) { + break; + } + lastIncludedBatch = batch; + builder.addBytes(fileLogRecords.channel(), batch.position(), batchSize); + accumulatedSize += batchSize; + } else { + // With projection: project first, then check size with projected size + BytesView projectedBytesView = projection.projectRecordBatch(batch); + int projectedSize = projectedBytesView.getBytesLength(); + if (projectedSize > 0) { + if (firstIncludedBatch == null) { + firstIncludedBatch = batch; + adjustedMaxSize = + minOneMessage ? Math.max(maxSize, projectedSize) : maxSize; + } else if (accumulatedSize + projectedSize > adjustedMaxSize) { + break; + } + lastIncludedBatch = batch; + builder.addBytes(projectedBytesView); + accumulatedSize += projectedSize; + } + } + } + + if (firstIncludedBatch == null) { + // All batches were filtered out in this segment. Return a filtered empty response + // with filteredEndOffset so the caller can advance past this segment's filtered + // range and decide whether to continue scanning subsequent segments. + if (lastScannedBatch != null) { + LogOffsetMetadata offsetMetadata = + new LogOffsetMetadata(startOffset, this.baseOffset, startPosition); + return FetchDataInfo.createFilteredEmptyResponse( + offsetMetadata, lastScannedBatch.nextLogOffset()); + } + return null; + } + + LogOffsetMetadata offsetMetadata = + new LogOffsetMetadata(startOffset, this.baseOffset, startPosition); + + return new FetchDataInfo(offsetMetadata, new BytesViewLogRecords(builder.build())); } private void ensureOffsetInRange(long offset) throws IOException { diff --git a/fluss-server/src/main/java/org/apache/fluss/server/log/LogTablet.java b/fluss-server/src/main/java/org/apache/fluss/server/log/LogTablet.java index a1fa20d5a5..b5968ca59f 100644 --- a/fluss-server/src/main/java/org/apache/fluss/server/log/LogTablet.java +++ b/fluss-server/src/main/java/org/apache/fluss/server/log/LogTablet.java @@ -32,6 +32,7 @@ import org.apache.fluss.metadata.TablePath; import org.apache.fluss.metrics.MetricNames; import org.apache.fluss.metrics.groups.MetricGroup; +import org.apache.fluss.predicate.Predicate; import org.apache.fluss.record.DefaultLogRecordBatch; import org.apache.fluss.record.FileLogProjection; import org.apache.fluss.record.FileLogRecords; @@ -395,13 +396,15 @@ public LogAppendInfo appendAsFollower(MemoryLogRecords records) throws Exception return append(records, false); } - /** Read messages from the local log. */ public FetchDataInfo read( long readOffset, int maxLength, FetchIsolation fetchIsolation, boolean minOneMessage, - @Nullable FileLogProjection projection) + @Nullable FileLogProjection projection, + @Nullable Predicate recordBatchFilter, + @Nullable LogRecordBatch.ReadContext readContext, + @Nullable PredicateSchemaResolver predicateResolver) throws IOException { LogOffsetMetadata maxOffsetMetadata = null; if (fetchIsolation == FetchIsolation.LOG_END) { @@ -410,7 +413,15 @@ public FetchDataInfo read( maxOffsetMetadata = fetchHighWatermarkMetadata(); } - return localLog.read(readOffset, maxLength, minOneMessage, maxOffsetMetadata, projection); + return localLog.read( + readOffset, + maxLength, + minOneMessage, + maxOffsetMetadata, + projection, + recordBatchFilter, + readContext, + predicateResolver); } /** diff --git a/fluss-server/src/main/java/org/apache/fluss/server/log/PredicateSchemaResolver.java b/fluss-server/src/main/java/org/apache/fluss/server/log/PredicateSchemaResolver.java new file mode 100644 index 0000000000..3e9425d870 --- /dev/null +++ b/fluss-server/src/main/java/org/apache/fluss/server/log/PredicateSchemaResolver.java @@ -0,0 +1,136 @@ +/* + * 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.server.log; + +import org.apache.fluss.metadata.Schema; +import org.apache.fluss.metadata.SchemaGetter; +import org.apache.fluss.predicate.Predicate; +import org.apache.fluss.predicate.PredicateBuilder; +import org.apache.fluss.utils.SchemaUtil; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import javax.annotation.Nullable; + +import java.util.HashMap; +import java.util.Map; +import java.util.Optional; + +/** + * Resolves a filter predicate for a given batch schema ID, handling schema evolution transparently. + * + *

When the batch schema matches the predicate schema, the original predicate is returned + * directly. When they differ, the predicate is adapted using field index mapping and the result is + * cached to avoid redundant adaptation across batches and segments. + * + *

Adapted predicates are cached in a simple HashMap since the resolver is created per-request + * and does not require thread-safety or eviction policies. + */ +public final class PredicateSchemaResolver { + + private static final Logger LOG = LoggerFactory.getLogger(PredicateSchemaResolver.class); + + private final Predicate predicate; + private final int predicateSchemaId; + @Nullable private final Schema predicateSchema; + @Nullable private final SchemaGetter schemaGetter; + + private final Map> cache = new HashMap<>(); + + public PredicateSchemaResolver( + Predicate predicate, int predicateSchemaId, @Nullable SchemaGetter schemaGetter) { + this.predicate = predicate; + this.predicateSchemaId = predicateSchemaId; + this.schemaGetter = schemaGetter; + + // Pre-resolve predicate schema once + Schema resolved = null; + if (schemaGetter != null && predicateSchemaId >= 0) { + try { + resolved = schemaGetter.getSchema(predicateSchemaId); + } catch (Exception e) { + LOG.warn( + "Failed to get predicate schema (schemaId={}), " + + "server-side filter will be disabled for cross-schema batches.", + predicateSchemaId, + e); + } + } + this.predicateSchema = resolved; + } + + /** + * Resolve the effective predicate for a batch with the given schema ID. + * + * @return the adapted predicate, or {@code null} if adaptation is not possible (safe fallback: + * include the batch) + */ + @Nullable + public Predicate resolve(int batchSchemaId) { + // Fast path: same schema + if (predicateSchemaId == batchSchemaId || predicateSchemaId < 0) { + return predicate; + } + + // Check cache + Optional cached = cache.get(batchSchemaId); + if (cached != null) { + return cached.orElse(null); + } + + // No schema getter or predicate schema, cannot adapt + if (schemaGetter == null || predicateSchema == null) { + LOG.warn( + "Cannot adapt predicate for batch schemaId={}: " + + "schema getter or predicate schema unavailable, " + + "skipping filter for this batch.", + batchSchemaId); + cache.put(batchSchemaId, Optional.empty()); + return null; + } + + try { + Schema batchSchema = schemaGetter.getSchema(batchSchemaId); + if (batchSchema == null) { + LOG.warn( + "Batch schema not found (schemaId={}), " + + "skipping filter for this batch.", + batchSchemaId); + cache.put(batchSchemaId, Optional.empty()); + return null; + } + + // indexMapping[predicateIdx] = batchIdx + int[] indexMapping = SchemaUtil.getIndexMapping(batchSchema, predicateSchema); + Optional adapted = + PredicateBuilder.transformFieldMapping(predicate, indexMapping); + cache.put(batchSchemaId, adapted); + return adapted.orElse(null); + } catch (Exception e) { + LOG.warn( + "Failed to adapt predicate for batch schemaId={} " + + "(predicate schemaId={}), skipping filter for this batch.", + batchSchemaId, + predicateSchemaId, + e); + cache.put(batchSchemaId, Optional.empty()); + return null; + } + } +} diff --git a/fluss-server/src/main/java/org/apache/fluss/server/replica/Replica.java b/fluss-server/src/main/java/org/apache/fluss/server/replica/Replica.java index dd83c58b1d..fc06e1ecdb 100644 --- a/fluss-server/src/main/java/org/apache/fluss/server/replica/Replica.java +++ b/fluss-server/src/main/java/org/apache/fluss/server/replica/Replica.java @@ -34,6 +34,7 @@ import org.apache.fluss.metadata.ChangelogImage; import org.apache.fluss.metadata.LogFormat; import org.apache.fluss.metadata.PhysicalTablePath; +import org.apache.fluss.metadata.Schema; import org.apache.fluss.metadata.SchemaGetter; import org.apache.fluss.metadata.TableBucket; import org.apache.fluss.metadata.TableInfo; @@ -41,12 +42,15 @@ import org.apache.fluss.metrics.Counter; import org.apache.fluss.metrics.MetricNames; import org.apache.fluss.metrics.groups.MetricGroup; +import org.apache.fluss.predicate.Predicate; import org.apache.fluss.record.DefaultValueRecordBatch; import org.apache.fluss.record.KvRecordBatch; +import org.apache.fluss.record.LogRecordReadContext; import org.apache.fluss.record.LogRecords; import org.apache.fluss.record.MemoryLogRecords; import org.apache.fluss.rpc.protocol.Errors; import org.apache.fluss.rpc.protocol.MergeMode; +import org.apache.fluss.rpc.util.PredicateMessageUtils; import org.apache.fluss.server.SequenceIDCounter; import org.apache.fluss.server.coordinator.CoordinatorContext; import org.apache.fluss.server.entity.NotifyLeaderAndIsrData; @@ -67,6 +71,7 @@ import org.apache.fluss.server.log.FetchDataInfo; import org.apache.fluss.server.log.FetchIsolation; import org.apache.fluss.server.log.FetchParams; +import org.apache.fluss.server.log.FilterInfo; import org.apache.fluss.server.log.ListOffsetsParam; import org.apache.fluss.server.log.LogAppendInfo; import org.apache.fluss.server.log.LogManager; @@ -74,6 +79,7 @@ import org.apache.fluss.server.log.LogOffsetSnapshot; import org.apache.fluss.server.log.LogReadInfo; import org.apache.fluss.server.log.LogTablet; +import org.apache.fluss.server.log.PredicateSchemaResolver; import org.apache.fluss.server.log.checkpoint.OffsetCheckpointFile; import org.apache.fluss.server.log.remote.RemoteLogManager; import org.apache.fluss.server.metadata.ServerMetadataCache; @@ -90,6 +96,7 @@ import org.apache.fluss.server.zk.ZooKeeperClient; import org.apache.fluss.server.zk.data.LeaderAndIsr; import org.apache.fluss.server.zk.data.ZkData; +import org.apache.fluss.types.RowType; import org.apache.fluss.utils.CloseableRegistry; import org.apache.fluss.utils.FlussPaths; import org.apache.fluss.utils.IOUtils; @@ -1335,6 +1342,9 @@ public LogRecords limitLogScan(int limit) { Integer.MAX_VALUE, FetchIsolation.HIGH_WATERMARK, true, + null, + null, + null, null); return dataInfo.getRecords(); } catch (IOException e) { @@ -1494,13 +1504,71 @@ private LogReadInfo readRecords(FetchParams fetchParams, LogTablet logTablet) // todo validate fetched epoch. - FetchDataInfo fetchDataInfo = - logTablet.read( - readOffset, - fetchParams.maxFetchBytes(), - fetchParams.isolation(), - fetchParams.minOneMessage(), - fetchParams.projection()); + // Create ReadContext for batch filtering if needed. + // Only ARROW format has batch-level statistics (V1+ magic) for filter evaluation. + // INDEXED and COMPACTED formats use V0 magic without statistics, so filter pushdown + // would be a no-op — skip it entirely to avoid unnecessary overhead. + Predicate resolvedFilter = null; + LogRecordReadContext readContext = null; + PredicateSchemaResolver predicateResolver = null; + FilterInfo filterInfo = fetchParams.getFilterInfo(tableBucket.getTableId()); + if (filterInfo != null && logFormat == LogFormat.ARROW) { + try { + int filterSchemaId = filterInfo.getSchemaId(); + RowType rowType; + int schemaIdForContext; + if (filterSchemaId >= 0) { + Schema filterSchema = schemaGetter.getSchema(filterSchemaId); + rowType = filterSchema.getRowType(); + schemaIdForContext = filterSchemaId; + } else { + rowType = tableInfo.getSchema().getRowType(); + schemaIdForContext = tableInfo.getSchemaId(); + } + resolvedFilter = + PredicateMessageUtils.toPredicate(filterInfo.getPbPredicate(), rowType); + if (resolvedFilter != null) { + readContext = + LogRecordReadContext.createArrowReadContext( + rowType, schemaIdForContext, schemaGetter); + predicateResolver = + new PredicateSchemaResolver( + resolvedFilter, schemaIdForContext, schemaGetter); + } + } catch (Exception e) { + LOG.warn( + "Failed to initialize filter context for {}, " + + "falling back to unfiltered read.", + tableBucket, + e); + // Safe fallback: read without filter. resolvedFilter/readContext/predicateResolver + // remain null, so the read proceeds as if no filter was requested. + } + } + + FetchDataInfo fetchDataInfo; + try { + fetchDataInfo = + logTablet.read( + readOffset, + fetchParams.maxFetchBytes(), + fetchParams.isolation(), + fetchParams.minOneMessage(), + fetchParams.projection(), + resolvedFilter, + readContext, + predicateResolver); + } finally { + // Close readContext eagerly — it is only used for statistics extraction during + // batch filtering and is NOT referenced by the returned FetchDataInfo records. + if (readContext != null) { + try { + readContext.close(); + } catch (Exception e) { + // ignore close exception + } + } + } return new LogReadInfo(fetchDataInfo, initialHighWatermark, initialLogEndOffset); } diff --git a/fluss-server/src/main/java/org/apache/fluss/server/replica/ReplicaManager.java b/fluss-server/src/main/java/org/apache/fluss/server/replica/ReplicaManager.java index c57f1c6fb7..b568600eff 100644 --- a/fluss-server/src/main/java/org/apache/fluss/server/replica/ReplicaManager.java +++ b/fluss-server/src/main/java/org/apache/fluss/server/replica/ReplicaManager.java @@ -1412,13 +1412,21 @@ public Map readFromLog( fetchParams.markReadOneMessage(); } limitBytes = Math.max(0, limitBytes - recordBatchSize); - + FetchLogResultForBucket fetchLogResult; + if (fetchedData.hasFilteredEndOffset()) { + fetchLogResult = + new FetchLogResultForBucket( + tb, + readInfo.getHighWatermark(), + fetchedData.getFilteredEndOffset()); + } else { + fetchLogResult = + new FetchLogResultForBucket( + tb, fetchedData.getRecords(), readInfo.getHighWatermark()); + } logReadResult.put( tb, - new LogReadResult( - new FetchLogResultForBucket( - tb, fetchedData.getRecords(), readInfo.getHighWatermark()), - fetchedData.getFetchOffsetMetadata())); + new LogReadResult(fetchLogResult, fetchedData.getFetchOffsetMetadata())); // update metrics if (isFromFollower) { diff --git a/fluss-server/src/main/java/org/apache/fluss/server/tablet/TabletService.java b/fluss-server/src/main/java/org/apache/fluss/server/tablet/TabletService.java index b511f21785..548ef29299 100644 --- a/fluss-server/src/main/java/org/apache/fluss/server/tablet/TabletService.java +++ b/fluss-server/src/main/java/org/apache/fluss/server/tablet/TabletService.java @@ -75,6 +75,8 @@ import org.apache.fluss.server.entity.NotifyLeaderAndIsrData; import org.apache.fluss.server.entity.UserContext; import org.apache.fluss.server.log.FetchParams; +import org.apache.fluss.server.log.FetchParamsBuilder; +import org.apache.fluss.server.log.FilterInfo; import org.apache.fluss.server.log.ListOffsetsParam; import org.apache.fluss.server.metadata.TabletServerMetadataCache; import org.apache.fluss.server.metadata.TabletServerMetadataProvider; @@ -108,6 +110,7 @@ import static org.apache.fluss.server.utils.ServerRpcMessageUtils.getProduceLogData; import static org.apache.fluss.server.utils.ServerRpcMessageUtils.getPutKvData; import static org.apache.fluss.server.utils.ServerRpcMessageUtils.getStopReplicaData; +import static org.apache.fluss.server.utils.ServerRpcMessageUtils.getTableFilterInfoMap; import static org.apache.fluss.server.utils.ServerRpcMessageUtils.getTableStatsRequestData; import static org.apache.fluss.server.utils.ServerRpcMessageUtils.getTargetColumns; import static org.apache.fluss.server.utils.ServerRpcMessageUtils.getUpdateMetadataRequestData; @@ -209,18 +212,24 @@ public CompletableFuture fetchLog(FetchLogRequest request) { private static FetchParams getFetchParams(FetchLogRequest request) { FetchParams fetchParams; + Map tableFilterInfoMap = getTableFilterInfoMap(request); if (request.hasMinBytes()) { fetchParams = - new FetchParams( - request.getFollowerServerId(), - request.getMaxBytes(), - request.getMinBytes(), - request.hasMaxWaitMs() - ? request.getMaxWaitMs() - : DEFAULT_MAX_WAIT_MS_WHEN_MIN_BYTES_ENABLE); + new FetchParamsBuilder(request.getFollowerServerId(), request.getMaxBytes()) + .withMinFetchBytes(request.getMinBytes()) + .withMaxWaitMs( + request.hasMaxWaitMs() + ? request.getMaxWaitMs() + : DEFAULT_MAX_WAIT_MS_WHEN_MIN_BYTES_ENABLE) + .withTableFilterInfoMap(tableFilterInfoMap) + .build(); } else { - fetchParams = new FetchParams(request.getFollowerServerId(), request.getMaxBytes()); + fetchParams = + new FetchParamsBuilder(request.getFollowerServerId(), request.getMaxBytes()) + .withTableFilterInfoMap(tableFilterInfoMap) + .build(); } + return fetchParams; } diff --git a/fluss-server/src/main/java/org/apache/fluss/server/utils/ServerRpcMessageUtils.java b/fluss-server/src/main/java/org/apache/fluss/server/utils/ServerRpcMessageUtils.java index df5f8034cb..bbd48016df 100644 --- a/fluss-server/src/main/java/org/apache/fluss/server/utils/ServerRpcMessageUtils.java +++ b/fluss-server/src/main/java/org/apache/fluss/server/utils/ServerRpcMessageUtils.java @@ -186,6 +186,7 @@ import org.apache.fluss.server.kv.snapshot.CompletedSnapshot; import org.apache.fluss.server.kv.snapshot.CompletedSnapshotJsonSerde; import org.apache.fluss.server.kv.snapshot.KvSnapshotHandle; +import org.apache.fluss.server.log.FilterInfo; import org.apache.fluss.server.metadata.BucketMetadata; import org.apache.fluss.server.metadata.ClusterMetadata; import org.apache.fluss.server.metadata.PartitionMetadata; @@ -221,6 +222,7 @@ import static org.apache.fluss.rpc.util.CommonRpcMessageUtils.toByteBuffer; import static org.apache.fluss.rpc.util.CommonRpcMessageUtils.toPbAclInfo; import static org.apache.fluss.utils.Preconditions.checkNotNull; +import static org.apache.fluss.utils.Preconditions.checkState; /** * Utils for making rpc request/response from inner object or convert inner class to rpc @@ -887,6 +889,22 @@ public static ProduceLogResponse makeProduceLogResponse( return produceResponse; } + public static @Nullable Map getTableFilterInfoMap(FetchLogRequest request) { + Map result = null; + for (PbFetchLogReqForTable tableReq : request.getTablesReqsList()) { + if (tableReq.hasFilterPredicate()) { + if (result == null) { + result = new HashMap<>(); + } + int schemaId = tableReq.hasFilterSchemaId() ? tableReq.getFilterSchemaId() : -1; + result.put( + tableReq.getTableId(), + new FilterInfo(tableReq.getFilterPredicate(), schemaId)); + } + } + return result; + } + public static Map getFetchLogData(FetchLogRequest request) { Map fetchDataMap = new HashMap<>(); for (PbFetchLogReqForTable fetchLogReqForTable : request.getTablesReqsList()) { @@ -933,6 +951,15 @@ public static FetchLogResponse makeFetchLogResponse( FetchLogResultForBucket bucketResult = entry.getValue(); PbFetchLogRespForBucket fetchLogRespForBucket = new PbFetchLogRespForBucket().setBucketId(tb.getBucket()); + if (bucketResult.hasFilteredEndOffset()) { + fetchLogRespForBucket.setFilteredEndOffset(bucketResult.getFilteredEndOffset()); + // filteredEndOffset and records are mutually exclusive: when all batches are + // filtered out, there should be no record data to send. + checkState( + bucketResult.recordsOrEmpty().sizeInBytes() == 0, + "filteredEndOffset is set but records are not empty for bucket %s", + tb); + } if (tb.getPartitionId() != null) { fetchLogRespForBucket.setPartitionId(tb.getPartitionId()); } diff --git a/fluss-server/src/test/java/org/apache/fluss/server/kv/KvTabletMergeModeTest.java b/fluss-server/src/test/java/org/apache/fluss/server/kv/KvTabletMergeModeTest.java index ce18423090..17f51c62a5 100644 --- a/fluss-server/src/test/java/org/apache/fluss/server/kv/KvTabletMergeModeTest.java +++ b/fluss-server/src/test/java/org/apache/fluss/server/kv/KvTabletMergeModeTest.java @@ -535,7 +535,15 @@ void testDefaultMergeModeIsDefault() throws Exception { private LogRecords readLogRecords(long startOffset) throws Exception { return logTablet - .read(startOffset, Integer.MAX_VALUE, FetchIsolation.LOG_END, false, null) + .read( + startOffset, + Integer.MAX_VALUE, + FetchIsolation.LOG_END, + false, + null, + null, + null, + null) .getRecords(); } diff --git a/fluss-server/src/test/java/org/apache/fluss/server/kv/KvTabletTest.java b/fluss-server/src/test/java/org/apache/fluss/server/kv/KvTabletTest.java index da918f45e8..6e18c1cf6e 100644 --- a/fluss-server/src/test/java/org/apache/fluss/server/kv/KvTabletTest.java +++ b/fluss-server/src/test/java/org/apache/fluss/server/kv/KvTabletTest.java @@ -1393,7 +1393,15 @@ private LogRecords readLogRecords( LogTablet logTablet, long startOffset, @Nullable FileLogProjection projection) throws Exception { return logTablet - .read(startOffset, Integer.MAX_VALUE, FetchIsolation.LOG_END, false, projection) + .read( + startOffset, + Integer.MAX_VALUE, + FetchIsolation.LOG_END, + false, + projection, + null, + null, + null) .getRecords(); } diff --git a/fluss-server/src/test/java/org/apache/fluss/server/log/FetchParamsTest.java b/fluss-server/src/test/java/org/apache/fluss/server/log/FetchParamsTest.java index d4c2be57b8..da3ea953d2 100644 --- a/fluss-server/src/test/java/org/apache/fluss/server/log/FetchParamsTest.java +++ b/fluss-server/src/test/java/org/apache/fluss/server/log/FetchParamsTest.java @@ -82,4 +82,14 @@ void testSetCurrentFetch() { // the FileLogProjection should be cached assertThat(fetchParams.projection()).isNotNull().isSameAs(prevProjection); } + + @Test + void testBuilderDefaultsToLeaderOnly() { + FetchParams fetchParams = new FetchParamsBuilder(-1, 100).build(); + assertThat(fetchParams.fetchOnlyLeader()).isTrue(); + + FetchParams overrideFetchParams = + new FetchParamsBuilder(-1, 100).withFetchOnlyLeader(false).build(); + assertThat(overrideFetchParams.fetchOnlyLeader()).isFalse(); + } } diff --git a/fluss-server/src/test/java/org/apache/fluss/server/log/LocalLogTest.java b/fluss-server/src/test/java/org/apache/fluss/server/log/LocalLogTest.java index ad3d0c6caf..e85abad620 100644 --- a/fluss-server/src/test/java/org/apache/fluss/server/log/LocalLogTest.java +++ b/fluss-server/src/test/java/org/apache/fluss/server/log/LocalLogTest.java @@ -21,10 +21,17 @@ import org.apache.fluss.exception.FlussRuntimeException; import org.apache.fluss.metadata.LogFormat; import org.apache.fluss.metadata.TableBucket; +import org.apache.fluss.predicate.Predicate; +import org.apache.fluss.predicate.PredicateBuilder; +import org.apache.fluss.record.LogRecord; +import org.apache.fluss.record.LogRecordBatch; +import org.apache.fluss.record.LogRecordBatchStatisticsTestUtils; +import org.apache.fluss.record.LogRecordReadContext; import org.apache.fluss.record.LogTestBase; import org.apache.fluss.record.MemoryLogRecords; import org.apache.fluss.server.log.LocalLog.SegmentDeletionReason; import org.apache.fluss.server.metrics.group.TestingMetricGroups; +import org.apache.fluss.utils.CloseableIterator; import org.junit.jupiter.api.AfterEach; import org.junit.jupiter.api.BeforeEach; @@ -40,6 +47,9 @@ import java.util.Optional; import java.util.stream.Collectors; +import static org.apache.fluss.record.TestData.DATA1_ROW_TYPE; +import static org.apache.fluss.record.TestData.DEFAULT_SCHEMA_ID; +import static org.apache.fluss.record.TestData.TEST_SCHEMA_GETTER; import static org.apache.fluss.testutils.DataTestUtils.assertLogRecordsEquals; import static org.apache.fluss.testutils.DataTestUtils.genMemoryLogRecordsWithBaseOffset; import static org.assertj.core.api.Assertions.assertThat; @@ -392,9 +402,344 @@ private List nonActiveBaseOffsetsFrom(long offset) { .collect(Collectors.toList()); } + @Test + void testReadWithFilterAcrossMultipleSegments() throws Exception { + // Segment 1 (offsets 0-2): values 1,2,3 — should be filtered out by "a > 5" + MemoryLogRecords seg1Records = + LogRecordBatchStatisticsTestUtils.createLogRecordsWithStatistics( + Arrays.asList( + new Object[] {1, "a"}, + new Object[] {2, "b"}, + new Object[] {3, "c"}), + DATA1_ROW_TYPE, + 0L, + DEFAULT_SCHEMA_ID); + localLog.append(2, -1L, 0L, seg1Records); + localLog.roll(Optional.empty()); + + // Segment 2 (offsets 3-5): values 4,5,3 — should be filtered out by "a > 5" + MemoryLogRecords seg2Records = + LogRecordBatchStatisticsTestUtils.createLogRecordsWithStatistics( + Arrays.asList( + new Object[] {4, "d"}, + new Object[] {5, "e"}, + new Object[] {3, "f"}), + DATA1_ROW_TYPE, + 3L, + DEFAULT_SCHEMA_ID); + localLog.append(5, -1L, 0L, seg2Records); + localLog.roll(Optional.empty()); + + // Segment 3 (offsets 6-8): values 7,8,9 — should PASS filter "a > 5" + MemoryLogRecords seg3Records = + LogRecordBatchStatisticsTestUtils.createLogRecordsWithStatistics( + Arrays.asList( + new Object[] {7, "g"}, + new Object[] {8, "h"}, + new Object[] {9, "i"}), + DATA1_ROW_TYPE, + 6L, + DEFAULT_SCHEMA_ID); + localLog.append(8, -1L, 0L, seg3Records); + + assertThat(localLog.getSegments().numberOfSegments()).isEqualTo(3); + + // Create filter: a > 5 + PredicateBuilder builder = new PredicateBuilder(DATA1_ROW_TYPE); + Predicate filter = builder.greaterThan(0, 5); + + try (LogRecordReadContext readContext = + LogRecordReadContext.createArrowReadContext( + DATA1_ROW_TYPE, DEFAULT_SCHEMA_ID, TEST_SCHEMA_GETTER)) { + FetchDataInfo result = + localLog.read( + 0, + Integer.MAX_VALUE, + false, + localLog.getLocalLogEndOffsetMetadata(), + null, + filter, + readContext, + null); + + assertThat(result).isNotNull(); + // Should have found data from segment 3, skipping segments 1 and 2 + assertThat(result.getRecords().sizeInBytes()).isGreaterThan(0); + + // Verify the returned records are from segment 3 (values > 5) + int recordCount = 0; + for (LogRecordBatch batch : result.getRecords().batches()) { + try (CloseableIterator iter = batch.records(readContext)) { + while (iter.hasNext()) { + LogRecord record = iter.next(); + assertThat(record.getRow().getInt(0)).isGreaterThan(5); + recordCount++; + } + } + } + assertThat(recordCount).isEqualTo(3); + } + } + + @Test + void testReadWithFilterAllSegmentsFilteredOut() throws Exception { + // Segment 1 (offsets 0-2): values 1,2,3 + MemoryLogRecords seg1Records = + LogRecordBatchStatisticsTestUtils.createLogRecordsWithStatistics( + Arrays.asList( + new Object[] {1, "a"}, + new Object[] {2, "b"}, + new Object[] {3, "c"}), + DATA1_ROW_TYPE, + 0L, + DEFAULT_SCHEMA_ID); + localLog.append(2, -1L, 0L, seg1Records); + localLog.roll(Optional.empty()); + + // Segment 2 (offsets 3-5): values 4,5,3 + MemoryLogRecords seg2Records = + LogRecordBatchStatisticsTestUtils.createLogRecordsWithStatistics( + Arrays.asList( + new Object[] {4, "d"}, + new Object[] {5, "e"}, + new Object[] {3, "f"}), + DATA1_ROW_TYPE, + 3L, + DEFAULT_SCHEMA_ID); + localLog.append(5, -1L, 0L, seg2Records); + + assertThat(localLog.getSegments().numberOfSegments()).isEqualTo(2); + + // Create filter: a > 100 — nothing matches + PredicateBuilder builder = new PredicateBuilder(DATA1_ROW_TYPE); + Predicate filter = builder.greaterThan(0, 100); + + try (LogRecordReadContext readContext = + LogRecordReadContext.createArrowReadContext( + DATA1_ROW_TYPE, DEFAULT_SCHEMA_ID, TEST_SCHEMA_GETTER)) { + FetchDataInfo result = + localLog.read( + 0, + Integer.MAX_VALUE, + false, + localLog.getLocalLogEndOffsetMetadata(), + null, + filter, + readContext, + null); + + assertThat(result).isNotNull(); + // All segments filtered out — should return empty records with filteredEndOffset + assertThat(result.getRecords().sizeInBytes()).isEqualTo(0); + assertThat(result.getFilteredEndOffset()).isEqualTo(6); + } + } + + @Test + void testReadWithFilterSingleBatchSegmentFilteredNextHasData() throws Exception { + // Segment 1: single batch with values [1,2,3] — will be filtered by a > 5 + List seg1Data = + Arrays.asList(new Object[] {1, "a"}, new Object[] {2, "b"}, new Object[] {3, "c"}); + MemoryLogRecords seg1Records = + LogRecordBatchStatisticsTestUtils.createLogRecordsWithStatistics( + seg1Data, DATA1_ROW_TYPE, 0L, DEFAULT_SCHEMA_ID); + localLog.append(2L, -1L, 0L, seg1Records); + localLog.roll(Optional.empty()); + + // Segment 2: single batch with values [6,7,8] — should pass filter + List seg2Data = + Arrays.asList(new Object[] {6, "d"}, new Object[] {7, "e"}, new Object[] {8, "f"}); + MemoryLogRecords seg2Records = + LogRecordBatchStatisticsTestUtils.createLogRecordsWithStatistics( + seg2Data, DATA1_ROW_TYPE, 3L, DEFAULT_SCHEMA_ID); + localLog.append(5L, -1L, 0L, seg2Records); + + PredicateBuilder builder = new PredicateBuilder(DATA1_ROW_TYPE); + Predicate filter = builder.greaterThan(0, 5); + + try (LogRecordReadContext readContext = + LogRecordReadContext.createArrowReadContext( + DATA1_ROW_TYPE, DEFAULT_SCHEMA_ID, TEST_SCHEMA_GETTER)) { + FetchDataInfo result = + localLog.read( + 0, + Integer.MAX_VALUE, + false, + localLog.getLocalLogEndOffsetMetadata(), + null, + filter, + readContext, + null); + + assertThat(result).isNotNull(); + assertThat(result.getRecords().sizeInBytes()).isGreaterThan(0); + assertThat(result.getFilteredEndOffset()).isEqualTo(-1L); + + int recordCount = 0; + for (LogRecordBatch batch : result.getRecords().batches()) { + try (CloseableIterator iter = batch.records(readContext)) { + while (iter.hasNext()) { + LogRecord record = iter.next(); + assertThat(record.getRow().getInt(0)).isGreaterThan(5); + recordCount++; + } + } + } + assertThat(recordCount).isEqualTo(3); + } + } + + @Test + void testReadWithFilteredEndOffsetAtSegmentBoundary() throws Exception { + // Segment 1: offsets 0-4, values [1,2,3,4,5] — all filtered by a > 10 + List seg1Data = + Arrays.asList( + new Object[] {1, "a"}, + new Object[] {2, "b"}, + new Object[] {3, "c"}, + new Object[] {4, "d"}, + new Object[] {5, "e"}); + MemoryLogRecords seg1Records = + LogRecordBatchStatisticsTestUtils.createLogRecordsWithStatistics( + seg1Data, DATA1_ROW_TYPE, 0L, DEFAULT_SCHEMA_ID); + localLog.append(4L, -1L, 0L, seg1Records); + localLog.roll(Optional.empty()); + + // Segment 2: offsets 5-9, values [6,7,8,9,10] — also filtered by a > 10 + List seg2Data = + Arrays.asList( + new Object[] {6, "f"}, + new Object[] {7, "g"}, + new Object[] {8, "h"}, + new Object[] {9, "i"}, + new Object[] {10, "j"}); + MemoryLogRecords seg2Records = + LogRecordBatchStatisticsTestUtils.createLogRecordsWithStatistics( + seg2Data, DATA1_ROW_TYPE, 5L, DEFAULT_SCHEMA_ID); + localLog.append(9L, -1L, 0L, seg2Records); + localLog.roll(Optional.empty()); + + // Segment 3: offsets 10-12, values [11,12,13] — passes filter + List seg3Data = + Arrays.asList( + new Object[] {11, "k"}, new Object[] {12, "l"}, new Object[] {13, "m"}); + MemoryLogRecords seg3Records = + LogRecordBatchStatisticsTestUtils.createLogRecordsWithStatistics( + seg3Data, DATA1_ROW_TYPE, 10L, DEFAULT_SCHEMA_ID); + localLog.append(12L, -1L, 0L, seg3Records); + + PredicateBuilder builder = new PredicateBuilder(DATA1_ROW_TYPE); + Predicate filter = builder.greaterThan(0, 10); + + try (LogRecordReadContext readContext = + LogRecordReadContext.createArrowReadContext( + DATA1_ROW_TYPE, DEFAULT_SCHEMA_ID, TEST_SCHEMA_GETTER)) { + FetchDataInfo result = + localLog.read( + 0, + Integer.MAX_VALUE, + false, + localLog.getLocalLogEndOffsetMetadata(), + null, + filter, + readContext, + null); + + assertThat(result).isNotNull(); + assertThat(result.getRecords().sizeInBytes()).isGreaterThan(0); + + // Verify returned records are from segment 3 only + int recordCount = 0; + for (LogRecordBatch batch : result.getRecords().batches()) { + try (CloseableIterator iter = batch.records(readContext)) { + while (iter.hasNext()) { + LogRecord record = iter.next(); + assertThat(record.getRow().getInt(0)).isGreaterThan(10); + recordCount++; + } + } + } + assertThat(recordCount).isEqualTo(3); + } + } + + @Test + void testReadWithFilterLastSegmentLastBatchFiltered() throws Exception { + // Segment 1: offsets 0-4, values [6,7,8,9,10] — passes filter a > 5 + List seg1Data = + Arrays.asList( + new Object[] {6, "a"}, + new Object[] {7, "b"}, + new Object[] {8, "c"}, + new Object[] {9, "d"}, + new Object[] {10, "e"}); + MemoryLogRecords seg1Records = + LogRecordBatchStatisticsTestUtils.createLogRecordsWithStatistics( + seg1Data, DATA1_ROW_TYPE, 0L, DEFAULT_SCHEMA_ID); + localLog.append(4L, -1L, 0L, seg1Records); + localLog.roll(Optional.empty()); + + // Segment 2: offsets 5-9, values [1,2,3,4,5] — filtered by a > 5 + List seg2Data = + Arrays.asList( + new Object[] {1, "f"}, + new Object[] {2, "g"}, + new Object[] {3, "h"}, + new Object[] {4, "i"}, + new Object[] {5, "j"}); + MemoryLogRecords seg2Records = + LogRecordBatchStatisticsTestUtils.createLogRecordsWithStatistics( + seg2Data, DATA1_ROW_TYPE, 5L, DEFAULT_SCHEMA_ID); + localLog.append(9L, -1L, 0L, seg2Records); + + PredicateBuilder builder = new PredicateBuilder(DATA1_ROW_TYPE); + Predicate filter = builder.greaterThan(0, 5); + + try (LogRecordReadContext readContext = + LogRecordReadContext.createArrowReadContext( + DATA1_ROW_TYPE, DEFAULT_SCHEMA_ID, TEST_SCHEMA_GETTER)) { + // Read starting from offset 0 — should return segment 1 data + FetchDataInfo result1 = + localLog.read( + 0, + Integer.MAX_VALUE, + false, + localLog.getLocalLogEndOffsetMetadata(), + null, + filter, + readContext, + null); + assertThat(result1).isNotNull(); + assertThat(result1.getRecords().sizeInBytes()).isGreaterThan(0); + + // Read starting from offset 5 (segment 2) — all filtered + FetchDataInfo result2 = + localLog.read( + 5, + Integer.MAX_VALUE, + false, + localLog.getLocalLogEndOffsetMetadata(), + null, + filter, + readContext, + null); + assertThat(result2).isNotNull(); + assertThat(result2.getRecords().sizeInBytes()).isEqualTo(0); + // filteredEndOffset should be set to advance past the filtered data + assertThat(result2.getFilteredEndOffset()).isEqualTo(10); + } + } + private FetchDataInfo readLog(LocalLog log, long startOffset, int maxLength) throws Exception { return log.read( - startOffset, maxLength, false, localLog.getLocalLogEndOffsetMetadata(), null); + startOffset, + maxLength, + false, + localLog.getLocalLogEndOffsetMetadata(), + null, + null, + null, + null); } private LocalLog createLocalLogWithActiveSegment( diff --git a/fluss-server/src/test/java/org/apache/fluss/server/log/LogManagerTest.java b/fluss-server/src/test/java/org/apache/fluss/server/log/LogManagerTest.java index bdaa5d6464..b7cb6c57fe 100644 --- a/fluss-server/src/test/java/org/apache/fluss/server/log/LogManagerTest.java +++ b/fluss-server/src/test/java/org/apache/fluss/server/log/LogManagerTest.java @@ -322,7 +322,7 @@ private void initTableBuckets(@Nullable String partitionName) { } private FetchDataInfo readLog(LogTablet log) throws Exception { - return log.read(0, Integer.MAX_VALUE, FetchIsolation.LOG_END, true, null); + return log.read(0, Integer.MAX_VALUE, FetchIsolation.LOG_END, true, null, null, null, null); } @AfterEach diff --git a/fluss-server/src/test/java/org/apache/fluss/server/log/LogSegmentTest.java b/fluss-server/src/test/java/org/apache/fluss/server/log/LogSegmentTest.java index 274a64264d..1bdff51b6b 100644 --- a/fluss-server/src/test/java/org/apache/fluss/server/log/LogSegmentTest.java +++ b/fluss-server/src/test/java/org/apache/fluss/server/log/LogSegmentTest.java @@ -21,12 +21,18 @@ import org.apache.fluss.config.MemorySize; import org.apache.fluss.exception.LogSegmentOffsetOverflowException; import org.apache.fluss.metadata.LogFormat; +import org.apache.fluss.predicate.Predicate; +import org.apache.fluss.predicate.PredicateBuilder; +import org.apache.fluss.record.FileLogProjection; import org.apache.fluss.record.LogRecord; import org.apache.fluss.record.LogRecordBatch; +import org.apache.fluss.record.LogRecordBatchStatisticsTestUtils; import org.apache.fluss.record.LogRecordReadContext; import org.apache.fluss.record.LogRecords; import org.apache.fluss.record.LogTestBase; import org.apache.fluss.record.MemoryLogRecords; +import org.apache.fluss.record.ProjectionPushdownCache; +import org.apache.fluss.utils.CloseableIterator; import org.junit.jupiter.api.Test; import org.junit.jupiter.api.io.TempDir; @@ -36,14 +42,20 @@ import java.io.File; import java.io.IOException; +import java.util.ArrayList; import java.util.Arrays; import java.util.Collections; +import java.util.List; +import java.util.function.IntPredicate; import java.util.stream.Stream; +import static org.apache.fluss.record.LogRecordBatch.CURRENT_LOG_MAGIC_VALUE; +import static org.apache.fluss.record.TestData.DATA1; import static org.apache.fluss.record.TestData.DATA1_ROW_TYPE; import static org.apache.fluss.record.TestData.DEFAULT_SCHEMA_ID; import static org.apache.fluss.record.TestData.TEST_SCHEMA_GETTER; import static org.apache.fluss.testutils.DataTestUtils.assertLogRecordsEquals; +import static org.apache.fluss.testutils.DataTestUtils.createRecordsWithoutBaseLogOffset; import static org.apache.fluss.testutils.DataTestUtils.genLogRecordsWithBaseOffsetAndTimestamp; import static org.apache.fluss.testutils.DataTestUtils.genMemoryLogRecordsWithBaseOffset; import static org.assertj.core.api.Assertions.assertThat; @@ -435,6 +447,537 @@ void testCreateWithInitFileSizeClearShutdown() throws Exception { assertThat(position).isEqualTo(oldPosition); } + @Test + void testReadWithFilterEqualPredicate() throws Exception { + PredicateBuilder builder = new PredicateBuilder(DATA1_ROW_TYPE); + Predicate equalPredicate = builder.equal(0, 5); + assertFilteredReadFindsMatchingRecord(equalPredicate, v -> v == 5); + } + + @Test + void testReadWithFilterGreaterThanPredicate() throws Exception { + PredicateBuilder builder = new PredicateBuilder(DATA1_ROW_TYPE); + Predicate greaterThanPredicate = builder.greaterThan(0, 3); + assertFilteredReadFindsMatchingRecord(greaterThanPredicate, v -> v > 3); + } + + @Test + void testReadWithFilterLessThanPredicate() throws Exception { + PredicateBuilder builder = new PredicateBuilder(DATA1_ROW_TYPE); + Predicate lessThanPredicate = builder.lessThan(0, 7); + assertFilteredReadFindsMatchingRecord(lessThanPredicate, v -> v < 7); + } + + @Test + void testReadWithFilterComplexPredicate() throws Exception { + PredicateBuilder builder = new PredicateBuilder(DATA1_ROW_TYPE); + Predicate greaterThan = builder.greaterThan(0, 3); + Predicate lessThan = builder.lessThan(0, 7); + Predicate complexPredicate = PredicateBuilder.and(greaterThan, lessThan); + assertFilteredReadFindsMatchingRecord(complexPredicate, v -> v > 3 && v < 7); + } + + /** + * Helper method to test filtered read with a given predicate and value matcher. + * + * @param predicate the predicate to apply + * @param valueMatcher function to check if first field value matches expectation + */ + private void assertFilteredReadFindsMatchingRecord( + Predicate predicate, IntPredicate valueMatcher) throws Exception { + LogSegment segment = createSegment(40); + MemoryLogRecords memoryRecords = + LogRecordBatchStatisticsTestUtils.createLogRecordsWithStatistics( + DATA1, DATA1_ROW_TYPE, 50, DEFAULT_SCHEMA_ID); + segment.append(59, -1L, -1L, memoryRecords); + + try (LogRecordReadContext readContext = + LogRecordReadContext.createArrowReadContext( + DATA1_ROW_TYPE, DEFAULT_SCHEMA_ID, TEST_SCHEMA_GETTER)) { + FetchDataInfo read = + segment.read( + 50, 300, segment.getSizeInBytes(), true, null, predicate, readContext); + assertThat(read).isNotNull(); + assertThat(read.getRecords().sizeInBytes()).isGreaterThan(0); + + boolean foundMatchingRecord = false; + for (LogRecordBatch batch : read.getRecords().batches()) { + try (CloseableIterator iterator = batch.records(readContext)) { + while (iterator.hasNext()) { + LogRecord record = iterator.next(); + if (valueMatcher.test(record.getRow().getInt(0))) { + foundMatchingRecord = true; + break; + } + } + } + } + assertThat(foundMatchingRecord).isTrue(); + } + } + + @Test + void testReadWithFilterMultipleBatches() throws Exception { + // Test reading with filter across multiple batches + LogSegment segment = createSegment(40); + + // Create multiple batches with different data + List batch1Data = + Arrays.asList(new Object[] {1, "a"}, new Object[] {2, "b"}, new Object[] {3, "c"}); + List batch2Data = + Arrays.asList(new Object[] {4, "d"}, new Object[] {5, "e"}, new Object[] {6, "f"}); + List batch3Data = + Arrays.asList(new Object[] {7, "g"}, new Object[] {8, "h"}, new Object[] {9, "i"}); + + MemoryLogRecords batch1 = + LogRecordBatchStatisticsTestUtils.createLogRecordsWithStatistics( + batch1Data, DATA1_ROW_TYPE, 50, DEFAULT_SCHEMA_ID); + MemoryLogRecords batch2 = + LogRecordBatchStatisticsTestUtils.createLogRecordsWithStatistics( + batch2Data, DATA1_ROW_TYPE, 53, DEFAULT_SCHEMA_ID); + MemoryLogRecords batch3 = + LogRecordBatchStatisticsTestUtils.createLogRecordsWithStatistics( + batch3Data, DATA1_ROW_TYPE, 56, DEFAULT_SCHEMA_ID); + + segment.append(52, -1L, -1L, batch1); + segment.append(55, -1L, -1L, batch2); + segment.append(58, -1L, -1L, batch3); + + // Create predicate (first field greater than 3) + PredicateBuilder builder = new PredicateBuilder(DATA1_ROW_TYPE); + Predicate greaterThanPredicate = builder.greaterThan(0, 3); + + // Verify that filtered records contain records from multiple batches + try (LogRecordReadContext readContext = + LogRecordReadContext.createArrowReadContext( + DATA1_ROW_TYPE, DEFAULT_SCHEMA_ID, TEST_SCHEMA_GETTER)) { + // Read with filter + FetchDataInfo read = + segment.read( + 50, + 1000, + segment.getSizeInBytes(), + true, + null, + greaterThanPredicate, + readContext); + assertThat(read).isNotNull(); + + List returnedValues = new ArrayList<>(); + for (LogRecordBatch batch : read.getRecords().batches()) { + try (CloseableIterator iterator = batch.records(readContext)) { + while (iterator.hasNext()) { + LogRecord record = iterator.next(); + returnedValues.add(record.getRow().getInt(0)); + } + } + } + // batch1 (values 1,2,3) should be EXCLUDED — all values <= 3 + assertThat(returnedValues).doesNotContain(1, 2, 3); + // batch2 (4,5,6) and batch3 (7,8,9) should be INCLUDED + assertThat(returnedValues).containsExactly(4, 5, 6, 7, 8, 9); + assertThat(returnedValues).hasSize(6); + } + } + + @Test + void testReadWithFilterEmptySegment() throws Exception { + // Test reading with filter on empty segment + LogSegment segment = createSegment(40); + + // Create predicate + PredicateBuilder builder = new PredicateBuilder(DATA1_ROW_TYPE); + Predicate equalPredicate = builder.equal(0, 5); + + try (LogRecordReadContext readContext = + LogRecordReadContext.createArrowReadContext( + DATA1_ROW_TYPE, DEFAULT_SCHEMA_ID, TEST_SCHEMA_GETTER)) { + // Read with filter on empty segment should return null + FetchDataInfo read = + segment.read( + 40, + 300, + segment.getSizeInBytes(), + true, + null, + equalPredicate, + readContext); + assertThat(read).isNull(); + } + } + + @Test + void testReadWithFilterRejectsAllBatchesInSegment() throws Exception { + // Test that when a filter rejects ALL batches in a non-empty segment, + // we get a filtered empty response with a valid filteredEndOffset. + LogSegment segment = createSegment(40); + + // Create multiple batches with values 1-9 (all <= 9) + List batch1Data = + Arrays.asList(new Object[] {1, "a"}, new Object[] {2, "b"}, new Object[] {3, "c"}); + List batch2Data = + Arrays.asList(new Object[] {4, "d"}, new Object[] {5, "e"}, new Object[] {6, "f"}); + List batch3Data = + Arrays.asList(new Object[] {7, "g"}, new Object[] {8, "h"}, new Object[] {9, "i"}); + + MemoryLogRecords batch1 = + LogRecordBatchStatisticsTestUtils.createLogRecordsWithStatistics( + batch1Data, DATA1_ROW_TYPE, 50, DEFAULT_SCHEMA_ID); + MemoryLogRecords batch2 = + LogRecordBatchStatisticsTestUtils.createLogRecordsWithStatistics( + batch2Data, DATA1_ROW_TYPE, 53, DEFAULT_SCHEMA_ID); + MemoryLogRecords batch3 = + LogRecordBatchStatisticsTestUtils.createLogRecordsWithStatistics( + batch3Data, DATA1_ROW_TYPE, 56, DEFAULT_SCHEMA_ID); + + segment.append(52, -1L, -1L, batch1); + segment.append(55, -1L, -1L, batch2); + segment.append(58, -1L, -1L, batch3); + + // Predicate: first field > 100 — no batch can match (max value is 9) + PredicateBuilder builder = new PredicateBuilder(DATA1_ROW_TYPE); + Predicate predicate = builder.greaterThan(0, 100); + + try (LogRecordReadContext readContext = + LogRecordReadContext.createArrowReadContext( + DATA1_ROW_TYPE, DEFAULT_SCHEMA_ID, TEST_SCHEMA_GETTER)) { + FetchDataInfo read = + segment.read( + 50, 1000, segment.getSizeInBytes(), true, null, predicate, readContext); + assertThat(read).isNotNull(); + // All batches filtered out — records should be empty + assertThat(read.getRecords().sizeInBytes()).isEqualTo(0); + // filteredEndOffset should be set to allow client to advance + assertThat(read.hasFilteredEndOffset()).isTrue(); + assertThat(read.getFilteredEndOffset()).isGreaterThanOrEqualTo(50L); + } + } + + @Test + void testReadWithFilterExceptionInPredicateFallsBackToInclude() throws Exception { + LogSegment segment = createSegment(40); + MemoryLogRecords records = + LogRecordBatchStatisticsTestUtils.createLogRecordsWithStatistics( + DATA1, DATA1_ROW_TYPE, 50, DEFAULT_SCHEMA_ID); + segment.append(59, -1L, -1L, records); + + // Create a predicate that always throws during test() + Predicate throwingPredicate = + new Predicate() { + @Override + public boolean test(org.apache.fluss.row.InternalRow row) { + throw new RuntimeException("Simulated filter failure"); + } + + @Override + public boolean test( + long rowCount, + org.apache.fluss.row.InternalRow minValues, + org.apache.fluss.row.InternalRow maxValues, + Long[] nullCounts) { + throw new RuntimeException("Simulated filter failure"); + } + + @Override + public java.util.Optional negate() { + return java.util.Optional.empty(); + } + + @Override + public T visit(org.apache.fluss.predicate.PredicateVisitor visitor) { + throw new RuntimeException("Simulated filter failure"); + } + }; + + try (LogRecordReadContext readContext = + LogRecordReadContext.createArrowReadContext( + DATA1_ROW_TYPE, DEFAULT_SCHEMA_ID, TEST_SCHEMA_GETTER)) { + // Should NOT throw — should fall back to including the batch + FetchDataInfo read = + segment.read( + 50, + 300, + segment.getSizeInBytes(), + true, + null, + throwingPredicate, + readContext); + assertThat(read).isNotNull(); + assertThat(read.getRecords().sizeInBytes()).isGreaterThan(0); + } + } + + @Test + void testReadWithFilterAndProjection() throws Exception { + // Test filter + projection combination: filter selects batches, projection selects columns + LogSegment segment = createSegment(40); + + // Batch 1: values 1,2,3 — should be filtered out by "a > 5" + MemoryLogRecords batch1 = + LogRecordBatchStatisticsTestUtils.createLogRecordsWithStatistics( + Arrays.asList( + new Object[] {1, "a"}, + new Object[] {2, "b"}, + new Object[] {3, "c"}), + DATA1_ROW_TYPE, + 50, + DEFAULT_SCHEMA_ID); + // Batch 2: values 7,8,9 — should pass filter "a > 5" + MemoryLogRecords batch2 = + LogRecordBatchStatisticsTestUtils.createLogRecordsWithStatistics( + Arrays.asList( + new Object[] {7, "g"}, + new Object[] {8, "h"}, + new Object[] {9, "i"}), + DATA1_ROW_TYPE, + 53, + DEFAULT_SCHEMA_ID); + + segment.append(52, -1L, -1L, batch1); + segment.append(55, -1L, -1L, batch2); + + // Create filter: a > 5 + PredicateBuilder builder = new PredicateBuilder(DATA1_ROW_TYPE); + Predicate filter = builder.greaterThan(0, 5); + + // Create projection: project only column 0 (field "a") + FileLogProjection projection = new FileLogProjection(new ProjectionPushdownCache()); + projection.setCurrentProjection( + 1L, + TEST_SCHEMA_GETTER, + org.apache.fluss.compression.ArrowCompressionInfo.DEFAULT_COMPRESSION, + new int[] {0}); + + try (LogRecordReadContext readContext = + LogRecordReadContext.createArrowReadContext( + DATA1_ROW_TYPE, DEFAULT_SCHEMA_ID, TEST_SCHEMA_GETTER)) { + FetchDataInfo read = + segment.read( + 50, + 1000, + segment.getSizeInBytes(), + true, + projection, + filter, + readContext); + assertThat(read).isNotNull(); + assertThat(read.getRecords().sizeInBytes()).isGreaterThan(0); + + // Verify: only batch2 data returned, projected to single column + org.apache.fluss.types.RowType projectedType = DATA1_ROW_TYPE.project(new int[] {0}); + try (LogRecordReadContext projectedContext = + LogRecordReadContext.createArrowReadContext( + projectedType, DEFAULT_SCHEMA_ID, TEST_SCHEMA_GETTER)) { + int recordCount = 0; + for (LogRecordBatch b : read.getRecords().batches()) { + try (CloseableIterator iter = b.records(projectedContext)) { + while (iter.hasNext()) { + LogRecord record = iter.next(); + // Projected row should have 1 field + assertThat(record.getRow().getFieldCount()).isEqualTo(1); + // Values should be from batch2 (> 5) + assertThat(record.getRow().getInt(0)).isGreaterThan(5); + recordCount++; + } + } + } + assertThat(recordCount).isEqualTo(3); + } + } + } + + @Test + void testReadWithFilterAndProjectionOnEmptyBatchReturnsSkipOffset() throws Exception { + LogSegment segment = createSegment(40); + MemoryLogRecords emptyBatch = + createRecordsWithoutBaseLogOffset( + DATA1_ROW_TYPE, + DEFAULT_SCHEMA_ID, + 50L, + System.currentTimeMillis(), + CURRENT_LOG_MAGIC_VALUE, + Collections.emptyList(), + LogFormat.ARROW); + segment.append(49, -1L, -1L, emptyBatch); + + PredicateBuilder builder = new PredicateBuilder(DATA1_ROW_TYPE); + Predicate filter = builder.greaterThan(0, 5); + + FileLogProjection projection = new FileLogProjection(new ProjectionPushdownCache()); + projection.setCurrentProjection( + 1L, + TEST_SCHEMA_GETTER, + org.apache.fluss.compression.ArrowCompressionInfo.DEFAULT_COMPRESSION, + new int[] {0}); + + try (LogRecordReadContext readContext = + LogRecordReadContext.createArrowReadContext( + DATA1_ROW_TYPE, DEFAULT_SCHEMA_ID, TEST_SCHEMA_GETTER)) { + FetchDataInfo read = + segment.read( + 50, + 1000, + segment.getSizeInBytes(), + true, + projection, + filter, + readContext); + assertThat(read).isNotNull(); + assertThat(read.getRecords().sizeInBytes()).isEqualTo(0); + assertThat(read.getFilteredEndOffset()).isGreaterThanOrEqualTo(50L); + } + } + + @Test + void testReadWithFilterOffsetMetadataUsesStartOffset() throws Exception { + LogSegment segment = createSegment(40); + + // Batch 1: values 1,2,3 — will be filtered out by a > 5 + MemoryLogRecords batch1 = + LogRecordBatchStatisticsTestUtils.createLogRecordsWithStatistics( + Arrays.asList( + new Object[] {1, "a"}, + new Object[] {2, "b"}, + new Object[] {3, "c"}), + DATA1_ROW_TYPE, + 50, + DEFAULT_SCHEMA_ID); + // Batch 2: values 7,8,9 — passes filter + MemoryLogRecords batch2 = + LogRecordBatchStatisticsTestUtils.createLogRecordsWithStatistics( + Arrays.asList( + new Object[] {7, "g"}, + new Object[] {8, "h"}, + new Object[] {9, "i"}), + DATA1_ROW_TYPE, + 53, + DEFAULT_SCHEMA_ID); + + segment.append(52, -1L, -1L, batch1); + segment.append(55, -1L, -1L, batch2); + + PredicateBuilder builder = new PredicateBuilder(DATA1_ROW_TYPE); + Predicate filter = builder.greaterThan(0, 5); + + try (LogRecordReadContext readContext = + LogRecordReadContext.createArrowReadContext( + DATA1_ROW_TYPE, DEFAULT_SCHEMA_ID, TEST_SCHEMA_GETTER)) { + FetchDataInfo read = + segment.read( + 50, 1000, segment.getSizeInBytes(), true, null, filter, readContext); + assertThat(read).isNotNull(); + // The offset metadata should use the requested startOffset (50), + // NOT the first included batch's baseLogOffset (53) + assertThat(read.getFetchOffsetMetadata().getMessageOffset()).isEqualTo(50); + } + } + + @Test + void testReadWithFilterAndProjectionSizeLimitUsesProjectedSize() throws Exception { + LogSegment segment = createSegment(40); + + // Create 2 batches — all values > 5 so both pass filter "a > 5" + // Use long strings to make unprojected size much larger than projected + List batch1Data = + Arrays.asList( + new Object[] {6, "aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa"}, + new Object[] {7, "bbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbb"}, + new Object[] {8, "cccccccccccccccccccccccccccccccccccccccc"}); + List batch2Data = + Arrays.asList( + new Object[] {9, "dddddddddddddddddddddddddddddddddddddd"}, + new Object[] {10, "eeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeee"}, + new Object[] {11, "ffffffffffffffffffffffffffffffffffffffff"}); + + MemoryLogRecords batch1 = + LogRecordBatchStatisticsTestUtils.createLogRecordsWithStatistics( + batch1Data, DATA1_ROW_TYPE, 50, DEFAULT_SCHEMA_ID); + MemoryLogRecords batch2 = + LogRecordBatchStatisticsTestUtils.createLogRecordsWithStatistics( + batch2Data, DATA1_ROW_TYPE, 53, DEFAULT_SCHEMA_ID); + + segment.append(52, -1L, -1L, batch1); + segment.append(55, -1L, -1L, batch2); + + PredicateBuilder builder = new PredicateBuilder(DATA1_ROW_TYPE); + Predicate filter = builder.greaterThan(0, 5); + + // Project only column 0 (int field "a") — much smaller than full row + FileLogProjection projection = new FileLogProjection(new ProjectionPushdownCache()); + projection.setCurrentProjection( + 1L, + TEST_SCHEMA_GETTER, + org.apache.fluss.compression.ArrowCompressionInfo.DEFAULT_COMPRESSION, + new int[] {0}); + + int unprojectedBatch1Size = batch1.sizeInBytes(); + + try (LogRecordReadContext readContext = + LogRecordReadContext.createArrowReadContext( + DATA1_ROW_TYPE, DEFAULT_SCHEMA_ID, TEST_SCHEMA_GETTER)) { + // Set maxSize to slightly more than one unprojected batch but less than two. + // With the bug: only batch1 returned (break triggered by unprojected size). + // With the fix: both batches fit (projected sizes are smaller). + FetchDataInfo read = + segment.read( + 50, + unprojectedBatch1Size + 1, + segment.getSizeInBytes(), + true, + projection, + filter, + readContext); + assertThat(read).isNotNull(); + + int batchCount = 0; + for (LogRecordBatch b : read.getRecords().batches()) { + batchCount++; + } + assertThat(batchCount).isEqualTo(2); + } + } + + @Test + void testReadWithFilterRequiresReadContext() throws Exception { + LogSegment segment = createSegment(40); + PredicateBuilder builder = new PredicateBuilder(DATA1_ROW_TYPE); + Predicate predicate = builder.greaterThan(0, 3); + + // filter non-null but readContext null → should throw + assertThatThrownBy( + () -> + segment.read( + 40, + 300, + segment.getSizeInBytes(), + true, + null, + predicate, + null)) + .isInstanceOf(IllegalArgumentException.class); + } + + @Test + void testReadWithReadContextRequiresFilter() throws Exception { + LogSegment segment = createSegment(40); + try (LogRecordReadContext readContext = + LogRecordReadContext.createArrowReadContext( + DATA1_ROW_TYPE, DEFAULT_SCHEMA_ID, TEST_SCHEMA_GETTER)) { + // readContext non-null but filter null → should throw + assertThatThrownBy( + () -> + segment.read( + 40, + 300, + segment.getSizeInBytes(), + true, + null, + null, + readContext)) + .isInstanceOf(IllegalArgumentException.class); + } + } + private LogSegment createSegment(long baseOffset) throws IOException { return createSegment(baseOffset, 10); } diff --git a/fluss-server/src/test/java/org/apache/fluss/server/log/LogTabletTest.java b/fluss-server/src/test/java/org/apache/fluss/server/log/LogTabletTest.java index 7b88a53a52..3cdef1bade 100644 --- a/fluss-server/src/test/java/org/apache/fluss/server/log/LogTabletTest.java +++ b/fluss-server/src/test/java/org/apache/fluss/server/log/LogTabletTest.java @@ -584,7 +584,7 @@ private FetchDataInfo readLog( FetchIsolation isolation, boolean minOneMessage) throws Exception { - return logTablet.read(offset, maxLength, isolation, minOneMessage, null); + return logTablet.read(offset, maxLength, isolation, minOneMessage, null, null, null, null); } private void assertValidLogOffsetMetadata(LogOffsetMetadata offsetMetadata) throws IOException { diff --git a/fluss-server/src/test/java/org/apache/fluss/server/log/PredicateSchemaResolverTest.java b/fluss-server/src/test/java/org/apache/fluss/server/log/PredicateSchemaResolverTest.java new file mode 100644 index 0000000000..9c81549dae --- /dev/null +++ b/fluss-server/src/test/java/org/apache/fluss/server/log/PredicateSchemaResolverTest.java @@ -0,0 +1,231 @@ +/* + * 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.server.log; + +import org.apache.fluss.metadata.Schema; +import org.apache.fluss.metadata.SchemaInfo; +import org.apache.fluss.predicate.LeafPredicate; +import org.apache.fluss.predicate.Predicate; +import org.apache.fluss.predicate.PredicateBuilder; +import org.apache.fluss.record.TestingSchemaGetter; +import org.apache.fluss.types.DataTypes; +import org.apache.fluss.types.RowType; + +import org.junit.jupiter.api.Test; + +import java.util.Arrays; + +import static org.assertj.core.api.Assertions.assertThat; + +/** Tests for {@link PredicateSchemaResolver}. */ +class PredicateSchemaResolverTest { + + private static final int SCHEMA_ID_1 = 1; + private static final int SCHEMA_ID_2 = 2; + + // Schema v1: columns (a INT colId=1, b STRING colId=2) + private static final Schema SCHEMA_V1 = + Schema.newBuilder() + .fromColumns( + Arrays.asList( + new Schema.Column("a", DataTypes.INT(), null, 1), + new Schema.Column("b", DataTypes.STRING(), null, 2))) + .build(); + + private static final RowType ROW_TYPE_V1 = + DataTypes.ROW( + DataTypes.FIELD("a", DataTypes.INT()), + DataTypes.FIELD("b", DataTypes.STRING())); + + // Schema v2: columns (a INT, b STRING, c STRING) with column IDs 1, 2, 3 + // Simulates adding a new column "c" + private static final Schema SCHEMA_V2 = + Schema.newBuilder() + .fromColumns( + Arrays.asList( + new Schema.Column("a", DataTypes.INT(), null, 1), + new Schema.Column("b", DataTypes.STRING(), null, 2), + new Schema.Column("c", DataTypes.STRING(), null, 3))) + .build(); + + @Test + void testSameSchemaIdReturnsSamePredicate() { + Predicate predicate = new PredicateBuilder(ROW_TYPE_V1).equal(0, 42); + TestingSchemaGetter schemaGetter = new TestingSchemaGetter(SCHEMA_ID_1, SCHEMA_V1); + + PredicateSchemaResolver resolver = + new PredicateSchemaResolver(predicate, SCHEMA_ID_1, schemaGetter); + + Predicate result = resolver.resolve(SCHEMA_ID_1); + assertThat(result).isSameAs(predicate); + } + + @Test + void testNegativePredicateSchemaIdAlwaysReturnsOriginal() { + Predicate predicate = new PredicateBuilder(ROW_TYPE_V1).equal(0, 42); + + // Negative predicate schema ID means schema tracking is disabled + PredicateSchemaResolver resolver = new PredicateSchemaResolver(predicate, -1, null); + + // Should return original predicate regardless of batch schema ID + assertThat(resolver.resolve(SCHEMA_ID_1)).isSameAs(predicate); + assertThat(resolver.resolve(SCHEMA_ID_2)).isSameAs(predicate); + assertThat(resolver.resolve(999)).isSameAs(predicate); + } + + @Test + void testNullSchemaGetterDifferentSchemaReturnsNull() { + Predicate predicate = new PredicateBuilder(ROW_TYPE_V1).equal(0, 42); + + PredicateSchemaResolver resolver = + new PredicateSchemaResolver(predicate, SCHEMA_ID_1, null); + + // Same schema still works + assertThat(resolver.resolve(SCHEMA_ID_1)).isSameAs(predicate); + // Different schema returns null (safe fallback: include batch) + assertThat(resolver.resolve(SCHEMA_ID_2)).isNull(); + } + + @Test + void testCacheHitReturnsSameResult() { + TestingSchemaGetter schemaGetter = new TestingSchemaGetter(SCHEMA_ID_1, SCHEMA_V1); + schemaGetter.updateLatestSchemaInfo(new SchemaInfo(SCHEMA_V2, SCHEMA_ID_2)); + + Predicate predicate = new PredicateBuilder(ROW_TYPE_V1).equal(0, 42); + PredicateSchemaResolver resolver = + new PredicateSchemaResolver(predicate, SCHEMA_ID_1, schemaGetter); + + // First call populates cache + Predicate first = resolver.resolve(SCHEMA_ID_2); + // Second call should return the same cached instance + Predicate second = resolver.resolve(SCHEMA_ID_2); + + assertThat(first).isNotNull(); + assertThat(second).isSameAs(first); + } + + @Test + void testSchemaGetterThrowsInConstructor() { + // SchemaGetter that throws for the predicate schema ID + TestingSchemaGetter schemaGetter = new TestingSchemaGetter(); + + Predicate predicate = new PredicateBuilder(ROW_TYPE_V1).equal(0, 42); + + // Constructor should NOT throw even though getSchema will throw + PredicateSchemaResolver resolver = + new PredicateSchemaResolver(predicate, SCHEMA_ID_1, schemaGetter); + + // Same schema still works (fast path doesn't need predicateSchema) + assertThat(resolver.resolve(SCHEMA_ID_1)).isSameAs(predicate); + // Different schema returns null because predicateSchema couldn't be resolved + assertThat(resolver.resolve(SCHEMA_ID_2)).isNull(); + } + + @Test + void testCrossSchemaAdaptation() { + // Predicate built on schema v1 (a=0, b=1), predicate on column "a" at index 0 + Predicate predicate = new PredicateBuilder(ROW_TYPE_V1).equal(0, 42); + + TestingSchemaGetter schemaGetter = new TestingSchemaGetter(SCHEMA_ID_1, SCHEMA_V1); + schemaGetter.updateLatestSchemaInfo(new SchemaInfo(SCHEMA_V2, SCHEMA_ID_2)); + + PredicateSchemaResolver resolver = + new PredicateSchemaResolver(predicate, SCHEMA_ID_1, schemaGetter); + + // Resolve for schema v2 which has an extra column "c" + Predicate adapted = resolver.resolve(SCHEMA_ID_2); + + // The adapted predicate should still reference column "a" at index 0 in schema v2 + // because column IDs match (both have column ID 1 for "a" at index 0) + assertThat(adapted).isNotNull(); + assertThat(adapted).isInstanceOf(LeafPredicate.class); + LeafPredicate leaf = (LeafPredicate) adapted; + assertThat(leaf.index()).isEqualTo(0); + assertThat(leaf.fieldName()).isEqualTo("a"); + } + + @Test + void testCrossSchemaAdaptationWithReorderedColumns() { + // Schema v1: (a INT colId=1, b STRING colId=2) + // Schema v3: (b STRING colId=2, a INT colId=1) — columns reordered + Schema schemaV3 = + Schema.newBuilder() + .fromColumns( + Arrays.asList( + new Schema.Column("b", DataTypes.STRING(), null, 2), + new Schema.Column("a", DataTypes.INT(), null, 1))) + .build(); + int schemaId3 = 3; + + // Predicate on column "a" at index 0 in schema v1 + Predicate predicate = new PredicateBuilder(ROW_TYPE_V1).equal(0, 42); + + TestingSchemaGetter schemaGetter = new TestingSchemaGetter(SCHEMA_ID_1, SCHEMA_V1); + schemaGetter.updateLatestSchemaInfo(new SchemaInfo(schemaV3, schemaId3)); + + PredicateSchemaResolver resolver = + new PredicateSchemaResolver(predicate, SCHEMA_ID_1, schemaGetter); + + Predicate adapted = resolver.resolve(schemaId3); + + // In schema v3, column "a" (colId=1) is at index 1 (second position) + assertThat(adapted).isNotNull(); + assertThat(adapted).isInstanceOf(LeafPredicate.class); + LeafPredicate leaf = (LeafPredicate) adapted; + assertThat(leaf.index()).isEqualTo(1); + assertThat(leaf.fieldName()).isEqualTo("a"); + } + + @Test + void testBatchSchemaNotFoundReturnsNull() { + TestingSchemaGetter schemaGetter = new TestingSchemaGetter(SCHEMA_ID_1, SCHEMA_V1); + // Schema ID 99 is not registered in the getter + + Predicate predicate = new PredicateBuilder(ROW_TYPE_V1).equal(0, 42); + PredicateSchemaResolver resolver = + new PredicateSchemaResolver(predicate, SCHEMA_ID_1, schemaGetter); + + // Should return null because batch schema 99 doesn't exist + assertThat(resolver.resolve(99)).isNull(); + } + + @Test + void testAdaptationForDroppedColumnReturnsNull() { + // Schema v1: (a INT colId=1, b STRING colId=2) + // Schema v4: (c STRING colId=3) — column "a" was dropped + Schema schemaV4 = + Schema.newBuilder() + .fromColumns( + Arrays.asList(new Schema.Column("c", DataTypes.STRING(), null, 3))) + .build(); + int schemaId4 = 4; + + // Predicate on column "a" at index 0 in schema v1 + Predicate predicate = new PredicateBuilder(ROW_TYPE_V1).equal(0, 42); + + TestingSchemaGetter schemaGetter = new TestingSchemaGetter(SCHEMA_ID_1, SCHEMA_V1); + schemaGetter.updateLatestSchemaInfo(new SchemaInfo(schemaV4, schemaId4)); + + PredicateSchemaResolver resolver = + new PredicateSchemaResolver(predicate, SCHEMA_ID_1, schemaGetter); + + // Column "a" doesn't exist in schema v4, so adaptation should return null + // (transformFieldMapping returns Optional.empty for unmapped fields) + assertThat(resolver.resolve(schemaId4)).isNull(); + } +} diff --git a/fluss-server/src/test/java/org/apache/fluss/server/replica/fetcher/ReplicaFetcherITCase.java b/fluss-server/src/test/java/org/apache/fluss/server/replica/fetcher/ReplicaFetcherITCase.java index 1e787a345c..e2bb80f96f 100644 --- a/fluss-server/src/test/java/org/apache/fluss/server/replica/fetcher/ReplicaFetcherITCase.java +++ b/fluss-server/src/test/java/org/apache/fluss/server/replica/fetcher/ReplicaFetcherITCase.java @@ -155,7 +155,7 @@ void testProduceLogNeedAck() throws Exception { new CompletableFuture<>(); // mock client fetch from follower. replicaManager.fetchLogRecords( - new FetchParams(-1, false, Integer.MAX_VALUE, -1, -1), + new FetchParams(-1, false, Integer.MAX_VALUE, -1, -1, null), Collections.singletonMap(tb, new FetchReqInfo(tableId, 0L, 1024 * 1024)), null, future::complete); @@ -232,7 +232,7 @@ void testPutKvNeedAck() throws Exception { new CompletableFuture<>(); // mock client fetch from follower. replicaManager.fetchLogRecords( - new FetchParams(-1, false, Integer.MAX_VALUE, -1, -1), + new FetchParams(-1, false, Integer.MAX_VALUE, -1, -1, null), Collections.singletonMap(tb, new FetchReqInfo(tableId, 0L, 1024 * 1024)), null, future::complete); From 59811c26402893c86f42cb2e0ec1acbd4f0fcc59 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?=E7=99=BD=E9=B5=BA?= Date: Mon, 30 Mar 2026 15:02:27 +0800 Subject: [PATCH 2/2] [server] Fix code review issues for filter pushdown --- .../table/scanner/log/CompletedFetch.java | 10 ++++---- ...rTest.java => LogFetcherFilterITCase.java} | 10 ++++---- .../fluss/record/FileLogProjection.java | 3 ++- .../apache/fluss/record/FileLogRecords.java | 4 ++++ .../fluss/record/FileLogProjectionTest.java | 6 +++-- .../fluss/server/log/FetchParamsBuilder.java | 8 ++++--- .../apache/fluss/server/log/FilterInfo.java | 19 +++++++++++++++ .../apache/fluss/server/log/LogSegment.java | 3 ++- .../apache/fluss/server/replica/Replica.java | 5 ++-- .../server/utils/ServerRpcMessageUtils.java | 21 +++++++++++------ .../fluss/server/log/FetchParamsTest.java | 23 +++++++++++++++++++ 11 files changed, 85 insertions(+), 27 deletions(-) rename fluss-client/src/test/java/org/apache/fluss/client/table/scanner/log/{LogFetcherFilterTest.java => LogFetcherFilterITCase.java} (97%) diff --git a/fluss-client/src/main/java/org/apache/fluss/client/table/scanner/log/CompletedFetch.java b/fluss-client/src/main/java/org/apache/fluss/client/table/scanner/log/CompletedFetch.java index fe18a457e6..0113335210 100644 --- a/fluss-client/src/main/java/org/apache/fluss/client/table/scanner/log/CompletedFetch.java +++ b/fluss-client/src/main/java/org/apache/fluss/client/table/scanner/log/CompletedFetch.java @@ -42,9 +42,9 @@ import static org.apache.fluss.utils.Preconditions.checkArgument; /** - * {@link CompletedFetch} represents the result that was returned from the tablet server via a - * {@link FetchLogRequest}, which can be a {@link LogRecordBatch} or remote log segments path. It - * contains logic to maintain state between calls to {@link #fetchRecords(int)}. + * {@link CompletedFetch} represents the result that was returned from the tablet server via a fetch + * log request, which can be a {@link LogRecordBatch} or remote log segments path. It contains logic + * to maintain state between calls to {@link #fetchRecords(int)}. */ @Internal abstract class CompletedFetch { @@ -198,8 +198,8 @@ public List fetchRecords(int maxRecords) { ScanRecord record = toScanRecord(lastRecord); scanRecords.add(record); recordsRead++; - // Update nextFetchOffset based on the current record - // This will be overridden by batch-level nextLogOffset when batch is complete + // Per-record offset is a best-effort value; the authoritative offset + // comes from the batch's nextLogOffset once the batch is fully consumed. nextFetchOffset = lastRecord.logOffset() + 1; cachedRecordException = null; } diff --git a/fluss-client/src/test/java/org/apache/fluss/client/table/scanner/log/LogFetcherFilterTest.java b/fluss-client/src/test/java/org/apache/fluss/client/table/scanner/log/LogFetcherFilterITCase.java similarity index 97% rename from fluss-client/src/test/java/org/apache/fluss/client/table/scanner/log/LogFetcherFilterTest.java rename to fluss-client/src/test/java/org/apache/fluss/client/table/scanner/log/LogFetcherFilterITCase.java index 962b131aec..a5bd5a4eef 100644 --- a/fluss-client/src/test/java/org/apache/fluss/client/table/scanner/log/LogFetcherFilterTest.java +++ b/fluss-client/src/test/java/org/apache/fluss/client/table/scanner/log/LogFetcherFilterITCase.java @@ -52,8 +52,8 @@ import static org.apache.fluss.testutils.common.CommonTestUtils.retry; import static org.assertj.core.api.Assertions.assertThat; -/** Test for {@link LogFetcher} with recordBatchFilter pushdown scenarios. */ -public class LogFetcherFilterTest extends ClientToServerITCaseBase { +/** Integration test for {@link LogFetcher} with recordBatchFilter pushdown scenarios. */ +public class LogFetcherFilterITCase extends ClientToServerITCaseBase { private LogFetcher logFetcher; private long tableId; private final int bucketId0 = 0; @@ -181,7 +181,7 @@ void testFetchWithRecordBatchFilter() throws Exception { } @Test - void testFetchWithMixedData() throws Exception { + void testBatchLevelFilterIncludesEntireBatchWhenStatisticsOverlap() throws Exception { TableBucket tb0 = new TableBucket(tableId, bucketId0); // Create mixed data: some matching, some not matching (using DATA1 structure) @@ -275,9 +275,7 @@ void testFilterCompletelyRejectsNonMatchingBatch() throws Exception { // For a batch where max value = 5 and filter is a > 5, the entire batch should be // filtered out at the server side. collectFetch() may omit the bucket entirely because // there are no user-visible records to return. - if (records.containsKey(tb0)) { - assertThat(records.get(tb0)).isEmpty(); - } + assertThat(records.getOrDefault(tb0, Collections.emptyList())).isEmpty(); } @Test diff --git a/fluss-common/src/main/java/org/apache/fluss/record/FileLogProjection.java b/fluss-common/src/main/java/org/apache/fluss/record/FileLogProjection.java index 87e52731f2..c6af730496 100644 --- a/fluss-common/src/main/java/org/apache/fluss/record/FileLogProjection.java +++ b/fluss-common/src/main/java/org/apache/fluss/record/FileLogProjection.java @@ -152,7 +152,8 @@ public BytesView projectRecordBatch(FileChannelLogRecordBatch batch) throws IOEx MultiBytesView.Builder builder = MultiBytesView.builder(); - // Return empty if meets empty batch + // Empty batches (header-only) can occur for CDC log batches with no changes; + // return empty projection to preserve offset advancement. if (batchSizeInBytes == recordBatchHeaderSize) { return builder.build(); } diff --git a/fluss-common/src/main/java/org/apache/fluss/record/FileLogRecords.java b/fluss-common/src/main/java/org/apache/fluss/record/FileLogRecords.java index 46db535fb4..daf9aba3b0 100644 --- a/fluss-common/src/main/java/org/apache/fluss/record/FileLogRecords.java +++ b/fluss-common/src/main/java/org/apache/fluss/record/FileLogRecords.java @@ -322,6 +322,10 @@ private AbstractIterator batchIterator(int start) */ public AbstractIterator batchIterator(int start, int endPosition) throws IOException { + if (endPosition >= 0 && start > endPosition) { + throw new IllegalArgumentException( + "start (" + start + ") must be <= endPosition (" + endPosition + ")"); + } final int end; if (endPosition >= 0) { end = endPosition; diff --git a/fluss-common/src/test/java/org/apache/fluss/record/FileLogProjectionTest.java b/fluss-common/src/test/java/org/apache/fluss/record/FileLogProjectionTest.java index c2602dda62..f29ea605f0 100644 --- a/fluss-common/src/test/java/org/apache/fluss/record/FileLogProjectionTest.java +++ b/fluss-common/src/test/java/org/apache/fluss/record/FileLogProjectionTest.java @@ -745,9 +745,11 @@ void testProjectRecordBatchNoProjectionSet() throws Exception { fileLogRecords.batchIterator(0, -1).next(); // Should throw exception when no projection is set - // getOrCreateProjectionInfo throws NPE because no projection has been registered + // getOrCreateProjectionInfo returns null because no projection has been registered, + // causing a NullPointerException downstream. This is an implementation detail — + // the key contract is that calling projectRecordBatch without setting a projection fails. assertThatThrownBy(() -> projection.projectRecordBatch(batch)) - .isInstanceOf(NullPointerException.class); + .isInstanceOf(Exception.class); } @ParameterizedTest diff --git a/fluss-server/src/main/java/org/apache/fluss/server/log/FetchParamsBuilder.java b/fluss-server/src/main/java/org/apache/fluss/server/log/FetchParamsBuilder.java index 30d7e80245..0bb2c0058f 100644 --- a/fluss-server/src/main/java/org/apache/fluss/server/log/FetchParamsBuilder.java +++ b/fluss-server/src/main/java/org/apache/fluss/server/log/FetchParamsBuilder.java @@ -19,6 +19,8 @@ import org.apache.fluss.annotation.VisibleForTesting; +import javax.annotation.Nullable; + import java.util.Map; import static org.apache.fluss.server.log.FetchParams.DEFAULT_MAX_WAIT_MS; @@ -26,10 +28,10 @@ /** Builder of FetchParams. */ public final class FetchParamsBuilder { - private int replicaId; + private final int replicaId; private boolean fetchOnlyLeader = true; - private int maxFetchBytes; - private Map tableFilterInfoMap; + private final int maxFetchBytes; + @Nullable private Map tableFilterInfoMap; private int minFetchBytes; private long maxWaitMs; diff --git a/fluss-server/src/main/java/org/apache/fluss/server/log/FilterInfo.java b/fluss-server/src/main/java/org/apache/fluss/server/log/FilterInfo.java index b020ec7ce2..8fa972d177 100644 --- a/fluss-server/src/main/java/org/apache/fluss/server/log/FilterInfo.java +++ b/fluss-server/src/main/java/org/apache/fluss/server/log/FilterInfo.java @@ -19,6 +19,8 @@ import org.apache.fluss.rpc.messages.PbPredicate; +import java.util.Objects; + /** Holds the raw filter predicate and schema ID from a fetch request. */ public final class FilterInfo { private final PbPredicate pbPredicate; @@ -36,4 +38,21 @@ public PbPredicate getPbPredicate() { public int getSchemaId() { return schemaId; } + + @Override + public boolean equals(Object o) { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + FilterInfo that = (FilterInfo) o; + return schemaId == that.schemaId && Objects.equals(pbPredicate, that.pbPredicate); + } + + @Override + public int hashCode() { + return Objects.hash(pbPredicate, schemaId); + } } diff --git a/fluss-server/src/main/java/org/apache/fluss/server/log/LogSegment.java b/fluss-server/src/main/java/org/apache/fluss/server/log/LogSegment.java index c299a73412..2a73b4c2a7 100644 --- a/fluss-server/src/main/java/org/apache/fluss/server/log/LogSegment.java +++ b/fluss-server/src/main/java/org/apache/fluss/server/log/LogSegment.java @@ -718,10 +718,11 @@ private FetchDataInfo readWithFilter( } } catch (Exception e) { LOG.warn( - "Failed to evaluate filter for batch at offset {} in segment {}, " + "Failed to evaluate filter for batch at offset {} in segment {} ({}), " + "including batch as safe fallback.", batch.baseLogOffset(), fileLogRecords, + e.getClass().getSimpleName(), e); include = true; } diff --git a/fluss-server/src/main/java/org/apache/fluss/server/replica/Replica.java b/fluss-server/src/main/java/org/apache/fluss/server/replica/Replica.java index fc06e1ecdb..e75f787538 100644 --- a/fluss-server/src/main/java/org/apache/fluss/server/replica/Replica.java +++ b/fluss-server/src/main/java/org/apache/fluss/server/replica/Replica.java @@ -1537,9 +1537,10 @@ private LogReadInfo readRecords(FetchParams fetchParams, LogTablet logTablet) } } catch (Exception e) { LOG.warn( - "Failed to initialize filter context for {}, " + "Failed to initialize filter context for {} ({}), " + "falling back to unfiltered read.", tableBucket, + e.getClass().getSimpleName(), e); // Safe fallback: read without filter. resolvedFilter/readContext/predicateResolver // remain null, so the read proceeds as if no filter was requested. @@ -1565,7 +1566,7 @@ private LogReadInfo readRecords(FetchParams fetchParams, LogTablet logTablet) try { readContext.close(); } catch (Exception e) { - // ignore close exception + LOG.debug("Failed to close readContext for {}", tableBucket, e); } } } diff --git a/fluss-server/src/main/java/org/apache/fluss/server/utils/ServerRpcMessageUtils.java b/fluss-server/src/main/java/org/apache/fluss/server/utils/ServerRpcMessageUtils.java index bbd48016df..b6e007e259 100644 --- a/fluss-server/src/main/java/org/apache/fluss/server/utils/ServerRpcMessageUtils.java +++ b/fluss-server/src/main/java/org/apache/fluss/server/utils/ServerRpcMessageUtils.java @@ -201,6 +201,9 @@ import org.apache.fluss.utils.json.JsonSerdeUtils; import org.apache.fluss.utils.json.TableBucketOffsets; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + import javax.annotation.Nullable; import java.nio.ByteBuffer; @@ -222,7 +225,6 @@ import static org.apache.fluss.rpc.util.CommonRpcMessageUtils.toByteBuffer; import static org.apache.fluss.rpc.util.CommonRpcMessageUtils.toPbAclInfo; import static org.apache.fluss.utils.Preconditions.checkNotNull; -import static org.apache.fluss.utils.Preconditions.checkState; /** * Utils for making rpc request/response from inner object or convert inner class to rpc @@ -230,6 +232,8 @@ */ public class ServerRpcMessageUtils { + private static final Logger LOG = LoggerFactory.getLogger(ServerRpcMessageUtils.class); + public static TablePath toTablePath(PbTablePath pbTablePath) { return new TablePath(pbTablePath.getDatabaseName(), pbTablePath.getTableName()); } @@ -902,7 +906,7 @@ public static ProduceLogResponse makeProduceLogResponse( new FilterInfo(tableReq.getFilterPredicate(), schemaId)); } } - return result; + return result == null ? null : Collections.unmodifiableMap(result); } public static Map getFetchLogData(FetchLogRequest request) { @@ -952,13 +956,16 @@ public static FetchLogResponse makeFetchLogResponse( PbFetchLogRespForBucket fetchLogRespForBucket = new PbFetchLogRespForBucket().setBucketId(tb.getBucket()); if (bucketResult.hasFilteredEndOffset()) { - fetchLogRespForBucket.setFilteredEndOffset(bucketResult.getFilteredEndOffset()); // filteredEndOffset and records are mutually exclusive: when all batches are // filtered out, there should be no record data to send. - checkState( - bucketResult.recordsOrEmpty().sizeInBytes() == 0, - "filteredEndOffset is set but records are not empty for bucket %s", - tb); + if (bucketResult.recordsOrEmpty().sizeInBytes() != 0) { + LOG.warn( + "filteredEndOffset is set but records are not empty for bucket {}, " + + "clearing filteredEndOffset as fallback.", + tb); + } else { + fetchLogRespForBucket.setFilteredEndOffset(bucketResult.getFilteredEndOffset()); + } } if (tb.getPartitionId() != null) { fetchLogRespForBucket.setPartitionId(tb.getPartitionId()); diff --git a/fluss-server/src/test/java/org/apache/fluss/server/log/FetchParamsTest.java b/fluss-server/src/test/java/org/apache/fluss/server/log/FetchParamsTest.java index da3ea953d2..c8922d95c2 100644 --- a/fluss-server/src/test/java/org/apache/fluss/server/log/FetchParamsTest.java +++ b/fluss-server/src/test/java/org/apache/fluss/server/log/FetchParamsTest.java @@ -22,9 +22,12 @@ import org.apache.fluss.record.ProjectionPushdownCache; import org.apache.fluss.record.TestData; import org.apache.fluss.record.TestingSchemaGetter; +import org.apache.fluss.rpc.messages.PbPredicate; import org.junit.jupiter.api.Test; +import java.util.Collections; + import static org.apache.fluss.compression.ArrowCompressionInfo.DEFAULT_COMPRESSION; import static org.assertj.core.api.AssertionsForClassTypes.assertThat; @@ -92,4 +95,24 @@ void testBuilderDefaultsToLeaderOnly() { new FetchParamsBuilder(-1, 100).withFetchOnlyLeader(false).build(); assertThat(overrideFetchParams.fetchOnlyLeader()).isFalse(); } + + @Test + void testBuilderPropagatesFilterInfo() { + PbPredicate pbPredicate = new PbPredicate(); + FilterInfo filterInfo = new FilterInfo(pbPredicate, 42); + + FetchParams fetchParams = + new FetchParamsBuilder(-1, 100) + .withTableFilterInfoMap(Collections.singletonMap(1L, filterInfo)) + .build(); + + assertThat(fetchParams.getFilterInfo(1L)).isEqualTo(filterInfo); + assertThat(fetchParams.getFilterInfo(999L)).isNull(); + } + + @Test + void testBuilderWithoutFilterInfo() { + FetchParams fetchParams = new FetchParamsBuilder(-1, 100).build(); + assertThat(fetchParams.getFilterInfo(1L)).isNull(); + } }