From 43d1e5c54dd2fda8962bce648ca4589b8045f128 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?=E7=99=BD=E9=B5=BA?= Date: Tue, 31 Mar 2026 14:40:51 +0800 Subject: [PATCH] [server][client][flink] Add retract support for Aggregation Merge Engine --- .../client/table/writer/UpsertWriter.java | 16 + .../client/table/writer/UpsertWriterImpl.java | 35 + .../client/utils/ClientRpcMessageUtils.java | 1 - .../write/AbstractRowLogWriteBatch.java | 7 +- .../client/write/ArrowLogWriteBatch.java | 7 +- .../fluss/client/write/KvWriteBatch.java | 63 +- .../fluss/client/write/RecordAccumulator.java | 33 +- .../apache/fluss/client/write/WriteBatch.java | 16 +- .../fluss/client/write/WriteRecord.java | 82 +- .../utils/ClientRpcMessageUtilsTest.java | 1 + .../client/write/ArrowLogWriteBatchTest.java | 24 +- .../write/CompactedLogWriteBatchTest.java | 32 +- .../write/IndexedLogWriteBatchTest.java | 32 +- .../fluss/client/write/KvWriteBatchTest.java | 234 +++++- .../fluss/client/write/WriteRecordTest.java | 228 ++++++ .../apache/fluss/metadata/AggFunction.java | 10 + .../fluss/metadata/AggFunctionType.java | 24 + .../apache/fluss/record/DefaultKvRecord.java | 123 ++- .../fluss/record/DefaultKvRecordBatch.java | 32 +- .../org/apache/fluss/record/KvRecord.java | 9 + .../apache/fluss/record/KvRecordBatch.java | 8 + .../fluss/record/KvRecordBatchBuilder.java | 68 +- .../org/apache/fluss/record/MutationType.java | 63 ++ .../apache/fluss/rpc/protocol/MergeMode.java | 34 +- .../fluss/metadata/AggFunctionTypeTest.java | 39 + .../fluss/metadata/TableDescriptorTest.java | 25 + .../record/DefaultKvRecordBatchTest.java | 187 +++++ .../fluss/record/DefaultKvRecordTest.java | 115 ++- .../fluss/record/KvRecordTestUtils.java | 51 ++ .../apache/fluss/record/MutationTypeTest.java | 56 ++ .../fluss/rpc/protocol/MergeModeTest.java | 59 ++ .../flink/catalog/FlinkTableFactory.java | 13 +- .../apache/fluss/flink/row/OperationType.java | 20 +- .../apache/fluss/flink/sink/FlinkSink.java | 8 +- .../fluss/flink/sink/FlinkTableSink.java | 108 ++- .../fluss/flink/sink/FlussSinkBuilder.java | 3 +- .../RowDataSerializationSchema.java | 42 +- .../sink/serializer/SinkOperationMode.java | 95 +++ .../flink/sink/writer/UpsertSinkWriter.java | 62 +- .../fluss/flink/utils/FlinkConversions.java | 45 + .../apache/fluss/flink/row/RowWithOpTest.java | 20 + .../sink/FlinkRowDataChannelComputerTest.java | 3 +- .../flink/sink/FlinkTableSinkITCase.java | 73 ++ .../fluss/flink/sink/FlinkTableSinkTest.java | 133 +++ .../fluss/flink/sink/FlussSinkITCase.java | 14 +- .../fluss/flink/sink/UndoRecoveryITCase.java | 7 +- .../RowDataSerializationSchemaTest.java | 202 +++++ .../serializer/SinkOperationModeTest.java | 97 +++ .../shuffle/DataStatisticOperatorTest.java | 3 +- .../DataStatisticsCoordinatorTest.java | 3 +- ...StatisticsOrRecordChannelComputerTest.java | 3 +- .../flink/sink/undo/UndoComputerTest.java | 34 + .../sink/writer/FlinkSinkWriterTest.java | 3 +- .../sink/writer/UpsertSinkWriterTest.java | 109 +++ .../flink/utils/FlinkConversionsTest.java | 127 +++ .../fluss/flink/utils/TestUpsertWriter.java | 31 + .../apache/fluss/rpc/protocol/ApiKeys.java | 4 +- .../org/apache/fluss/server/kv/KvTablet.java | 439 +++++++++- .../kv/rowmerger/AggregateRowMerger.java | 159 +++- .../fluss/server/kv/rowmerger/RowMerger.java | 29 + .../aggregate/AggregateFieldsProcessor.java | 554 +++++++------ .../aggregate/functions/FieldAggregator.java | 16 + .../functions/FieldLastNonNullValueAgg.java | 17 + .../functions/FieldLastValueAgg.java | 16 + .../aggregate/functions/FieldProductAgg.java | 24 +- .../aggregate/functions/FieldSumAgg.java | 79 +- .../aggregate/functions/NarrowMathUtils.java | 89 ++ .../apache/fluss/server/replica/Replica.java | 6 +- .../fluss/server/replica/ReplicaManager.java | 6 +- .../fluss/server/tablet/TabletService.java | 7 +- .../server/kv/KvTabletMergeModeTest.java | 770 +++++++++++++++++- .../kv/rowmerger/AggregateRowMergerTest.java | 413 ++++++++++ .../kv/rowmerger/RowMergerCreateTest.java | 60 ++ .../FieldAggregatorParameterizedTest.java | 265 ++++++ .../server/replica/ReplicaManagerTest.java | 3 - .../fluss/server/replica/ReplicaTest.java | 3 +- .../table-design/merge-engines/aggregation.md | 35 +- 77 files changed, 5415 insertions(+), 551 deletions(-) create mode 100644 fluss-client/src/test/java/org/apache/fluss/client/write/WriteRecordTest.java create mode 100644 fluss-common/src/main/java/org/apache/fluss/record/MutationType.java create mode 100644 fluss-common/src/test/java/org/apache/fluss/metadata/AggFunctionTypeTest.java create mode 100644 fluss-common/src/test/java/org/apache/fluss/record/MutationTypeTest.java create mode 100644 fluss-common/src/test/java/org/apache/fluss/rpc/protocol/MergeModeTest.java create mode 100644 fluss-flink/fluss-flink-common/src/main/java/org/apache/fluss/flink/sink/serializer/SinkOperationMode.java create mode 100644 fluss-flink/fluss-flink-common/src/test/java/org/apache/fluss/flink/sink/FlinkTableSinkTest.java create mode 100644 fluss-flink/fluss-flink-common/src/test/java/org/apache/fluss/flink/sink/serializer/RowDataSerializationSchemaTest.java create mode 100644 fluss-flink/fluss-flink-common/src/test/java/org/apache/fluss/flink/sink/serializer/SinkOperationModeTest.java create mode 100644 fluss-flink/fluss-flink-common/src/test/java/org/apache/fluss/flink/sink/writer/UpsertSinkWriterTest.java create mode 100644 fluss-server/src/main/java/org/apache/fluss/server/kv/rowmerger/aggregate/functions/NarrowMathUtils.java diff --git a/fluss-client/src/main/java/org/apache/fluss/client/table/writer/UpsertWriter.java b/fluss-client/src/main/java/org/apache/fluss/client/table/writer/UpsertWriter.java index e4d751747d..84eb216030 100644 --- a/fluss-client/src/main/java/org/apache/fluss/client/table/writer/UpsertWriter.java +++ b/fluss-client/src/main/java/org/apache/fluss/client/table/writer/UpsertWriter.java @@ -38,6 +38,22 @@ public interface UpsertWriter extends TableWriter { */ CompletableFuture upsert(InternalRow record); + /** + * Retracts a previous aggregation contribution for the given row. The row must contain the + * primary key fields and the old aggregation values to retract. + * + *

This is used when a Flink aggregate operator emits UPDATE_BEFORE (retract old value) for a + * key. The retract record is sent independently with {@link + * org.apache.fluss.record.MutationType#RETRACT} mutation type. + * + * @param row the old aggregation value to retract (UPDATE_BEFORE). + * @return A {@link CompletableFuture} that returns upsert result when complete normally. + */ + default CompletableFuture retract(InternalRow row) { + throw new UnsupportedOperationException( + "retract() is not supported by this UpsertWriter implementation."); + } + /** * Delete a certain record from the Fluss table. The input must contain the primary key fields. * diff --git a/fluss-client/src/main/java/org/apache/fluss/client/table/writer/UpsertWriterImpl.java b/fluss-client/src/main/java/org/apache/fluss/client/table/writer/UpsertWriterImpl.java index 6b7f821a1e..ae5fa133a7 100644 --- a/fluss-client/src/main/java/org/apache/fluss/client/table/writer/UpsertWriterImpl.java +++ b/fluss-client/src/main/java/org/apache/fluss/client/table/writer/UpsertWriterImpl.java @@ -21,6 +21,7 @@ import org.apache.fluss.client.write.WriteRecord; import org.apache.fluss.client.write.WriterClient; import org.apache.fluss.metadata.KvFormat; +import org.apache.fluss.metadata.MergeEngineType; import org.apache.fluss.metadata.TableInfo; import org.apache.fluss.metadata.TablePath; import org.apache.fluss.row.BinaryRow; @@ -57,6 +58,9 @@ class UpsertWriterImpl extends AbstractTableWriter implements UpsertWriter { /** The merge mode for this writer. This controls how the server handles data merging. */ private final MergeMode mergeMode; + /** Whether this table supports retract (only AGGREGATION merge engine). */ + private final boolean supportsRetract; + UpsertWriterImpl( TablePath tablePath, TableInfo tableInfo, @@ -102,6 +106,12 @@ class UpsertWriterImpl extends AbstractTableWriter implements UpsertWriter { this.tableInfo = tableInfo; this.mergeMode = mergeMode; + this.supportsRetract = + tableInfo + .getTableConfig() + .getMergeEngineType() + .filter(t -> t == MergeEngineType.AGGREGATION) + .isPresent(); } private static void sanityCheck( @@ -217,6 +227,31 @@ public CompletableFuture delete(InternalRow row) { return sendWithResult(record, DeleteResult::new); } + @Override + public CompletableFuture retract(InternalRow row) { + if (!supportsRetract) { + throw new IllegalStateException( + "retract() is only supported for tables with AGGREGATION merge engine. " + + "Table: " + + tablePath); + } + checkFieldCount(row); + byte[] key = primaryKeyEncoder.encodeKey(row); + byte[] bucketKey = + bucketKeyEncoder == primaryKeyEncoder ? key : bucketKeyEncoder.encodeKey(row); + WriteRecord record = + WriteRecord.forRetract( + tableInfo, + getPhysicalPath(row), + encodeRow(row), + key, + bucketKey, + writeFormat, + targetColumns, + mergeMode); + return sendWithResult(record, UpsertResult::new); + } + private BinaryRow encodeRow(InternalRow row) { if (kvFormat == KvFormat.INDEXED && row instanceof IndexedRow) { return (IndexedRow) row; diff --git a/fluss-client/src/main/java/org/apache/fluss/client/utils/ClientRpcMessageUtils.java b/fluss-client/src/main/java/org/apache/fluss/client/utils/ClientRpcMessageUtils.java index 0bd67da17d..5c34cf8aa2 100644 --- a/fluss-client/src/main/java/org/apache/fluss/client/utils/ClientRpcMessageUtils.java +++ b/fluss-client/src/main/java/org/apache/fluss/client/utils/ClientRpcMessageUtils.java @@ -179,7 +179,6 @@ public static PutKvRequest makePutKvRequest( if (targetColumns != null) { request.setTargetColumns(targetColumns); } - // Set mergeMode in the request - this is the proper way to pass mergeMode to server request.setAggMode(mergeMode.getProtoValue()); readyWriteBatches.forEach( diff --git a/fluss-client/src/main/java/org/apache/fluss/client/write/AbstractRowLogWriteBatch.java b/fluss-client/src/main/java/org/apache/fluss/client/write/AbstractRowLogWriteBatch.java index 3c61cce82d..a66919334e 100644 --- a/fluss-client/src/main/java/org/apache/fluss/client/write/AbstractRowLogWriteBatch.java +++ b/fluss-client/src/main/java/org/apache/fluss/client/write/AbstractRowLogWriteBatch.java @@ -57,7 +57,8 @@ protected AbstractRowLogWriteBatch( } @Override - public boolean tryAppend(WriteRecord writeRecord, WriteCallback callback) throws Exception { + public AppendResult tryAppend(WriteRecord writeRecord, WriteCallback callback) + throws Exception { checkNotNull(callback, "write callback must be not null"); InternalRow rowObj = writeRecord.getRow(); checkNotNull(rowObj, "row must not be null for log record"); @@ -68,12 +69,12 @@ public boolean tryAppend(WriteRecord writeRecord, WriteCallback callback) throws R row = requireAndCastRow(rowObj); if (!recordsBuilder.hasRoomFor(row) || isClosed()) { - return false; + return AppendResult.BATCH_FULL; } recordsBuilder.append(ChangeType.APPEND_ONLY, row); recordCount++; callbacks.add(callback); - return true; + return AppendResult.APPENDED; } protected abstract R requireAndCastRow(InternalRow row); diff --git a/fluss-client/src/main/java/org/apache/fluss/client/write/ArrowLogWriteBatch.java b/fluss-client/src/main/java/org/apache/fluss/client/write/ArrowLogWriteBatch.java index bee407cf07..f252c8de90 100644 --- a/fluss-client/src/main/java/org/apache/fluss/client/write/ArrowLogWriteBatch.java +++ b/fluss-client/src/main/java/org/apache/fluss/client/write/ArrowLogWriteBatch.java @@ -72,7 +72,8 @@ public boolean isLogBatch() { } @Override - public boolean tryAppend(WriteRecord writeRecord, WriteCallback callback) throws Exception { + public AppendResult tryAppend(WriteRecord writeRecord, WriteCallback callback) + throws Exception { InternalRow row = writeRecord.getRow(); checkArgument( writeRecord.getTargetColumns() == null, @@ -81,12 +82,12 @@ public boolean tryAppend(WriteRecord writeRecord, WriteCallback callback) throws checkNotNull(row != null, "row must not be null for log record"); checkNotNull(callback, "write callback must be not null"); if (recordsBuilder.isClosed() || recordsBuilder.isFull()) { - return false; + return AppendResult.BATCH_FULL; } else { recordsBuilder.append(ChangeType.APPEND_ONLY, row); recordCount++; callbacks.add(callback); - return true; + return AppendResult.APPENDED; } } diff --git a/fluss-client/src/main/java/org/apache/fluss/client/write/KvWriteBatch.java b/fluss-client/src/main/java/org/apache/fluss/client/write/KvWriteBatch.java index a72d36717a..8924b5e50f 100644 --- a/fluss-client/src/main/java/org/apache/fluss/client/write/KvWriteBatch.java +++ b/fluss-client/src/main/java/org/apache/fluss/client/write/KvWriteBatch.java @@ -24,6 +24,7 @@ import org.apache.fluss.metadata.KvFormat; import org.apache.fluss.metadata.PhysicalTablePath; import org.apache.fluss.record.KvRecordBatchBuilder; +import org.apache.fluss.record.MutationType; import org.apache.fluss.record.bytesview.BytesView; import org.apache.fluss.row.BinaryRow; import org.apache.fluss.row.InternalRow; @@ -53,6 +54,7 @@ public class KvWriteBatch extends WriteBatch { private final @Nullable int[] targetColumns; private final int schemaId; private final MergeMode mergeMode; + private final boolean v2Format; public KvWriteBatch( int bucketId, @@ -63,14 +65,16 @@ public KvWriteBatch( AbstractPagedOutputView outputView, @Nullable int[] targetColumns, MergeMode mergeMode, + boolean v2Format, long createdMs) { super(bucketId, physicalTablePath, createdMs); this.outputView = outputView; this.recordsBuilder = - KvRecordBatchBuilder.builder(schemaId, writeLimit, outputView, kvFormat); + KvRecordBatchBuilder.builder(schemaId, writeLimit, outputView, kvFormat, v2Format); this.targetColumns = targetColumns; this.schemaId = schemaId; this.mergeMode = mergeMode; + this.v2Format = v2Format; } @Override @@ -79,17 +83,48 @@ public boolean isLogBatch() { } @Override - public boolean tryAppend(WriteRecord writeRecord, WriteCallback callback) throws Exception { + public AppendResult tryAppend(WriteRecord writeRecord, WriteCallback callback) + throws Exception { + validateRecordConsistency(writeRecord); + + // V0 batch cannot carry RETRACT records — caller must create a new V2 batch. + if (!v2Format && writeRecord.getMutationType() == MutationType.RETRACT) { + return AppendResult.FORMAT_MISMATCH; + } + + byte[] key = writeRecord.getKey(); + checkNotNull(key, "key must be not null for kv record"); + checkNotNull(callback, "write callback must be not null"); + BinaryRow row = checkRow(writeRecord.getRow()); + + if (v2Format) { + if (!recordsBuilder.hasRoomForV2(key, row) || isClosed()) { + return AppendResult.BATCH_FULL; + } else { + recordsBuilder.appendV2(writeRecord.getMutationType(), key, row); + callbacks.add(callback); + recordCount++; + return AppendResult.APPENDED; + } + } else { + if (!recordsBuilder.hasRoomFor(key, row) || isClosed()) { + return AppendResult.BATCH_FULL; + } else { + recordsBuilder.append(key, row); + callbacks.add(callback); + recordCount++; + return AppendResult.APPENDED; + } + } + } + + private void validateRecordConsistency(WriteRecord writeRecord) { if (schemaId != writeRecord.getSchemaId()) { throw new IllegalStateException( String.format( "schema id %d of the write record to append is not the same as the current schema id %d in the batch.", writeRecord.getSchemaId(), schemaId)); } - - // currently, we throw exception directly when the target columns of the write record is - // not the same as the current target columns in the batch. - // this should be quite fast as they should be the same objects. if (!Arrays.equals(targetColumns, writeRecord.getTargetColumns())) { throw new IllegalStateException( String.format( @@ -97,9 +132,6 @@ public boolean tryAppend(WriteRecord writeRecord, WriteCallback callback) throws Arrays.toString(writeRecord.getTargetColumns()), Arrays.toString(targetColumns))); } - - // Validate mergeMode consistency - records with different mergeMode cannot be batched - // together if (writeRecord.getMergeMode() != this.mergeMode) { throw new IllegalStateException( String.format( @@ -107,19 +139,6 @@ public boolean tryAppend(WriteRecord writeRecord, WriteCallback callback) throws + "Batch mergeMode: %s, Record mergeMode: %s", this.mergeMode, writeRecord.getMergeMode())); } - - byte[] key = writeRecord.getKey(); - checkNotNull(key, "key must be not null for kv record"); - checkNotNull(callback, "write callback must be not null"); - BinaryRow row = checkRow(writeRecord.getRow()); - if (!recordsBuilder.hasRoomFor(key, row) || isClosed()) { - return false; - } else { - recordsBuilder.append(key, row); - callbacks.add(callback); - recordCount++; - return true; - } } @Nullable diff --git a/fluss-client/src/main/java/org/apache/fluss/client/write/RecordAccumulator.java b/fluss-client/src/main/java/org/apache/fluss/client/write/RecordAccumulator.java index 991ea7c8ea..f5c0179a93 100644 --- a/fluss-client/src/main/java/org/apache/fluss/client/write/RecordAccumulator.java +++ b/fluss-client/src/main/java/org/apache/fluss/client/write/RecordAccumulator.java @@ -28,6 +28,7 @@ import org.apache.fluss.memory.LazyMemorySegmentPool; import org.apache.fluss.memory.MemorySegment; import org.apache.fluss.memory.PreAllocatedPagedOutputView; +import org.apache.fluss.metadata.MergeEngineType; import org.apache.fluss.metadata.PhysicalTablePath; import org.apache.fluss.metadata.TableBucket; import org.apache.fluss.metadata.TableInfo; @@ -592,7 +593,14 @@ private RecordAppendResult appendNewBatch( outputView, schemaId); - batch.tryAppend(writeRecord, callback); + WriteBatch.AppendResult appendResult2 = batch.tryAppend(writeRecord, callback); + if (appendResult2 != WriteBatch.AppendResult.APPENDED) { + batch.close(); + throw new FlussRuntimeException( + "Failed to append record to a freshly created batch (result=" + + appendResult2 + + "). The record may be too large for the configured batch size."); + } deque.addLast(batch); incomplete.add(batch); return new RecordAppendResult(deque.size() > 1 || batch.isClosed(), true, false); @@ -619,6 +627,11 @@ private WriteBatch createWriteBatch( outputView, writeRecord.getTargetColumns(), writeRecord.getMergeMode(), + tableInfo + .getTableConfig() + .getMergeEngineType() + .filter(t -> t == MergeEngineType.AGGREGATION) + .isPresent(), clock.milliseconds()); case ARROW_LOG: @@ -675,17 +688,17 @@ private RecordAppendResult tryAppend( } WriteBatch last = deque.peekLast(); if (last != null) { - boolean success = last.tryAppend(writeRecord, callback); - if (!success) { - // TODO For ArrowLogWriteBatch, close here is a heavy operation (including build - // logic), we need to avoid do that in an lock which locked dq. However, why we not - // remove build logic out of close for ArrowLogWriteBatch is that we want to release - // non-heap memory hold by arrowWriter as soon as possible to avoid OOM. Maybe we - // need to introduce a more reasonable way to solve these two problems. - last.close(); - } else { + WriteBatch.AppendResult result = last.tryAppend(writeRecord, callback); + if (result == WriteBatch.AppendResult.APPENDED) { return new RecordAppendResult(deque.size() > 1 || last.isClosed(), false, false); } + // BATCH_FULL or FORMAT_MISMATCH — close the current batch so a new one is created. + // TODO For ArrowLogWriteBatch, close here is a heavy operation (including build + // logic), we need to avoid do that in an lock which locked dq. However, why we not + // remove build logic out of close for ArrowLogWriteBatch is that we want to release + // non-heap memory hold by arrowWriter as soon as possible to avoid OOM. Maybe we + // need to introduce a more reasonable way to solve these two problems. + last.close(); } return null; } diff --git a/fluss-client/src/main/java/org/apache/fluss/client/write/WriteBatch.java b/fluss-client/src/main/java/org/apache/fluss/client/write/WriteBatch.java index cb83943cfc..51c9cd83d4 100644 --- a/fluss-client/src/main/java/org/apache/fluss/client/write/WriteBatch.java +++ b/fluss-client/src/main/java/org/apache/fluss/client/write/WriteBatch.java @@ -71,14 +71,24 @@ public WriteBatch(int bucketId, PhysicalTablePath physicalTablePath, long create */ public abstract boolean isLogBatch(); + /** Result of a {@link #tryAppend} call. */ + public enum AppendResult { + /** Record was successfully appended to the batch. */ + APPENDED, + /** Batch has no room for the record or is closed. */ + BATCH_FULL, + /** Batch format does not support this record's mutation type (e.g. V0 batch + RETRACT). */ + FORMAT_MISMATCH + } + /** - * try to append one write record to the record batch. + * Try to append one write record to the record batch. * * @param writeRecord the record to write * @param callback the callback to send back to writer - * @return true if append success, false if the batch is full. + * @return the result of the append attempt */ - public abstract boolean tryAppend(WriteRecord writeRecord, WriteCallback callback) + public abstract AppendResult tryAppend(WriteRecord writeRecord, WriteCallback callback) throws Exception; /** diff --git a/fluss-client/src/main/java/org/apache/fluss/client/write/WriteRecord.java b/fluss-client/src/main/java/org/apache/fluss/client/write/WriteRecord.java index f25562be72..f102806a8c 100644 --- a/fluss-client/src/main/java/org/apache/fluss/client/write/WriteRecord.java +++ b/fluss-client/src/main/java/org/apache/fluss/client/write/WriteRecord.java @@ -23,6 +23,7 @@ import org.apache.fluss.record.CompactedLogRecord; import org.apache.fluss.record.DefaultKvRecord; import org.apache.fluss.record.IndexedLogRecord; +import org.apache.fluss.record.MutationType; import org.apache.fluss.row.BinaryRow; import org.apache.fluss.row.InternalRow; import org.apache.fluss.row.compacted.CompactedRow; @@ -88,7 +89,39 @@ public static WriteRecord forUpsert( writeFormat, targetColumns, estimatedSizeInBytes, - mergeMode); + mergeMode, + MutationType.UPSERT); + } + + /** + * Create a write record for retract operation. Same shape as upsert but carries {@link + * MutationType#RETRACT} so the server knows to retract the old aggregation value. + */ + public static WriteRecord forRetract( + TableInfo tableInfo, + PhysicalTablePath tablePath, + BinaryRow row, + byte[] key, + byte[] bucketKey, + WriteFormat writeFormat, + @Nullable int[] targetColumns, + MergeMode mergeMode) { + checkNotNull(row, "row must not be null"); + checkNotNull(key, "key must not be null"); + checkNotNull(bucketKey, "bucketKey must not be null"); + checkArgument(writeFormat.isKv(), "writeFormat must be a KV format"); + int estimatedSizeInBytes = DefaultKvRecord.sizeOfV2(key, row) + RECORD_BATCH_HEADER_SIZE; + return new WriteRecord( + tableInfo, + tablePath, + key, + bucketKey, + row, + writeFormat, + targetColumns, + estimatedSizeInBytes, + mergeMode, + MutationType.RETRACT); } /** Create a write record for delete operation and partial-delete update. */ @@ -119,7 +152,7 @@ public static WriteRecord forDelete( @Nullable int[] targetColumns, MergeMode mergeMode) { checkNotNull(key, "key must not be null"); - checkNotNull(bucketKey, "key must not be null"); + checkNotNull(bucketKey, "bucketKey must not be null"); checkArgument(writeFormat.isKv(), "writeFormat must be a KV format"); int estimatedSizeInBytes = DefaultKvRecord.sizeOf(key, null) + RECORD_BATCH_HEADER_SIZE; return new WriteRecord( @@ -131,7 +164,8 @@ public static WriteRecord forDelete( writeFormat, targetColumns, estimatedSizeInBytes, - mergeMode); + mergeMode, + MutationType.DELETE); } /** Create a write record for append operation for indexed format. */ @@ -152,7 +186,8 @@ public static WriteRecord forIndexedAppend( WriteFormat.INDEXED_LOG, null, estimatedSizeInBytes, - MergeMode.DEFAULT); + MergeMode.DEFAULT, + MutationType.UPSERT); } /** Creates a write record for append operation for Arrow format. */ @@ -174,7 +209,8 @@ public static WriteRecord forArrowAppend( WriteFormat.ARROW_LOG, null, estimatedSizeInBytes, - MergeMode.DEFAULT); + MergeMode.DEFAULT, + MutationType.UPSERT); } /** Creates a write record for append operation for Compacted format. */ @@ -195,7 +231,8 @@ public static WriteRecord forCompactedAppend( WriteFormat.COMPACTED_LOG, null, estimatedSizeInBytes, - MergeMode.DEFAULT); + MergeMode.DEFAULT, + MutationType.UPSERT); } // ------------------------------------------------------------------------------------------ @@ -213,15 +250,13 @@ public static WriteRecord forCompactedAppend( private final TableInfo tableInfo; /** - * The merge mode for this record. This controls how the server handles data merging. - * - *

+ * The merge mode for this record: DEFAULT (normal merge) or OVERWRITE (bypass merge engine). */ private final MergeMode mergeMode; + /** The per-record mutation type: UPSERT, DELETE, or RETRACT. */ + private final MutationType mutationType; + private WriteRecord( TableInfo tableInfo, PhysicalTablePath physicalTablePath, @@ -231,7 +266,8 @@ private WriteRecord( WriteFormat writeFormat, @Nullable int[] targetColumns, int estimatedSizeInBytes, - MergeMode mergeMode) { + MergeMode mergeMode, + MutationType mutationType) { this.tableInfo = tableInfo; this.physicalTablePath = physicalTablePath; this.key = key; @@ -241,6 +277,7 @@ private WriteRecord( this.targetColumns = targetColumns; this.estimatedSizeInBytes = estimatedSizeInBytes; this.mergeMode = mergeMode; + this.mutationType = mutationType; } public PhysicalTablePath getPhysicalTablePath() { @@ -272,22 +309,17 @@ public WriteFormat getWriteFormat() { return writeFormat; } - /** - * Get the merge mode for this record. - * - * @return the merge mode - */ + /** Get the merge mode for this record. */ public MergeMode getMergeMode() { return mergeMode; } - /** - * Get the estimated size in bytes of the record with batch header. - * - * @return the estimated size in bytes of the record with batch header - * @throws IllegalStateException if the estimated size in bytes is not supported for the write - * format - */ + /** Get the per-record mutation type (UPSERT, DELETE, or RETRACT). */ + public MutationType getMutationType() { + return mutationType; + } + + /** Returns the estimated size in bytes of the record with batch header. */ public int getEstimatedSizeInBytes() { if (estimatedSizeInBytes < 0) { throw new IllegalStateException( diff --git a/fluss-client/src/test/java/org/apache/fluss/client/utils/ClientRpcMessageUtilsTest.java b/fluss-client/src/test/java/org/apache/fluss/client/utils/ClientRpcMessageUtilsTest.java index 5b1ca58c71..8f200d830f 100644 --- a/fluss-client/src/test/java/org/apache/fluss/client/utils/ClientRpcMessageUtilsTest.java +++ b/fluss-client/src/test/java/org/apache/fluss/client/utils/ClientRpcMessageUtilsTest.java @@ -139,6 +139,7 @@ private KvWriteBatch createKvWriteBatch(int bucketId, MergeMode mergeMode) throw outputView, null, mergeMode, + false, System.currentTimeMillis()); } } diff --git a/fluss-client/src/test/java/org/apache/fluss/client/write/ArrowLogWriteBatchTest.java b/fluss-client/src/test/java/org/apache/fluss/client/write/ArrowLogWriteBatchTest.java index 07c4aa3383..3a3369b612 100644 --- a/fluss-client/src/test/java/org/apache/fluss/client/write/ArrowLogWriteBatchTest.java +++ b/fluss-client/src/test/java/org/apache/fluss/client/write/ArrowLogWriteBatchTest.java @@ -43,6 +43,7 @@ import java.util.List; import java.util.concurrent.CompletableFuture; +import static org.apache.fluss.client.write.WriteBatch.AppendResult; import static org.apache.fluss.compression.ArrowCompressionInfo.DEFAULT_COMPRESSION; import static org.apache.fluss.record.LogRecordReadContext.createArrowReadContext; import static org.apache.fluss.record.TestData.DATA1_PHYSICAL_TABLE_PATH; @@ -80,14 +81,15 @@ void testAppend() throws Exception { createArrowLogWriteBatch(new TableBucket(DATA1_TABLE_ID, bucketId), maxSizeInBytes); int count = 0; while (arrowLogWriteBatch.tryAppend( - createWriteRecord(row(count, "a" + count)), newWriteCallback())) { + createWriteRecord(row(count, "a" + count)), newWriteCallback()) + == AppendResult.APPENDED) { count++; } // batch full. - boolean appendResult = + AppendResult appendResult = arrowLogWriteBatch.tryAppend(createWriteRecord(row(1, "a")), newWriteCallback()); - assertThat(appendResult).isFalse(); + assertThat(appendResult).isEqualTo(AppendResult.BATCH_FULL); // close this batch. arrowLogWriteBatch.close(); @@ -142,14 +144,15 @@ void testAppendWithPreAllocatedMemorySegments() throws Exception { int count = 0; while (arrowLogWriteBatch.tryAppend( - createWriteRecord(row(count, "a" + count)), newWriteCallback())) { + createWriteRecord(row(count, "a" + count)), newWriteCallback()) + == AppendResult.APPENDED) { count++; } // batch full. - boolean appendResult = + AppendResult appendResult = arrowLogWriteBatch.tryAppend(createWriteRecord(row(1, "a")), newWriteCallback()); - assertThat(appendResult).isFalse(); + assertThat(appendResult).isEqualTo(AppendResult.BATCH_FULL); // close this batch. arrowLogWriteBatch.close(); @@ -216,16 +219,17 @@ void testArrowCompressionRatioEstimated() throws Exception { int recordCount = 0; while (arrowLogWriteBatch.tryAppend( - createWriteRecord(row(recordCount, RandomStringUtils.random(100))), - newWriteCallback())) { + createWriteRecord(row(recordCount, RandomStringUtils.random(100))), + newWriteCallback()) + == AppendResult.APPENDED) { recordCount++; } // batch full. - boolean appendResult = + AppendResult appendResult = arrowLogWriteBatch.tryAppend( createWriteRecord(row(1, "a")), newWriteCallback()); - assertThat(appendResult).isFalse(); + assertThat(appendResult).isEqualTo(AppendResult.BATCH_FULL); // close this batch and recycle the writer. arrowLogWriteBatch.close(); diff --git a/fluss-client/src/test/java/org/apache/fluss/client/write/CompactedLogWriteBatchTest.java b/fluss-client/src/test/java/org/apache/fluss/client/write/CompactedLogWriteBatchTest.java index 4f8b379193..30cea5ba59 100644 --- a/fluss-client/src/test/java/org/apache/fluss/client/write/CompactedLogWriteBatchTest.java +++ b/fluss-client/src/test/java/org/apache/fluss/client/write/CompactedLogWriteBatchTest.java @@ -39,6 +39,7 @@ import java.util.List; import java.util.concurrent.CompletableFuture; +import static org.apache.fluss.client.write.WriteBatch.AppendResult; import static org.apache.fluss.record.LogRecordBatch.CURRENT_LOG_MAGIC_VALUE; import static org.apache.fluss.record.LogRecordBatchFormat.recordBatchHeaderSize; import static org.apache.fluss.record.TestData.DATA1_PHYSICAL_TABLE_PATH; @@ -75,14 +76,15 @@ void testTryAppendWithWriteLimit() throws Exception { (writeLimit - recordBatchHeaderSize(CURRENT_LOG_MAGIC_VALUE)) / estimatedSizeInBytes; for (int i = 0; i < maxRecordsPerBatch; i++) { - boolean appendResult = + AppendResult appendResult = logProducerBatch.tryAppend(createWriteRecord(), newWriteCallback()); - assertThat(appendResult).isTrue(); + assertThat(appendResult).isEqualTo(AppendResult.APPENDED); } // batch full. - boolean appendResult = logProducerBatch.tryAppend(createWriteRecord(), newWriteCallback()); - assertThat(appendResult).isFalse(); + AppendResult appendResult = + logProducerBatch.tryAppend(createWriteRecord(), newWriteCallback()); + assertThat(appendResult).isEqualTo(AppendResult.BATCH_FULL); } @Test @@ -90,8 +92,9 @@ void testToBytes() throws Exception { int bucketId = 0; CompactedLogWriteBatch logProducerBatch = createLogWriteBatch(new TableBucket(DATA1_TABLE_ID, bucketId), 0L); - boolean appendResult = logProducerBatch.tryAppend(createWriteRecord(), newWriteCallback()); - assertThat(appendResult).isTrue(); + AppendResult appendResult = + logProducerBatch.tryAppend(createWriteRecord(), newWriteCallback()); + assertThat(appendResult).isEqualTo(AppendResult.APPENDED); logProducerBatch.close(); BytesView bytesView = logProducerBatch.build(); @@ -106,8 +109,9 @@ void testCompleteTwice() throws Exception { int bucketId = 0; CompactedLogWriteBatch logWriteBatch = createLogWriteBatch(new TableBucket(DATA1_TABLE_ID, bucketId), 0L); - boolean appendResult = logWriteBatch.tryAppend(createWriteRecord(), newWriteCallback()); - assertThat(appendResult).isTrue(); + AppendResult appendResult = + logWriteBatch.tryAppend(createWriteRecord(), newWriteCallback()); + assertThat(appendResult).isEqualTo(AppendResult.APPENDED); assertThat(logWriteBatch.complete()).isTrue(); assertThatThrownBy(logWriteBatch::complete) @@ -121,8 +125,9 @@ void testFailedTwice() throws Exception { int bucketId = 0; CompactedLogWriteBatch logWriteBatch = createLogWriteBatch(new TableBucket(DATA1_TABLE_ID, bucketId), 0L); - boolean appendResult = logWriteBatch.tryAppend(createWriteRecord(), newWriteCallback()); - assertThat(appendResult).isTrue(); + AppendResult appendResult = + logWriteBatch.tryAppend(createWriteRecord(), newWriteCallback()); + assertThat(appendResult).isEqualTo(AppendResult.APPENDED); assertThat(logWriteBatch.completeExceptionally(new IllegalStateException("test failed."))) .isTrue(); @@ -136,14 +141,15 @@ void testClose() throws Exception { int bucketId = 0; CompactedLogWriteBatch logProducerBatch = createLogWriteBatch(new TableBucket(DATA1_TABLE_ID, bucketId), 0L); - boolean appendResult = logProducerBatch.tryAppend(createWriteRecord(), newWriteCallback()); - assertThat(appendResult).isTrue(); + AppendResult appendResult = + logProducerBatch.tryAppend(createWriteRecord(), newWriteCallback()); + assertThat(appendResult).isEqualTo(AppendResult.APPENDED); logProducerBatch.close(); assertThat(logProducerBatch.isClosed()).isTrue(); appendResult = logProducerBatch.tryAppend(createWriteRecord(), newWriteCallback()); - assertThat(appendResult).isFalse(); + assertThat(appendResult).isEqualTo(AppendResult.BATCH_FULL); } @Test diff --git a/fluss-client/src/test/java/org/apache/fluss/client/write/IndexedLogWriteBatchTest.java b/fluss-client/src/test/java/org/apache/fluss/client/write/IndexedLogWriteBatchTest.java index 683013b893..200eb35dd8 100644 --- a/fluss-client/src/test/java/org/apache/fluss/client/write/IndexedLogWriteBatchTest.java +++ b/fluss-client/src/test/java/org/apache/fluss/client/write/IndexedLogWriteBatchTest.java @@ -41,6 +41,7 @@ import java.util.List; import java.util.concurrent.CompletableFuture; +import static org.apache.fluss.client.write.WriteBatch.AppendResult; import static org.apache.fluss.record.LogRecordBatch.CURRENT_LOG_MAGIC_VALUE; import static org.apache.fluss.record.LogRecordBatchFormat.recordBatchHeaderSize; import static org.apache.fluss.record.TestData.DATA1_PHYSICAL_TABLE_PATH; @@ -78,14 +79,15 @@ void testTryAppendWithWriteLimit() throws Exception { (writeLimit - recordBatchHeaderSize(CURRENT_LOG_MAGIC_VALUE)) / estimatedSizeInBytes; for (int i = 0; i < maxRecordsPerBatch; i++) { - boolean appendResult = + AppendResult appendResult = logProducerBatch.tryAppend(createWriteRecord(), newWriteCallback()); - assertThat(appendResult).isTrue(); + assertThat(appendResult).isEqualTo(AppendResult.APPENDED); } // batch full. - boolean appendResult = logProducerBatch.tryAppend(createWriteRecord(), newWriteCallback()); - assertThat(appendResult).isFalse(); + AppendResult appendResult = + logProducerBatch.tryAppend(createWriteRecord(), newWriteCallback()); + assertThat(appendResult).isEqualTo(AppendResult.BATCH_FULL); } @Test @@ -93,8 +95,9 @@ void testToBytes() throws Exception { int bucketId = 0; IndexedLogWriteBatch logProducerBatch = createLogWriteBatch(new TableBucket(DATA1_TABLE_ID, bucketId), 0L); - boolean appendResult = logProducerBatch.tryAppend(createWriteRecord(), newWriteCallback()); - assertThat(appendResult).isTrue(); + AppendResult appendResult = + logProducerBatch.tryAppend(createWriteRecord(), newWriteCallback()); + assertThat(appendResult).isEqualTo(AppendResult.APPENDED); logProducerBatch.close(); BytesView bytesView = logProducerBatch.build(); @@ -110,8 +113,9 @@ void testCompleteTwice() throws Exception { int bucketId = 0; IndexedLogWriteBatch logWriteBatch = createLogWriteBatch(new TableBucket(DATA1_TABLE_ID, bucketId), 0L); - boolean appendResult = logWriteBatch.tryAppend(createWriteRecord(), newWriteCallback()); - assertThat(appendResult).isTrue(); + AppendResult appendResult = + logWriteBatch.tryAppend(createWriteRecord(), newWriteCallback()); + assertThat(appendResult).isEqualTo(AppendResult.APPENDED); assertThat(logWriteBatch.complete()).isTrue(); assertThatThrownBy(logWriteBatch::complete) @@ -125,8 +129,9 @@ void testFailedTwice() throws Exception { int bucketId = 0; IndexedLogWriteBatch logWriteBatch = createLogWriteBatch(new TableBucket(DATA1_TABLE_ID, bucketId), 0L); - boolean appendResult = logWriteBatch.tryAppend(createWriteRecord(), newWriteCallback()); - assertThat(appendResult).isTrue(); + AppendResult appendResult = + logWriteBatch.tryAppend(createWriteRecord(), newWriteCallback()); + assertThat(appendResult).isEqualTo(AppendResult.APPENDED); assertThat(logWriteBatch.completeExceptionally(new IllegalStateException("test failed."))) .isTrue(); @@ -140,14 +145,15 @@ void testClose() throws Exception { int bucketId = 0; IndexedLogWriteBatch logProducerBatch = createLogWriteBatch(new TableBucket(DATA1_TABLE_ID, bucketId), 0L); - boolean appendResult = logProducerBatch.tryAppend(createWriteRecord(), newWriteCallback()); - assertThat(appendResult).isTrue(); + AppendResult appendResult = + logProducerBatch.tryAppend(createWriteRecord(), newWriteCallback()); + assertThat(appendResult).isEqualTo(AppendResult.APPENDED); logProducerBatch.close(); assertThat(logProducerBatch.isClosed()).isTrue(); appendResult = logProducerBatch.tryAppend(createWriteRecord(), newWriteCallback()); - assertThat(appendResult).isFalse(); + assertThat(appendResult).isEqualTo(AppendResult.BATCH_FULL); } @Test diff --git a/fluss-client/src/test/java/org/apache/fluss/client/write/KvWriteBatchTest.java b/fluss-client/src/test/java/org/apache/fluss/client/write/KvWriteBatchTest.java index e2340d1709..e3e42195d8 100644 --- a/fluss-client/src/test/java/org/apache/fluss/client/write/KvWriteBatchTest.java +++ b/fluss-client/src/test/java/org/apache/fluss/client/write/KvWriteBatchTest.java @@ -30,6 +30,7 @@ import org.apache.fluss.record.DefaultKvRecordBatch; import org.apache.fluss.record.KvRecord; import org.apache.fluss.record.KvRecordReadContext; +import org.apache.fluss.record.MutationType; import org.apache.fluss.record.TestingSchemaGetter; import org.apache.fluss.row.BinaryRow; import org.apache.fluss.row.encode.CompactedKeyEncoder; @@ -45,6 +46,7 @@ import java.util.List; import java.util.concurrent.CompletableFuture; +import static org.apache.fluss.client.write.WriteBatch.AppendResult; 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.DATA1_SCHEMA_PK; @@ -78,35 +80,35 @@ void setup() { @Test void testTryAppendWithWriteLimit() throws Exception { - int writeLimit = 100; + // Use a write limit that fits exactly N records with no leftover space + int headerSize = DefaultKvRecordBatch.RECORD_BATCH_HEADER_SIZE; + int numRecords = 5; + int writeLimit = headerSize + numRecords * estimatedSizeInBytes; KvWriteBatch kvProducerBatch = createKvWriteBatch( new TableBucket(DATA1_TABLE_ID_PK, 0), writeLimit, MemorySegment.allocateHeapMemory(writeLimit)); - for (int i = 0; - i - < (writeLimit - DefaultKvRecordBatch.RECORD_BATCH_HEADER_SIZE) - / estimatedSizeInBytes; - i++) { - boolean appendResult = + for (int i = 0; i < numRecords; i++) { + AppendResult appendResult = kvProducerBatch.tryAppend(createWriteRecord(), newWriteCallback()); - assertThat(appendResult).isTrue(); + assertThat(appendResult).isEqualTo(AppendResult.APPENDED); } // batch full. - boolean appendResult = kvProducerBatch.tryAppend(createWriteRecord(), newWriteCallback()); - assertThat(appendResult).isFalse(); + AppendResult appendResult = + kvProducerBatch.tryAppend(createWriteRecord(), newWriteCallback()); + assertThat(appendResult).isEqualTo(AppendResult.BATCH_FULL); } @Test void testToBytes() throws Exception { KvWriteBatch kvProducerBatch = createKvWriteBatch(new TableBucket(DATA1_TABLE_ID_PK, 0)); - boolean appendResult = kvProducerBatch.tryAppend(createWriteRecord(), newWriteCallback()); - assertThat(appendResult).isTrue(); + assertThat(kvProducerBatch.tryAppend(createWriteRecord(), newWriteCallback())) + .isEqualTo(AppendResult.APPENDED); DefaultKvRecordBatch kvRecords = DefaultKvRecordBatch.pointToBytesView(kvProducerBatch.build()); assertDefaultKvRecordBatchEquals(kvRecords); @@ -116,8 +118,8 @@ void testToBytes() throws Exception { void testCompleteTwice() throws Exception { KvWriteBatch kvWriteBatch = createKvWriteBatch(new TableBucket(DATA1_TABLE_ID_PK, 0)); - boolean appendResult = kvWriteBatch.tryAppend(createWriteRecord(), newWriteCallback()); - assertThat(appendResult).isTrue(); + assertThat(kvWriteBatch.tryAppend(createWriteRecord(), newWriteCallback())) + .isEqualTo(AppendResult.APPENDED); assertThat(kvWriteBatch.complete()).isTrue(); assertThatThrownBy(kvWriteBatch::complete) @@ -130,8 +132,8 @@ void testCompleteTwice() throws Exception { void testFailedTwice() throws Exception { KvWriteBatch kvWriteBatch = createKvWriteBatch(new TableBucket(DATA1_TABLE_ID_PK, 0)); - boolean appendResult = kvWriteBatch.tryAppend(createWriteRecord(), newWriteCallback()); - assertThat(appendResult).isTrue(); + assertThat(kvWriteBatch.tryAppend(createWriteRecord(), newWriteCallback())) + .isEqualTo(AppendResult.APPENDED); assertThat(kvWriteBatch.completeExceptionally(new IllegalStateException("test failed."))) .isTrue(); @@ -143,14 +145,14 @@ void testFailedTwice() throws Exception { @Test void testClose() throws Exception { KvWriteBatch kvProducerBatch = createKvWriteBatch(new TableBucket(DATA1_TABLE_ID_PK, 0)); - boolean appendResult = kvProducerBatch.tryAppend(createWriteRecord(), newWriteCallback()); - assertThat(appendResult).isTrue(); + assertThat(kvProducerBatch.tryAppend(createWriteRecord(), newWriteCallback())) + .isEqualTo(AppendResult.APPENDED); kvProducerBatch.close(); assertThat(kvProducerBatch.isClosed()).isTrue(); - appendResult = kvProducerBatch.tryAppend(createWriteRecord(), newWriteCallback()); - assertThat(appendResult).isFalse(); + assertThat(kvProducerBatch.tryAppend(createWriteRecord(), newWriteCallback())) + .isEqualTo(AppendResult.BATCH_FULL); } @Test @@ -229,6 +231,7 @@ private KvWriteBatch createKvWriteBatch( outputView, null, MergeMode.DEFAULT, + false, System.currentTimeMillis()); } @@ -268,7 +271,8 @@ void testMergeModeConsistencyValidation() throws Exception { // Append record with DEFAULT mode should succeed WriteRecord defaultRecord = createWriteRecordWithMergeMode(MergeMode.DEFAULT); - assertThat(defaultBatch.tryAppend(defaultRecord, newWriteCallback())).isTrue(); + assertThat(defaultBatch.tryAppend(defaultRecord, newWriteCallback())) + .isEqualTo(AppendResult.APPENDED); // Append record with OVERWRITE mode should fail WriteRecord overwriteRecord = createWriteRecordWithMergeMode(MergeMode.OVERWRITE); @@ -292,7 +296,8 @@ void testOverwriteModeBatch() throws Exception { // Append record with OVERWRITE mode should succeed WriteRecord overwriteRecord = createWriteRecordWithMergeMode(MergeMode.OVERWRITE); - assertThat(overwriteBatch.tryAppend(overwriteRecord, newWriteCallback())).isTrue(); + assertThat(overwriteBatch.tryAppend(overwriteRecord, newWriteCallback())) + .isEqualTo(AppendResult.APPENDED); // Append record with DEFAULT mode should fail WriteRecord defaultRecord = createWriteRecordWithMergeMode(MergeMode.DEFAULT); @@ -324,6 +329,7 @@ private KvWriteBatch createKvWriteBatchWithMergeMode(TableBucket tb, MergeMode m outputView, null, mergeMode, + false, System.currentTimeMillis()); } @@ -338,4 +344,188 @@ private WriteRecord createWriteRecordWithMergeMode(MergeMode mergeMode) { null, mergeMode); } + + // ==================== V2 Format Tests ==================== + + @Test + void testV0BatchRejectsRetract() throws Exception { + KvWriteBatch v0Batch = createKvWriteBatch(new TableBucket(DATA1_TABLE_ID_PK, 0)); + + // UPSERT should succeed on V0 batch + WriteRecord upsertRecord = createWriteRecord(); + assertThat(v0Batch.tryAppend(upsertRecord, newWriteCallback())) + .isEqualTo(AppendResult.APPENDED); + + // RETRACT should be rejected on V0 batch + WriteRecord retractRecord = + WriteRecord.forRetract( + DATA1_TABLE_INFO_PK, + PhysicalTablePath.of(DATA1_TABLE_PATH_PK), + row, + key, + key, + WriteFormat.COMPACTED_KV, + null, + MergeMode.DEFAULT); + assertThat(v0Batch.tryAppend(retractRecord, newWriteCallback())) + .isEqualTo(AppendResult.FORMAT_MISMATCH); + + // Only the UPSERT record should be in the batch + assertThat(v0Batch.getRecordCount()).isEqualTo(1); + } + + @Test + void testV2BatchAcceptsMixed() throws Exception { + KvWriteBatch v2Batch = createV2KvWriteBatch(new TableBucket(DATA1_TABLE_ID_PK, 0)); + + // RETRACT should succeed on V2 batch + WriteRecord retractRecord = + WriteRecord.forRetract( + DATA1_TABLE_INFO_PK, + PhysicalTablePath.of(DATA1_TABLE_PATH_PK), + row, + key, + key, + WriteFormat.COMPACTED_KV, + null, + MergeMode.DEFAULT); + assertThat(v2Batch.tryAppend(retractRecord, newWriteCallback())) + .isEqualTo(AppendResult.APPENDED); + + // UPSERT should also succeed on V2 batch + WriteRecord upsertRecord = createWriteRecord(); + assertThat(v2Batch.tryAppend(upsertRecord, newWriteCallback())) + .isEqualTo(AppendResult.APPENDED); + + // Both records should be in the batch + assertThat(v2Batch.getRecordCount()).isEqualTo(2); + + // Verify mutation types survive serialization + DefaultKvRecordBatch kvRecords = DefaultKvRecordBatch.pointToBytesView(v2Batch.build()); + KvRecordReadContext v2ReadContext = + KvRecordReadContext.createReadContext( + KvFormat.COMPACTED, new TestingSchemaGetter(1, DATA1_SCHEMA), true); + Iterator iterator = kvRecords.records(v2ReadContext).iterator(); + assertThat(iterator.next().getMutationType()).isEqualTo(MutationType.RETRACT); + assertThat(iterator.next().getMutationType()).isEqualTo(MutationType.UPSERT); + } + + @Test + void testV2FormatRetractRecordRoundTrip() throws Exception { + KvWriteBatch v2Batch = createV2KvWriteBatch(new TableBucket(DATA1_TABLE_ID_PK, 0)); + + // Append a RETRACT record + WriteRecord retractRecord = + WriteRecord.forRetract( + DATA1_TABLE_INFO_PK, + PhysicalTablePath.of(DATA1_TABLE_PATH_PK), + row, + key, + key, + WriteFormat.COMPACTED_KV, + null, + MergeMode.DEFAULT); + assertThat(retractRecord.getMutationType()).isEqualTo(MutationType.RETRACT); + assertThat(v2Batch.tryAppend(retractRecord, newWriteCallback())) + .isEqualTo(AppendResult.APPENDED); + + // Build and read back with V2 ReadContext + DefaultKvRecordBatch kvRecords = DefaultKvRecordBatch.pointToBytesView(v2Batch.build()); + assertThat(kvRecords.getRecordCount()).isEqualTo(1); + + KvRecordReadContext v2ReadContext = + KvRecordReadContext.createReadContext( + KvFormat.COMPACTED, new TestingSchemaGetter(1, DATA1_SCHEMA), true); + Iterator iterator = kvRecords.records(v2ReadContext).iterator(); + assertThat(iterator.hasNext()).isTrue(); + KvRecord kvRecord = iterator.next(); + assertThat(toArray(kvRecord.getKey())).isEqualTo(key); + assertThat(kvRecord.getRow()).isEqualTo(row); + assertThat(kvRecord.getMutationType()).isEqualTo(MutationType.RETRACT); + } + + @Test + void testV2FormatUpsertRecordRoundTrip() throws Exception { + KvWriteBatch v2Batch = createV2KvWriteBatch(new TableBucket(DATA1_TABLE_ID_PK, 0)); + + // Append an UPSERT record + WriteRecord upsertRecord = createWriteRecord(); + assertThat(v2Batch.tryAppend(upsertRecord, newWriteCallback())) + .isEqualTo(AppendResult.APPENDED); + + // Build and read back with V2 ReadContext + DefaultKvRecordBatch kvRecords = DefaultKvRecordBatch.pointToBytesView(v2Batch.build()); + assertThat(kvRecords.getRecordCount()).isEqualTo(1); + + KvRecordReadContext v2ReadContext = + KvRecordReadContext.createReadContext( + KvFormat.COMPACTED, new TestingSchemaGetter(1, DATA1_SCHEMA), true); + Iterator iterator = kvRecords.records(v2ReadContext).iterator(); + assertThat(iterator.hasNext()).isTrue(); + KvRecord kvRecord = iterator.next(); + assertThat(toArray(kvRecord.getKey())).isEqualTo(key); + assertThat(kvRecord.getRow()).isEqualTo(row); + assertThat(kvRecord.getMutationType()).isEqualTo(MutationType.UPSERT); + } + + @Test + void testV2FormatMixedUpsertAndRetract() throws Exception { + KvWriteBatch v2Batch = createV2KvWriteBatch(new TableBucket(DATA1_TABLE_ID_PK, 0)); + + // Append an UPSERT record + WriteRecord upsertRecord = createWriteRecord(); + assertThat(v2Batch.tryAppend(upsertRecord, newWriteCallback())) + .isEqualTo(AppendResult.APPENDED); + + // Append a RETRACT record + WriteRecord retractRecord = + WriteRecord.forRetract( + DATA1_TABLE_INFO_PK, + PhysicalTablePath.of(DATA1_TABLE_PATH_PK), + row, + key, + key, + WriteFormat.COMPACTED_KV, + null, + MergeMode.DEFAULT); + assertThat(v2Batch.tryAppend(retractRecord, newWriteCallback())) + .isEqualTo(AppendResult.APPENDED); + assertThat(v2Batch.getRecordCount()).isEqualTo(2); + + // Build and read back with V2 ReadContext + DefaultKvRecordBatch kvRecords = DefaultKvRecordBatch.pointToBytesView(v2Batch.build()); + assertThat(kvRecords.getRecordCount()).isEqualTo(2); + + KvRecordReadContext v2ReadContext = + KvRecordReadContext.createReadContext( + KvFormat.COMPACTED, new TestingSchemaGetter(1, DATA1_SCHEMA), true); + Iterator iterator = kvRecords.records(v2ReadContext).iterator(); + + // First record: UPSERT + assertThat(iterator.hasNext()).isTrue(); + KvRecord first = iterator.next(); + assertThat(first.getMutationType()).isEqualTo(MutationType.UPSERT); + + // Second record: RETRACT + assertThat(iterator.hasNext()).isTrue(); + KvRecord second = iterator.next(); + assertThat(second.getMutationType()).isEqualTo(MutationType.RETRACT); + } + + private KvWriteBatch createV2KvWriteBatch(TableBucket tb) throws Exception { + PreAllocatedPagedOutputView outputView = + new PreAllocatedPagedOutputView( + Collections.singletonList(memoryPool.nextSegment())); + return new KvWriteBatch( + tb.getBucket(), + PhysicalTablePath.of(DATA1_TABLE_PATH_PK), + DATA1_TABLE_INFO_PK.getSchemaId(), + KvFormat.COMPACTED, + Integer.MAX_VALUE, + outputView, + null, + MergeMode.DEFAULT, + true, + System.currentTimeMillis()); + } } diff --git a/fluss-client/src/test/java/org/apache/fluss/client/write/WriteRecordTest.java b/fluss-client/src/test/java/org/apache/fluss/client/write/WriteRecordTest.java new file mode 100644 index 0000000000..a0b296dcd7 --- /dev/null +++ b/fluss-client/src/test/java/org/apache/fluss/client/write/WriteRecordTest.java @@ -0,0 +1,228 @@ +/* + * 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.write; + +import org.apache.fluss.metadata.PhysicalTablePath; +import org.apache.fluss.record.MutationType; +import org.apache.fluss.row.BinaryRow; +import org.apache.fluss.row.compacted.CompactedRow; +import org.apache.fluss.row.encode.CompactedKeyEncoder; + +import org.junit.jupiter.api.Test; + +import static org.apache.fluss.record.TestData.DATA1_ROW_TYPE; +import static org.apache.fluss.record.TestData.DATA1_SCHEMA_PK; +import static org.apache.fluss.record.TestData.DATA1_TABLE_INFO_PK; +import static org.apache.fluss.record.TestData.DATA1_TABLE_PATH_PK; +import static org.apache.fluss.testutils.DataTestUtils.compactedRow; +import static org.assertj.core.api.Assertions.assertThat; +import static org.assertj.core.api.Assertions.assertThatThrownBy; + +/** Tests for {@link WriteRecord}. */ +class WriteRecordTest { + + @Test + void testKvUpsertRecordHasRow() { + BinaryRow row = compactedRow(DATA1_ROW_TYPE, new Object[] {1, "a"}); + byte[] key = + new CompactedKeyEncoder(DATA1_ROW_TYPE, DATA1_SCHEMA_PK.getPrimaryKeyIndexes()) + .encodeKey(row); + WriteRecord writeRecord = + WriteRecord.forUpsert( + DATA1_TABLE_INFO_PK, + PhysicalTablePath.of(DATA1_TABLE_PATH_PK), + row, + key, + key, + WriteFormat.COMPACTED_KV, + null); + + assertThat(writeRecord.getRow()).isNotNull(); + assertThat(writeRecord.getKey()).isEqualTo(key); + } + + @Test + void testKvDeleteRecordHasNullRow() { + CompactedRow row = compactedRow(DATA1_ROW_TYPE, new Object[] {1, "a"}); + byte[] key = + new CompactedKeyEncoder(DATA1_ROW_TYPE, DATA1_SCHEMA_PK.getPrimaryKeyIndexes()) + .encodeKey(row); + WriteRecord writeRecord = + WriteRecord.forDelete( + DATA1_TABLE_INFO_PK, + PhysicalTablePath.of(DATA1_TABLE_PATH_PK), + key, + key, + WriteFormat.COMPACTED_KV, + null); + + assertThat(writeRecord.getRow()).isNull(); + assertThat(writeRecord.getKey()).isEqualTo(key); + } + + @Test + void testAppendOnlyRecordHasNoKey() { + CompactedRow row = compactedRow(DATA1_ROW_TYPE, new Object[] {1, "a"}); + WriteRecord writeRecord = + WriteRecord.forCompactedAppend( + DATA1_TABLE_INFO_PK, PhysicalTablePath.of(DATA1_TABLE_PATH_PK), row, null); + + assertThat(writeRecord.getRow()).isNotNull(); + assertThat(writeRecord.getKey()).isNull(); + } + + @Test + void testUpsertMutationType() { + BinaryRow row = compactedRow(DATA1_ROW_TYPE, new Object[] {1, "a"}); + byte[] key = + new CompactedKeyEncoder(DATA1_ROW_TYPE, DATA1_SCHEMA_PK.getPrimaryKeyIndexes()) + .encodeKey(row); + WriteRecord writeRecord = + WriteRecord.forUpsert( + DATA1_TABLE_INFO_PK, + PhysicalTablePath.of(DATA1_TABLE_PATH_PK), + row, + key, + key, + WriteFormat.COMPACTED_KV, + null); + + assertThat(writeRecord.getMutationType()).isEqualTo(MutationType.UPSERT); + } + + @Test + void testDeleteMutationType() { + CompactedRow row = compactedRow(DATA1_ROW_TYPE, new Object[] {1, "a"}); + byte[] key = + new CompactedKeyEncoder(DATA1_ROW_TYPE, DATA1_SCHEMA_PK.getPrimaryKeyIndexes()) + .encodeKey(row); + WriteRecord writeRecord = + WriteRecord.forDelete( + DATA1_TABLE_INFO_PK, + PhysicalTablePath.of(DATA1_TABLE_PATH_PK), + key, + key, + WriteFormat.COMPACTED_KV, + null); + + assertThat(writeRecord.getMutationType()).isEqualTo(MutationType.DELETE); + } + + @Test + void testRetractMutationType() { + BinaryRow row = compactedRow(DATA1_ROW_TYPE, new Object[] {1, "a"}); + byte[] key = + new CompactedKeyEncoder(DATA1_ROW_TYPE, DATA1_SCHEMA_PK.getPrimaryKeyIndexes()) + .encodeKey(row); + WriteRecord writeRecord = + WriteRecord.forRetract( + DATA1_TABLE_INFO_PK, + PhysicalTablePath.of(DATA1_TABLE_PATH_PK), + row, + key, + key, + WriteFormat.COMPACTED_KV, + null, + org.apache.fluss.rpc.protocol.MergeMode.DEFAULT); + + assertThat(writeRecord.getMutationType()).isEqualTo(MutationType.RETRACT); + assertThat(writeRecord.getRow()).isNotNull(); + assertThat(writeRecord.getKey()).isEqualTo(key); + } + + @Test + void testForRetractNullRowThrows() { + BinaryRow row = compactedRow(DATA1_ROW_TYPE, new Object[] {1, "a"}); + byte[] key = + new CompactedKeyEncoder(DATA1_ROW_TYPE, DATA1_SCHEMA_PK.getPrimaryKeyIndexes()) + .encodeKey(row); + assertThatThrownBy( + () -> + WriteRecord.forRetract( + DATA1_TABLE_INFO_PK, + PhysicalTablePath.of(DATA1_TABLE_PATH_PK), + null, + key, + key, + WriteFormat.COMPACTED_KV, + null, + org.apache.fluss.rpc.protocol.MergeMode.DEFAULT)) + .isInstanceOf(NullPointerException.class) + .hasMessageContaining("row must not be null"); + } + + @Test + void testForRetractNullKeyThrows() { + BinaryRow row = compactedRow(DATA1_ROW_TYPE, new Object[] {1, "a"}); + assertThatThrownBy( + () -> + WriteRecord.forRetract( + DATA1_TABLE_INFO_PK, + PhysicalTablePath.of(DATA1_TABLE_PATH_PK), + row, + null, + new byte[] {1}, + WriteFormat.COMPACTED_KV, + null, + org.apache.fluss.rpc.protocol.MergeMode.DEFAULT)) + .isInstanceOf(NullPointerException.class) + .hasMessageContaining("key must not be null"); + } + + @Test + void testForRetractNullBucketKeyThrows() { + BinaryRow row = compactedRow(DATA1_ROW_TYPE, new Object[] {1, "a"}); + byte[] key = + new CompactedKeyEncoder(DATA1_ROW_TYPE, DATA1_SCHEMA_PK.getPrimaryKeyIndexes()) + .encodeKey(row); + assertThatThrownBy( + () -> + WriteRecord.forRetract( + DATA1_TABLE_INFO_PK, + PhysicalTablePath.of(DATA1_TABLE_PATH_PK), + row, + key, + null, + WriteFormat.COMPACTED_KV, + null, + org.apache.fluss.rpc.protocol.MergeMode.DEFAULT)) + .isInstanceOf(NullPointerException.class) + .hasMessageContaining("bucketKey must not be null"); + } + + @Test + void testForRetractNonKvFormatThrows() { + BinaryRow row = compactedRow(DATA1_ROW_TYPE, new Object[] {1, "a"}); + byte[] key = + new CompactedKeyEncoder(DATA1_ROW_TYPE, DATA1_SCHEMA_PK.getPrimaryKeyIndexes()) + .encodeKey(row); + assertThatThrownBy( + () -> + WriteRecord.forRetract( + DATA1_TABLE_INFO_PK, + PhysicalTablePath.of(DATA1_TABLE_PATH_PK), + row, + key, + key, + WriteFormat.INDEXED_LOG, + null, + org.apache.fluss.rpc.protocol.MergeMode.DEFAULT)) + .isInstanceOf(IllegalArgumentException.class) + .hasMessageContaining("writeFormat must be a KV format"); + } +} diff --git a/fluss-common/src/main/java/org/apache/fluss/metadata/AggFunction.java b/fluss-common/src/main/java/org/apache/fluss/metadata/AggFunction.java index 1f31a8c222..f909697ecc 100644 --- a/fluss-common/src/main/java/org/apache/fluss/metadata/AggFunction.java +++ b/fluss-common/src/main/java/org/apache/fluss/metadata/AggFunction.java @@ -104,6 +104,16 @@ public boolean hasParameters() { return !parameters.isEmpty(); } + /** + * Returns whether this aggregation function supports retract (UPDATE_BEFORE) records. + * + * @return true if this function supports retract, false otherwise + * @see AggFunctionType#supportsRetract() + */ + public boolean supportsRetract() { + return type.supportsRetract(); + } + /** * Validates data type of this aggregation function. This method checks that the field data type * is valid diff --git a/fluss-common/src/main/java/org/apache/fluss/metadata/AggFunctionType.java b/fluss-common/src/main/java/org/apache/fluss/metadata/AggFunctionType.java index f0026fbba7..8fbdc289e1 100644 --- a/fluss-common/src/main/java/org/apache/fluss/metadata/AggFunctionType.java +++ b/fluss-common/src/main/java/org/apache/fluss/metadata/AggFunctionType.java @@ -198,6 +198,30 @@ public void validateDataType(DataType fieldType) { fieldType); } + /** + * Returns whether this aggregation function has a retract implementation. + * + *

Retract quality varies by function: + * + *

    + *
  • Strict inverse: {@link #SUM} — subtraction is mathematically exact. + *
  • Best-effort: {@link #LAST_VALUE} and {@link #LAST_VALUE_IGNORE_NULLS} — + * unconditionally clear to null without reconstructing prior history. + *
+ * + * @return true if this function has a retract implementation, false otherwise + */ + public boolean supportsRetract() { + switch (this) { + case SUM: + case LAST_VALUE: + case LAST_VALUE_IGNORE_NULLS: + return true; + default: + return false; + } + } + /** * Converts a string to an AggFunctionType enum value. * diff --git a/fluss-common/src/main/java/org/apache/fluss/record/DefaultKvRecord.java b/fluss-common/src/main/java/org/apache/fluss/record/DefaultKvRecord.java index 9bda95b196..882934d022 100644 --- a/fluss-common/src/main/java/org/apache/fluss/record/DefaultKvRecord.java +++ b/fluss-common/src/main/java/org/apache/fluss/record/DefaultKvRecord.java @@ -38,7 +38,7 @@ /** * This class is an immutable kv record. Different from {@link IndexedLogRecord}, it isn't designed - * for persistence. The schema is as follows: + * for persistence. The V0 schema is as follows: * *
    *
  • Length => int32 @@ -47,6 +47,16 @@ *
  • Row => {@link BinaryRow} *
* + *

The V2 schema adds a MutationType byte after Length: + * + *

    + *
  • Length => int32 + *
  • MutationType => int8 (see {@link MutationType}) + *
  • KeyLength => unsigned varint + *
  • Key => bytes + *
  • Row => {@link BinaryRow} + *
+ * *

When the row is null, no any byte will be remained after Key. * * @since 0.1 @@ -55,6 +65,10 @@ public class DefaultKvRecord implements KvRecord { static final int LENGTH_LENGTH = 4; + + /** Size of the MutationType byte in V2 format. */ + static final int MUTATION_TYPE_LENGTH = 1; + private final RowDecoder rowDecoder; private MemorySegment segment; @@ -64,6 +78,13 @@ public class DefaultKvRecord implements KvRecord { private ByteBuffer key; private BinaryRow value; + // Null signals a legacy V0/V1 record so getMutationType() can fall back to + // row-presence heuristic without requiring a format migration. + @Nullable private MutationType mutationType; + + /** Minimum V2 record body size: 1 byte MutationType + 1 byte varint key length. */ + static final int MIN_V2_RECORD_SIZE = MUTATION_TYPE_LENGTH + 1; + private DefaultKvRecord(RowDecoder rowDecoder) { this.rowDecoder = rowDecoder; } @@ -99,20 +120,14 @@ public int hashCode() { public static int writeTo(OutputView outputView, byte[] key, @Nullable BinaryRow row) throws IOException { - // bytes for key length + bytes for key + bytes for row int sizeInBytes = sizeWithoutLength(key, row); // TODO using varint instead int to reduce storage size. - // write record total bytes size. outputView.writeInt(sizeInBytes); - - // write key length, unsigned var int; VarLengthUtils.writeUnsignedVarInt(key.length, outputView); - // write the real key outputView.write(key); if (row != null) { - // write internal row, which is the value. serializeInternalRow(outputView, row); } return sizeInBytes + LENGTH_LENGTH; @@ -134,6 +149,86 @@ public static int sizeOf(byte[] key, @Nullable BinaryRow row) { return sizeWithoutLength(key, row) + LENGTH_LENGTH; } + // ----------------------- V2 format methods ------------------------------- + + /** + * Write a V2 format KvRecord to the output view. V2 layout: + * + *

    + *
  • Length => int32 (covers MutationType + KeyLength + Key + Row) + *
  • MutationType => int8 + *
  • KeyLength => unsigned varint + *
  • Key => bytes + *
  • Row => {@link BinaryRow} (absent when row is null) + *
+ */ + public static int writeToV2( + OutputView outputView, MutationType mutationType, byte[] key, @Nullable BinaryRow row) + throws IOException { + int sizeInBytes = sizeWithoutLengthV2(key, row); + + outputView.writeInt(sizeInBytes); + outputView.writeByte(mutationType.getValue()); + VarLengthUtils.writeUnsignedVarInt(key.length, outputView); + outputView.write(key); + + if (row != null) { + serializeInternalRow(outputView, row); + } + return sizeInBytes + LENGTH_LENGTH; + } + + /** + * Read a V2 format KvRecord from the given memory segment. The MutationType byte is read and + * stored explicitly. Must set mutationType BEFORE pointTo so that readKeyAndRow knows to skip + * the extra byte. + */ + public static KvRecord readFromV2( + MemorySegment segment, + int position, + short schemaId, + KvRecordBatch.ReadContext readContext) { + int sizeInBytes = segment.getInt(position); + if (sizeInBytes < MIN_V2_RECORD_SIZE) { + throw new InvalidRecordException( + "Invalid V2 KvRecord size: " + + sizeInBytes + + " at position " + + position + + ". Minimum size is " + + MIN_V2_RECORD_SIZE + + " bytes."); + } + DefaultKvRecord kvRecord = new DefaultKvRecord(readContext.getRowDecoder(schemaId)); + // Read the mutation type byte before pointTo, so readKeyAndRow can skip it. + byte mutationByte = segment.get(position + LENGTH_LENGTH); + try { + kvRecord.mutationType = MutationType.fromValue(mutationByte); + } catch (IllegalArgumentException e) { + throw new InvalidRecordException( + "Invalid MutationType byte 0x" + + Integer.toHexString(mutationByte & 0xFF) + + " at position " + + (position + LENGTH_LENGTH) + + ".", + e); + } + kvRecord.pointTo(segment, position, sizeInBytes + LENGTH_LENGTH); + return kvRecord; + } + + /** + * Calculate the size of a V2 kv record written to batch, including {@link #LENGTH_LENGTH}. This + * is {@link #sizeOf(byte[], BinaryRow)} + 1 byte for MutationType. + */ + public static int sizeOfV2(byte[] key, @Nullable BinaryRow row) { + return sizeWithoutLengthV2(key, row) + LENGTH_LENGTH; + } + + private static int sizeWithoutLengthV2(byte[] key, @Nullable BinaryRow row) { + return MUTATION_TYPE_LENGTH + sizeWithoutLength(key, row); + } + private static int sizeWithoutLength(byte[] key, @Nullable BinaryRow row) { return VarLengthUtils.sizeOfUnsignedVarInt(key.length) + key.length @@ -141,8 +236,11 @@ private static int sizeWithoutLength(byte[] key, @Nullable BinaryRow row) { } private void readKeyAndRow() throws IOException { - // start to read from the offset of key int currentOffset = offset + LENGTH_LENGTH; + // For V2 records, skip the MutationType byte (already read before pointTo). + if (mutationType != null) { + currentOffset += MUTATION_TYPE_LENGTH; + } // now, read key; // read the length of key size int keyLength = VarLengthUtils.readUnsignedVarInt(segment, currentOffset); @@ -173,6 +271,15 @@ public ByteBuffer getKey() { return value; } + @Override + public MutationType getMutationType() { + if (mutationType != null) { + return mutationType; + } + // V0/V1 fallback: infer from row presence. + return value == null ? MutationType.DELETE : MutationType.UPSERT; + } + @Override public int getSizeInBytes() { return sizeInBytes; diff --git a/fluss-common/src/main/java/org/apache/fluss/record/DefaultKvRecordBatch.java b/fluss-common/src/main/java/org/apache/fluss/record/DefaultKvRecordBatch.java index 1fe83650d1..1a139c0252 100644 --- a/fluss-common/src/main/java/org/apache/fluss/record/DefaultKvRecordBatch.java +++ b/fluss-common/src/main/java/org/apache/fluss/record/DefaultKvRecordBatch.java @@ -56,7 +56,7 @@ * *
  * -----------------------------------------------------------------------------------------------
- * | Unused (0-8)
+ * | Unused (0-7)
  * -----------------------------------------------------------------------------------------------
  * 
* @@ -85,6 +85,12 @@ public class DefaultKvRecordBatch implements KvRecordBatch { static final int RECORDS_OFFSET = RECORDS_COUNT_OFFSET + RECORDS_COUNT_LENGTH; public static final int RECORD_BATCH_HEADER_SIZE = RECORDS_OFFSET; + /** Bit mask for V2 record format in the attributes byte. */ + static final byte V2_FORMAT_ATTRIBUTE_MASK = 0x01; + + /** Mask of all attribute bits that this version understands. */ + static final byte KNOWN_ATTRIBUTES_MASK = V2_FORMAT_ATTRIBUTE_MASK; + public static final int KV_OVERHEAD = LENGTH_OFFSET + LENGTH_LENGTH; private MemorySegment segment; @@ -111,6 +117,17 @@ public void ensureValid() { + RECORD_BATCH_HEADER_SIZE + ")"); } + // Reject batches that use attribute bits this version does not understand. + // This prevents an old server from silently misreading V2 records as V0. + byte attributes = segment.get(position + ATTRIBUTES_OFFSET); + byte unknownBits = (byte) (attributes & ~KNOWN_ATTRIBUTES_MASK); + if (unknownBits != 0) { + throw new CorruptMessageException( + "Record batch uses unsupported attribute bits: 0x" + + Integer.toHexString(unknownBits & 0xFF) + + ". This may indicate a newer record format that this server version" + + " does not support."); + } if (!isValid()) { throw new CorruptMessageException( "Record batch is corrupt (stored crc = " @@ -156,6 +173,11 @@ public int getRecordCount() { return segment.getInt(position + RECORDS_COUNT_OFFSET); } + @Override + public boolean isV2Format() { + return (segment.get(position + ATTRIBUTES_OFFSET) & V2_FORMAT_ATTRIBUTE_MASK) != 0; + } + public MemorySegment getMemorySegment() { return segment; } @@ -181,8 +203,12 @@ private Iterator iterator(ReadContext readContext) { @Override protected KvRecord readNext() { - KvRecord kvRecord = - DefaultKvRecord.readFrom(segment, position, schemaId, readContext); + KvRecord kvRecord; + if (DefaultKvRecordBatch.this.isV2Format()) { + kvRecord = DefaultKvRecord.readFromV2(segment, position, schemaId, readContext); + } else { + kvRecord = DefaultKvRecord.readFrom(segment, position, schemaId, readContext); + } iteratorNumber++; position += kvRecord.getSizeInBytes(); return kvRecord; diff --git a/fluss-common/src/main/java/org/apache/fluss/record/KvRecord.java b/fluss-common/src/main/java/org/apache/fluss/record/KvRecord.java index b3c4602b46..bb89042cbc 100644 --- a/fluss-common/src/main/java/org/apache/fluss/record/KvRecord.java +++ b/fluss-common/src/main/java/org/apache/fluss/record/KvRecord.java @@ -53,4 +53,13 @@ public interface KvRecord { * @return the size of the record in bytes */ int getSizeInBytes(); + + /** + * Get the mutation type of this record. V0/V1 records infer the type from the row: {@link + * MutationType#UPSERT} for non-null row, {@link MutationType#DELETE} for null row. V2 records + * carry an explicit mutation type byte. + */ + default MutationType getMutationType() { + return getRow() == null ? MutationType.DELETE : MutationType.UPSERT; + } } diff --git a/fluss-common/src/main/java/org/apache/fluss/record/KvRecordBatch.java b/fluss-common/src/main/java/org/apache/fluss/record/KvRecordBatch.java index 0689ecadd1..6894f97f17 100644 --- a/fluss-common/src/main/java/org/apache/fluss/record/KvRecordBatch.java +++ b/fluss-common/src/main/java/org/apache/fluss/record/KvRecordBatch.java @@ -97,6 +97,14 @@ public interface KvRecordBatch { */ int getRecordCount(); + /** + * Whether records in this batch use V2 wire format (with per-record MutationType byte). V2 + * batches have attributes bit 0 set. + */ + default boolean isV2Format() { + return false; + } + /** * Get the iterable of {@link KvRecord} in this batch. * diff --git a/fluss-common/src/main/java/org/apache/fluss/record/KvRecordBatchBuilder.java b/fluss-common/src/main/java/org/apache/fluss/record/KvRecordBatchBuilder.java index 12c3217101..dcb3d7f9ce 100644 --- a/fluss-common/src/main/java/org/apache/fluss/record/KvRecordBatchBuilder.java +++ b/fluss-common/src/main/java/org/apache/fluss/record/KvRecordBatchBuilder.java @@ -58,6 +58,7 @@ public class KvRecordBatchBuilder implements AutoCloseable { private int sizeInBytes; private volatile boolean isClosed; private final KvFormat kvFormat; + private final boolean v2Format; private boolean aborted = false; private KvRecordBatchBuilder( @@ -65,7 +66,8 @@ private KvRecordBatchBuilder( byte magic, int writeLimit, AbstractPagedOutputView pagedOutputView, - KvFormat kvFormat) { + KvFormat kvFormat, + boolean v2Format) { checkArgument( schemaId <= Short.MAX_VALUE, "schemaId shouldn't be greater than the max value of short: " + Short.MAX_VALUE); @@ -83,12 +85,22 @@ private KvRecordBatchBuilder( pagedOutputView.setPosition(RECORD_BATCH_HEADER_SIZE); this.sizeInBytes = RECORD_BATCH_HEADER_SIZE; this.kvFormat = kvFormat; + this.v2Format = v2Format; } public static KvRecordBatchBuilder builder( int schemaId, int writeLimit, AbstractPagedOutputView outputView, KvFormat kvFormat) { + return builder(schemaId, writeLimit, outputView, kvFormat, false); + } + + public static KvRecordBatchBuilder builder( + int schemaId, + int writeLimit, + AbstractPagedOutputView outputView, + KvFormat kvFormat, + boolean v2Format) { return new KvRecordBatchBuilder( - schemaId, CURRENT_KV_MAGIC_VALUE, writeLimit, outputView, kvFormat); + schemaId, CURRENT_KV_MAGIC_VALUE, writeLimit, outputView, kvFormat, v2Format); } /** @@ -96,7 +108,17 @@ public static KvRecordBatchBuilder builder( * appended, then this returns true. */ public boolean hasRoomFor(byte[] key, @Nullable BinaryRow row) { - return sizeInBytes + DefaultKvRecord.sizeOf(key, row) <= writeLimit; + int recordSize = DefaultKvRecord.sizeOf(key, row); + return sizeInBytes + recordSize <= writeLimit; + } + + /** + * Check if we have room for a new V2 record containing the given row. V2 records have an extra + * MutationType byte. + */ + public boolean hasRoomForV2(byte[] key, @Nullable BinaryRow row) { + int recordSize = DefaultKvRecord.sizeOfV2(key, row); + return sizeInBytes + recordSize <= writeLimit; } /** @@ -107,6 +129,10 @@ public boolean hasRoomFor(byte[] key, @Nullable BinaryRow row) { * KvRecord is for delete the corresponding key. */ public void append(byte[] key, @Nullable BinaryRow row) throws IOException { + if (v2Format) { + throw new IllegalStateException( + "append() must not be used on a V2 format builder. Use appendV2() instead."); + } if (aborted) { throw new IllegalStateException( "Tried to append a record, but KvRecordBatchBuilder has already been aborted"); @@ -126,6 +152,40 @@ public void append(byte[] key, @Nullable BinaryRow row) throws IOException { sizeInBytes += recordByteSizes; } + /** + * Append a V2 format KvRecord with an explicit {@link MutationType} to the batch. + * + * @param mutationType the mutation type of this record + * @param key the key in the KvRecord to be appended + * @param row the value in the KvRecord to be appended. Null for DELETE records. + */ + public void appendV2(MutationType mutationType, byte[] key, @Nullable BinaryRow row) + throws IOException { + if (!v2Format) { + throw new IllegalStateException( + "appendV2() requires a V2 format builder. Use builder(..., v2Format=true)."); + } + if (aborted) { + throw new IllegalStateException( + "Tried to append a record, but KvRecordBatchBuilder has already been aborted"); + } + + if (isClosed) { + throw new IllegalStateException( + "Tried to put a record, but KvRecordBatchBuilder is closed for record puts."); + } + int recordByteSizes = + DefaultKvRecord.writeToV2( + pagedOutputView, mutationType, key, validateRowFormat(row)); + currentRecordNumber++; + if (currentRecordNumber == Integer.MAX_VALUE) { + throw new IllegalArgumentException( + "Maximum number of records per batch exceeded, max records: " + + Integer.MAX_VALUE); + } + sizeInBytes += recordByteSizes; + } + public void setWriterState(long writerId, int batchBaseSequence) { this.writerId = writerId; this.batchSequence = batchBaseSequence; @@ -208,7 +268,7 @@ private void writeBatchHeader() throws IOException { } private byte computeAttributes() { - return 0; + return v2Format ? DefaultKvRecordBatch.V2_FORMAT_ATTRIBUTE_MASK : 0; } /** Validate the row instance according to the kv format. */ diff --git a/fluss-common/src/main/java/org/apache/fluss/record/MutationType.java b/fluss-common/src/main/java/org/apache/fluss/record/MutationType.java new file mode 100644 index 0000000000..0119bc0244 --- /dev/null +++ b/fluss-common/src/main/java/org/apache/fluss/record/MutationType.java @@ -0,0 +1,63 @@ +/* + * 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.annotation.PublicEvolving; + +/** + * The mutation type of a {@link KvRecord} in V2 format. + * + *

This is a per-record field inside the KvRecord body, independent of the batch/request-level + * {@link org.apache.fluss.rpc.protocol.MergeMode}. + * + * @since 0.10 + */ +@PublicEvolving +public enum MutationType { + /** Normal upsert record. Row must not be null. */ + UPSERT((byte) 0), + + /** Explicit delete record. Row must be null. */ + DELETE((byte) 1), + + /** Retract record carrying the old value to be retracted. Row must not be null. */ + RETRACT((byte) 2); + + private final byte value; + + MutationType(byte value) { + this.value = value; + } + + public byte getValue() { + return value; + } + + public static MutationType fromValue(byte value) { + switch (value) { + case 0: + return UPSERT; + case 1: + return DELETE; + case 2: + return RETRACT; + default: + throw new IllegalArgumentException("Unknown MutationType value: " + value); + } + } +} diff --git a/fluss-common/src/main/java/org/apache/fluss/rpc/protocol/MergeMode.java b/fluss-common/src/main/java/org/apache/fluss/rpc/protocol/MergeMode.java index eb3cafbade..294416d283 100644 --- a/fluss-common/src/main/java/org/apache/fluss/rpc/protocol/MergeMode.java +++ b/fluss-common/src/main/java/org/apache/fluss/rpc/protocol/MergeMode.java @@ -19,6 +19,9 @@ import org.apache.fluss.annotation.PublicEvolving; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + /** * Merge mode for write operations on tables with merge engines. * @@ -60,6 +63,8 @@ public enum MergeMode { */ OVERWRITE(1); + private static final Logger LOG = LoggerFactory.getLogger(MergeMode.class); + private final int value; MergeMode(int value) { @@ -92,7 +97,8 @@ public int getProtoValue() { * Converts an integer value to a MergeMode enum. * * @param value the integer value - * @return the corresponding MergeMode, or DEFAULT if the value is invalid + * @return the corresponding MergeMode + * @throws IllegalArgumentException if the value is unknown */ public static MergeMode fromValue(int value) { switch (value) { @@ -101,19 +107,37 @@ public static MergeMode fromValue(int value) { case 1: return OVERWRITE; default: - return DEFAULT; + throw new IllegalArgumentException("Unknown MergeMode value: " + value); } } /** * Converts a proto value to a MergeMode enum. * - *

This is an alias for {@link #fromValue(int)} for clarity when working with proto messages. + *

Unlike {@link #fromValue(int)}, this method is lenient: unknown values return {@link + * #DEFAULT} with a warning log, providing backward compatibility when a newer server sends an + * unrecognized merge mode to an older client. * * @param protoValue the proto value - * @return the corresponding MergeMode, or DEFAULT if the value is invalid + * @return the corresponding MergeMode, or {@link #DEFAULT} for unknown values */ + private static volatile boolean unknownMergeModeWarned = false; + public static MergeMode fromProtoValue(int protoValue) { - return fromValue(protoValue); + switch (protoValue) { + case 0: + return DEFAULT; + case 1: + return OVERWRITE; + default: + if (!unknownMergeModeWarned) { + unknownMergeModeWarned = true; + LOG.warn( + "Unknown MergeMode proto value: {}, falling back to DEFAULT. " + + "This warning is logged only once.", + protoValue); + } + return DEFAULT; + } } } diff --git a/fluss-common/src/test/java/org/apache/fluss/metadata/AggFunctionTypeTest.java b/fluss-common/src/test/java/org/apache/fluss/metadata/AggFunctionTypeTest.java new file mode 100644 index 0000000000..082cd63f64 --- /dev/null +++ b/fluss-common/src/test/java/org/apache/fluss/metadata/AggFunctionTypeTest.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.metadata; + +import org.junit.jupiter.params.ParameterizedTest; +import org.junit.jupiter.params.provider.EnumSource; + +import static org.assertj.core.api.Assertions.assertThat; + +/** Tests for {@link AggFunctionType}. */ +class AggFunctionTypeTest { + + @ParameterizedTest(name = "{0}.supportsRetract()") + @EnumSource(AggFunctionType.class) + void testSupportsRetract(AggFunctionType type) { + if (type == AggFunctionType.SUM + || type == AggFunctionType.LAST_VALUE + || type == AggFunctionType.LAST_VALUE_IGNORE_NULLS) { + assertThat(type.supportsRetract()).as("%s should support retract", type).isTrue(); + } else { + assertThat(type.supportsRetract()).as("%s should NOT support retract", type).isFalse(); + } + } +} diff --git a/fluss-common/src/test/java/org/apache/fluss/metadata/TableDescriptorTest.java b/fluss-common/src/test/java/org/apache/fluss/metadata/TableDescriptorTest.java index bd99610752..5558b4341e 100644 --- a/fluss-common/src/test/java/org/apache/fluss/metadata/TableDescriptorTest.java +++ b/fluss-common/src/test/java/org/apache/fluss/metadata/TableDescriptorTest.java @@ -388,4 +388,29 @@ void testValidateAggFunctionWithDataType() { AggFunctions.of(AggFunctionType.LAST_VALUE, params).validateDataType(DataTypes.STRING()); AggFunctions.of(AggFunctionType.LISTAGG, params).validateDataType(DataTypes.STRING()); } + + @Test + void testRetractSafeAggFunctionSubset() { + // SUM, LAST_VALUE, LAST_VALUE_IGNORE_NULLS support retract + assertThat(AggFunctions.SUM().getType().supportsRetract()).isTrue(); + assertThat(AggFunctions.SUM().supportsRetract()).isTrue(); + assertThat(AggFunctions.PRODUCT().getType().supportsRetract()).isFalse(); + assertThat(AggFunctions.LAST_VALUE().getType().supportsRetract()).isTrue(); + assertThat(AggFunctions.LAST_VALUE_IGNORE_NULLS().getType().supportsRetract()).isTrue(); + + // All other functions must NOT support retract + assertThat(AggFunctions.MAX().getType().supportsRetract()).isFalse(); + assertThat(AggFunctions.MIN().getType().supportsRetract()).isFalse(); + assertThat(AggFunctions.FIRST_VALUE().getType().supportsRetract()).isFalse(); + assertThat(AggFunctions.FIRST_VALUE_IGNORE_NULLS().getType().supportsRetract()).isFalse(); + assertThat(AggFunctions.LISTAGG().getType().supportsRetract()).isFalse(); + assertThat(AggFunctions.BOOL_AND().getType().supportsRetract()).isFalse(); + assertThat(AggFunctions.BOOL_OR().getType().supportsRetract()).isFalse(); + + // Verify the AggFunction delegating helper is consistent with AggFunctionType + for (AggFunctionType type : AggFunctionType.values()) { + AggFunction func = AggFunctions.of(type, null); + assertThat(func.supportsRetract()).isEqualTo(type.supportsRetract()); + } + } } diff --git a/fluss-common/src/test/java/org/apache/fluss/record/DefaultKvRecordBatchTest.java b/fluss-common/src/test/java/org/apache/fluss/record/DefaultKvRecordBatchTest.java index 6c8b483a0b..9ee117220d 100644 --- a/fluss-common/src/test/java/org/apache/fluss/record/DefaultKvRecordBatchTest.java +++ b/fluss-common/src/test/java/org/apache/fluss/record/DefaultKvRecordBatchTest.java @@ -19,8 +19,10 @@ import org.apache.fluss.memory.UnmanagedPagedOutputView; import org.apache.fluss.metadata.KvFormat; +import org.apache.fluss.row.BinaryString; import org.apache.fluss.row.TestInternalRowGenerator; import org.apache.fluss.row.compacted.CompactedRow; +import org.apache.fluss.row.encode.CompactedRowEncoder; import org.junit.jupiter.api.Test; @@ -76,4 +78,189 @@ KvFormat.COMPACTED, new TestingSchemaGetter(1, DATA1_SCHEMA)))) { builder.close(); } + + @Test + void testNullRowIsNormalizedAsDelete() throws Exception { + KvRecordBatchBuilder builder = + KvRecordBatchBuilder.builder( + schemaId, + Integer.MAX_VALUE, + new UnmanagedPagedOutputView(100), + KvFormat.COMPACTED); + byte[] key = new byte[] {1, 2}; + builder.append(key, null); + + DefaultKvRecordBatch batch = DefaultKvRecordBatch.pointToBytesView(builder.build()); + batch.ensureValid(); + + KvRecord record = + batch.records( + KvRecordReadContext.createReadContext( + KvFormat.COMPACTED, + new TestingSchemaGetter(1, DATA1_SCHEMA))) + .iterator() + .next(); + assertThat(record.getRow()).isNull(); + + builder.close(); + } + + @Test + void testV0BatchDefaultMutationType() throws Exception { + KvRecordBatchBuilder builder = + KvRecordBatchBuilder.builder( + schemaId, + Integer.MAX_VALUE, + new UnmanagedPagedOutputView(100), + KvFormat.COMPACTED); + + CompactedRow row; + try (CompactedRowEncoder writer = new CompactedRowEncoder(baseRowFieldTypes)) { + writer.startNewRow(); + writer.encodeField(0, 1); + writer.encodeField(1, BinaryString.fromString("v0")); + row = writer.finishRow(); + } + + builder.append(new byte[] {1}, row); + builder.append(new byte[] {2}, null); + + DefaultKvRecordBatch batch = DefaultKvRecordBatch.pointToBytesView(builder.build()); + batch.ensureValid(); + + // V0 read context (v2Format=false) + KvRecordReadContext readContext = + KvRecordReadContext.createReadContext( + KvFormat.COMPACTED, new TestingSchemaGetter(1, DATA1_SCHEMA)); + + List records = new ArrayList<>(); + for (KvRecord record : batch.records(readContext)) { + records.add(record); + } + + assertThat(records).hasSize(2); + // Non-null row -> UPSERT + assertThat(records.get(0).getMutationType()).isEqualTo(MutationType.UPSERT); + assertThat(records.get(0).getRow()).isNotNull(); + // Null row -> DELETE + assertThat(records.get(1).getMutationType()).isEqualTo(MutationType.DELETE); + assertThat(records.get(1).getRow()).isNull(); + + builder.close(); + } + + @Test + void testV2BatchMixedMutationTypes() throws Exception { + KvRecordBatchBuilder builder = + KvRecordBatchBuilder.builder( + schemaId, + Integer.MAX_VALUE, + new UnmanagedPagedOutputView(100), + KvFormat.COMPACTED, + true); + + CompactedRow upsertRow; + try (CompactedRowEncoder writer = new CompactedRowEncoder(baseRowFieldTypes)) { + writer.startNewRow(); + writer.encodeField(0, 10); + writer.encodeField(1, BinaryString.fromString("upsert")); + upsertRow = writer.finishRow(); + } + + CompactedRow retractRow; + try (CompactedRowEncoder writer = new CompactedRowEncoder(baseRowFieldTypes)) { + writer.startNewRow(); + writer.encodeField(0, 20); + writer.encodeField(1, BinaryString.fromString("retract")); + retractRow = writer.finishRow(); + } + + byte[] key1 = new byte[] {1}; + byte[] key2 = new byte[] {2}; + byte[] key3 = new byte[] {3}; + + builder.appendV2(MutationType.UPSERT, key1, upsertRow); + builder.appendV2(MutationType.DELETE, key2, null); + builder.appendV2(MutationType.RETRACT, key3, retractRow); + + DefaultKvRecordBatch batch = DefaultKvRecordBatch.pointToBytesView(builder.build()); + batch.ensureValid(); + + assertThat(batch.getRecordCount()).isEqualTo(3); + + // V2 read context + KvRecordReadContext readContext = + KvRecordReadContext.createReadContext( + KvFormat.COMPACTED, new TestingSchemaGetter(1, DATA1_SCHEMA)); + + List records = new ArrayList<>(); + for (KvRecord record : batch.records(readContext)) { + records.add(record); + } + + assertThat(records).hasSize(3); + + // Record 0: UPSERT + assertThat(keyToBytes(records.get(0))).isEqualTo(key1); + assertThat(records.get(0).getRow()).isEqualTo(upsertRow); + assertThat(records.get(0).getMutationType()).isEqualTo(MutationType.UPSERT); + + // Record 1: DELETE + assertThat(keyToBytes(records.get(1))).isEqualTo(key2); + assertThat(records.get(1).getRow()).isNull(); + assertThat(records.get(1).getMutationType()).isEqualTo(MutationType.DELETE); + + // Record 2: RETRACT + assertThat(keyToBytes(records.get(2))).isEqualTo(key3); + assertThat(records.get(2).getRow()).isEqualTo(retractRow); + assertThat(records.get(2).getMutationType()).isEqualTo(MutationType.RETRACT); + + builder.close(); + } + + @Test + void testV2FormatAttributeRoundTrip() throws Exception { + // V2 batch: isV2Format() should be true + KvRecordBatchBuilder v2Builder = + KvRecordBatchBuilder.builder( + schemaId, + Integer.MAX_VALUE, + new UnmanagedPagedOutputView(100), + KvFormat.COMPACTED, + true); + + CompactedRow row; + try (CompactedRowEncoder writer = new CompactedRowEncoder(baseRowFieldTypes)) { + writer.startNewRow(); + writer.encodeField(0, 42); + writer.encodeField(1, BinaryString.fromString("v2test")); + row = writer.finishRow(); + } + + v2Builder.appendV2(MutationType.UPSERT, new byte[] {1}, row); + + DefaultKvRecordBatch v2Batch = DefaultKvRecordBatch.pointToBytesView(v2Builder.build()); + v2Batch.ensureValid(); + assertThat(v2Batch.isV2Format()).isTrue(); + assertThat(v2Batch.getRecordCount()).isEqualTo(1); + + v2Builder.close(); + + // V0 batch: isV2Format() should be false + KvRecordBatchBuilder v0Builder = + KvRecordBatchBuilder.builder( + schemaId, + Integer.MAX_VALUE, + new UnmanagedPagedOutputView(100), + KvFormat.COMPACTED); + + v0Builder.append(new byte[] {2}, row); + + DefaultKvRecordBatch v0Batch = DefaultKvRecordBatch.pointToBytesView(v0Builder.build()); + v0Batch.ensureValid(); + assertThat(v0Batch.isV2Format()).isFalse(); + assertThat(v0Batch.getRecordCount()).isEqualTo(1); + + v0Builder.close(); + } } diff --git a/fluss-common/src/test/java/org/apache/fluss/record/DefaultKvRecordTest.java b/fluss-common/src/test/java/org/apache/fluss/record/DefaultKvRecordTest.java index 8e67742218..90c0172531 100644 --- a/fluss-common/src/test/java/org/apache/fluss/record/DefaultKvRecordTest.java +++ b/fluss-common/src/test/java/org/apache/fluss/record/DefaultKvRecordTest.java @@ -103,7 +103,7 @@ void testWriteToAndReadFromWithRandomData() throws Exception { RowType rowType = TestInternalRowGenerator.createAllRowType(); DataType[] colTypes = rowType.getChildren().toArray(new DataType[0]); - // Test read form. + // Test read from. KvRecord kvRecord = DefaultKvRecord.readFrom( MemorySegment.wrap(outputView.getCopyOfBuffer()), @@ -117,4 +117,117 @@ void testWriteToAndReadFromWithRandomData() throws Exception { // check value assertThat(kvRecord.getRow()).isEqualTo(row); } + + @Test + void testV0RecordDefaultMutationType() throws Exception { + KvRecordReadContext readContext = + KvRecordReadContext.createReadContext( + KvFormat.COMPACTED, new TestingSchemaGetter(1, DATA1_SCHEMA)); + + // V0 record with non-null row should return UPSERT. + CompactedRow row; + try (CompactedRowEncoder writer = new CompactedRowEncoder(baseRowFieldTypes)) { + writer.startNewRow(); + writer.encodeField(0, 42); + writer.encodeField(1, BinaryString.fromString("v0")); + row = writer.finishRow(); + } + byte[] key = new byte[] {1, 2}; + DefaultKvRecord.writeTo(outputView, key, row); + KvRecord kvRecord = + DefaultKvRecord.readFrom( + MemorySegment.wrap(outputView.getCopyOfBuffer()), 0, schemaId, readContext); + assertThat(kvRecord.getMutationType()).isEqualTo(MutationType.UPSERT); + + // V0 record with null row should return DELETE. + outputView = new MemorySegmentOutputView(100); + DefaultKvRecord.writeTo(outputView, key, null); + kvRecord = + DefaultKvRecord.readFrom( + MemorySegment.wrap(outputView.getCopyOfBuffer()), 0, schemaId, readContext); + assertThat(kvRecord.getMutationType()).isEqualTo(MutationType.DELETE); + } + + @Test + void testV2UpsertRoundTrip() throws Exception { + KvRecordReadContext readContext = + KvRecordReadContext.createReadContext( + KvFormat.COMPACTED, new TestingSchemaGetter(1, DATA1_SCHEMA)); + + CompactedRow row; + try (CompactedRowEncoder writer = new CompactedRowEncoder(baseRowFieldTypes)) { + writer.startNewRow(); + writer.encodeField(0, 10); + writer.encodeField(1, BinaryString.fromString("abc")); + row = writer.finishRow(); + } + + byte[] key = new byte[] {1, 2}; + DefaultKvRecord.writeToV2(outputView, MutationType.UPSERT, key, row); + + KvRecord kvRecord = + DefaultKvRecord.readFromV2( + MemorySegment.wrap(outputView.getCopyOfBuffer()), 0, schemaId, readContext); + + // V2 size = V0 size + 1 byte for MutationType + // 4 (length) + 1 (mutationType) + 1 (keyLen varint) + 2 (key) + 6 (row) = 14 + assertThat(kvRecord.getSizeInBytes()).isEqualTo(14); + assertThat(keyToBytes(kvRecord)).isEqualTo(key); + assertThat(kvRecord.getRow()).isEqualTo(row); + assertThat(kvRecord.getMutationType()).isEqualTo(MutationType.UPSERT); + } + + @Test + void testV2DeleteRoundTrip() throws Exception { + KvRecordReadContext readContext = + KvRecordReadContext.createReadContext( + KvFormat.COMPACTED, new TestingSchemaGetter(1, DATA1_SCHEMA)); + + byte[] key = new byte[] {3, 4}; + DefaultKvRecord.writeToV2(outputView, MutationType.DELETE, key, null); + + KvRecord kvRecord = + DefaultKvRecord.readFromV2( + MemorySegment.wrap(outputView.getCopyOfBuffer()), 0, schemaId, readContext); + + // 4 (length) + 1 (mutationType) + 1 (keyLen varint) + 2 (key) = 8 + assertThat(kvRecord.getSizeInBytes()).isEqualTo(8); + assertThat(keyToBytes(kvRecord)).isEqualTo(key); + assertThat(kvRecord.getRow()).isNull(); + assertThat(kvRecord.getMutationType()).isEqualTo(MutationType.DELETE); + } + + @Test + void testV2RetractRoundTrip() throws Exception { + KvRecordReadContext readContext = + KvRecordReadContext.createReadContext( + KvFormat.COMPACTED, new TestingSchemaGetter(1, DATA1_SCHEMA)); + + CompactedRow row; + try (CompactedRowEncoder writer = new CompactedRowEncoder(baseRowFieldTypes)) { + writer.startNewRow(); + writer.encodeField(0, 99); + writer.encodeField(1, BinaryString.fromString("old")); + row = writer.finishRow(); + } + + byte[] key = new byte[] {5, 6}; + DefaultKvRecord.writeToV2(outputView, MutationType.RETRACT, key, row); + + KvRecord kvRecord = + DefaultKvRecord.readFromV2( + MemorySegment.wrap(outputView.getCopyOfBuffer()), 0, schemaId, readContext); + + assertThat(keyToBytes(kvRecord)).isEqualTo(key); + assertThat(kvRecord.getRow()).isEqualTo(row); + assertThat(kvRecord.getMutationType()).isEqualTo(MutationType.RETRACT); + } + + @Test + void testV2SizeOf() { + byte[] key = new byte[] {1, 2}; + // V2 size should be V0 size + 1 (MutationType byte) + assertThat(DefaultKvRecord.sizeOfV2(key, null)) + .isEqualTo(DefaultKvRecord.sizeOf(key, null) + 1); + } } diff --git a/fluss-common/src/test/java/org/apache/fluss/record/KvRecordTestUtils.java b/fluss-common/src/test/java/org/apache/fluss/record/KvRecordTestUtils.java index 679920a215..7becd3e63f 100644 --- a/fluss-common/src/test/java/org/apache/fluss/record/KvRecordTestUtils.java +++ b/fluss-common/src/test/java/org/apache/fluss/record/KvRecordTestUtils.java @@ -82,6 +82,57 @@ public KvRecordBatch ofRecords( builder.close(); return kvRecords; } + + /** + * Build a V2 format batch from a list of (MutationType, KvRecord) entries. V2 records carry + * an explicit MutationType byte per record. + */ + public KvRecordBatch ofV2Records(List entries) throws IOException { + return ofV2Records(entries, NO_WRITER_ID, NO_BATCH_SEQUENCE); + } + + /** + * Build a V2 format batch from a list of (MutationType, KvRecord) entries with writer + * state. + */ + public KvRecordBatch ofV2Records( + List entries, long writeClientId, int batchSequenceId) + throws IOException { + KvRecordBatchBuilder builder = + KvRecordBatchBuilder.builder( + schemaId, + Integer.MAX_VALUE, + new UnmanagedPagedOutputView(100), + KvFormat.COMPACTED, + true); + for (V2RecordEntry entry : entries) { + builder.appendV2( + entry.mutationType, + BytesUtils.toArray(entry.record.getKey()), + entry.record.getRow()); + } + + builder.setWriterState(writeClientId, batchSequenceId); + KvRecordBatch kvRecords = DefaultKvRecordBatch.pointToBytesView(builder.build()); + kvRecords.ensureValid(); + builder.close(); + return kvRecords; + } + } + + /** An entry pairing a {@link MutationType} with a {@link KvRecord} for V2 batch building. */ + public static class V2RecordEntry { + public final MutationType mutationType; + public final KvRecord record; + + public V2RecordEntry(MutationType mutationType, KvRecord record) { + this.mutationType = mutationType; + this.record = record; + } + + public static V2RecordEntry of(MutationType mutationType, KvRecord record) { + return new V2RecordEntry(mutationType, record); + } } /** A factory to create {@link KvRecord} whose key and value is specified by user. */ diff --git a/fluss-common/src/test/java/org/apache/fluss/record/MutationTypeTest.java b/fluss-common/src/test/java/org/apache/fluss/record/MutationTypeTest.java new file mode 100644 index 0000000000..d3ec10c586 --- /dev/null +++ b/fluss-common/src/test/java/org/apache/fluss/record/MutationTypeTest.java @@ -0,0 +1,56 @@ +/* + * 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.junit.jupiter.api.Test; + +import static org.assertj.core.api.Assertions.assertThat; +import static org.assertj.core.api.Assertions.assertThatThrownBy; + +class MutationTypeTest { + + @Test + void testEnumValues() { + assertThat(MutationType.UPSERT.getValue()).isEqualTo((byte) 0); + assertThat(MutationType.DELETE.getValue()).isEqualTo((byte) 1); + assertThat(MutationType.RETRACT.getValue()).isEqualTo((byte) 2); + } + + @Test + void testFromValue() { + assertThat(MutationType.fromValue((byte) 0)).isEqualTo(MutationType.UPSERT); + assertThat(MutationType.fromValue((byte) 1)).isEqualTo(MutationType.DELETE); + assertThat(MutationType.fromValue((byte) 2)).isEqualTo(MutationType.RETRACT); + } + + @Test + void testFromValueUnknownThrows() { + assertThatThrownBy(() -> MutationType.fromValue((byte) 3)) + .isInstanceOf(IllegalArgumentException.class) + .hasMessageContaining("Unknown MutationType"); + assertThatThrownBy(() -> MutationType.fromValue((byte) -1)) + .isInstanceOf(IllegalArgumentException.class); + assertThatThrownBy(() -> MutationType.fromValue(Byte.MAX_VALUE)) + .isInstanceOf(IllegalArgumentException.class); + } + + @Test + void testValuesCount() { + assertThat(MutationType.values()).hasSize(3); + } +} diff --git a/fluss-common/src/test/java/org/apache/fluss/rpc/protocol/MergeModeTest.java b/fluss-common/src/test/java/org/apache/fluss/rpc/protocol/MergeModeTest.java new file mode 100644 index 0000000000..e9e1716ae7 --- /dev/null +++ b/fluss-common/src/test/java/org/apache/fluss/rpc/protocol/MergeModeTest.java @@ -0,0 +1,59 @@ +/* + * 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.rpc.protocol; + +import org.junit.jupiter.api.Test; + +import static org.assertj.core.api.Assertions.assertThat; +import static org.assertj.core.api.Assertions.assertThatThrownBy; + +/** Test for {@link MergeMode}. */ +class MergeModeTest { + + @Test + void testExistingModesUnchanged() { + assertThat(MergeMode.DEFAULT.getValue()).isEqualTo(0); + assertThat(MergeMode.OVERWRITE.getValue()).isEqualTo(1); + assertThat(MergeMode.fromValue(0)).isEqualTo(MergeMode.DEFAULT); + assertThat(MergeMode.fromValue(1)).isEqualTo(MergeMode.OVERWRITE); + } + + @Test + void testFromValueUnknownThrows() { + assertThatThrownBy(() -> MergeMode.fromValue(99)) + .isInstanceOf(IllegalArgumentException.class) + .hasMessageContaining("Unknown MergeMode value: 99"); + assertThatThrownBy(() -> MergeMode.fromValue(-1)) + .isInstanceOf(IllegalArgumentException.class) + .hasMessageContaining("Unknown MergeMode value: -1"); + } + + @Test + void testFromProtoValueUnknownReturnDefault() { + assertThat(MergeMode.fromProtoValue(99)).isEqualTo(MergeMode.DEFAULT); + assertThat(MergeMode.fromProtoValue(-1)).isEqualTo(MergeMode.DEFAULT); + } + + @Test + void testAllValuesRoundTrip() { + for (MergeMode mode : MergeMode.values()) { + assertThat(MergeMode.fromValue(mode.getValue())).isEqualTo(mode); + assertThat(MergeMode.fromProtoValue(mode.getProtoValue())).isEqualTo(mode); + } + } +} diff --git a/fluss-flink/fluss-flink-common/src/main/java/org/apache/fluss/flink/catalog/FlinkTableFactory.java b/fluss-flink/fluss-flink-common/src/main/java/org/apache/fluss/flink/catalog/FlinkTableFactory.java index 77f7909f24..c4456a4678 100644 --- a/fluss-flink/fluss-flink-common/src/main/java/org/apache/fluss/flink/catalog/FlinkTableFactory.java +++ b/fluss-flink/fluss-flink-common/src/main/java/org/apache/fluss/flink/catalog/FlinkTableFactory.java @@ -29,6 +29,7 @@ import org.apache.fluss.flink.source.FlinkTableSource; import org.apache.fluss.flink.source.reader.LeaseContext; import org.apache.fluss.flink.utils.FlinkConnectorOptionsUtils; +import org.apache.fluss.flink.utils.FlinkConversions; import org.apache.fluss.metadata.MergeEngineType; import org.apache.fluss.metadata.TablePath; @@ -192,6 +193,15 @@ public DynamicTableSink createDynamicTableSink(Context context) { validateDistributionModeForMergeEngine(mergeEngineType, distributionMode); } + org.apache.flink.configuration.Configuration resolvedOptions = + org.apache.flink.configuration.Configuration.fromMap( + resolvedCatalogTable.getOptions()); + // Compute retract capability once so planner-time gating and runtime writer configuration + // use the same effective aggregation functions. + Set nonRetractableColumns = + FlinkConversions.computeNonRetractableAggregationColumns( + resolvedCatalogTable.getResolvedSchema(), resolvedOptions); + return new FlinkTableSink( toFlussTablePath(context.getObjectIdentifier()), toFlussClientConfig( @@ -207,7 +217,8 @@ public DynamicTableSink createDynamicTableSink(Context context) { tableOptions.get(FlinkConnectorOptions.BUCKET_NUMBER), getBucketKeys(tableOptions), distributionMode, - tableOptions.getOptional(FlinkConnectorOptions.SINK_PRODUCER_ID).orElse(null)); + tableOptions.getOptional(FlinkConnectorOptions.SINK_PRODUCER_ID).orElse(null), + nonRetractableColumns); } @Override diff --git a/fluss-flink/fluss-flink-common/src/main/java/org/apache/fluss/flink/row/OperationType.java b/fluss-flink/fluss-flink-common/src/main/java/org/apache/fluss/flink/row/OperationType.java index d203b40c64..06d36a14c8 100644 --- a/fluss-flink/fluss-flink-common/src/main/java/org/apache/fluss/flink/row/OperationType.java +++ b/fluss-flink/fluss-flink-common/src/main/java/org/apache/fluss/flink/row/OperationType.java @@ -17,22 +17,7 @@ package org.apache.fluss.flink.row; -/** - * Enumeration of row operation types used in Fluss-Flink data processing. - * - *

This enum represents the type of operation associated with a row, such as an append (insert), - * upsert (update or insert), delete, or ignore. It is used to indicate how a row should be - * interpreted or processed in downstream systems. - * - *

    - *
  • {@link #APPEND} - Represents an append-only (insert) operation. - *
  • {@link #UPSERT} - Represents an upsert operation (update or insert). - *
  • {@link #DELETE} - Represents a delete operation. - *
  • {@link #IGNORE} - Represents an operation that should be ignored. - *
- * - * @see org.apache.fluss.flink.row.RowWithOp - */ +/** Enumeration of row operation types used in Fluss-Flink data processing. */ public enum OperationType { /** Represents an append-only (insert) operation. */ APPEND, @@ -43,6 +28,9 @@ public enum OperationType { /** Represents a delete operation. */ DELETE, + /** Represents a retract operation for aggregation merge engine tables. */ + RETRACT, + /** Represents an operation that should be ignored. */ IGNORE } diff --git a/fluss-flink/fluss-flink-common/src/main/java/org/apache/fluss/flink/sink/FlinkSink.java b/fluss-flink/fluss-flink-common/src/main/java/org/apache/fluss/flink/sink/FlinkSink.java index 159b42764c..885b691e7a 100644 --- a/fluss-flink/fluss-flink-common/src/main/java/org/apache/fluss/flink/sink/FlinkSink.java +++ b/fluss-flink/fluss-flink-common/src/main/java/org/apache/fluss/flink/sink/FlinkSink.java @@ -251,6 +251,7 @@ static class UpsertSinkWriterBuilder private final FlussSerializationSchema flussSerializationSchema; private final boolean enableUndoRecovery; @Nullable private final String producerId; + private final boolean schemaSupportsRetract; /** * Optional context for reporting offsets to the upstream UndoRecoveryOperator. @@ -276,7 +277,8 @@ static class UpsertSinkWriterBuilder DistributionMode distributionMode, FlussSerializationSchema flussSerializationSchema, boolean enableUndoRecovery, - @Nullable String producerId) { + @Nullable String producerId, + boolean schemaSupportsRetract) { this.tablePath = tablePath; this.flussConfig = flussConfig; this.tableRowType = tableRowType; @@ -289,6 +291,7 @@ static class UpsertSinkWriterBuilder this.flussSerializationSchema = flussSerializationSchema; this.enableUndoRecovery = enableUndoRecovery; this.producerId = producerId; + this.schemaSupportsRetract = schemaSupportsRetract; } @Override @@ -300,7 +303,8 @@ public UpsertSinkWriter createWriter(MailboxExecutor mailboxExecutor) { targetColumnIndexes, mailboxExecutor, flussSerializationSchema, - offsetReporter); + offsetReporter, + schemaSupportsRetract); } @Override diff --git a/fluss-flink/fluss-flink-common/src/main/java/org/apache/fluss/flink/sink/FlinkTableSink.java b/fluss-flink/fluss-flink-common/src/main/java/org/apache/fluss/flink/sink/FlinkTableSink.java index 8035028dec..5d3328066a 100644 --- a/fluss-flink/fluss-flink-common/src/main/java/org/apache/fluss/flink/sink/FlinkTableSink.java +++ b/fluss-flink/fluss-flink-common/src/main/java/org/apache/fluss/flink/sink/FlinkTableSink.java @@ -19,6 +19,7 @@ import org.apache.fluss.config.Configuration; import org.apache.fluss.flink.sink.serializer.RowDataSerializationSchema; +import org.apache.fluss.flink.sink.serializer.SinkOperationMode; import org.apache.fluss.flink.sink.shuffle.DistributionMode; import org.apache.fluss.flink.sink.writer.FlinkSinkWriter; import org.apache.fluss.flink.utils.PushdownUtils; @@ -87,6 +88,23 @@ public class FlinkTableSink private final @Nullable DataLakeFormat lakeFormat; @Nullable private final String producerId; + /** + * Whether the table uses the AGGREGATION merge engine. Used to enable undo recovery for + * aggregation tables. Note: this does NOT control UPDATE_BEFORE admission — that is gated by + * {@link #schemaSupportsRetract}, which additionally requires all non-PK aggregation columns to + * support retract. + */ + private final boolean isAggregationTable; + + /** + * Whether all non-PK aggregation columns in the full schema support retract. This is the + * planner-time safety gate for accepting update changelog into an aggregation table. + */ + private final boolean schemaSupportsRetract; + + /** Non-PK aggregation columns whose effective aggregation function is not retract-capable. */ + private final Set nonRetractableColumns; + private boolean appliedUpdates = false; @Nullable private GenericRow deleteRow; @@ -104,7 +122,8 @@ public FlinkTableSink( int numBucket, List bucketKeys, DistributionMode distributionMode, - @Nullable String producerId) { + @Nullable String producerId, + Set nonRetractableColumns) { this.tablePath = tablePath; this.flussConfig = flussConfig; this.tableRowType = tableRowType; @@ -119,27 +138,46 @@ public FlinkTableSink( this.distributionMode = distributionMode; this.lakeFormat = lakeFormat; this.producerId = producerId; + this.isAggregationTable = mergeEngineType == MergeEngineType.AGGREGATION; + this.nonRetractableColumns = new HashSet<>(nonRetractableColumns); + this.schemaSupportsRetract = isAggregationTable && this.nonRetractableColumns.isEmpty(); } @Override public ChangelogMode getChangelogMode(ChangelogMode requestedMode) { if (!streaming) { return ChangelogMode.insertOnly(); - } else { - if (primaryKeyIndexes.length > 0 || sinkIgnoreDelete) { - // primary-key table or ignore_delete mode can accept RowKind.DELETE - ChangelogMode.Builder builder = ChangelogMode.newBuilder(); - for (RowKind kind : requestedMode.getContainedKinds()) { - // optimize out the update_before messages - if (kind != RowKind.UPDATE_BEFORE) { - builder.addContainedKind(kind); - } + } + + if (isAggregationTable + && !schemaSupportsRetract + && containsUpdateChangelog(requestedMode)) { + throw new ValidationException( + String.format( + "Table %s uses the 'aggregation' merge engine with non-retractable aggregation columns %s. " + + "It cannot consume UPDATE changelog safely. Please use append-only input or change those columns to retract-capable aggregate functions.", + tablePath, sortedNonRetractable())); + } + + if (primaryKeyIndexes.length > 0 || sinkIgnoreDelete) { + // Accept UPDATE_BEFORE in two cases: + // 1. writerSupportsRetract=true: retract-then-aggregate semantics + // 2. sink.ignore-delete=true on non-aggregation tables: serializer drops UPDATE_BEFORE + // and DELETE as IGNORE + if (schemaSupportsRetract || sinkIgnoreDelete) { + return requestedMode; + } + // primary-key table without retract or ignore-delete: strip UPDATE_BEFORE + ChangelogMode.Builder builder = ChangelogMode.newBuilder(); + for (RowKind kind : requestedMode.getContainedKinds()) { + if (kind != RowKind.UPDATE_BEFORE) { + builder.addContainedKind(kind); } - return builder.build(); - } else { - return ChangelogMode.insertOnly(); } + return builder.build(); } + + return ChangelogMode.insertOnly(); } @Override @@ -208,7 +246,8 @@ public DataStreamSink consumeDataStream( private FlinkSink getFlinkSink(int[] targetColumnIndexes) { // Enable undo recovery for aggregation tables - boolean enableUndoRecovery = mergeEngineType == MergeEngineType.AGGREGATION; + boolean enableUndoRecovery = isAggregationTable; + boolean writerSupportsRetract = supportsRetract(targetColumnIndexes); FlinkSink.SinkWriterBuilder flinkSinkWriterBuilder = (primaryKeyIndexes.length > 0) @@ -222,9 +261,14 @@ private FlinkSink getFlinkSink(int[] targetColumnIndexes) { partitionKeys, lakeFormat, distributionMode, - new RowDataSerializationSchema(false, sinkIgnoreDelete), + new RowDataSerializationSchema( + isAggregationTable + ? SinkOperationMode.aggregation( + sinkIgnoreDelete, writerSupportsRetract) + : SinkOperationMode.upsert(sinkIgnoreDelete)), enableUndoRecovery, - producerId) + producerId, + writerSupportsRetract) : new FlinkSink.AppendSinkWriterBuilder<>( tablePath, flussConfig, @@ -234,7 +278,8 @@ private FlinkSink getFlinkSink(int[] targetColumnIndexes) { partitionKeys, lakeFormat, distributionMode, - new RowDataSerializationSchema(true, sinkIgnoreDelete)); + new RowDataSerializationSchema( + SinkOperationMode.appendOnly(sinkIgnoreDelete))); return new FlinkSink<>(flinkSinkWriterBuilder, tablePath); } @@ -247,6 +292,32 @@ private List columns(int[] columnIndexes) { return columns; } + private boolean supportsRetract(@Nullable int[] targetColumnIndexes) { + if (!isAggregationTable) { + return false; + } + if (targetColumnIndexes == null + || targetColumnIndexes.length == tableRowType.getFieldCount()) { + return schemaSupportsRetract; + } + + for (String column : columns(targetColumnIndexes)) { + if (nonRetractableColumns.contains(column)) { + return false; + } + } + return true; + } + + private boolean containsUpdateChangelog(ChangelogMode requestedMode) { + return requestedMode.contains(RowKind.UPDATE_BEFORE) + || requestedMode.contains(RowKind.UPDATE_AFTER); + } + + private List sortedNonRetractable() { + return nonRetractableColumns.stream().sorted().collect(Collectors.toList()); + } + @Override public DynamicTableSink copy() { FlinkTableSink sink = @@ -264,7 +335,8 @@ public DynamicTableSink copy() { numBucket, bucketKeys, distributionMode, - producerId); + producerId, + nonRetractableColumns); sink.appliedUpdates = appliedUpdates; sink.deleteRow = deleteRow; return sink; diff --git a/fluss-flink/fluss-flink-common/src/main/java/org/apache/fluss/flink/sink/FlussSinkBuilder.java b/fluss-flink/fluss-flink-common/src/main/java/org/apache/fluss/flink/sink/FlussSinkBuilder.java index c8808f12f8..0b488d4be4 100644 --- a/fluss-flink/fluss-flink-common/src/main/java/org/apache/fluss/flink/sink/FlussSinkBuilder.java +++ b/fluss-flink/fluss-flink-common/src/main/java/org/apache/fluss/flink/sink/FlussSinkBuilder.java @@ -237,7 +237,8 @@ public FlussSink build() { distributionMode, serializationSchema, enableUndoRecovery, - producerId); + producerId, + false); } else { LOG.info("Initializing Fluss append sink writer ..."); writerBuilder = diff --git a/fluss-flink/fluss-flink-common/src/main/java/org/apache/fluss/flink/sink/serializer/RowDataSerializationSchema.java b/fluss-flink/fluss-flink-common/src/main/java/org/apache/fluss/flink/sink/serializer/RowDataSerializationSchema.java index ac80f938c4..b4e0eada43 100644 --- a/fluss-flink/fluss-flink-common/src/main/java/org/apache/fluss/flink/sink/serializer/RowDataSerializationSchema.java +++ b/fluss-flink/fluss-flink-common/src/main/java/org/apache/fluss/flink/sink/serializer/RowDataSerializationSchema.java @@ -87,6 +87,20 @@ public class RowDataSerializationSchema implements FlussSerializationSchemaSemantic precedence for UPDATE_BEFORE handling (evaluated in order): + * + *
    + *
  1. {@code ignoreDelete} — if true, DELETE and UPDATE_BEFORE are silently dropped + *
  2. {@code appendOnly} — if true, all operations map to APPEND + *
  3. {@code supportsRetract} — if true, UPDATE_BEFORE maps to RETRACT + *
  4. {@code aggregationTable} — if true and retract not supported, UPDATE_BEFORE fails fast + *
+ */ +public final class SinkOperationMode implements Serializable { + private static final long serialVersionUID = 1L; + + private final boolean appendOnly; + private final boolean ignoreDelete; + private final boolean aggregationTable; + private final boolean supportsRetract; + + private SinkOperationMode( + boolean appendOnly, + boolean ignoreDelete, + boolean aggregationTable, + boolean supportsRetract) { + if (supportsRetract && !aggregationTable) { + throw new IllegalArgumentException( + "supportsRetract requires aggregationTable to be true."); + } + this.appendOnly = appendOnly; + this.ignoreDelete = ignoreDelete; + this.aggregationTable = aggregationTable; + this.supportsRetract = supportsRetract; + } + + /** Mode for append-only (log) tables. */ + public static SinkOperationMode appendOnly(boolean ignoreDelete) { + return new SinkOperationMode(true, ignoreDelete, false, false); + } + + /** Mode for upsert (primary key) tables without aggregation. */ + public static SinkOperationMode upsert(boolean ignoreDelete) { + return new SinkOperationMode(false, ignoreDelete, false, false); + } + + /** Mode for aggregation tables. */ + public static SinkOperationMode aggregation(boolean ignoreDelete, boolean supportsRetract) { + return new SinkOperationMode(false, ignoreDelete, true, supportsRetract); + } + + public boolean isAppendOnly() { + return appendOnly; + } + + public boolean isIgnoreDelete() { + return ignoreDelete; + } + + public boolean isAggregationTable() { + return aggregationTable; + } + + public boolean supportsRetract() { + return supportsRetract; + } + + @Override + public String toString() { + return String.format( + "SinkOperationMode{appendOnly=%s, ignoreDelete=%s, aggregationTable=%s, supportsRetract=%s}", + appendOnly, ignoreDelete, aggregationTable, supportsRetract); + } +} diff --git a/fluss-flink/fluss-flink-common/src/main/java/org/apache/fluss/flink/sink/writer/UpsertSinkWriter.java b/fluss-flink/fluss-flink-common/src/main/java/org/apache/fluss/flink/sink/writer/UpsertSinkWriter.java index 73ac142f04..1c2a402e27 100644 --- a/fluss-flink/fluss-flink-common/src/main/java/org/apache/fluss/flink/sink/writer/UpsertSinkWriter.java +++ b/fluss-flink/fluss-flink-common/src/main/java/org/apache/fluss/flink/sink/writer/UpsertSinkWriter.java @@ -17,6 +17,7 @@ package org.apache.fluss.flink.sink.writer; +import org.apache.fluss.annotation.VisibleForTesting; import org.apache.fluss.client.table.writer.DeleteResult; import org.apache.fluss.client.table.writer.TableWriter; import org.apache.fluss.client.table.writer.Upsert; @@ -26,6 +27,7 @@ import org.apache.fluss.flink.row.OperationType; import org.apache.fluss.flink.sink.serializer.FlussSerializationSchema; import org.apache.fluss.flink.sink.undo.ProducerOffsetReporter; +import org.apache.fluss.metadata.TableBucket; import org.apache.fluss.metadata.TablePath; import org.apache.fluss.row.InternalRow; @@ -41,7 +43,7 @@ /** An upsert sink writer or fluss primary key table. */ public class UpsertSinkWriter extends FlinkSinkWriter { - private transient UpsertWriter upsertWriter; + @VisibleForTesting transient UpsertWriter upsertWriter; /** * Optional context for reporting offsets to the upstream UndoRecoveryOperator. @@ -53,6 +55,12 @@ public class UpsertSinkWriter extends FlinkSinkWriter { */ @Nullable private final ProducerOffsetReporter offsetReporter; + /** + * Whether this writer supports RETRACT operations. For aggregation tables this may be enabled + * either for the full schema or for a retract-safe partial-update column subset. + */ + private final boolean schemaSupportsRetract; + /** * Creates a new UpsertSinkWriter with ProducerOffsetReporter for UndoRecoveryOperator * integration. @@ -64,6 +72,8 @@ public class UpsertSinkWriter extends FlinkSinkWriter { * @param mailboxExecutor the mailbox executor for async operations * @param flussSerializationSchema the serialization schema for input records * @param offsetReporter optional reporter for reporting offsets to upstream operator + * @param schemaSupportsRetract whether this writer can use retract semantics for its target + * columns */ public UpsertSinkWriter( TablePath tablePath, @@ -72,7 +82,8 @@ public UpsertSinkWriter( @Nullable int[] targetColumnIndexes, MailboxExecutor mailboxExecutor, FlussSerializationSchema flussSerializationSchema, - @Nullable ProducerOffsetReporter offsetReporter) { + @Nullable ProducerOffsetReporter offsetReporter, + boolean schemaSupportsRetract) { super( tablePath, flussConfig, @@ -81,6 +92,7 @@ public UpsertSinkWriter( mailboxExecutor, flussSerializationSchema); this.offsetReporter = offsetReporter; + this.schemaSupportsRetract = schemaSupportsRetract; } @Override @@ -96,30 +108,36 @@ public void initialize(SinkWriterMetricGroup metricGroup) { @Override CompletableFuture writeRow(OperationType opType, InternalRow internalRow) { - if (opType == OperationType.UPSERT) { + if (opType == OperationType.RETRACT) { + if (!schemaSupportsRetract) { + throw new UnsupportedOperationException( + "Received RETRACT operation but the schema does not support retract. " + + "This should have been rejected at plan time or serialization."); + } + CompletableFuture future = upsertWriter.retract(internalRow); + if (offsetReporter != null) { + return future.thenAccept( + result -> + reportOffsetIfAvailable( + result.getBucket(), result.getLogEndOffset())); + } + return future; + } else if (opType == OperationType.UPSERT) { CompletableFuture future = upsertWriter.upsert(internalRow); - // Report offset to upstream UndoRecoveryOperator if reporter provided if (offsetReporter != null) { return future.thenAccept( - result -> { - if (result.getBucket() != null && result.getLogEndOffset() >= 0) { - offsetReporter.reportOffset( - result.getBucket(), result.getLogEndOffset()); - } - }); + result -> + reportOffsetIfAvailable( + result.getBucket(), result.getLogEndOffset())); } return future; } else if (opType == OperationType.DELETE) { CompletableFuture future = upsertWriter.delete(internalRow); - // Report offset to upstream UndoRecoveryOperator if reporter provided if (offsetReporter != null) { return future.thenAccept( - result -> { - if (result.getBucket() != null && result.getLogEndOffset() >= 0) { - offsetReporter.reportOffset( - result.getBucket(), result.getLogEndOffset()); - } - }); + result -> + reportOffsetIfAvailable( + result.getBucket(), result.getLogEndOffset())); } return future; } else { @@ -127,6 +145,16 @@ CompletableFuture writeRow(OperationType opType, InternalRow internalRow) { } } + /** + * Reports the offset to the upstream UndoRecoveryOperator if the bucket and offset are + * available. + */ + private void reportOffsetIfAvailable(@Nullable TableBucket bucket, long logEndOffset) { + if (offsetReporter != null && bucket != null && logEndOffset >= 0) { + offsetReporter.reportOffset(bucket, logEndOffset); + } + } + @Override public void flush(boolean endOfInput) throws IOException, InterruptedException { upsertWriter.flush(); diff --git a/fluss-flink/fluss-flink-common/src/main/java/org/apache/fluss/flink/utils/FlinkConversions.java b/fluss-flink/fluss-flink-common/src/main/java/org/apache/fluss/flink/utils/FlinkConversions.java index 01f9a9817e..1dea7ac286 100644 --- a/fluss-flink/fluss-flink-common/src/main/java/org/apache/fluss/flink/utils/FlinkConversions.java +++ b/fluss-flink/fluss-flink-common/src/main/java/org/apache/fluss/flink/utils/FlinkConversions.java @@ -24,6 +24,7 @@ import org.apache.fluss.flink.adapter.CatalogTableAdapter; import org.apache.fluss.flink.catalog.FlinkCatalogFactory; import org.apache.fluss.metadata.AggFunction; +import org.apache.fluss.metadata.AggFunctionType; import org.apache.fluss.metadata.DatabaseDescriptor; import org.apache.fluss.metadata.MergeEngineType; import org.apache.fluss.metadata.Schema; @@ -59,10 +60,12 @@ import java.util.Collection; import java.util.Collections; import java.util.HashMap; +import java.util.HashSet; import java.util.List; import java.util.Map; import java.util.Objects; import java.util.Optional; +import java.util.Set; import java.util.stream.Collectors; import static org.apache.flink.table.factories.FactoryUtil.CONNECTOR; @@ -139,6 +142,7 @@ public static CatalogBaseTable toFlinkTable(TableInfo tableInfo) { column.getName(), column.getAggFunction().get(), newOptions); } } + List physicalColumns = schema.getColumnNames(); int columnCount = physicalColumns.size() @@ -660,6 +664,47 @@ private static CatalogMaterializedTable toFlinkMaterializedTable( return builder.build(); } + /** + * Compute non-PK aggregation columns whose effective aggregation function is not retractable. + * + *

For columns without an explicit {@code fields..agg} option, the server defaults to + * {@link AggFunctionType#LAST_VALUE_IGNORE_NULLS}. This method mirrors that default to keep the + * Flink-side check consistent with the server-side behavior. + */ + public static Set computeNonRetractableAggregationColumns( + ResolvedSchema resolvedSchema, Configuration tableOptions) { + if (!isAggregationMergeEngine(tableOptions)) { + return Collections.emptySet(); + } + + Set nonRetractableColumns = new HashSet<>(); + Set pkSet = + new HashSet<>( + resolvedSchema + .getPrimaryKey() + .map(pk -> pk.getColumns()) + .orElse(Collections.emptyList())); + for (Column column : resolvedSchema.getColumns()) { + if (!column.isPhysical() || pkSet.contains(column.getName())) { + continue; + } + String columnName = column.getName(); + DataType flussDataType = toFlussType(column.getDataType()); + Optional aggFunc = + FlinkAggFunctionParser.parseAggFunction( + columnName, flussDataType, tableOptions); + // Mirror the server default: columns without an explicit agg function + // default to LAST_VALUE_IGNORE_NULLS (see AggregationContext.getAggFunction). + AggFunctionType effectiveType = + aggFunc.map(AggFunction::getType) + .orElse(AggFunctionType.LAST_VALUE_IGNORE_NULLS); + if (!effectiveType.supportsRetract()) { + nonRetractableColumns.add(columnName); + } + } + return Collections.unmodifiableSet(nonRetractableColumns); + } + /** * Check if the table uses aggregation merge engine. * diff --git a/fluss-flink/fluss-flink-common/src/test/java/org/apache/fluss/flink/row/RowWithOpTest.java b/fluss-flink/fluss-flink-common/src/test/java/org/apache/fluss/flink/row/RowWithOpTest.java index 1bf1f01bb9..472ec461fc 100644 --- a/fluss-flink/fluss-flink-common/src/test/java/org/apache/fluss/flink/row/RowWithOpTest.java +++ b/fluss-flink/fluss-flink-common/src/test/java/org/apache/fluss/flink/row/RowWithOpTest.java @@ -105,4 +105,24 @@ void testHashCode_withEqualObjects() { assertThat(rowWithOp1.hashCode()).isEqualTo(rowWithOp2.hashCode()); } + + // --- RowWithOp equality/hashCode with RETRACT --- + + @Test + void testRowWithOpEqualityWithRetract() { + GenericRow row1 = new GenericRow(2); + GenericRow row2 = new GenericRow(2); + RowWithOp a = new RowWithOp(row1, OperationType.RETRACT); + RowWithOp b = new RowWithOp(row2, OperationType.RETRACT); + assertThat(a).isEqualTo(b); + assertThat(a.hashCode()).isEqualTo(b.hashCode()); + } + + @Test + void testRowWithOpRetractNotEqualToDelete() { + InternalRow row = new GenericRow(2); + RowWithOp retractOp = new RowWithOp(row, OperationType.RETRACT); + RowWithOp deleteOp = new RowWithOp(row, OperationType.DELETE); + assertThat(retractOp).isNotEqualTo(deleteOp); + } } diff --git a/fluss-flink/fluss-flink-common/src/test/java/org/apache/fluss/flink/sink/FlinkRowDataChannelComputerTest.java b/fluss-flink/fluss-flink-common/src/test/java/org/apache/fluss/flink/sink/FlinkRowDataChannelComputerTest.java index 93d5254fdd..8c7945b76d 100644 --- a/fluss-flink/fluss-flink-common/src/test/java/org/apache/fluss/flink/sink/FlinkRowDataChannelComputerTest.java +++ b/fluss-flink/fluss-flink-common/src/test/java/org/apache/fluss/flink/sink/FlinkRowDataChannelComputerTest.java @@ -20,6 +20,7 @@ import org.apache.fluss.flink.sink.serializer.FlussSerializationSchema; import org.apache.fluss.flink.sink.serializer.RowDataSerializationSchema; import org.apache.fluss.flink.sink.serializer.SerializerInitContextImpl; +import org.apache.fluss.flink.sink.serializer.SinkOperationMode; import org.apache.flink.table.data.GenericRowData; import org.apache.flink.table.data.RowData; @@ -36,7 +37,7 @@ class FlinkRowDataChannelComputerTest { private static final FlussSerializationSchema serializationSchema = - new RowDataSerializationSchema(false, false); + new RowDataSerializationSchema(SinkOperationMode.upsert(false)); @BeforeAll static void init() throws Exception { diff --git a/fluss-flink/fluss-flink-common/src/test/java/org/apache/fluss/flink/sink/FlinkTableSinkITCase.java b/fluss-flink/fluss-flink-common/src/test/java/org/apache/fluss/flink/sink/FlinkTableSinkITCase.java index 02a8af15f3..74a0c35346 100644 --- a/fluss-flink/fluss-flink-common/src/test/java/org/apache/fluss/flink/sink/FlinkTableSinkITCase.java +++ b/fluss-flink/fluss-flink-common/src/test/java/org/apache/fluss/flink/sink/FlinkTableSinkITCase.java @@ -1472,6 +1472,79 @@ void testPartialUpdateOnAggregationMergeEngine() throws Exception { assertResultsIgnoreOrder(rowIter, expectedRows, true); } + /** + * End-to-end test for retract behavior on aggregation tables. Mirrors Paimon's {@code + * SumRetractionAggregationITCase}: a Flink streaming GROUP BY produces UPDATE_BEFORE rows that + * are mapped to RETRACT operations on the aggregation merge engine. + * + *

Flow: INSERT INTO agg_table SELECT k, SUM(b) FROM pk_input GROUP BY k + * + *

When pk_input is updated, Flink's GROUP BY emits retract (UPDATE_BEFORE) + accumulate + * (UPDATE_AFTER). The aggregation table's merge engine applies retract (subtract old SUM) then + * accumulate (add new SUM), producing the correct final result. + */ + @Test + void testRetractOnAggregationTableEndToEnd() throws Exception { + // Step 1: Create a PK input table (updates produce changelog with UPDATE_BEFORE) + tEnv.executeSql( + "CREATE TABLE retract_input (" + + "k INT NOT NULL, " + + "b BIGINT, " + + "PRIMARY KEY (k) NOT ENFORCED" + + ")"); + + // Step 2: Create an aggregation target table with SUM + tEnv.executeSql( + "CREATE TABLE retract_agg (" + + "k INT NOT NULL, " + + "total BIGINT, " + + "PRIMARY KEY (k) NOT ENFORCED" + + ") WITH (" + + "'table.merge-engine' = 'aggregation', " + + "'fields.total.agg' = 'sum'" + + ")"); + + // Step 3: Start streaming pipeline — GROUP BY produces retract changelog + TableResult pipelineResult = + tEnv.executeSql( + "INSERT INTO retract_agg SELECT k, SUM(b) FROM retract_input GROUP BY k"); + + // Step 4: Start reading changelog from the aggregation table + CloseableIterator rowIter = tEnv.executeSql("SELECT * FROM retract_agg").collect(); + + try { + // Step 5: Insert initial data into the input table + tEnv.executeSql("INSERT INTO retract_input VALUES (1, 10), (2, 20)").await(); + + // Expect: initial aggregation results (+I for each key) + // Flink GROUP BY emits SUM(10)=10 for k=1, SUM(20)=20 for k=2 + // These arrive as UPSERT at the aggregation table → INSERT changelog + List initialExpected = Arrays.asList("+I[1, 10]", "+I[2, 20]"); + assertResultsIgnoreOrder(rowIter, initialExpected, false); + + // Step 6: Update the input table — triggers retract in the GROUP BY + // Updating k=1 from b=10 to b=30: + // Flink GROUP BY emits: UPDATE_BEFORE(k=1, SUM=10), UPDATE_AFTER(k=1, SUM=30) + // The sink sends each record with per-record MutationType (RETRACT / INSERT). + // Server processes: retract 10, then aggregate 30. + // Net effect: total = 10 - 10 + 30 = 30 → changelog: -U[1,10], +U[1,30] + // Updating k=2 from b=20 to b=40: + // Net effect: total = 20 - 20 + 40 = 40 → changelog: -U[2,20], +U[2,40] + tEnv.executeSql("INSERT INTO retract_input VALUES (1, 30), (2, 40)").await(); + + // Expect: each key produces 2 changelog records (UPDATE_BEFORE/UPDATE_AFTER) + // No intermediate states because retract and upsert are processed sequentially. + List updateExpected = + Arrays.asList( + "-U[1, 10]", "+U[1, 30]", + "-U[2, 20]", "+U[2, 40]"); + assertResultsIgnoreOrder(rowIter, updateExpected, false); + } finally { + rowIter.close(); + pipelineResult.getJobClient().ifPresent(JobClient::cancel); + } + } + private InsertAndExpectValues rowsToInsertInto(Collection partitions) { List insertValues = new ArrayList<>(); List expectedValues = new ArrayList<>(); diff --git a/fluss-flink/fluss-flink-common/src/test/java/org/apache/fluss/flink/sink/FlinkTableSinkTest.java b/fluss-flink/fluss-flink-common/src/test/java/org/apache/fluss/flink/sink/FlinkTableSinkTest.java new file mode 100644 index 0000000000..6650c3b52b --- /dev/null +++ b/fluss-flink/fluss-flink-common/src/test/java/org/apache/fluss/flink/sink/FlinkTableSinkTest.java @@ -0,0 +1,133 @@ +/* + * 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.flink.sink; + +import org.apache.fluss.config.Configuration; +import org.apache.fluss.flink.sink.shuffle.DistributionMode; +import org.apache.fluss.metadata.DeleteBehavior; +import org.apache.fluss.metadata.MergeEngineType; +import org.apache.fluss.metadata.TablePath; + +import org.apache.flink.table.connector.ChangelogMode; +import org.apache.flink.types.RowKind; +import org.junit.jupiter.api.Test; + +import java.util.ArrayList; +import java.util.Collections; + +import static org.assertj.core.api.Assertions.assertThat; +import static org.assertj.core.api.Assertions.assertThatThrownBy; + +/** Tests for {@link FlinkTableSink}. */ +class FlinkTableSinkTest { + + @Test + void testGetChangelogModeIncludesUpdateBeforeForRetractCapableAggregationTable() { + FlinkTableSink sink = + createPkTableSink(MergeEngineType.AGGREGATION, /* schemaSupportsRetract= */ true); + ChangelogMode requestedMode = allKindsChangelogMode(); + + ChangelogMode result = sink.getChangelogMode(requestedMode); + + assertThat(result.contains(RowKind.INSERT)).isTrue(); + assertThat(result.contains(RowKind.UPDATE_AFTER)).isTrue(); + assertThat(result.contains(RowKind.DELETE)).isTrue(); + assertThat(result.contains(RowKind.UPDATE_BEFORE)).isTrue(); + } + + @Test + void testGetChangelogModeExcludesUpdateBeforeForNonRetractAggregationTable() { + FlinkTableSink sink = + createPkTableSink(MergeEngineType.AGGREGATION, /* schemaSupportsRetract= */ false); + ChangelogMode requestedMode = allKindsChangelogMode(); + + assertThatThrownBy(() -> sink.getChangelogMode(requestedMode)) + .isInstanceOf(org.apache.flink.table.api.ValidationException.class) + .hasMessageContaining("cannot consume UPDATE changelog safely"); + } + + @Test + void testGetChangelogModeExcludesUpdateBeforeForNonAggregationTable() { + FlinkTableSink sink = createPkTableSink(null, /* schemaSupportsRetract= */ false); + ChangelogMode requestedMode = allKindsChangelogMode(); + + ChangelogMode result = sink.getChangelogMode(requestedMode); + + assertThat(result.contains(RowKind.INSERT)).isTrue(); + assertThat(result.contains(RowKind.UPDATE_AFTER)).isTrue(); + assertThat(result.contains(RowKind.DELETE)).isTrue(); + assertThat(result.contains(RowKind.UPDATE_BEFORE)).isFalse(); + } + + @Test + void testGetChangelogModeRejectsUpdateBeforeForNonRetractAggregationEvenWithIgnoreDelete() { + FlinkTableSink sink = + createPkTableSink( + MergeEngineType.AGGREGATION, + /* schemaSupportsRetract= */ false, + /* sinkIgnoreDelete= */ true); + ChangelogMode requestedMode = allKindsChangelogMode(); + + assertThatThrownBy(() -> sink.getChangelogMode(requestedMode)) + .isInstanceOf(org.apache.flink.table.api.ValidationException.class) + .hasMessageContaining("cannot consume UPDATE changelog safely"); + } + + /** Creates a PK table FlinkTableSink for changelog mode testing. */ + private static FlinkTableSink createPkTableSink( + MergeEngineType mergeEngineType, boolean schemaSupportsRetract) { + return createPkTableSink(mergeEngineType, schemaSupportsRetract, false); + } + + private static FlinkTableSink createPkTableSink( + MergeEngineType mergeEngineType, + boolean schemaSupportsRetract, + boolean sinkIgnoreDelete) { + return new FlinkTableSink( + TablePath.of("test_db", "test_table"), + new Configuration(), + org.apache.flink.table.types.logical.RowType.of( + new org.apache.flink.table.types.logical.LogicalType[] { + new org.apache.flink.table.types.logical.IntType(), + new org.apache.flink.table.types.logical.IntType() + }, + new String[] {"id", "value"}), + new int[] {0}, // primary key on first column + Collections.emptyList(), + true, // streaming + mergeEngineType, + null, // lakeFormat + sinkIgnoreDelete, + DeleteBehavior.IGNORE, + 1, // numBucket + new ArrayList<>(), + DistributionMode.BUCKET, + null, // producerId + schemaSupportsRetract ? Collections.emptySet() : Collections.singleton("value")); + } + + /** Builds a ChangelogMode containing all four RowKinds. */ + private static ChangelogMode allKindsChangelogMode() { + return ChangelogMode.newBuilder() + .addContainedKind(RowKind.INSERT) + .addContainedKind(RowKind.UPDATE_BEFORE) + .addContainedKind(RowKind.UPDATE_AFTER) + .addContainedKind(RowKind.DELETE) + .build(); + } +} diff --git a/fluss-flink/fluss-flink-common/src/test/java/org/apache/fluss/flink/sink/FlussSinkITCase.java b/fluss-flink/fluss-flink-common/src/test/java/org/apache/fluss/flink/sink/FlussSinkITCase.java index 1a5e98eebb..99fc0ac2b2 100644 --- a/fluss-flink/fluss-flink-common/src/test/java/org/apache/fluss/flink/sink/FlussSinkITCase.java +++ b/fluss-flink/fluss-flink-common/src/test/java/org/apache/fluss/flink/sink/FlussSinkITCase.java @@ -26,6 +26,7 @@ import org.apache.fluss.flink.row.RowWithOp; import org.apache.fluss.flink.sink.serializer.FlussSerializationSchema; import org.apache.fluss.flink.sink.serializer.RowDataSerializationSchema; +import org.apache.fluss.flink.sink.serializer.SinkOperationMode; import org.apache.fluss.flink.utils.FlinkTestBase; import org.apache.fluss.flink.utils.FlussRowToFlinkRowConverter; import org.apache.fluss.metadata.Schema; @@ -132,7 +133,7 @@ public void testRowDataTablePKSink() throws Exception { inputRows.add(row7); RowDataSerializationSchema serializationSchema = - new RowDataSerializationSchema(false, true); + new RowDataSerializationSchema(SinkOperationMode.upsert(true)); DataStream stream = env.fromData(inputRows); @@ -205,7 +206,8 @@ public void testRowDataTableLogSink() throws Exception { inputRows.add(row4); inputRows.add(row5); - RowDataSerializationSchema serializationSchema = new RowDataSerializationSchema(true, true); + RowDataSerializationSchema serializationSchema = + new RowDataSerializationSchema(SinkOperationMode.appendOnly(true)); DataStream stream = env.fromData(inputRows); @@ -449,7 +451,7 @@ public void testPartialUpdateWithTwoWritersWithRD() throws Exception { StreamExecutionEnvironment insertEnv = StreamExecutionEnvironment.getExecutionEnvironment(); DataStream insertStream = insertEnv.fromData(inserts); RowDataSerializationSchema insertSerializationSchema = - new RowDataSerializationSchema(false, true); + new RowDataSerializationSchema(SinkOperationMode.upsert(true)); FlinkSink insertSink = FlussSink.builder() .setBootstrapServers(bootstrapServers) @@ -469,7 +471,7 @@ public void testPartialUpdateWithTwoWritersWithRD() throws Exception { DataStream streamWriter2 = updatesEnv.fromData(updatesWriter2); RowDataSerializationSchema updatesSerializationSchema = - new RowDataSerializationSchema(false, true); + new RowDataSerializationSchema(SinkOperationMode.upsert(true)); // Partial update sink 1: orderId + address FlinkSink partialSink1 = @@ -582,7 +584,7 @@ public void testPartialUpdateSingleWriterNullRemainder() throws Exception { StreamExecutionEnvironment insertEnv = StreamExecutionEnvironment.getExecutionEnvironment(); DataStream insertStream = insertEnv.fromData(inserts); RowDataSerializationSchema insertSerializationSchema = - new RowDataSerializationSchema(false, true); + new RowDataSerializationSchema(SinkOperationMode.upsert(true)); FlinkSink insertSink = FlussSink.builder() .setBootstrapServers(bootstrapServers) @@ -597,7 +599,7 @@ public void testPartialUpdateSingleWriterNullRemainder() throws Exception { StreamExecutionEnvironment updateEnv = StreamExecutionEnvironment.getExecutionEnvironment(); DataStream updateStream = updateEnv.fromData(updates); RowDataSerializationSchema updateSerializationSchema = - new RowDataSerializationSchema(false, true); + new RowDataSerializationSchema(SinkOperationMode.upsert(true)); FlinkSink partialSink = FlussSink.builder() .setBootstrapServers(bootstrapServers) diff --git a/fluss-flink/fluss-flink-common/src/test/java/org/apache/fluss/flink/sink/UndoRecoveryITCase.java b/fluss-flink/fluss-flink-common/src/test/java/org/apache/fluss/flink/sink/UndoRecoveryITCase.java index df023ed7a7..2ceec87f6e 100644 --- a/fluss-flink/fluss-flink-common/src/test/java/org/apache/fluss/flink/sink/UndoRecoveryITCase.java +++ b/fluss-flink/fluss-flink-common/src/test/java/org/apache/fluss/flink/sink/UndoRecoveryITCase.java @@ -25,6 +25,7 @@ import org.apache.fluss.client.table.Table; import org.apache.fluss.config.ConfigOptions; import org.apache.fluss.flink.sink.serializer.RowDataSerializationSchema; +import org.apache.fluss.flink.sink.serializer.SinkOperationMode; import org.apache.fluss.flink.sink.testutils.CountingSource; import org.apache.fluss.flink.sink.testutils.FailingCountingSource; import org.apache.fluss.metadata.TablePath; @@ -821,7 +822,8 @@ private JobClient startBoundedJob( .setBootstrapServers(bootstrapServers) .setDatabase(tablePath.getDatabaseName()) .setTable(tablePath.getTableName()) - .setSerializationSchema(new RowDataSerializationSchema(false, true)); + .setSerializationSchema( + new RowDataSerializationSchema(SinkOperationMode.upsert(true))); if (producerId != null) { sinkBuilder.setProducerId(producerId); } @@ -890,7 +892,8 @@ private JobClient startFailoverJob( .setBootstrapServers(bootstrapServers) .setDatabase(tablePath.getDatabaseName()) .setTable(tablePath.getTableName()) - .setSerializationSchema(new RowDataSerializationSchema(false, true)); + .setSerializationSchema( + new RowDataSerializationSchema(SinkOperationMode.upsert(true))); if (producerId != null) { sinkBuilder.setProducerId(producerId); } diff --git a/fluss-flink/fluss-flink-common/src/test/java/org/apache/fluss/flink/sink/serializer/RowDataSerializationSchemaTest.java b/fluss-flink/fluss-flink-common/src/test/java/org/apache/fluss/flink/sink/serializer/RowDataSerializationSchemaTest.java new file mode 100644 index 0000000000..4f3ba79e48 --- /dev/null +++ b/fluss-flink/fluss-flink-common/src/test/java/org/apache/fluss/flink/sink/serializer/RowDataSerializationSchemaTest.java @@ -0,0 +1,202 @@ +/* + * 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.flink.sink.serializer; + +import org.apache.fluss.flink.row.OperationType; +import org.apache.fluss.flink.row.RowWithOp; +import org.apache.fluss.types.IntType; +import org.apache.fluss.types.RowType; + +import org.apache.flink.table.data.GenericRowData; +import org.apache.flink.types.RowKind; +import org.junit.jupiter.api.Test; + +import static org.assertj.core.api.Assertions.assertThat; +import static org.assertj.core.api.Assertions.assertThatThrownBy; + +/** Tests for {@link RowDataSerializationSchema}. */ +class RowDataSerializationSchemaTest { + + @Test + void testUpdateBeforeMapsToRetractWhenRetractCompatible() throws Exception { + RowDataSerializationSchema schema = + new RowDataSerializationSchema(SinkOperationMode.aggregation(false, true)); + schema.open(new TestInitializationContext()); + + GenericRowData rowData = new GenericRowData(2); + rowData.setField(0, 1); + rowData.setField(1, 2); + rowData.setRowKind(RowKind.UPDATE_BEFORE); + + RowWithOp result = schema.serialize(rowData); + assertThat(result.getOperationType()).isEqualTo(OperationType.RETRACT); + } + + @Test + void testDeleteStillMapsToDeleteWhenRetractCompatible() throws Exception { + RowDataSerializationSchema schema = + new RowDataSerializationSchema(SinkOperationMode.aggregation(false, true)); + schema.open(new TestInitializationContext()); + + GenericRowData rowData = new GenericRowData(2); + rowData.setField(0, 1); + rowData.setField(1, 2); + rowData.setRowKind(RowKind.DELETE); + + RowWithOp result = schema.serialize(rowData); + assertThat(result.getOperationType()).isEqualTo(OperationType.DELETE); + } + + @Test + void testUpdateBeforeMapsToDeleteWhenNotRetractCompatible() throws Exception { + RowDataSerializationSchema schema = + new RowDataSerializationSchema(SinkOperationMode.upsert(false)); + schema.open(new TestInitializationContext()); + + GenericRowData rowData = new GenericRowData(2); + rowData.setField(0, 1); + rowData.setField(1, 2); + rowData.setRowKind(RowKind.UPDATE_BEFORE); + + RowWithOp result = schema.serialize(rowData); + assertThat(result.getOperationType()).isEqualTo(OperationType.DELETE); + } + + @Test + void testUpdateBeforeFailsFastForAggregationTableWithoutRetract() throws Exception { + RowDataSerializationSchema schema = + new RowDataSerializationSchema(SinkOperationMode.aggregation(false, false)); + schema.open(new TestInitializationContext()); + + GenericRowData rowData = new GenericRowData(2); + rowData.setField(0, 1); + rowData.setField(1, 2); + rowData.setRowKind(RowKind.UPDATE_BEFORE); + + assertThatThrownBy(() -> schema.serialize(rowData)) + .isInstanceOf(UnsupportedOperationException.class) + .hasMessageContaining("does not support retract"); + } + + @Test + void testNonAggregationTableKeepsCurrentBehavior() throws Exception { + RowDataSerializationSchema schema = + new RowDataSerializationSchema(SinkOperationMode.upsert(false)); + schema.open(new TestInitializationContext()); + + // INSERT -> UPSERT + GenericRowData insert = new GenericRowData(2); + insert.setField(0, 1); + insert.setField(1, 2); + insert.setRowKind(RowKind.INSERT); + assertThat(schema.serialize(insert).getOperationType()).isEqualTo(OperationType.UPSERT); + + // UPDATE_AFTER -> UPSERT + GenericRowData updateAfter = new GenericRowData(2); + updateAfter.setField(0, 1); + updateAfter.setField(1, 2); + updateAfter.setRowKind(RowKind.UPDATE_AFTER); + assertThat(schema.serialize(updateAfter).getOperationType()) + .isEqualTo(OperationType.UPSERT); + + // DELETE -> DELETE + GenericRowData delete = new GenericRowData(2); + delete.setField(0, 1); + delete.setField(1, 2); + delete.setRowKind(RowKind.DELETE); + assertThat(schema.serialize(delete).getOperationType()).isEqualTo(OperationType.DELETE); + + // UPDATE_BEFORE -> DELETE (not RETRACT) + GenericRowData updateBefore = new GenericRowData(2); + updateBefore.setField(0, 1); + updateBefore.setField(1, 2); + updateBefore.setRowKind(RowKind.UPDATE_BEFORE); + assertThat(schema.serialize(updateBefore).getOperationType()) + .isEqualTo(OperationType.DELETE); + } + + @Test + void testInsertAndUpdateAfterStillMapToUpsertWhenAggregationTable() throws Exception { + RowDataSerializationSchema schema = + new RowDataSerializationSchema(SinkOperationMode.aggregation(false, true)); + schema.open(new TestInitializationContext()); + + // INSERT -> UPSERT even when isAggregationTable=true + GenericRowData insert = new GenericRowData(2); + insert.setField(0, 1); + insert.setField(1, 2); + insert.setRowKind(RowKind.INSERT); + assertThat(schema.serialize(insert).getOperationType()).isEqualTo(OperationType.UPSERT); + + // UPDATE_AFTER -> UPSERT even when isAggregationTable=true + GenericRowData updateAfter = new GenericRowData(2); + updateAfter.setField(0, 1); + updateAfter.setField(1, 2); + updateAfter.setRowKind(RowKind.UPDATE_AFTER); + assertThat(schema.serialize(updateAfter).getOperationType()) + .isEqualTo(OperationType.UPSERT); + } + + @Test + void testIgnoreDeleteOverridesAggregationTable() throws Exception { + // When ignoreDelete=true, UPDATE_BEFORE should be IGNORE even if aggregation table + RowDataSerializationSchema schema = + new RowDataSerializationSchema(SinkOperationMode.aggregation(true, true)); + schema.open(new TestInitializationContext()); + + GenericRowData rowData = new GenericRowData(2); + rowData.setField(0, 1); + rowData.setField(1, 2); + rowData.setRowKind(RowKind.UPDATE_BEFORE); + + RowWithOp result = schema.serialize(rowData); + assertThat(result.getOperationType()).isEqualTo(OperationType.IGNORE); + } + + /** + * Minimal InitializationContext for unit tests that don't need schema evolution or statistics. + */ + static class TestInitializationContext + implements FlussSerializationSchema.InitializationContext { + + private static final RowType FLUSS_ROW_TYPE = + RowType.of( + new org.apache.fluss.types.DataType[] {new IntType(), new IntType()}, + new String[] {"a", "b"}); + + private static final org.apache.flink.table.types.logical.RowType FLINK_ROW_TYPE = + org.apache.flink.table.types.logical.RowType.of( + new org.apache.flink.table.types.logical.IntType(), + new org.apache.flink.table.types.logical.IntType()); + + @Override + public RowType getRowSchema() { + return FLUSS_ROW_TYPE; + } + + @Override + public org.apache.flink.table.types.logical.RowType getInputRowSchema() { + return FLINK_ROW_TYPE; + } + + @Override + public boolean isStatisticEnabled() { + return false; + } + } +} diff --git a/fluss-flink/fluss-flink-common/src/test/java/org/apache/fluss/flink/sink/serializer/SinkOperationModeTest.java b/fluss-flink/fluss-flink-common/src/test/java/org/apache/fluss/flink/sink/serializer/SinkOperationModeTest.java new file mode 100644 index 0000000000..6ebc4ba9a3 --- /dev/null +++ b/fluss-flink/fluss-flink-common/src/test/java/org/apache/fluss/flink/sink/serializer/SinkOperationModeTest.java @@ -0,0 +1,97 @@ +/* + * 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.flink.sink.serializer; + +import org.junit.jupiter.api.Test; + +import static org.assertj.core.api.Assertions.assertThat; + +/** Tests for {@link SinkOperationMode}. */ +class SinkOperationModeTest { + + @Test + void testAppendOnlyMode() { + SinkOperationMode mode = SinkOperationMode.appendOnly(false); + assertThat(mode.isAppendOnly()).isTrue(); + assertThat(mode.isIgnoreDelete()).isFalse(); + assertThat(mode.isAggregationTable()).isFalse(); + assertThat(mode.supportsRetract()).isFalse(); + } + + @Test + void testAppendOnlyModeWithIgnoreDelete() { + SinkOperationMode mode = SinkOperationMode.appendOnly(true); + assertThat(mode.isAppendOnly()).isTrue(); + assertThat(mode.isIgnoreDelete()).isTrue(); + assertThat(mode.isAggregationTable()).isFalse(); + assertThat(mode.supportsRetract()).isFalse(); + } + + @Test + void testUpsertMode() { + SinkOperationMode mode = SinkOperationMode.upsert(false); + assertThat(mode.isAppendOnly()).isFalse(); + assertThat(mode.isIgnoreDelete()).isFalse(); + assertThat(mode.isAggregationTable()).isFalse(); + assertThat(mode.supportsRetract()).isFalse(); + } + + @Test + void testUpsertModeWithIgnoreDelete() { + SinkOperationMode mode = SinkOperationMode.upsert(true); + assertThat(mode.isAppendOnly()).isFalse(); + assertThat(mode.isIgnoreDelete()).isTrue(); + assertThat(mode.isAggregationTable()).isFalse(); + assertThat(mode.supportsRetract()).isFalse(); + } + + @Test + void testAggregationModeWithoutRetract() { + SinkOperationMode mode = SinkOperationMode.aggregation(false, false); + assertThat(mode.isAppendOnly()).isFalse(); + assertThat(mode.isIgnoreDelete()).isFalse(); + assertThat(mode.isAggregationTable()).isTrue(); + assertThat(mode.supportsRetract()).isFalse(); + } + + @Test + void testAggregationModeWithRetract() { + SinkOperationMode mode = SinkOperationMode.aggregation(false, true); + assertThat(mode.isAppendOnly()).isFalse(); + assertThat(mode.isIgnoreDelete()).isFalse(); + assertThat(mode.isAggregationTable()).isTrue(); + assertThat(mode.supportsRetract()).isTrue(); + } + + @Test + void testAggregationModeWithRetractIsValid() { + // aggregation() always sets aggregationTable=true, so this should NOT throw. + SinkOperationMode mode = SinkOperationMode.aggregation(false, true); + assertThat(mode.isAggregationTable()).isTrue(); + assertThat(mode.supportsRetract()).isTrue(); + } + + @Test + void testAggregationModeWithIgnoreDeleteAndRetract() { + SinkOperationMode mode = SinkOperationMode.aggregation(true, true); + assertThat(mode.isAppendOnly()).isFalse(); + assertThat(mode.isIgnoreDelete()).isTrue(); + assertThat(mode.isAggregationTable()).isTrue(); + assertThat(mode.supportsRetract()).isTrue(); + } +} diff --git a/fluss-flink/fluss-flink-common/src/test/java/org/apache/fluss/flink/sink/shuffle/DataStatisticOperatorTest.java b/fluss-flink/fluss-flink-common/src/test/java/org/apache/fluss/flink/sink/shuffle/DataStatisticOperatorTest.java index d8a1c098ce..0d1a7b4ebe 100644 --- a/fluss-flink/fluss-flink-common/src/test/java/org/apache/fluss/flink/sink/shuffle/DataStatisticOperatorTest.java +++ b/fluss-flink/fluss-flink-common/src/test/java/org/apache/fluss/flink/sink/shuffle/DataStatisticOperatorTest.java @@ -18,6 +18,7 @@ package org.apache.fluss.flink.sink.shuffle; import org.apache.fluss.flink.sink.serializer.RowDataSerializationSchema; +import org.apache.fluss.flink.sink.serializer.SinkOperationMode; import org.apache.flink.runtime.operators.coordination.OperatorEvent; import org.apache.flink.streaming.runtime.streamrecord.StreamRecord; @@ -47,7 +48,7 @@ void testProcessElement() throws Exception { new DataStatisticsOperatorFactory<>( DATA1_ROW_TYPE, Collections.singletonList("b"), - new RowDataSerializationSchema(false, false)); + new RowDataSerializationSchema(SinkOperationMode.upsert(false))); List> inputRecords = Arrays.asList( new StreamRecord<>(GenericRowData.of(1, StringData.fromString("a"))), diff --git a/fluss-flink/fluss-flink-common/src/test/java/org/apache/fluss/flink/sink/shuffle/DataStatisticsCoordinatorTest.java b/fluss-flink/fluss-flink-common/src/test/java/org/apache/fluss/flink/sink/shuffle/DataStatisticsCoordinatorTest.java index d9f303b668..3e558913c6 100644 --- a/fluss-flink/fluss-flink-common/src/test/java/org/apache/fluss/flink/sink/shuffle/DataStatisticsCoordinatorTest.java +++ b/fluss-flink/fluss-flink-common/src/test/java/org/apache/fluss/flink/sink/shuffle/DataStatisticsCoordinatorTest.java @@ -18,6 +18,7 @@ package org.apache.fluss.flink.sink.shuffle; import org.apache.fluss.flink.sink.serializer.RowDataSerializationSchema; +import org.apache.fluss.flink.sink.serializer.SinkOperationMode; import org.apache.flink.runtime.executiongraph.ExecutionAttemptID; import org.apache.flink.runtime.executiongraph.ExecutionGraphID; @@ -125,7 +126,7 @@ private DataStatisticsCoordinator getDataStatisticsCoordinator() throws Exceptio new DataStatisticsOperatorFactory( DATA1_ROW_TYPE, Collections.singletonList("b"), - new RowDataSerializationSchema(false, false)); + new RowDataSerializationSchema(SinkOperationMode.upsert(false))); OperatorID operatorID = new OperatorID(); DataStatisticsCoordinatorProvider provider = (DataStatisticsCoordinatorProvider) diff --git a/fluss-flink/fluss-flink-common/src/test/java/org/apache/fluss/flink/sink/shuffle/StatisticsOrRecordChannelComputerTest.java b/fluss-flink/fluss-flink-common/src/test/java/org/apache/fluss/flink/sink/shuffle/StatisticsOrRecordChannelComputerTest.java index c636909517..13cd108f16 100644 --- a/fluss-flink/fluss-flink-common/src/test/java/org/apache/fluss/flink/sink/shuffle/StatisticsOrRecordChannelComputerTest.java +++ b/fluss-flink/fluss-flink-common/src/test/java/org/apache/fluss/flink/sink/shuffle/StatisticsOrRecordChannelComputerTest.java @@ -20,6 +20,7 @@ import org.apache.fluss.flink.sink.serializer.FlussSerializationSchema; import org.apache.fluss.flink.sink.serializer.RowDataSerializationSchema; import org.apache.fluss.flink.sink.serializer.SerializerInitContextImpl; +import org.apache.fluss.flink.sink.serializer.SinkOperationMode; import org.apache.flink.table.data.GenericRowData; import org.apache.flink.table.data.RowData; @@ -46,7 +47,7 @@ /** Unit tests for {@link StatisticsOrRecordChannelComputer}. */ class StatisticsOrRecordChannelComputerTest { private static final FlussSerializationSchema serializationSchema = - new RowDataSerializationSchema(false, false); + new RowDataSerializationSchema(SinkOperationMode.upsert(false)); @BeforeAll static void init() throws Exception { diff --git a/fluss-flink/fluss-flink-common/src/test/java/org/apache/fluss/flink/sink/undo/UndoComputerTest.java b/fluss-flink/fluss-flink-common/src/test/java/org/apache/fluss/flink/sink/undo/UndoComputerTest.java index 8a450bb04f..bb04c2c00d 100644 --- a/fluss-flink/fluss-flink-common/src/test/java/org/apache/fluss/flink/sink/undo/UndoComputerTest.java +++ b/fluss-flink/fluss-flink-common/src/test/java/org/apache/fluss/flink/sink/undo/UndoComputerTest.java @@ -196,4 +196,38 @@ void testUpdateAfterDoesNotBlockSubsequentRecords() { assertThat(mockWriter.getDeleteCount()).isEqualTo(1); assertThat(processedKeys).hasSize(1); } + + // ==================== Undo vs Retract Boundary ==================== + + /** + * Test that undo operations never use retract — they use upsert/delete with OVERWRITE mode. + * + *

This is a critical boundary test: undo recovery restores exact checkpoint state via + * OVERWRITE, while retract applies inverse aggregation via RETRACT. The two mechanisms are + * independent and must not be confused. + */ + @Test + void testUndoNeverCallsRetract() { + Set processedKeys = new HashSet<>(); + + // Process all change types that trigger undo operations + undoComputer.processRecord( + new ScanRecord(0L, 0L, ChangeType.INSERT, row(1, "a", 10)), processedKeys); + processedKeys.clear(); // allow reprocessing for test + + undoComputer.processRecord( + new ScanRecord(1L, 0L, ChangeType.UPDATE_BEFORE, row(2, "b", 20)), processedKeys); + processedKeys.clear(); + + undoComputer.processRecord( + new ScanRecord(2L, 0L, ChangeType.DELETE, row(3, "c", 30)), processedKeys); + + // Undo should only use upsert and delete, never retract + assertThat(mockWriter.getDeleteCount()).isEqualTo(1); // INSERT -> delete + assertThat(mockWriter.getUpsertCount()) + .isEqualTo(2); // UPDATE_BEFORE -> upsert, DELETE -> upsert + assertThat(mockWriter.getRetractCount()) + .as("Undo operations must never call retract") + .isEqualTo(0); + } } diff --git a/fluss-flink/fluss-flink-common/src/test/java/org/apache/fluss/flink/sink/writer/FlinkSinkWriterTest.java b/fluss-flink/fluss-flink-common/src/test/java/org/apache/fluss/flink/sink/writer/FlinkSinkWriterTest.java index 6eebba4197..61afb59946 100644 --- a/fluss-flink/fluss-flink-common/src/test/java/org/apache/fluss/flink/sink/writer/FlinkSinkWriterTest.java +++ b/fluss-flink/fluss-flink-common/src/test/java/org/apache/fluss/flink/sink/writer/FlinkSinkWriterTest.java @@ -24,6 +24,7 @@ import org.apache.fluss.config.Configuration; import org.apache.fluss.exception.NetworkException; import org.apache.fluss.flink.sink.serializer.RowDataSerializationSchema; +import org.apache.fluss.flink.sink.serializer.SinkOperationMode; import org.apache.fluss.flink.utils.FlinkTestBase; import org.apache.fluss.metadata.DatabaseDescriptor; import org.apache.fluss.metadata.Schema; @@ -264,7 +265,7 @@ private FlinkSinkWriter createSinkWriter( new LogicalType[] {new IntType(), new CharType(10)}, new String[] {"id", "name"}); RowDataSerializationSchema serializationSchema = - new RowDataSerializationSchema(true, false); + new RowDataSerializationSchema(SinkOperationMode.appendOnly(false)); return new AppendSinkWriter<>( DEFAULT_SINK_TABLE_PATH, configuration, diff --git a/fluss-flink/fluss-flink-common/src/test/java/org/apache/fluss/flink/sink/writer/UpsertSinkWriterTest.java b/fluss-flink/fluss-flink-common/src/test/java/org/apache/fluss/flink/sink/writer/UpsertSinkWriterTest.java new file mode 100644 index 0000000000..9e30bfa2a0 --- /dev/null +++ b/fluss-flink/fluss-flink-common/src/test/java/org/apache/fluss/flink/sink/writer/UpsertSinkWriterTest.java @@ -0,0 +1,109 @@ +/* + * 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.flink.sink.writer; + +import org.apache.fluss.config.Configuration; +import org.apache.fluss.flink.row.OperationType; +import org.apache.fluss.flink.sink.serializer.RowDataSerializationSchema; +import org.apache.fluss.flink.sink.serializer.SinkOperationMode; +import org.apache.fluss.flink.utils.TestUpsertWriter; +import org.apache.fluss.metadata.TablePath; +import org.apache.fluss.row.GenericRow; + +import org.apache.flink.runtime.metrics.util.InterceptingOperatorMetricGroup; +import org.apache.flink.table.data.RowData; +import org.apache.flink.table.types.logical.IntType; +import org.apache.flink.table.types.logical.RowType; +import org.junit.jupiter.api.Test; + +import static org.assertj.core.api.Assertions.assertThat; +import static org.assertj.core.api.Assertions.assertThatThrownBy; + +/** Tests for {@link UpsertSinkWriter}. */ +class UpsertSinkWriterTest { + + @Test + void testRetractSentDirectlyWithoutBuffering() throws Exception { + UpsertSinkWriter writer = createTestingWriter(true); + TestUpsertWriter mockWriter = getTestUpsertWriter(writer); + + writer.writeRow(OperationType.RETRACT, genericRow(1, 10L)); + + assertThat(mockWriter.getRetractCount()).isEqualTo(1); + assertThat(mockWriter.getUpsertCount()).isEqualTo(0); + } + + @Test + void testRetractOnNonRetractSchemaThrows() throws Exception { + UpsertSinkWriter writer = createTestingWriter(false); + + assertThatThrownBy(() -> writer.writeRow(OperationType.RETRACT, genericRow(1, 10L))) + .isInstanceOf(UnsupportedOperationException.class) + .hasMessageContaining("does not support retract"); + } + + @Test + void testMixedRetractAndUpsertInSequence() throws Exception { + UpsertSinkWriter writer = createTestingWriter(true); + TestUpsertWriter mockWriter = getTestUpsertWriter(writer); + + writer.writeRow(OperationType.RETRACT, genericRow(1, 10L)); + writer.writeRow(OperationType.UPSERT, genericRow(1, 11L)); + + assertThat(mockWriter.getRetractCount()).isEqualTo(1); + assertThat(mockWriter.getUpsertCount()).isEqualTo(1); + } + + private static UpsertSinkWriter createTestingWriter(boolean schemaSupportsRetract) + throws Exception { + UpsertSinkWriter writer = + new UpsertSinkWriter<>( + TablePath.of("test_db", "test_table"), + new Configuration(), + flinkRowTypeForSimplePk(), + null, + new MockWriterInitContext(new InterceptingOperatorMetricGroup()) + .getMailboxExecutor(), + new RowDataSerializationSchema(SinkOperationMode.upsert(false)), + null, + schemaSupportsRetract); + + writer.upsertWriter = new TestUpsertWriter(); + return writer; + } + + private static TestUpsertWriter getTestUpsertWriter(UpsertSinkWriter writer) { + return (TestUpsertWriter) writer.upsertWriter; + } + + private static RowType flinkRowTypeForSimplePk() { + return RowType.of( + new org.apache.flink.table.types.logical.LogicalType[] { + new IntType(), new org.apache.flink.table.types.logical.BigIntType() + }, + new String[] {"id", "value"}); + } + + private static GenericRow genericRow(Object... values) { + GenericRow row = new GenericRow(values.length); + for (int i = 0; i < values.length; i++) { + row.setField(i, values[i]); + } + return row; + } +} diff --git a/fluss-flink/fluss-flink-common/src/test/java/org/apache/fluss/flink/utils/FlinkConversionsTest.java b/fluss-flink/fluss-flink-common/src/test/java/org/apache/fluss/flink/utils/FlinkConversionsTest.java index efc386eb64..31fa96097e 100644 --- a/fluss-flink/fluss-flink-common/src/test/java/org/apache/fluss/flink/utils/FlinkConversionsTest.java +++ b/fluss-flink/fluss-flink-common/src/test/java/org/apache/fluss/flink/utils/FlinkConversionsTest.java @@ -520,6 +520,133 @@ void testAggregationFunctionRoundTrip() { assertThat(convertedFlinkTable.getOptions()).containsAllEntriesOf(options); } + // --- computeNonRetractableAggregationColumns() tests --- + + @Test + void testComputeSchemaSupportsRetractAllRetractable() { + // SUM supports retract — should return true + ResolvedSchema schema = + new ResolvedSchema( + Arrays.asList( + Column.physical( + "id", org.apache.flink.table.api.DataTypes.INT().notNull()), + Column.physical( + "val", org.apache.flink.table.api.DataTypes.BIGINT())), + Collections.emptyList(), + UniqueConstraint.primaryKey("PK_id", Collections.singletonList("id"))); + + Map options = new HashMap<>(); + options.put("table.merge-engine", "aggregation"); + options.put("fields.val.agg", "sum"); + + org.apache.flink.configuration.Configuration conf = + org.apache.flink.configuration.Configuration.fromMap(options); + + assertThat(FlinkConversions.computeNonRetractableAggregationColumns(schema, conf).isEmpty()) + .isTrue(); + } + + @Test + void testComputeSchemaSupportsRetractNonRetractableFunction() { + // MAX does not support retract — should return false + ResolvedSchema schema = + new ResolvedSchema( + Arrays.asList( + Column.physical( + "id", org.apache.flink.table.api.DataTypes.INT().notNull()), + Column.physical("val", org.apache.flink.table.api.DataTypes.INT())), + Collections.emptyList(), + UniqueConstraint.primaryKey("PK_id", Collections.singletonList("id"))); + + Map options = new HashMap<>(); + options.put("table.merge-engine", "aggregation"); + options.put("fields.val.agg", "max"); + + org.apache.flink.configuration.Configuration conf = + org.apache.flink.configuration.Configuration.fromMap(options); + + assertThat(FlinkConversions.computeNonRetractableAggregationColumns(schema, conf).isEmpty()) + .isFalse(); + } + + @Test + void testComputeSchemaSupportsRetractMixedRetractability() { + // SUM (retractable) + MIN (non-retractable) — should return false + ResolvedSchema schema = + new ResolvedSchema( + Arrays.asList( + Column.physical( + "id", org.apache.flink.table.api.DataTypes.INT().notNull()), + Column.physical( + "sum_val", org.apache.flink.table.api.DataTypes.BIGINT()), + Column.physical( + "min_val", org.apache.flink.table.api.DataTypes.INT())), + Collections.emptyList(), + UniqueConstraint.primaryKey("PK_id", Collections.singletonList("id"))); + + Map options = new HashMap<>(); + options.put("table.merge-engine", "aggregation"); + options.put("fields.sum_val.agg", "sum"); + options.put("fields.min_val.agg", "min"); + + org.apache.flink.configuration.Configuration conf = + org.apache.flink.configuration.Configuration.fromMap(options); + + assertThat(FlinkConversions.computeNonRetractableAggregationColumns(schema, conf).isEmpty()) + .isFalse(); + } + + @Test + void testComputeNonRetractableAggregationColumns() { + ResolvedSchema schema = + new ResolvedSchema( + Arrays.asList( + Column.physical( + "id", org.apache.flink.table.api.DataTypes.INT().notNull()), + Column.physical( + "sum_val", org.apache.flink.table.api.DataTypes.BIGINT()), + Column.physical( + "min_val", org.apache.flink.table.api.DataTypes.INT())), + Collections.emptyList(), + UniqueConstraint.primaryKey("PK_id", Collections.singletonList("id"))); + + Map options = new HashMap<>(); + options.put("table.merge-engine", "aggregation"); + options.put("fields.sum_val.agg", "sum"); + options.put("fields.min_val.agg", "min"); + + org.apache.flink.configuration.Configuration conf = + org.apache.flink.configuration.Configuration.fromMap(options); + + assertThat(FlinkConversions.computeNonRetractableAggregationColumns(schema, conf)) + .containsExactlyInAnyOrder("min_val"); + } + + @Test + void testComputeSchemaSupportsRetractImplicitDefault() { + // Non-PK column without explicit agg option: the method mirrors the server + // default (LAST_VALUE_IGNORE_NULLS) which supports retract, so returns true. + ResolvedSchema schema = + new ResolvedSchema( + Arrays.asList( + Column.physical( + "id", org.apache.flink.table.api.DataTypes.INT().notNull()), + Column.physical( + "name", org.apache.flink.table.api.DataTypes.STRING())), + Collections.emptyList(), + UniqueConstraint.primaryKey("PK_id", Collections.singletonList("id"))); + + Map options = new HashMap<>(); + options.put("table.merge-engine", "aggregation"); + // No fields.name.agg specified — implicit default + + org.apache.flink.configuration.Configuration conf = + org.apache.flink.configuration.Configuration.fromMap(options); + + assertThat(FlinkConversions.computeNonRetractableAggregationColumns(schema, conf).isEmpty()) + .isTrue(); + } + /** Test refresh handler for testing purpose. */ public static class TestRefreshHandler implements RefreshHandler { diff --git a/fluss-flink/fluss-flink-common/src/test/java/org/apache/fluss/flink/utils/TestUpsertWriter.java b/fluss-flink/fluss-flink-common/src/test/java/org/apache/fluss/flink/utils/TestUpsertWriter.java index fceaa027f9..2c86c89bc7 100644 --- a/fluss-flink/fluss-flink-common/src/test/java/org/apache/fluss/flink/utils/TestUpsertWriter.java +++ b/fluss-flink/fluss-flink-common/src/test/java/org/apache/fluss/flink/utils/TestUpsertWriter.java @@ -41,6 +41,9 @@ public class TestUpsertWriter implements UpsertWriter { private InternalRow lastDeletedRow; private final List allUpsertedRows = new ArrayList<>(); private final List allDeletedRows = new ArrayList<>(); + private int retractCount = 0; + private InternalRow lastRetractedRow; + private final List allRetractedRows = new ArrayList<>(); @Override public CompletableFuture upsert(InternalRow record) { @@ -55,6 +58,19 @@ public CompletableFuture upsert(InternalRow record) { return CompletableFuture.completedFuture(new UpsertResult(new TableBucket(1L, 0), 0L)); } + @Override + public CompletableFuture retract(InternalRow row) { + if (shouldFail) { + CompletableFuture future = new CompletableFuture<>(); + future.completeExceptionally(new RuntimeException("Simulated write failure")); + return future; + } + retractCount++; + lastRetractedRow = row; + allRetractedRows.add(row); + return CompletableFuture.completedFuture(new UpsertResult(new TableBucket(1L, 0), 0L)); + } + @Override public CompletableFuture delete(InternalRow record) { if (shouldFail) { @@ -101,6 +117,18 @@ public List getAllDeletedRows() { return allDeletedRows; } + public int getRetractCount() { + return retractCount; + } + + public InternalRow getLastRetractedRow() { + return lastRetractedRow; + } + + public List getAllRetractedRows() { + return allRetractedRows; + } + public void setShouldFail(boolean shouldFail) { this.shouldFail = shouldFail; } @@ -108,11 +136,14 @@ public void setShouldFail(boolean shouldFail) { public void reset() { upsertCount = 0; deleteCount = 0; + retractCount = 0; flushCalled = false; shouldFail = false; lastUpsertedRow = null; lastDeletedRow = null; + lastRetractedRow = null; allUpsertedRows.clear(); allDeletedRows.clear(); + allRetractedRows.clear(); } } diff --git a/fluss-rpc/src/main/java/org/apache/fluss/rpc/protocol/ApiKeys.java b/fluss-rpc/src/main/java/org/apache/fluss/rpc/protocol/ApiKeys.java index 605967f67f..5b35a372b3 100644 --- a/fluss-rpc/src/main/java/org/apache/fluss/rpc/protocol/ApiKeys.java +++ b/fluss-rpc/src/main/java/org/apache/fluss/rpc/protocol/ApiKeys.java @@ -48,7 +48,9 @@ public enum ApiKeys { // Version 0: Uses lake's encoder for primary key encoding (legacy behavior). // Version 1: Uses CompactedKeyEncoder for primary key encoding when bucket key differs from // primary key, enabling prefix lookup support. - PUT_KV(1016, 0, 1, PUBLIC), + // Version 2: Adds per-record MutationType byte to KvRecord format, enabling + // UPSERT/DELETE/RETRACT identification at the record level. + PUT_KV(1016, 0, 2, PUBLIC), // Version 0: Uses lake's encoder for primary key encoding (legacy behavior). // Version 1: Uses CompactedKeyEncoder for primary key encoding when bucket key differs from diff --git a/fluss-server/src/main/java/org/apache/fluss/server/kv/KvTablet.java b/fluss-server/src/main/java/org/apache/fluss/server/kv/KvTablet.java index e72428a02f..1458b28e23 100644 --- a/fluss-server/src/main/java/org/apache/fluss/server/kv/KvTablet.java +++ b/fluss-server/src/main/java/org/apache/fluss/server/kv/KvTablet.java @@ -22,6 +22,7 @@ import org.apache.fluss.config.ConfigOptions; import org.apache.fluss.config.Configuration; import org.apache.fluss.exception.DeletionDisabledException; +import org.apache.fluss.exception.InvalidRecordException; import org.apache.fluss.exception.InvalidTableException; import org.apache.fluss.exception.KvStorageException; import org.apache.fluss.exception.SchemaNotExistException; @@ -41,6 +42,7 @@ import org.apache.fluss.record.KvRecord; import org.apache.fluss.record.KvRecordBatch; import org.apache.fluss.record.KvRecordReadContext; +import org.apache.fluss.record.MutationType; import org.apache.fluss.row.BinaryRow; import org.apache.fluss.row.PaddingRow; import org.apache.fluss.row.arrow.ArrowWriterPool; @@ -86,6 +88,7 @@ import java.io.File; import java.io.IOException; import java.util.Collection; +import java.util.Iterator; import java.util.List; import java.util.Map; import java.util.concurrent.Executor; @@ -330,14 +333,30 @@ public TabletState getTabletState() { * Put the KvRecordBatch into the kv storage with default DEFAULT mode. * *

This is a convenience method that calls {@link #putAsLeader(KvRecordBatch, int[], - * MergeMode)} with {@link MergeMode#DEFAULT}. + * MergeMode, short)} with {@link MergeMode#DEFAULT} and apiVersion 0. * * @param kvRecords the kv records to put into * @param targetColumns the target columns to put, null if put all columns */ public LogAppendInfo putAsLeader(KvRecordBatch kvRecords, @Nullable int[] targetColumns) throws Exception { - return putAsLeader(kvRecords, targetColumns, MergeMode.DEFAULT); + return putAsLeader(kvRecords, targetColumns, MergeMode.DEFAULT, (short) 0); + } + + /** + * Put the KvRecordBatch into the kv storage with the given merge mode and default apiVersion 0. + * + *

This is a convenience method that calls {@link #putAsLeader(KvRecordBatch, int[], + * MergeMode, short)} with apiVersion 0 (V0 format). + * + * @param kvRecords the kv records to put into + * @param targetColumns the target columns to put, null if put all columns + * @param mergeMode the merge mode (DEFAULT or OVERWRITE) + */ + public LogAppendInfo putAsLeader( + KvRecordBatch kvRecords, @Nullable int[] targetColumns, MergeMode mergeMode) + throws Exception { + return putAsLeader(kvRecords, targetColumns, mergeMode, (short) 0); } /** @@ -360,9 +379,14 @@ public LogAppendInfo putAsLeader(KvRecordBatch kvRecords, @Nullable int[] target * @param kvRecords the kv records to put into * @param targetColumns the target columns to put, null if put all columns * @param mergeMode the merge mode (DEFAULT or OVERWRITE) + * @param apiVersion the client API version; V2 (apiVersion >= 2) enables per-record + * MutationType */ public LogAppendInfo putAsLeader( - KvRecordBatch kvRecords, @Nullable int[] targetColumns, MergeMode mergeMode) + KvRecordBatch kvRecords, + @Nullable int[] targetColumns, + MergeMode mergeMode, + short apiVersion) throws Exception { return inWriteLock( kvLock, @@ -410,7 +434,8 @@ public LogAppendInfo putAsLeader( currentAutoIncrementUpdater, walBuilder, latestSchemaRow, - logEndOffsetOfPrevBatch); + logEndOffsetOfPrevBatch, + apiVersion); // There will be a situation that these batches of kvRecordBatch have not // generated any CDC logs, for example, when client attempts to delete @@ -464,43 +489,281 @@ private void processKvRecords( AutoIncrementUpdater autoIncrementUpdater, WalBuilder walBuilder, PaddingRow latestSchemaRow, - long startLogOffset) + long startLogOffset, + short apiVersion) throws Exception { long logOffset = startLogOffset; // TODO: reuse the read context and decoder + boolean isV2 = kvRecords.isV2Format(); KvRecordBatch.ReadContext readContext = KvRecordReadContext.createReadContext(kvFormat, schemaGetter); ValueDecoder valueDecoder = new ValueDecoder(schemaGetter, kvFormat); - for (KvRecord kvRecord : kvRecords.records(readContext)) { - byte[] keyBytes = BytesUtils.toArray(kvRecord.getKey()); - KvPreWriteBuffer.Key key = KvPreWriteBuffer.Key.of(keyBytes); - BinaryRow row = kvRecord.getRow(); - BinaryValue currentValue = row == null ? null : new BinaryValue(schemaIdOfNewData, row); + if (!isV2) { + // Fast path for V0/V1 batches: no RETRACT possible, skip peekable wrapper + // to avoid per-record NormalizedKvMutation allocation overhead. + for (KvRecord kvRecord : kvRecords.records(readContext)) { + byte[] keyBytes = BytesUtils.toArray(kvRecord.getKey()); + KvPreWriteBuffer.Key key = KvPreWriteBuffer.Key.of(keyBytes); + BinaryRow row = kvRecord.getRow(); + BinaryValue currentValue = + row == null ? null : new BinaryValue(schemaIdOfNewData, row); + if (currentValue == null) { + logOffset = + processDeletion( + key, + currentMerger, + valueDecoder, + walBuilder, + latestSchemaRow, + logOffset); + } else { + logOffset = + processUpsert( + key, + currentValue, + currentMerger, + autoIncrementUpdater, + valueDecoder, + walBuilder, + latestSchemaRow, + logOffset); + } + } + return; + } - if (currentValue == null) { - logOffset = - processDeletion( - key, - currentMerger, - valueDecoder, - walBuilder, - latestSchemaRow, - logOffset); + // V2 path: uses PeekableKvMutationIterator for RETRACT+UPSERT pairing optimization + Iterator iter = kvRecords.records(readContext).iterator(); + PeekableKvMutationIterator peekableIter = + new PeekableKvMutationIterator(iter, schemaIdOfNewData); + while (peekableIter.hasNext()) { + KvRecord kvRecord = peekableIter.next(); + NormalizedKvMutation mutation = peekableIter.nextMutation(kvRecord); + MutationType mutationType = kvRecord.getMutationType(); + + switch (mutationType) { + case UPSERT: + { + if (mutation.isDelete()) { + throw new InvalidRecordException( + "V2 UPSERT record must carry a non-null row value."); + } + logOffset = + dispatchMutation( + mutation, + currentMerger, + autoIncrementUpdater, + valueDecoder, + walBuilder, + latestSchemaRow, + logOffset); + break; + } + case DELETE: + { + logOffset = + dispatchMutation( + mutation, + currentMerger, + autoIncrementUpdater, + valueDecoder, + walBuilder, + latestSchemaRow, + logOffset); + break; + } + case RETRACT: + { + if (!currentMerger.supportsRetract()) { + throw new InvalidRecordException( + "RETRACT records are only supported for aggregation merge " + + "engine tables with retract-safe functions."); + } + if (mutation.isDelete()) { + throw new InvalidRecordException( + "RETRACT record must carry a non-null row value."); + } + + // Snapshot key/value from the reusable NormalizedKvMutation to decouple + // from the reusable object's lifecycle during multi-step retract + // processing. + final KvPreWriteBuffer.Key retractKey = mutation.key; + final BinaryValue retractValue = mutation.value; + + byte[] oldValueBytes = getFromBufferOrKv(retractKey); + if (oldValueBytes == null) { + // Retract on non-existent key — nothing to undo, safe to skip. + LOG.debug( + "Retract on non-existent key in kv tablet for {}, " + + "ignoring as the key does not exist.", + tableBucket); + break; + } + + BinaryValue oldValue = valueDecoder.decodeValue(oldValueBytes); + + // Best-effort merge optimization: peek next record for same key + KvRecord nextRecord = peekableIter.peek(); + if (nextRecord != null) { + NormalizedKvMutation nextMutation = peekableIter.peekMutation(); + if (nextMutation.key.equals(retractKey) + && nextRecord.getMutationType() == MutationType.UPSERT + && !nextMutation.isDelete()) { + // Consume the peeked record — it's part of this merged pair + peekableIter.consumePeeked(); + // Merged retract+upsert pair + BinaryValue intermediate = + currentMerger.retract(oldValue, retractValue); + BinaryValue newValue; + if (intermediate == null) { + // Retract fully removed the row (intermediate == null). + // Treat the subsequent upsert as a fresh insert by merging + // with null accumulator — this is the intended semantic. + newValue = currentMerger.merge(null, nextMutation.value); + } else { + newValue = + currentMerger.merge(intermediate, nextMutation.value); + } + + if (newValue == null) { + DeleteBehavior deleteBehavior = currentMerger.deleteBehavior(); + if (deleteBehavior == DeleteBehavior.IGNORE) { + // Merged retract+upsert produced null but delete is + // ignored. + // The old value remains in the KV store — this is + // intentional: + // IGNORE means "do not delete keys", so the pre-retract + // value + // is preserved. Subsequent lookups will return the old + // value. + // No CDC output is emitted for this no-op. + } else if (deleteBehavior == DeleteBehavior.DISABLE) { + throw new DeletionDisabledException( + "Delete operations are disabled for this table." + + " The table.delete.behavior is set to 'disable'."); + } else { + logOffset = + applyDelete( + retractKey, + oldValue, + walBuilder, + latestSchemaRow, + logOffset); + } + } + // Use value equality (not reference equality) because + // retract()+merge() always creates new BinaryValue instances + // even when the result is logically unchanged. + else if (newValue.equals(oldValue)) { + // Merged retract+upsert produced same value as before: + // intentionally no CDC output and no offset consumed. + } else { + logOffset = + applyUpdate( + retractKey, + oldValue, + newValue, + walBuilder, + latestSchemaRow, + logOffset); + } + break; + } + // Non-matching peeked record stays in the iterator for the + // next loop iteration — no need to manually save it. + } + + // Independent retract (no matching upsert follows) + logOffset = + processIndependentRetract( + retractKey, + retractValue, + oldValue, + currentMerger, + walBuilder, + latestSchemaRow, + logOffset); + break; + } + default: + // Defensive: should be unreachable; MutationType is validated + // during deserialization. + throw new InvalidRecordException("Unknown MutationType: " + mutationType); + } + } + } + + /** + * Process an independent retract (not paired with a subsequent upsert). Retracts the given + * value from the old value and applies the result. + */ + private long processIndependentRetract( + KvPreWriteBuffer.Key key, + BinaryValue retractValue, + BinaryValue oldValue, + RowMerger currentMerger, + WalBuilder walBuilder, + PaddingRow latestSchemaRow, + long logOffset) + throws Exception { + BinaryValue intermediate = currentMerger.retract(oldValue, retractValue); + + if (intermediate == null) { + // Retract resulted in removal + DeleteBehavior deleteBehavior = currentMerger.deleteBehavior(); + if (deleteBehavior == DeleteBehavior.IGNORE) { + // Retract produced null but delete is ignored. + // The old value remains in the KV store — this is intentional: + // IGNORE means "do not delete keys", so the pre-retract value + // is preserved. Subsequent lookups will return the old value. + return logOffset; + } else if (deleteBehavior == DeleteBehavior.DISABLE) { + throw new DeletionDisabledException( + "Delete operations are disabled for this table." + + " The table.delete.behavior is set to 'disable'."); } else { - logOffset = - processUpsert( - key, - currentValue, - currentMerger, - autoIncrementUpdater, - valueDecoder, - walBuilder, - latestSchemaRow, - logOffset); + return applyDelete(key, oldValue, walBuilder, latestSchemaRow, logOffset); } } + // Use value equality (not reference equality) because retract()+merge() always + // creates new BinaryValue instances even when the result is logically unchanged. + else if (intermediate.equals(oldValue)) { + return logOffset; // no change, no offset consumed + } else { + return applyUpdate(key, oldValue, intermediate, walBuilder, latestSchemaRow, logOffset); + } + } + + private long dispatchMutation( + NormalizedKvMutation mutation, + RowMerger currentMerger, + AutoIncrementUpdater autoIncrementUpdater, + ValueDecoder valueDecoder, + WalBuilder walBuilder, + PaddingRow latestSchemaRow, + long logOffset) + throws Exception { + if (mutation.isDelete()) { + return processDeletion( + mutation.key, + currentMerger, + valueDecoder, + walBuilder, + latestSchemaRow, + logOffset); + } else { + return processUpsert( + mutation.key, + mutation.value, + currentMerger, + autoIncrementUpdater, + valueDecoder, + walBuilder, + latestSchemaRow, + logOffset); + } } private long processDeletion( @@ -517,8 +780,8 @@ private long processDeletion( return logOffset; } else if (deleteBehavior == DeleteBehavior.DISABLE) { throw new DeletionDisabledException( - "Delete operations are disabled for this table. " - + "The table.delete.behavior is set to 'disable'."); + "Delete operations are disabled for this table." + + " The table.delete.behavior is set to 'disable'."); } byte[] oldValueBytes = getFromBufferOrKv(key); @@ -575,7 +838,11 @@ private long processUpsert( BinaryValue newValue = currentMerger.merge(oldValue, currentValue); if (newValue == oldValue) { - // no actual change, skip this record + // Reference equality: RowMerger returns the same oldValue reference + // to signal "no actual change" (e.g., FirstRowRowMerger). + // Do NOT use .equals() here — DefaultRowMerger.merge() returns + // currentValue (a new object), and value-equality would incorrectly + // skip CDC events when the user writes identical data. return logOffset; } @@ -628,6 +895,114 @@ private long applyUpdate( } } + /** + * A peekable iterator over KvRecords that pre-computes {@link NormalizedKvMutation} for each + * record. Supports one-element lookahead via {@link #peek()} / {@link #peekMutation()} without + * consuming the record, eliminating the need for manual pendingRecord/pendingMutation state + * management in the caller. + */ + private static final class PeekableKvMutationIterator { + private final Iterator inner; + private final short schemaIdOfNewData; + + private @Nullable KvRecord peekedRecord; + private @Nullable NormalizedKvMutation peekedMutation; + // Holds the mutation from a consumed peeked record so nextMutation() can return + // it without recomputing. + private @Nullable NormalizedKvMutation lastConsumedMutation; + // Reusable mutation object for the non-peeked hot path to avoid per-record allocation. + private final NormalizedKvMutation reusable; + + PeekableKvMutationIterator(Iterator inner, short schemaIdOfNewData) { + this.inner = inner; + this.schemaIdOfNewData = schemaIdOfNewData; + this.reusable = new NormalizedKvMutation(null, null); + } + + boolean hasNext() { + return peekedRecord != null || inner.hasNext(); + } + + /** Returns the next record, consuming it. */ + KvRecord next() { + if (peekedRecord != null) { + KvRecord record = peekedRecord; + lastConsumedMutation = peekedMutation; + peekedRecord = null; + peekedMutation = null; + return record; + } + lastConsumedMutation = null; + return inner.next(); + } + + /** Returns the mutation for the record returned by the last {@link #next()} call. */ + NormalizedKvMutation nextMutation(KvRecord kvRecord) { + if (lastConsumedMutation != null) { + NormalizedKvMutation m = lastConsumedMutation; + lastConsumedMutation = null; + return m; + } + // Non-peeked hot path: reuse the mutation object to avoid allocation. + KvPreWriteBuffer.Key key = + KvPreWriteBuffer.Key.of(BytesUtils.toArray(kvRecord.getKey())); + BinaryRow row = kvRecord.getRow(); + BinaryValue value = row == null ? null : new BinaryValue(schemaIdOfNewData, row); + reusable.reset(key, value); + return reusable; + } + + /** Peeks at the next record without consuming it. Returns null if no more records. */ + @Nullable + KvRecord peek() { + if (peekedRecord == null && inner.hasNext()) { + peekedRecord = inner.next(); + peekedMutation = normalize(peekedRecord); + } + return peekedRecord; + } + + /** Returns the pre-computed mutation for the peeked record. */ + @Nullable + NormalizedKvMutation peekMutation() { + return peekedMutation; + } + + /** Consumes the peeked record (must be called after peek()). */ + void consumePeeked() { + peekedRecord = null; + peekedMutation = null; + } + + private NormalizedKvMutation normalize(KvRecord kvRecord) { + KvPreWriteBuffer.Key key = + KvPreWriteBuffer.Key.of(BytesUtils.toArray(kvRecord.getKey())); + BinaryRow row = kvRecord.getRow(); + BinaryValue value = row == null ? null : new BinaryValue(schemaIdOfNewData, row); + return new NormalizedKvMutation(key, value); + } + } + + private static final class NormalizedKvMutation { + private KvPreWriteBuffer.Key key; + private @Nullable BinaryValue value; + + private NormalizedKvMutation(KvPreWriteBuffer.Key key, @Nullable BinaryValue value) { + this.key = key; + this.value = value; + } + + void reset(KvPreWriteBuffer.Key key, @Nullable BinaryValue value) { + this.key = key; + this.value = value; + } + + /** A delete mutation has null value. */ + boolean isDelete() { + return value == null; + } + } + private WalBuilder createWalBuilder(int schemaId, RowType rowType) throws Exception { switch (logFormat) { case INDEXED: diff --git a/fluss-server/src/main/java/org/apache/fluss/server/kv/rowmerger/AggregateRowMerger.java b/fluss-server/src/main/java/org/apache/fluss/server/kv/rowmerger/AggregateRowMerger.java index 54583a90b3..a12caf4ae5 100644 --- a/fluss-server/src/main/java/org/apache/fluss/server/kv/rowmerger/AggregateRowMerger.java +++ b/fluss-server/src/main/java/org/apache/fluss/server/kv/rowmerger/AggregateRowMerger.java @@ -28,6 +28,7 @@ import org.apache.fluss.server.kv.rowmerger.aggregate.AggregateFieldsProcessor; import org.apache.fluss.server.kv.rowmerger.aggregate.AggregationContext; import org.apache.fluss.server.kv.rowmerger.aggregate.AggregationContextCache; +import org.apache.fluss.server.kv.rowmerger.aggregate.functions.FieldAggregator; import com.github.benmanes.caffeine.cache.Cache; import com.github.benmanes.caffeine.cache.Caffeine; @@ -90,28 +91,58 @@ public BinaryValue merge(BinaryValue oldValue, BinaryValue newValue) { if (oldValue == null || oldValue.row == null) { return newValue; } + return applyAllFields( + oldValue, newValue, AggregateFieldsProcessor::aggregateAllFieldsWithTargetSchema); + } + + @Override + public BinaryValue delete(BinaryValue oldValue) { + // Remove the entire row (returns null to indicate deletion) + return null; + } + + @Override + public BinaryValue retract(BinaryValue oldValue, BinaryValue retractValue) { + // No existing row: nothing to retract from. + // Returns null to signal the row should be removed (same contract as delete()). + if (oldValue == null || oldValue.row == null) { + return null; + } + BinaryValue result = + applyAllFields( + oldValue, + retractValue, + AggregateFieldsProcessor::retractAllFieldsWithTargetSchema); + // If all non-PK fields are null after retract, the row is fully retracted — remove it. + if (isAllNonPkFieldsNull(result.row, contextCache.getContext(result.schemaId))) { + return null; + } + return result; + } - // Get contexts for schema evolution support + /** Shared implementation for merge() and retract() on all fields. */ + private BinaryValue applyAllFields( + BinaryValue oldValue, BinaryValue inputValue, AllFieldsOperation operation) { AggregationContext oldContext = contextCache.getContext(oldValue.schemaId); - AggregationContext newContext = contextCache.getContext(newValue.schemaId); + AggregationContext inputContext = contextCache.getContext(inputValue.schemaId); AggregationContext targetContext = contextCache.getContext(targetSchemaId); - // Use target schema encoder to ensure merged row uses latest schema RowEncoder encoder = targetContext.getRowEncoder(); encoder.startNewRow(); - - // Aggregate using target schema context to ensure output uses server's latest schema - AggregateFieldsProcessor.aggregateAllFieldsWithTargetSchema( - oldValue.row, newValue.row, oldContext, newContext, targetContext, encoder); - BinaryRow mergedRow = encoder.finishRow(); - - return new BinaryValue(targetSchemaId, mergedRow); + operation.apply( + oldValue.row, inputValue.row, oldContext, inputContext, targetContext, encoder); + return new BinaryValue(targetSchemaId, encoder.finishRow()); } - @Override - public BinaryValue delete(BinaryValue oldValue) { - // Remove the entire row (returns null to indicate deletion) - return null; + @FunctionalInterface + private interface AllFieldsOperation { + void apply( + BinaryRow oldRow, + BinaryRow inputRow, + AggregationContext oldContext, + AggregationContext inputContext, + AggregationContext targetContext, + RowEncoder encoder); } @Override @@ -119,6 +150,11 @@ public DeleteBehavior deleteBehavior() { return deleteBehavior; } + @Override + public boolean supportsRetract() { + return allAggregatorsSupportRetract(contextCache.getContext(targetSchemaId)); + } + @Override public RowMerger configureTargetColumns( @Nullable int[] targetColumns, short latestSchemaId, Schema latestSchema) { @@ -162,6 +198,31 @@ public RowMerger configureTargetColumns( }); } + private static boolean allAggregatorsSupportRetract(AggregationContext context) { + for (FieldAggregator agg : context.getAggregators()) { + if (agg != null && !agg.supportsRetract()) { + return false; + } + } + return true; + } + + /** + * Checks whether all non-primary-key fields in the given row are null. + * + *

When a retract operation zeroes out every non-PK field, the row is effectively empty and + * should be removed rather than kept as a zombie row with only PK values. + */ + private static boolean isAllNonPkFieldsNull(BinaryRow row, AggregationContext context) { + BitSet pkBitSet = context.getPrimaryKeyColsBitSet(); + for (int i = 0; i < context.getFieldCount(); i++) { + if (!pkBitSet.get(i) && !row.isNullAt(i)) { + return false; + } + } + return true; + } + /** * Cache key for PartialAggregateRowMerger instances. * @@ -273,28 +334,60 @@ public BinaryValue merge(BinaryValue oldValue, BinaryValue newValue) { if (oldValue == null || oldValue.row == null) { return newValue; } + return applyTargetFields( + oldValue, + newValue, + AggregateFieldsProcessor::aggregateTargetFieldsWithTargetSchema); + } + + @Override + public BinaryValue retract(BinaryValue oldValue, BinaryValue retractValue) { + // No existing row: nothing to retract from + if (oldValue == null || oldValue.row == null) { + return null; + } + BinaryValue result = + applyTargetFields( + oldValue, + retractValue, + AggregateFieldsProcessor::retractTargetFieldsWithTargetSchema); + // If all non-PK fields are null after retract, remove the row. + if (isAllNonPkFieldsNull(result.row, contextCache.getContext(result.schemaId))) { + return null; + } + return result; + } - // Get contexts for schema evolution support + /** Shared implementation for merge() and retract() on target fields. */ + private BinaryValue applyTargetFields( + BinaryValue oldValue, BinaryValue inputValue, TargetFieldsOperation operation) { AggregationContext oldContext = contextCache.getContext(oldValue.schemaId); - AggregationContext newContext = contextCache.getContext(newValue.schemaId); + AggregationContext inputContext = contextCache.getContext(inputValue.schemaId); AggregationContext targetContext = contextCache.getContext(targetSchemaId); - // Use target schema encoder to ensure merged row uses latest schema RowEncoder encoder = targetContext.getRowEncoder(); encoder.startNewRow(); - - // Aggregate using target schema to ensure output uses server's latest schema - AggregateFieldsProcessor.aggregateTargetFieldsWithTargetSchema( + operation.apply( oldValue.row, - newValue.row, + inputValue.row, oldContext, - newContext, + inputContext, targetContext, targetColumnIdBitSet, encoder); - BinaryRow mergedRow = encoder.finishRow(); + return new BinaryValue(targetSchemaId, encoder.finishRow()); + } - return new BinaryValue(targetSchemaId, mergedRow); + @FunctionalInterface + private interface TargetFieldsOperation { + void apply( + BinaryRow oldRow, + BinaryRow inputRow, + AggregationContext oldContext, + AggregationContext inputContext, + AggregationContext targetContext, + BitSet targetColumnIdBitSet, + RowEncoder encoder); } @Override @@ -352,6 +445,24 @@ public DeleteBehavior deleteBehavior() { return deleteBehavior; } + @Override + public boolean supportsRetract() { + // Only check aggregators for target columns, not all columns. + // A partial retract touching only SUM columns should succeed even if + // the table has non-retractable columns like MAX/MIN. + AggregationContext context = contextCache.getContext(targetSchemaId); + List columns = context.getSchema().getColumns(); + FieldAggregator[] aggregators = context.getAggregators(); + for (int i = 0; i < aggregators.length; i++) { + if (aggregators[i] != null + && targetColumnIdBitSet.get(columns.get(i).getColumnId()) + && !aggregators[i].supportsRetract()) { + return false; + } + } + return true; + } + @Override public RowMerger configureTargetColumns( @Nullable int[] targetColumns, short latestSchemaId, Schema latestSchema) { diff --git a/fluss-server/src/main/java/org/apache/fluss/server/kv/rowmerger/RowMerger.java b/fluss-server/src/main/java/org/apache/fluss/server/kv/rowmerger/RowMerger.java index ed8b37a658..9eea59432d 100644 --- a/fluss-server/src/main/java/org/apache/fluss/server/kv/rowmerger/RowMerger.java +++ b/fluss-server/src/main/java/org/apache/fluss/server/kv/rowmerger/RowMerger.java @@ -55,6 +55,35 @@ public interface RowMerger { @Nullable BinaryValue delete(BinaryValue oldRow); + /** + * Retract (reverse) a previously merged value from the accumulator. + * + *

Only supported by merge engines whose aggregation functions have retract implementations + * (e.g., aggregation merge engine with SUM). Retract quality varies by function — see {@link + * org.apache.fluss.metadata.AggFunctionType#supportsRetract()} for details. By default, this + * method throws {@link UnsupportedOperationException}. + * + * @param oldValue the current accumulated value + * @param retractValue the value to retract + * @return the result after retraction, or null if the row should be removed + */ + @Nullable + default BinaryValue retract(BinaryValue oldValue, BinaryValue retractValue) { + throw new UnsupportedOperationException( + String.format("%s does not support retract", this.getClass().getSimpleName())); + } + + /** + * Returns whether this merger has a retract implementation. + * + *

When true, {@link #retract(BinaryValue, BinaryValue)} can be called. When false, calling + * retract will throw {@link UnsupportedOperationException}. Note that retract quality varies by + * function — returning true does not guarantee a mathematically exact inverse. + */ + default boolean supportsRetract() { + return false; + } + /** * The behavior of delete operations on primary key tables. * diff --git a/fluss-server/src/main/java/org/apache/fluss/server/kv/rowmerger/aggregate/AggregateFieldsProcessor.java b/fluss-server/src/main/java/org/apache/fluss/server/kv/rowmerger/aggregate/AggregateFieldsProcessor.java index 0429a2015b..9c5702b218 100644 --- a/fluss-server/src/main/java/org/apache/fluss/server/kv/rowmerger/aggregate/AggregateFieldsProcessor.java +++ b/fluss-server/src/main/java/org/apache/fluss/server/kv/rowmerger/aggregate/AggregateFieldsProcessor.java @@ -41,25 +41,28 @@ public final class AggregateFieldsProcessor { // A FieldGetter that always returns null, used for non-existent columns in schema evolution private static final InternalRow.FieldGetter NULL_FIELD_GETTER = row -> null; + /** Functional interface for field-level operations (agg or retract). */ + @FunctionalInterface + private interface FieldOperation { + Object apply(FieldAggregator aggregator, Object accumulator, Object inputField); + } + + // Allows processAllFields/processTargetFields to be reused for both aggregate and retract + // paths without branching — the caller simply passes the appropriate operation. + private static final FieldOperation AGGREGATE_OP = FieldAggregator::agg; + + /** Field operation: applies aggregator.retract(accumulator, input). */ + private static final FieldOperation RETRACT_OP = FieldAggregator::retract; + // Private constructor to prevent instantiation private AggregateFieldsProcessor() {} + // ======================== Public aggregate methods ======================== + /** * Aggregate all fields from old and new rows with explicit target schema (full aggregation). - * - *

This variant allows specifying a different target schema for the output, which is useful - * when the client's newValue uses an outdated schema but we want to output using the latest - * server schema. - * - *

This method handles schema evolution by matching fields using column IDs across three - * potentially different schemas: old row schema, new row schema, and target output schema. - * - * @param oldRow the old row - * @param newRow the new row - * @param oldContext context for the old row schema - * @param newInputContext context for the new row schema (for reading newRow) - * @param targetContext context for the target output schema - * @param encoder the row encoder to encode results (should match targetContext) + * Handles schema evolution by matching fields using column IDs across three potentially + * different schemas. */ public static void aggregateAllFieldsWithTargetSchema( BinaryRow oldRow, @@ -68,318 +71,411 @@ public static void aggregateAllFieldsWithTargetSchema( AggregationContext newInputContext, AggregationContext targetContext, RowEncoder encoder) { + processAllFieldsWithTargetSchema( + oldRow, + newRow, + oldContext, + newInputContext, + targetContext, + encoder, + AGGREGATE_OP, + false); + } + + /** + * Aggregate target fields with explicit target schema (partial aggregation). For target + * columns, aggregate with the aggregation function. For non-target columns, keep the old value + * unchanged. + */ + public static void aggregateTargetFieldsWithTargetSchema( + BinaryRow oldRow, + BinaryRow newRow, + AggregationContext oldContext, + AggregationContext newInputContext, + AggregationContext targetContext, + BitSet targetColumnIdBitSet, + RowEncoder encoder) { + processTargetFieldsWithTargetSchema( + oldRow, + newRow, + oldContext, + newInputContext, + targetContext, + targetColumnIdBitSet, + encoder, + AGGREGATE_OP, + false); + } + + // ======================== Public retract methods ======================== + + /** + * Retract all fields with explicit target schema (full retract). Primary key columns are never + * retracted; their old values are copied directly. + */ + public static void retractAllFieldsWithTargetSchema( + BinaryRow oldRow, + BinaryRow retractRow, + AggregationContext oldContext, + AggregationContext retractInputContext, + AggregationContext targetContext, + RowEncoder encoder) { + processAllFieldsWithTargetSchema( + oldRow, + retractRow, + oldContext, + retractInputContext, + targetContext, + encoder, + RETRACT_OP, + true); + } + + /** + * Retract target fields with explicit target schema (partial retract). For target columns + * (excluding primary keys), retract with the aggregation function. For non-target columns and + * primary key columns, keep the old value unchanged. + */ + public static void retractTargetFieldsWithTargetSchema( + BinaryRow oldRow, + BinaryRow retractRow, + AggregationContext oldContext, + AggregationContext retractInputContext, + AggregationContext targetContext, + BitSet targetColumnIdBitSet, + RowEncoder encoder) { + processTargetFieldsWithTargetSchema( + oldRow, + retractRow, + oldContext, + retractInputContext, + targetContext, + targetColumnIdBitSet, + encoder, + RETRACT_OP, + true); + } + + // ======================== Public delete methods ======================== + + /** + * Encode a partial delete when schemas are identical. Set target columns (except primary key) + * to null, keep other columns unchanged. + */ + public static void encodePartialDeleteWithSameSchema( + BinaryRow oldRow, + AggregationContext context, + BitSet targetPosBitSet, + BitSet pkPosBitSet, + RowEncoder encoder) { + InternalRow.FieldGetter[] fieldGetters = context.getFieldGetters(); + + for (int i = 0; i < oldRow.getFieldCount(); i++) { + if (targetPosBitSet.get(i) && !pkPosBitSet.get(i)) { + // Target column (not primary key): set to null + encoder.encodeField(i, null); + } else { + // Non-target column or primary key: keep old value + copyOldValueAndEncode(fieldGetters[i], oldRow, i, encoder); + } + } + } + + /** + * Encode a partial delete with schema evolution. Set target columns (except primary key) to + * null, keep other columns unchanged. + */ + public static void encodePartialDeleteWithDifferentSchema( + BinaryRow oldRow, + AggregationContext oldContext, + AggregationContext targetContext, + BitSet targetColumnIdBitSet, + BitSet pkPosBitSet, + RowEncoder encoder) { + InternalRow.FieldGetter[] oldFieldGetters = oldContext.getFieldGetters(); + List targetColumns = targetContext.getSchema().getColumns(); + + for (int targetIdx = 0; targetIdx < targetColumns.size(); targetIdx++) { + Schema.Column targetColumn = targetColumns.get(targetIdx); + int columnId = targetColumn.getColumnId(); + + // Find corresponding field in old schema using column ID + Integer oldIdx = oldContext.getFieldIndex(columnId); + + // Check if this is a target column using columnId (not position) + boolean isTargetColumn = targetColumnIdBitSet.get(columnId); + boolean isPrimaryKey = pkPosBitSet.get(targetIdx); + + if (isTargetColumn && !isPrimaryKey) { + // Target column (not primary key): set to null + encoder.encodeField(targetIdx, null); + } else if (oldIdx != null) { + // Column exists in old schema: copy value from old row + copyOldValueAndEncode(oldFieldGetters[oldIdx], oldRow, targetIdx, encoder); + } else { + // New column that doesn't exist in old schema: set to null + encoder.encodeField(targetIdx, null); + } + } + } + + // ======================== Public utility methods ======================== + + /** Check if there are any non-null fields in non-target columns. */ + public static boolean hasNonTargetNonNullField(BinaryRow row, BitSet targetPosBitSet) { + int fieldCount = row.getFieldCount(); + // Use nextClearBit to iterate over non-target fields (bits not set in targetPosBitSet) + for (int pos = targetPosBitSet.nextClearBit(0); + pos < fieldCount; + pos = targetPosBitSet.nextClearBit(pos + 1)) { + if (!row.isNullAt(pos)) { + return true; + } + } + return false; + } + + // ======================== Unified internal methods ======================== + + /** + * Process all fields with explicit target schema. Unified implementation for both aggregate and + * retract operations. + * + * @param skipPrimaryKeys if true, primary key columns are copied directly (retract mode) + */ + private static void processAllFieldsWithTargetSchema( + BinaryRow oldRow, + BinaryRow inputRow, + AggregationContext oldContext, + AggregationContext inputContext, + AggregationContext targetContext, + RowEncoder encoder, + FieldOperation fieldOp, + boolean skipPrimaryKeys) { // Fast path: all three schemas are the same - if (targetContext == oldContext && targetContext == newInputContext) { - aggregateAllFieldsWithSameSchema(oldRow, newRow, targetContext, encoder); + if (targetContext == oldContext && targetContext == inputContext) { + processAllFieldsWithSameSchema( + oldRow, inputRow, targetContext, encoder, fieldOp, skipPrimaryKeys); return; } - // General path: iterate over target schema columns and aggregate using column ID matching + // General path: iterate over target schema columns InternalRow.FieldGetter[] oldFieldGetters = oldContext.getFieldGetters(); - InternalRow.FieldGetter[] newFieldGetters = newInputContext.getFieldGetters(); + InternalRow.FieldGetter[] inputFieldGetters = inputContext.getFieldGetters(); FieldAggregator[] targetAggregators = targetContext.getAggregators(); List targetColumns = targetContext.getSchema().getColumns(); + BitSet pkBitSet = skipPrimaryKeys ? targetContext.getPrimaryKeyColsBitSet() : null; for (int targetIdx = 0; targetIdx < targetColumns.size(); targetIdx++) { Schema.Column targetColumn = targetColumns.get(targetIdx); int columnId = targetColumn.getColumnId(); - // Find corresponding fields in old and new schemas using column ID Integer oldIdx = oldContext.getFieldIndex(columnId); - Integer newIdx = newInputContext.getFieldIndex(columnId); + + // Primary key columns: copy old value directly when in retract mode + if (pkBitSet != null && pkBitSet.get(targetIdx)) { + if (oldIdx != null) { + copyOldValueAndEncode(oldFieldGetters[oldIdx], oldRow, targetIdx, encoder); + } else { + encoder.encodeField(targetIdx, null); + } + continue; + } + + Integer inputIdx = inputContext.getFieldIndex(columnId); // Get field getters (use NULL_FIELD_GETTER if column doesn't exist in that schema) InternalRow.FieldGetter oldFieldGetter = (oldIdx != null) ? oldFieldGetters[oldIdx] : NULL_FIELD_GETTER; - InternalRow.FieldGetter newFieldGetter = - (newIdx != null) ? newFieldGetters[newIdx] : NULL_FIELD_GETTER; + InternalRow.FieldGetter inputFieldGetter = + (inputIdx != null) ? inputFieldGetters[inputIdx] : NULL_FIELD_GETTER; - // Aggregate and encode using target schema's aggregator - aggregateAndEncode( + applyAndEncode( oldFieldGetter, - newFieldGetter, + inputFieldGetter, oldRow, - newRow, + inputRow, targetAggregators[targetIdx], targetIdx, - encoder); + encoder, + fieldOp); } } /** - * Aggregate and encode a single field. - * - * @param oldFieldGetter getter for the old field - * @param newFieldGetter getter for the new field - * @param oldRow the old row - * @param newRow the new row - * @param aggregator the aggregator for this field - * @param targetIdx the target index to encode - * @param encoder the row encoder + * Process all fields when old and input schemas are identical. Fast path: field positions match + * directly, no column ID lookup needed. */ - private static void aggregateAndEncode( - InternalRow.FieldGetter oldFieldGetter, - InternalRow.FieldGetter newFieldGetter, + private static void processAllFieldsWithSameSchema( BinaryRow oldRow, - BinaryRow newRow, - FieldAggregator aggregator, - int targetIdx, - RowEncoder encoder) { - Object accumulator = oldFieldGetter.getFieldOrNull(oldRow); - Object inputField = newFieldGetter.getFieldOrNull(newRow); - Object mergedField = aggregator.agg(accumulator, inputField); - encoder.encodeField(targetIdx, mergedField); - } + BinaryRow inputRow, + AggregationContext context, + RowEncoder encoder, + FieldOperation fieldOp, + boolean skipPrimaryKeys) { + InternalRow.FieldGetter[] fieldGetters = context.getFieldGetters(); + FieldAggregator[] aggregators = context.getAggregators(); + int fieldCount = context.getFieldCount(); + BitSet pkBitSet = skipPrimaryKeys ? context.getPrimaryKeyColsBitSet() : null; - /** - * Copy and encode a field value from old row. - * - * @param fieldGetter getter for the field - * @param oldRow the old row - * @param targetIdx the target index to encode - * @param encoder the row encoder - */ - private static void copyOldValueAndEncode( - InternalRow.FieldGetter fieldGetter, - BinaryRow oldRow, - int targetIdx, - RowEncoder encoder) { - encoder.encodeField(targetIdx, fieldGetter.getFieldOrNull(oldRow)); + for (int idx = 0; idx < fieldCount; idx++) { + if (pkBitSet != null && pkBitSet.get(idx)) { + // Primary key columns: copy old value directly + copyOldValueAndEncode(fieldGetters[idx], oldRow, idx, encoder); + } else { + applyAndEncode( + fieldGetters[idx], + fieldGetters[idx], + oldRow, + inputRow, + aggregators[idx], + idx, + encoder, + fieldOp); + } + } } /** - * Aggregate target fields from old and new rows with explicit target schema (partial - * aggregation). - * - *

This variant allows specifying a different target schema for the output, which is useful - * when the client's newValue uses an outdated schema but we want to output using the latest - * server schema. - * - *

For target columns, aggregate with the aggregation function. For non-target columns, keep - * the old value unchanged. For columns that don't exist in old schema, copy from newRow. For - * columns that exist only in target schema, set to null. + * Process target fields with explicit target schema. Unified implementation for both aggregate + * and retract partial operations. * - * @param oldRow the old row - * @param newRow the new row - * @param oldContext context for the old row schema - * @param newInputContext context for the new row schema (for reading newRow) - * @param targetContext context for the target output schema - * @param targetColumnIdBitSet BitSet marking target columns by column ID - * @param encoder the row encoder to encode results (should match targetContext) + * @param skipPrimaryKeys if true, primary key columns are always copied (retract mode) */ - public static void aggregateTargetFieldsWithTargetSchema( + private static void processTargetFieldsWithTargetSchema( BinaryRow oldRow, - BinaryRow newRow, + BinaryRow inputRow, AggregationContext oldContext, - AggregationContext newInputContext, + AggregationContext inputContext, AggregationContext targetContext, BitSet targetColumnIdBitSet, - RowEncoder encoder) { + RowEncoder encoder, + FieldOperation fieldOp, + boolean skipPrimaryKeys) { // Fast path: all three schemas are the same - if (targetContext == oldContext && targetContext == newInputContext) { - aggregateTargetFieldsWithSameSchema( - oldRow, newRow, targetContext, targetColumnIdBitSet, encoder); + if (targetContext == oldContext && targetContext == inputContext) { + processTargetFieldsWithSameSchema( + oldRow, + inputRow, + targetContext, + targetColumnIdBitSet, + encoder, + fieldOp, + skipPrimaryKeys); return; } // General path: iterate over target schema columns InternalRow.FieldGetter[] oldFieldGetters = oldContext.getFieldGetters(); - InternalRow.FieldGetter[] newFieldGetters = newInputContext.getFieldGetters(); + InternalRow.FieldGetter[] inputFieldGetters = inputContext.getFieldGetters(); FieldAggregator[] targetAggregators = targetContext.getAggregators(); List targetColumns = targetContext.getSchema().getColumns(); + BitSet pkBitSet = skipPrimaryKeys ? targetContext.getPrimaryKeyColsBitSet() : null; for (int targetIdx = 0; targetIdx < targetColumns.size(); targetIdx++) { Schema.Column targetColumn = targetColumns.get(targetIdx); int columnId = targetColumn.getColumnId(); - // Find corresponding fields in old and new schemas using column ID Integer oldIdx = oldContext.getFieldIndex(columnId); - Integer newIdx = newInputContext.getFieldIndex(columnId); + + // Primary key columns: always copy old value when in retract mode + if (pkBitSet != null && pkBitSet.get(targetIdx)) { + if (oldIdx != null) { + copyOldValueAndEncode(oldFieldGetters[oldIdx], oldRow, targetIdx, encoder); + } else { + encoder.encodeField(targetIdx, null); + } + continue; + } if (targetColumnIdBitSet.get(columnId)) { - // Target column: aggregate and encode + Integer inputIdx = inputContext.getFieldIndex(columnId); InternalRow.FieldGetter oldFieldGetter = (oldIdx != null) ? oldFieldGetters[oldIdx] : NULL_FIELD_GETTER; - InternalRow.FieldGetter newFieldGetter = - (newIdx != null) ? newFieldGetters[newIdx] : NULL_FIELD_GETTER; - aggregateAndEncode( + InternalRow.FieldGetter inputFieldGetter = + (inputIdx != null) ? inputFieldGetters[inputIdx] : NULL_FIELD_GETTER; + applyAndEncode( oldFieldGetter, - newFieldGetter, + inputFieldGetter, oldRow, - newRow, + inputRow, targetAggregators[targetIdx], targetIdx, - encoder); + encoder, + fieldOp); } else if (oldIdx != null) { - // Non-target column that exists in old schema: copy old value copyOldValueAndEncode(oldFieldGetters[oldIdx], oldRow, targetIdx, encoder); } else { - // Non-target column that doesn't exist in old schema: set to null - // NOTE: In partial aggregation, non-target columns should not use values from - // newRow, even if they exist in newRow's schema, as only target columns are - // aggregated encoder.encodeField(targetIdx, null); } } } /** - * Aggregate all fields when old and new schemas are identical. - * - *

Fast path: field positions match directly, no column ID lookup needed. + * Process target fields when old and input schemas are identical. Fast path: field positions + * match directly, no column ID lookup needed. */ - private static void aggregateAllFieldsWithSameSchema( - BinaryRow oldRow, BinaryRow newRow, AggregationContext context, RowEncoder encoder) { - InternalRow.FieldGetter[] fieldGetters = context.getFieldGetters(); - FieldAggregator[] aggregators = context.getAggregators(); - int fieldCount = context.getFieldCount(); - - for (int idx = 0; idx < fieldCount; idx++) { - aggregateAndEncode( - fieldGetters[idx], - fieldGetters[idx], - oldRow, - newRow, - aggregators[idx], - idx, - encoder); - } - } - - /** - * Aggregate target fields when old and new schemas are identical. - * - *

Fast path: field positions match directly, no column ID lookup needed. - */ - private static void aggregateTargetFieldsWithSameSchema( + private static void processTargetFieldsWithSameSchema( BinaryRow oldRow, - BinaryRow newRow, + BinaryRow inputRow, AggregationContext context, BitSet targetColumnIdBitSet, - RowEncoder encoder) { + RowEncoder encoder, + FieldOperation fieldOp, + boolean skipPrimaryKeys) { InternalRow.FieldGetter[] fieldGetters = context.getFieldGetters(); FieldAggregator[] aggregators = context.getAggregators(); List columns = context.getSchema().getColumns(); int fieldCount = context.getFieldCount(); + BitSet pkBitSet = skipPrimaryKeys ? context.getPrimaryKeyColsBitSet() : null; for (int idx = 0; idx < fieldCount; idx++) { + // Primary key columns: always copy old value when in retract mode + if (pkBitSet != null && pkBitSet.get(idx)) { + copyOldValueAndEncode(fieldGetters[idx], oldRow, idx, encoder); + continue; + } + int columnId = columns.get(idx).getColumnId(); if (targetColumnIdBitSet.get(columnId)) { - // Target column: aggregate and encode - aggregateAndEncode( + applyAndEncode( fieldGetters[idx], fieldGetters[idx], oldRow, - newRow, + inputRow, aggregators[idx], idx, - encoder); + encoder, + fieldOp); } else { - // Non-target column: encode old value copyOldValueAndEncode(fieldGetters[idx], oldRow, idx, encoder); } } } - /** - * Encode a partial delete when old and new schemas are identical. - * - *

Set target columns (except primary key) to null, keep other columns unchanged. - * - *

Fast path: field positions match directly, no column ID lookup needed. - * - * @param oldRow the old row to partially delete - * @param context the aggregation context for encoding - * @param targetPosBitSet BitSet marking target column positions - * @param pkPosBitSet BitSet marking primary key positions - * @param encoder the row encoder to encode results - */ - public static void encodePartialDeleteWithSameSchema( + /** Apply a field operation (agg or retract) and encode the result. */ + private static void applyAndEncode( + InternalRow.FieldGetter oldFieldGetter, + InternalRow.FieldGetter inputFieldGetter, BinaryRow oldRow, - AggregationContext context, - BitSet targetPosBitSet, - BitSet pkPosBitSet, - RowEncoder encoder) { - InternalRow.FieldGetter[] fieldGetters = context.getFieldGetters(); - - for (int i = 0; i < oldRow.getFieldCount(); i++) { - if (targetPosBitSet.get(i) && !pkPosBitSet.get(i)) { - // Target column (not primary key): set to null - encoder.encodeField(i, null); - } else { - // Non-target column or primary key: keep old value - copyOldValueAndEncode(fieldGetters[i], oldRow, i, encoder); - } - } + BinaryRow inputRow, + FieldAggregator aggregator, + int targetIdx, + RowEncoder encoder, + FieldOperation fieldOp) { + Object accumulator = oldFieldGetter.getFieldOrNull(oldRow); + Object inputField = inputFieldGetter.getFieldOrNull(inputRow); + Object result = fieldOp.apply(aggregator, accumulator, inputField); + encoder.encodeField(targetIdx, result); } - /** - * Encode a partial delete when old and new schemas differ (schema evolution). - * - *

Set target columns (except primary key) to null, keep other columns unchanged. For new - * columns that don't exist in old schema, set to null. - * - *

Slow path: requires column ID matching to find corresponding fields between schemas. - * - * @param oldRow the old row to partially delete - * @param oldContext context for the old schema - * @param targetContext context for the target schema - * @param targetColumnIdBitSet BitSet marking target columns by column ID - * @param pkPosBitSet BitSet marking primary key positions in target schema - * @param encoder the row encoder to encode results - */ - public static void encodePartialDeleteWithDifferentSchema( + /** Copy and encode a field value from old row. */ + private static void copyOldValueAndEncode( + InternalRow.FieldGetter fieldGetter, BinaryRow oldRow, - AggregationContext oldContext, - AggregationContext targetContext, - BitSet targetColumnIdBitSet, - BitSet pkPosBitSet, + int targetIdx, RowEncoder encoder) { - InternalRow.FieldGetter[] oldFieldGetters = oldContext.getFieldGetters(); - List targetColumns = targetContext.getSchema().getColumns(); - - for (int targetIdx = 0; targetIdx < targetColumns.size(); targetIdx++) { - Schema.Column targetColumn = targetColumns.get(targetIdx); - int columnId = targetColumn.getColumnId(); - - // Find corresponding field in old schema using column ID - Integer oldIdx = oldContext.getFieldIndex(columnId); - - // Check if this is a target column using columnId (not position) - boolean isTargetColumn = targetColumnIdBitSet.get(columnId); - boolean isPrimaryKey = pkPosBitSet.get(targetIdx); - - if (isTargetColumn && !isPrimaryKey) { - // Target column (not primary key): set to null - encoder.encodeField(targetIdx, null); - } else if (oldIdx != null) { - // Column exists in old schema: copy value from old row - copyOldValueAndEncode(oldFieldGetters[oldIdx], oldRow, targetIdx, encoder); - } else { - // New column that doesn't exist in old schema: set to null - encoder.encodeField(targetIdx, null); - } - } - } - - /** - * Check if there are any non-null fields in non-target columns. - * - * @param row the row to check - * @param targetPosBitSet BitSet marking target column positions - * @return true if at least one non-target column has a non-null value - */ - public static boolean hasNonTargetNonNullField(BinaryRow row, BitSet targetPosBitSet) { - int fieldCount = row.getFieldCount(); - // Use nextClearBit to iterate over non-target fields (bits not set in targetPosBitSet) - for (int pos = targetPosBitSet.nextClearBit(0); - pos < fieldCount; - pos = targetPosBitSet.nextClearBit(pos + 1)) { - if (!row.isNullAt(pos)) { - return true; - } - } - return false; + encoder.encodeField(targetIdx, fieldGetter.getFieldOrNull(oldRow)); } } diff --git a/fluss-server/src/main/java/org/apache/fluss/server/kv/rowmerger/aggregate/functions/FieldAggregator.java b/fluss-server/src/main/java/org/apache/fluss/server/kv/rowmerger/aggregate/functions/FieldAggregator.java index f6ec77a494..db06c5ffb7 100644 --- a/fluss-server/src/main/java/org/apache/fluss/server/kv/rowmerger/aggregate/functions/FieldAggregator.java +++ b/fluss-server/src/main/java/org/apache/fluss/server/kv/rowmerger/aggregate/functions/FieldAggregator.java @@ -61,6 +61,22 @@ public Object aggReversed(Object accumulator, Object inputField) { return agg(inputField, accumulator); } + /** + * Retracts a previously aggregated value from the accumulator. By default throws {@link + * UnsupportedOperationException}. Subclasses that support retraction should override. + */ + public Object retract(Object accumulator, Object retractField) { + throw new UnsupportedOperationException( + String.format( + "Aggregate function '%s' does not support retraction.", + this.getClass().getSimpleName())); + } + + /** Returns whether this aggregator supports retract operations. */ + public boolean supportsRetract() { + return false; + } + /** Resets the aggregator to a clean start state. */ public void reset() {} } diff --git a/fluss-server/src/main/java/org/apache/fluss/server/kv/rowmerger/aggregate/functions/FieldLastNonNullValueAgg.java b/fluss-server/src/main/java/org/apache/fluss/server/kv/rowmerger/aggregate/functions/FieldLastNonNullValueAgg.java index b35f9ce477..51afbe907c 100644 --- a/fluss-server/src/main/java/org/apache/fluss/server/kv/rowmerger/aggregate/functions/FieldLastNonNullValueAgg.java +++ b/fluss-server/src/main/java/org/apache/fluss/server/kv/rowmerger/aggregate/functions/FieldLastNonNullValueAgg.java @@ -40,4 +40,21 @@ public FieldLastNonNullValueAgg(DataType dataType) { public Object agg(Object accumulator, Object inputField) { return (inputField == null) ? accumulator : inputField; } + + @Override + public boolean supportsRetract() { + return true; + } + + /** + * Retracts by clearing the accumulator to {@code null} when {@code retractField} is non-null. + * If {@code retractField} is null, the accumulator is unchanged (consistent with the forward + * path which ignores null inputs). This is a best-effort semantic aligned with Paimon: since + * {@code last_value_ignore_nulls} does not maintain history, precise retraction is not + * possible. + */ + @Override + public Object retract(Object accumulator, Object retractField) { + return retractField != null ? null : accumulator; + } } diff --git a/fluss-server/src/main/java/org/apache/fluss/server/kv/rowmerger/aggregate/functions/FieldLastValueAgg.java b/fluss-server/src/main/java/org/apache/fluss/server/kv/rowmerger/aggregate/functions/FieldLastValueAgg.java index 69593c6713..aece6d2eec 100644 --- a/fluss-server/src/main/java/org/apache/fluss/server/kv/rowmerger/aggregate/functions/FieldLastValueAgg.java +++ b/fluss-server/src/main/java/org/apache/fluss/server/kv/rowmerger/aggregate/functions/FieldLastValueAgg.java @@ -37,4 +37,20 @@ public FieldLastValueAgg(DataType dataType) { public Object agg(Object accumulator, Object inputField) { return inputField; } + + @Override + public boolean supportsRetract() { + return true; + } + + /** + * Retracts by unconditionally clearing the accumulator to {@code null}, regardless of whether + * {@code retractField} matches the current accumulator. This is a best-effort semantic aligned + * with Paimon: since {@code last_value} does not maintain history, precise retraction is not + * possible. + */ + @Override + public Object retract(Object accumulator, Object retractField) { + return null; + } } diff --git a/fluss-server/src/main/java/org/apache/fluss/server/kv/rowmerger/aggregate/functions/FieldProductAgg.java b/fluss-server/src/main/java/org/apache/fluss/server/kv/rowmerger/aggregate/functions/FieldProductAgg.java index da640abcb7..13141625d3 100644 --- a/fluss-server/src/main/java/org/apache/fluss/server/kv/rowmerger/aggregate/functions/FieldProductAgg.java +++ b/fluss-server/src/main/java/org/apache/fluss/server/kv/rowmerger/aggregate/functions/FieldProductAgg.java @@ -28,6 +28,9 @@ import java.math.BigDecimal; import static org.apache.fluss.row.Decimal.fromBigDecimal; +import static org.apache.fluss.server.kv.rowmerger.aggregate.functions.NarrowMathUtils.checkDecimalConsistency; +import static org.apache.fluss.server.kv.rowmerger.aggregate.functions.NarrowMathUtils.multiplyExactByte; +import static org.apache.fluss.server.kv.rowmerger.aggregate.functions.NarrowMathUtils.multiplyExactShort; /** Product aggregator - computes the product of numeric values. */ public class FieldProductAgg extends FieldAggregator { @@ -51,26 +54,23 @@ public Object agg(Object accumulator, Object inputField) { case DECIMAL: Decimal mergeFieldDD = (Decimal) accumulator; Decimal inFieldDD = (Decimal) inputField; - assert mergeFieldDD.scale() == inFieldDD.scale() - : "Inconsistent scale of aggregate Decimal!"; - assert mergeFieldDD.precision() == inFieldDD.precision() - : "Inconsistent precision of aggregate Decimal!"; + checkDecimalConsistency(mergeFieldDD, inFieldDD); BigDecimal bigDecimal = mergeFieldDD.toBigDecimal(); BigDecimal bigDecimal1 = inFieldDD.toBigDecimal(); BigDecimal mul = bigDecimal.multiply(bigDecimal1); product = fromBigDecimal(mul, mergeFieldDD.precision(), mergeFieldDD.scale()); break; case TINYINT: - product = (byte) ((byte) accumulator * (byte) inputField); + product = multiplyExactByte((byte) accumulator, (byte) inputField); break; case SMALLINT: - product = (short) ((short) accumulator * (short) inputField); + product = multiplyExactShort((short) accumulator, (short) inputField); break; case INTEGER: - product = (int) accumulator * (int) inputField; + product = Math.multiplyExact((int) accumulator, (int) inputField); break; case BIGINT: - product = (long) accumulator * (long) inputField; + product = Math.multiplyExact((long) accumulator, (long) inputField); break; case FLOAT: product = (float) accumulator * (float) inputField; @@ -81,9 +81,15 @@ public Object agg(Object accumulator, Object inputField) { default: String msg = String.format( - "type %s not support in %s", typeRoot, this.getClass().getName()); + "Type %s is not supported in %s", + typeRoot, this.getClass().getName()); throw new IllegalArgumentException(msg); } return product; } + + @Override + public boolean supportsRetract() { + return false; + } } diff --git a/fluss-server/src/main/java/org/apache/fluss/server/kv/rowmerger/aggregate/functions/FieldSumAgg.java b/fluss-server/src/main/java/org/apache/fluss/server/kv/rowmerger/aggregate/functions/FieldSumAgg.java index 3c2e101df3..c0e84f347d 100644 --- a/fluss-server/src/main/java/org/apache/fluss/server/kv/rowmerger/aggregate/functions/FieldSumAgg.java +++ b/fluss-server/src/main/java/org/apache/fluss/server/kv/rowmerger/aggregate/functions/FieldSumAgg.java @@ -26,6 +26,12 @@ import org.apache.fluss.types.DataType; import org.apache.fluss.utils.DecimalUtils; +import static org.apache.fluss.server.kv.rowmerger.aggregate.functions.NarrowMathUtils.addExactByte; +import static org.apache.fluss.server.kv.rowmerger.aggregate.functions.NarrowMathUtils.addExactShort; +import static org.apache.fluss.server.kv.rowmerger.aggregate.functions.NarrowMathUtils.checkDecimalConsistency; +import static org.apache.fluss.server.kv.rowmerger.aggregate.functions.NarrowMathUtils.subtractExactByte; +import static org.apache.fluss.server.kv.rowmerger.aggregate.functions.NarrowMathUtils.subtractExactShort; + /** Sum aggregator - computes the sum of numeric values. */ public class FieldSumAgg extends FieldAggregator { @@ -47,10 +53,7 @@ public Object agg(Object accumulator, Object inputField) { case DECIMAL: Decimal mergeFieldDD = (Decimal) accumulator; Decimal inFieldDD = (Decimal) inputField; - assert mergeFieldDD.scale() == inFieldDD.scale() - : "Inconsistent scale of aggregate Decimal!"; - assert mergeFieldDD.precision() == inFieldDD.precision() - : "Inconsistent precision of aggregate Decimal!"; + checkDecimalConsistency(mergeFieldDD, inFieldDD); sum = DecimalUtils.add( mergeFieldDD, @@ -59,16 +62,16 @@ public Object agg(Object accumulator, Object inputField) { mergeFieldDD.scale()); break; case TINYINT: - sum = (byte) ((byte) accumulator + (byte) inputField); + sum = addExactByte((byte) accumulator, (byte) inputField); break; case SMALLINT: - sum = (short) ((short) accumulator + (short) inputField); + sum = addExactShort((short) accumulator, (short) inputField); break; case INTEGER: - sum = (int) accumulator + (int) inputField; + sum = Math.addExact((int) accumulator, (int) inputField); break; case BIGINT: - sum = (long) accumulator + (long) inputField; + sum = Math.addExact((long) accumulator, (long) inputField); break; case FLOAT: sum = (float) accumulator + (float) inputField; @@ -79,9 +82,67 @@ public Object agg(Object accumulator, Object inputField) { default: String msg = String.format( - "type %s not support in %s", typeRoot, this.getClass().getName()); + "Type %s is not supported in %s", + typeRoot, this.getClass().getName()); throw new IllegalArgumentException(msg); } return sum; } + + @Override + public boolean supportsRetract() { + return true; + } + + @Override + public Object retract(Object accumulator, Object retractField) { + if (accumulator == null) { + return null; + } + if (retractField == null) { + return accumulator; + } + + Object result; + try { + result = subtractByType(accumulator, retractField); + } catch (ArithmeticException e) { + ArithmeticException wrapped = + new ArithmeticException( + String.format( + "Overflow in SUM retract (%s): %s - %s", + typeRoot, accumulator, retractField)); + wrapped.initCause(e); + throw wrapped; + } + return result; + } + + private Object subtractByType(Object accumulator, Object retractField) { + switch (typeRoot) { + case DECIMAL: + Decimal accDD = (Decimal) accumulator; + Decimal retDD = (Decimal) retractField; + checkDecimalConsistency(accDD, retDD); + return DecimalUtils.subtract(accDD, retDD, accDD.precision(), accDD.scale()); + case TINYINT: + return subtractExactByte((byte) accumulator, (byte) retractField); + case SMALLINT: + return subtractExactShort((short) accumulator, (short) retractField); + case INTEGER: + return Math.subtractExact((int) accumulator, (int) retractField); + case BIGINT: + return Math.subtractExact((long) accumulator, (long) retractField); + case FLOAT: + return (float) accumulator - (float) retractField; + case DOUBLE: + return (double) accumulator - (double) retractField; + default: + String msg = + String.format( + "Type %s is not supported in %s", + typeRoot, this.getClass().getName()); + throw new IllegalArgumentException(msg); + } + } } diff --git a/fluss-server/src/main/java/org/apache/fluss/server/kv/rowmerger/aggregate/functions/NarrowMathUtils.java b/fluss-server/src/main/java/org/apache/fluss/server/kv/rowmerger/aggregate/functions/NarrowMathUtils.java new file mode 100644 index 0000000000..e8ea2d8d45 --- /dev/null +++ b/fluss-server/src/main/java/org/apache/fluss/server/kv/rowmerger/aggregate/functions/NarrowMathUtils.java @@ -0,0 +1,89 @@ +/* + * 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.kv.rowmerger.aggregate.functions; + +import org.apache.fluss.row.Decimal; + +import static org.apache.fluss.utils.Preconditions.checkArgument; + +/** + * Overflow-safe arithmetic for narrow numeric types ({@code byte} and {@code short}) that {@link + * Math} does not cover, plus shared Decimal validation used by multiple aggregators. + */ +final class NarrowMathUtils { + + private NarrowMathUtils() {} + + static byte addExactByte(byte a, byte b) { + int r = a + b; + if (r < Byte.MIN_VALUE || r > Byte.MAX_VALUE) { + throw new ArithmeticException("byte addition out of range"); + } + return (byte) r; + } + + static short addExactShort(short a, short b) { + int r = a + b; + if (r < Short.MIN_VALUE || r > Short.MAX_VALUE) { + throw new ArithmeticException("short addition out of range"); + } + return (short) r; + } + + static byte subtractExactByte(byte a, byte b) { + int r = a - b; + if (r < Byte.MIN_VALUE || r > Byte.MAX_VALUE) { + throw new ArithmeticException("byte subtraction out of range"); + } + return (byte) r; + } + + static short subtractExactShort(short a, short b) { + int r = a - b; + if (r < Short.MIN_VALUE || r > Short.MAX_VALUE) { + throw new ArithmeticException("short subtraction out of range"); + } + return (short) r; + } + + static byte multiplyExactByte(byte a, byte b) { + int r = a * b; + if (r < Byte.MIN_VALUE || r > Byte.MAX_VALUE) { + throw new ArithmeticException("byte multiplication out of range"); + } + return (byte) r; + } + + static short multiplyExactShort(short a, short b) { + int r = a * b; + if (r < Short.MIN_VALUE || r > Short.MAX_VALUE) { + throw new ArithmeticException("short multiplication out of range"); + } + return (short) r; + } + + static void checkDecimalConsistency(Decimal a, Decimal b) { + checkArgument(a.scale() == b.scale(), "Inconsistent scale: %s vs %s", a.scale(), b.scale()); + checkArgument( + a.precision() == b.precision(), + "Inconsistent precision: %s vs %s", + a.precision(), + b.precision()); + } +} 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..c341fd0aea 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 @@ -991,7 +991,8 @@ public LogAppendInfo putRecordsToLeader( KvRecordBatch kvRecords, @Nullable int[] targetColumns, MergeMode mergeMode, - int requiredAcks) + int requiredAcks, + short apiVersion) throws Exception { return inReadLock( leaderIsrUpdateLock, @@ -1009,7 +1010,8 @@ public LogAppendInfo putRecordsToLeader( kv, "KvTablet for the replica to put kv records shouldn't be null."); LogAppendInfo logAppendInfo; try { - logAppendInfo = kv.putAsLeader(kvRecords, targetColumns, mergeMode); + logAppendInfo = + kv.putAsLeader(kvRecords, targetColumns, mergeMode, apiVersion); } catch (IOException e) { LOG.error("Error while putting records to {}", tableBucket, e); fatalErrorHandler.onFatalError(e); 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..3e5c81ba6c 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 @@ -1280,7 +1280,11 @@ private Map putToLocalKv( tableMetrics.totalPutKvRequests().inc(); LogAppendInfo appendInfo = replica.putRecordsToLeader( - entry.getValue(), targetColumns, mergeMode, requiredAcks); + entry.getValue(), + targetColumns, + mergeMode, + requiredAcks, + apiVersion); LOG.trace( "Written to local kv for {}, and the cdc log beginning at offset {} and ending at offset {}", tb, 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..eb6570128a 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 @@ -229,11 +229,8 @@ public CompletableFuture putKv(PutKvRequest request) { authorizeTable(WRITE, request.getTableId()); Map putKvData = getPutKvData(request); - // Get mergeMode from request, default to DEFAULT if not set - MergeMode mergeMode = - request.hasAggMode() - ? MergeMode.fromValue(request.getAggMode()) - : MergeMode.DEFAULT; + int aggMode = request.hasAggMode() ? request.getAggMode() : 0; + MergeMode mergeMode = MergeMode.fromProtoValue(aggMode); CompletableFuture response = new CompletableFuture<>(); replicaManager.putRecordsToKv( request.getTimeoutMs(), 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..25acc16357 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 @@ -19,6 +19,7 @@ import org.apache.fluss.config.Configuration; import org.apache.fluss.config.TableConfig; +import org.apache.fluss.exception.InvalidRecordException; import org.apache.fluss.memory.TestingMemorySegmentPool; import org.apache.fluss.metadata.AggFunctions; import org.apache.fluss.metadata.KvFormat; @@ -32,12 +33,16 @@ import org.apache.fluss.record.KvRecord; import org.apache.fluss.record.KvRecordBatch; import org.apache.fluss.record.KvRecordTestUtils; +import org.apache.fluss.record.KvRecordTestUtils.V2RecordEntry; +import org.apache.fluss.record.LogRecordBatch; import org.apache.fluss.record.LogRecords; import org.apache.fluss.record.MemoryLogRecords; +import org.apache.fluss.record.MutationType; import org.apache.fluss.record.TestingSchemaGetter; import org.apache.fluss.rpc.protocol.MergeMode; import org.apache.fluss.server.kv.autoinc.AutoIncrementManager; import org.apache.fluss.server.kv.autoinc.TestingSequenceGeneratorFactory; +import org.apache.fluss.server.kv.prewrite.KvPreWriteBuffer.Key; import org.apache.fluss.server.kv.rowmerger.RowMerger; import org.apache.fluss.server.log.FetchIsolation; import org.apache.fluss.server.log.LogTablet; @@ -67,6 +72,8 @@ import static org.apache.fluss.record.LogRecordBatchFormat.NO_WRITER_ID; import static org.apache.fluss.testutils.DataTestUtils.createBasicMemoryLogRecords; import static org.apache.fluss.testutils.LogRecordsAssert.assertThatLogRecords; +import static org.assertj.core.api.Assertions.assertThat; +import static org.assertj.core.api.Assertions.assertThatThrownBy; /** * Tests for {@link KvTablet} with {@link MergeMode} support. @@ -101,17 +108,48 @@ class KvTabletMergeModeTest { private static final RowType AGG_ROW_TYPE = AGG_SCHEMA.getRowType(); + // Schema with only retract-safe functions (SUM, LAST_VALUE) for retract tests + private static final Schema RETRACT_SAFE_SCHEMA = + Schema.newBuilder() + .column("id", DataTypes.INT()) + .column("count", DataTypes.BIGINT(), AggFunctions.SUM()) + .column("name", DataTypes.STRING(), AggFunctions.LAST_VALUE()) + .primaryKey("id") + .build(); + + private static final RowType RETRACT_SAFE_ROW_TYPE = RETRACT_SAFE_SCHEMA.getRowType(); + private final KvRecordTestUtils.KvRecordFactory kvRecordFactory = KvRecordTestUtils.KvRecordFactory.of(AGG_ROW_TYPE); @BeforeEach void setUp() throws Exception { + initTablets(AGG_SCHEMA); + } + + /** Reinitialize tablets with a different schema. Closes existing tablets first. */ + private void initTablets(Schema schema) throws Exception { + initTablets(schema, Collections.emptyMap()); + } + + /** + * Reinitialize tablets with a different schema and extra config. Closes existing tablets first. + */ + private void initTablets(Schema schema, Map extraConfig) throws Exception { + if (kvTablet != null) { + kvTablet.close(); + } + if (logTablet != null) { + logTablet.close(); + } + Map config = new HashMap<>(); config.put("table.merge-engine", "aggregation"); + config.putAll(extraConfig); TablePath tablePath = TablePath.of("testDb", "test_merge_mode"); PhysicalTablePath physicalTablePath = PhysicalTablePath.of(tablePath); - schemaGetter = new TestingSchemaGetter(new SchemaInfo(AGG_SCHEMA, SCHEMA_ID)); + schemaGetter = new TestingSchemaGetter(new SchemaInfo(schema, SCHEMA_ID)); File logTabletDir = LogTestUtils.makeRandomLogTabletDir( @@ -492,6 +530,549 @@ void testOverwriteModeWithMultipleKeys() throws Exception { .isEqualTo(expectedLogs); } + // ==================== Per-Record MutationType RETRACT Tests (V2 format) ==================== + + @Test + void testRetractPairedWithUpsertOnExistingKey() throws Exception { + initTablets(RETRACT_SAFE_SCHEMA); + + KvRecordTestUtils.KvRecordFactory retractKvRecordFactory = + KvRecordTestUtils.KvRecordFactory.of(RETRACT_SAFE_ROW_TYPE); + + // Insert initial state: key=k1, count=100, name="init" + KvRecordBatch initBatch = + kvRecordBatchFactory.ofRecords( + retractKvRecordFactory.ofRecord( + "k1".getBytes(), new Object[] {1, 100L, "init"})); + kvTablet.putAsLeader(initBatch, null, MergeMode.DEFAULT); + + long endOffset = logTablet.localLogEndOffset(); + + // Send V2 batch: RETRACT(20, "old") + UPSERT(30, "new") for same key + // count: 100-20+30=110, name: retract "old" then upsert "new" → "new" + List entries = + Arrays.asList( + V2RecordEntry.of( + MutationType.RETRACT, + retractKvRecordFactory.ofRecord( + "k1".getBytes(), new Object[] {1, 20L, "old"})), + V2RecordEntry.of( + MutationType.UPSERT, + retractKvRecordFactory.ofRecord( + "k1".getBytes(), new Object[] {1, 30L, "new"}))); + KvRecordBatch batch = kvRecordBatchFactory.ofV2Records(entries); + kvTablet.putAsLeader(batch, null, MergeMode.DEFAULT, (short) 2); + + // Verify UB(old) + UA(new) CDC entries + LogRecords actualLogRecords = readLogRecords(endOffset); + MemoryLogRecords expectedLogs = + logRecords( + RETRACT_SAFE_ROW_TYPE, + endOffset, + Arrays.asList(ChangeType.UPDATE_BEFORE, ChangeType.UPDATE_AFTER), + Arrays.asList( + new Object[] {1, 100L, "init"}, new Object[] {1, 110L, "new"})); + + assertThatLogRecords(actualLogRecords) + .withSchema(RETRACT_SAFE_ROW_TYPE) + .assertCheckSum(true) + .isEqualTo(expectedLogs); + } + + @Test + void testRetractUnpairedOnExistingKey() throws Exception { + initTablets(RETRACT_SAFE_SCHEMA); + + KvRecordTestUtils.KvRecordFactory retractKvRecordFactory = + KvRecordTestUtils.KvRecordFactory.of(RETRACT_SAFE_ROW_TYPE); + + // Initial state: key=k1, count=100, name="init" + KvRecordBatch initBatch = + kvRecordBatchFactory.ofRecords( + retractKvRecordFactory.ofRecord( + "k1".getBytes(), new Object[] {1, 100L, "init"})); + kvTablet.putAsLeader(initBatch, null, MergeMode.DEFAULT); + + long endOffset = logTablet.localLogEndOffset(); + + // Send V2 batch: single RETRACT(20, "old") — no following upsert + // count: 100-20=80, name: retract "old" → null (last_value retract semantics) + List entries = + Collections.singletonList( + V2RecordEntry.of( + MutationType.RETRACT, + retractKvRecordFactory.ofRecord( + "k1".getBytes(), new Object[] {1, 20L, "old"}))); + KvRecordBatch batch = kvRecordBatchFactory.ofV2Records(entries); + kvTablet.putAsLeader(batch, null, MergeMode.DEFAULT, (short) 2); + + // Verify UB(old) + UA(intermediate) CDC entries + LogRecords actualLogRecords = readLogRecords(endOffset); + MemoryLogRecords expectedLogs = + logRecords( + RETRACT_SAFE_ROW_TYPE, + endOffset, + Arrays.asList(ChangeType.UPDATE_BEFORE, ChangeType.UPDATE_AFTER), + Arrays.asList(new Object[] {1, 100L, "init"}, new Object[] {1, 80L, null})); + + assertThatLogRecords(actualLogRecords) + .withSchema(RETRACT_SAFE_ROW_TYPE) + .assertCheckSum(true) + .isEqualTo(expectedLogs); + } + + @Test + void testRetractOnNonExistentKey() throws Exception { + initTablets(RETRACT_SAFE_SCHEMA); + + KvRecordTestUtils.KvRecordFactory retractKvRecordFactory = + KvRecordTestUtils.KvRecordFactory.of(RETRACT_SAFE_ROW_TYPE); + + long startOffset = logTablet.localLogEndOffset(); + + // Send V2 batch: RETRACT on a key that doesn't exist — should be skipped + List entries = + Collections.singletonList( + V2RecordEntry.of( + MutationType.RETRACT, + retractKvRecordFactory.ofRecord( + "k1".getBytes(), new Object[] {1, 10L, "ghost"}))); + KvRecordBatch batch = kvRecordBatchFactory.ofV2Records(entries); + kvTablet.putAsLeader(batch, null, MergeMode.DEFAULT, (short) 2); + + // Verify no CDC records generated + LogRecords logRecords = readLogRecords(startOffset); + int totalRecordCount = 0; + for (LogRecordBatch logBatch : logRecords.batches()) { + totalRecordCount += logBatch.getRecordCount(); + } + assertThat(totalRecordCount) + .as("No changelog records should be emitted for retract on non-existent key") + .isEqualTo(0); + + // Verify the key is still absent from the KV store (no spurious write) + assertThat(kvTablet.getKvPreWriteBuffer().get(Key.of("k1".getBytes()))).isNull(); + } + + @Test + void testRetractIndependentLastValueProducesNullIntermediate() throws Exception { + // Schema where the only value column is LAST_VALUE. + // LAST_VALUE.retract() always returns null for the column value, so an unpaired + // RETRACT produces an intermediate row with all non-PK fields null. + // With the zombie row fix, this is detected as a fully-retracted row and + // treated as a deletion. With DeleteBehavior.ALLOW, it produces a DELETE CDC entry. + Schema lastValueOnlySchema = + Schema.newBuilder() + .column("id", DataTypes.INT()) + .column("name", DataTypes.STRING(), AggFunctions.LAST_VALUE()) + .primaryKey("id") + .build(); + Map allowDelete = new HashMap<>(); + allowDelete.put("table.delete.behavior", "allow"); + initTablets(lastValueOnlySchema, allowDelete); + + RowType lastValueRowType = lastValueOnlySchema.getRowType(); + KvRecordTestUtils.KvRecordFactory lastValueKvRecordFactory = + KvRecordTestUtils.KvRecordFactory.of(lastValueRowType); + + // Insert initial state: key=k1, name="hello" + KvRecordBatch initBatch = + kvRecordBatchFactory.ofRecords( + lastValueKvRecordFactory.ofRecord( + "k1".getBytes(), new Object[] {1, "hello"})); + kvTablet.putAsLeader(initBatch, null, MergeMode.DEFAULT); + + long endOffset = logTablet.localLogEndOffset(); + + // Send V2 batch: single unpaired RETRACT(k1, "hello") + // LAST_VALUE.retract("hello") → null column value → all non-PK fields null → row removed + List entries = + Collections.singletonList( + V2RecordEntry.of( + MutationType.RETRACT, + lastValueKvRecordFactory.ofRecord( + "k1".getBytes(), new Object[] {1, "hello"}))); + KvRecordBatch batch = kvRecordBatchFactory.ofV2Records(entries); + kvTablet.putAsLeader(batch, null, MergeMode.DEFAULT, (short) 2); + + // Verify CDC: DELETE(1, "hello") — zombie row is removed + LogRecords actualLogRecords = readLogRecords(endOffset); + MemoryLogRecords expectedLogs = + logRecords( + lastValueRowType, + endOffset, + Collections.singletonList(ChangeType.DELETE), + Collections.singletonList(new Object[] {1, "hello"})); + + assertThatLogRecords(actualLogRecords) + .withSchema(lastValueRowType) + .assertCheckSum(true) + .isEqualTo(expectedLogs); + } + + @Test + void testRetractRejectedForNonAggregationTable() throws Exception { + // Create a table with first_row merge engine (non-aggregation) + Schema defaultSchema = + Schema.newBuilder() + .column("id", DataTypes.INT()) + .column("name", DataTypes.STRING()) + .primaryKey("id") + .build(); + Map firstRowConfig = new HashMap<>(); + firstRowConfig.put("table.merge-engine", "first_row"); + initTablets(defaultSchema, firstRowConfig); + + RowType defaultRowType = defaultSchema.getRowType(); + KvRecordTestUtils.KvRecordFactory defaultKvRecordFactory = + KvRecordTestUtils.KvRecordFactory.of(defaultRowType); + + // Send a V2 RETRACT record to a non-aggregation table + List entries = + Collections.singletonList( + V2RecordEntry.of( + MutationType.RETRACT, + defaultKvRecordFactory.ofRecord( + "k1".getBytes(), new Object[] {1, "old"}))); + KvRecordBatch batch = kvRecordBatchFactory.ofV2Records(entries); + + assertThatThrownBy(() -> kvTablet.putAsLeader(batch, null, MergeMode.DEFAULT, (short) 2)) + .isInstanceOf(InvalidRecordException.class) + .hasMessageContaining("RETRACT") + .hasMessageContaining("aggregation"); + } + + @Test + void testRetractRejectedForNonRetractableFunction() throws Exception { + // AGG_SCHEMA has MAX which does not support retract + // Send a V2 RETRACT record — should be rejected + KvRecordTestUtils.KvRecordFactory aggKvRecordFactory = + KvRecordTestUtils.KvRecordFactory.of(AGG_ROW_TYPE); + + // Insert initial record first + KvRecordBatch initBatch = + kvRecordBatchFactory.ofRecords( + aggKvRecordFactory.ofRecord( + "k1".getBytes(), new Object[] {1, 10L, 100, "Alice"})); + kvTablet.putAsLeader(initBatch, null, MergeMode.DEFAULT); + + List entries = + Collections.singletonList( + V2RecordEntry.of( + MutationType.RETRACT, + aggKvRecordFactory.ofRecord( + "k1".getBytes(), new Object[] {1, 5L, 50, "old"}))); + KvRecordBatch batch = kvRecordBatchFactory.ofV2Records(entries); + + assertThatThrownBy(() -> kvTablet.putAsLeader(batch, null, MergeMode.DEFAULT, (short) 2)) + .isInstanceOf(InvalidRecordException.class) + .hasMessageContaining("RETRACT") + .hasMessageContaining("retract-safe"); + } + + @Test + void testMixedUpsertAndRetractInSameBatch() throws Exception { + initTablets(RETRACT_SAFE_SCHEMA); + + KvRecordTestUtils.KvRecordFactory retractKvRecordFactory = + KvRecordTestUtils.KvRecordFactory.of(RETRACT_SAFE_ROW_TYPE); + + // Insert initial state for k1 + KvRecordBatch initBatch = + kvRecordBatchFactory.ofRecords( + retractKvRecordFactory.ofRecord( + "k1".getBytes(), new Object[] {1, 100L, "init"})); + kvTablet.putAsLeader(initBatch, null, MergeMode.DEFAULT); + + long endOffset = logTablet.localLogEndOffset(); + + // Mixed batch: UPSERT(k2) + RETRACT(k1) + UPSERT(k1) + // k2 is new → INSERT + // k1: retract(20,"old") + upsert(30,"new") → 100-20+30=110 + List entries = + Arrays.asList( + V2RecordEntry.of( + MutationType.UPSERT, + retractKvRecordFactory.ofRecord( + "k2".getBytes(), new Object[] {2, 50L, "Bob"})), + V2RecordEntry.of( + MutationType.RETRACT, + retractKvRecordFactory.ofRecord( + "k1".getBytes(), new Object[] {1, 20L, "old"})), + V2RecordEntry.of( + MutationType.UPSERT, + retractKvRecordFactory.ofRecord( + "k1".getBytes(), new Object[] {1, 30L, "new"}))); + KvRecordBatch batch = kvRecordBatchFactory.ofV2Records(entries); + kvTablet.putAsLeader(batch, null, MergeMode.DEFAULT, (short) 2); + + // Verify: INSERT(k2) + UB(k1,old) + UA(k1,new) + LogRecords actualLogRecords = readLogRecords(endOffset); + MemoryLogRecords expectedLogs = + logRecords( + RETRACT_SAFE_ROW_TYPE, + endOffset, + Arrays.asList( + ChangeType.INSERT, + ChangeType.UPDATE_BEFORE, + ChangeType.UPDATE_AFTER), + Arrays.asList( + new Object[] {2, 50L, "Bob"}, + new Object[] {1, 100L, "init"}, + new Object[] {1, 110L, "new"})); + + assertThatLogRecords(actualLogRecords) + .withSchema(RETRACT_SAFE_ROW_TYPE) + .assertCheckSum(true) + .isEqualTo(expectedLogs); + } + + @Test + void testRetractMultiplePairsSameKeyInOneBatch() throws Exception { + initTablets(RETRACT_SAFE_SCHEMA); + + KvRecordTestUtils.KvRecordFactory retractKvRecordFactory = + KvRecordTestUtils.KvRecordFactory.of(RETRACT_SAFE_ROW_TYPE); + + // Initial state: key=k1, count=100, name="init" + KvRecordBatch initBatch = + kvRecordBatchFactory.ofRecords( + retractKvRecordFactory.ofRecord( + "k1".getBytes(), new Object[] {1, 100L, "init"})); + kvTablet.putAsLeader(initBatch, null, MergeMode.DEFAULT); + + long endOffset = logTablet.localLogEndOffset(); + + // Send 2 retract+upsert pairs for same key in one V2 batch: + // Pair 1: RETRACT(20, "old1") + UPSERT(30, "new1") + // count: 100-20+30=110, name: "new1" + // Pair 2: RETRACT(10, "old2") + UPSERT(50, "new2") + // count: 110-10+50=150, name: "new2" + List entries = + Arrays.asList( + V2RecordEntry.of( + MutationType.RETRACT, + retractKvRecordFactory.ofRecord( + "k1".getBytes(), new Object[] {1, 20L, "old1"})), + V2RecordEntry.of( + MutationType.UPSERT, + retractKvRecordFactory.ofRecord( + "k1".getBytes(), new Object[] {1, 30L, "new1"})), + V2RecordEntry.of( + MutationType.RETRACT, + retractKvRecordFactory.ofRecord( + "k1".getBytes(), new Object[] {1, 10L, "old2"})), + V2RecordEntry.of( + MutationType.UPSERT, + retractKvRecordFactory.ofRecord( + "k1".getBytes(), new Object[] {1, 50L, "new2"}))); + KvRecordBatch batch = kvRecordBatchFactory.ofV2Records(entries); + kvTablet.putAsLeader(batch, null, MergeMode.DEFAULT, (short) 2); + + // Verify changelog shows the transitions + LogRecords actualLogRecords = readLogRecords(endOffset); + + // Pair 1 produces: UB(100,"init") + UA(110,"new1") + // Pair 2 produces: UB(110,"new1") + UA(150,"new2") + MemoryLogRecords expectedLogs = + logRecords( + RETRACT_SAFE_ROW_TYPE, + endOffset, + Arrays.asList( + ChangeType.UPDATE_BEFORE, + ChangeType.UPDATE_AFTER, + ChangeType.UPDATE_BEFORE, + ChangeType.UPDATE_AFTER), + Arrays.asList( + new Object[] {1, 100L, "init"}, + new Object[] {1, 110L, "new1"}, + new Object[] {1, 110L, "new1"}, + new Object[] {1, 150L, "new2"})); + + assertThatLogRecords(actualLogRecords) + .withSchema(RETRACT_SAFE_ROW_TYPE) + .assertCheckSum(true) + .isEqualTo(expectedLogs); + } + + @Test + void testRetractToZeroThenRescue() throws Exception { + // Schema uses SUM for count. Initial state: key=k1, total=10 + Schema sumSchema = + Schema.newBuilder() + .column("id", DataTypes.INT()) + .column("total", DataTypes.BIGINT(), AggFunctions.SUM()) + .primaryKey("id") + .build(); + initTablets(sumSchema); + + RowType sumRowType = sumSchema.getRowType(); + KvRecordTestUtils.KvRecordFactory sumKvRecordFactory = + KvRecordTestUtils.KvRecordFactory.of(sumRowType); + + // Initial state: key=k1, total=10 + KvRecordBatch initBatch = + kvRecordBatchFactory.ofRecords( + sumKvRecordFactory.ofRecord("k1".getBytes(), new Object[] {1, 10L})); + kvTablet.putAsLeader(initBatch, null, MergeMode.DEFAULT); + + long endOffset = logTablet.localLogEndOffset(); + + // V2 batch: RETRACT(10) + UPSERT(30) + // Retract: 10-10=0 (intermediate) + // Upsert: 0+30=30 + List entries = + Arrays.asList( + V2RecordEntry.of( + MutationType.RETRACT, + sumKvRecordFactory.ofRecord( + "k1".getBytes(), new Object[] {1, 10L})), + V2RecordEntry.of( + MutationType.UPSERT, + sumKvRecordFactory.ofRecord( + "k1".getBytes(), new Object[] {1, 30L}))); + KvRecordBatch batch = kvRecordBatchFactory.ofV2Records(entries); + kvTablet.putAsLeader(batch, null, MergeMode.DEFAULT, (short) 2); + + // Verify key is NOT deleted and final value is 30 + LogRecords actualLogRecords = readLogRecords(endOffset); + MemoryLogRecords expectedLogs = + logRecords( + sumRowType, + endOffset, + Arrays.asList(ChangeType.UPDATE_BEFORE, ChangeType.UPDATE_AFTER), + Arrays.asList( + new Object[] {1, 10L}, // before + new Object[] {1, 30L} // after: rescued from zero + )); + + assertThatLogRecords(actualLogRecords) + .withSchema(sumRowType) + .assertCheckSum(true) + .isEqualTo(expectedLogs); + } + + @Test + void testRetractNoChangeOptimization() throws Exception { + // Use a simple SUM schema + Schema sumSchema = + Schema.newBuilder() + .column("id", DataTypes.INT()) + .column("total", DataTypes.BIGINT(), AggFunctions.SUM()) + .primaryKey("id") + .build(); + initTablets(sumSchema); + + RowType sumRowType = sumSchema.getRowType(); + KvRecordTestUtils.KvRecordFactory sumKvRecordFactory = + KvRecordTestUtils.KvRecordFactory.of(sumRowType); + + // Initial state: key=k1, total=10 + KvRecordBatch initBatch = + kvRecordBatchFactory.ofRecords( + sumKvRecordFactory.ofRecord("k1".getBytes(), new Object[] {1, 10L})); + kvTablet.putAsLeader(initBatch, null, MergeMode.DEFAULT); + + long endOffset = logTablet.localLogEndOffset(); + + // V2 batch: RETRACT(5) + UPSERT(5) → 10-5+5=10 (no change) + List entries = + Arrays.asList( + V2RecordEntry.of( + MutationType.RETRACT, + sumKvRecordFactory.ofRecord("k1".getBytes(), new Object[] {1, 5L})), + V2RecordEntry.of( + MutationType.UPSERT, + sumKvRecordFactory.ofRecord( + "k1".getBytes(), new Object[] {1, 5L}))); + KvRecordBatch retractBatch = kvRecordBatchFactory.ofV2Records(entries); + kvTablet.putAsLeader(retractBatch, null, MergeMode.DEFAULT, (short) 2); + + // Verify no changelog is emitted (the newValue.equals(oldValue) optimization). + LogRecords logRecords = readLogRecords(endOffset); + int totalRecordCount = 0; + for (LogRecordBatch logBatch : logRecords.batches()) { + totalRecordCount += logBatch.getRecordCount(); + } + assertThat(totalRecordCount) + .as("No changelog records should be emitted when retract+upsert produces no change") + .isEqualTo(0); + } + + @Test + void testConsecutiveRetractsThenUpsertOptimization() throws Exception { + // Verify that when a batch contains RETRACT → RETRACT → UPSERT for the same key, + // the second RETRACT gets its own peek-ahead and merges with the UPSERT, + // producing fewer CDC entries than fully independent processing. + Schema sumSchema = + Schema.newBuilder() + .column("id", DataTypes.INT()) + .column("total", DataTypes.BIGINT(), AggFunctions.SUM()) + .primaryKey("id") + .build(); + initTablets(sumSchema); + + RowType sumRowType = sumSchema.getRowType(); + KvRecordTestUtils.KvRecordFactory sumKvRecordFactory = + KvRecordTestUtils.KvRecordFactory.of(sumRowType); + + // Initial state: key=k1, total=100 + KvRecordBatch initBatch = + kvRecordBatchFactory.ofRecords( + sumKvRecordFactory.ofRecord("k1".getBytes(), new Object[] {1, 100L})); + kvTablet.putAsLeader(initBatch, null, MergeMode.DEFAULT); + + long endOffset = logTablet.localLogEndOffset(); + + // V2 batch: RETRACT(20) → RETRACT(20) → UPSERT(30) for same key + // Expected processing: + // 1st RETRACT peeks, sees 2nd RETRACT (not UPSERT) → independent: 100→80 + // CDC: UB(100) + UA(80) + // 2nd RETRACT peeks, sees UPSERT → merged: 80-20+30=90 + // CDC: UB(80) + UA(90) + // Total: 4 CDC entries (not 6 as would happen without the optimization) + List entries = + Arrays.asList( + V2RecordEntry.of( + MutationType.RETRACT, + sumKvRecordFactory.ofRecord( + "k1".getBytes(), new Object[] {1, 20L})), + V2RecordEntry.of( + MutationType.RETRACT, + sumKvRecordFactory.ofRecord( + "k1".getBytes(), new Object[] {1, 20L})), + V2RecordEntry.of( + MutationType.UPSERT, + sumKvRecordFactory.ofRecord( + "k1".getBytes(), new Object[] {1, 30L}))); + KvRecordBatch batch = kvRecordBatchFactory.ofV2Records(entries); + kvTablet.putAsLeader(batch, null, MergeMode.DEFAULT, (short) 2); + + // Verify changelog: 4 entries total + // 1st RETRACT (independent): UB(100) + UA(80) + // 2nd RETRACT+UPSERT (merged): UB(80) + UA(90) + LogRecords actualLogRecords = readLogRecords(endOffset); + MemoryLogRecords expectedLogs = + logRecords( + sumRowType, + endOffset, + Arrays.asList( + ChangeType.UPDATE_BEFORE, + ChangeType.UPDATE_AFTER, + ChangeType.UPDATE_BEFORE, + ChangeType.UPDATE_AFTER), + Arrays.asList( + new Object[] {1, 100L}, + new Object[] {1, 80L}, + new Object[] {1, 80L}, + new Object[] {1, 90L})); + + assertThatLogRecords(actualLogRecords) + .withSchema(sumRowType) + .assertCheckSum(true) + .isEqualTo(expectedLogs); + } + // ==================== Default MergeMode Tests ==================== @Test @@ -531,6 +1112,171 @@ void testDefaultMergeModeIsDefault() throws Exception { .isEqualTo(expectedLogs); } + // ==================== V2 Format Validation Tests ==================== + + @Test + void testV2UpsertWithNullRowThrows() throws Exception { + // Build a V2 batch with UPSERT mutation type but null row value + KvRecordTestUtils.KvRecordFactory aggKvRecordFactory = + KvRecordTestUtils.KvRecordFactory.of(AGG_ROW_TYPE); + + List entries = + Collections.singletonList( + V2RecordEntry.of( + MutationType.UPSERT, + aggKvRecordFactory.ofRecord("k1".getBytes(), null))); + KvRecordBatch batch = kvRecordBatchFactory.ofV2Records(entries); + + assertThatThrownBy(() -> kvTablet.putAsLeader(batch, null, MergeMode.DEFAULT, (short) 2)) + .isInstanceOf(InvalidRecordException.class) + .hasMessageContaining("V2 UPSERT record must carry a non-null row value."); + } + + @Test + void testV2RetractWithNullRowThrows() throws Exception { + initTablets(RETRACT_SAFE_SCHEMA); + + KvRecordTestUtils.KvRecordFactory retractKvRecordFactory = + KvRecordTestUtils.KvRecordFactory.of(RETRACT_SAFE_ROW_TYPE); + + List entries = + Collections.singletonList( + V2RecordEntry.of( + MutationType.RETRACT, + retractKvRecordFactory.ofRecord("k1".getBytes(), null))); + KvRecordBatch batch = kvRecordBatchFactory.ofV2Records(entries); + + assertThatThrownBy(() -> kvTablet.putAsLeader(batch, null, MergeMode.DEFAULT, (short) 2)) + .isInstanceOf(InvalidRecordException.class) + .hasMessageContaining("RETRACT record must carry a non-null row value."); + } + + @Test + void testV2DeleteOnAggregationTable() throws Exception { + // Verify that V2 DELETE behaves the same as V0 delete on an aggregation table. + // Insert initial record, then send a V2 batch with MutationType.DELETE. + // Must use DeleteBehavior.ALLOW so the delete is not silently dropped. + Map allowDelete = new HashMap<>(); + allowDelete.put("table.delete.behavior", "allow"); + initTablets(RETRACT_SAFE_SCHEMA, allowDelete); + + KvRecordTestUtils.KvRecordFactory retractKvRecordFactory = + KvRecordTestUtils.KvRecordFactory.of(RETRACT_SAFE_ROW_TYPE); + + // Insert initial state: key=k1, count=100, name="init" + KvRecordBatch initBatch = + kvRecordBatchFactory.ofRecords( + retractKvRecordFactory.ofRecord( + "k1".getBytes(), new Object[] {1, 100L, "init"})); + kvTablet.putAsLeader(initBatch, null, MergeMode.DEFAULT); + + long endOffset = logTablet.localLogEndOffset(); + + // Send V2 batch with DELETE mutation type (null row value) + List entries = + Collections.singletonList( + V2RecordEntry.of( + MutationType.DELETE, + retractKvRecordFactory.ofRecord("k1".getBytes(), null))); + KvRecordBatch batch = kvRecordBatchFactory.ofV2Records(entries); + kvTablet.putAsLeader(batch, null, MergeMode.DEFAULT, (short) 2); + + // Verify DELETE CDC is produced — same behavior as V0 delete + LogRecords actualLogRecords = readLogRecords(endOffset); + MemoryLogRecords expectedLogs = + logRecords( + RETRACT_SAFE_ROW_TYPE, + endOffset, + Collections.singletonList(ChangeType.DELETE), + Collections.singletonList(new Object[] {1, 100L, "init"})); + + assertThatLogRecords(actualLogRecords) + .withSchema(RETRACT_SAFE_ROW_TYPE) + .assertCheckSum(true) + .isEqualTo(expectedLogs); + } + + // ==================== Partial Update + Retract Tests ==================== + + @Test + void testRetractWithPartialUpdateTargetColumns() throws Exception { + // Schema with two SUM columns to verify partial update targets only one + Schema twoSumSchema = + Schema.newBuilder() + .column("id", DataTypes.INT()) + .column("col_a", DataTypes.BIGINT(), AggFunctions.SUM()) + .column("col_b", DataTypes.BIGINT(), AggFunctions.SUM()) + .primaryKey("id") + .build(); + initTablets(twoSumSchema); + + RowType twoSumRowType = twoSumSchema.getRowType(); + KvRecordTestUtils.KvRecordFactory twoSumKvRecordFactory = + KvRecordTestUtils.KvRecordFactory.of(twoSumRowType); + + // Step 1: Insert initial state: id=1, col_a=100, col_b=200 + KvRecordBatch initBatch = + kvRecordBatchFactory.ofRecords( + twoSumKvRecordFactory.ofRecord( + "k1".getBytes(), new Object[] {1, 100L, 200L})); + kvTablet.putAsLeader(initBatch, null, MergeMode.DEFAULT); + + long endOffset = logTablet.localLogEndOffset(); + + // Step 2: V2 RETRACT targeting only col_a (targetColumns = {0, 1} = id + col_a) + // retract col_a=30 → col_a: 100-30=70, col_b unchanged (200) + int[] targetColumns = new int[] {0, 1}; // id and col_a + List retractEntries = + Collections.singletonList( + V2RecordEntry.of( + MutationType.RETRACT, + twoSumKvRecordFactory.ofRecord( + "k1".getBytes(), new Object[] {1, 30L, null}))); + KvRecordBatch retractBatch = kvRecordBatchFactory.ofV2Records(retractEntries); + kvTablet.putAsLeader(retractBatch, targetColumns, MergeMode.DEFAULT, (short) 2); + + long afterRetractOffset = logTablet.localLogEndOffset(); + + // Verify CDC after retract: UB(100,200) + UA(70,200) + LogRecords retractCdc = readLogRecords(endOffset); + MemoryLogRecords expectedRetractLogs = + logRecords( + twoSumRowType, + endOffset, + Arrays.asList(ChangeType.UPDATE_BEFORE, ChangeType.UPDATE_AFTER), + Arrays.asList(new Object[] {1, 100L, 200L}, new Object[] {1, 70L, 200L})); + + assertThatLogRecords(retractCdc) + .withSchema(twoSumRowType) + .assertCheckSum(true) + .isEqualTo(expectedRetractLogs); + + // Step 3: V2 UPSERT targeting only col_a (upsert col_a=50) + // col_a: 70+50=120, col_b unchanged (200) + List upsertEntries = + Collections.singletonList( + V2RecordEntry.of( + MutationType.UPSERT, + twoSumKvRecordFactory.ofRecord( + "k1".getBytes(), new Object[] {1, 50L, null}))); + KvRecordBatch upsertBatch = kvRecordBatchFactory.ofV2Records(upsertEntries); + kvTablet.putAsLeader(upsertBatch, targetColumns, MergeMode.DEFAULT, (short) 2); + + // Verify CDC after upsert: UB(70,200) + UA(120,200) + LogRecords upsertCdc = readLogRecords(afterRetractOffset); + MemoryLogRecords expectedUpsertLogs = + logRecords( + twoSumRowType, + afterRetractOffset, + Arrays.asList(ChangeType.UPDATE_BEFORE, ChangeType.UPDATE_AFTER), + Arrays.asList(new Object[] {1, 70L, 200L}, new Object[] {1, 120L, 200L})); + + assertThatLogRecords(upsertCdc) + .withSchema(twoSumRowType) + .assertCheckSum(true) + .isEqualTo(expectedUpsertLogs); + } + // ==================== Helper Methods ==================== private LogRecords readLogRecords(long startOffset) throws Exception { @@ -540,10 +1286,21 @@ private LogRecords readLogRecords(long startOffset) throws Exception { } private MemoryLogRecords logRecords( - long baseOffset, List changeTypes, List rows) throws Exception { + RowType rowType, long baseOffset, List changeTypes, List rows) + throws Exception { + return logRecords(rowType, SCHEMA_ID, baseOffset, changeTypes, rows); + } + + private MemoryLogRecords logRecords( + RowType rowType, + short schemaId, + long baseOffset, + List changeTypes, + List rows) + throws Exception { return createBasicMemoryLogRecords( - AGG_ROW_TYPE, - SCHEMA_ID, + rowType, + schemaId, baseOffset, -1L, CURRENT_LOG_MAGIC_VALUE, @@ -554,4 +1311,9 @@ private MemoryLogRecords logRecords( LogFormat.ARROW, DEFAULT_COMPRESSION); } + + private MemoryLogRecords logRecords( + long baseOffset, List changeTypes, List rows) throws Exception { + return logRecords(AGG_ROW_TYPE, baseOffset, changeTypes, rows); + } } diff --git a/fluss-server/src/test/java/org/apache/fluss/server/kv/rowmerger/AggregateRowMergerTest.java b/fluss-server/src/test/java/org/apache/fluss/server/kv/rowmerger/AggregateRowMergerTest.java index 387fb95bb0..f279d6336f 100644 --- a/fluss-server/src/test/java/org/apache/fluss/server/kv/rowmerger/AggregateRowMergerTest.java +++ b/fluss-server/src/test/java/org/apache/fluss/server/kv/rowmerger/AggregateRowMergerTest.java @@ -144,6 +144,22 @@ void testDeleteBehaviorNotAllowed() { assertThat(deleted).isNull(); } + @Test + void testProductAggregationIsNotRetractSafe() { + Schema schema = + Schema.newBuilder() + .column("id", DataTypes.INT()) + .column("value", DataTypes.DOUBLE(), AggFunctions.PRODUCT()) + .primaryKey("id") + .build(); + + TableConfig tableConfig = new TableConfig(new Configuration()); + AggregateRowMerger merger = createMerger(schema, tableConfig); + merger.configureTargetColumns(null, SCHEMA_ID, schema); + + assertThat(merger.supportsRetract()).isFalse(); + } + @Test void testNullValueHandling() { Schema schema = @@ -1005,4 +1021,401 @@ private AggregateRowMerger createMerger(Schema schema, TableConfig tableConfig) new TestingSchemaGetter(new SchemaInfo(schema, SCHEMA_ID)); return new AggregateRowMerger(tableConfig, tableConfig.getKvFormat(), schemaGetter); } + + // ========== Retract Tests ========== + + @Test + void testBasicRetract() { + Schema schema = + Schema.newBuilder() + .column("id", DataTypes.INT()) + .column("sum_count", DataTypes.BIGINT(), AggFunctions.SUM()) + .column("sum_total", DataTypes.DOUBLE(), AggFunctions.SUM()) + .primaryKey("id") + .build(); + + TableConfig tableConfig = new TableConfig(new Configuration()); + AggregateRowMerger merger = createMerger(schema, tableConfig); + merger.configureTargetColumns(null, SCHEMA_ID, schema); + RowType rowType = schema.getRowType(); + + // Existing accumulated row: id=1, sum_count=100, sum_total=50.5 + BinaryRow oldRow = compactedRow(rowType, new Object[] {1, 100L, 50.5}); + // Retract row: id=1, sum_count=30, sum_total=10.5 + BinaryRow retractRow = compactedRow(rowType, new Object[] {1, 30L, 10.5}); + + BinaryValue result = merger.retract(toBinaryValue(oldRow), toBinaryValue(retractRow)); + + assertThat(result.schemaId).isEqualTo(SCHEMA_ID); + assertThat(result.row.getInt(0)).isEqualTo(1); // id unchanged + assertThat(result.row.getLong(1)).isEqualTo(70L); // 100 - 30 + assertThat(result.row.getDouble(2)).isEqualTo(40.0); // 50.5 - 10.5 + } + + @Test + void testRetractFirstWrite() { + TableConfig tableConfig = new TableConfig(new Configuration()); + AggregateRowMerger merger = createMerger(SCHEMA_SUM, tableConfig); + merger.configureTargetColumns(null, SCHEMA_ID, SCHEMA_SUM); + + // Retract with no existing row should return null (nothing to retract from) + BinaryRow retractRow = compactedRow(ROW_TYPE_SUM, new Object[] {1, 30L, 10.5}); + BinaryValue result = merger.retract(null, toBinaryValue(retractRow)); + assertThat(result).isNull(); + } + + @Test + void testRetractToZero() { + TableConfig tableConfig = new TableConfig(new Configuration()); + AggregateRowMerger merger = createMerger(SCHEMA_SUM, tableConfig); + merger.configureTargetColumns(null, SCHEMA_ID, SCHEMA_SUM); + RowType rowType = SCHEMA_SUM.getRowType(); + + // Existing accumulated row: id=1, sum_count=30, sum_total=10.5 + BinaryRow oldRow = compactedRow(rowType, new Object[] {1, 30L, 10.5}); + // Retract the same values — result should be zero, not null + BinaryRow retractRow = compactedRow(rowType, new Object[] {1, 30L, 10.5}); + + BinaryValue result = merger.retract(toBinaryValue(oldRow), toBinaryValue(retractRow)); + + assertThat(result).isNotNull(); + assertThat(result.row.getInt(0)).isEqualTo(1); // id unchanged + assertThat(result.row.getLong(1)).isEqualTo(0L); // 30 - 30 = 0 + assertThat(result.row.getDouble(2)).isEqualTo(0.0); // 10.5 - 10.5 = 0.0 + } + + @Test + void testRetractBeyondAccumulatorProducesNegative() { + TableConfig tableConfig = new TableConfig(new Configuration()); + AggregateRowMerger merger = createMerger(SCHEMA_SUM, tableConfig); + merger.configureTargetColumns(null, SCHEMA_ID, SCHEMA_SUM); + RowType rowType = SCHEMA_SUM.getRowType(); + + // Existing accumulated row: id=1, count=5, total=2.0 + BinaryRow oldRow = compactedRow(rowType, new Object[] {1, 5L, 2.0}); + // Retract more than accumulated — should produce negative values + BinaryRow retractRow = compactedRow(rowType, new Object[] {1, 10L, 7.0}); + + BinaryValue result = merger.retract(toBinaryValue(oldRow), toBinaryValue(retractRow)); + + assertThat(result).isNotNull(); + assertThat(result.row.getInt(0)).isEqualTo(1); // id unchanged + assertThat(result.row.getLong(1)).isEqualTo(-5L); // 5 - 10 = -5 + assertThat(result.row.getDouble(2)).isEqualTo(-5.0); // 2.0 - 7.0 = -5.0 + } + + @Test + void testRetractWithNullFields() { + TableConfig tableConfig = new TableConfig(new Configuration()); + AggregateRowMerger merger = createMerger(SCHEMA_SUM, tableConfig); + merger.configureTargetColumns(null, SCHEMA_ID, SCHEMA_SUM); + + // Existing row: id=1, count=100, total=50.5 + BinaryRow oldRow = compactedRow(ROW_TYPE_SUM, new Object[] {1, 100L, 50.5}); + // Retract row with null field: id=1, count=null, total=10.5 + BinaryRow retractRow = compactedRow(ROW_TYPE_SUM, new Object[] {1, null, 10.5}); + + BinaryValue result = merger.retract(toBinaryValue(oldRow), toBinaryValue(retractRow)); + + assertThat(result.row.getLong(1)).isEqualTo(100L); // null retract -> unchanged + assertThat(result.row.getDouble(2)).isEqualTo(40.0); // 50.5 - 10.5 + } + + @Test + void testRetractWithSchemaEvolution() { + // Old schema: id(columnId=0), sum_count(columnId=1) + Schema oldSchema = + Schema.newBuilder() + .column("id", DataTypes.INT()) + .column("sum_count", DataTypes.BIGINT(), AggFunctions.SUM()) + .primaryKey("id") + .build(); + + // New schema: id(columnId=0), sum_count(columnId=1), new_sum(columnId=2) + Schema newSchema = + Schema.newBuilder() + .fromColumns( + java.util.Arrays.asList( + new Schema.Column("id", DataTypes.INT(), null, 0), + new Schema.Column( + "sum_count", + DataTypes.BIGINT(), + null, + 1, + AggFunctions.SUM()), + new Schema.Column( + "new_sum", + DataTypes.BIGINT(), + null, + 2, + AggFunctions.SUM()))) + .primaryKey("id") + .build(); + + short oldSchemaId = (short) 1; + short newSchemaId = (short) 2; + + TestingSchemaGetter schemaGetter = + new TestingSchemaGetter(new SchemaInfo(newSchema, newSchemaId)); + schemaGetter.updateLatestSchemaInfo(new SchemaInfo(oldSchema, oldSchemaId)); + schemaGetter.updateLatestSchemaInfo(new SchemaInfo(newSchema, newSchemaId)); + + TableConfig tableConfig = new TableConfig(new Configuration()); + AggregateRowMerger merger = + new AggregateRowMerger(tableConfig, tableConfig.getKvFormat(), schemaGetter); + merger.configureTargetColumns(null, newSchemaId, newSchema); + + // Old row with old schema (v1): id=1, sum_count=100 + BinaryRow oldRow = compactedRow(oldSchema.getRowType(), new Object[] {1, 100L}); + BinaryValue oldValue = new BinaryValue(oldSchemaId, oldRow); + + // Retract row with new schema (v2): id=1, sum_count=30, new_sum=10 + BinaryRow retractRow = compactedRow(newSchema.getRowType(), new Object[] {1, 30L, 10L}); + BinaryValue retractValue = new BinaryValue(newSchemaId, retractRow); + + BinaryValue result = merger.retract(oldValue, retractValue); + + assertThat(result.schemaId).isEqualTo(newSchemaId); + assertThat(result.row.getFieldCount()).isEqualTo(3); + assertThat(result.row.getInt(0)).isEqualTo(1); // id unchanged + assertThat(result.row.getLong(1)).isEqualTo(70L); // sum_count: 100 - 30 + // new_sum: old row didn't have this column (null), retract 10 from null -> null + assertThat(result.row.isNullAt(2)).isTrue(); + } + + @Test + void testPartialRetract() { + Schema schema = + Schema.newBuilder() + .column("id", DataTypes.INT()) + .column("count", DataTypes.BIGINT(), AggFunctions.SUM()) + .column("total", DataTypes.DOUBLE(), AggFunctions.SUM()) + .primaryKey("id") + .build(); + + TableConfig tableConfig = new TableConfig(new Configuration()); + AggregateRowMerger merger = createMerger(schema, tableConfig); + + // Configure partial update for id and count (excluding total) + RowMerger partialMerger = + merger.configureTargetColumns(new int[] {0, 1}, SCHEMA_ID, schema); + + // Existing row: id=1, count=100, total=50.5 + BinaryRow oldRow = compactedRow(schema.getRowType(), new Object[] {1, 100L, 50.5}); + // Retract row: id=1, count=30, total=999.0 + BinaryRow retractRow = compactedRow(schema.getRowType(), new Object[] {1, 30L, 999.0}); + + BinaryValue result = + partialMerger.retract(toBinaryValue(oldRow), toBinaryValue(retractRow)); + + assertThat(result.row.getInt(0)).isEqualTo(1); // id unchanged + assertThat(result.row.getLong(1)).isEqualTo(70L); // count: 100 - 30 (target, retracted) + assertThat(result.row.getDouble(2)).isEqualTo(50.5); // total: unchanged (not target) + } + + @Test + void testRetractWithCompositePrimaryKey() { + Schema schema = + Schema.newBuilder() + .column("pk1", DataTypes.INT()) + .column("pk2", DataTypes.STRING()) + .column("sum_a", DataTypes.BIGINT(), AggFunctions.SUM()) + .column("sum_b", DataTypes.DOUBLE(), AggFunctions.SUM()) + .primaryKey("pk1", "pk2") + .build(); + + TableConfig tableConfig = new TableConfig(new Configuration()); + AggregateRowMerger merger = createMerger(schema, tableConfig); + merger.configureTargetColumns(null, SCHEMA_ID, schema); + RowType rowType = schema.getRowType(); + + // Existing accumulated row + BinaryRow oldRow = compactedRow(rowType, new Object[] {1, "key", 100L, 50.5}); + // Retract row + BinaryRow retractRow = compactedRow(rowType, new Object[] {1, "key", 30L, 10.5}); + + BinaryValue result = merger.retract(toBinaryValue(oldRow), toBinaryValue(retractRow)); + + assertThat(result.schemaId).isEqualTo(SCHEMA_ID); + assertThat(result.row.getFieldCount()).isEqualTo(4); + // PK columns preserved + assertThat(result.row.getInt(0)).isEqualTo(1); + assertThat(result.row.getString(1).toString()).isEqualTo("key"); + // SUM columns retracted + assertThat(result.row.getLong(2)).isEqualTo(70L); // 100 - 30 + assertThat(result.row.getDouble(3)).isEqualTo(40.0); // 50.5 - 10.5 + } + + @Test + void testRetractFastPathMatchesGeneralPath() { + // Create two schemas with identical column structures but different schema IDs. + // Same-schema path uses the fast path; different-schema IDs force the general path. + Schema schema = + Schema.newBuilder() + .column("id", DataTypes.INT()) + .column("sum_val", DataTypes.BIGINT(), AggFunctions.SUM()) + .primaryKey("id") + .build(); + + short schemaId1 = (short) 1; + short schemaId2 = (short) 2; + + // Fast path: both old and retract use the same schema ID + TableConfig tableConfig = new TableConfig(new Configuration()); + AggregateRowMerger fastMerger = createMerger(schema, tableConfig); + fastMerger.configureTargetColumns(null, schemaId1, schema); + + BinaryRow oldRow = compactedRow(schema.getRowType(), new Object[] {1, 100L}); + BinaryRow retractRow = compactedRow(schema.getRowType(), new Object[] {1, 30L}); + + BinaryValue fastResult = + fastMerger.retract( + new BinaryValue(schemaId1, oldRow), new BinaryValue(schemaId1, retractRow)); + + // General path: old row uses schemaId1, retract row uses schemaId2 + // Both schemas have identical column structures + TestingSchemaGetter schemaGetter = + new TestingSchemaGetter(new SchemaInfo(schema, schemaId2)); + schemaGetter.updateLatestSchemaInfo(new SchemaInfo(schema, schemaId1)); + schemaGetter.updateLatestSchemaInfo(new SchemaInfo(schema, schemaId2)); + + AggregateRowMerger generalMerger = + new AggregateRowMerger(tableConfig, tableConfig.getKvFormat(), schemaGetter); + generalMerger.configureTargetColumns(null, schemaId2, schema); + + BinaryValue generalResult = + generalMerger.retract( + new BinaryValue(schemaId1, oldRow), new BinaryValue(schemaId2, retractRow)); + + // Both paths should produce identical results + assertThat(fastResult.row.getInt(0)).isEqualTo(generalResult.row.getInt(0)); + assertThat(fastResult.row.getLong(1)).isEqualTo(generalResult.row.getLong(1)); + assertThat(fastResult.row.getInt(0)).isEqualTo(1); + assertThat(fastResult.row.getLong(1)).isEqualTo(70L); // 100 - 30 + } + + @Test + void testPartialRetractWithSchemaEvolution() { + // Old schema v1: id(columnId=0), sum_a(columnId=1) + Schema oldSchema = + Schema.newBuilder() + .column("id", DataTypes.INT()) + .column("sum_a", DataTypes.BIGINT(), AggFunctions.SUM()) + .primaryKey("id") + .build(); + + // New schema v2: id(columnId=0), sum_a(columnId=1), sum_b(columnId=2) + Schema newSchema = + Schema.newBuilder() + .fromColumns( + java.util.Arrays.asList( + new Schema.Column("id", DataTypes.INT(), null, 0), + new Schema.Column( + "sum_a", + DataTypes.BIGINT(), + null, + 1, + AggFunctions.SUM()), + new Schema.Column( + "sum_b", + DataTypes.BIGINT(), + null, + 2, + AggFunctions.SUM()))) + .primaryKey("id") + .build(); + + short oldSchemaId = (short) 1; + short newSchemaId = (short) 2; + + TestingSchemaGetter schemaGetter = + new TestingSchemaGetter(new SchemaInfo(newSchema, newSchemaId)); + schemaGetter.updateLatestSchemaInfo(new SchemaInfo(oldSchema, oldSchemaId)); + schemaGetter.updateLatestSchemaInfo(new SchemaInfo(newSchema, newSchemaId)); + + TableConfig tableConfig = new TableConfig(new Configuration()); + AggregateRowMerger merger = + new AggregateRowMerger(tableConfig, tableConfig.getKvFormat(), schemaGetter); + + // Partial retract: only target id(0) and sum_a(1), excluding sum_b(2) + RowMerger partialMerger = + merger.configureTargetColumns(new int[] {0, 1}, newSchemaId, newSchema); + + // Old row with old schema v1: id=1, sum_a=100 + BinaryRow oldRow = compactedRow(oldSchema.getRowType(), new Object[] {1, 100L}); + BinaryValue oldValue = new BinaryValue(oldSchemaId, oldRow); + + // Retract row with new schema v2: id=1, sum_a=30, sum_b=999 + BinaryRow retractRow = compactedRow(newSchema.getRowType(), new Object[] {1, 30L, 999L}); + BinaryValue retractValue = new BinaryValue(newSchemaId, retractRow); + + BinaryValue result = partialMerger.retract(oldValue, retractValue); + + assertThat(result.schemaId).isEqualTo(newSchemaId); + assertThat(result.row.getFieldCount()).isEqualTo(3); + assertThat(result.row.getInt(0)).isEqualTo(1); // id unchanged + assertThat(result.row.getLong(1)).isEqualTo(70L); // sum_a: 100 - 30 (target, retracted) + // sum_b: not a target column, old schema didn't have it -> null + assertThat(result.row.isNullAt(2)).isTrue(); + } + + @Test + void testRetractProducesNullWhenAllNonPkFieldsBecomeNull() { + // Schema where all non-PK columns use last_value (retract-safe). + // Retracting the exact same values should null out all non-PK fields, + // causing the merger to return null (row removal) instead of a zombie row. + Schema schema = + Schema.newBuilder() + .column("id", DataTypes.INT()) + .column("name", DataTypes.STRING(), AggFunctions.LAST_VALUE()) + .column("age", DataTypes.INT(), AggFunctions.LAST_VALUE()) + .primaryKey("id") + .build(); + + TableConfig tableConfig = new TableConfig(new Configuration()); + AggregateRowMerger merger = createMerger(schema, tableConfig); + merger.configureTargetColumns(null, SCHEMA_ID, schema); + RowType rowType = schema.getRowType(); + + // Existing row: id=1, name="Alice", age=30 + BinaryRow oldRow = compactedRow(rowType, new Object[] {1, "Alice", 30}); + // Retract the same values + BinaryRow retractRow = compactedRow(rowType, new Object[] {1, "Alice", 30}); + + BinaryValue result = merger.retract(toBinaryValue(oldRow), toBinaryValue(retractRow)); + + // All non-PK fields are null after retract -> row should be removed + assertThat(result).isNull(); + } + + @Test + void testPartialRetractProducesNullWhenAllNonPkFieldsBecomeNull() { + // Schema with partial update: retract on target columns that nulls them out, + // while non-target columns are already null -> zombie row should be removed. + Schema schema = + Schema.newBuilder() + .column("id", DataTypes.INT()) + .column("val", DataTypes.BIGINT(), AggFunctions.LAST_VALUE()) + .column("other", DataTypes.STRING(), AggFunctions.LAST_VALUE()) + .primaryKey("id") + .build(); + + TableConfig tableConfig = new TableConfig(new Configuration()); + AggregateRowMerger merger = createMerger(schema, tableConfig); + + // Partial update targeting id and val only + RowMerger partialMerger = + merger.configureTargetColumns(new int[] {0, 1}, SCHEMA_ID, schema); + + // Existing row: id=1, val=42, other=null (non-target already null) + BinaryRow oldRow = compactedRow(schema.getRowType(), new Object[] {1, 42L, null}); + // Retract val + BinaryRow retractRow = compactedRow(schema.getRowType(), new Object[] {1, 42L, null}); + + BinaryValue result = + partialMerger.retract(toBinaryValue(oldRow), toBinaryValue(retractRow)); + + // All non-PK fields are null after retract -> row should be removed + assertThat(result).isNull(); + } } diff --git a/fluss-server/src/test/java/org/apache/fluss/server/kv/rowmerger/RowMergerCreateTest.java b/fluss-server/src/test/java/org/apache/fluss/server/kv/rowmerger/RowMergerCreateTest.java index e6d7f48499..6cb0a20efe 100644 --- a/fluss-server/src/test/java/org/apache/fluss/server/kv/rowmerger/RowMergerCreateTest.java +++ b/fluss-server/src/test/java/org/apache/fluss/server/kv/rowmerger/RowMergerCreateTest.java @@ -207,6 +207,66 @@ void testCreateAggregateRowMergerWithCompositePrimaryKeyAndMultipleAggTypes() { assertThat(merger).isInstanceOf(AggregateRowMerger.class); } + @Test + void testDefaultRowMergerRetractThrows() { + Configuration conf = new Configuration(); + TableConfig tableConfig = new TableConfig(conf); + + RowMerger merger = + RowMerger.create(tableConfig, KvFormat.COMPACTED, createSchemaGetter(TEST_SCHEMA)); + + assertThat(merger).isInstanceOf(DefaultRowMerger.class); + + BinaryRow row = compactedRow(TEST_SCHEMA.getRowType(), new Object[] {1, 10L}); + assertThatThrownBy(() -> merger.retract(toBinaryValue(row), toBinaryValue(row))) + .isInstanceOf(UnsupportedOperationException.class) + .hasMessageContaining("does not support retract"); + } + + @Test + void testFirstRowRowMergerRetractThrows() { + Configuration conf = new Configuration(); + conf.setString(ConfigOptions.TABLE_MERGE_ENGINE.key(), MergeEngineType.FIRST_ROW.name()); + TableConfig tableConfig = new TableConfig(conf); + + RowMerger merger = + RowMerger.create(tableConfig, KvFormat.COMPACTED, createSchemaGetter(TEST_SCHEMA)); + + assertThat(merger).isInstanceOf(FirstRowRowMerger.class); + + BinaryRow row = compactedRow(TEST_SCHEMA.getRowType(), new Object[] {1, 10L}); + assertThatThrownBy(() -> merger.retract(toBinaryValue(row), toBinaryValue(row))) + .isInstanceOf(UnsupportedOperationException.class) + .hasMessageContaining("does not support retract"); + } + + @Test + void testVersionedRowMergerRetractThrows() { + Schema schema = + Schema.newBuilder() + .column("id", DataTypes.INT()) + .column("version", DataTypes.BIGINT()) + .column("value", DataTypes.STRING()) + .primaryKey("id") + .build(); + + Configuration conf = new Configuration(); + conf.setString(ConfigOptions.TABLE_MERGE_ENGINE.key(), MergeEngineType.VERSIONED.name()); + conf.setString(ConfigOptions.TABLE_MERGE_ENGINE_VERSION_COLUMN.key(), "version"); + TableConfig tableConfig = new TableConfig(conf); + + RowMerger merger = + RowMerger.create(tableConfig, KvFormat.COMPACTED, createSchemaGetter(schema)); + merger.configureTargetColumns(null, SCHEMA_ID, schema); + + assertThat(merger).isInstanceOf(VersionedRowMerger.class); + + BinaryRow row = compactedRow(schema.getRowType(), new Object[] {1, 1L, "hello"}); + assertThatThrownBy(() -> merger.retract(toBinaryValue(row), toBinaryValue(row))) + .isInstanceOf(UnsupportedOperationException.class) + .hasMessageContaining("does not support retract"); + } + @Test void testCreateAggregateRowMergerCaseInsensitive() { Configuration conf = new Configuration(); diff --git a/fluss-server/src/test/java/org/apache/fluss/server/kv/rowmerger/aggregate/FieldAggregatorParameterizedTest.java b/fluss-server/src/test/java/org/apache/fluss/server/kv/rowmerger/aggregate/FieldAggregatorParameterizedTest.java index 0968dffa08..394e00cee6 100644 --- a/fluss-server/src/test/java/org/apache/fluss/server/kv/rowmerger/aggregate/FieldAggregatorParameterizedTest.java +++ b/fluss-server/src/test/java/org/apache/fluss/server/kv/rowmerger/aggregate/FieldAggregatorParameterizedTest.java @@ -29,10 +29,26 @@ import org.apache.fluss.row.TimestampLtz; import org.apache.fluss.row.TimestampNtz; import org.apache.fluss.server.kv.rowmerger.AggregateRowMerger; +import org.apache.fluss.server.kv.rowmerger.aggregate.functions.FieldAggregator; +import org.apache.fluss.server.kv.rowmerger.aggregate.functions.FieldBoolAndAgg; +import org.apache.fluss.server.kv.rowmerger.aggregate.functions.FieldBoolOrAgg; +import org.apache.fluss.server.kv.rowmerger.aggregate.functions.FieldFirstNonNullValueAgg; +import org.apache.fluss.server.kv.rowmerger.aggregate.functions.FieldFirstValueAgg; +import org.apache.fluss.server.kv.rowmerger.aggregate.functions.FieldLastNonNullValueAgg; +import org.apache.fluss.server.kv.rowmerger.aggregate.functions.FieldLastValueAgg; +import org.apache.fluss.server.kv.rowmerger.aggregate.functions.FieldListaggAgg; +import org.apache.fluss.server.kv.rowmerger.aggregate.functions.FieldMaxAgg; +import org.apache.fluss.server.kv.rowmerger.aggregate.functions.FieldMinAgg; +import org.apache.fluss.server.kv.rowmerger.aggregate.functions.FieldProductAgg; +import org.apache.fluss.server.kv.rowmerger.aggregate.functions.FieldRoaringBitmap32Agg; +import org.apache.fluss.server.kv.rowmerger.aggregate.functions.FieldRoaringBitmap64Agg; +import org.apache.fluss.server.kv.rowmerger.aggregate.functions.FieldSumAgg; import org.apache.fluss.server.utils.RoaringBitmapUtils; +import org.apache.fluss.types.BooleanType; import org.apache.fluss.types.DataType; import org.apache.fluss.types.DataTypeChecks; import org.apache.fluss.types.DataTypes; +import org.apache.fluss.types.StringType; import org.junit.jupiter.api.Test; import org.junit.jupiter.params.ParameterizedTest; @@ -52,6 +68,7 @@ import static org.apache.fluss.testutils.DataTestUtils.compactedRow; import static org.assertj.core.api.Assertions.assertThat; +import static org.assertj.core.api.Assertions.assertThatThrownBy; /** Parameterized tests for all aggregation functions with different data types. */ class FieldAggregatorParameterizedTest { @@ -786,6 +803,171 @@ void testRbm64Aggregation() throws IOException { assertThat(merged.row.getBinary(1, expectedBytes.length)).isEqualTo(expectedBytes); } + // =================================================================================== + // Retract (Sum Subtraction) Tests + // =================================================================================== + + @ParameterizedTest(name = "sum retract with {0}") + @MethodSource("sumRetractTestData") + void testSumRetract( + String typeName, + DataType dataType, + Object accumulator, + Object retractVal, + Object expected) { + FieldSumAgg sumAgg = new FieldSumAgg(dataType); + Object result = sumAgg.retract(accumulator, retractVal); + assertRetractResult(result, expected); + } + + static Stream sumRetractTestData() { + return Stream.of( + Arguments.of("TINYINT", DataTypes.TINYINT(), (byte) 10, (byte) 3, (byte) 7), + Arguments.of( + "SMALLINT", DataTypes.SMALLINT(), (short) 300, (short) 100, (short) 200), + Arguments.of("INT", DataTypes.INT(), 3000, 1000, 2000), + Arguments.of("BIGINT", DataTypes.BIGINT(), 30000L, 10000L, 20000L), + Arguments.of("FLOAT", DataTypes.FLOAT(), 4.0f, 1.5f, 2.5f), + Arguments.of("DOUBLE", DataTypes.DOUBLE(), 31.0, 10.5, 20.5), + Arguments.of( + "DECIMAL(10,2)", + DataTypes.DECIMAL(10, 2), + Decimal.fromBigDecimal(new BigDecimal("301.25"), 10, 2), + Decimal.fromBigDecimal(new BigDecimal("100.50"), 10, 2), + Decimal.fromBigDecimal(new BigDecimal("200.75"), 10, 2)), + // Negative result cases: retract value exceeds accumulator + Arguments.of("INT_negative", DataTypes.INT(), 5, 10, -5), + Arguments.of("BIGINT_negative", DataTypes.BIGINT(), 5L, 10L, -5L)); + } + + @ParameterizedTest(name = "sum retract overflow with {0}") + @MethodSource("sumRetractOverflowTestData") + void testSumRetractOverflow( + String typeName, DataType dataType, Object accumulator, Object retractVal) { + FieldSumAgg sumAgg = new FieldSumAgg(dataType); + assertThatThrownBy(() -> sumAgg.retract(accumulator, retractVal)) + .isInstanceOf(ArithmeticException.class); + } + + static Stream sumRetractOverflowTestData() { + return Stream.of( + // Underflow cases + Arguments.of("TINYINT_underflow", DataTypes.TINYINT(), Byte.MIN_VALUE, (byte) 1), + Arguments.of( + "SMALLINT_underflow", DataTypes.SMALLINT(), Short.MIN_VALUE, (short) 1), + Arguments.of("INT_underflow", DataTypes.INT(), Integer.MIN_VALUE, 1), + Arguments.of("BIGINT_underflow", DataTypes.BIGINT(), Long.MIN_VALUE, 1L), + // Overflow cases (subtracting a negative value) + Arguments.of("TINYINT_overflow", DataTypes.TINYINT(), Byte.MAX_VALUE, (byte) -1), + Arguments.of( + "SMALLINT_overflow", DataTypes.SMALLINT(), Short.MAX_VALUE, (short) -1), + Arguments.of("INT_overflow", DataTypes.INT(), Integer.MAX_VALUE, -1), + Arguments.of("BIGINT_overflow", DataTypes.BIGINT(), Long.MAX_VALUE, -1L)); + } + + @ParameterizedTest(name = "sum retract with null - {0}") + @MethodSource("sumRetractNullTestData") + void testSumRetractWithNull( + String typeName, + DataType dataType, + Object accumulator, + Object retractVal, + Object expected) { + FieldSumAgg sumAgg = new FieldSumAgg(dataType); + Object result = sumAgg.retract(accumulator, retractVal); + assertThat(result).isEqualTo(expected); + } + + static Stream sumRetractNullTestData() { + return Stream.of( + // null accumulator, non-null retract -> null (cannot subtract from nothing) + Arguments.of("BIGINT_null_acc", DataTypes.BIGINT(), null, 10L, null), + // non-null accumulator, null retract -> accumulator unchanged + Arguments.of("BIGINT_null_retract", DataTypes.BIGINT(), 10L, null, 10L), + // both null -> null + Arguments.of("BIGINT_both_null", DataTypes.BIGINT(), null, null, null)); + } + + // =================================================================================== + // Retract (Last Value) Tests + // =================================================================================== + + @ParameterizedTest(name = "last_value retract - {0}") + @MethodSource("lastValueRetractTestData") + void testLastValueRetract( + String caseName, Object accumulator, Object retractVal, Object expected) { + FieldLastValueAgg agg = new FieldLastValueAgg(DataTypes.INT()); + Object result = agg.retract(accumulator, retractVal); + assertThat(result).isEqualTo(expected); + } + + static Stream lastValueRetractTestData() { + return Stream.of( + // last_value retract always returns null regardless of inputs + Arguments.of("non_null_acc_non_null_retract", 10, 5, null), + Arguments.of("non_null_acc_null_retract", 10, null, null), + Arguments.of("null_acc_non_null_retract", null, 5, null), + Arguments.of("both_null", null, null, null)); + } + + @ParameterizedTest(name = "last_value_ignore_nulls retract - {0}") + @MethodSource("lastNonNullValueRetractTestData") + void testLastNonNullValueRetract( + String caseName, Object accumulator, Object retractVal, Object expected) { + FieldLastNonNullValueAgg agg = new FieldLastNonNullValueAgg(DataTypes.INT()); + Object result = agg.retract(accumulator, retractVal); + assertThat(result).isEqualTo(expected); + } + + static Stream lastNonNullValueRetractTestData() { + return Stream.of( + // non-null retract -> null (retract the value) + Arguments.of("non_null_acc_non_null_retract", 10, 5, null), + // null retract -> accumulator unchanged + Arguments.of("non_null_acc_null_retract", 10, null, 10), + // null accumulator, non-null retract -> null + Arguments.of("null_acc_non_null_retract", null, 5, null), + // both null -> null (accumulator is null, retract is null -> keep accumulator) + Arguments.of("both_null", null, null, null)); + } + + @ParameterizedTest(name = "non-retractable function throws: {0}") + @MethodSource("nonRetractableAggregators") + void testNonRetractableFunctionThrows( + String name, FieldAggregator agg, Object acc, Object val) { + assertThatThrownBy(() -> agg.retract(acc, val)) + .isInstanceOf(UnsupportedOperationException.class) + .hasMessageContaining("does not support retract"); + } + + static Stream nonRetractableAggregators() { + return Stream.of( + Arguments.of("FieldMaxAgg", new FieldMaxAgg(DataTypes.INT()), 10, 5), + Arguments.of("FieldMinAgg", new FieldMinAgg(DataTypes.INT()), 10, 5), + Arguments.of("FieldFirstValueAgg", new FieldFirstValueAgg(DataTypes.INT()), 10, 5), + Arguments.of( + "FieldFirstNonNullValueAgg", + new FieldFirstNonNullValueAgg(DataTypes.INT()), + 10, + 5), + Arguments.of("FieldProductAgg", new FieldProductAgg(DataTypes.INT()), 10, 5), + Arguments.of( + "FieldListaggAgg", new FieldListaggAgg(new StringType(), ","), "a,b", "a"), + Arguments.of( + "FieldBoolAndAgg", new FieldBoolAndAgg(new BooleanType()), true, false), + Arguments.of("FieldBoolOrAgg", new FieldBoolOrAgg(new BooleanType()), true, false), + Arguments.of( + "FieldRoaringBitmap32Agg", + new FieldRoaringBitmap32Agg(DataTypes.BYTES()), + new byte[] {1, 2}, + new byte[] {3, 4}), + Arguments.of( + "FieldRoaringBitmap64Agg", + new FieldRoaringBitmap64Agg(DataTypes.BYTES()), + new byte[] {1, 2}, + new byte[] {3, 4})); + } + // =================================================================================== // Helper Methods // =================================================================================== @@ -799,6 +981,15 @@ private AggregateRowMerger createMerger(Schema schema, TableConfig tableConfig) return merger; } + private static void assertRetractResult(Object result, Object expected) { + if (expected instanceof Decimal) { + assertThat(((Decimal) result).toBigDecimal()) + .isEqualByComparingTo(((Decimal) expected).toBigDecimal()); + } else { + assertThat(result).isEqualTo(expected); + } + } + private void assertAggregatedValue(BinaryRow row, int pos, DataType dataType, Object expected) { if (expected == null) { assertThat(row.isNullAt(pos)).isTrue(); @@ -887,4 +1078,78 @@ private static TimestampLtz timestampLtz(String timestamp) { Instant instant = LocalDateTime.parse(timestamp).toInstant(ZoneOffset.UTC); return TimestampLtz.fromInstant(instant); } + + // =================================================================================== + // Retract Overflow / Boundary Tests (M10) + // =================================================================================== + + @Test + void testSumRetractFloatInfinity() { + Schema schema = + Schema.newBuilder() + .column("id", DataTypes.INT()) + .column("val", DataTypes.FLOAT(), AggFunctions.SUM()) + .primaryKey("id") + .build(); + + AggregateRowMerger merger = createMerger(schema, new TableConfig(new Configuration())); + + // Retract Float.MAX_VALUE from -Float.MAX_VALUE → should produce -Infinity + BinaryRow oldRow = compactedRow(schema.getRowType(), new Object[] {1, -Float.MAX_VALUE}); + BinaryRow retractRow = compactedRow(schema.getRowType(), new Object[] {1, Float.MAX_VALUE}); + + BinaryValue result = merger.retract(toBinaryValue(oldRow), toBinaryValue(retractRow)); + assertThat(result).isNotNull(); + assertThat(Float.isInfinite(result.row.getFloat(1))).isTrue(); + } + + @Test + void testSumRetractDoubleInfinity() { + Schema schema = + Schema.newBuilder() + .column("id", DataTypes.INT()) + .column("val", DataTypes.DOUBLE(), AggFunctions.SUM()) + .primaryKey("id") + .build(); + + AggregateRowMerger merger = createMerger(schema, new TableConfig(new Configuration())); + + // Retract Double.MAX_VALUE from -Double.MAX_VALUE → should produce -Infinity + BinaryRow oldRow = compactedRow(schema.getRowType(), new Object[] {1, -Double.MAX_VALUE}); + BinaryRow retractRow = + compactedRow(schema.getRowType(), new Object[] {1, Double.MAX_VALUE}); + + BinaryValue result = merger.retract(toBinaryValue(oldRow), toBinaryValue(retractRow)); + assertThat(result).isNotNull(); + assertThat(Double.isInfinite(result.row.getDouble(1))).isTrue(); + } + + @Test + void testSumRetractDecimalOverflow() { + // DECIMAL(10,2): max is 99999999.99 + DataType decimalType = DataTypes.DECIMAL(10, 2); + Schema schema = + Schema.newBuilder() + .column("id", DataTypes.INT()) + .column("val", decimalType, AggFunctions.SUM()) + .primaryKey("id") + .build(); + + AggregateRowMerger merger = createMerger(schema, new TableConfig(new Configuration())); + + // old = 99999999.99, retract = -99999999.99 → result would be 199999999.98 which overflows + // DECIMAL(10,2). The behavior depends on the Decimal implementation — verify it doesn't + // throw an unhandled exception. + Decimal maxDecimal = Decimal.fromBigDecimal(new BigDecimal("99999999.99"), 10, 2); + Decimal negMaxDecimal = Decimal.fromBigDecimal(new BigDecimal("-99999999.99"), 10, 2); + + BinaryRow oldRow = compactedRow(schema.getRowType(), new Object[] {1, maxDecimal}); + BinaryRow retractRow = compactedRow(schema.getRowType(), new Object[] {1, negMaxDecimal}); + + // This should either produce a result or handle overflow gracefully + BinaryValue result = merger.retract(toBinaryValue(oldRow), toBinaryValue(retractRow)); + assertThat(result).isNotNull(); + // The result decimal may be null (overflow) or a valid value + // Just verify no unhandled exception is thrown + } } diff --git a/fluss-server/src/test/java/org/apache/fluss/server/replica/ReplicaManagerTest.java b/fluss-server/src/test/java/org/apache/fluss/server/replica/ReplicaManagerTest.java index 0d9f5bc565..97a64ac8ff 100644 --- a/fluss-server/src/test/java/org/apache/fluss/server/replica/ReplicaManagerTest.java +++ b/fluss-server/src/test/java/org/apache/fluss/server/replica/ReplicaManagerTest.java @@ -777,9 +777,6 @@ void testLookup() throws Exception { MergeMode.DEFAULT, PUT_KV_VERSION, future1::complete); - assertThat(future1.get()).containsOnly(new PutKvResultForBucket(tb, 8)); - - // second lookup key in table, key = 1, value = 1, "a1". Object[] value1 = DATA_1_WITH_KEY_AND_VALUE.get(3).f1; byte[] value1Bytes = ValueEncoder.encodeValue(DEFAULT_SCHEMA_ID, compactedRow(DATA1_ROW_TYPE, value1)); diff --git a/fluss-server/src/test/java/org/apache/fluss/server/replica/ReplicaTest.java b/fluss-server/src/test/java/org/apache/fluss/server/replica/ReplicaTest.java index 04331fb6ab..cf6971db4c 100644 --- a/fluss-server/src/test/java/org/apache/fluss/server/replica/ReplicaTest.java +++ b/fluss-server/src/test/java/org/apache/fluss/server/replica/ReplicaTest.java @@ -871,7 +871,8 @@ private LogAppendInfo putRecordsToLeader( Replica replica, KvRecordBatch kvRecords, int[] targetColumns) throws Exception { // Use DEFAULT mode as default for tests LogAppendInfo logAppendInfo = - replica.putRecordsToLeader(kvRecords, targetColumns, MergeMode.DEFAULT, 0); + replica.putRecordsToLeader( + kvRecords, targetColumns, MergeMode.DEFAULT, 0, (short) 0); KvTablet kvTablet = checkNotNull(replica.getKvTablet()); // flush to make data visible kvTablet.flush(replica.getLocalLogEndOffset(), NOPErrorHandler.INSTANCE); diff --git a/website/docs/table-design/merge-engines/aggregation.md b/website/docs/table-design/merge-engines/aggregation.md index 4375404af0..25d54180f1 100644 --- a/website/docs/table-design/merge-engines/aggregation.md +++ b/website/docs/table-design/merge-engines/aggregation.md @@ -513,6 +513,25 @@ TableDescriptor.builder() **Key behavior:** Null values overwrite existing values, treating null as a valid value to be stored. +:::warning Retract Behavior Warning +When using `last_value` with retract semantics (e.g., in Flink streaming aggregations): + +- **No history tracking**: `last_value` does not maintain historical values. When a retract (`UPDATE_BEFORE`) is received, it unconditionally clears the accumulator to `null`, regardless of whether the retracted value matches the current state. +- **Best-effort semantic**: This is a simplified implementation aligned with Paimon. Precise retraction is not possible without full history. +- **Use case limitation**: Only suitable for scenarios where each key receives exactly one update. If multiple updates occur before a retract, the result may be incorrect (the field becomes `null` instead of reverting to a previous value). +- **Alternative**: If you need accurate retract behavior, consider using `sum` or other mathematically reversible functions instead. + +**Example of incorrect retract scenario:** +``` +Initial: INSERT (id=1, status='A') → status='A' +Update 1: UPDATE (id=1, status='B') → status='B' +Update 2: UPDATE (id=1, status='C') → status='C' +Retract: RETRACT (id=1, status='B') → status=null (WRONG! Expected 'C') +``` + +In this example, retracting 'B' when the current value is 'C' results in `null` instead of the expected 'C', because `last_value` has no memory of the intermediate state. +::: + ### last_value_ignore_nulls Replaces the previous value with the latest non-null value. This is the **default aggregate function** when no function is specified. @@ -816,11 +835,6 @@ SELECT * FROM test_string_agg WHERE id = 1; ```java -Schema schema = Schema.newBuilder() - .column("id", DataTypes.BIGINT()) - .column("tags", DataTypes.STRING(), AggFunctions.STRING_AGG(";")) // Specify delimiter inline - .primaryKey("id") - .build(); Schema schema = Schema.newBuilder() .column("id", DataTypes.BIGINT()) .column("tags1", DataTypes.STRING(), AggFunctions.STRING_AGG()) @@ -1137,12 +1151,13 @@ partialWriter.delete(primaryKeyRow); ``` :::note -**Current Limitation**: The aggregation merge engine does not support retraction semantics (e.g., subtracting from a sum, reverting a max). - -- **Full update mode**: Delete operations can only remove the entire record -- **Partial update mode**: Delete operations can only null out target columns, not retract aggregated values +**Retract Support**: The aggregation merge engine supports retract semantics for aggregate functions that have a retract implementation. Retract quality varies by function — `sum` provides exact inverse (subtraction), while others are best-effort. When using Flink SQL with a retract-compatible aggregation table, `UPDATE_BEFORE` rows are automatically treated as retract operations rather than deletes. -Future versions may support fine-grained retraction by enhancing the protocol to carry row data with delete operations. +- **Strict inverse**: `sum` — subtraction is mathematically exact. +- **Best-effort**: `last_value` (unconditionally clears to null), `last_value_ignore_nulls` (clears to null when retracting a non-null value). These do not reconstruct prior history. +- Retract is only supported when **all** target aggregation fields use retract-capable functions. +- **Undo vs Retract**: Undo recovery (used during Flink failover) restores exact checkpoint state via OVERWRITE mode. Retract applies inverse aggregation during normal streaming. These are independent mechanisms. +- **Non-retract-capable functions** (e.g., `max`, `min`, `first_value`, `listagg`, `bool_and`, `bool_or`): These functions throw `UnsupportedOperationException` on retract. They cannot be used in retract-enabled tables. ::: ## Limitations