From 98ecaa764f71a5564d9fa272a1deaedbe97e3c49 Mon Sep 17 00:00:00 2001 From: ipolyzos Date: Mon, 17 Nov 2025 16:40:51 +0200 Subject: [PATCH 01/33] Introduce Generics and Typed Classes # Conflicts: # fluss-client/src/main/java/org/apache/fluss/client/lookup/Lookuper.java # fluss-client/src/main/java/org/apache/fluss/client/lookup/PrefixKeyLookuper.java # fluss-client/src/main/java/org/apache/fluss/client/lookup/PrimaryKeyLookuper.java --- .../apache/fluss/client/lookup/Lookup.java | 42 ++++--- .../fluss/client/lookup/LookupResult.java | 4 +- .../apache/fluss/client/lookup/Lookuper.java | 38 +++++-- .../client/lookup/PrefixKeyLookuper.java | 8 +- .../client/lookup/PrimaryKeyLookuper.java | 8 +- .../fluss/client/lookup/TableLookup.java | 7 +- .../fluss/client/lookup/TypedLookuper.java | 70 ++++++++++++ .../fluss/client/table/scanner/Scan.java | 12 +- .../client/table/scanner/ScanRecord.java | 39 ++++--- .../fluss/client/table/scanner/TableScan.java | 10 +- .../table/scanner/log/CompletedFetch.java | 8 +- .../table/scanner/log/LogFetchCollector.java | 17 +-- .../client/table/scanner/log/LogFetcher.java | 3 +- .../client/table/scanner/log/LogScanner.java | 4 +- .../table/scanner/log/LogScannerImpl.java | 21 ++-- .../client/table/scanner/log/ScanRecords.java | 36 +++--- .../table/scanner/log/TypedLogScanner.java | 99 ++++++++++++++++ .../fluss/client/table/writer/Append.java | 9 +- .../client/table/writer/AppendWriter.java | 13 ++- .../client/table/writer/AppendWriterImpl.java | 2 +- .../client/table/writer/TableAppend.java | 10 +- .../client/table/writer/TableUpsert.java | 10 +- .../table/writer/TypedAppendWriter.java | 58 ++++++++++ .../table/writer/TypedUpsertWriter.java | 106 ++++++++++++++++++ .../fluss/client/table/writer/Upsert.java | 27 ++--- .../client/table/writer/UpsertWriter.java | 23 ++-- .../client/table/writer/UpsertWriterImpl.java | 2 +- 27 files changed, 554 insertions(+), 132 deletions(-) create mode 100644 fluss-client/src/main/java/org/apache/fluss/client/lookup/TypedLookuper.java create mode 100644 fluss-client/src/main/java/org/apache/fluss/client/table/scanner/log/TypedLogScanner.java create mode 100644 fluss-client/src/main/java/org/apache/fluss/client/table/writer/TypedAppendWriter.java create mode 100644 fluss-client/src/main/java/org/apache/fluss/client/table/writer/TypedUpsertWriter.java diff --git a/fluss-client/src/main/java/org/apache/fluss/client/lookup/Lookup.java b/fluss-client/src/main/java/org/apache/fluss/client/lookup/Lookup.java index 31faed2f77..61d976b4b2 100644 --- a/fluss-client/src/main/java/org/apache/fluss/client/lookup/Lookup.java +++ b/fluss-client/src/main/java/org/apache/fluss/client/lookup/Lookup.java @@ -24,31 +24,37 @@ /** * Used to configure and create a {@link Lookuper} to lookup rows of a primary key table. The built - * Lookuper can be a primary key lookuper that lookups by the primary key, or a prefix key lookup - * that lookups by the prefix key of the primary key. + * lookuper can lookup by the full primary key, or by a prefix of the primary key when configured + * via {@link #lookupBy}. * *

{@link Lookup} objects are immutable and can be shared between threads. Refinement methods, - * like {@link #lookupBy}, create new Lookup instances. + * like {@link #lookupBy}, create new {@code Lookup} instances. * - *

Example1: Create a Primary Key Lookuper. Given a table with primary key column [k STRING]. + *

Examples + * + *

Example 1: Primary Key Lookuper using an InternalRow key. Given a table with primary key + * column [k STRING]: * *

{@code
- * Lookuper lookuper = table.newLookup().createLookuper();
+ * Lookuper lookuper = table.newLookup().createLookuper();
  * CompletableFuture resultFuture = lookuper.lookup(GenericRow.of("key1"));
- * resultFuture.get().getRows().forEach(row -> {
- *    System.out.println(row);
- * });
+ * resultFuture.get().getRowList().forEach(System.out::println);
  * }
* - *

Example2: Create a Prefix Key Lookuper. Given a table with primary key column [a INT, b - * STRING, c BIGINT] and bucket key [a, b]. + *

Example 2: Prefix Key Lookuper using an InternalRow key. Given a table with primary key + * columns [a INT, b STRING, c BIGINT] and bucket key [a, b]: * *

{@code
- * Lookuper lookuper = table.newLookup().lookupBy("a", "b").createLookuper();
+ * Lookuper lookuper = table.newLookup().lookupBy("a", "b").createLookuper();
  * CompletableFuture resultFuture = lookuper.lookup(GenericRow.of(1, "b1"));
- * resultFuture.get().getRows().forEach(row -> {
- *   System.out.println(row);
- * });
+ * resultFuture.get().getRowList().forEach(System.out::println);
+ * }
+ * + *

Example 3: Using a POJO key (conversion handled internally): + * + *

{@code
+ * Lookuper lookuper = table.newLookup().createLookuper();
+ * LookupResult result = lookuper.lookup(new MyKeyPojo(...)).get();
  * }
* * @since 0.6 @@ -93,7 +99,11 @@ default Lookup lookupBy(String... lookupColumnNames) { * lookup columns. By default, the lookup columns are the primary key columns, but can be * changed with ({@link #lookupBy(List)}) method. * + *

The returned lookuper accepts generic keys of type {@code K}. If a key is a POJO, + * the client implementation will convert it to an {@code InternalRow} based on the table + * schema and the active lookup columns. + * * @return the lookuper */ - Lookuper createLookuper(); -} + Lookuper createLookuper(); +} \ No newline at end of file diff --git a/fluss-client/src/main/java/org/apache/fluss/client/lookup/LookupResult.java b/fluss-client/src/main/java/org/apache/fluss/client/lookup/LookupResult.java index b4536640ed..83497d37cf 100644 --- a/fluss-client/src/main/java/org/apache/fluss/client/lookup/LookupResult.java +++ b/fluss-client/src/main/java/org/apache/fluss/client/lookup/LookupResult.java @@ -27,7 +27,9 @@ import java.util.Objects; /** - * The result of {@link Lookuper#lookup(InternalRow)}. + * The result of a lookup operation performed by a {@link Lookuper}. It carries zero, one, or many + * {@link org.apache.fluss.row.InternalRow} values depending on whether the underlying lookup is a + * primary-key lookup (at most one) or a prefix-key lookup (zero or more). * * @since 0.1 */ diff --git a/fluss-client/src/main/java/org/apache/fluss/client/lookup/Lookuper.java b/fluss-client/src/main/java/org/apache/fluss/client/lookup/Lookuper.java index 37157d9b12..651f9b5590 100644 --- a/fluss-client/src/main/java/org/apache/fluss/client/lookup/Lookuper.java +++ b/fluss-client/src/main/java/org/apache/fluss/client/lookup/Lookuper.java @@ -21,21 +21,37 @@ import org.apache.fluss.row.InternalRow; import javax.annotation.concurrent.NotThreadSafe; - import java.util.concurrent.CompletableFuture; /** - * The lookup-er is used to lookup row of a primary key table by primary key or prefix key. The - * lookuper has retriable ability to handle transient errors during lookup operations which is - * configured by {@link org.apache.fluss.config.ConfigOptions#CLIENT_LOOKUP_MAX_RETRIES}. + * A lookuper performs key-based lookups against a primary key table, using either the full primary + * key or a prefix of the primary key (when configured via {@code Lookup#lookupBy}). + * + *

This interface is generic on the key type {@code K}: + * - When used in row mode, implementations are typically declared as {@code Lookuper} + * and accept an {@link org.apache.fluss.row.InternalRow} containing the key fields in the + * configured order. + * - When used with POJO keys, client-provided lookuper instances can also accept a POJO key type + * (for example {@code Lookuper}) and will transparently convert the POJO to an + * {@code InternalRow} using the table schema and active lookup columns. + * + *

Usage examples: * - *

Note: Lookuper instances are not thread-safe. + *

{@code
+ * // Row-based key (InternalRow)
+ * Lookuper lookuper = table.newLookup().createLookuper();
+ * LookupResult res = lookuper.lookup(keyRow).get();
+ *
+ * // POJO key (converted internally)
+ * Lookuper lookuperPojo = table.newLookup().createLookuper();
+ * LookupResult res2 = lookuperPojo.lookup(new MyKeyPojo(...)).get();
+ * }
* * @since 0.6 */ @PublicEvolving @NotThreadSafe -public interface Lookuper { +public interface Lookuper { /** * Lookups certain row from the given lookup key. @@ -44,8 +60,12 @@ public interface Lookuper { * {@code table.newLookup().createLookuper()}), or be the prefix key if the lookuper is a Prefix * Key Lookuper (created by {@code table.newLookup().lookupBy(prefixKeys).createLookuper()}). * - * @param lookupKey the lookup key. + *

The key can be either an {@link org.apache.fluss.row.InternalRow} or a POJO representing + * the lookup key. Client-provided implementations returned by the Fluss client handle POJO-to- + * row conversion internally when necessary. + * + * @param lookupKey the lookup key * @return the result of lookup. */ - CompletableFuture lookup(InternalRow lookupKey); -} + CompletableFuture lookup(K lookupKey); +} \ No newline at end of file diff --git a/fluss-client/src/main/java/org/apache/fluss/client/lookup/PrefixKeyLookuper.java b/fluss-client/src/main/java/org/apache/fluss/client/lookup/PrefixKeyLookuper.java index a2d1029d25..c5ba215209 100644 --- a/fluss-client/src/main/java/org/apache/fluss/client/lookup/PrefixKeyLookuper.java +++ b/fluss-client/src/main/java/org/apache/fluss/client/lookup/PrefixKeyLookuper.java @@ -46,7 +46,13 @@ * of the primary key. */ @NotThreadSafe -class PrefixKeyLookuper extends AbstractLookuper { +class PrefixKeyLookuper implements Lookuper { + + private final TableInfo tableInfo; + + private final MetadataUpdater metadataUpdater; + + private final LookupClient lookupClient; /** Extract bucket key from prefix lookup key row. */ private final KeyEncoder bucketKeyEncoder; diff --git a/fluss-client/src/main/java/org/apache/fluss/client/lookup/PrimaryKeyLookuper.java b/fluss-client/src/main/java/org/apache/fluss/client/lookup/PrimaryKeyLookuper.java index 6c32647636..f7c79879b3 100644 --- a/fluss-client/src/main/java/org/apache/fluss/client/lookup/PrimaryKeyLookuper.java +++ b/fluss-client/src/main/java/org/apache/fluss/client/lookup/PrimaryKeyLookuper.java @@ -40,7 +40,13 @@ /** An implementation of {@link Lookuper} that lookups by primary key. */ @NotThreadSafe -class PrimaryKeyLookuper extends AbstractLookuper { +class PrimaryKeyLookuper implements Lookuper { + + private final TableInfo tableInfo; + + private final MetadataUpdater metadataUpdater; + + private final LookupClient lookupClient; private final KeyEncoder primaryKeyEncoder; diff --git a/fluss-client/src/main/java/org/apache/fluss/client/lookup/TableLookup.java b/fluss-client/src/main/java/org/apache/fluss/client/lookup/TableLookup.java index 84c9bdb211..a4eb3cb28d 100644 --- a/fluss-client/src/main/java/org/apache/fluss/client/lookup/TableLookup.java +++ b/fluss-client/src/main/java/org/apache/fluss/client/lookup/TableLookup.java @@ -20,6 +20,7 @@ import org.apache.fluss.client.metadata.MetadataUpdater; import org.apache.fluss.metadata.SchemaGetter; import org.apache.fluss.metadata.TableInfo; +import org.apache.fluss.row.InternalRow; import javax.annotation.Nullable; @@ -63,12 +64,14 @@ public Lookup lookupBy(List lookupColumnNames) { } @Override - public Lookuper createLookuper() { + public Lookuper createLookuper() { + Lookuper lookuper; if (lookupColumnNames == null) { return new PrimaryKeyLookuper(tableInfo, schemaGetter, metadataUpdater, lookupClient); } else { return new PrefixKeyLookuper( tableInfo, schemaGetter, metadataUpdater, lookupClient, lookupColumnNames); } + return new TypedLookuper(lookuper, tableInfo, lookupColumnNames); } -} +} \ No newline at end of file diff --git a/fluss-client/src/main/java/org/apache/fluss/client/lookup/TypedLookuper.java b/fluss-client/src/main/java/org/apache/fluss/client/lookup/TypedLookuper.java new file mode 100644 index 0000000000..b4fb3518e1 --- /dev/null +++ b/fluss-client/src/main/java/org/apache/fluss/client/lookup/TypedLookuper.java @@ -0,0 +1,70 @@ +package org.apache.fluss.client.lookup; + +/* + * 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. + */ + +import org.apache.fluss.client.converter.PojoToRowConverter; +import org.apache.fluss.metadata.TableInfo; +import org.apache.fluss.row.InternalRow; +import org.apache.fluss.types.RowType; + +import javax.annotation.Nullable; +import java.util.List; +import java.util.concurrent.CompletableFuture; + +/** + * Decorator for {@link Lookuper} that enables generic key lookup via {@link Lookuper#lookup(Object)}. + * Converts POJO keys to {@link InternalRow} using existing converters based on table schema and + * active lookup columns, and directly delegates when the key is already an {@link InternalRow}. + */ +final class TypedLookuper implements Lookuper { + + private final Lookuper delegate; + private final TableInfo tableInfo; + @Nullable private final List lookupColumnNames; + + TypedLookuper(Lookuper delegate, + TableInfo tableInfo, + @Nullable List lookupColumnNames) { + this.delegate = delegate; + this.tableInfo = tableInfo; + this.lookupColumnNames = lookupColumnNames; + } + + @Override + public CompletableFuture lookup(K key) { + if (key == null) { + throw new IllegalArgumentException("key must not be null"); + } + // Fast-path: already an InternalRow + if (key instanceof InternalRow) { + return delegate.lookup((InternalRow) key); + } + RowType tableSchema = tableInfo.getRowType(); + RowType keyProjection; + if (lookupColumnNames == null) { + keyProjection = tableSchema.project(tableInfo.getPrimaryKeys()); + } else { + keyProjection = tableSchema.project(lookupColumnNames); + } + @SuppressWarnings("unchecked") + Class keyClass = (Class) key.getClass(); + PojoToRowConverter keyConv = PojoToRowConverter.of(keyClass, tableSchema, keyProjection); + InternalRow keyRow = keyConv.toRow(key); + return delegate.lookup(keyRow); + } +} diff --git a/fluss-client/src/main/java/org/apache/fluss/client/table/scanner/Scan.java b/fluss-client/src/main/java/org/apache/fluss/client/table/scanner/Scan.java index 97e37847f6..e15e3894b7 100644 --- a/fluss-client/src/main/java/org/apache/fluss/client/table/scanner/Scan.java +++ b/fluss-client/src/main/java/org/apache/fluss/client/table/scanner/Scan.java @@ -21,6 +21,7 @@ import org.apache.fluss.client.table.scanner.batch.BatchScanner; import org.apache.fluss.client.table.scanner.log.LogScanner; import org.apache.fluss.metadata.TableBucket; +import org.apache.fluss.row.InternalRow; import javax.annotation.Nullable; @@ -63,7 +64,14 @@ public interface Scan { * *

Note: this API doesn't support pre-configured with {@link #limit(int)}. */ - LogScanner createLogScanner(); + LogScanner createLogScanner(); + + /** + * Creates a typed LogScanner to continuously read log data as POJOs of the given class. + * + *

Note: this API doesn't support pre-configured with {@link #limit(int)}. + */ + LogScanner createLogScanner(Class pojoClass); /** * Creates a {@link BatchScanner} to read current data in the given table bucket for this scan. @@ -80,4 +88,4 @@ public interface Scan { * #limit(int)} and only support for Primary Key Tables. */ BatchScanner createBatchScanner(TableBucket tableBucket, long snapshotId); -} +} \ No newline at end of file diff --git a/fluss-client/src/main/java/org/apache/fluss/client/table/scanner/ScanRecord.java b/fluss-client/src/main/java/org/apache/fluss/client/table/scanner/ScanRecord.java index d303b72329..074875b9c8 100644 --- a/fluss-client/src/main/java/org/apache/fluss/client/table/scanner/ScanRecord.java +++ b/fluss-client/src/main/java/org/apache/fluss/client/table/scanner/ScanRecord.java @@ -19,52 +19,51 @@ import org.apache.fluss.annotation.Internal; import org.apache.fluss.record.ChangeType; -import org.apache.fluss.record.LogRecord; -import org.apache.fluss.row.InternalRow; import java.util.Objects; /** one scan record. */ -// TODO: replace this with GenericRecord in the future @Internal -public class ScanRecord implements LogRecord { +public class ScanRecord { private static final long INVALID = -1L; private final long offset; private final long timestamp; private final ChangeType changeType; - private final InternalRow row; + private final T value; - public ScanRecord(InternalRow row) { - this(INVALID, INVALID, ChangeType.INSERT, row); + public ScanRecord(T value) { + this(INVALID, INVALID, ChangeType.INSERT, value); } - public ScanRecord(long offset, long timestamp, ChangeType changeType, InternalRow row) { + public ScanRecord(long offset, long timestamp, ChangeType changeType, T value) { this.offset = offset; this.timestamp = timestamp; this.changeType = changeType; - this.row = row; + this.value = value; } /** The position of this record in the corresponding fluss table bucket. */ - @Override public long logOffset() { return offset; } - @Override public long timestamp() { return timestamp; } - @Override public ChangeType getChangeType() { return changeType; } - @Override - public InternalRow getRow() { - return row; + /** Returns the carried record as InternalRow for backward compatibility. */ + public org.apache.fluss.row.InternalRow getRow() { + return (org.apache.fluss.row.InternalRow) value; + } + + /** Returns the carried record value. */ + public T getValue() { + return value; } @Override @@ -75,19 +74,19 @@ public boolean equals(Object o) { if (o == null || getClass() != o.getClass()) { return false; } - ScanRecord that = (ScanRecord) o; + ScanRecord that = (ScanRecord) o; return offset == that.offset && changeType == that.changeType - && Objects.equals(row, that.row); + && Objects.equals(value, that.value); } @Override public int hashCode() { - return Objects.hash(offset, changeType, row); + return Objects.hash(offset, changeType, value); } @Override public String toString() { - return changeType.shortString() + row.toString() + "@" + offset; + return changeType.shortString() + value + "@" + offset; } -} +} \ No newline at end of file diff --git a/fluss-client/src/main/java/org/apache/fluss/client/table/scanner/TableScan.java b/fluss-client/src/main/java/org/apache/fluss/client/table/scanner/TableScan.java index 0ca1952cf6..93afcf58f3 100644 --- a/fluss-client/src/main/java/org/apache/fluss/client/table/scanner/TableScan.java +++ b/fluss-client/src/main/java/org/apache/fluss/client/table/scanner/TableScan.java @@ -25,11 +25,13 @@ import org.apache.fluss.client.table.scanner.batch.LimitBatchScanner; import org.apache.fluss.client.table.scanner.log.LogScanner; import org.apache.fluss.client.table.scanner.log.LogScannerImpl; +import org.apache.fluss.client.table.scanner.log.TypedLogScanner; import org.apache.fluss.config.ConfigOptions; import org.apache.fluss.exception.FlussRuntimeException; import org.apache.fluss.metadata.SchemaGetter; import org.apache.fluss.metadata.TableBucket; import org.apache.fluss.metadata.TableInfo; +import org.apache.fluss.row.InternalRow; import org.apache.fluss.types.RowType; import javax.annotation.Nullable; @@ -113,6 +115,12 @@ public LogScanner createLogScanner() { schemaGetter); } + @Override + public LogScanner createLogScanner(Class pojoClass) { + LogScanner base = createLogScanner(); + return new TypedLogScanner<>(base, pojoClass, tableInfo, projectedColumns); + } + @Override public BatchScanner createBatchScanner(TableBucket tableBucket) { if (limit == null) { @@ -163,4 +171,4 @@ public BatchScanner createBatchScanner(TableBucket tableBucket, long snapshotId) tableInfo.getTableConfig().getKvFormat(), conn.getOrCreateRemoteFileDownloader()); } -} +} \ No newline at end of file diff --git a/fluss-client/src/main/java/org/apache/fluss/client/table/scanner/log/CompletedFetch.java b/fluss-client/src/main/java/org/apache/fluss/client/table/scanner/log/CompletedFetch.java index 8f29f3ef34..88780b9ab6 100644 --- a/fluss-client/src/main/java/org/apache/fluss/client/table/scanner/log/CompletedFetch.java +++ b/fluss-client/src/main/java/org/apache/fluss/client/table/scanner/log/CompletedFetch.java @@ -95,7 +95,7 @@ public CompletedFetch( // TODO: optimize this to avoid deep copying the record. // refactor #fetchRecords to return an iterator which lazily deserialize // from underlying record stream and arrow buffer. - ScanRecord toScanRecord(LogRecord record) { + ScanRecord toScanRecord(LogRecord record) { GenericRow newRow = new GenericRow(selectedFieldGetters.length); InternalRow internalRow = record.getRow(); for (int i = 0; i < selectedFieldGetters.length; i++) { @@ -148,7 +148,7 @@ void drain() { * maxRecords} * @return {@link ScanRecord scan records} */ - public List fetchRecords(int maxRecords) { + public List> fetchRecords(int maxRecords) { if (corruptLastRecord) { throw new FetchException( "Received exception when fetching the next record from " @@ -161,7 +161,7 @@ public List fetchRecords(int maxRecords) { return Collections.emptyList(); } - List scanRecords = new ArrayList<>(); + List> scanRecords = new ArrayList<>(); try { for (int i = 0; i < maxRecords; i++) { // Only move to next record if there was no exception in the last fetch. @@ -255,4 +255,4 @@ private void maybeCloseRecordStream() { records = null; } } -} +} \ No newline at end of file diff --git a/fluss-client/src/main/java/org/apache/fluss/client/table/scanner/log/LogFetchCollector.java b/fluss-client/src/main/java/org/apache/fluss/client/table/scanner/log/LogFetchCollector.java index 34a3ec86e6..daf3cf409f 100644 --- a/fluss-client/src/main/java/org/apache/fluss/client/table/scanner/log/LogFetchCollector.java +++ b/fluss-client/src/main/java/org/apache/fluss/client/table/scanner/log/LogFetchCollector.java @@ -20,6 +20,7 @@ import org.apache.fluss.annotation.Internal; import org.apache.fluss.client.metadata.MetadataUpdater; import org.apache.fluss.client.table.scanner.ScanRecord; +import org.apache.fluss.row.InternalRow; import org.apache.fluss.config.ConfigOptions; import org.apache.fluss.config.Configuration; import org.apache.fluss.exception.AuthorizationException; @@ -83,8 +84,8 @@ public LogFetchCollector( * @throws LogOffsetOutOfRangeException If there is OffsetOutOfRange error in fetchResponse and * the defaultResetPolicy is NONE */ - public Map> collectFetch(final LogFetchBuffer logFetchBuffer) { - Map> fetched = new HashMap<>(); + public Map>> collectFetch(final LogFetchBuffer logFetchBuffer) { + Map>> fetched = new HashMap<>(); int recordsRemaining = maxPollRecords; try { @@ -115,10 +116,10 @@ public Map> collectFetch(final LogFetchBuffer logF logFetchBuffer.poll(); } else { - List records = fetchRecords(nextInLineFetch, recordsRemaining); + List> records = fetchRecords(nextInLineFetch, recordsRemaining); if (!records.isEmpty()) { TableBucket tableBucket = nextInLineFetch.tableBucket; - List currentRecords = fetched.get(tableBucket); + List> currentRecords = fetched.get(tableBucket); if (currentRecords == null) { fetched.put(tableBucket, records); } else { @@ -126,7 +127,7 @@ public Map> collectFetch(final LogFetchBuffer logF // a time per bucket, but it might conceivably happen in some rare // cases (such as bucket leader changes). we have to copy to a new list // because the old one may be immutable - List newScanRecords = + List> newScanRecords = new ArrayList<>(records.size() + currentRecords.size()); newScanRecords.addAll(currentRecords); newScanRecords.addAll(records); @@ -146,7 +147,7 @@ public Map> collectFetch(final LogFetchBuffer logF return fetched; } - private List fetchRecords(CompletedFetch nextInLineFetch, int maxRecords) { + private List> fetchRecords(CompletedFetch nextInLineFetch, int maxRecords) { TableBucket tb = nextInLineFetch.tableBucket; Long offset = logScannerStatus.getBucketOffset(tb); if (offset == null) { @@ -157,7 +158,7 @@ private List fetchRecords(CompletedFetch nextInLineFetch, int maxRec nextInLineFetch.nextFetchOffset()); } else { if (nextInLineFetch.nextFetchOffset() == offset) { - List records = nextInLineFetch.fetchRecords(maxRecords); + List> records = nextInLineFetch.fetchRecords(maxRecords); LOG.trace( "Returning {} fetched records at offset {} for assigned bucket {}.", records.size(), @@ -289,4 +290,4 @@ private void handleInitializeErrors( error, fetchOffset, tb, error.exception(errorMessage))); } } -} +} \ No newline at end of file diff --git a/fluss-client/src/main/java/org/apache/fluss/client/table/scanner/log/LogFetcher.java b/fluss-client/src/main/java/org/apache/fluss/client/table/scanner/log/LogFetcher.java index a97bffac04..c74d814468 100644 --- a/fluss-client/src/main/java/org/apache/fluss/client/table/scanner/log/LogFetcher.java +++ b/fluss-client/src/main/java/org/apache/fluss/client/table/scanner/log/LogFetcher.java @@ -42,6 +42,7 @@ import org.apache.fluss.record.MemoryLogRecords; import org.apache.fluss.remote.RemoteLogFetchInfo; import org.apache.fluss.remote.RemoteLogSegment; +import org.apache.fluss.row.InternalRow; import org.apache.fluss.rpc.entity.FetchLogResultForBucket; import org.apache.fluss.rpc.gateway.TabletServerGateway; import org.apache.fluss.rpc.messages.FetchLogRequest; @@ -161,7 +162,7 @@ public boolean hasAvailableFetches() { return !logFetchBuffer.isEmpty(); } - public Map> collectFetch() { + public Map>> collectFetch() { return logFetchCollector.collectFetch(logFetchBuffer); } diff --git a/fluss-client/src/main/java/org/apache/fluss/client/table/scanner/log/LogScanner.java b/fluss-client/src/main/java/org/apache/fluss/client/table/scanner/log/LogScanner.java index e8d9ba6a0b..a085b9c4ee 100644 --- a/fluss-client/src/main/java/org/apache/fluss/client/table/scanner/log/LogScanner.java +++ b/fluss-client/src/main/java/org/apache/fluss/client/table/scanner/log/LogScanner.java @@ -27,7 +27,7 @@ * @since 0.1 */ @PublicEvolving -public interface LogScanner extends AutoCloseable { +public interface LogScanner extends AutoCloseable { /** * The earliest offset to fetch from. Fluss uses "-2" to indicate fetching from log start @@ -48,7 +48,7 @@ public interface LogScanner extends AutoCloseable { * @throws java.lang.IllegalStateException if the scanner is not subscribed to any buckets to * read from. */ - ScanRecords poll(Duration timeout); + ScanRecords poll(Duration timeout); /** * Subscribe to the given table bucket in given offset dynamically. If the table bucket is diff --git a/fluss-client/src/main/java/org/apache/fluss/client/table/scanner/log/LogScannerImpl.java b/fluss-client/src/main/java/org/apache/fluss/client/table/scanner/log/LogScannerImpl.java index b50bc3622a..dbd89925a6 100644 --- a/fluss-client/src/main/java/org/apache/fluss/client/table/scanner/log/LogScannerImpl.java +++ b/fluss-client/src/main/java/org/apache/fluss/client/table/scanner/log/LogScannerImpl.java @@ -28,6 +28,7 @@ import org.apache.fluss.metadata.TableBucket; import org.apache.fluss.metadata.TableInfo; import org.apache.fluss.metadata.TablePath; +import org.apache.fluss.row.InternalRow; import org.apache.fluss.rpc.metrics.ClientMetricGroup; import org.apache.fluss.types.RowType; import org.apache.fluss.utils.Projection; @@ -56,7 +57,7 @@ * @since 0.1 */ @PublicEvolving -public class LogScannerImpl implements LogScanner { +public class LogScannerImpl implements LogScanner { private static final Logger LOG = LoggerFactory.getLogger(LogScannerImpl.class); private static final long NO_CURRENT_THREAD = -1L; @@ -130,7 +131,7 @@ private Projection sanityProjection(@Nullable int[] projectedFields, TableInfo t } @Override - public ScanRecords poll(Duration timeout) { + public ScanRecords poll(Duration timeout) { acquireAndEnsureOpen(); try { if (!logScannerStatus.prepareToPoll()) { @@ -141,17 +142,17 @@ public ScanRecords poll(Duration timeout) { long timeoutNanos = timeout.toNanos(); long startNanos = System.nanoTime(); do { - Map> fetchRecords = pollForFetches(); + Map>> fetchRecords = pollForFetches(); if (fetchRecords.isEmpty()) { try { if (!logFetcher.awaitNotEmpty(startNanos + timeoutNanos)) { // logFetcher waits for the timeout and no data in buffer, // so we return empty - return new ScanRecords(fetchRecords); + return new ScanRecords<>(fetchRecords); } } catch (WakeupException e) { // wakeup() is called, we need to return empty - return new ScanRecords(fetchRecords); + return new ScanRecords<>(fetchRecords); } } else { // before returning the fetched records, we can send off the next round of @@ -159,11 +160,11 @@ public ScanRecords poll(Duration timeout) { // while the user is handling the fetched records. logFetcher.sendFetches(); - return new ScanRecords(fetchRecords); + return new ScanRecords<>(fetchRecords); } } while (System.nanoTime() - startNanos < timeoutNanos); - return ScanRecords.EMPTY; + return ScanRecords.empty(); } finally { release(); scannerMetricGroup.recordPollEnd(System.currentTimeMillis()); @@ -230,8 +231,8 @@ public void wakeup() { logFetcher.wakeup(); } - private Map> pollForFetches() { - Map> fetchedRecords = logFetcher.collectFetch(); + private Map>> pollForFetches() { + Map>> fetchedRecords = logFetcher.collectFetch(); if (!fetchedRecords.isEmpty()) { return fetchedRecords; } @@ -305,4 +306,4 @@ public void close() { release(); } } -} +} \ No newline at end of file diff --git a/fluss-client/src/main/java/org/apache/fluss/client/table/scanner/log/ScanRecords.java b/fluss-client/src/main/java/org/apache/fluss/client/table/scanner/log/ScanRecords.java index 9d58c22b49..ea38f6adaf 100644 --- a/fluss-client/src/main/java/org/apache/fluss/client/table/scanner/log/ScanRecords.java +++ b/fluss-client/src/main/java/org/apache/fluss/client/table/scanner/log/ScanRecords.java @@ -36,12 +36,12 @@ * @since 0.1 */ @PublicEvolving -public class ScanRecords implements Iterable { - public static final ScanRecords EMPTY = new ScanRecords(Collections.emptyMap()); +public class ScanRecords implements Iterable> { + public static final ScanRecords empty() { return new ScanRecords<>(Collections.emptyMap()); } - private final Map> records; + private final Map>> records; - public ScanRecords(Map> records) { + public ScanRecords(Map>> records) { this.records = records; } @@ -50,8 +50,8 @@ public ScanRecords(Map> records) { * * @param scanBucket The bucket to get records for */ - public List records(TableBucket scanBucket) { - List recs = records.get(scanBucket); + public List> records(TableBucket scanBucket) { + List> recs = records.get(scanBucket); if (recs == null) { return Collections.emptyList(); } @@ -71,7 +71,7 @@ public Set buckets() { /** The number of records for all buckets. */ public int count() { int count = 0; - for (List recs : records.values()) { + for (List> recs : records.values()) { count += recs.size(); } return count; @@ -82,25 +82,25 @@ public boolean isEmpty() { } @Override - public Iterator iterator() { - return new ConcatenatedIterable(records.values()).iterator(); + public Iterator> iterator() { + return new ConcatenatedIterable<>(records.values()).iterator(); } - private static class ConcatenatedIterable implements Iterable { + private static class ConcatenatedIterable implements Iterable> { - private final Iterable> iterables; + private final Iterable>> iterables; - public ConcatenatedIterable(Iterable> iterables) { + public ConcatenatedIterable(Iterable>> iterables) { this.iterables = iterables; } @Override - public Iterator iterator() { - return new AbstractIterator() { - final Iterator> iters = iterables.iterator(); - Iterator current; + public Iterator> iterator() { + return new AbstractIterator>() { + final Iterator>> iters = iterables.iterator(); + Iterator> current; - public ScanRecord makeNext() { + public ScanRecord makeNext() { while (current == null || !current.hasNext()) { if (iters.hasNext()) { current = iters.next().iterator(); @@ -113,4 +113,4 @@ public ScanRecord makeNext() { }; } } -} +} \ No newline at end of file diff --git a/fluss-client/src/main/java/org/apache/fluss/client/table/scanner/log/TypedLogScanner.java b/fluss-client/src/main/java/org/apache/fluss/client/table/scanner/log/TypedLogScanner.java new file mode 100644 index 0000000000..a13ded7fe9 --- /dev/null +++ b/fluss-client/src/main/java/org/apache/fluss/client/table/scanner/log/TypedLogScanner.java @@ -0,0 +1,99 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.fluss.client.table.scanner.log; + +import org.apache.fluss.client.converter.RowToPojoConverter; +import org.apache.fluss.client.table.scanner.ScanRecord; +import org.apache.fluss.metadata.TableBucket; +import org.apache.fluss.metadata.TableInfo; +import org.apache.fluss.row.InternalRow; +import org.apache.fluss.types.RowType; + +import java.time.Duration; +import java.util.ArrayList; +import java.util.HashMap; +import java.util.List; +import java.util.Map; + +/** + * Adapter that converts {@link InternalRow} records from a {@link LogScanner} into POJOs of type T. + */ +public class TypedLogScanner implements LogScanner { + + private final LogScanner delegate; + private final RowToPojoConverter converter; + + public TypedLogScanner(LogScanner delegate, + Class pojoClass, + TableInfo tableInfo, + int[] projectedColumns) { + this.delegate = delegate; + RowType tableSchema = tableInfo.getRowType(); + RowType projection = projectedColumns == null ? tableSchema : tableSchema.project(projectedColumns); + this.converter = RowToPojoConverter.of(pojoClass, tableSchema, projection); + } + + @Override + public ScanRecords poll(Duration timeout) { + ScanRecords records = delegate.poll(timeout); + if (records == null || records.isEmpty()) { + return ScanRecords.empty(); + } + Map>> out = new HashMap<>(); + for (TableBucket bucket : records.buckets()) { + List> list = records.records(bucket); + List> converted = new ArrayList<>(list.size()); + for (ScanRecord r : list) { + InternalRow row = r.getValue(); + T pojo = converter.fromRow(row); + converted.add(new ScanRecord<>(r.logOffset(), r.timestamp(), r.getChangeType(), pojo)); + } + out.put(bucket, converted); + } + return new ScanRecords<>(out); + } + + @Override + public void subscribe(int bucket, long offset) { + delegate.subscribe(bucket, offset); + } + + @Override + public void subscribe(long partitionId, int bucket, long offset) { + delegate.subscribe(partitionId, bucket, offset); + } + + @Override + public void unsubscribe(long partitionId, int bucket) { + delegate.unsubscribe(partitionId, bucket); + } + + @Override + public void wakeup() { + delegate.wakeup(); + } + + @Override + public void close() { + try { + delegate.close(); + } catch (Exception e) { + throw new RuntimeException(e); + } + } +} \ No newline at end of file diff --git a/fluss-client/src/main/java/org/apache/fluss/client/table/writer/Append.java b/fluss-client/src/main/java/org/apache/fluss/client/table/writer/Append.java index 6a5b10523d..24dd8a6845 100644 --- a/fluss-client/src/main/java/org/apache/fluss/client/table/writer/Append.java +++ b/fluss-client/src/main/java/org/apache/fluss/client/table/writer/Append.java @@ -32,6 +32,9 @@ public interface Append { // TODO: Add more methods to configure the AppendWriter, such as apply static partitions, // apply overwrites, etc. - /** Create a new {@link AppendWriter} to write data to a Log Table. */ - AppendWriter createWriter(); -} + /** Create a new {@link AppendWriter} to write data to a Log Table using InternalRow. */ + AppendWriter createWriter(); + + /** Create a new typed {@link AppendWriter} to write POJOs directly. */ + AppendWriter createWriter(Class pojoClass); +} \ No newline at end of file diff --git a/fluss-client/src/main/java/org/apache/fluss/client/table/writer/AppendWriter.java b/fluss-client/src/main/java/org/apache/fluss/client/table/writer/AppendWriter.java index 4edcc2471d..ff0f62eaae 100644 --- a/fluss-client/src/main/java/org/apache/fluss/client/table/writer/AppendWriter.java +++ b/fluss-client/src/main/java/org/apache/fluss/client/table/writer/AppendWriter.java @@ -28,13 +28,16 @@ * @since 0.2 */ @PublicEvolving -public interface AppendWriter extends TableWriter { +public interface AppendWriter extends TableWriter { /** - * Append row into a Log Table. + * Append a record into a Log Table. * - * @param row the row to append. + *

If {@code T} is {@link InternalRow}, the row will be written directly. Otherwise, the + * client will convert the POJO into an {@link InternalRow} using the configured converters. + * + * @param record the record to append. * @return A {@link CompletableFuture} that always returns append result when complete normally. */ - CompletableFuture append(InternalRow row); -} + CompletableFuture append(T record); +} \ No newline at end of file diff --git a/fluss-client/src/main/java/org/apache/fluss/client/table/writer/AppendWriterImpl.java b/fluss-client/src/main/java/org/apache/fluss/client/table/writer/AppendWriterImpl.java index d702e9621e..823413b2f0 100644 --- a/fluss-client/src/main/java/org/apache/fluss/client/table/writer/AppendWriterImpl.java +++ b/fluss-client/src/main/java/org/apache/fluss/client/table/writer/AppendWriterImpl.java @@ -37,7 +37,7 @@ import java.util.concurrent.CompletableFuture; /** The writer to write data to the log table. */ -class AppendWriterImpl extends AbstractTableWriter implements AppendWriter { +class AppendWriterImpl extends AbstractTableWriter implements AppendWriter { private static final AppendResult APPEND_SUCCESS = new AppendResult(); private final @Nullable KeyEncoder bucketKeyEncoder; diff --git a/fluss-client/src/main/java/org/apache/fluss/client/table/writer/TableAppend.java b/fluss-client/src/main/java/org/apache/fluss/client/table/writer/TableAppend.java index 50e416d563..5d3340f180 100644 --- a/fluss-client/src/main/java/org/apache/fluss/client/table/writer/TableAppend.java +++ b/fluss-client/src/main/java/org/apache/fluss/client/table/writer/TableAppend.java @@ -35,7 +35,13 @@ public TableAppend(TablePath tablePath, TableInfo tableInfo, WriterClient writer } @Override - public AppendWriter createWriter() { + public AppendWriter createWriter() { return new AppendWriterImpl(tablePath, tableInfo, writerClient); } -} + + @Override + public AppendWriter createWriter(Class pojoClass) { + AppendWriterImpl delegate = new AppendWriterImpl(tablePath, tableInfo, writerClient); + return new TypedAppendWriter<>(delegate, pojoClass, tableInfo); + } +} \ No newline at end of file diff --git a/fluss-client/src/main/java/org/apache/fluss/client/table/writer/TableUpsert.java b/fluss-client/src/main/java/org/apache/fluss/client/table/writer/TableUpsert.java index 7d90a9ccd3..d5e8921e04 100644 --- a/fluss-client/src/main/java/org/apache/fluss/client/table/writer/TableUpsert.java +++ b/fluss-client/src/main/java/org/apache/fluss/client/table/writer/TableUpsert.java @@ -92,7 +92,13 @@ public Upsert partialUpdate(String... targetColumnNames) { } @Override - public UpsertWriter createWriter() { + public UpsertWriter createWriter() { return new UpsertWriterImpl(tablePath, tableInfo, targetColumns, writerClient); } -} + + @Override + public UpsertWriter createWriter(Class pojoClass) { + UpsertWriterImpl delegate = new UpsertWriterImpl(tablePath, tableInfo, targetColumns, writerClient); + return new TypedUpsertWriter<>(delegate, pojoClass, tableInfo, targetColumns); + } +} \ No newline at end of file diff --git a/fluss-client/src/main/java/org/apache/fluss/client/table/writer/TypedAppendWriter.java b/fluss-client/src/main/java/org/apache/fluss/client/table/writer/TypedAppendWriter.java new file mode 100644 index 0000000000..67f4113c28 --- /dev/null +++ b/fluss-client/src/main/java/org/apache/fluss/client/table/writer/TypedAppendWriter.java @@ -0,0 +1,58 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.fluss.client.table.writer; + +import org.apache.fluss.client.converter.PojoToRowConverter; +import org.apache.fluss.metadata.TableInfo; +import org.apache.fluss.row.InternalRow; +import org.apache.fluss.types.RowType; + +import java.util.concurrent.CompletableFuture; + +/** + * A typed {@link AppendWriter} that converts POJOs to {@link InternalRow} and delegates to the + * existing internal-row based writer implementation. + */ +class TypedAppendWriter implements AppendWriter { + + private final AppendWriterImpl delegate; + private final Class pojoClass; + private final RowType tableSchema; + + TypedAppendWriter(AppendWriterImpl delegate, Class pojoClass, TableInfo tableInfo) { + this.delegate = delegate; + this.pojoClass = pojoClass; + this.tableSchema = tableInfo.getRowType(); + } + + @Override + public void flush() { + delegate.flush(); + } + + @Override + public CompletableFuture append(T record) { + if (record instanceof InternalRow) { + return delegate.append((InternalRow) record); + } + // TODO: initialize this on the constructor and reuse + PojoToRowConverter converter = PojoToRowConverter.of(pojoClass, tableSchema, tableSchema); + InternalRow row = converter.toRow(record); + return delegate.append(row); + } +} \ No newline at end of file diff --git a/fluss-client/src/main/java/org/apache/fluss/client/table/writer/TypedUpsertWriter.java b/fluss-client/src/main/java/org/apache/fluss/client/table/writer/TypedUpsertWriter.java new file mode 100644 index 0000000000..fe895b3bb4 --- /dev/null +++ b/fluss-client/src/main/java/org/apache/fluss/client/table/writer/TypedUpsertWriter.java @@ -0,0 +1,106 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.fluss.client.table.writer; + +import org.apache.fluss.client.converter.PojoToRowConverter; +import org.apache.fluss.metadata.TableInfo; +import org.apache.fluss.row.GenericRow; +import org.apache.fluss.row.InternalRow; +import org.apache.fluss.types.RowType; + +import java.util.concurrent.CompletableFuture; + +/** + * A typed {@link UpsertWriter} that converts POJOs to {@link InternalRow} and delegates to the + * existing internal-row based writer implementation. + */ +class TypedUpsertWriter implements UpsertWriter { + + @Override + public void flush() { + delegate.flush(); + } + + private final UpsertWriterImpl delegate; + private final Class pojoClass; + private final TableInfo tableInfo; + private final RowType tableSchema; + private final int[] targetColumns; // may be null + + TypedUpsertWriter(UpsertWriterImpl delegate, Class pojoClass, TableInfo tableInfo, int[] targetColumns) { + this.delegate = delegate; + this.pojoClass = pojoClass; + this.tableInfo = tableInfo; + this.tableSchema = tableInfo.getRowType(); + this.targetColumns = targetColumns; + } + + @Override + public CompletableFuture upsert(T record) { + if (record instanceof InternalRow) { + return delegate.upsert((InternalRow) record); + } + InternalRow row = convertPojo(record, /*forDelete=*/false); + return delegate.upsert(row); + } + + @Override + public CompletableFuture delete(T record) { + if (record instanceof InternalRow) { + return delegate.delete((InternalRow) record); + } + InternalRow pkOnly = convertPojo(record, /*forDelete=*/true); + return delegate.delete(pkOnly); + } + + private InternalRow convertPojo(T pojo, boolean forDelete) { + RowType projection; + if (forDelete) { + // for delete we only need primary key columns + projection = tableSchema.project(tableInfo.getPhysicalPrimaryKeys()); + } else if (targetColumns != null) { + projection = tableSchema.project(targetColumns); + } else { + projection = tableSchema; + } + + // TODO: initialize this on the constructor and reuse + PojoToRowConverter converter = PojoToRowConverter.of(pojoClass, tableSchema, projection); + GenericRow projected = converter.toRow(pojo); + if (projection == tableSchema) { + return projected; + } + // expand projected row to full row if needed + GenericRow full = new GenericRow(tableSchema.getFieldCount()); + if (forDelete) { + // set PK fields, others null + for (String pk : tableInfo.getPhysicalPrimaryKeys()) { + int projIndex = projection.getFieldIndex(pk); + int fullIndex = tableSchema.getFieldIndex(pk); + full.setField(fullIndex, projected.getField(projIndex)); + } + } else if (targetColumns != null) { + for (int i = 0; i < projection.getFieldCount(); i++) { + String name = projection.getFieldNames().get(i); + int fullIdx = tableSchema.getFieldIndex(name); + full.setField(fullIdx, projected.getField(i)); + } + } + return full; + } +} \ No newline at end of file diff --git a/fluss-client/src/main/java/org/apache/fluss/client/table/writer/Upsert.java b/fluss-client/src/main/java/org/apache/fluss/client/table/writer/Upsert.java index d3c3608ee8..89cc8d8a6d 100644 --- a/fluss-client/src/main/java/org/apache/fluss/client/table/writer/Upsert.java +++ b/fluss-client/src/main/java/org/apache/fluss/client/table/writer/Upsert.java @@ -18,7 +18,6 @@ package org.apache.fluss.client.table.writer; import org.apache.fluss.annotation.PublicEvolving; -import org.apache.fluss.row.InternalRow; import javax.annotation.Nullable; @@ -37,16 +36,14 @@ public interface Upsert { /** * Apply partial update columns and returns a new Upsert instance. * - *

For {@link UpsertWriter#upsert(InternalRow)} operation, only the specified columns will be - * updated and other columns will remain unchanged if the row exists or set to null if the row - * doesn't exist. + *

For upsert operations, only the specified columns will be updated and other columns will + * remain unchanged if the row exists or set to null if the row doesn't exist. * - *

For {@link UpsertWriter#delete(InternalRow)} operation, the entire row will not be - * removed, but only the specified columns except primary key will be set to null. The entire - * row will be removed when all columns except primary key are null after a {@link - * UpsertWriter#delete(InternalRow)} operation. + *

For delete operations, the entire row will not be removed immediately, but only the + * specified columns except primary key will be set to null. The entire row will be removed when + * all columns except primary key are null after a delete operation. * - *

Note: The specified columns must be a contains all columns of primary key, and all columns + *

Note: The specified columns must contain all columns of primary key, and all columns * except primary key should be nullable. * * @param targetColumns the column indexes to partial update @@ -60,8 +57,12 @@ public interface Upsert { Upsert partialUpdate(String... targetColumnNames); /** - * Create a new {@link UpsertWriter} with the optional {@link #partialUpdate(String...)} - * information to upsert and delete data to a Primary Key Table. + * Create a new {@link UpsertWriter} using {@code InternalRow} with the optional + * {@link #partialUpdate(String...)} information to upsert and delete data to a Primary Key + * Table. */ - UpsertWriter createWriter(); -} + UpsertWriter createWriter(); + + /** Create a new typed {@link UpsertWriter} to write POJOs directly. */ + UpsertWriter createWriter(Class pojoClass); +} \ No newline at end of file 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 6648eebfc0..6fbd607ba4 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 @@ -28,22 +28,27 @@ * @since 0.2 */ @PublicEvolving -public interface UpsertWriter extends TableWriter { +public interface UpsertWriter extends TableWriter { /** - * Inserts row into Fluss table if they do not already exist, or updates them if they do exist. + * Inserts a record into Fluss table if it does not already exist, or updates it if it does. * - * @param row the row to upsert. + *

If {@code T} is {@link InternalRow}, the row will be written directly. Otherwise, the + * client will convert the POJO into an {@link InternalRow} using the configured converters. + * + * @param record the record to upsert. * @return A {@link CompletableFuture} that always returns upsert result when complete normally. */ - CompletableFuture upsert(InternalRow row); + CompletableFuture upsert(T record); /** - * Delete certain row by the input row in Fluss table, the input row must contain the primary - * key. + * Delete a certain record from the Fluss table. The input must contain the primary key fields. + * + *

If {@code T} is {@link InternalRow}, the row will be used directly. Otherwise, the client + * will extract primary key fields from the POJO using the configured converters. * - * @param row the row to delete. + * @param record the record to delete. * @return A {@link CompletableFuture} that always delete result when complete normally. */ - CompletableFuture delete(InternalRow row); -} + CompletableFuture delete(T record); +} \ No newline at end of file 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 39f65592c1..626a985f3b 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 @@ -39,7 +39,7 @@ import java.util.concurrent.CompletableFuture; /** The writer to write data to the primary key table. */ -class UpsertWriterImpl extends AbstractTableWriter implements UpsertWriter { +class UpsertWriterImpl extends AbstractTableWriter implements UpsertWriter { private static final UpsertResult UPSERT_SUCCESS = new UpsertResult(); private static final DeleteResult DELETE_SUCCESS = new DeleteResult(); From dceb71e9aad527aa024294ca6885ec9a22482bac Mon Sep 17 00:00:00 2001 From: ipolyzos Date: Mon, 17 Nov 2025 16:52:08 +0200 Subject: [PATCH 02/33] fix checkstyle violation --- .../src/main/java/org/apache/fluss/client/lookup/Lookuper.java | 1 - 1 file changed, 1 deletion(-) diff --git a/fluss-client/src/main/java/org/apache/fluss/client/lookup/Lookuper.java b/fluss-client/src/main/java/org/apache/fluss/client/lookup/Lookuper.java index 651f9b5590..485a31cc2d 100644 --- a/fluss-client/src/main/java/org/apache/fluss/client/lookup/Lookuper.java +++ b/fluss-client/src/main/java/org/apache/fluss/client/lookup/Lookuper.java @@ -18,7 +18,6 @@ package org.apache.fluss.client.lookup; import org.apache.fluss.annotation.PublicEvolving; -import org.apache.fluss.row.InternalRow; import javax.annotation.concurrent.NotThreadSafe; import java.util.concurrent.CompletableFuture; From 47e50f81f042105e4ec76c694edfe25242f9917d Mon Sep 17 00:00:00 2001 From: ipolyzos Date: Mon, 17 Nov 2025 16:56:44 +0200 Subject: [PATCH 03/33] fix checkstyle violation --- .../org/apache/fluss/client/lookup/Lookup.java | 8 ++++---- .../apache/fluss/client/lookup/Lookuper.java | 15 +++++++-------- .../apache/fluss/client/lookup/TableLookup.java | 2 +- .../fluss/client/lookup/TypedLookuper.java | 15 +++++++++------ .../apache/fluss/client/table/scanner/Scan.java | 2 +- .../fluss/client/table/scanner/ScanRecord.java | 2 +- .../fluss/client/table/scanner/TableScan.java | 2 +- .../table/scanner/log/CompletedFetch.java | 2 +- .../table/scanner/log/LogFetchCollector.java | 13 ++++++++----- .../table/scanner/log/LogScannerImpl.java | 2 +- .../client/table/scanner/log/ScanRecords.java | 6 ++++-- .../table/scanner/log/TypedLogScanner.java | 17 ++++++++++------- .../fluss/client/table/writer/Append.java | 2 +- .../fluss/client/table/writer/AppendWriter.java | 2 +- .../fluss/client/table/writer/TableAppend.java | 2 +- .../fluss/client/table/writer/TableUpsert.java | 5 +++-- .../client/table/writer/TypedAppendWriter.java | 5 +++-- .../client/table/writer/TypedUpsertWriter.java | 12 ++++++++---- .../fluss/client/table/writer/Upsert.java | 7 +++---- .../fluss/client/table/writer/UpsertWriter.java | 2 +- 20 files changed, 69 insertions(+), 54 deletions(-) diff --git a/fluss-client/src/main/java/org/apache/fluss/client/lookup/Lookup.java b/fluss-client/src/main/java/org/apache/fluss/client/lookup/Lookup.java index 61d976b4b2..5411a0c91d 100644 --- a/fluss-client/src/main/java/org/apache/fluss/client/lookup/Lookup.java +++ b/fluss-client/src/main/java/org/apache/fluss/client/lookup/Lookup.java @@ -99,11 +99,11 @@ default Lookup lookupBy(String... lookupColumnNames) { * lookup columns. By default, the lookup columns are the primary key columns, but can be * changed with ({@link #lookupBy(List)}) method. * - *

The returned lookuper accepts generic keys of type {@code K}. If a key is a POJO, - * the client implementation will convert it to an {@code InternalRow} based on the table - * schema and the active lookup columns. + *

The returned lookuper accepts generic keys of type {@code K}. If a key is a POJO, the + * client implementation will convert it to an {@code InternalRow} based on the table schema and + * the active lookup columns. * * @return the lookuper */ Lookuper createLookuper(); -} \ No newline at end of file +} diff --git a/fluss-client/src/main/java/org/apache/fluss/client/lookup/Lookuper.java b/fluss-client/src/main/java/org/apache/fluss/client/lookup/Lookuper.java index 485a31cc2d..b70adf8a4f 100644 --- a/fluss-client/src/main/java/org/apache/fluss/client/lookup/Lookuper.java +++ b/fluss-client/src/main/java/org/apache/fluss/client/lookup/Lookuper.java @@ -26,13 +26,12 @@ * A lookuper performs key-based lookups against a primary key table, using either the full primary * key or a prefix of the primary key (when configured via {@code Lookup#lookupBy}). * - *

This interface is generic on the key type {@code K}: - * - When used in row mode, implementations are typically declared as {@code Lookuper} - * and accept an {@link org.apache.fluss.row.InternalRow} containing the key fields in the - * configured order. - * - When used with POJO keys, client-provided lookuper instances can also accept a POJO key type - * (for example {@code Lookuper}) and will transparently convert the POJO to an - * {@code InternalRow} using the table schema and active lookup columns. + *

This interface is generic on the key type {@code K}: - When used in row mode, implementations + * are typically declared as {@code Lookuper} and accept an {@link + * org.apache.fluss.row.InternalRow} containing the key fields in the configured order. - When used + * with POJO keys, client-provided lookuper instances can also accept a POJO key type (for example + * {@code Lookuper}) and will transparently convert the POJO to an {@code InternalRow} + * using the table schema and active lookup columns. * *

Usage examples: * @@ -67,4 +66,4 @@ public interface Lookuper { * @return the result of lookup. */ CompletableFuture lookup(K lookupKey); -} \ No newline at end of file +} diff --git a/fluss-client/src/main/java/org/apache/fluss/client/lookup/TableLookup.java b/fluss-client/src/main/java/org/apache/fluss/client/lookup/TableLookup.java index a4eb3cb28d..975196801f 100644 --- a/fluss-client/src/main/java/org/apache/fluss/client/lookup/TableLookup.java +++ b/fluss-client/src/main/java/org/apache/fluss/client/lookup/TableLookup.java @@ -74,4 +74,4 @@ public Lookuper createLookuper() { } return new TypedLookuper(lookuper, tableInfo, lookupColumnNames); } -} \ No newline at end of file +} diff --git a/fluss-client/src/main/java/org/apache/fluss/client/lookup/TypedLookuper.java b/fluss-client/src/main/java/org/apache/fluss/client/lookup/TypedLookuper.java index b4fb3518e1..bba49366b8 100644 --- a/fluss-client/src/main/java/org/apache/fluss/client/lookup/TypedLookuper.java +++ b/fluss-client/src/main/java/org/apache/fluss/client/lookup/TypedLookuper.java @@ -23,13 +23,15 @@ import org.apache.fluss.types.RowType; import javax.annotation.Nullable; + import java.util.List; import java.util.concurrent.CompletableFuture; /** - * Decorator for {@link Lookuper} that enables generic key lookup via {@link Lookuper#lookup(Object)}. - * Converts POJO keys to {@link InternalRow} using existing converters based on table schema and - * active lookup columns, and directly delegates when the key is already an {@link InternalRow}. + * Decorator for {@link Lookuper} that enables generic key lookup via {@link + * Lookuper#lookup(Object)}. Converts POJO keys to {@link InternalRow} using existing converters + * based on table schema and active lookup columns, and directly delegates when the key is already + * an {@link InternalRow}. */ final class TypedLookuper implements Lookuper { @@ -37,9 +39,10 @@ final class TypedLookuper implements Lookuper { private final TableInfo tableInfo; @Nullable private final List lookupColumnNames; - TypedLookuper(Lookuper delegate, - TableInfo tableInfo, - @Nullable List lookupColumnNames) { + TypedLookuper( + Lookuper delegate, + TableInfo tableInfo, + @Nullable List lookupColumnNames) { this.delegate = delegate; this.tableInfo = tableInfo; this.lookupColumnNames = lookupColumnNames; diff --git a/fluss-client/src/main/java/org/apache/fluss/client/table/scanner/Scan.java b/fluss-client/src/main/java/org/apache/fluss/client/table/scanner/Scan.java index e15e3894b7..c8d47a9796 100644 --- a/fluss-client/src/main/java/org/apache/fluss/client/table/scanner/Scan.java +++ b/fluss-client/src/main/java/org/apache/fluss/client/table/scanner/Scan.java @@ -88,4 +88,4 @@ public interface Scan { * #limit(int)} and only support for Primary Key Tables. */ BatchScanner createBatchScanner(TableBucket tableBucket, long snapshotId); -} \ No newline at end of file +} diff --git a/fluss-client/src/main/java/org/apache/fluss/client/table/scanner/ScanRecord.java b/fluss-client/src/main/java/org/apache/fluss/client/table/scanner/ScanRecord.java index 074875b9c8..0f96e06abf 100644 --- a/fluss-client/src/main/java/org/apache/fluss/client/table/scanner/ScanRecord.java +++ b/fluss-client/src/main/java/org/apache/fluss/client/table/scanner/ScanRecord.java @@ -89,4 +89,4 @@ public int hashCode() { public String toString() { return changeType.shortString() + value + "@" + offset; } -} \ No newline at end of file +} diff --git a/fluss-client/src/main/java/org/apache/fluss/client/table/scanner/TableScan.java b/fluss-client/src/main/java/org/apache/fluss/client/table/scanner/TableScan.java index 93afcf58f3..ac530ab2bb 100644 --- a/fluss-client/src/main/java/org/apache/fluss/client/table/scanner/TableScan.java +++ b/fluss-client/src/main/java/org/apache/fluss/client/table/scanner/TableScan.java @@ -171,4 +171,4 @@ public BatchScanner createBatchScanner(TableBucket tableBucket, long snapshotId) tableInfo.getTableConfig().getKvFormat(), conn.getOrCreateRemoteFileDownloader()); } -} \ No newline at end of file +} diff --git a/fluss-client/src/main/java/org/apache/fluss/client/table/scanner/log/CompletedFetch.java b/fluss-client/src/main/java/org/apache/fluss/client/table/scanner/log/CompletedFetch.java index 88780b9ab6..a4c0ee4e90 100644 --- a/fluss-client/src/main/java/org/apache/fluss/client/table/scanner/log/CompletedFetch.java +++ b/fluss-client/src/main/java/org/apache/fluss/client/table/scanner/log/CompletedFetch.java @@ -255,4 +255,4 @@ private void maybeCloseRecordStream() { records = null; } } -} \ No newline at end of file +} diff --git a/fluss-client/src/main/java/org/apache/fluss/client/table/scanner/log/LogFetchCollector.java b/fluss-client/src/main/java/org/apache/fluss/client/table/scanner/log/LogFetchCollector.java index daf3cf409f..87e738fc58 100644 --- a/fluss-client/src/main/java/org/apache/fluss/client/table/scanner/log/LogFetchCollector.java +++ b/fluss-client/src/main/java/org/apache/fluss/client/table/scanner/log/LogFetchCollector.java @@ -20,7 +20,6 @@ import org.apache.fluss.annotation.Internal; import org.apache.fluss.client.metadata.MetadataUpdater; import org.apache.fluss.client.table.scanner.ScanRecord; -import org.apache.fluss.row.InternalRow; import org.apache.fluss.config.ConfigOptions; import org.apache.fluss.config.Configuration; import org.apache.fluss.exception.AuthorizationException; @@ -30,6 +29,7 @@ import org.apache.fluss.metadata.TablePath; import org.apache.fluss.record.LogRecord; import org.apache.fluss.record.LogRecordBatch; +import org.apache.fluss.row.InternalRow; import org.apache.fluss.rpc.protocol.ApiError; import org.apache.fluss.rpc.protocol.Errors; @@ -84,7 +84,8 @@ public LogFetchCollector( * @throws LogOffsetOutOfRangeException If there is OffsetOutOfRange error in fetchResponse and * the defaultResetPolicy is NONE */ - public Map>> collectFetch(final LogFetchBuffer logFetchBuffer) { + public Map>> collectFetch( + final LogFetchBuffer logFetchBuffer) { Map>> fetched = new HashMap<>(); int recordsRemaining = maxPollRecords; @@ -116,7 +117,8 @@ public Map>> collectFetch(final LogFet logFetchBuffer.poll(); } else { - List> records = fetchRecords(nextInLineFetch, recordsRemaining); + List> records = + fetchRecords(nextInLineFetch, recordsRemaining); if (!records.isEmpty()) { TableBucket tableBucket = nextInLineFetch.tableBucket; List> currentRecords = fetched.get(tableBucket); @@ -147,7 +149,8 @@ public Map>> collectFetch(final LogFet return fetched; } - private List> fetchRecords(CompletedFetch nextInLineFetch, int maxRecords) { + private List> fetchRecords( + CompletedFetch nextInLineFetch, int maxRecords) { TableBucket tb = nextInLineFetch.tableBucket; Long offset = logScannerStatus.getBucketOffset(tb); if (offset == null) { @@ -290,4 +293,4 @@ private void handleInitializeErrors( error, fetchOffset, tb, error.exception(errorMessage))); } } -} \ No newline at end of file +} diff --git a/fluss-client/src/main/java/org/apache/fluss/client/table/scanner/log/LogScannerImpl.java b/fluss-client/src/main/java/org/apache/fluss/client/table/scanner/log/LogScannerImpl.java index dbd89925a6..491773a7c8 100644 --- a/fluss-client/src/main/java/org/apache/fluss/client/table/scanner/log/LogScannerImpl.java +++ b/fluss-client/src/main/java/org/apache/fluss/client/table/scanner/log/LogScannerImpl.java @@ -306,4 +306,4 @@ public void close() { release(); } } -} \ No newline at end of file +} diff --git a/fluss-client/src/main/java/org/apache/fluss/client/table/scanner/log/ScanRecords.java b/fluss-client/src/main/java/org/apache/fluss/client/table/scanner/log/ScanRecords.java index ea38f6adaf..1473954c51 100644 --- a/fluss-client/src/main/java/org/apache/fluss/client/table/scanner/log/ScanRecords.java +++ b/fluss-client/src/main/java/org/apache/fluss/client/table/scanner/log/ScanRecords.java @@ -37,7 +37,9 @@ */ @PublicEvolving public class ScanRecords implements Iterable> { - public static final ScanRecords empty() { return new ScanRecords<>(Collections.emptyMap()); } + public static final ScanRecords empty() { + return new ScanRecords<>(Collections.emptyMap()); + } private final Map>> records; @@ -113,4 +115,4 @@ public ScanRecord makeNext() { }; } } -} \ No newline at end of file +} diff --git a/fluss-client/src/main/java/org/apache/fluss/client/table/scanner/log/TypedLogScanner.java b/fluss-client/src/main/java/org/apache/fluss/client/table/scanner/log/TypedLogScanner.java index a13ded7fe9..f8b7cdb857 100644 --- a/fluss-client/src/main/java/org/apache/fluss/client/table/scanner/log/TypedLogScanner.java +++ b/fluss-client/src/main/java/org/apache/fluss/client/table/scanner/log/TypedLogScanner.java @@ -38,13 +38,15 @@ public class TypedLogScanner implements LogScanner { private final LogScanner delegate; private final RowToPojoConverter converter; - public TypedLogScanner(LogScanner delegate, - Class pojoClass, - TableInfo tableInfo, - int[] projectedColumns) { + public TypedLogScanner( + LogScanner delegate, + Class pojoClass, + TableInfo tableInfo, + int[] projectedColumns) { this.delegate = delegate; RowType tableSchema = tableInfo.getRowType(); - RowType projection = projectedColumns == null ? tableSchema : tableSchema.project(projectedColumns); + RowType projection = + projectedColumns == null ? tableSchema : tableSchema.project(projectedColumns); this.converter = RowToPojoConverter.of(pojoClass, tableSchema, projection); } @@ -61,7 +63,8 @@ public ScanRecords poll(Duration timeout) { for (ScanRecord r : list) { InternalRow row = r.getValue(); T pojo = converter.fromRow(row); - converted.add(new ScanRecord<>(r.logOffset(), r.timestamp(), r.getChangeType(), pojo)); + converted.add( + new ScanRecord<>(r.logOffset(), r.timestamp(), r.getChangeType(), pojo)); } out.put(bucket, converted); } @@ -96,4 +99,4 @@ public void close() { throw new RuntimeException(e); } } -} \ No newline at end of file +} diff --git a/fluss-client/src/main/java/org/apache/fluss/client/table/writer/Append.java b/fluss-client/src/main/java/org/apache/fluss/client/table/writer/Append.java index 24dd8a6845..594ef7cbfe 100644 --- a/fluss-client/src/main/java/org/apache/fluss/client/table/writer/Append.java +++ b/fluss-client/src/main/java/org/apache/fluss/client/table/writer/Append.java @@ -37,4 +37,4 @@ public interface Append { /** Create a new typed {@link AppendWriter} to write POJOs directly. */ AppendWriter createWriter(Class pojoClass); -} \ No newline at end of file +} diff --git a/fluss-client/src/main/java/org/apache/fluss/client/table/writer/AppendWriter.java b/fluss-client/src/main/java/org/apache/fluss/client/table/writer/AppendWriter.java index ff0f62eaae..7223f2f639 100644 --- a/fluss-client/src/main/java/org/apache/fluss/client/table/writer/AppendWriter.java +++ b/fluss-client/src/main/java/org/apache/fluss/client/table/writer/AppendWriter.java @@ -40,4 +40,4 @@ public interface AppendWriter extends TableWriter { * @return A {@link CompletableFuture} that always returns append result when complete normally. */ CompletableFuture append(T record); -} \ No newline at end of file +} diff --git a/fluss-client/src/main/java/org/apache/fluss/client/table/writer/TableAppend.java b/fluss-client/src/main/java/org/apache/fluss/client/table/writer/TableAppend.java index 5d3340f180..975acdd969 100644 --- a/fluss-client/src/main/java/org/apache/fluss/client/table/writer/TableAppend.java +++ b/fluss-client/src/main/java/org/apache/fluss/client/table/writer/TableAppend.java @@ -44,4 +44,4 @@ public AppendWriter createWriter(Class pojoClass) { AppendWriterImpl delegate = new AppendWriterImpl(tablePath, tableInfo, writerClient); return new TypedAppendWriter<>(delegate, pojoClass, tableInfo); } -} \ No newline at end of file +} diff --git a/fluss-client/src/main/java/org/apache/fluss/client/table/writer/TableUpsert.java b/fluss-client/src/main/java/org/apache/fluss/client/table/writer/TableUpsert.java index d5e8921e04..cd8d747848 100644 --- a/fluss-client/src/main/java/org/apache/fluss/client/table/writer/TableUpsert.java +++ b/fluss-client/src/main/java/org/apache/fluss/client/table/writer/TableUpsert.java @@ -98,7 +98,8 @@ public UpsertWriter createWriter() { @Override public UpsertWriter createWriter(Class pojoClass) { - UpsertWriterImpl delegate = new UpsertWriterImpl(tablePath, tableInfo, targetColumns, writerClient); + UpsertWriterImpl delegate = + new UpsertWriterImpl(tablePath, tableInfo, targetColumns, writerClient); return new TypedUpsertWriter<>(delegate, pojoClass, tableInfo, targetColumns); } -} \ No newline at end of file +} diff --git a/fluss-client/src/main/java/org/apache/fluss/client/table/writer/TypedAppendWriter.java b/fluss-client/src/main/java/org/apache/fluss/client/table/writer/TypedAppendWriter.java index 67f4113c28..256305ab1b 100644 --- a/fluss-client/src/main/java/org/apache/fluss/client/table/writer/TypedAppendWriter.java +++ b/fluss-client/src/main/java/org/apache/fluss/client/table/writer/TypedAppendWriter.java @@ -51,8 +51,9 @@ public CompletableFuture append(T record) { return delegate.append((InternalRow) record); } // TODO: initialize this on the constructor and reuse - PojoToRowConverter converter = PojoToRowConverter.of(pojoClass, tableSchema, tableSchema); + PojoToRowConverter converter = + PojoToRowConverter.of(pojoClass, tableSchema, tableSchema); InternalRow row = converter.toRow(record); return delegate.append(row); } -} \ No newline at end of file +} diff --git a/fluss-client/src/main/java/org/apache/fluss/client/table/writer/TypedUpsertWriter.java b/fluss-client/src/main/java/org/apache/fluss/client/table/writer/TypedUpsertWriter.java index fe895b3bb4..68976f8e29 100644 --- a/fluss-client/src/main/java/org/apache/fluss/client/table/writer/TypedUpsertWriter.java +++ b/fluss-client/src/main/java/org/apache/fluss/client/table/writer/TypedUpsertWriter.java @@ -42,7 +42,11 @@ public void flush() { private final RowType tableSchema; private final int[] targetColumns; // may be null - TypedUpsertWriter(UpsertWriterImpl delegate, Class pojoClass, TableInfo tableInfo, int[] targetColumns) { + TypedUpsertWriter( + UpsertWriterImpl delegate, + Class pojoClass, + TableInfo tableInfo, + int[] targetColumns) { this.delegate = delegate; this.pojoClass = pojoClass; this.tableInfo = tableInfo; @@ -55,7 +59,7 @@ public CompletableFuture upsert(T record) { if (record instanceof InternalRow) { return delegate.upsert((InternalRow) record); } - InternalRow row = convertPojo(record, /*forDelete=*/false); + InternalRow row = convertPojo(record, /*forDelete=*/ false); return delegate.upsert(row); } @@ -64,7 +68,7 @@ public CompletableFuture delete(T record) { if (record instanceof InternalRow) { return delegate.delete((InternalRow) record); } - InternalRow pkOnly = convertPojo(record, /*forDelete=*/true); + InternalRow pkOnly = convertPojo(record, /*forDelete=*/ true); return delegate.delete(pkOnly); } @@ -103,4 +107,4 @@ private InternalRow convertPojo(T pojo, boolean forDelete) { } return full; } -} \ No newline at end of file +} diff --git a/fluss-client/src/main/java/org/apache/fluss/client/table/writer/Upsert.java b/fluss-client/src/main/java/org/apache/fluss/client/table/writer/Upsert.java index 89cc8d8a6d..c5e3c1c762 100644 --- a/fluss-client/src/main/java/org/apache/fluss/client/table/writer/Upsert.java +++ b/fluss-client/src/main/java/org/apache/fluss/client/table/writer/Upsert.java @@ -57,12 +57,11 @@ public interface Upsert { Upsert partialUpdate(String... targetColumnNames); /** - * Create a new {@link UpsertWriter} using {@code InternalRow} with the optional - * {@link #partialUpdate(String...)} information to upsert and delete data to a Primary Key - * Table. + * Create a new {@link UpsertWriter} using {@code InternalRow} with the optional {@link + * #partialUpdate(String...)} information to upsert and delete data to a Primary Key Table. */ UpsertWriter createWriter(); /** Create a new typed {@link UpsertWriter} to write POJOs directly. */ UpsertWriter createWriter(Class pojoClass); -} \ No newline at end of file +} 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 6fbd607ba4..c0a4c51f2d 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 @@ -51,4 +51,4 @@ public interface UpsertWriter extends TableWriter { * @return A {@link CompletableFuture} that always delete result when complete normally. */ CompletableFuture delete(T record); -} \ No newline at end of file +} From a24c32a19df594a4d3a7b74af17554b2456577fd Mon Sep 17 00:00:00 2001 From: ipolyzos Date: Mon, 17 Nov 2025 17:09:30 +0200 Subject: [PATCH 04/33] update tests --- .../admin/ClientToServerITCaseBase.java | 17 ++-- .../table/AutoPartitionedTableITCase.java | 14 ++-- .../table/FlussFailServerTableITCase.java | 8 +- .../client/table/FlussLakeTableITCase.java | 11 +-- .../fluss/client/table/FlussTableITCase.java | 80 ++++++++++--------- .../client/table/PartitionedTableITCase.java | 2 +- .../log/DefaultCompletedFetchTest.java | 10 +-- .../scanner/log/LogFetchCollectorTest.java | 5 +- .../table/scanner/log/LogScannerITCase.java | 44 +++++----- .../scanner/log/RemoteCompletedFetchTest.java | 15 ++-- .../scanner/log/RemoteLogScannerITCase.java | 16 ++-- 11 files changed, 117 insertions(+), 105 deletions(-) diff --git a/fluss-client/src/test/java/org/apache/fluss/client/admin/ClientToServerITCaseBase.java b/fluss-client/src/test/java/org/apache/fluss/client/admin/ClientToServerITCaseBase.java index a62c1b70d3..c57ef9027c 100644 --- a/fluss-client/src/test/java/org/apache/fluss/client/admin/ClientToServerITCaseBase.java +++ b/fluss-client/src/test/java/org/apache/fluss/client/admin/ClientToServerITCaseBase.java @@ -129,15 +129,15 @@ private static Configuration initConfig() { return conf; } - protected static LogScanner createLogScanner(Table table) { + protected static LogScanner createLogScanner(Table table) { return table.newScan().createLogScanner(); } - protected static LogScanner createLogScanner(Table table, int[] projectFields) { + protected static LogScanner createLogScanner(Table table, int[] projectFields) { return table.newScan().project(projectFields).createLogScanner(); } - protected static void subscribeFromBeginning(LogScanner logScanner, Table table) { + protected static void subscribeFromBeginning(LogScanner logScanner, Table table) { int bucketCount = table.getTableInfo().getNumBuckets(); for (int i = 0; i < bucketCount; i++) { logScanner.subscribeFromBeginning(i); @@ -216,10 +216,10 @@ public static void verifyPartitionLogs( logScanner.subscribeFromBeginning(partitionId, 0); } while (scanRecordCount < totalRecords) { - ScanRecords scanRecords = logScanner.poll(Duration.ofSeconds(1)); + ScanRecords scanRecords = logScanner.poll(Duration.ofSeconds(1)); for (TableBucket scanBucket : scanRecords.buckets()) { - List records = scanRecords.records(scanBucket); - for (ScanRecord scanRecord : records) { + List> records = scanRecords.records(scanBucket); + for (ScanRecord scanRecord : records) { actualRows .computeIfAbsent( scanBucket.getPartitionId(), k -> new ArrayList<>()) @@ -272,13 +272,14 @@ protected static void verifyPutAndLookup(Table table, Object[] fields) throws Ex upsertWriter.upsert(row); upsertWriter.flush(); // lookup this key. - Lookuper lookuper = table.newLookup().createLookuper(); + Lookuper lookuper = table.newLookup().createLookuper(); ProjectedRow keyRow = ProjectedRow.from(schema.getPrimaryKeyIndexes()); keyRow.replaceRow(row); assertThatRow(lookupRow(lookuper, keyRow)).withSchema(schema.getRowType()).isEqualTo(row); } - protected static InternalRow lookupRow(Lookuper lookuper, InternalRow keyRow) throws Exception { + protected static InternalRow lookupRow(Lookuper lookuper, InternalRow keyRow) + throws Exception { // lookup this key. return lookuper.lookup(keyRow).get().getSingletonRow(); } diff --git a/fluss-client/src/test/java/org/apache/fluss/client/table/AutoPartitionedTableITCase.java b/fluss-client/src/test/java/org/apache/fluss/client/table/AutoPartitionedTableITCase.java index 6ae9ed7be7..d3166b1c09 100644 --- a/fluss-client/src/test/java/org/apache/fluss/client/table/AutoPartitionedTableITCase.java +++ b/fluss-client/src/test/java/org/apache/fluss/client/table/AutoPartitionedTableITCase.java @@ -93,7 +93,7 @@ void testPartitionedPrimaryKeyTable() throws Exception { } upsertWriter.flush(); - Lookuper lookuper = table.newLookup().createLookuper(); + Lookuper lookuper = table.newLookup().createLookuper(); // now, let's lookup the written data by look up for (String partition : partitionIdByNames.keySet()) { for (int i = 0; i < recordsPerPartition; i++) { @@ -285,14 +285,14 @@ private Map> writeRows( } private Map> pollRecords( - LogScanner logScanner, int expectRecordsCount) { + LogScanner logScanner, int expectRecordsCount) { int scanRecordCount = 0; Map> actualRows = new HashMap<>(); while (scanRecordCount < expectRecordsCount) { - ScanRecords scanRecords = logScanner.poll(Duration.ofSeconds(1)); + ScanRecords scanRecords = logScanner.poll(Duration.ofSeconds(1)); for (TableBucket scanBucket : scanRecords.buckets()) { - List records = scanRecords.records(scanBucket); - for (ScanRecord scanRecord : records) { + List> records = scanRecords.records(scanBucket); + for (ScanRecord scanRecord : records) { actualRows .computeIfAbsent(scanBucket.getPartitionId(), k -> new ArrayList<>()) .add(scanRecord.getRow()); @@ -308,7 +308,7 @@ void testOperateNotExistPartitionShouldThrowException() throws Exception { createPartitionedTable(DATA1_TABLE_PATH_PK, true); Table table = conn.getTable(DATA1_TABLE_PATH_PK); String partitionName = "notExistPartition"; - Lookuper lookuper = table.newLookup().createLookuper(); + Lookuper lookuper = table.newLookup().createLookuper(); // lookup a not exist partition will return null. assertThat(lookuper.lookup(row(1, partitionName)).get().getSingletonRow()).isEqualTo(null); @@ -324,7 +324,7 @@ void testOperateNotExistPartitionShouldThrowException() throws Exception { PhysicalTablePath.of(DATA1_TABLE_PATH_PK, partitionName)); // test scan a not exist partition's log - LogScanner logScanner = table.newScan().createLogScanner(); + LogScanner logScanner = table.newScan().createLogScanner(); assertThatThrownBy(() -> logScanner.subscribe(100L, 0, 0)) .isInstanceOf(PartitionNotExistException.class) .hasMessageContaining("The partition id '100' does not exist"); diff --git a/fluss-client/src/test/java/org/apache/fluss/client/table/FlussFailServerTableITCase.java b/fluss-client/src/test/java/org/apache/fluss/client/table/FlussFailServerTableITCase.java index 341c9a80d8..ccc22af76e 100644 --- a/fluss-client/src/test/java/org/apache/fluss/client/table/FlussFailServerTableITCase.java +++ b/fluss-client/src/test/java/org/apache/fluss/client/table/FlussFailServerTableITCase.java @@ -109,13 +109,13 @@ void testLogScan() throws Exception { // append one row. GenericRow row = row(1, "a"); try (Table table = conn.getTable(DATA1_TABLE_PATH); - LogScanner logScanner = createLogScanner(table)) { + LogScanner logScanner = createLogScanner(table)) { subscribeFromBeginning(logScanner, table); AppendWriter appendWriter = table.newAppend().createWriter(); appendWriter.append(row).get(); // poll data util we get one record - ScanRecords scanRecords; + ScanRecords scanRecords; do { scanRecords = logScanner.poll(Duration.ofSeconds(1)); } while (scanRecords.isEmpty()); @@ -147,9 +147,9 @@ void testLogScan() throws Exception { } } - private List toRows(ScanRecords scanRecords) { + private List toRows(ScanRecords scanRecords) { List rows = new ArrayList<>(); - for (ScanRecord scanRecord : scanRecords) { + for (ScanRecord scanRecord : scanRecords) { rows.add(scanRecord.getRow()); } return rows; diff --git a/fluss-client/src/test/java/org/apache/fluss/client/table/FlussLakeTableITCase.java b/fluss-client/src/test/java/org/apache/fluss/client/table/FlussLakeTableITCase.java index 5ad6a10ddf..4e41c80ddd 100644 --- a/fluss-client/src/test/java/org/apache/fluss/client/table/FlussLakeTableITCase.java +++ b/fluss-client/src/test/java/org/apache/fluss/client/table/FlussLakeTableITCase.java @@ -138,7 +138,7 @@ void testDeleteOnPrimaryKeyTable() throws Exception { tableWriter.delete(row(2, null)); tableWriter.flush(); - Lookuper lookuper = table.newLookup().createLookuper(); + Lookuper lookuper = table.newLookup().createLookuper(); List row1 = lookuper.lookup(row(1)).get().getRowList(); assertThatRows(row1) .withSchema(TestData.DATA1_SCHEMA_PK.getRowType()) @@ -208,7 +208,8 @@ void testPrimaryKeyTable(boolean isPartitioned, boolean isDefaultBucketKey) thro } // lookup try (Table table = conn.getTable(tablePath)) { - Lookuper lookuper = table.newLookup().lookupBy(lookUpColumns).createLookuper(); + Lookuper lookuper = + table.newLookup().lookupBy(lookUpColumns).createLookuper(); for (InternalRow row : allRows) { GenericRow lookupKeyRow = new GenericRow(lookUpFieldGetter.size()); for (int i = 0; i < lookUpFieldGetter.size(); i++) { @@ -317,7 +318,7 @@ private Map> writeRowsAndVerifyBucket( int scanCount = 0; Map> actualRows = new HashMap<>(); try (Table table = conn.getTable(tablePath); - LogScanner logScanner = table.newScan().createLogScanner()) { + LogScanner logScanner = table.newScan().createLogScanner()) { for (int bucket = 0; bucket < DEFAULT_BUCKET_COUNT; bucket++) { if (partitionIdByNames != null) { for (long partitionId : partitionIdByNames.values()) { @@ -328,13 +329,13 @@ private Map> writeRowsAndVerifyBucket( } } while (scanCount < totalRows) { - ScanRecords scanRecords = logScanner.poll(Duration.ofSeconds(1)); + ScanRecords scanRecords = logScanner.poll(Duration.ofSeconds(1)); for (TableBucket tableBucket : scanRecords.buckets()) { actualRows .computeIfAbsent(tableBucket, (k) -> new ArrayList<>()) .addAll( scanRecords.records(tableBucket).stream() - .map(ScanRecord::getRow) + .map(ScanRecord::getRow) .collect(Collectors.toList())); } scanCount += scanRecords.count(); diff --git a/fluss-client/src/test/java/org/apache/fluss/client/table/FlussTableITCase.java b/fluss-client/src/test/java/org/apache/fluss/client/table/FlussTableITCase.java index 0cc2f7f10f..12963105c5 100644 --- a/fluss-client/src/test/java/org/apache/fluss/client/table/FlussTableITCase.java +++ b/fluss-client/src/test/java/org/apache/fluss/client/table/FlussTableITCase.java @@ -144,12 +144,12 @@ void testAppendWithSmallBuffer(boolean indexedFormat) throws Exception { appendWriter.flush(); // assert the written data - LogScanner logScanner = createLogScanner(table); + LogScanner logScanner = createLogScanner(table); subscribeFromBeginning(logScanner, table); int count = 0; while (count < expectedSize) { - ScanRecords scanRecords = logScanner.poll(Duration.ofSeconds(1)); - for (ScanRecord scanRecord : scanRecords) { + ScanRecords scanRecords = logScanner.poll(Duration.ofSeconds(1)); + for (ScanRecord scanRecord : scanRecords) { assertThat(scanRecord.getChangeType()).isEqualTo(ChangeType.APPEND_ONLY); InternalRow row = scanRecord.getRow(); assertThat(row.getInt(0)).isEqualTo(1); @@ -178,13 +178,13 @@ void testPollOnce() throws Exception { appendWriter.flush(); // assert the written data - LogScanner logScanner = createLogScanner(table); + LogScanner logScanner = createLogScanner(table); subscribeFromBeginning(logScanner, table); int count = 0; while (count < expectedSize) { - ScanRecords scanRecords = logScanner.poll(Duration.ofSeconds(1)); + ScanRecords scanRecords = logScanner.poll(Duration.ofSeconds(1)); assertThat(scanRecords.isEmpty()).isFalse(); - for (ScanRecord scanRecord : scanRecords) { + for (ScanRecord scanRecord : scanRecords) { assertThat(scanRecord.getChangeType()).isEqualTo(ChangeType.APPEND_ONLY); InternalRow row = scanRecord.getRow(); assertThat(row.getInt(0)).isEqualTo(1); @@ -218,12 +218,12 @@ void testUpsertWithSmallBuffer() throws Exception { upsertWriter.flush(); // assert the written data - LogScanner logScanner = createLogScanner(table); + LogScanner logScanner = createLogScanner(table); subscribeFromBeginning(logScanner, table); int count = 0; while (count < expectedSize) { - ScanRecords scanRecords = logScanner.poll(Duration.ofSeconds(1)); - for (ScanRecord scanRecord : scanRecords) { + ScanRecords scanRecords = logScanner.poll(Duration.ofSeconds(1)); + for (ScanRecord scanRecord : scanRecords) { assertThat(scanRecord.getChangeType()).isEqualTo(ChangeType.INSERT); InternalRow row = scanRecord.getRow(); assertThat(row.getInt(0)).isEqualTo(count); @@ -467,7 +467,7 @@ void testLookupForNotReadyTable() throws Exception { // if you want to test the lookup for not ready table, you can comment the following line. waitAllReplicasReady(tableId, 10); Table table = conn.getTable(tablePath); - Lookuper lookuper = table.newLookup().createLookuper(); + Lookuper lookuper = table.newLookup().createLookuper(); assertThat(lookupRow(lookuper, rowKey)).isNull(); } @@ -597,7 +597,7 @@ void testPartialPutAndDelete() throws Exception { UpsertWriter upsertWriter = table.newUpsert().partialUpdate(new int[] {0, 1}).createWriter(); upsertWriter.upsert(row(1, "aaa", null, null)).get(); - Lookuper lookuper = table.newLookup().createLookuper(); + Lookuper lookuper = table.newLookup().createLookuper(); // check the row GenericRow rowKey = row(1); @@ -696,7 +696,7 @@ void testDelete() throws Exception { try (Table table = conn.getTable(DATA1_TABLE_PATH_PK)) { UpsertWriter upsertWriter = table.newUpsert().createWriter(); upsertWriter.upsert(row).get(); - Lookuper lookuper = table.newLookup().createLookuper(); + Lookuper lookuper = table.newLookup().createLookuper(); // lookup this key. IndexedRow keyRow = keyRow(DATA1_SCHEMA_PK, new Object[] {1, "a"}); @@ -727,12 +727,12 @@ void testAppendWhileTableMaybeNotReady() throws Exception { appendWriter.append(row).get(); // fetch data. - LogScanner logScanner = createLogScanner(table); + LogScanner logScanner = createLogScanner(table); subscribeFromBeginning(logScanner, table); InternalRow result = null; while (result == null) { - ScanRecords scanRecords = logScanner.poll(Duration.ofSeconds(1)); - for (ScanRecord scanRecord : scanRecords) { + ScanRecords scanRecords = logScanner.poll(Duration.ofSeconds(1)); + for (ScanRecord scanRecord : scanRecords) { assertThat(scanRecord.getChangeType()).isEqualTo(ChangeType.APPEND_ONLY); result = scanRecord.getRow(); } @@ -808,12 +808,12 @@ void verifyAppendOrPut(boolean append, String logFormat, @Nullable String kvForm // fetch data. try (Table table = conn.getTable(DATA1_TABLE_PATH); - LogScanner logScanner = createLogScanner(table)) { + LogScanner logScanner = createLogScanner(table)) { subscribeFromBeginning(logScanner, table); int count = 0; while (count < expectedSize) { - ScanRecords scanRecords = logScanner.poll(Duration.ofSeconds(1)); - for (ScanRecord scanRecord : scanRecords) { + ScanRecords scanRecords = logScanner.poll(Duration.ofSeconds(1)); + for (ScanRecord scanRecord : scanRecords) { if (append) { assertThat(scanRecord.getChangeType()).isEqualTo(ChangeType.APPEND_ONLY); } else { @@ -869,12 +869,12 @@ void testAppendAndProject(String format) throws Exception { } // fetch data. - LogScanner logScanner = createLogScanner(table, new int[] {0, 2}); + LogScanner logScanner = createLogScanner(table, new int[] {0, 2}); subscribeFromBeginning(logScanner, table); int count = 0; while (count < expectedSize) { - ScanRecords scanRecords = logScanner.poll(Duration.ofSeconds(1)); - for (ScanRecord scanRecord : scanRecords) { + ScanRecords scanRecords = logScanner.poll(Duration.ofSeconds(1)); + for (ScanRecord scanRecord : scanRecords) { assertThat(scanRecord.getChangeType()).isEqualTo(ChangeType.APPEND_ONLY); assertThat(scanRecord.getRow().getFieldCount()).isEqualTo(2); assertThat(scanRecord.getRow().getInt(0)).isEqualTo(count); @@ -896,8 +896,8 @@ void testAppendAndProject(String format) throws Exception { subscribeFromBeginning(logScanner, table); count = 0; while (count < expectedSize) { - ScanRecords scanRecords = logScanner.poll(Duration.ofSeconds(1)); - for (ScanRecord scanRecord : scanRecords) { + ScanRecords scanRecords = logScanner.poll(Duration.ofSeconds(1)); + for (ScanRecord scanRecord : scanRecords) { assertThat(scanRecord.getChangeType()).isEqualTo(ChangeType.APPEND_ONLY); assertThat(scanRecord.getRow().getFieldCount()).isEqualTo(2); assertThat(scanRecord.getRow().getInt(1)).isEqualTo(count); @@ -1200,7 +1200,7 @@ void testFirstRowMergeEngine(boolean doProjection) throws Exception { upsertWriter.flush(); - Lookuper lookuper = table.newLookup().createLookuper(); + Lookuper lookuper = table.newLookup().createLookuper(); // now, get rows by lookup for (int id = 0; id < rows; id++) { InternalRow gotRow = lookuper.lookup(row(id)).get().getSingletonRow(); @@ -1211,18 +1211,20 @@ void testFirstRowMergeEngine(boolean doProjection) throws Exception { if (doProjection) { scan = scan.project(new int[] {0}); // do projection. } - LogScanner logScanner = scan.createLogScanner(); + LogScanner logScanner = scan.createLogScanner(); logScanner.subscribeFromBeginning(0); - List actualLogRecords = new ArrayList<>(0); + List> actualLogRecords = new ArrayList<>(0); while (actualLogRecords.size() < rows) { - ScanRecords scanRecords = logScanner.poll(Duration.ofSeconds(1)); - scanRecords.forEach(actualLogRecords::add); + ScanRecords scanRecords = logScanner.poll(Duration.ofSeconds(1)); + for (ScanRecord rec : scanRecords) { + actualLogRecords.add(rec); + } } logScanner.close(); assertThat(actualLogRecords).hasSize(rows); for (int i = 0; i < actualLogRecords.size(); i++) { - ScanRecord scanRecord = actualLogRecords.get(i); + ScanRecord scanRecord = actualLogRecords.get(i); assertThat(scanRecord.getChangeType()).isEqualTo(ChangeType.INSERT); assertThatRow(scanRecord.getRow()) .withSchema(doProjection ? rowType.project(new int[] {0}) : rowType) @@ -1265,12 +1267,12 @@ void testArrowCompressionAndProject(String compression, String level) throws Exc } // fetch data without project. - LogScanner logScanner = createLogScanner(table); + LogScanner logScanner = createLogScanner(table); subscribeFromBeginning(logScanner, table); int count = 0; while (count < expectedSize) { - ScanRecords scanRecords = logScanner.poll(Duration.ofSeconds(1)); - for (ScanRecord scanRecord : scanRecords) { + ScanRecords scanRecords = logScanner.poll(Duration.ofSeconds(1)); + for (ScanRecord scanRecord : scanRecords) { assertThat(scanRecord.getChangeType()).isEqualTo(ChangeType.APPEND_ONLY); assertThat(scanRecord.getRow().getInt(0)).isEqualTo(count); @@ -1293,8 +1295,8 @@ void testArrowCompressionAndProject(String compression, String level) throws Exc subscribeFromBeginning(logScanner, table); count = 0; while (count < expectedSize) { - ScanRecords scanRecords = logScanner.poll(Duration.ofSeconds(1)); - for (ScanRecord scanRecord : scanRecords) { + ScanRecords scanRecords = logScanner.poll(Duration.ofSeconds(1)); + for (ScanRecord scanRecord : scanRecords) { assertThat(scanRecord.getChangeType()).isEqualTo(ChangeType.APPEND_ONLY); assertThat(scanRecord.getRow().getFieldCount()).isEqualTo(2); assertThat(scanRecord.getRow().getInt(0)).isEqualTo(count); @@ -1381,12 +1383,14 @@ void testMergeEngineWithVersion(boolean doProjection) throws Exception { if (doProjection) { scan = scan.project(new int[] {0}); // do projection. } - LogScanner logScanner = scan.createLogScanner(); + LogScanner logScanner = scan.createLogScanner(); logScanner.subscribeFromBeginning(0); - List actualLogRecords = new ArrayList<>(rows); + List> actualLogRecords = new ArrayList<>(rows); while (actualLogRecords.size() < rows) { - ScanRecords scanRecords = logScanner.poll(Duration.ofSeconds(1)); - scanRecords.forEach(actualLogRecords::add); + ScanRecords scanRecords = logScanner.poll(Duration.ofSeconds(1)); + for (ScanRecord rec : scanRecords) { + actualLogRecords.add(rec); + } } logScanner.close(); diff --git a/fluss-client/src/test/java/org/apache/fluss/client/table/PartitionedTableITCase.java b/fluss-client/src/test/java/org/apache/fluss/client/table/PartitionedTableITCase.java index d204d087a4..438a9c0ed6 100644 --- a/fluss-client/src/test/java/org/apache/fluss/client/table/PartitionedTableITCase.java +++ b/fluss-client/src/test/java/org/apache/fluss/client/table/PartitionedTableITCase.java @@ -90,7 +90,7 @@ void testPartitionedPrimaryKeyTable() throws Exception { upsertWriter.flush(); - Lookuper lookuper = table.newLookup().createLookuper(); + Lookuper lookuper = table.newLookup().createLookuper(); // now, let's lookup the written data by look up. for (PartitionInfo partitionInfo : partitionInfos) { String partitionName = partitionInfo.getPartitionName(); diff --git a/fluss-client/src/test/java/org/apache/fluss/client/table/scanner/log/DefaultCompletedFetchTest.java b/fluss-client/src/test/java/org/apache/fluss/client/table/scanner/log/DefaultCompletedFetchTest.java index 72739ef265..3c8f3a7674 100644 --- a/fluss-client/src/test/java/org/apache/fluss/client/table/scanner/log/DefaultCompletedFetchTest.java +++ b/fluss-client/src/test/java/org/apache/fluss/client/table/scanner/log/DefaultCompletedFetchTest.java @@ -103,7 +103,7 @@ void testSimple(byte recordBatchMagic) throws Exception { tb, createMemoryLogRecords(DATA2, LogFormat.ARROW, recordBatchMagic), 10L); DefaultCompletedFetch defaultCompletedFetch = makeCompletedFetch(tb, resultForBucket0, fetchOffset); - List scanRecords = defaultCompletedFetch.fetchRecords(8); + List> scanRecords = defaultCompletedFetch.fetchRecords(8); assertThat(scanRecords.size()).isEqualTo(8); assertThat(scanRecords.get(0).logOffset()).isEqualTo(0L); @@ -126,7 +126,7 @@ void testNegativeFetchCount(byte recordBatchMagic) throws Exception { tb, createMemoryLogRecords(DATA2, LogFormat.ARROW, recordBatchMagic), 10L); DefaultCompletedFetch defaultCompletedFetch = makeCompletedFetch(tb, resultForBucket0, fetchOffset); - List scanRecords = defaultCompletedFetch.fetchRecords(-10); + List> scanRecords = defaultCompletedFetch.fetchRecords(-10); assertThat(scanRecords.size()).isEqualTo(0); } @@ -139,7 +139,7 @@ void testNoRecordsInFetch() { new FetchLogResultForBucket(tb, MemoryLogRecords.EMPTY, 0L); DefaultCompletedFetch defaultCompletedFetch = makeCompletedFetch(tb, resultForBucket0, fetchOffset); - List scanRecords = defaultCompletedFetch.fetchRecords(10); + List> scanRecords = defaultCompletedFetch.fetchRecords(10); assertThat(scanRecords.size()).isEqualTo(0); } @@ -181,7 +181,7 @@ void testProjection(LogFormat logFormat, byte magic) throws Exception { new FetchLogResultForBucket(tb, memoryLogRecords, 10L); DefaultCompletedFetch defaultCompletedFetch = makeCompletedFetch(tb, resultForBucket0, fetchOffset, projection); - List scanRecords = defaultCompletedFetch.fetchRecords(8); + List> scanRecords = defaultCompletedFetch.fetchRecords(8); List expectedObjects = Arrays.asList( new Object[] {1, "hello"}, @@ -195,7 +195,7 @@ void testProjection(LogFormat logFormat, byte magic) throws Exception { assertThat(scanRecords.size()).isEqualTo(8); for (int i = 0; i < scanRecords.size(); i++) { Object[] expectObject = expectedObjects.get(i); - ScanRecord actualRecord = scanRecords.get(i); + ScanRecord actualRecord = scanRecords.get(i); assertThat(actualRecord.logOffset()).isEqualTo(i); assertThat(actualRecord.getChangeType()).isEqualTo(ChangeType.APPEND_ONLY); InternalRow row = actualRecord.getRow(); diff --git a/fluss-client/src/test/java/org/apache/fluss/client/table/scanner/log/LogFetchCollectorTest.java b/fluss-client/src/test/java/org/apache/fluss/client/table/scanner/log/LogFetchCollectorTest.java index 99a108e2f3..4bba0e1682 100644 --- a/fluss-client/src/test/java/org/apache/fluss/client/table/scanner/log/LogFetchCollectorTest.java +++ b/fluss-client/src/test/java/org/apache/fluss/client/table/scanner/log/LogFetchCollectorTest.java @@ -23,6 +23,7 @@ import org.apache.fluss.config.Configuration; import org.apache.fluss.metadata.TableBucket; import org.apache.fluss.record.LogRecordReadContext; +import org.apache.fluss.row.InternalRow; import org.apache.fluss.rpc.entity.FetchLogResultForBucket; import org.junit.jupiter.api.AfterEach; @@ -97,7 +98,7 @@ void testNormal() throws Exception { assertThat(completedFetch.isInitialized()).isFalse(); // Fetch the data and validate that we get all the records we want back. - Map> bucketAndRecords = + Map>> bucketAndRecords = logFetchCollector.collectFetch(logFetchBuffer); assertThat(bucketAndRecords.size()).isEqualTo(1); assertThat(bucketAndRecords.get(tb)).size().isEqualTo(10); @@ -147,7 +148,7 @@ void testCollectAfterUnassign() throws Exception { // unassign bucket 2 logScannerStatus.unassignScanBuckets(Collections.singletonList(tb2)); - Map> bucketAndRecords = + Map>> bucketAndRecords = logFetchCollector.collectFetch(logFetchBuffer); // should only contain records for bucket 1 assertThat(bucketAndRecords.keySet()).containsExactly(tb1); diff --git a/fluss-client/src/test/java/org/apache/fluss/client/table/scanner/log/LogScannerITCase.java b/fluss-client/src/test/java/org/apache/fluss/client/table/scanner/log/LogScannerITCase.java index cfe215fb0b..5540760197 100644 --- a/fluss-client/src/test/java/org/apache/fluss/client/table/scanner/log/LogScannerITCase.java +++ b/fluss-client/src/test/java/org/apache/fluss/client/table/scanner/log/LogScannerITCase.java @@ -70,12 +70,12 @@ void testPoll() throws Exception { appendWriter.append(row).get(); } - LogScanner logScanner = createLogScanner(table); + LogScanner logScanner = createLogScanner(table); subscribeFromBeginning(logScanner, table); List rowList = new ArrayList<>(); while (rowList.size() < recordSize) { - ScanRecords scanRecords = logScanner.poll(Duration.ofSeconds(1)); - for (ScanRecord scanRecord : scanRecords) { + ScanRecords scanRecords = logScanner.poll(Duration.ofSeconds(1)); + for (ScanRecord scanRecord : scanRecords) { assertThat(scanRecord.getChangeType()).isEqualTo(ChangeType.APPEND_ONLY); InternalRow row = scanRecord.getRow(); rowList.add(row(row.getInt(0), row.getString(1))); @@ -105,12 +105,12 @@ void testPollWhileCreateTableNotReady() throws Exception { appendWriter.append(row).get(); } - LogScanner logScanner = createLogScanner(table); + LogScanner logScanner = createLogScanner(table); subscribeFromBeginning(logScanner, table); List rowList = new ArrayList<>(); while (rowList.size() < recordSize) { - ScanRecords scanRecords = logScanner.poll(Duration.ofSeconds(1)); - for (ScanRecord scanRecord : scanRecords) { + ScanRecords scanRecords = logScanner.poll(Duration.ofSeconds(1)); + for (ScanRecord scanRecord : scanRecords) { assertThat(scanRecord.getChangeType()).isEqualTo(ChangeType.APPEND_ONLY); InternalRow row = scanRecord.getRow(); rowList.add(row(row.getInt(0), row.getString(1))); @@ -136,7 +136,7 @@ void testLogScannerMultiThreadAccess() throws Exception { appendWriter.append(row).get(); } - LogScanner logScanner = table.newScan().createLogScanner(); + LogScanner logScanner = table.newScan().createLogScanner(); ExecutorService executor = Executors.newSingleThreadExecutor(); // subscribe in thread1 executor.submit(() -> logScanner.subscribe(0, LogScanner.EARLIEST_OFFSET)).get(); @@ -148,8 +148,8 @@ void testLogScannerMultiThreadAccess() throws Exception { // should be able to poll data from all buckets List rowList = new ArrayList<>(); while (rowList.size() < recordSize) { - ScanRecords scanRecords = logScanner.poll(Duration.ofSeconds(1)); - for (ScanRecord scanRecord : scanRecords) { + ScanRecords scanRecords = logScanner.poll(Duration.ofSeconds(1)); + for (ScanRecord scanRecord : scanRecords) { assertThat(scanRecord.getChangeType()).isEqualTo(ChangeType.APPEND_ONLY); InternalRow row = scanRecord.getRow(); rowList.add(row(row.getInt(0), row.getString(1))); @@ -196,13 +196,13 @@ void testLogHeavyWriteAndScan() throws Exception { } appendWriter.flush(); - LogScanner logScanner = createLogScanner(table); + LogScanner logScanner = createLogScanner(table); subscribeFromBeginning(logScanner, table); long scanned = 0; long total = 0; while (scanned < recordSize) { - ScanRecords scanRecords = logScanner.poll(Duration.ofSeconds(1)); - for (ScanRecord scanRecord : scanRecords) { + ScanRecords scanRecords = logScanner.poll(Duration.ofSeconds(1)); + for (ScanRecord scanRecord : scanRecords) { assertThat(scanRecord.getChangeType()).isEqualTo(ChangeType.APPEND_ONLY); assertThat(scanRecord.getRow().getString(0).getSizeInBytes()).isEqualTo(10); assertThat(scanRecord.getRow().getLong(1)).isEqualTo(scanned); @@ -252,13 +252,13 @@ void testKvHeavyWriteAndScan() throws Exception { } upsertWriter.flush(); - LogScanner logScanner = createLogScanner(table); + LogScanner logScanner = createLogScanner(table); subscribeFromBeginning(logScanner, table); long scanned = 0; long total = 0; while (scanned < recordSize) { - ScanRecords scanRecords = logScanner.poll(Duration.ofSeconds(1)); - for (ScanRecord scanRecord : scanRecords) { + ScanRecords scanRecords = logScanner.poll(Duration.ofSeconds(1)); + for (ScanRecord scanRecord : scanRecords) { assertThat(scanRecord.getChangeType()).isEqualTo(ChangeType.INSERT); assertThat(scanRecord.getRow().getString(0).getSizeInBytes()).isEqualTo(10); assertThat(scanRecord.getRow().getLong(1)).isEqualTo(scanned); @@ -320,7 +320,7 @@ void testScanFromStartTimestamp(boolean isPartitioned) throws Exception { // as early as possible to avoid potential time backwards long secondStartTimestamp = System.currentTimeMillis(); - LogScanner logScanner = createLogScanner(table); + LogScanner logScanner = createLogScanner(table); // try to fetch from firstStartTimestamp, which smaller than the first batch commit // timestamp. subscribeFromTimestamp( @@ -333,8 +333,8 @@ void testScanFromStartTimestamp(boolean isPartitioned) throws Exception { firstStartTimestamp); List rowList = new ArrayList<>(); while (rowList.size() < batchRecordSize) { - ScanRecords scanRecords = logScanner.poll(Duration.ofSeconds(1)); - for (ScanRecord scanRecord : scanRecords) { + ScanRecords scanRecords = logScanner.poll(Duration.ofSeconds(1)); + for (ScanRecord scanRecord : scanRecords) { assertThat(scanRecord.getChangeType()).isEqualTo(ChangeType.APPEND_ONLY); InternalRow row = scanRecord.getRow(); rowList.add(row(row.getInt(0), row.getString(1))); @@ -363,8 +363,8 @@ void testScanFromStartTimestamp(boolean isPartitioned) throws Exception { secondStartTimestamp); rowList = new ArrayList<>(); while (rowList.size() < batchRecordSize) { - ScanRecords scanRecords = logScanner.poll(Duration.ofSeconds(1)); - for (ScanRecord scanRecord : scanRecords) { + ScanRecords scanRecords = logScanner.poll(Duration.ofSeconds(1)); + for (ScanRecord scanRecord : scanRecords) { assertThat(scanRecord.getChangeType()).isEqualTo(ChangeType.APPEND_ONLY); InternalRow row = scanRecord.getRow(); rowList.add(row(row.getInt(0), row.getString(1))); @@ -424,8 +424,8 @@ void testScanFromLatestOffsets(boolean isPartitioned) throws Exception { List rowList = new ArrayList<>(); while (rowList.size() < batchRecordSize) { - ScanRecords scanRecords = logScanner.poll(Duration.ofSeconds(1)); - for (ScanRecord scanRecord : scanRecords) { + ScanRecords scanRecords = logScanner.poll(Duration.ofSeconds(1)); + for (ScanRecord scanRecord : scanRecords) { assertThat(scanRecord.getChangeType()).isEqualTo(ChangeType.APPEND_ONLY); InternalRow row = scanRecord.getRow(); rowList.add(row(row.getInt(0), row.getString(1))); diff --git a/fluss-client/src/test/java/org/apache/fluss/client/table/scanner/log/RemoteCompletedFetchTest.java b/fluss-client/src/test/java/org/apache/fluss/client/table/scanner/log/RemoteCompletedFetchTest.java index edc47a7ecb..65292e47fe 100644 --- a/fluss-client/src/test/java/org/apache/fluss/client/table/scanner/log/RemoteCompletedFetchTest.java +++ b/fluss-client/src/test/java/org/apache/fluss/client/table/scanner/log/RemoteCompletedFetchTest.java @@ -113,7 +113,8 @@ void testSimple() throws Exception { null, () -> recycleCalled.set(true)); - List scanRecords = completedFetch.fetchRecords(8); + List> scanRecords = + completedFetch.fetchRecords(8); assertThat(scanRecords.size()).isEqualTo(8); assertThat(scanRecords.get(0).logOffset()).isEqualTo(0L); @@ -146,7 +147,8 @@ void testFetchForPartitionTable() throws Exception { makeCompletedFetch( tb, fileLogRecords, fetchOffset, null, () -> recycleCalled.set(true)); - List scanRecords = completedFetch.fetchRecords(8); + List> scanRecords = + completedFetch.fetchRecords(8); assertThat(scanRecords.size()).isEqualTo(8); assertThat(scanRecords.get(0).logOffset()).isEqualTo(0L); @@ -174,7 +176,8 @@ void testNegativeFetchCount() throws Exception { RemoteCompletedFetch completedFetch = makeCompletedFetch(tableBucket, fileLogRecords, fetchOffset, null); - List scanRecords = completedFetch.fetchRecords(-10); + List> scanRecords = + completedFetch.fetchRecords(-10); assertThat(scanRecords.size()).isEqualTo(0); } @@ -191,7 +194,8 @@ void testNoRecordsInFetch() throws Exception { RemoteCompletedFetch completedFetch = makeCompletedFetch(tableBucket, fileLogRecords, fetchOffset, null); - List scanRecords = completedFetch.fetchRecords(10); + List> scanRecords = + completedFetch.fetchRecords(10); assertThat(scanRecords.size()).isEqualTo(0); } @@ -231,7 +235,8 @@ void testProjection(String format) throws Exception { fetchOffset, Projection.of(new int[] {0, 2}, schema)); - List scanRecords = completedFetch.fetchRecords(8); + List> scanRecords = + completedFetch.fetchRecords(8); List expectedObjects = Arrays.asList( new Object[] {1, "hello"}, diff --git a/fluss-client/src/test/java/org/apache/fluss/client/table/scanner/log/RemoteLogScannerITCase.java b/fluss-client/src/test/java/org/apache/fluss/client/table/scanner/log/RemoteLogScannerITCase.java index 84b50af2c7..b9a1fce7e6 100644 --- a/fluss-client/src/test/java/org/apache/fluss/client/table/scanner/log/RemoteLogScannerITCase.java +++ b/fluss-client/src/test/java/org/apache/fluss/client/table/scanner/log/RemoteLogScannerITCase.java @@ -99,12 +99,12 @@ void testScanFromRemote() throws Exception { FLUSS_CLUSTER_EXTENSION.waitUntilSomeLogSegmentsCopyToRemote(new TableBucket(tableId, 0)); // test fetch. - LogScanner logScanner = table.newScan().createLogScanner(); + LogScanner logScanner = table.newScan().createLogScanner(); logScanner.subscribeFromBeginning(0); List rowList = new ArrayList<>(); while (rowList.size() < recordSize) { - ScanRecords scanRecords = logScanner.poll(Duration.ofSeconds(1)); - for (ScanRecord scanRecord : scanRecords) { + ScanRecords scanRecords = logScanner.poll(Duration.ofSeconds(1)); + for (ScanRecord scanRecord : scanRecords) { assertThat(scanRecord.getChangeType()).isEqualTo(ChangeType.APPEND_ONLY); InternalRow row = scanRecord.getRow(); rowList.add(row(row.getInt(0), row.getString(1))); @@ -150,12 +150,12 @@ void testScanFromRemoteAndProject(String format) throws Exception { FLUSS_CLUSTER_EXTENSION.waitUntilSomeLogSegmentsCopyToRemote(new TableBucket(tableId, 0)); // test fetch. - LogScanner logScanner = createLogScanner(table, new int[] {0, 2}); + LogScanner logScanner = createLogScanner(table, new int[] {0, 2}); logScanner.subscribeFromBeginning(0); int count = 0; while (count < expectedSize) { - ScanRecords scanRecords = logScanner.poll(Duration.ofSeconds(1)); - for (ScanRecord scanRecord : scanRecords) { + ScanRecords scanRecords = logScanner.poll(Duration.ofSeconds(1)); + for (ScanRecord scanRecord : scanRecords) { assertThat(scanRecord.getChangeType()).isEqualTo(ChangeType.APPEND_ONLY); assertThat(scanRecord.getRow().getFieldCount()).isEqualTo(2); assertThat(scanRecord.getRow().getInt(0)).isEqualTo(count); @@ -177,8 +177,8 @@ void testScanFromRemoteAndProject(String format) throws Exception { logScanner.subscribeFromBeginning(0); count = 0; while (count < expectedSize) { - ScanRecords scanRecords = logScanner.poll(Duration.ofSeconds(1)); - for (ScanRecord scanRecord : scanRecords) { + ScanRecords scanRecords = logScanner.poll(Duration.ofSeconds(1)); + for (ScanRecord scanRecord : scanRecords) { assertThat(scanRecord.getChangeType()).isEqualTo(ChangeType.APPEND_ONLY); assertThat(scanRecord.getRow().getFieldCount()).isEqualTo(2); assertThat(scanRecord.getRow().getInt(1)).isEqualTo(count); From 7dc648dc516337c098a8e68048dd113f0080d670 Mon Sep 17 00:00:00 2001 From: ipolyzos Date: Mon, 17 Nov 2025 17:34:50 +0200 Subject: [PATCH 05/33] add end2end tests --- .../client/converter/ConverterCommons.java | 17 + .../client/converter/PojoToRowConverter.java | 25 +- .../fluss/client/converter/PojoType.java | 36 +- .../client/table/FlussTypedClientITCase.java | 444 ++++++++++++++++++ .../scanner/log/RemoteCompletedFetchTest.java | 9 +- 5 files changed, 514 insertions(+), 17 deletions(-) create mode 100644 fluss-client/src/test/java/org/apache/fluss/client/table/FlussTypedClientITCase.java diff --git a/fluss-client/src/main/java/org/apache/fluss/client/converter/ConverterCommons.java b/fluss-client/src/main/java/org/apache/fluss/client/converter/ConverterCommons.java index 5f4f1b6c12..700456edae 100644 --- a/fluss-client/src/main/java/org/apache/fluss/client/converter/ConverterCommons.java +++ b/fluss-client/src/main/java/org/apache/fluss/client/converter/ConverterCommons.java @@ -83,6 +83,23 @@ static void validatePojoMatchesTable(PojoType pojoType, RowType tableSchema) } } + static void validatePojoMatchesProjection(PojoType pojoType, RowType projection) { + Set pojoNames = pojoType.getProperties().keySet(); + List fieldNames = projection.getFieldNames(); + if (!pojoNames.containsAll(fieldNames)) { + throw new IllegalArgumentException( + String.format( + "POJO fields %s must contain all projection fields %s.", + pojoNames, fieldNames)); + } + for (int i = 0; i < projection.getFieldCount(); i++) { + String name = fieldNames.get(i); + DataType dt = projection.getTypeAt(i); + PojoType.Property prop = pojoType.getProperty(name); + validateCompatibility(dt, prop); + } + } + static void validateProjectionSubset(RowType projection, RowType tableSchema) { Set tableNames = new HashSet<>(tableSchema.getFieldNames()); for (String n : projection.getFieldNames()) { diff --git a/fluss-client/src/main/java/org/apache/fluss/client/converter/PojoToRowConverter.java b/fluss-client/src/main/java/org/apache/fluss/client/converter/PojoToRowConverter.java index 4d75aa91a5..bd5a580662 100644 --- a/fluss-client/src/main/java/org/apache/fluss/client/converter/PojoToRowConverter.java +++ b/fluss-client/src/main/java/org/apache/fluss/client/converter/PojoToRowConverter.java @@ -30,6 +30,7 @@ import javax.annotation.Nullable; import java.math.BigDecimal; +import java.math.RoundingMode; import java.time.Instant; import java.time.LocalDate; import java.time.LocalDateTime; @@ -54,7 +55,9 @@ private PojoToRowConverter(PojoType pojoType, RowType tableSchema, RowType pr this.tableSchema = tableSchema; this.projection = projection; this.projectionFieldNames = projection.getFieldNames(); - ConverterCommons.validatePojoMatchesTable(pojoType, tableSchema); + // For writer path, allow POJO to be a superset of the projection. It must contain all + // projected fields. + ConverterCommons.validatePojoMatchesProjection(pojoType, projection); ConverterCommons.validateProjectionSubset(projection, tableSchema); this.fieldConverters = createFieldConverters(); } @@ -177,8 +180,24 @@ private static FieldToRow createFieldConverter(PojoType.Property prop, DataType String.format( "Field %s is not a BigDecimal. Cannot convert to Decimal.", prop.name)); } - return Decimal.fromBigDecimal( - (BigDecimal) v, decimalType.getPrecision(), decimalType.getScale()); + final int precision = decimalType.getPrecision(); + final int scale = decimalType.getScale(); + + // Scale with a deterministic rounding mode to avoid ArithmeticException when rounding is + // needed. + BigDecimal bd = (BigDecimal) v; + BigDecimal scaled = bd.setScale(scale, RoundingMode.HALF_UP); + + // Validate precision after scaling; precision is the number of digits in the unscaled + // value. + if (scaled.precision() > precision) { + throw new IllegalArgumentException( + String.format( + "Decimal value for field %s exceeds precision %d after scaling to %d: %s", + prop.name, precision, scale, scaled)); + } + + return Decimal.fromBigDecimal(scaled, precision, scale); } /** Converts a LocalDate POJO property to number of days since epoch. */ diff --git a/fluss-client/src/main/java/org/apache/fluss/client/converter/PojoType.java b/fluss-client/src/main/java/org/apache/fluss/client/converter/PojoType.java index 7790b91438..158a45e774 100644 --- a/fluss-client/src/main/java/org/apache/fluss/client/converter/PojoType.java +++ b/fluss-client/src/main/java/org/apache/fluss/client/converter/PojoType.java @@ -37,6 +37,8 @@ final class PojoType { private final Class pojoClass; private final Constructor defaultConstructor; private final Map properties; // property name -> property + // Mapping of primitive types to their boxed counterparts to avoid long if-chains + private static final Map, Class> PRIMITIVE_TO_BOXED = createPrimitiveToBoxedMap(); private PojoType(Class pojoClass, Constructor ctor, Map props) { this.pojoClass = pojoClass; @@ -73,12 +75,8 @@ static PojoType of(Class pojoClass) { for (Map.Entry e : allFields.entrySet()) { String name = e.getKey(); Field field = e.getValue(); - if (field.getType().isPrimitive()) { - throw new IllegalArgumentException( - String.format( - "POJO class %s has primitive field '%s' of type %s. Primitive types are not allowed; all fields must be nullable (use wrapper types).", - pojoClass.getName(), name, field.getType().getName())); - } + // Allow primitive types by treating them as their boxed counterparts for compatibility + Class effectiveType = boxIfPrimitive(field.getType()); boolean publicField = Modifier.isPublic(field.getModifiers()); Method getter = getters.get(name); Method setter = setters.get(name); @@ -94,8 +92,7 @@ static PojoType of(Class pojoClass) { } props.put( name, - new Property( - name, field.getType(), publicField ? field : null, getter, setter)); + new Property(name, effectiveType, publicField ? field : null, getter, setter)); } return new PojoType<>(pojoClass, ctor, props); @@ -235,6 +232,29 @@ private static String decapitalize(String s) { return s.substring(0, 1).toLowerCase(Locale.ROOT) + s.substring(1); } + private static Map, Class> createPrimitiveToBoxedMap() { + Map, Class> map = new HashMap<>(); + map.put(boolean.class, Boolean.class); + map.put(byte.class, Byte.class); + map.put(short.class, Short.class); + map.put(int.class, Integer.class); + map.put(long.class, Long.class); + map.put(float.class, Float.class); + map.put(double.class, Double.class); + map.put(char.class, Character.class); + // void shouldn't appear as a field type, but handle defensively + map.put(void.class, Void.class); + return map; + } + + private static Class boxIfPrimitive(Class type) { + if (!type.isPrimitive()) { + return type; + } + Class boxed = PRIMITIVE_TO_BOXED.get(type); + return boxed != null ? boxed : type; + } + static final class Property { final String name; final Class type; diff --git a/fluss-client/src/test/java/org/apache/fluss/client/table/FlussTypedClientITCase.java b/fluss-client/src/test/java/org/apache/fluss/client/table/FlussTypedClientITCase.java new file mode 100644 index 0000000000..ce65ad8ae7 --- /dev/null +++ b/fluss-client/src/test/java/org/apache/fluss/client/table/FlussTypedClientITCase.java @@ -0,0 +1,444 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.fluss.client.table; + +import org.apache.fluss.client.admin.ClientToServerITCaseBase; +import org.apache.fluss.client.converter.RowToPojoConverter; +import org.apache.fluss.client.lookup.LookupResult; +import org.apache.fluss.client.lookup.Lookuper; +import org.apache.fluss.client.table.scanner.Scan; +import org.apache.fluss.client.table.scanner.ScanRecord; +import org.apache.fluss.client.table.scanner.log.LogScanner; +import org.apache.fluss.client.table.scanner.log.ScanRecords; +import org.apache.fluss.client.table.writer.AppendWriter; +import org.apache.fluss.client.table.writer.Upsert; +import org.apache.fluss.client.table.writer.UpsertWriter; +import org.apache.fluss.metadata.Schema; +import org.apache.fluss.metadata.TableDescriptor; +import org.apache.fluss.metadata.TablePath; +import org.apache.fluss.record.ChangeType; +import org.apache.fluss.row.GenericRow; +import org.apache.fluss.row.InternalRow; +import org.apache.fluss.types.DataTypes; +import org.apache.fluss.types.RowType; + +import org.junit.jupiter.api.Test; + +import java.math.BigDecimal; +import java.math.RoundingMode; +import java.time.Duration; +import java.time.Instant; +import java.time.LocalDate; +import java.time.LocalDateTime; +import java.time.LocalTime; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.List; + +import static org.assertj.core.api.Assertions.assertThat; + +/** End-to-end tests for writing and scanning POJOs via client API. */ +public class FlussTypedClientITCase extends ClientToServerITCaseBase { + + /** Test POJO containing all supported field types used by converters. */ + public static class AllTypesPojo { + // primary key + public Integer a; + // all supported converter fields + public Boolean bool1; + public Byte tiny; + public Short small; + public Integer intv; + public Long big; + public Float flt; + public Double dbl; + public Character ch; + public String str; + public byte[] bin; + public byte[] bytes; + public java.math.BigDecimal dec; + public java.time.LocalDate dt; + public java.time.LocalTime tm; + public java.time.LocalDateTime tsNtz; + public java.time.Instant tsLtz; + + public AllTypesPojo() {} + + public AllTypesPojo( + Integer a, + Boolean bool1, + Byte tiny, + Short small, + Integer intv, + Long big, + Float flt, + Double dbl, + Character ch, + String str, + byte[] bin, + byte[] bytes, + java.math.BigDecimal dec, + java.time.LocalDate dt, + java.time.LocalTime tm, + java.time.LocalDateTime tsNtz, + java.time.Instant tsLtz) { + this.a = a; + this.bool1 = bool1; + this.tiny = tiny; + this.small = small; + this.intv = intv; + this.big = big; + this.flt = flt; + this.dbl = dbl; + this.ch = ch; + this.str = str; + this.bin = bin; + this.bytes = bytes; + this.dec = dec; + this.dt = dt; + this.tm = tm; + this.tsNtz = tsNtz; + this.tsLtz = tsLtz; + } + } + + /** Minimal POJO representing the primary key for {@link AllTypesPojo}. */ + public static class PLookupKey { + public Integer a; + + public PLookupKey() {} + + public PLookupKey(Integer a) { + this.a = a; + } + } + + private static Schema allTypesLogSchema() { + return Schema.newBuilder() + .column("a", DataTypes.INT()) + .column("bool1", DataTypes.BOOLEAN()) + .column("tiny", DataTypes.TINYINT()) + .column("small", DataTypes.SMALLINT()) + .column("intv", DataTypes.INT()) + .column("big", DataTypes.BIGINT()) + .column("flt", DataTypes.FLOAT()) + .column("dbl", DataTypes.DOUBLE()) + .column("ch", DataTypes.CHAR(1)) + .column("str", DataTypes.STRING()) + .column("bin", DataTypes.BINARY(3)) + .column("bytes", DataTypes.BYTES()) + .column("dec", DataTypes.DECIMAL(10, 2)) + .column("dt", DataTypes.DATE()) + .column("tm", DataTypes.TIME()) + .column("tsNtz", DataTypes.TIMESTAMP(3)) + .column("tsLtz", DataTypes.TIMESTAMP_LTZ(3)) + .build(); + } + + private static Schema allTypesPkSchema() { + // Same columns as log schema but with PK on 'a' + return Schema.newBuilder() + .column("a", DataTypes.INT()) + .column("bool1", DataTypes.BOOLEAN()) + .column("tiny", DataTypes.TINYINT()) + .column("small", DataTypes.SMALLINT()) + .column("intv", DataTypes.INT()) + .column("big", DataTypes.BIGINT()) + .column("flt", DataTypes.FLOAT()) + .column("dbl", DataTypes.DOUBLE()) + .column("ch", DataTypes.CHAR(1)) + .column("str", DataTypes.STRING()) + .column("bin", DataTypes.BINARY(3)) + .column("bytes", DataTypes.BYTES()) + .column("dec", DataTypes.DECIMAL(10, 2)) + .column("dt", DataTypes.DATE()) + .column("tm", DataTypes.TIME()) + .column("tsNtz", DataTypes.TIMESTAMP(3)) + .column("tsLtz", DataTypes.TIMESTAMP_LTZ(3)) + .primaryKey("a") + .build(); + } + + private static AllTypesPojo newAllTypesPojo(int i) { + Integer a = i; + Boolean bool1 = (i % 2) == 0; + Byte tiny = (byte) (i - 5); + Short small = (short) (100 + i); + Integer intv = 1000 + i; + Long big = 100000L + i; + Float flt = 1.5f + i; + Double dbl = 2.5 + i; + Character ch = (char) ('a' + (i % 26)); + String str = "s" + i; + byte[] bin = new byte[] {(byte) i, (byte) (i + 1), (byte) (i + 2)}; + byte[] bytes = new byte[] {(byte) (10 + i), (byte) (20 + i)}; + BigDecimal dec = new BigDecimal("12345." + (10 + i)).setScale(2, RoundingMode.HALF_UP); + LocalDate dt = LocalDate.of(2024, 1, 1).plusDays(i); + LocalTime tm = LocalTime.of(12, (i * 7) % 60, (i * 11) % 60); + LocalDateTime tsNtz = LocalDateTime.of(2024, 1, 1, 0, 0).plusSeconds(i).withNano(0); + Instant tsLtz = Instant.ofEpochMilli(1700000000000L + (i * 1000L)); + return new AllTypesPojo( + a, bool1, tiny, small, intv, big, flt, dbl, ch, str, bin, bytes, dec, dt, tm, tsNtz, + tsLtz); + } + + @Test + void testTypedAppendWriteAndScan() throws Exception { + // Build all-types log table schema + Schema schema = + Schema.newBuilder() + .column("a", DataTypes.INT()) + .column("bool1", DataTypes.BOOLEAN()) + .column("tiny", DataTypes.TINYINT()) + .column("small", DataTypes.SMALLINT()) + .column("intv", DataTypes.INT()) + .column("big", DataTypes.BIGINT()) + .column("flt", DataTypes.FLOAT()) + .column("dbl", DataTypes.DOUBLE()) + .column("ch", DataTypes.CHAR(1)) + .column("str", DataTypes.STRING()) + .column("bin", DataTypes.BINARY(3)) + .column("bytes", DataTypes.BYTES()) + .column("dec", DataTypes.DECIMAL(10, 2)) + .column("dt", DataTypes.DATE()) + .column("tm", DataTypes.TIME()) + .column("tsNtz", DataTypes.TIMESTAMP(3)) + .column("tsLtz", DataTypes.TIMESTAMP_LTZ(3)) + .build(); + TablePath path = TablePath.of("pojo_db", "all_types_log"); + TableDescriptor td = TableDescriptor.builder().schema(schema).distributedBy(2).build(); + createTable(path, td, true); + + try (Table table = conn.getTable(path)) { + // write + AppendWriter writer = table.newAppend().createWriter(AllTypesPojo.class); + List expected = new ArrayList<>(); + for (int i = 0; i < 5; i++) { + AllTypesPojo u = newAllTypesPojo(i); + expected.add(u); + writer.append(u).get(); + } + writer.flush(); + + // read + Scan scan = table.newScan(); + LogScanner scanner = scan.createLogScanner(AllTypesPojo.class); + subscribeFromBeginning(scanner, table); + + List actual = new ArrayList<>(); + while (actual.size() < expected.size()) { + ScanRecords recs = scanner.poll(Duration.ofSeconds(2)); + for (ScanRecord r : recs) { + assertThat(r.getChangeType()).isEqualTo(ChangeType.APPEND_ONLY); + actual.add(r.getValue()); + } + } + assertThat(actual) + .usingRecursiveFieldByFieldElementComparator() + .containsExactlyInAnyOrderElementsOf(expected); + } + } + + @Test + void testTypedUpsertWriteAndScan() throws Exception { + // Build all-types PK table schema (PK on 'a') + Schema schema = allTypesPkSchema(); + TablePath path = TablePath.of("pojo_db", "all_types_pk"); + TableDescriptor td = TableDescriptor.builder().schema(schema).distributedBy(2, "a").build(); + createTable(path, td, true); + + try (Table table = conn.getTable(path)) { + Upsert upsert = table.newUpsert(); + UpsertWriter writer = upsert.createWriter(AllTypesPojo.class); + + AllTypesPojo p1 = newAllTypesPojo(1); + AllTypesPojo p2 = newAllTypesPojo(2); + writer.upsert(p1).get(); + writer.upsert(p2).get(); + + // update key 1: change a couple of fields + AllTypesPojo p1Updated = newAllTypesPojo(1); + p1Updated.str = "a1"; + p1Updated.dec = new java.math.BigDecimal("42.42"); + writer.upsert(p1Updated).get(); + writer.flush(); + + // scan as POJOs and verify change types and values + LogScanner scanner = table.newScan().createLogScanner(AllTypesPojo.class); + subscribeFromBeginning(scanner, table); + + List changes = new ArrayList<>(); + List values = new ArrayList<>(); + while (values.size() < 4) { // INSERT 1, INSERT 2, UPDATE_BEFORE 1, UPDATE_AFTER 1 + ScanRecords recs = scanner.poll(Duration.ofSeconds(2)); + for (ScanRecord r : recs) { + changes.add(r.getChangeType()); + values.add(r.getValue()); + } + } + assertThat(changes) + .contains(ChangeType.INSERT, ChangeType.UPDATE_BEFORE, ChangeType.UPDATE_AFTER); + // ensure the last update_after reflects new value + int lastIdx = changes.lastIndexOf(ChangeType.UPDATE_AFTER); + assertThat(values.get(lastIdx).str).isEqualTo("a1"); + } + } + + @Test + void testTypedLookups() throws Exception { + Schema schema = allTypesPkSchema(); + TablePath path = TablePath.of("pojo_db", "lookup_pk"); + TableDescriptor td = TableDescriptor.builder().schema(schema).distributedBy(2, "a").build(); + createTable(path, td, true); + + try (Table table = conn.getTable(path)) { + UpsertWriter writer = table.newUpsert().createWriter(AllTypesPojo.class); + writer.upsert(newAllTypesPojo(1)).get(); + writer.upsert(newAllTypesPojo(2)).get(); + writer.flush(); + + // primary key lookup using Lookuper API with POJO key + Lookuper lookuper = table.newLookup().createLookuper(); + RowType tableSchema = table.getTableInfo().getRowType(); + RowToPojoConverter rowConv = + RowToPojoConverter.of(AllTypesPojo.class, tableSchema, tableSchema); + + LookupResult lr = lookuper.lookup(new PLookupKey(1)).get(); + AllTypesPojo one = rowConv.fromRow(lr.getSingletonRow()); + assertThat(one.str).isEqualTo("s1"); + } + } + + @Test + void testInternalRowLookup() throws Exception { + Schema schema = allTypesPkSchema(); + TablePath path = TablePath.of("pojo_db", "lookup_internalrow"); + TableDescriptor td = TableDescriptor.builder().schema(schema).distributedBy(2, "a").build(); + createTable(path, td, true); + + try (Table table = conn.getTable(path)) { + // write a couple of rows via POJO writer + UpsertWriter writer = table.newUpsert().createWriter(AllTypesPojo.class); + writer.upsert(newAllTypesPojo(101)).get(); + writer.upsert(newAllTypesPojo(202)).get(); + writer.flush(); + + // now perform lookup using the raw InternalRow path to ensure it's still supported + Lookuper lookuper = table.newLookup().createLookuper(); + RowType tableSchema = table.getTableInfo().getRowType(); + RowType keyProjection = tableSchema.project(table.getTableInfo().getPrimaryKeys()); + + // Build the key row directly using GenericRow to avoid any POJO conversion + GenericRow keyRow = new GenericRow(keyProjection.getFieldCount()); + keyRow.setField(0, 101); // primary key field 'a' + + LookupResult lr = lookuper.lookup(keyRow).get(); + RowToPojoConverter rowConv = + RowToPojoConverter.of(AllTypesPojo.class, tableSchema, tableSchema); + AllTypesPojo pojo = rowConv.fromRow(lr.getSingletonRow()); + assertThat(pojo).isNotNull(); + assertThat(pojo.a).isEqualTo(101); + assertThat(pojo.str).isEqualTo("s101"); + } + } + + @Test + void testTypedProjections() throws Exception { + TablePath path = TablePath.of("pojo_db", "proj_log"); + TableDescriptor td = + TableDescriptor.builder().schema(allTypesLogSchema()).distributedBy(1).build(); + createTable(path, td, true); + + try (Table table = conn.getTable(path)) { + AppendWriter writer = table.newAppend().createWriter(AllTypesPojo.class); + writer.append(newAllTypesPojo(10)).get(); + writer.append(newAllTypesPojo(11)).get(); + writer.flush(); + + // Project only a subset of fields + LogScanner scanner = + table.newScan() + .project(Arrays.asList("a", "str")) + .createLogScanner(AllTypesPojo.class); + subscribeFromBeginning(scanner, table); + ScanRecords recs = scanner.poll(Duration.ofSeconds(2)); + for (ScanRecord r : recs) { + AllTypesPojo u = r.getValue(); + assertThat(u.a).isNotNull(); + assertThat(u.str).isNotNull(); + // non-projected fields should be null + assertThat(u.bool1).isNull(); + assertThat(u.bin).isNull(); + assertThat(u.bytes).isNull(); + assertThat(u.dec).isNull(); + assertThat(u.dt).isNull(); + assertThat(u.tm).isNull(); + assertThat(u.tsNtz).isNull(); + assertThat(u.tsLtz).isNull(); + } + } + } + + @Test + void testTypedPartialUpdates() throws Exception { + // Use full PK schema and update a subset of fields + Schema schema = allTypesPkSchema(); + TablePath path = TablePath.of("pojo_db", "pk_partial"); + TableDescriptor td = TableDescriptor.builder().schema(schema).distributedBy(1, "a").build(); + createTable(path, td, true); + + try (Table table = conn.getTable(path)) { + Upsert upsert = table.newUpsert().partialUpdate("a", "str", "dec"); + UpsertWriter writer = upsert.createWriter(AllTypesPojo.class); + + // initial full row + writer.upsert(newAllTypesPojo(1)).get(); + + // partial update: only PK + subset fields + AllTypesPojo patch = new AllTypesPojo(); + patch.a = 1; + patch.str = "second"; + patch.dec = new java.math.BigDecimal("99.99"); + writer.upsert(patch).get(); + writer.flush(); + + // verify via lookup and scan using Lookuper + POJO key + Lookuper lookuper = table.newLookup().createLookuper(); + RowType tableSchema = table.getTableInfo().getRowType(); + RowToPojoConverter rowConv = + RowToPojoConverter.of(AllTypesPojo.class, tableSchema, tableSchema); + AllTypesPojo lookedUp = + rowConv.fromRow(lookuper.lookup(new PLookupKey(1)).get().getSingletonRow()); + assertThat(lookedUp.str).isEqualTo("second"); + assertThat(lookedUp.dec).isEqualByComparingTo("99.99"); + + LogScanner scanner = table.newScan().createLogScanner(AllTypesPojo.class); + subscribeFromBeginning(scanner, table); + boolean sawUpdateAfter = false; + while (!sawUpdateAfter) { + ScanRecords recs = scanner.poll(Duration.ofSeconds(2)); + for (ScanRecord r : recs) { + if (r.getChangeType() == ChangeType.UPDATE_AFTER) { + assertThat(r.getValue().str).isEqualTo("second"); + sawUpdateAfter = true; + } + } + } + } + } +} diff --git a/fluss-client/src/test/java/org/apache/fluss/client/table/scanner/log/RemoteCompletedFetchTest.java b/fluss-client/src/test/java/org/apache/fluss/client/table/scanner/log/RemoteCompletedFetchTest.java index 65292e47fe..64729804e8 100644 --- a/fluss-client/src/test/java/org/apache/fluss/client/table/scanner/log/RemoteCompletedFetchTest.java +++ b/fluss-client/src/test/java/org/apache/fluss/client/table/scanner/log/RemoteCompletedFetchTest.java @@ -176,8 +176,7 @@ void testNegativeFetchCount() throws Exception { RemoteCompletedFetch completedFetch = makeCompletedFetch(tableBucket, fileLogRecords, fetchOffset, null); - List> scanRecords = - completedFetch.fetchRecords(-10); + List> scanRecords = completedFetch.fetchRecords(-10); assertThat(scanRecords.size()).isEqualTo(0); } @@ -194,8 +193,7 @@ void testNoRecordsInFetch() throws Exception { RemoteCompletedFetch completedFetch = makeCompletedFetch(tableBucket, fileLogRecords, fetchOffset, null); - List> scanRecords = - completedFetch.fetchRecords(10); + List> scanRecords = completedFetch.fetchRecords(10); assertThat(scanRecords.size()).isEqualTo(0); } @@ -235,8 +233,7 @@ void testProjection(String format) throws Exception { fetchOffset, Projection.of(new int[] {0, 2}, schema)); - List> scanRecords = - completedFetch.fetchRecords(8); + List> scanRecords = completedFetch.fetchRecords(8); List expectedObjects = Arrays.asList( new Object[] {1, "hello"}, From 4dc25d6818b3eaac40e8c9818ef7c5904d152669 Mon Sep 17 00:00:00 2001 From: ipolyzos Date: Mon, 17 Nov 2025 18:05:18 +0200 Subject: [PATCH 06/33] add required parameterized types --- .../client/converter/ConverterCommons.java | 2 +- .../fluss/client/converter/PojoType.java | 7 +++- .../LakeSnapshotAndLogSplitScanner.java | 4 +-- .../source/emitter/FlinkRecordEmitter.java | 35 ++++++++++++++++++- .../lookup/FlinkAsyncLookupFunction.java | 5 +-- .../source/lookup/FlinkLookupFunction.java | 2 +- 6 files changed, 47 insertions(+), 8 deletions(-) diff --git a/fluss-client/src/main/java/org/apache/fluss/client/converter/ConverterCommons.java b/fluss-client/src/main/java/org/apache/fluss/client/converter/ConverterCommons.java index 700456edae..c43f2ae9a8 100644 --- a/fluss-client/src/main/java/org/apache/fluss/client/converter/ConverterCommons.java +++ b/fluss-client/src/main/java/org/apache/fluss/client/converter/ConverterCommons.java @@ -89,7 +89,7 @@ static void validatePojoMatchesProjection(PojoType pojoType, RowType projecti if (!pojoNames.containsAll(fieldNames)) { throw new IllegalArgumentException( String.format( - "POJO fields %s must contain all projection fields %s.", + "POJO fields %s must contain all projection fields %s. For full-table writes, POJO fields must exactly match table schema fields.", pojoNames, fieldNames)); } for (int i = 0; i < projection.getFieldCount(); i++) { diff --git a/fluss-client/src/main/java/org/apache/fluss/client/converter/PojoType.java b/fluss-client/src/main/java/org/apache/fluss/client/converter/PojoType.java index 158a45e774..9253b4f835 100644 --- a/fluss-client/src/main/java/org/apache/fluss/client/converter/PojoType.java +++ b/fluss-client/src/main/java/org/apache/fluss/client/converter/PojoType.java @@ -75,7 +75,12 @@ static PojoType of(Class pojoClass) { for (Map.Entry e : allFields.entrySet()) { String name = e.getKey(); Field field = e.getValue(); - // Allow primitive types by treating them as their boxed counterparts for compatibility + // Enforce nullable fields: primitives are not allowed in POJO definitions. + if (field.getType().isPrimitive()) { + throw new IllegalArgumentException( + "Primitive types are not allowed; all fields must be nullable (use wrapper types)."); + } + // use boxed type as effective type Class effectiveType = boxIfPrimitive(field.getType()); boolean publicField = Modifier.isPublic(field.getModifiers()); Method getter = getters.get(name); diff --git a/fluss-flink/fluss-flink-common/src/main/java/org/apache/fluss/flink/lake/reader/LakeSnapshotAndLogSplitScanner.java b/fluss-flink/fluss-flink-common/src/main/java/org/apache/fluss/flink/lake/reader/LakeSnapshotAndLogSplitScanner.java index 9d3ce73790..e734eb3083 100644 --- a/fluss-flink/fluss-flink-common/src/main/java/org/apache/fluss/flink/lake/reader/LakeSnapshotAndLogSplitScanner.java +++ b/fluss-flink/fluss-flink-common/src/main/java/org/apache/fluss/flink/lake/reader/LakeSnapshotAndLogSplitScanner.java @@ -219,8 +219,8 @@ public CloseableIterator pollBatch(Duration timeout) throws IOExcep } private void pollLogRecords(Duration timeout) { - ScanRecords scanRecords = logScanner.poll(timeout); - for (ScanRecord scanRecord : scanRecords) { + ScanRecords scanRecords = logScanner.poll(timeout); + for (ScanRecord scanRecord : scanRecords) { boolean isDelete = scanRecord.getChangeType() == ChangeType.DELETE || scanRecord.getChangeType() == ChangeType.UPDATE_BEFORE; diff --git a/fluss-flink/fluss-flink-common/src/main/java/org/apache/fluss/flink/source/emitter/FlinkRecordEmitter.java b/fluss-flink/fluss-flink-common/src/main/java/org/apache/fluss/flink/source/emitter/FlinkRecordEmitter.java index 82a3ed87e6..c536766da0 100644 --- a/fluss-flink/fluss-flink-common/src/main/java/org/apache/fluss/flink/source/emitter/FlinkRecordEmitter.java +++ b/fluss-flink/fluss-flink-common/src/main/java/org/apache/fluss/flink/source/emitter/FlinkRecordEmitter.java @@ -24,6 +24,9 @@ import org.apache.fluss.flink.source.reader.RecordAndPos; import org.apache.fluss.flink.source.split.HybridSnapshotLogSplitState; import org.apache.fluss.flink.source.split.SourceSplitState; +import org.apache.fluss.record.ChangeType; +import org.apache.fluss.record.LogRecord; +import org.apache.fluss.row.InternalRow; import org.apache.flink.api.connector.source.SourceOutput; import org.apache.flink.connector.base.source.reader.RecordEmitter; @@ -88,7 +91,7 @@ public void emitRecord( private void processAndEmitRecord(ScanRecord scanRecord, SourceOutput sourceOutput) { OUT record; try { - record = deserializationSchema.deserialize(scanRecord); + record = deserializationSchema.deserialize(new ScanRecordLogRecord(scanRecord)); } catch (Exception e) { throw new RuntimeException( "Failed to deserialize record: " + scanRecord + ". Cause: " + e.getMessage(), @@ -104,4 +107,34 @@ record = deserializationSchema.deserialize(scanRecord); } } } + + /** Lightweight adapter to view a ScanRecord as a LogRecord. */ + private static final class ScanRecordLogRecord implements LogRecord { + private final ScanRecord delegate; + + private ScanRecordLogRecord(ScanRecord delegate) { + // unchecked, but producer in this module always uses InternalRow + this.delegate = (ScanRecord) delegate; + } + + @Override + public long logOffset() { + return delegate.logOffset(); + } + + @Override + public long timestamp() { + return delegate.timestamp(); + } + + @Override + public ChangeType getChangeType() { + return delegate.getChangeType(); + } + + @Override + public InternalRow getRow() { + return delegate.getRow(); + } + } } diff --git a/fluss-flink/fluss-flink-common/src/main/java/org/apache/fluss/flink/source/lookup/FlinkAsyncLookupFunction.java b/fluss-flink/fluss-flink-common/src/main/java/org/apache/fluss/flink/source/lookup/FlinkAsyncLookupFunction.java index 82fd1bbc36..5258e80d30 100644 --- a/fluss-flink/fluss-flink-common/src/main/java/org/apache/fluss/flink/source/lookup/FlinkAsyncLookupFunction.java +++ b/fluss-flink/fluss-flink-common/src/main/java/org/apache/fluss/flink/source/lookup/FlinkAsyncLookupFunction.java @@ -20,6 +20,7 @@ import org.apache.fluss.client.Connection; import org.apache.fluss.client.ConnectionFactory; import org.apache.fluss.client.lookup.Lookup; +import org.apache.fluss.client.lookup.LookupResult; import org.apache.fluss.client.lookup.LookupType; import org.apache.fluss.client.lookup.Lookuper; import org.apache.fluss.client.table.Table; @@ -67,7 +68,7 @@ public class FlinkAsyncLookupFunction extends AsyncLookupFunction { private transient FlussRowToFlinkRowConverter flussRowToFlinkRowConverter; private transient Connection connection; private transient Table table; - private transient Lookuper lookuper; + private transient Lookuper lookuper; private transient FlinkAsFlussRow lookupRow; public FlinkAsyncLookupFunction( @@ -132,7 +133,7 @@ public CompletableFuture> asyncLookup(RowData keyRow) { CompletableFuture> future = new CompletableFuture<>(); lookuper.lookup(flussKeyRow) .whenComplete( - (result, throwable) -> { + (LookupResult result, Throwable throwable) -> { if (throwable != null) { if (ExceptionUtils.findThrowable( throwable, TableNotExistException.class) diff --git a/fluss-flink/fluss-flink-common/src/main/java/org/apache/fluss/flink/source/lookup/FlinkLookupFunction.java b/fluss-flink/fluss-flink-common/src/main/java/org/apache/fluss/flink/source/lookup/FlinkLookupFunction.java index 4776667375..acd3f73112 100644 --- a/fluss-flink/fluss-flink-common/src/main/java/org/apache/fluss/flink/source/lookup/FlinkLookupFunction.java +++ b/fluss-flink/fluss-flink-common/src/main/java/org/apache/fluss/flink/source/lookup/FlinkLookupFunction.java @@ -62,7 +62,7 @@ public class FlinkLookupFunction extends LookupFunction { private transient FlussRowToFlinkRowConverter flussRowToFlinkRowConverter; private transient Connection connection; private transient Table table; - private transient Lookuper lookuper; + private transient Lookuper lookuper; private transient FlinkAsFlussRow lookupRow; @Nullable private transient ProjectedRow projectedRow; From f291a0fc1fc1c0c674a47772eb5cb8e1ecefd633 Mon Sep 17 00:00:00 2001 From: ipolyzos Date: Mon, 17 Nov 2025 18:10:53 +0200 Subject: [PATCH 07/33] fix checkstyle violation --- .../org/apache/fluss/client/converter/ConverterCommons.java | 2 +- .../apache/fluss/flink/source/emitter/FlinkRecordEmitter.java | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/fluss-client/src/main/java/org/apache/fluss/client/converter/ConverterCommons.java b/fluss-client/src/main/java/org/apache/fluss/client/converter/ConverterCommons.java index c43f2ae9a8..700456edae 100644 --- a/fluss-client/src/main/java/org/apache/fluss/client/converter/ConverterCommons.java +++ b/fluss-client/src/main/java/org/apache/fluss/client/converter/ConverterCommons.java @@ -89,7 +89,7 @@ static void validatePojoMatchesProjection(PojoType pojoType, RowType projecti if (!pojoNames.containsAll(fieldNames)) { throw new IllegalArgumentException( String.format( - "POJO fields %s must contain all projection fields %s. For full-table writes, POJO fields must exactly match table schema fields.", + "POJO fields %s must contain all projection fields %s.", pojoNames, fieldNames)); } for (int i = 0; i < projection.getFieldCount(); i++) { diff --git a/fluss-flink/fluss-flink-common/src/main/java/org/apache/fluss/flink/source/emitter/FlinkRecordEmitter.java b/fluss-flink/fluss-flink-common/src/main/java/org/apache/fluss/flink/source/emitter/FlinkRecordEmitter.java index c536766da0..a413ca7ffc 100644 --- a/fluss-flink/fluss-flink-common/src/main/java/org/apache/fluss/flink/source/emitter/FlinkRecordEmitter.java +++ b/fluss-flink/fluss-flink-common/src/main/java/org/apache/fluss/flink/source/emitter/FlinkRecordEmitter.java @@ -108,7 +108,7 @@ record = deserializationSchema.deserialize(new ScanRecordLogRecord(scanRecord)); } } - /** Lightweight adapter to view a ScanRecord as a LogRecord. */ + /** Lightweight adapter to view a {@code ScanRecord} as a {@link LogRecord}. */ private static final class ScanRecordLogRecord implements LogRecord { private final ScanRecord delegate; From 8297380fb278c7cf4886fce58de0ab51f93ee738 Mon Sep 17 00:00:00 2001 From: ipolyzos Date: Mon, 17 Nov 2025 18:24:00 +0200 Subject: [PATCH 08/33] add missing types to flink module --- .../LakeSnapshotAndLogSplitScanner.java | 2 +- .../source/reader/FlinkSourceSplitReader.java | 7 +-- .../tiering/source/TieringSplitReader.java | 48 ++++++++++++++++--- 3 files changed, 46 insertions(+), 11 deletions(-) diff --git a/fluss-flink/fluss-flink-common/src/main/java/org/apache/fluss/flink/lake/reader/LakeSnapshotAndLogSplitScanner.java b/fluss-flink/fluss-flink-common/src/main/java/org/apache/fluss/flink/lake/reader/LakeSnapshotAndLogSplitScanner.java index e734eb3083..d84963dc84 100644 --- a/fluss-flink/fluss-flink-common/src/main/java/org/apache/fluss/flink/lake/reader/LakeSnapshotAndLogSplitScanner.java +++ b/fluss-flink/fluss-flink-common/src/main/java/org/apache/fluss/flink/lake/reader/LakeSnapshotAndLogSplitScanner.java @@ -65,7 +65,7 @@ public class LakeSnapshotAndLogSplitScanner implements BatchScanner { // the sorted logs in memory, mapping from key -> value private Map logRows; - private final LogScanner logScanner; + private final LogScanner logScanner; private final long stoppingOffset; private boolean logScanFinished; diff --git a/fluss-flink/fluss-flink-common/src/main/java/org/apache/fluss/flink/source/reader/FlinkSourceSplitReader.java b/fluss-flink/fluss-flink-common/src/main/java/org/apache/fluss/flink/source/reader/FlinkSourceSplitReader.java index ce50fe4068..6d6664ca4e 100644 --- a/fluss-flink/fluss-flink-common/src/main/java/org/apache/fluss/flink/source/reader/FlinkSourceSplitReader.java +++ b/fluss-flink/fluss-flink-common/src/main/java/org/apache/fluss/flink/source/reader/FlinkSourceSplitReader.java @@ -39,6 +39,7 @@ import org.apache.fluss.lake.source.LakeSplit; import org.apache.fluss.metadata.TableBucket; import org.apache.fluss.metadata.TablePath; +import org.apache.fluss.row.InternalRow; import org.apache.fluss.types.RowType; import org.apache.fluss.utils.CloseableIterator; import org.apache.fluss.utils.ExceptionUtils; @@ -410,7 +411,7 @@ private void checkSnapshotSplitOrStartNext() { } } - private FlinkRecordsWithSplitIds forLogRecords(ScanRecords scanRecords) { + private FlinkRecordsWithSplitIds forLogRecords(ScanRecords scanRecords) { // For calculating the currentFetchEventTimeLag long fetchTimestamp = System.currentTimeMillis(); long maxConsumerRecordTimestampInFetch = -1; @@ -429,7 +430,7 @@ private FlinkRecordsWithSplitIds forLogRecords(ScanRecords scanRecords) { } splitIdByTableBucket.put(scanBucket, splitId); tableScanBuckets.add(scanBucket); - List bucketScanRecords = scanRecords.records(scanBucket); + List> bucketScanRecords = scanRecords.records(scanBucket); if (!bucketScanRecords.isEmpty()) { final ScanRecord lastRecord = bucketScanRecords.get(bucketScanRecords.size() - 1); // We keep the maximum message timestamp in the fetch for calculating lags @@ -481,7 +482,7 @@ public String next() { } private CloseableIterator toRecordAndPos( - Iterator recordAndPosIterator) { + Iterator> recordAndPosIterator) { return new CloseableIterator() { @Override diff --git a/fluss-flink/fluss-flink-common/src/main/java/org/apache/fluss/flink/tiering/source/TieringSplitReader.java b/fluss-flink/fluss-flink-common/src/main/java/org/apache/fluss/flink/tiering/source/TieringSplitReader.java index b9fe79e3d3..649e14c2dc 100644 --- a/fluss-flink/fluss-flink-common/src/main/java/org/apache/fluss/flink/tiering/source/TieringSplitReader.java +++ b/fluss-flink/fluss-flink-common/src/main/java/org/apache/fluss/flink/tiering/source/TieringSplitReader.java @@ -32,6 +32,9 @@ import org.apache.fluss.metadata.TableBucket; import org.apache.fluss.metadata.TableInfo; import org.apache.fluss.metadata.TablePath; +import org.apache.fluss.record.ChangeType; +import org.apache.fluss.record.LogRecord; +import org.apache.fluss.row.InternalRow; import org.apache.fluss.utils.CloseableIterator; import org.apache.flink.connector.base.source.reader.RecordsWithSplitIds; @@ -249,11 +252,12 @@ private void mayCreateLogScanner() { } private RecordsWithSplitIds> forLogRecords( - ScanRecords scanRecords) throws IOException { + ScanRecords scanRecords) throws IOException { Map> writeResults = new HashMap<>(); Map finishedSplitIds = new HashMap<>(); for (TableBucket bucket : scanRecords.buckets()) { - List bucketScanRecords = scanRecords.records(bucket); + List> bucketScanRecords = + scanRecords.records(bucket); if (bucketScanRecords.isEmpty()) { continue; } @@ -265,13 +269,14 @@ private RecordsWithSplitIds> forLogRecords( LakeWriter lakeWriter = getOrCreateLakeWriter( bucket, currentTableSplitsByBucket.get(bucket).getPartitionName()); - for (ScanRecord record : bucketScanRecords) { + for (ScanRecord record : bucketScanRecords) { // if record is less than stopping offset if (record.logOffset() < stoppingOffset) { - lakeWriter.write(record); + lakeWriter.write(new ScanRecordLogRecord(record)); } } - ScanRecord lastRecord = bucketScanRecords.get(bucketScanRecords.size() - 1); + ScanRecord lastRecord = + bucketScanRecords.get(bucketScanRecords.size() - 1); // has arrived into the end of the split, if (lastRecord.logOffset() >= stoppingOffset - 1) { currentTableStoppingOffsets.remove(bucket); @@ -390,8 +395,8 @@ private TableBucketWriteResultWithSplitIds forSnapshotSplitRecords( getOrCreateLakeWriter( bucket, checkNotNull(currentSnapshotSplit).getPartitionName()); while (recordIterator.hasNext()) { - ScanRecord scanRecord = recordIterator.next().record(); - lakeWriter.write(scanRecord); + ScanRecord scanRecord = recordIterator.next().record(); + lakeWriter.write(new ScanRecordLogRecord(scanRecord)); } recordIterator.close(); return emptyTableBucketWriteResultWithSplitIds(); @@ -460,6 +465,35 @@ public void close() throws Exception { // don't need to close connection, will be closed by TieringSourceReader } + /** Lightweight adapter to view a {@code ScanRecord} as a {@link LogRecord}. */ + private static final class ScanRecordLogRecord implements LogRecord { + private final ScanRecord delegate; + + private ScanRecordLogRecord(ScanRecord delegate) { + this.delegate = delegate; + } + + @Override + public long logOffset() { + return delegate.logOffset(); + } + + @Override + public long timestamp() { + return delegate.timestamp(); + } + + @Override + public ChangeType getChangeType() { + return delegate.getChangeType(); + } + + @Override + public InternalRow getRow() { + return delegate.getRow(); + } + } + private void subscribeLog(TieringLogSplit logSplit) { // assign bucket offset dynamically TableBucket tableBucket = logSplit.getTableBucket(); From 2a8a2b55655e417ad8245cba82f340e043f91a63 Mon Sep 17 00:00:00 2001 From: ipolyzos Date: Mon, 17 Nov 2025 18:37:15 +0200 Subject: [PATCH 09/33] patch tests # Conflicts: # fluss-flink/fluss-flink-common/src/test/java/org/apache/fluss/flink/utils/FlussRowToFlinkRowConverterTest.java --- .../lake/reader/SortMergeReaderTest.java | 39 +++++++++++- .../flink/sink/FlinkTableSinkITCase.java | 6 +- .../fluss/flink/sink/FlussSinkITCase.java | 16 ++--- .../FlussDeserializationSchemaTest.java | 59 +++++++++++++++---- .../RowDataDeserializationSchemaTest.java | 36 ++++++++++- .../FlussRowToFlinkRowConverterTest.java | 31 ++++++++++ 6 files changed, 159 insertions(+), 28 deletions(-) diff --git a/fluss-flink/fluss-flink-common/src/test/java/org/apache/fluss/flink/lake/reader/SortMergeReaderTest.java b/fluss-flink/fluss-flink-common/src/test/java/org/apache/fluss/flink/lake/reader/SortMergeReaderTest.java index edeee58291..b6828c3773 100644 --- a/fluss-flink/fluss-flink-common/src/test/java/org/apache/fluss/flink/lake/reader/SortMergeReaderTest.java +++ b/fluss-flink/fluss-flink-common/src/test/java/org/apache/fluss/flink/lake/reader/SortMergeReaderTest.java @@ -17,7 +17,6 @@ package org.apache.fluss.flink.lake.reader; -import org.apache.fluss.client.table.scanner.ScanRecord; import org.apache.fluss.record.ChangeType; import org.apache.fluss.record.LogRecord; import org.apache.fluss.row.BinaryString; @@ -153,8 +152,44 @@ private List createRecords(int startId, int count, boolean isLog) { startId + i, BinaryString.fromString(isLog ? "a" + "_updated" : "a"), BinaryString.fromString(isLog ? "A" + "_updated" : "A")); - logRecords.add(new ScanRecord(i, System.currentTimeMillis(), ChangeType.INSERT, row)); + logRecords.add( + new TestLogRecord(i, System.currentTimeMillis(), ChangeType.INSERT, row)); } return logRecords; } + + /** Simple LogRecord for tests. */ + private static final class TestLogRecord implements LogRecord { + private final long offset; + private final long ts; + private final ChangeType ct; + private final InternalRow row; + + private TestLogRecord(long offset, long ts, ChangeType ct, InternalRow row) { + this.offset = offset; + this.ts = ts; + this.ct = ct; + this.row = row; + } + + @Override + public long logOffset() { + return offset; + } + + @Override + public long timestamp() { + return ts; + } + + @Override + public ChangeType getChangeType() { + return ct; + } + + @Override + public InternalRow getRow() { + return row; + } + } } 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 68fa741d48..edbc6fc73a 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 @@ -295,17 +295,17 @@ void testAppendLogWithRoundRobin() throws Exception { Configuration clientConf = FLUSS_CLUSTER_EXTENSION.getClientConfig(); try (Connection conn = ConnectionFactory.createConnection(clientConf); Table table = conn.getTable(TablePath.of(DEFAULT_DB, "sink_test")); - LogScanner logScanner = table.newScan().createLogScanner()) { + LogScanner logScanner = table.newScan().createLogScanner()) { logScanner.subscribeFromBeginning(0); logScanner.subscribeFromBeginning(1); logScanner.subscribeFromBeginning(2); long scanned = 0; while (scanned < 6) { - ScanRecords scanRecords = logScanner.poll(Duration.ofSeconds(1)); + ScanRecords scanRecords = logScanner.poll(Duration.ofSeconds(1)); for (TableBucket bucket : scanRecords.buckets()) { List rowsBucket = rows.computeIfAbsent(bucket.getBucket(), k -> new ArrayList<>()); - for (ScanRecord record : scanRecords.records(bucket)) { + for (ScanRecord record : scanRecords.records(bucket)) { InternalRow row = record.getRow(); rowsBucket.add( Row.of(row.getInt(0), row.getLong(1), row.getString(2).toString()) 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 b2d4e47086..bfebff0ddb 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 @@ -149,7 +149,7 @@ public void testRowDataTablePKSink() throws Exception { env.executeAsync("Test RowData Fluss Sink"); Table table = conn.getTable(new TablePath(DEFAULT_DB, pkTableName)); - LogScanner logScanner = table.newScan().createLogScanner(); + LogScanner logScanner = table.newScan().createLogScanner(); int numBuckets = table.getTableInfo().getNumBuckets(); for (int i = 0; i < numBuckets; i++) { @@ -159,9 +159,9 @@ public void testRowDataTablePKSink() throws Exception { List rows = new ArrayList<>(); while (rows.size() < inputRows.size()) { - ScanRecords scanRecords = logScanner.poll(Duration.ofSeconds(1)); + ScanRecords scanRecords = logScanner.poll(Duration.ofSeconds(1)); for (TableBucket bucket : scanRecords.buckets()) { - for (ScanRecord record : scanRecords.records(bucket)) { + for (ScanRecord record : scanRecords.records(bucket)) { RowData row = converter.toFlinkRowData(record.getRow()); row.setRowKind(toFlinkRowKind(record.getChangeType())); rows.add(row); @@ -222,7 +222,7 @@ public void testRowDataTableLogSink() throws Exception { env.executeAsync("Test RowData Fluss Sink"); Table table = conn.getTable(new TablePath(DEFAULT_DB, logTableName)); - LogScanner logScanner = table.newScan().createLogScanner(); + LogScanner logScanner = table.newScan().createLogScanner(); int numBuckets = table.getTableInfo().getNumBuckets(); for (int i = 0; i < numBuckets; i++) { @@ -232,9 +232,9 @@ public void testRowDataTableLogSink() throws Exception { List rows = new ArrayList<>(); while (rows.size() < inputRows.size()) { - ScanRecords scanRecords = logScanner.poll(Duration.ofSeconds(1)); + ScanRecords scanRecords = logScanner.poll(Duration.ofSeconds(1)); for (TableBucket bucket : scanRecords.buckets()) { - for (ScanRecord record : scanRecords.records(bucket)) { + for (ScanRecord record : scanRecords.records(bucket)) { RowData row = converter.toFlinkRowData(record.getRow()); row.setRowKind(toFlinkRowKind(record.getChangeType())); rows.add(row); @@ -288,9 +288,9 @@ public void testOrdersTablePKSink() throws Exception { List rows = new ArrayList<>(); while (rows.size() < orders.size()) { - ScanRecords scanRecords = logScanner.poll(Duration.ofSeconds(1)); + ScanRecords scanRecords = logScanner.poll(Duration.ofSeconds(1)); for (TableBucket bucket : scanRecords.buckets()) { - for (ScanRecord record : scanRecords.records(bucket)) { + for (ScanRecord record : scanRecords.records(bucket)) { InternalRow row = record.getRow(); TestOrder order = new TestOrder( diff --git a/fluss-flink/fluss-flink-common/src/test/java/org/apache/fluss/flink/source/deserializer/FlussDeserializationSchemaTest.java b/fluss-flink/fluss-flink-common/src/test/java/org/apache/fluss/flink/source/deserializer/FlussDeserializationSchemaTest.java index 8a5f3b81d5..001982969e 100644 --- a/fluss-flink/fluss-flink-common/src/test/java/org/apache/fluss/flink/source/deserializer/FlussDeserializationSchemaTest.java +++ b/fluss-flink/fluss-flink-common/src/test/java/org/apache/fluss/flink/source/deserializer/FlussDeserializationSchemaTest.java @@ -21,9 +21,11 @@ import org.apache.fluss.flink.source.testutils.Order; import org.apache.fluss.flink.source.testutils.OrderDeserializationSchema; import org.apache.fluss.record.ChangeType; +import org.apache.fluss.record.LogRecord; import org.apache.fluss.row.BinaryString; import org.apache.fluss.row.Decimal; import org.apache.fluss.row.GenericRow; +import org.apache.fluss.row.InternalRow; import org.apache.fluss.row.TimestampLtz; import org.apache.fluss.row.TimestampNtz; import org.apache.fluss.types.DataField; @@ -50,6 +52,36 @@ * conversion from Fluss records to various target formats. */ public class FlussDeserializationSchemaTest { + + /** Lightweight adapter to view a {@code ScanRecord} as a {@link LogRecord}. */ + private static final class ScanRecordLogRecord implements LogRecord { + private final ScanRecord delegate; + + private ScanRecordLogRecord(ScanRecord delegate) { + this.delegate = delegate; + } + + @Override + public long logOffset() { + return delegate.logOffset(); + } + + @Override + public long timestamp() { + return delegate.timestamp(); + } + + @Override + public ChangeType getChangeType() { + return delegate.getChangeType(); + } + + @Override + public InternalRow getRow() { + return delegate.getRow(); + } + } + @Test public void testDeserialize() throws Exception { // Create GenericRow with proper types @@ -59,10 +91,10 @@ public void testDeserialize() throws Exception { row.setField(2, 3); row.setField(3, BinaryString.fromString("123 Main St")); - ScanRecord scanRecord = new ScanRecord(row); + ScanRecord scanRecord = new ScanRecord<>(row); OrderDeserializationSchema deserializer = new OrderDeserializationSchema(); - Order result = deserializer.deserialize(scanRecord); + Order result = deserializer.deserialize(new ScanRecordLogRecord(scanRecord)); assertThat(result.getOrderId()).isEqualTo(1001L); assertThat(result.getItemId()).isEqualTo(5001L); @@ -78,10 +110,10 @@ public void testDeserializeWithNumericConversion() throws Exception { row.setField(2, 4); row.setField(3, BinaryString.fromString("456 Oak Ave")); - ScanRecord scanRecord = new ScanRecord(row); + ScanRecord scanRecord = new ScanRecord<>(row); OrderDeserializationSchema schema = new OrderDeserializationSchema(); - Order result = schema.deserialize(scanRecord); + Order result = schema.deserialize(new ScanRecordLogRecord(scanRecord)); assertThat(result.getOrderId()).isEqualTo(1002L); assertThat(result.getItemId()).isEqualTo(5002L); @@ -97,10 +129,10 @@ public void testDeserializeWithNullValues() throws Exception { row.setField(2, 5); row.setField(3, null); - ScanRecord scanRecord = new ScanRecord(row); + ScanRecord scanRecord = new ScanRecord<>(row); OrderDeserializationSchema schema = new OrderDeserializationSchema(); - Order result = schema.deserialize(scanRecord); + Order result = schema.deserialize(new ScanRecordLogRecord(scanRecord)); assertThat(result.getOrderId()).isEqualTo(1003L); assertThat(result.getItemId()).isEqualTo(5003L); @@ -176,13 +208,13 @@ public void testJsonStringDeserialize() throws Exception { row.setField(14, TimestampNtz.fromMillis(testTimestampInSeconds * 1000)); row.setField(15, TimestampLtz.fromEpochMillis(testTimestampInSeconds * 1000)); row.setField(16, null); - ScanRecord scanRecord = new ScanRecord(row); + ScanRecord scanRecord = new ScanRecord<>(row); // Create deserializer JsonStringDeserializationSchema deserializer = new JsonStringDeserializationSchema(); // Test deserialization - deserializer.open(new DeserializerInitContextImpl(null, null, sourceRowType)); - String result = deserializer.deserialize(scanRecord); + deserializer.open(new DeserializerInitContextImpl(null, null, rowType)); + String result = deserializer.deserialize(new ScanRecordLogRecord(scanRecord)); String rowJson = "{" @@ -213,8 +245,9 @@ public void testJsonStringDeserialize() throws Exception { + "}"); // Verify with offset and timestamp - ScanRecord scanRecord2 = new ScanRecord(1001, 1743261788400L, ChangeType.DELETE, row); - String result2 = deserializer.deserialize(scanRecord2); + ScanRecord scanRecord2 = + new ScanRecord<>(1001, 1743261788400L, ChangeType.DELETE, row); + String result2 = deserializer.deserialize(new ScanRecordLogRecord(scanRecord2)); assertThat(result2).isNotNull(); assertThat(result2) .isEqualTo( @@ -227,8 +260,8 @@ public void testJsonStringDeserialize() throws Exception { row.setField(2, true); row.setField(8, 512); row.setField(13, 72000000); - ScanRecord changedRecord = new ScanRecord(row); - String changedResult = deserializer.deserialize(changedRecord); + ScanRecord changedRecord = new ScanRecord<>(row); + String changedResult = deserializer.deserialize(new ScanRecordLogRecord(changedRecord)); String changedRowJson = "{" + "\"char\":\"b\"," diff --git a/fluss-flink/fluss-flink-common/src/test/java/org/apache/fluss/flink/source/deserializer/RowDataDeserializationSchemaTest.java b/fluss-flink/fluss-flink-common/src/test/java/org/apache/fluss/flink/source/deserializer/RowDataDeserializationSchemaTest.java index 8abbbac8bb..146031c2ff 100644 --- a/fluss-flink/fluss-flink-common/src/test/java/org/apache/fluss/flink/source/deserializer/RowDataDeserializationSchemaTest.java +++ b/fluss-flink/fluss-flink-common/src/test/java/org/apache/fluss/flink/source/deserializer/RowDataDeserializationSchemaTest.java @@ -18,8 +18,11 @@ package org.apache.fluss.flink.source.deserializer; import org.apache.fluss.client.table.scanner.ScanRecord; +import org.apache.fluss.record.ChangeType; +import org.apache.fluss.record.LogRecord; import org.apache.fluss.row.BinaryString; import org.apache.fluss.row.GenericRow; +import org.apache.fluss.row.InternalRow; import org.apache.fluss.types.DataField; import org.apache.fluss.types.DataTypes; import org.apache.fluss.types.RowType; @@ -77,10 +80,10 @@ public void testDeserialize() throws Exception { row.setField(2, 45); row.setField(3, BinaryString.fromString("Test addr")); - ScanRecord scanRecord = new ScanRecord(row); + ScanRecord scanRecord = new ScanRecord<>(row); RowDataDeserializationSchema deserializer = getRowDataDeserializationSchema(rowType); - RowData result = deserializer.deserialize(scanRecord); + RowData result = deserializer.deserialize(new ScanRecordLogRecord(scanRecord)); assertThat(result.getArity()).isEqualTo(4); assertThat(result.getLong(0)).isEqualTo(100L); @@ -89,6 +92,35 @@ public void testDeserialize() throws Exception { assertThat(result.getString(3).toString()).isEqualTo("Test addr"); } + /** Lightweight adapter to view a {@code ScanRecord} as a {@link LogRecord}. */ + private static final class ScanRecordLogRecord implements LogRecord { + private final ScanRecord delegate; + + private ScanRecordLogRecord(ScanRecord delegate) { + this.delegate = delegate; + } + + @Override + public long logOffset() { + return delegate.logOffset(); + } + + @Override + public long timestamp() { + return delegate.timestamp(); + } + + @Override + public ChangeType getChangeType() { + return delegate.getChangeType(); + } + + @Override + public InternalRow getRow() { + return delegate.getRow(); + } + } + private @NotNull RowDataDeserializationSchema getRowDataDeserializationSchema(RowType rowType) throws Exception { RowDataDeserializationSchema deserializationSchema = new RowDataDeserializationSchema(); diff --git a/fluss-flink/fluss-flink-common/src/test/java/org/apache/fluss/flink/utils/FlussRowToFlinkRowConverterTest.java b/fluss-flink/fluss-flink-common/src/test/java/org/apache/fluss/flink/utils/FlussRowToFlinkRowConverterTest.java index 0c7515f3ae..0475c8e00c 100644 --- a/fluss-flink/fluss-flink-common/src/test/java/org/apache/fluss/flink/utils/FlussRowToFlinkRowConverterTest.java +++ b/fluss-flink/fluss-flink-common/src/test/java/org/apache/fluss/flink/utils/FlussRowToFlinkRowConverterTest.java @@ -20,6 +20,8 @@ import org.apache.fluss.client.table.scanner.ScanRecord; import org.apache.fluss.flink.row.FlinkAsFlussArray; import org.apache.fluss.record.ChangeType; +import org.apache.fluss.record.LogRecord; +import org.apache.fluss.row.InternalRow; import org.apache.fluss.row.BinaryString; import org.apache.fluss.row.indexed.IndexedRow; import org.apache.fluss.row.indexed.IndexedRowWriter; @@ -126,4 +128,33 @@ void testConverter() throws Exception { .isEqualTo(new BinaryString[] {fromString("hello"), fromString("world")}); } } + + /** Lightweight adapter to view a {@code ScanRecord} as a {@link LogRecord}. */ + private static final class ScanRecordLogRecord implements LogRecord { + private final ScanRecord delegate; + + private ScanRecordLogRecord(ScanRecord delegate) { + this.delegate = delegate; + } + + @Override + public long logOffset() { + return delegate.logOffset(); + } + + @Override + public long timestamp() { + return delegate.timestamp(); + } + + @Override + public ChangeType getChangeType() { + return delegate.getChangeType(); + } + + @Override + public InternalRow getRow() { + return delegate.getRow(); + } + } } From eabb51da2e5bb8f6ff9308af359bf7c52f18deee Mon Sep 17 00:00:00 2001 From: ipolyzos Date: Mon, 17 Nov 2025 19:13:16 +0200 Subject: [PATCH 10/33] improve test coverage --- .../client/converter/ConverterCommons.java | 3 +- .../source/ScanRecordLogRecordTest.java | 66 +++++++++++++++++++ 2 files changed, 68 insertions(+), 1 deletion(-) create mode 100644 fluss-flink/fluss-flink-common/src/test/java/org/apache/fluss/flink/tiering/source/ScanRecordLogRecordTest.java diff --git a/fluss-client/src/main/java/org/apache/fluss/client/converter/ConverterCommons.java b/fluss-client/src/main/java/org/apache/fluss/client/converter/ConverterCommons.java index 700456edae..aeb36419a6 100644 --- a/fluss-client/src/main/java/org/apache/fluss/client/converter/ConverterCommons.java +++ b/fluss-client/src/main/java/org/apache/fluss/client/converter/ConverterCommons.java @@ -89,7 +89,8 @@ static void validatePojoMatchesProjection(PojoType pojoType, RowType projecti if (!pojoNames.containsAll(fieldNames)) { throw new IllegalArgumentException( String.format( - "POJO fields %s must contain all projection fields %s.", + "POJO fields %s must contain all projection fields %s. " + + "For full-table writes, POJO fields must exactly match table schema fields.", pojoNames, fieldNames)); } for (int i = 0; i < projection.getFieldCount(); i++) { diff --git a/fluss-flink/fluss-flink-common/src/test/java/org/apache/fluss/flink/tiering/source/ScanRecordLogRecordTest.java b/fluss-flink/fluss-flink-common/src/test/java/org/apache/fluss/flink/tiering/source/ScanRecordLogRecordTest.java new file mode 100644 index 0000000000..7524e9e78d --- /dev/null +++ b/fluss-flink/fluss-flink-common/src/test/java/org/apache/fluss/flink/tiering/source/ScanRecordLogRecordTest.java @@ -0,0 +1,66 @@ +/* + * 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.tiering.source; + +import org.apache.fluss.client.table.scanner.ScanRecord; +import org.apache.fluss.record.ChangeType; +import org.apache.fluss.record.LogRecord; +import org.apache.fluss.row.GenericRow; +import org.apache.fluss.row.InternalRow; + +import org.junit.jupiter.api.Test; + +import java.lang.reflect.Constructor; + +import static org.assertj.core.api.Assertions.assertThat; + +/** Tests for the private adapter TieringSplitReader.ScanRecordLogRecord */ +class ScanRecordLogRecordTest { + + @Test + void testAdapterDelegatesAllMethods() throws Exception { + // Prepare a simple InternalRow and wrap it in a ScanRecord with known attributes + GenericRow row = new GenericRow(1); + row.setField(0, 42); // content is irrelevant for this test + + long offset = 1234L; + long timestamp = 987654321L; + ChangeType changeType = ChangeType.UPDATE_AFTER; + ScanRecord scanRecord = new ScanRecord<>(offset, timestamp, changeType, row); + + // Reflectively construct the private static inner class + Class clazz = + Class.forName( + "org.apache.fluss.flink.tiering.source.TieringSplitReader$ScanRecordLogRecord"); + @SuppressWarnings("unchecked") + Constructor ctor = + (Constructor) clazz.getDeclaredConstructor(ScanRecord.class); + ctor.setAccessible(true); + LogRecord adapter = ctor.newInstance(scanRecord); + + // Verify delegation + assertThat(adapter.logOffset()).isEqualTo(offset); + assertThat(adapter.timestamp()).isEqualTo(timestamp); + assertThat(adapter.getChangeType()).isEqualTo(changeType); + assertThat(adapter.getRow()).isSameAs(row); + + // Call twice to make sure repeated invocations are fine and to bump coverage + assertThat(adapter.logOffset()).isEqualTo(offset); + assertThat(adapter.getRow()).isSameAs(row); + } +} From 367461d1f3b8a87108c5850a8e7581b4131abbea Mon Sep 17 00:00:00 2001 From: ipolyzos Date: Mon, 17 Nov 2025 19:17:54 +0200 Subject: [PATCH 11/33] fix checkstyle violation --- .../fluss/flink/tiering/source/ScanRecordLogRecordTest.java | 3 +-- 1 file changed, 1 insertion(+), 2 deletions(-) diff --git a/fluss-flink/fluss-flink-common/src/test/java/org/apache/fluss/flink/tiering/source/ScanRecordLogRecordTest.java b/fluss-flink/fluss-flink-common/src/test/java/org/apache/fluss/flink/tiering/source/ScanRecordLogRecordTest.java index 7524e9e78d..d4cc961d11 100644 --- a/fluss-flink/fluss-flink-common/src/test/java/org/apache/fluss/flink/tiering/source/ScanRecordLogRecordTest.java +++ b/fluss-flink/fluss-flink-common/src/test/java/org/apache/fluss/flink/tiering/source/ScanRecordLogRecordTest.java @@ -29,7 +29,7 @@ import static org.assertj.core.api.Assertions.assertThat; -/** Tests for the private adapter TieringSplitReader.ScanRecordLogRecord */ +/** Tests for the private adapter TieringSplitReader.ScanRecordLogRecord. */ class ScanRecordLogRecordTest { @Test @@ -59,7 +59,6 @@ void testAdapterDelegatesAllMethods() throws Exception { assertThat(adapter.getChangeType()).isEqualTo(changeType); assertThat(adapter.getRow()).isSameAs(row); - // Call twice to make sure repeated invocations are fine and to bump coverage assertThat(adapter.logOffset()).isEqualTo(offset); assertThat(adapter.getRow()).isSameAs(row); } From e06df33cbe2b52dd345cb438936331728cb402d7 Mon Sep 17 00:00:00 2001 From: ipolyzos Date: Tue, 18 Nov 2025 09:11:45 +0200 Subject: [PATCH 12/33] instantiate converters once --- .../client/converter/PojoToRowConverter.java | 2 - .../fluss/client/converter/PojoType.java | 1 - .../table/writer/TypedAppendWriter.java | 7 ++-- .../table/writer/TypedUpsertWriter.java | 37 +++++++++++++++---- 4 files changed, 33 insertions(+), 14 deletions(-) diff --git a/fluss-client/src/main/java/org/apache/fluss/client/converter/PojoToRowConverter.java b/fluss-client/src/main/java/org/apache/fluss/client/converter/PojoToRowConverter.java index bd5a580662..0f4f6d8a4e 100644 --- a/fluss-client/src/main/java/org/apache/fluss/client/converter/PojoToRowConverter.java +++ b/fluss-client/src/main/java/org/apache/fluss/client/converter/PojoToRowConverter.java @@ -188,8 +188,6 @@ private static FieldToRow createFieldConverter(PojoType.Property prop, DataType BigDecimal bd = (BigDecimal) v; BigDecimal scaled = bd.setScale(scale, RoundingMode.HALF_UP); - // Validate precision after scaling; precision is the number of digits in the unscaled - // value. if (scaled.precision() > precision) { throw new IllegalArgumentException( String.format( diff --git a/fluss-client/src/main/java/org/apache/fluss/client/converter/PojoType.java b/fluss-client/src/main/java/org/apache/fluss/client/converter/PojoType.java index 9253b4f835..ea098070d4 100644 --- a/fluss-client/src/main/java/org/apache/fluss/client/converter/PojoType.java +++ b/fluss-client/src/main/java/org/apache/fluss/client/converter/PojoType.java @@ -37,7 +37,6 @@ final class PojoType { private final Class pojoClass; private final Constructor defaultConstructor; private final Map properties; // property name -> property - // Mapping of primitive types to their boxed counterparts to avoid long if-chains private static final Map, Class> PRIMITIVE_TO_BOXED = createPrimitiveToBoxedMap(); private PojoType(Class pojoClass, Constructor ctor, Map props) { diff --git a/fluss-client/src/main/java/org/apache/fluss/client/table/writer/TypedAppendWriter.java b/fluss-client/src/main/java/org/apache/fluss/client/table/writer/TypedAppendWriter.java index 256305ab1b..d02ec583a2 100644 --- a/fluss-client/src/main/java/org/apache/fluss/client/table/writer/TypedAppendWriter.java +++ b/fluss-client/src/main/java/org/apache/fluss/client/table/writer/TypedAppendWriter.java @@ -33,11 +33,13 @@ class TypedAppendWriter implements AppendWriter { private final AppendWriterImpl delegate; private final Class pojoClass; private final RowType tableSchema; + private final PojoToRowConverter pojoToRowConverter; TypedAppendWriter(AppendWriterImpl delegate, Class pojoClass, TableInfo tableInfo) { this.delegate = delegate; this.pojoClass = pojoClass; this.tableSchema = tableInfo.getRowType(); + this.pojoToRowConverter = PojoToRowConverter.of(pojoClass, tableSchema, tableSchema); } @Override @@ -50,10 +52,7 @@ public CompletableFuture append(T record) { if (record instanceof InternalRow) { return delegate.append((InternalRow) record); } - // TODO: initialize this on the constructor and reuse - PojoToRowConverter converter = - PojoToRowConverter.of(pojoClass, tableSchema, tableSchema); - InternalRow row = converter.toRow(record); + InternalRow row = pojoToRowConverter.toRow(record); return delegate.append(row); } } diff --git a/fluss-client/src/main/java/org/apache/fluss/client/table/writer/TypedUpsertWriter.java b/fluss-client/src/main/java/org/apache/fluss/client/table/writer/TypedUpsertWriter.java index 68976f8e29..f240c6b27c 100644 --- a/fluss-client/src/main/java/org/apache/fluss/client/table/writer/TypedUpsertWriter.java +++ b/fluss-client/src/main/java/org/apache/fluss/client/table/writer/TypedUpsertWriter.java @@ -23,6 +23,8 @@ import org.apache.fluss.row.InternalRow; import org.apache.fluss.types.RowType; +import javax.annotation.Nullable; + import java.util.concurrent.CompletableFuture; /** @@ -42,6 +44,13 @@ public void flush() { private final RowType tableSchema; private final int[] targetColumns; // may be null + private final RowType pkProjection; + @Nullable private final RowType targetProjection; + + private final PojoToRowConverter pojoToRowConverter; + private final PojoToRowConverter pkConverter; + @Nullable private final PojoToRowConverter targetConverter; + TypedUpsertWriter( UpsertWriterImpl delegate, Class pojoClass, @@ -52,6 +61,19 @@ public void flush() { this.tableInfo = tableInfo; this.tableSchema = tableInfo.getRowType(); this.targetColumns = targetColumns; + + // Precompute projections + this.pkProjection = this.tableSchema.project(tableInfo.getPhysicalPrimaryKeys()); + this.targetProjection = + (targetColumns == null) ? null : this.tableSchema.project(targetColumns); + + // Initialize reusable converters + this.pojoToRowConverter = PojoToRowConverter.of(pojoClass, tableSchema, tableSchema); + this.pkConverter = PojoToRowConverter.of(pojoClass, tableSchema, pkProjection); + this.targetConverter = + (targetProjection == null) + ? null + : PojoToRowConverter.of(pojoClass, tableSchema, targetProjection); } @Override @@ -73,18 +95,19 @@ public CompletableFuture delete(T record) { } private InternalRow convertPojo(T pojo, boolean forDelete) { - RowType projection; + final RowType projection; + final PojoToRowConverter converter; if (forDelete) { - // for delete we only need primary key columns - projection = tableSchema.project(tableInfo.getPhysicalPrimaryKeys()); - } else if (targetColumns != null) { - projection = tableSchema.project(targetColumns); + projection = pkProjection; + converter = pkConverter; + } else if (targetProjection != null && targetConverter != null) { + projection = targetProjection; + converter = targetConverter; } else { projection = tableSchema; + converter = pojoToRowConverter; } - // TODO: initialize this on the constructor and reuse - PojoToRowConverter converter = PojoToRowConverter.of(pojoClass, tableSchema, projection); GenericRow projected = converter.toRow(pojo); if (projection == tableSchema) { return projected; From 040fa903e56b90045b361e70726f4c845fa2ba52 Mon Sep 17 00:00:00 2001 From: ipolyzos Date: Mon, 17 Nov 2025 16:40:51 +0200 Subject: [PATCH 13/33] Introduce Generics and Typed Classes --- .../apache/fluss/client/lookup/Lookup.java | 8 +-- .../apache/fluss/client/lookup/Lookuper.java | 16 +++--- .../fluss/client/lookup/TableLookup.java | 2 +- .../fluss/client/lookup/TypedLookuper.java | 15 +++--- .../fluss/client/table/scanner/Scan.java | 2 +- .../client/table/scanner/ScanRecord.java | 2 +- .../fluss/client/table/scanner/TableScan.java | 2 +- .../table/scanner/log/CompletedFetch.java | 2 +- .../table/scanner/log/LogFetchCollector.java | 13 ++--- .../table/scanner/log/LogScannerImpl.java | 2 +- .../client/table/scanner/log/ScanRecords.java | 6 +-- .../table/scanner/log/TypedLogScanner.java | 17 +++---- .../fluss/client/table/writer/Append.java | 2 +- .../client/table/writer/AppendWriter.java | 2 +- .../client/table/writer/TableAppend.java | 2 +- .../client/table/writer/TableUpsert.java | 5 +- .../table/writer/TypedAppendWriter.java | 8 +-- .../table/writer/TypedUpsertWriter.java | 49 +++++-------------- .../fluss/client/table/writer/Upsert.java | 7 +-- .../client/table/writer/UpsertWriter.java | 2 +- 20 files changed, 64 insertions(+), 100 deletions(-) diff --git a/fluss-client/src/main/java/org/apache/fluss/client/lookup/Lookup.java b/fluss-client/src/main/java/org/apache/fluss/client/lookup/Lookup.java index 5411a0c91d..61d976b4b2 100644 --- a/fluss-client/src/main/java/org/apache/fluss/client/lookup/Lookup.java +++ b/fluss-client/src/main/java/org/apache/fluss/client/lookup/Lookup.java @@ -99,11 +99,11 @@ default Lookup lookupBy(String... lookupColumnNames) { * lookup columns. By default, the lookup columns are the primary key columns, but can be * changed with ({@link #lookupBy(List)}) method. * - *

The returned lookuper accepts generic keys of type {@code K}. If a key is a POJO, the - * client implementation will convert it to an {@code InternalRow} based on the table schema and - * the active lookup columns. + *

The returned lookuper accepts generic keys of type {@code K}. If a key is a POJO, + * the client implementation will convert it to an {@code InternalRow} based on the table + * schema and the active lookup columns. * * @return the lookuper */ Lookuper createLookuper(); -} +} \ No newline at end of file diff --git a/fluss-client/src/main/java/org/apache/fluss/client/lookup/Lookuper.java b/fluss-client/src/main/java/org/apache/fluss/client/lookup/Lookuper.java index b70adf8a4f..651f9b5590 100644 --- a/fluss-client/src/main/java/org/apache/fluss/client/lookup/Lookuper.java +++ b/fluss-client/src/main/java/org/apache/fluss/client/lookup/Lookuper.java @@ -18,6 +18,7 @@ package org.apache.fluss.client.lookup; import org.apache.fluss.annotation.PublicEvolving; +import org.apache.fluss.row.InternalRow; import javax.annotation.concurrent.NotThreadSafe; import java.util.concurrent.CompletableFuture; @@ -26,12 +27,13 @@ * A lookuper performs key-based lookups against a primary key table, using either the full primary * key or a prefix of the primary key (when configured via {@code Lookup#lookupBy}). * - *

This interface is generic on the key type {@code K}: - When used in row mode, implementations - * are typically declared as {@code Lookuper} and accept an {@link - * org.apache.fluss.row.InternalRow} containing the key fields in the configured order. - When used - * with POJO keys, client-provided lookuper instances can also accept a POJO key type (for example - * {@code Lookuper}) and will transparently convert the POJO to an {@code InternalRow} - * using the table schema and active lookup columns. + *

This interface is generic on the key type {@code K}: + * - When used in row mode, implementations are typically declared as {@code Lookuper} + * and accept an {@link org.apache.fluss.row.InternalRow} containing the key fields in the + * configured order. + * - When used with POJO keys, client-provided lookuper instances can also accept a POJO key type + * (for example {@code Lookuper}) and will transparently convert the POJO to an + * {@code InternalRow} using the table schema and active lookup columns. * *

Usage examples: * @@ -66,4 +68,4 @@ public interface Lookuper { * @return the result of lookup. */ CompletableFuture lookup(K lookupKey); -} +} \ No newline at end of file diff --git a/fluss-client/src/main/java/org/apache/fluss/client/lookup/TableLookup.java b/fluss-client/src/main/java/org/apache/fluss/client/lookup/TableLookup.java index 975196801f..a4eb3cb28d 100644 --- a/fluss-client/src/main/java/org/apache/fluss/client/lookup/TableLookup.java +++ b/fluss-client/src/main/java/org/apache/fluss/client/lookup/TableLookup.java @@ -74,4 +74,4 @@ public Lookuper createLookuper() { } return new TypedLookuper(lookuper, tableInfo, lookupColumnNames); } -} +} \ No newline at end of file diff --git a/fluss-client/src/main/java/org/apache/fluss/client/lookup/TypedLookuper.java b/fluss-client/src/main/java/org/apache/fluss/client/lookup/TypedLookuper.java index bba49366b8..b4fb3518e1 100644 --- a/fluss-client/src/main/java/org/apache/fluss/client/lookup/TypedLookuper.java +++ b/fluss-client/src/main/java/org/apache/fluss/client/lookup/TypedLookuper.java @@ -23,15 +23,13 @@ import org.apache.fluss.types.RowType; import javax.annotation.Nullable; - import java.util.List; import java.util.concurrent.CompletableFuture; /** - * Decorator for {@link Lookuper} that enables generic key lookup via {@link - * Lookuper#lookup(Object)}. Converts POJO keys to {@link InternalRow} using existing converters - * based on table schema and active lookup columns, and directly delegates when the key is already - * an {@link InternalRow}. + * Decorator for {@link Lookuper} that enables generic key lookup via {@link Lookuper#lookup(Object)}. + * Converts POJO keys to {@link InternalRow} using existing converters based on table schema and + * active lookup columns, and directly delegates when the key is already an {@link InternalRow}. */ final class TypedLookuper implements Lookuper { @@ -39,10 +37,9 @@ final class TypedLookuper implements Lookuper { private final TableInfo tableInfo; @Nullable private final List lookupColumnNames; - TypedLookuper( - Lookuper delegate, - TableInfo tableInfo, - @Nullable List lookupColumnNames) { + TypedLookuper(Lookuper delegate, + TableInfo tableInfo, + @Nullable List lookupColumnNames) { this.delegate = delegate; this.tableInfo = tableInfo; this.lookupColumnNames = lookupColumnNames; diff --git a/fluss-client/src/main/java/org/apache/fluss/client/table/scanner/Scan.java b/fluss-client/src/main/java/org/apache/fluss/client/table/scanner/Scan.java index c8d47a9796..e15e3894b7 100644 --- a/fluss-client/src/main/java/org/apache/fluss/client/table/scanner/Scan.java +++ b/fluss-client/src/main/java/org/apache/fluss/client/table/scanner/Scan.java @@ -88,4 +88,4 @@ public interface Scan { * #limit(int)} and only support for Primary Key Tables. */ BatchScanner createBatchScanner(TableBucket tableBucket, long snapshotId); -} +} \ No newline at end of file diff --git a/fluss-client/src/main/java/org/apache/fluss/client/table/scanner/ScanRecord.java b/fluss-client/src/main/java/org/apache/fluss/client/table/scanner/ScanRecord.java index 0f96e06abf..074875b9c8 100644 --- a/fluss-client/src/main/java/org/apache/fluss/client/table/scanner/ScanRecord.java +++ b/fluss-client/src/main/java/org/apache/fluss/client/table/scanner/ScanRecord.java @@ -89,4 +89,4 @@ public int hashCode() { public String toString() { return changeType.shortString() + value + "@" + offset; } -} +} \ No newline at end of file diff --git a/fluss-client/src/main/java/org/apache/fluss/client/table/scanner/TableScan.java b/fluss-client/src/main/java/org/apache/fluss/client/table/scanner/TableScan.java index ac530ab2bb..93afcf58f3 100644 --- a/fluss-client/src/main/java/org/apache/fluss/client/table/scanner/TableScan.java +++ b/fluss-client/src/main/java/org/apache/fluss/client/table/scanner/TableScan.java @@ -171,4 +171,4 @@ public BatchScanner createBatchScanner(TableBucket tableBucket, long snapshotId) tableInfo.getTableConfig().getKvFormat(), conn.getOrCreateRemoteFileDownloader()); } -} +} \ No newline at end of file diff --git a/fluss-client/src/main/java/org/apache/fluss/client/table/scanner/log/CompletedFetch.java b/fluss-client/src/main/java/org/apache/fluss/client/table/scanner/log/CompletedFetch.java index a4c0ee4e90..88780b9ab6 100644 --- a/fluss-client/src/main/java/org/apache/fluss/client/table/scanner/log/CompletedFetch.java +++ b/fluss-client/src/main/java/org/apache/fluss/client/table/scanner/log/CompletedFetch.java @@ -255,4 +255,4 @@ private void maybeCloseRecordStream() { records = null; } } -} +} \ No newline at end of file diff --git a/fluss-client/src/main/java/org/apache/fluss/client/table/scanner/log/LogFetchCollector.java b/fluss-client/src/main/java/org/apache/fluss/client/table/scanner/log/LogFetchCollector.java index 87e738fc58..daf3cf409f 100644 --- a/fluss-client/src/main/java/org/apache/fluss/client/table/scanner/log/LogFetchCollector.java +++ b/fluss-client/src/main/java/org/apache/fluss/client/table/scanner/log/LogFetchCollector.java @@ -20,6 +20,7 @@ import org.apache.fluss.annotation.Internal; import org.apache.fluss.client.metadata.MetadataUpdater; import org.apache.fluss.client.table.scanner.ScanRecord; +import org.apache.fluss.row.InternalRow; import org.apache.fluss.config.ConfigOptions; import org.apache.fluss.config.Configuration; import org.apache.fluss.exception.AuthorizationException; @@ -29,7 +30,6 @@ import org.apache.fluss.metadata.TablePath; import org.apache.fluss.record.LogRecord; import org.apache.fluss.record.LogRecordBatch; -import org.apache.fluss.row.InternalRow; import org.apache.fluss.rpc.protocol.ApiError; import org.apache.fluss.rpc.protocol.Errors; @@ -84,8 +84,7 @@ public LogFetchCollector( * @throws LogOffsetOutOfRangeException If there is OffsetOutOfRange error in fetchResponse and * the defaultResetPolicy is NONE */ - public Map>> collectFetch( - final LogFetchBuffer logFetchBuffer) { + public Map>> collectFetch(final LogFetchBuffer logFetchBuffer) { Map>> fetched = new HashMap<>(); int recordsRemaining = maxPollRecords; @@ -117,8 +116,7 @@ public Map>> collectFetch( logFetchBuffer.poll(); } else { - List> records = - fetchRecords(nextInLineFetch, recordsRemaining); + List> records = fetchRecords(nextInLineFetch, recordsRemaining); if (!records.isEmpty()) { TableBucket tableBucket = nextInLineFetch.tableBucket; List> currentRecords = fetched.get(tableBucket); @@ -149,8 +147,7 @@ public Map>> collectFetch( return fetched; } - private List> fetchRecords( - CompletedFetch nextInLineFetch, int maxRecords) { + private List> fetchRecords(CompletedFetch nextInLineFetch, int maxRecords) { TableBucket tb = nextInLineFetch.tableBucket; Long offset = logScannerStatus.getBucketOffset(tb); if (offset == null) { @@ -293,4 +290,4 @@ private void handleInitializeErrors( error, fetchOffset, tb, error.exception(errorMessage))); } } -} +} \ No newline at end of file diff --git a/fluss-client/src/main/java/org/apache/fluss/client/table/scanner/log/LogScannerImpl.java b/fluss-client/src/main/java/org/apache/fluss/client/table/scanner/log/LogScannerImpl.java index 491773a7c8..dbd89925a6 100644 --- a/fluss-client/src/main/java/org/apache/fluss/client/table/scanner/log/LogScannerImpl.java +++ b/fluss-client/src/main/java/org/apache/fluss/client/table/scanner/log/LogScannerImpl.java @@ -306,4 +306,4 @@ public void close() { release(); } } -} +} \ No newline at end of file diff --git a/fluss-client/src/main/java/org/apache/fluss/client/table/scanner/log/ScanRecords.java b/fluss-client/src/main/java/org/apache/fluss/client/table/scanner/log/ScanRecords.java index 1473954c51..ea38f6adaf 100644 --- a/fluss-client/src/main/java/org/apache/fluss/client/table/scanner/log/ScanRecords.java +++ b/fluss-client/src/main/java/org/apache/fluss/client/table/scanner/log/ScanRecords.java @@ -37,9 +37,7 @@ */ @PublicEvolving public class ScanRecords implements Iterable> { - public static final ScanRecords empty() { - return new ScanRecords<>(Collections.emptyMap()); - } + public static final ScanRecords empty() { return new ScanRecords<>(Collections.emptyMap()); } private final Map>> records; @@ -115,4 +113,4 @@ public ScanRecord makeNext() { }; } } -} +} \ No newline at end of file diff --git a/fluss-client/src/main/java/org/apache/fluss/client/table/scanner/log/TypedLogScanner.java b/fluss-client/src/main/java/org/apache/fluss/client/table/scanner/log/TypedLogScanner.java index f8b7cdb857..a13ded7fe9 100644 --- a/fluss-client/src/main/java/org/apache/fluss/client/table/scanner/log/TypedLogScanner.java +++ b/fluss-client/src/main/java/org/apache/fluss/client/table/scanner/log/TypedLogScanner.java @@ -38,15 +38,13 @@ public class TypedLogScanner implements LogScanner { private final LogScanner delegate; private final RowToPojoConverter converter; - public TypedLogScanner( - LogScanner delegate, - Class pojoClass, - TableInfo tableInfo, - int[] projectedColumns) { + public TypedLogScanner(LogScanner delegate, + Class pojoClass, + TableInfo tableInfo, + int[] projectedColumns) { this.delegate = delegate; RowType tableSchema = tableInfo.getRowType(); - RowType projection = - projectedColumns == null ? tableSchema : tableSchema.project(projectedColumns); + RowType projection = projectedColumns == null ? tableSchema : tableSchema.project(projectedColumns); this.converter = RowToPojoConverter.of(pojoClass, tableSchema, projection); } @@ -63,8 +61,7 @@ public ScanRecords poll(Duration timeout) { for (ScanRecord r : list) { InternalRow row = r.getValue(); T pojo = converter.fromRow(row); - converted.add( - new ScanRecord<>(r.logOffset(), r.timestamp(), r.getChangeType(), pojo)); + converted.add(new ScanRecord<>(r.logOffset(), r.timestamp(), r.getChangeType(), pojo)); } out.put(bucket, converted); } @@ -99,4 +96,4 @@ public void close() { throw new RuntimeException(e); } } -} +} \ No newline at end of file diff --git a/fluss-client/src/main/java/org/apache/fluss/client/table/writer/Append.java b/fluss-client/src/main/java/org/apache/fluss/client/table/writer/Append.java index 594ef7cbfe..24dd8a6845 100644 --- a/fluss-client/src/main/java/org/apache/fluss/client/table/writer/Append.java +++ b/fluss-client/src/main/java/org/apache/fluss/client/table/writer/Append.java @@ -37,4 +37,4 @@ public interface Append { /** Create a new typed {@link AppendWriter} to write POJOs directly. */ AppendWriter createWriter(Class pojoClass); -} +} \ No newline at end of file diff --git a/fluss-client/src/main/java/org/apache/fluss/client/table/writer/AppendWriter.java b/fluss-client/src/main/java/org/apache/fluss/client/table/writer/AppendWriter.java index 7223f2f639..ff0f62eaae 100644 --- a/fluss-client/src/main/java/org/apache/fluss/client/table/writer/AppendWriter.java +++ b/fluss-client/src/main/java/org/apache/fluss/client/table/writer/AppendWriter.java @@ -40,4 +40,4 @@ public interface AppendWriter extends TableWriter { * @return A {@link CompletableFuture} that always returns append result when complete normally. */ CompletableFuture append(T record); -} +} \ No newline at end of file diff --git a/fluss-client/src/main/java/org/apache/fluss/client/table/writer/TableAppend.java b/fluss-client/src/main/java/org/apache/fluss/client/table/writer/TableAppend.java index 975acdd969..5d3340f180 100644 --- a/fluss-client/src/main/java/org/apache/fluss/client/table/writer/TableAppend.java +++ b/fluss-client/src/main/java/org/apache/fluss/client/table/writer/TableAppend.java @@ -44,4 +44,4 @@ public AppendWriter createWriter(Class pojoClass) { AppendWriterImpl delegate = new AppendWriterImpl(tablePath, tableInfo, writerClient); return new TypedAppendWriter<>(delegate, pojoClass, tableInfo); } -} +} \ No newline at end of file diff --git a/fluss-client/src/main/java/org/apache/fluss/client/table/writer/TableUpsert.java b/fluss-client/src/main/java/org/apache/fluss/client/table/writer/TableUpsert.java index cd8d747848..d5e8921e04 100644 --- a/fluss-client/src/main/java/org/apache/fluss/client/table/writer/TableUpsert.java +++ b/fluss-client/src/main/java/org/apache/fluss/client/table/writer/TableUpsert.java @@ -98,8 +98,7 @@ public UpsertWriter createWriter() { @Override public UpsertWriter createWriter(Class pojoClass) { - UpsertWriterImpl delegate = - new UpsertWriterImpl(tablePath, tableInfo, targetColumns, writerClient); + UpsertWriterImpl delegate = new UpsertWriterImpl(tablePath, tableInfo, targetColumns, writerClient); return new TypedUpsertWriter<>(delegate, pojoClass, tableInfo, targetColumns); } -} +} \ No newline at end of file diff --git a/fluss-client/src/main/java/org/apache/fluss/client/table/writer/TypedAppendWriter.java b/fluss-client/src/main/java/org/apache/fluss/client/table/writer/TypedAppendWriter.java index d02ec583a2..67f4113c28 100644 --- a/fluss-client/src/main/java/org/apache/fluss/client/table/writer/TypedAppendWriter.java +++ b/fluss-client/src/main/java/org/apache/fluss/client/table/writer/TypedAppendWriter.java @@ -33,13 +33,11 @@ class TypedAppendWriter implements AppendWriter { private final AppendWriterImpl delegate; private final Class pojoClass; private final RowType tableSchema; - private final PojoToRowConverter pojoToRowConverter; TypedAppendWriter(AppendWriterImpl delegate, Class pojoClass, TableInfo tableInfo) { this.delegate = delegate; this.pojoClass = pojoClass; this.tableSchema = tableInfo.getRowType(); - this.pojoToRowConverter = PojoToRowConverter.of(pojoClass, tableSchema, tableSchema); } @Override @@ -52,7 +50,9 @@ public CompletableFuture append(T record) { if (record instanceof InternalRow) { return delegate.append((InternalRow) record); } - InternalRow row = pojoToRowConverter.toRow(record); + // TODO: initialize this on the constructor and reuse + PojoToRowConverter converter = PojoToRowConverter.of(pojoClass, tableSchema, tableSchema); + InternalRow row = converter.toRow(record); return delegate.append(row); } -} +} \ No newline at end of file diff --git a/fluss-client/src/main/java/org/apache/fluss/client/table/writer/TypedUpsertWriter.java b/fluss-client/src/main/java/org/apache/fluss/client/table/writer/TypedUpsertWriter.java index f240c6b27c..fe895b3bb4 100644 --- a/fluss-client/src/main/java/org/apache/fluss/client/table/writer/TypedUpsertWriter.java +++ b/fluss-client/src/main/java/org/apache/fluss/client/table/writer/TypedUpsertWriter.java @@ -23,8 +23,6 @@ import org.apache.fluss.row.InternalRow; import org.apache.fluss.types.RowType; -import javax.annotation.Nullable; - import java.util.concurrent.CompletableFuture; /** @@ -44,36 +42,12 @@ public void flush() { private final RowType tableSchema; private final int[] targetColumns; // may be null - private final RowType pkProjection; - @Nullable private final RowType targetProjection; - - private final PojoToRowConverter pojoToRowConverter; - private final PojoToRowConverter pkConverter; - @Nullable private final PojoToRowConverter targetConverter; - - TypedUpsertWriter( - UpsertWriterImpl delegate, - Class pojoClass, - TableInfo tableInfo, - int[] targetColumns) { + TypedUpsertWriter(UpsertWriterImpl delegate, Class pojoClass, TableInfo tableInfo, int[] targetColumns) { this.delegate = delegate; this.pojoClass = pojoClass; this.tableInfo = tableInfo; this.tableSchema = tableInfo.getRowType(); this.targetColumns = targetColumns; - - // Precompute projections - this.pkProjection = this.tableSchema.project(tableInfo.getPhysicalPrimaryKeys()); - this.targetProjection = - (targetColumns == null) ? null : this.tableSchema.project(targetColumns); - - // Initialize reusable converters - this.pojoToRowConverter = PojoToRowConverter.of(pojoClass, tableSchema, tableSchema); - this.pkConverter = PojoToRowConverter.of(pojoClass, tableSchema, pkProjection); - this.targetConverter = - (targetProjection == null) - ? null - : PojoToRowConverter.of(pojoClass, tableSchema, targetProjection); } @Override @@ -81,7 +55,7 @@ public CompletableFuture upsert(T record) { if (record instanceof InternalRow) { return delegate.upsert((InternalRow) record); } - InternalRow row = convertPojo(record, /*forDelete=*/ false); + InternalRow row = convertPojo(record, /*forDelete=*/false); return delegate.upsert(row); } @@ -90,24 +64,23 @@ public CompletableFuture delete(T record) { if (record instanceof InternalRow) { return delegate.delete((InternalRow) record); } - InternalRow pkOnly = convertPojo(record, /*forDelete=*/ true); + InternalRow pkOnly = convertPojo(record, /*forDelete=*/true); return delegate.delete(pkOnly); } private InternalRow convertPojo(T pojo, boolean forDelete) { - final RowType projection; - final PojoToRowConverter converter; + RowType projection; if (forDelete) { - projection = pkProjection; - converter = pkConverter; - } else if (targetProjection != null && targetConverter != null) { - projection = targetProjection; - converter = targetConverter; + // for delete we only need primary key columns + projection = tableSchema.project(tableInfo.getPhysicalPrimaryKeys()); + } else if (targetColumns != null) { + projection = tableSchema.project(targetColumns); } else { projection = tableSchema; - converter = pojoToRowConverter; } + // TODO: initialize this on the constructor and reuse + PojoToRowConverter converter = PojoToRowConverter.of(pojoClass, tableSchema, projection); GenericRow projected = converter.toRow(pojo); if (projection == tableSchema) { return projected; @@ -130,4 +103,4 @@ private InternalRow convertPojo(T pojo, boolean forDelete) { } return full; } -} +} \ No newline at end of file diff --git a/fluss-client/src/main/java/org/apache/fluss/client/table/writer/Upsert.java b/fluss-client/src/main/java/org/apache/fluss/client/table/writer/Upsert.java index c5e3c1c762..89cc8d8a6d 100644 --- a/fluss-client/src/main/java/org/apache/fluss/client/table/writer/Upsert.java +++ b/fluss-client/src/main/java/org/apache/fluss/client/table/writer/Upsert.java @@ -57,11 +57,12 @@ public interface Upsert { Upsert partialUpdate(String... targetColumnNames); /** - * Create a new {@link UpsertWriter} using {@code InternalRow} with the optional {@link - * #partialUpdate(String...)} information to upsert and delete data to a Primary Key Table. + * Create a new {@link UpsertWriter} using {@code InternalRow} with the optional + * {@link #partialUpdate(String...)} information to upsert and delete data to a Primary Key + * Table. */ UpsertWriter createWriter(); /** Create a new typed {@link UpsertWriter} to write POJOs directly. */ UpsertWriter createWriter(Class pojoClass); -} +} \ No newline at end of file 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 c0a4c51f2d..6fbd607ba4 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 @@ -51,4 +51,4 @@ public interface UpsertWriter extends TableWriter { * @return A {@link CompletableFuture} that always delete result when complete normally. */ CompletableFuture delete(T record); -} +} \ No newline at end of file From f07a84b23fdd346ef4d5b80b46766ca2ba061757 Mon Sep 17 00:00:00 2001 From: ipolyzos Date: Mon, 17 Nov 2025 16:52:08 +0200 Subject: [PATCH 14/33] fix checkstyle violation --- .../src/main/java/org/apache/fluss/client/lookup/Lookuper.java | 1 - 1 file changed, 1 deletion(-) diff --git a/fluss-client/src/main/java/org/apache/fluss/client/lookup/Lookuper.java b/fluss-client/src/main/java/org/apache/fluss/client/lookup/Lookuper.java index 651f9b5590..485a31cc2d 100644 --- a/fluss-client/src/main/java/org/apache/fluss/client/lookup/Lookuper.java +++ b/fluss-client/src/main/java/org/apache/fluss/client/lookup/Lookuper.java @@ -18,7 +18,6 @@ package org.apache.fluss.client.lookup; import org.apache.fluss.annotation.PublicEvolving; -import org.apache.fluss.row.InternalRow; import javax.annotation.concurrent.NotThreadSafe; import java.util.concurrent.CompletableFuture; From a7cad6d8556136fbbb514086f78acd9d6125d58b Mon Sep 17 00:00:00 2001 From: ipolyzos Date: Mon, 17 Nov 2025 16:56:44 +0200 Subject: [PATCH 15/33] fix checkstyle violation --- .../org/apache/fluss/client/lookup/Lookup.java | 8 ++++---- .../apache/fluss/client/lookup/Lookuper.java | 15 +++++++-------- .../fluss/client/lookup/TypedLookuper.java | 15 +++++++++------ .../apache/fluss/client/table/scanner/Scan.java | 2 +- .../fluss/client/table/scanner/ScanRecord.java | 2 +- .../fluss/client/table/scanner/TableScan.java | 2 +- .../table/scanner/log/CompletedFetch.java | 2 +- .../table/scanner/log/LogFetchCollector.java | 13 ++++++++----- .../table/scanner/log/LogScannerImpl.java | 2 +- .../client/table/scanner/log/ScanRecords.java | 6 ++++-- .../table/scanner/log/TypedLogScanner.java | 17 ++++++++++------- .../fluss/client/table/writer/Append.java | 2 +- .../fluss/client/table/writer/AppendWriter.java | 2 +- .../fluss/client/table/writer/TableAppend.java | 2 +- .../fluss/client/table/writer/TableUpsert.java | 5 +++-- .../client/table/writer/TypedAppendWriter.java | 5 +++-- .../client/table/writer/TypedUpsertWriter.java | 12 ++++++++---- .../fluss/client/table/writer/Upsert.java | 7 +++---- .../fluss/client/table/writer/UpsertWriter.java | 2 +- 19 files changed, 68 insertions(+), 53 deletions(-) diff --git a/fluss-client/src/main/java/org/apache/fluss/client/lookup/Lookup.java b/fluss-client/src/main/java/org/apache/fluss/client/lookup/Lookup.java index 61d976b4b2..5411a0c91d 100644 --- a/fluss-client/src/main/java/org/apache/fluss/client/lookup/Lookup.java +++ b/fluss-client/src/main/java/org/apache/fluss/client/lookup/Lookup.java @@ -99,11 +99,11 @@ default Lookup lookupBy(String... lookupColumnNames) { * lookup columns. By default, the lookup columns are the primary key columns, but can be * changed with ({@link #lookupBy(List)}) method. * - *

The returned lookuper accepts generic keys of type {@code K}. If a key is a POJO, - * the client implementation will convert it to an {@code InternalRow} based on the table - * schema and the active lookup columns. + *

The returned lookuper accepts generic keys of type {@code K}. If a key is a POJO, the + * client implementation will convert it to an {@code InternalRow} based on the table schema and + * the active lookup columns. * * @return the lookuper */ Lookuper createLookuper(); -} \ No newline at end of file +} diff --git a/fluss-client/src/main/java/org/apache/fluss/client/lookup/Lookuper.java b/fluss-client/src/main/java/org/apache/fluss/client/lookup/Lookuper.java index 485a31cc2d..b70adf8a4f 100644 --- a/fluss-client/src/main/java/org/apache/fluss/client/lookup/Lookuper.java +++ b/fluss-client/src/main/java/org/apache/fluss/client/lookup/Lookuper.java @@ -26,13 +26,12 @@ * A lookuper performs key-based lookups against a primary key table, using either the full primary * key or a prefix of the primary key (when configured via {@code Lookup#lookupBy}). * - *

This interface is generic on the key type {@code K}: - * - When used in row mode, implementations are typically declared as {@code Lookuper} - * and accept an {@link org.apache.fluss.row.InternalRow} containing the key fields in the - * configured order. - * - When used with POJO keys, client-provided lookuper instances can also accept a POJO key type - * (for example {@code Lookuper}) and will transparently convert the POJO to an - * {@code InternalRow} using the table schema and active lookup columns. + *

This interface is generic on the key type {@code K}: - When used in row mode, implementations + * are typically declared as {@code Lookuper} and accept an {@link + * org.apache.fluss.row.InternalRow} containing the key fields in the configured order. - When used + * with POJO keys, client-provided lookuper instances can also accept a POJO key type (for example + * {@code Lookuper}) and will transparently convert the POJO to an {@code InternalRow} + * using the table schema and active lookup columns. * *

Usage examples: * @@ -67,4 +66,4 @@ public interface Lookuper { * @return the result of lookup. */ CompletableFuture lookup(K lookupKey); -} \ No newline at end of file +} diff --git a/fluss-client/src/main/java/org/apache/fluss/client/lookup/TypedLookuper.java b/fluss-client/src/main/java/org/apache/fluss/client/lookup/TypedLookuper.java index b4fb3518e1..bba49366b8 100644 --- a/fluss-client/src/main/java/org/apache/fluss/client/lookup/TypedLookuper.java +++ b/fluss-client/src/main/java/org/apache/fluss/client/lookup/TypedLookuper.java @@ -23,13 +23,15 @@ import org.apache.fluss.types.RowType; import javax.annotation.Nullable; + import java.util.List; import java.util.concurrent.CompletableFuture; /** - * Decorator for {@link Lookuper} that enables generic key lookup via {@link Lookuper#lookup(Object)}. - * Converts POJO keys to {@link InternalRow} using existing converters based on table schema and - * active lookup columns, and directly delegates when the key is already an {@link InternalRow}. + * Decorator for {@link Lookuper} that enables generic key lookup via {@link + * Lookuper#lookup(Object)}. Converts POJO keys to {@link InternalRow} using existing converters + * based on table schema and active lookup columns, and directly delegates when the key is already + * an {@link InternalRow}. */ final class TypedLookuper implements Lookuper { @@ -37,9 +39,10 @@ final class TypedLookuper implements Lookuper { private final TableInfo tableInfo; @Nullable private final List lookupColumnNames; - TypedLookuper(Lookuper delegate, - TableInfo tableInfo, - @Nullable List lookupColumnNames) { + TypedLookuper( + Lookuper delegate, + TableInfo tableInfo, + @Nullable List lookupColumnNames) { this.delegate = delegate; this.tableInfo = tableInfo; this.lookupColumnNames = lookupColumnNames; diff --git a/fluss-client/src/main/java/org/apache/fluss/client/table/scanner/Scan.java b/fluss-client/src/main/java/org/apache/fluss/client/table/scanner/Scan.java index e15e3894b7..c8d47a9796 100644 --- a/fluss-client/src/main/java/org/apache/fluss/client/table/scanner/Scan.java +++ b/fluss-client/src/main/java/org/apache/fluss/client/table/scanner/Scan.java @@ -88,4 +88,4 @@ public interface Scan { * #limit(int)} and only support for Primary Key Tables. */ BatchScanner createBatchScanner(TableBucket tableBucket, long snapshotId); -} \ No newline at end of file +} diff --git a/fluss-client/src/main/java/org/apache/fluss/client/table/scanner/ScanRecord.java b/fluss-client/src/main/java/org/apache/fluss/client/table/scanner/ScanRecord.java index 074875b9c8..0f96e06abf 100644 --- a/fluss-client/src/main/java/org/apache/fluss/client/table/scanner/ScanRecord.java +++ b/fluss-client/src/main/java/org/apache/fluss/client/table/scanner/ScanRecord.java @@ -89,4 +89,4 @@ public int hashCode() { public String toString() { return changeType.shortString() + value + "@" + offset; } -} \ No newline at end of file +} diff --git a/fluss-client/src/main/java/org/apache/fluss/client/table/scanner/TableScan.java b/fluss-client/src/main/java/org/apache/fluss/client/table/scanner/TableScan.java index 93afcf58f3..ac530ab2bb 100644 --- a/fluss-client/src/main/java/org/apache/fluss/client/table/scanner/TableScan.java +++ b/fluss-client/src/main/java/org/apache/fluss/client/table/scanner/TableScan.java @@ -171,4 +171,4 @@ public BatchScanner createBatchScanner(TableBucket tableBucket, long snapshotId) tableInfo.getTableConfig().getKvFormat(), conn.getOrCreateRemoteFileDownloader()); } -} \ No newline at end of file +} diff --git a/fluss-client/src/main/java/org/apache/fluss/client/table/scanner/log/CompletedFetch.java b/fluss-client/src/main/java/org/apache/fluss/client/table/scanner/log/CompletedFetch.java index 88780b9ab6..a4c0ee4e90 100644 --- a/fluss-client/src/main/java/org/apache/fluss/client/table/scanner/log/CompletedFetch.java +++ b/fluss-client/src/main/java/org/apache/fluss/client/table/scanner/log/CompletedFetch.java @@ -255,4 +255,4 @@ private void maybeCloseRecordStream() { records = null; } } -} \ No newline at end of file +} diff --git a/fluss-client/src/main/java/org/apache/fluss/client/table/scanner/log/LogFetchCollector.java b/fluss-client/src/main/java/org/apache/fluss/client/table/scanner/log/LogFetchCollector.java index daf3cf409f..87e738fc58 100644 --- a/fluss-client/src/main/java/org/apache/fluss/client/table/scanner/log/LogFetchCollector.java +++ b/fluss-client/src/main/java/org/apache/fluss/client/table/scanner/log/LogFetchCollector.java @@ -20,7 +20,6 @@ import org.apache.fluss.annotation.Internal; import org.apache.fluss.client.metadata.MetadataUpdater; import org.apache.fluss.client.table.scanner.ScanRecord; -import org.apache.fluss.row.InternalRow; import org.apache.fluss.config.ConfigOptions; import org.apache.fluss.config.Configuration; import org.apache.fluss.exception.AuthorizationException; @@ -30,6 +29,7 @@ import org.apache.fluss.metadata.TablePath; import org.apache.fluss.record.LogRecord; import org.apache.fluss.record.LogRecordBatch; +import org.apache.fluss.row.InternalRow; import org.apache.fluss.rpc.protocol.ApiError; import org.apache.fluss.rpc.protocol.Errors; @@ -84,7 +84,8 @@ public LogFetchCollector( * @throws LogOffsetOutOfRangeException If there is OffsetOutOfRange error in fetchResponse and * the defaultResetPolicy is NONE */ - public Map>> collectFetch(final LogFetchBuffer logFetchBuffer) { + public Map>> collectFetch( + final LogFetchBuffer logFetchBuffer) { Map>> fetched = new HashMap<>(); int recordsRemaining = maxPollRecords; @@ -116,7 +117,8 @@ public Map>> collectFetch(final LogFet logFetchBuffer.poll(); } else { - List> records = fetchRecords(nextInLineFetch, recordsRemaining); + List> records = + fetchRecords(nextInLineFetch, recordsRemaining); if (!records.isEmpty()) { TableBucket tableBucket = nextInLineFetch.tableBucket; List> currentRecords = fetched.get(tableBucket); @@ -147,7 +149,8 @@ public Map>> collectFetch(final LogFet return fetched; } - private List> fetchRecords(CompletedFetch nextInLineFetch, int maxRecords) { + private List> fetchRecords( + CompletedFetch nextInLineFetch, int maxRecords) { TableBucket tb = nextInLineFetch.tableBucket; Long offset = logScannerStatus.getBucketOffset(tb); if (offset == null) { @@ -290,4 +293,4 @@ private void handleInitializeErrors( error, fetchOffset, tb, error.exception(errorMessage))); } } -} \ No newline at end of file +} diff --git a/fluss-client/src/main/java/org/apache/fluss/client/table/scanner/log/LogScannerImpl.java b/fluss-client/src/main/java/org/apache/fluss/client/table/scanner/log/LogScannerImpl.java index dbd89925a6..491773a7c8 100644 --- a/fluss-client/src/main/java/org/apache/fluss/client/table/scanner/log/LogScannerImpl.java +++ b/fluss-client/src/main/java/org/apache/fluss/client/table/scanner/log/LogScannerImpl.java @@ -306,4 +306,4 @@ public void close() { release(); } } -} \ No newline at end of file +} diff --git a/fluss-client/src/main/java/org/apache/fluss/client/table/scanner/log/ScanRecords.java b/fluss-client/src/main/java/org/apache/fluss/client/table/scanner/log/ScanRecords.java index ea38f6adaf..1473954c51 100644 --- a/fluss-client/src/main/java/org/apache/fluss/client/table/scanner/log/ScanRecords.java +++ b/fluss-client/src/main/java/org/apache/fluss/client/table/scanner/log/ScanRecords.java @@ -37,7 +37,9 @@ */ @PublicEvolving public class ScanRecords implements Iterable> { - public static final ScanRecords empty() { return new ScanRecords<>(Collections.emptyMap()); } + public static final ScanRecords empty() { + return new ScanRecords<>(Collections.emptyMap()); + } private final Map>> records; @@ -113,4 +115,4 @@ public ScanRecord makeNext() { }; } } -} \ No newline at end of file +} diff --git a/fluss-client/src/main/java/org/apache/fluss/client/table/scanner/log/TypedLogScanner.java b/fluss-client/src/main/java/org/apache/fluss/client/table/scanner/log/TypedLogScanner.java index a13ded7fe9..f8b7cdb857 100644 --- a/fluss-client/src/main/java/org/apache/fluss/client/table/scanner/log/TypedLogScanner.java +++ b/fluss-client/src/main/java/org/apache/fluss/client/table/scanner/log/TypedLogScanner.java @@ -38,13 +38,15 @@ public class TypedLogScanner implements LogScanner { private final LogScanner delegate; private final RowToPojoConverter converter; - public TypedLogScanner(LogScanner delegate, - Class pojoClass, - TableInfo tableInfo, - int[] projectedColumns) { + public TypedLogScanner( + LogScanner delegate, + Class pojoClass, + TableInfo tableInfo, + int[] projectedColumns) { this.delegate = delegate; RowType tableSchema = tableInfo.getRowType(); - RowType projection = projectedColumns == null ? tableSchema : tableSchema.project(projectedColumns); + RowType projection = + projectedColumns == null ? tableSchema : tableSchema.project(projectedColumns); this.converter = RowToPojoConverter.of(pojoClass, tableSchema, projection); } @@ -61,7 +63,8 @@ public ScanRecords poll(Duration timeout) { for (ScanRecord r : list) { InternalRow row = r.getValue(); T pojo = converter.fromRow(row); - converted.add(new ScanRecord<>(r.logOffset(), r.timestamp(), r.getChangeType(), pojo)); + converted.add( + new ScanRecord<>(r.logOffset(), r.timestamp(), r.getChangeType(), pojo)); } out.put(bucket, converted); } @@ -96,4 +99,4 @@ public void close() { throw new RuntimeException(e); } } -} \ No newline at end of file +} diff --git a/fluss-client/src/main/java/org/apache/fluss/client/table/writer/Append.java b/fluss-client/src/main/java/org/apache/fluss/client/table/writer/Append.java index 24dd8a6845..594ef7cbfe 100644 --- a/fluss-client/src/main/java/org/apache/fluss/client/table/writer/Append.java +++ b/fluss-client/src/main/java/org/apache/fluss/client/table/writer/Append.java @@ -37,4 +37,4 @@ public interface Append { /** Create a new typed {@link AppendWriter} to write POJOs directly. */ AppendWriter createWriter(Class pojoClass); -} \ No newline at end of file +} diff --git a/fluss-client/src/main/java/org/apache/fluss/client/table/writer/AppendWriter.java b/fluss-client/src/main/java/org/apache/fluss/client/table/writer/AppendWriter.java index ff0f62eaae..7223f2f639 100644 --- a/fluss-client/src/main/java/org/apache/fluss/client/table/writer/AppendWriter.java +++ b/fluss-client/src/main/java/org/apache/fluss/client/table/writer/AppendWriter.java @@ -40,4 +40,4 @@ public interface AppendWriter extends TableWriter { * @return A {@link CompletableFuture} that always returns append result when complete normally. */ CompletableFuture append(T record); -} \ No newline at end of file +} diff --git a/fluss-client/src/main/java/org/apache/fluss/client/table/writer/TableAppend.java b/fluss-client/src/main/java/org/apache/fluss/client/table/writer/TableAppend.java index 5d3340f180..975acdd969 100644 --- a/fluss-client/src/main/java/org/apache/fluss/client/table/writer/TableAppend.java +++ b/fluss-client/src/main/java/org/apache/fluss/client/table/writer/TableAppend.java @@ -44,4 +44,4 @@ public AppendWriter createWriter(Class pojoClass) { AppendWriterImpl delegate = new AppendWriterImpl(tablePath, tableInfo, writerClient); return new TypedAppendWriter<>(delegate, pojoClass, tableInfo); } -} \ No newline at end of file +} diff --git a/fluss-client/src/main/java/org/apache/fluss/client/table/writer/TableUpsert.java b/fluss-client/src/main/java/org/apache/fluss/client/table/writer/TableUpsert.java index d5e8921e04..cd8d747848 100644 --- a/fluss-client/src/main/java/org/apache/fluss/client/table/writer/TableUpsert.java +++ b/fluss-client/src/main/java/org/apache/fluss/client/table/writer/TableUpsert.java @@ -98,7 +98,8 @@ public UpsertWriter createWriter() { @Override public UpsertWriter createWriter(Class pojoClass) { - UpsertWriterImpl delegate = new UpsertWriterImpl(tablePath, tableInfo, targetColumns, writerClient); + UpsertWriterImpl delegate = + new UpsertWriterImpl(tablePath, tableInfo, targetColumns, writerClient); return new TypedUpsertWriter<>(delegate, pojoClass, tableInfo, targetColumns); } -} \ No newline at end of file +} diff --git a/fluss-client/src/main/java/org/apache/fluss/client/table/writer/TypedAppendWriter.java b/fluss-client/src/main/java/org/apache/fluss/client/table/writer/TypedAppendWriter.java index 67f4113c28..256305ab1b 100644 --- a/fluss-client/src/main/java/org/apache/fluss/client/table/writer/TypedAppendWriter.java +++ b/fluss-client/src/main/java/org/apache/fluss/client/table/writer/TypedAppendWriter.java @@ -51,8 +51,9 @@ public CompletableFuture append(T record) { return delegate.append((InternalRow) record); } // TODO: initialize this on the constructor and reuse - PojoToRowConverter converter = PojoToRowConverter.of(pojoClass, tableSchema, tableSchema); + PojoToRowConverter converter = + PojoToRowConverter.of(pojoClass, tableSchema, tableSchema); InternalRow row = converter.toRow(record); return delegate.append(row); } -} \ No newline at end of file +} diff --git a/fluss-client/src/main/java/org/apache/fluss/client/table/writer/TypedUpsertWriter.java b/fluss-client/src/main/java/org/apache/fluss/client/table/writer/TypedUpsertWriter.java index fe895b3bb4..68976f8e29 100644 --- a/fluss-client/src/main/java/org/apache/fluss/client/table/writer/TypedUpsertWriter.java +++ b/fluss-client/src/main/java/org/apache/fluss/client/table/writer/TypedUpsertWriter.java @@ -42,7 +42,11 @@ public void flush() { private final RowType tableSchema; private final int[] targetColumns; // may be null - TypedUpsertWriter(UpsertWriterImpl delegate, Class pojoClass, TableInfo tableInfo, int[] targetColumns) { + TypedUpsertWriter( + UpsertWriterImpl delegate, + Class pojoClass, + TableInfo tableInfo, + int[] targetColumns) { this.delegate = delegate; this.pojoClass = pojoClass; this.tableInfo = tableInfo; @@ -55,7 +59,7 @@ public CompletableFuture upsert(T record) { if (record instanceof InternalRow) { return delegate.upsert((InternalRow) record); } - InternalRow row = convertPojo(record, /*forDelete=*/false); + InternalRow row = convertPojo(record, /*forDelete=*/ false); return delegate.upsert(row); } @@ -64,7 +68,7 @@ public CompletableFuture delete(T record) { if (record instanceof InternalRow) { return delegate.delete((InternalRow) record); } - InternalRow pkOnly = convertPojo(record, /*forDelete=*/true); + InternalRow pkOnly = convertPojo(record, /*forDelete=*/ true); return delegate.delete(pkOnly); } @@ -103,4 +107,4 @@ private InternalRow convertPojo(T pojo, boolean forDelete) { } return full; } -} \ No newline at end of file +} diff --git a/fluss-client/src/main/java/org/apache/fluss/client/table/writer/Upsert.java b/fluss-client/src/main/java/org/apache/fluss/client/table/writer/Upsert.java index 89cc8d8a6d..c5e3c1c762 100644 --- a/fluss-client/src/main/java/org/apache/fluss/client/table/writer/Upsert.java +++ b/fluss-client/src/main/java/org/apache/fluss/client/table/writer/Upsert.java @@ -57,12 +57,11 @@ public interface Upsert { Upsert partialUpdate(String... targetColumnNames); /** - * Create a new {@link UpsertWriter} using {@code InternalRow} with the optional - * {@link #partialUpdate(String...)} information to upsert and delete data to a Primary Key - * Table. + * Create a new {@link UpsertWriter} using {@code InternalRow} with the optional {@link + * #partialUpdate(String...)} information to upsert and delete data to a Primary Key Table. */ UpsertWriter createWriter(); /** Create a new typed {@link UpsertWriter} to write POJOs directly. */ UpsertWriter createWriter(Class pojoClass); -} \ No newline at end of file +} 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 6fbd607ba4..c0a4c51f2d 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 @@ -51,4 +51,4 @@ public interface UpsertWriter extends TableWriter { * @return A {@link CompletableFuture} that always delete result when complete normally. */ CompletableFuture delete(T record); -} \ No newline at end of file +} From b22defd7b9fbf1bac48c36c698c7e15518df737f Mon Sep 17 00:00:00 2001 From: ipolyzos Date: Mon, 17 Nov 2025 17:09:30 +0200 Subject: [PATCH 16/33] update tests --- .../table/scanner/log/RemoteCompletedFetchTest.java | 9 ++++++--- 1 file changed, 6 insertions(+), 3 deletions(-) diff --git a/fluss-client/src/test/java/org/apache/fluss/client/table/scanner/log/RemoteCompletedFetchTest.java b/fluss-client/src/test/java/org/apache/fluss/client/table/scanner/log/RemoteCompletedFetchTest.java index 64729804e8..65292e47fe 100644 --- a/fluss-client/src/test/java/org/apache/fluss/client/table/scanner/log/RemoteCompletedFetchTest.java +++ b/fluss-client/src/test/java/org/apache/fluss/client/table/scanner/log/RemoteCompletedFetchTest.java @@ -176,7 +176,8 @@ void testNegativeFetchCount() throws Exception { RemoteCompletedFetch completedFetch = makeCompletedFetch(tableBucket, fileLogRecords, fetchOffset, null); - List> scanRecords = completedFetch.fetchRecords(-10); + List> scanRecords = + completedFetch.fetchRecords(-10); assertThat(scanRecords.size()).isEqualTo(0); } @@ -193,7 +194,8 @@ void testNoRecordsInFetch() throws Exception { RemoteCompletedFetch completedFetch = makeCompletedFetch(tableBucket, fileLogRecords, fetchOffset, null); - List> scanRecords = completedFetch.fetchRecords(10); + List> scanRecords = + completedFetch.fetchRecords(10); assertThat(scanRecords.size()).isEqualTo(0); } @@ -233,7 +235,8 @@ void testProjection(String format) throws Exception { fetchOffset, Projection.of(new int[] {0, 2}, schema)); - List> scanRecords = completedFetch.fetchRecords(8); + List> scanRecords = + completedFetch.fetchRecords(8); List expectedObjects = Arrays.asList( new Object[] {1, "hello"}, From 950ed3b01cd25d2ef08e3e5963578415874278e8 Mon Sep 17 00:00:00 2001 From: ipolyzos Date: Mon, 17 Nov 2025 17:34:50 +0200 Subject: [PATCH 17/33] add end2end tests --- .../apache/fluss/client/converter/ConverterCommons.java | 3 +-- .../fluss/client/converter/PojoToRowConverter.java | 2 ++ .../java/org/apache/fluss/client/converter/PojoType.java | 8 ++------ .../table/scanner/log/RemoteCompletedFetchTest.java | 9 +++------ 4 files changed, 8 insertions(+), 14 deletions(-) diff --git a/fluss-client/src/main/java/org/apache/fluss/client/converter/ConverterCommons.java b/fluss-client/src/main/java/org/apache/fluss/client/converter/ConverterCommons.java index aeb36419a6..700456edae 100644 --- a/fluss-client/src/main/java/org/apache/fluss/client/converter/ConverterCommons.java +++ b/fluss-client/src/main/java/org/apache/fluss/client/converter/ConverterCommons.java @@ -89,8 +89,7 @@ static void validatePojoMatchesProjection(PojoType pojoType, RowType projecti if (!pojoNames.containsAll(fieldNames)) { throw new IllegalArgumentException( String.format( - "POJO fields %s must contain all projection fields %s. " - + "For full-table writes, POJO fields must exactly match table schema fields.", + "POJO fields %s must contain all projection fields %s.", pojoNames, fieldNames)); } for (int i = 0; i < projection.getFieldCount(); i++) { diff --git a/fluss-client/src/main/java/org/apache/fluss/client/converter/PojoToRowConverter.java b/fluss-client/src/main/java/org/apache/fluss/client/converter/PojoToRowConverter.java index 0f4f6d8a4e..bd5a580662 100644 --- a/fluss-client/src/main/java/org/apache/fluss/client/converter/PojoToRowConverter.java +++ b/fluss-client/src/main/java/org/apache/fluss/client/converter/PojoToRowConverter.java @@ -188,6 +188,8 @@ private static FieldToRow createFieldConverter(PojoType.Property prop, DataType BigDecimal bd = (BigDecimal) v; BigDecimal scaled = bd.setScale(scale, RoundingMode.HALF_UP); + // Validate precision after scaling; precision is the number of digits in the unscaled + // value. if (scaled.precision() > precision) { throw new IllegalArgumentException( String.format( diff --git a/fluss-client/src/main/java/org/apache/fluss/client/converter/PojoType.java b/fluss-client/src/main/java/org/apache/fluss/client/converter/PojoType.java index ea098070d4..158a45e774 100644 --- a/fluss-client/src/main/java/org/apache/fluss/client/converter/PojoType.java +++ b/fluss-client/src/main/java/org/apache/fluss/client/converter/PojoType.java @@ -37,6 +37,7 @@ final class PojoType { private final Class pojoClass; private final Constructor defaultConstructor; private final Map properties; // property name -> property + // Mapping of primitive types to their boxed counterparts to avoid long if-chains private static final Map, Class> PRIMITIVE_TO_BOXED = createPrimitiveToBoxedMap(); private PojoType(Class pojoClass, Constructor ctor, Map props) { @@ -74,12 +75,7 @@ static PojoType of(Class pojoClass) { for (Map.Entry e : allFields.entrySet()) { String name = e.getKey(); Field field = e.getValue(); - // Enforce nullable fields: primitives are not allowed in POJO definitions. - if (field.getType().isPrimitive()) { - throw new IllegalArgumentException( - "Primitive types are not allowed; all fields must be nullable (use wrapper types)."); - } - // use boxed type as effective type + // Allow primitive types by treating them as their boxed counterparts for compatibility Class effectiveType = boxIfPrimitive(field.getType()); boolean publicField = Modifier.isPublic(field.getModifiers()); Method getter = getters.get(name); diff --git a/fluss-client/src/test/java/org/apache/fluss/client/table/scanner/log/RemoteCompletedFetchTest.java b/fluss-client/src/test/java/org/apache/fluss/client/table/scanner/log/RemoteCompletedFetchTest.java index 65292e47fe..64729804e8 100644 --- a/fluss-client/src/test/java/org/apache/fluss/client/table/scanner/log/RemoteCompletedFetchTest.java +++ b/fluss-client/src/test/java/org/apache/fluss/client/table/scanner/log/RemoteCompletedFetchTest.java @@ -176,8 +176,7 @@ void testNegativeFetchCount() throws Exception { RemoteCompletedFetch completedFetch = makeCompletedFetch(tableBucket, fileLogRecords, fetchOffset, null); - List> scanRecords = - completedFetch.fetchRecords(-10); + List> scanRecords = completedFetch.fetchRecords(-10); assertThat(scanRecords.size()).isEqualTo(0); } @@ -194,8 +193,7 @@ void testNoRecordsInFetch() throws Exception { RemoteCompletedFetch completedFetch = makeCompletedFetch(tableBucket, fileLogRecords, fetchOffset, null); - List> scanRecords = - completedFetch.fetchRecords(10); + List> scanRecords = completedFetch.fetchRecords(10); assertThat(scanRecords.size()).isEqualTo(0); } @@ -235,8 +233,7 @@ void testProjection(String format) throws Exception { fetchOffset, Projection.of(new int[] {0, 2}, schema)); - List> scanRecords = - completedFetch.fetchRecords(8); + List> scanRecords = completedFetch.fetchRecords(8); List expectedObjects = Arrays.asList( new Object[] {1, "hello"}, From 1fdbdfe2b2976b520ea32a1a2a0ad7809e64d3de Mon Sep 17 00:00:00 2001 From: ipolyzos Date: Mon, 17 Nov 2025 18:05:18 +0200 Subject: [PATCH 18/33] add required parameterized types --- .../apache/fluss/client/converter/ConverterCommons.java | 2 +- .../java/org/apache/fluss/client/converter/PojoType.java | 7 ++++++- .../fluss/flink/source/emitter/FlinkRecordEmitter.java | 2 +- 3 files changed, 8 insertions(+), 3 deletions(-) diff --git a/fluss-client/src/main/java/org/apache/fluss/client/converter/ConverterCommons.java b/fluss-client/src/main/java/org/apache/fluss/client/converter/ConverterCommons.java index 700456edae..c43f2ae9a8 100644 --- a/fluss-client/src/main/java/org/apache/fluss/client/converter/ConverterCommons.java +++ b/fluss-client/src/main/java/org/apache/fluss/client/converter/ConverterCommons.java @@ -89,7 +89,7 @@ static void validatePojoMatchesProjection(PojoType pojoType, RowType projecti if (!pojoNames.containsAll(fieldNames)) { throw new IllegalArgumentException( String.format( - "POJO fields %s must contain all projection fields %s.", + "POJO fields %s must contain all projection fields %s. For full-table writes, POJO fields must exactly match table schema fields.", pojoNames, fieldNames)); } for (int i = 0; i < projection.getFieldCount(); i++) { diff --git a/fluss-client/src/main/java/org/apache/fluss/client/converter/PojoType.java b/fluss-client/src/main/java/org/apache/fluss/client/converter/PojoType.java index 158a45e774..9253b4f835 100644 --- a/fluss-client/src/main/java/org/apache/fluss/client/converter/PojoType.java +++ b/fluss-client/src/main/java/org/apache/fluss/client/converter/PojoType.java @@ -75,7 +75,12 @@ static PojoType of(Class pojoClass) { for (Map.Entry e : allFields.entrySet()) { String name = e.getKey(); Field field = e.getValue(); - // Allow primitive types by treating them as their boxed counterparts for compatibility + // Enforce nullable fields: primitives are not allowed in POJO definitions. + if (field.getType().isPrimitive()) { + throw new IllegalArgumentException( + "Primitive types are not allowed; all fields must be nullable (use wrapper types)."); + } + // use boxed type as effective type Class effectiveType = boxIfPrimitive(field.getType()); boolean publicField = Modifier.isPublic(field.getModifiers()); Method getter = getters.get(name); diff --git a/fluss-flink/fluss-flink-common/src/main/java/org/apache/fluss/flink/source/emitter/FlinkRecordEmitter.java b/fluss-flink/fluss-flink-common/src/main/java/org/apache/fluss/flink/source/emitter/FlinkRecordEmitter.java index a413ca7ffc..c536766da0 100644 --- a/fluss-flink/fluss-flink-common/src/main/java/org/apache/fluss/flink/source/emitter/FlinkRecordEmitter.java +++ b/fluss-flink/fluss-flink-common/src/main/java/org/apache/fluss/flink/source/emitter/FlinkRecordEmitter.java @@ -108,7 +108,7 @@ record = deserializationSchema.deserialize(new ScanRecordLogRecord(scanRecord)); } } - /** Lightweight adapter to view a {@code ScanRecord} as a {@link LogRecord}. */ + /** Lightweight adapter to view a ScanRecord as a LogRecord. */ private static final class ScanRecordLogRecord implements LogRecord { private final ScanRecord delegate; From 5393618bd6be424e0ad5125602edfca039583fb6 Mon Sep 17 00:00:00 2001 From: ipolyzos Date: Mon, 17 Nov 2025 18:10:53 +0200 Subject: [PATCH 19/33] fix checkstyle violation --- .../org/apache/fluss/client/converter/ConverterCommons.java | 2 +- .../apache/fluss/flink/source/emitter/FlinkRecordEmitter.java | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/fluss-client/src/main/java/org/apache/fluss/client/converter/ConverterCommons.java b/fluss-client/src/main/java/org/apache/fluss/client/converter/ConverterCommons.java index c43f2ae9a8..700456edae 100644 --- a/fluss-client/src/main/java/org/apache/fluss/client/converter/ConverterCommons.java +++ b/fluss-client/src/main/java/org/apache/fluss/client/converter/ConverterCommons.java @@ -89,7 +89,7 @@ static void validatePojoMatchesProjection(PojoType pojoType, RowType projecti if (!pojoNames.containsAll(fieldNames)) { throw new IllegalArgumentException( String.format( - "POJO fields %s must contain all projection fields %s. For full-table writes, POJO fields must exactly match table schema fields.", + "POJO fields %s must contain all projection fields %s.", pojoNames, fieldNames)); } for (int i = 0; i < projection.getFieldCount(); i++) { diff --git a/fluss-flink/fluss-flink-common/src/main/java/org/apache/fluss/flink/source/emitter/FlinkRecordEmitter.java b/fluss-flink/fluss-flink-common/src/main/java/org/apache/fluss/flink/source/emitter/FlinkRecordEmitter.java index c536766da0..a413ca7ffc 100644 --- a/fluss-flink/fluss-flink-common/src/main/java/org/apache/fluss/flink/source/emitter/FlinkRecordEmitter.java +++ b/fluss-flink/fluss-flink-common/src/main/java/org/apache/fluss/flink/source/emitter/FlinkRecordEmitter.java @@ -108,7 +108,7 @@ record = deserializationSchema.deserialize(new ScanRecordLogRecord(scanRecord)); } } - /** Lightweight adapter to view a ScanRecord as a LogRecord. */ + /** Lightweight adapter to view a {@code ScanRecord} as a {@link LogRecord}. */ private static final class ScanRecordLogRecord implements LogRecord { private final ScanRecord delegate; From b579655523297b70fb8301299e5b3998871a78c4 Mon Sep 17 00:00:00 2001 From: ipolyzos Date: Mon, 17 Nov 2025 18:37:15 +0200 Subject: [PATCH 20/33] patch tests --- .../FlussRowToFlinkRowConverterTest.java | 42 +++---------------- 1 file changed, 5 insertions(+), 37 deletions(-) diff --git a/fluss-flink/fluss-flink-common/src/test/java/org/apache/fluss/flink/utils/FlussRowToFlinkRowConverterTest.java b/fluss-flink/fluss-flink-common/src/test/java/org/apache/fluss/flink/utils/FlussRowToFlinkRowConverterTest.java index 0475c8e00c..a8a4191c71 100644 --- a/fluss-flink/fluss-flink-common/src/test/java/org/apache/fluss/flink/utils/FlussRowToFlinkRowConverterTest.java +++ b/fluss-flink/fluss-flink-common/src/test/java/org/apache/fluss/flink/utils/FlussRowToFlinkRowConverterTest.java @@ -18,15 +18,12 @@ package org.apache.fluss.flink.utils; import org.apache.fluss.client.table.scanner.ScanRecord; -import org.apache.fluss.flink.row.FlinkAsFlussArray; import org.apache.fluss.record.ChangeType; import org.apache.fluss.record.LogRecord; import org.apache.fluss.row.InternalRow; -import org.apache.fluss.row.BinaryString; import org.apache.fluss.row.indexed.IndexedRow; import org.apache.fluss.row.indexed.IndexedRowWriter; import org.apache.fluss.types.DataType; -import org.apache.fluss.types.DataTypes; import org.apache.fluss.types.RowType; import org.apache.fluss.utils.DateTimeUtils; @@ -39,7 +36,6 @@ import java.time.LocalDate; import java.time.LocalTime; -import static org.apache.fluss.row.BinaryString.fromString; import static org.apache.fluss.row.TestInternalRowGenerator.createAllRowType; import static org.apache.fluss.row.TestInternalRowGenerator.createAllTypes; import static org.apache.fluss.row.indexed.IndexedRowTest.genRecordForAllTypes; @@ -58,9 +54,10 @@ void testConverter() throws Exception { try (IndexedRowWriter writer = genRecordForAllTypes(createAllTypes())) { row.pointTo(writer.segment(), 0, writer.position()); - ScanRecord scanRecord = new ScanRecord(0, 1L, ChangeType.UPDATE_BEFORE, row); - - RowData flinkRow = flussRowToFlinkRowConverter.toFlinkRowData(scanRecord); + ScanRecord scanRecord = + new ScanRecord<>(0, 1L, ChangeType.UPDATE_BEFORE, row); + RowData flinkRow = + flussRowToFlinkRowConverter.toFlinkRowData(new ScanRecordLogRecord(scanRecord)); assertThat(flinkRow.getArity()).isEqualTo(rowType.getFieldCount()); assertThat(flinkRow.getRowKind()).isEqualTo(RowKind.UPDATE_BEFORE); @@ -96,36 +93,7 @@ void testConverter() throws Exception { assertThat(flinkRow.getTimestamp(17, 1).toString()) .isEqualTo("2023-10-25T12:01:13.182"); - assertThat(flinkRow.getTimestamp(18, 5).toString()) - .isEqualTo("2023-10-25T12:01:13.182"); - - // array of int - Integer[] array1 = - new FlinkAsFlussArray(flinkRow.getArray(19)).toObjectArray(DataTypes.INT()); - assertThat(array1).isEqualTo(new Integer[] {1, 2, 3, 4, 5, -11, null, 444, 102234}); - - // array of float - Float[] array2 = - new FlinkAsFlussArray(flinkRow.getArray(20)).toObjectArray(DataTypes.FLOAT()); - assertThat(array2) - .isEqualTo( - new Float[] { - 0.1f, 1.1f, -0.5f, 6.6f, Float.MAX_VALUE, Float.MIN_VALUE - }); - - // array of string - assertThat(flinkRow.getArray(21).size()).isEqualTo(3); - BinaryString[] stringArray1 = - new FlinkAsFlussArray(flinkRow.getArray(21).getArray(0)) - .toObjectArray(DataTypes.STRING()); - assertThat(stringArray1) - .isEqualTo(new BinaryString[] {fromString("a"), null, fromString("c")}); - assertThat(flinkRow.getArray(21).isNullAt(1)).isTrue(); - BinaryString[] stringArray2 = - new FlinkAsFlussArray(flinkRow.getArray(21).getArray(2)) - .toObjectArray(DataTypes.STRING()); - assertThat(stringArray2) - .isEqualTo(new BinaryString[] {fromString("hello"), fromString("world")}); + assertThat(flinkRow.isNullAt(18)).isTrue(); } } From 3d0e607d505e7c133ab66b0567b94179c98e638e Mon Sep 17 00:00:00 2001 From: ipolyzos Date: Mon, 17 Nov 2025 19:13:16 +0200 Subject: [PATCH 21/33] improve test coverage --- .../org/apache/fluss/client/converter/ConverterCommons.java | 3 ++- .../fluss/flink/tiering/source/ScanRecordLogRecordTest.java | 3 ++- 2 files changed, 4 insertions(+), 2 deletions(-) diff --git a/fluss-client/src/main/java/org/apache/fluss/client/converter/ConverterCommons.java b/fluss-client/src/main/java/org/apache/fluss/client/converter/ConverterCommons.java index 700456edae..aeb36419a6 100644 --- a/fluss-client/src/main/java/org/apache/fluss/client/converter/ConverterCommons.java +++ b/fluss-client/src/main/java/org/apache/fluss/client/converter/ConverterCommons.java @@ -89,7 +89,8 @@ static void validatePojoMatchesProjection(PojoType pojoType, RowType projecti if (!pojoNames.containsAll(fieldNames)) { throw new IllegalArgumentException( String.format( - "POJO fields %s must contain all projection fields %s.", + "POJO fields %s must contain all projection fields %s. " + + "For full-table writes, POJO fields must exactly match table schema fields.", pojoNames, fieldNames)); } for (int i = 0; i < projection.getFieldCount(); i++) { diff --git a/fluss-flink/fluss-flink-common/src/test/java/org/apache/fluss/flink/tiering/source/ScanRecordLogRecordTest.java b/fluss-flink/fluss-flink-common/src/test/java/org/apache/fluss/flink/tiering/source/ScanRecordLogRecordTest.java index d4cc961d11..7524e9e78d 100644 --- a/fluss-flink/fluss-flink-common/src/test/java/org/apache/fluss/flink/tiering/source/ScanRecordLogRecordTest.java +++ b/fluss-flink/fluss-flink-common/src/test/java/org/apache/fluss/flink/tiering/source/ScanRecordLogRecordTest.java @@ -29,7 +29,7 @@ import static org.assertj.core.api.Assertions.assertThat; -/** Tests for the private adapter TieringSplitReader.ScanRecordLogRecord. */ +/** Tests for the private adapter TieringSplitReader.ScanRecordLogRecord */ class ScanRecordLogRecordTest { @Test @@ -59,6 +59,7 @@ void testAdapterDelegatesAllMethods() throws Exception { assertThat(adapter.getChangeType()).isEqualTo(changeType); assertThat(adapter.getRow()).isSameAs(row); + // Call twice to make sure repeated invocations are fine and to bump coverage assertThat(adapter.logOffset()).isEqualTo(offset); assertThat(adapter.getRow()).isSameAs(row); } From 5f9a2b1272ad08ae0bb7cf17037c4ef42b8fbda0 Mon Sep 17 00:00:00 2001 From: ipolyzos Date: Mon, 17 Nov 2025 19:17:54 +0200 Subject: [PATCH 22/33] fix checkstyle violation --- .../fluss/flink/tiering/source/ScanRecordLogRecordTest.java | 3 +-- 1 file changed, 1 insertion(+), 2 deletions(-) diff --git a/fluss-flink/fluss-flink-common/src/test/java/org/apache/fluss/flink/tiering/source/ScanRecordLogRecordTest.java b/fluss-flink/fluss-flink-common/src/test/java/org/apache/fluss/flink/tiering/source/ScanRecordLogRecordTest.java index 7524e9e78d..d4cc961d11 100644 --- a/fluss-flink/fluss-flink-common/src/test/java/org/apache/fluss/flink/tiering/source/ScanRecordLogRecordTest.java +++ b/fluss-flink/fluss-flink-common/src/test/java/org/apache/fluss/flink/tiering/source/ScanRecordLogRecordTest.java @@ -29,7 +29,7 @@ import static org.assertj.core.api.Assertions.assertThat; -/** Tests for the private adapter TieringSplitReader.ScanRecordLogRecord */ +/** Tests for the private adapter TieringSplitReader.ScanRecordLogRecord. */ class ScanRecordLogRecordTest { @Test @@ -59,7 +59,6 @@ void testAdapterDelegatesAllMethods() throws Exception { assertThat(adapter.getChangeType()).isEqualTo(changeType); assertThat(adapter.getRow()).isSameAs(row); - // Call twice to make sure repeated invocations are fine and to bump coverage assertThat(adapter.logOffset()).isEqualTo(offset); assertThat(adapter.getRow()).isSameAs(row); } From 421a70e7832266f6d224f3b48277d0b00253455b Mon Sep 17 00:00:00 2001 From: ipolyzos Date: Tue, 18 Nov 2025 09:11:45 +0200 Subject: [PATCH 23/33] instantiate converters once --- .../client/converter/PojoToRowConverter.java | 2 - .../fluss/client/converter/PojoType.java | 1 - .../table/writer/TypedAppendWriter.java | 7 ++-- .../table/writer/TypedUpsertWriter.java | 37 +++++++++++++++---- 4 files changed, 33 insertions(+), 14 deletions(-) diff --git a/fluss-client/src/main/java/org/apache/fluss/client/converter/PojoToRowConverter.java b/fluss-client/src/main/java/org/apache/fluss/client/converter/PojoToRowConverter.java index bd5a580662..0f4f6d8a4e 100644 --- a/fluss-client/src/main/java/org/apache/fluss/client/converter/PojoToRowConverter.java +++ b/fluss-client/src/main/java/org/apache/fluss/client/converter/PojoToRowConverter.java @@ -188,8 +188,6 @@ private static FieldToRow createFieldConverter(PojoType.Property prop, DataType BigDecimal bd = (BigDecimal) v; BigDecimal scaled = bd.setScale(scale, RoundingMode.HALF_UP); - // Validate precision after scaling; precision is the number of digits in the unscaled - // value. if (scaled.precision() > precision) { throw new IllegalArgumentException( String.format( diff --git a/fluss-client/src/main/java/org/apache/fluss/client/converter/PojoType.java b/fluss-client/src/main/java/org/apache/fluss/client/converter/PojoType.java index 9253b4f835..ea098070d4 100644 --- a/fluss-client/src/main/java/org/apache/fluss/client/converter/PojoType.java +++ b/fluss-client/src/main/java/org/apache/fluss/client/converter/PojoType.java @@ -37,7 +37,6 @@ final class PojoType { private final Class pojoClass; private final Constructor defaultConstructor; private final Map properties; // property name -> property - // Mapping of primitive types to their boxed counterparts to avoid long if-chains private static final Map, Class> PRIMITIVE_TO_BOXED = createPrimitiveToBoxedMap(); private PojoType(Class pojoClass, Constructor ctor, Map props) { diff --git a/fluss-client/src/main/java/org/apache/fluss/client/table/writer/TypedAppendWriter.java b/fluss-client/src/main/java/org/apache/fluss/client/table/writer/TypedAppendWriter.java index 256305ab1b..d02ec583a2 100644 --- a/fluss-client/src/main/java/org/apache/fluss/client/table/writer/TypedAppendWriter.java +++ b/fluss-client/src/main/java/org/apache/fluss/client/table/writer/TypedAppendWriter.java @@ -33,11 +33,13 @@ class TypedAppendWriter implements AppendWriter { private final AppendWriterImpl delegate; private final Class pojoClass; private final RowType tableSchema; + private final PojoToRowConverter pojoToRowConverter; TypedAppendWriter(AppendWriterImpl delegate, Class pojoClass, TableInfo tableInfo) { this.delegate = delegate; this.pojoClass = pojoClass; this.tableSchema = tableInfo.getRowType(); + this.pojoToRowConverter = PojoToRowConverter.of(pojoClass, tableSchema, tableSchema); } @Override @@ -50,10 +52,7 @@ public CompletableFuture append(T record) { if (record instanceof InternalRow) { return delegate.append((InternalRow) record); } - // TODO: initialize this on the constructor and reuse - PojoToRowConverter converter = - PojoToRowConverter.of(pojoClass, tableSchema, tableSchema); - InternalRow row = converter.toRow(record); + InternalRow row = pojoToRowConverter.toRow(record); return delegate.append(row); } } diff --git a/fluss-client/src/main/java/org/apache/fluss/client/table/writer/TypedUpsertWriter.java b/fluss-client/src/main/java/org/apache/fluss/client/table/writer/TypedUpsertWriter.java index 68976f8e29..f240c6b27c 100644 --- a/fluss-client/src/main/java/org/apache/fluss/client/table/writer/TypedUpsertWriter.java +++ b/fluss-client/src/main/java/org/apache/fluss/client/table/writer/TypedUpsertWriter.java @@ -23,6 +23,8 @@ import org.apache.fluss.row.InternalRow; import org.apache.fluss.types.RowType; +import javax.annotation.Nullable; + import java.util.concurrent.CompletableFuture; /** @@ -42,6 +44,13 @@ public void flush() { private final RowType tableSchema; private final int[] targetColumns; // may be null + private final RowType pkProjection; + @Nullable private final RowType targetProjection; + + private final PojoToRowConverter pojoToRowConverter; + private final PojoToRowConverter pkConverter; + @Nullable private final PojoToRowConverter targetConverter; + TypedUpsertWriter( UpsertWriterImpl delegate, Class pojoClass, @@ -52,6 +61,19 @@ public void flush() { this.tableInfo = tableInfo; this.tableSchema = tableInfo.getRowType(); this.targetColumns = targetColumns; + + // Precompute projections + this.pkProjection = this.tableSchema.project(tableInfo.getPhysicalPrimaryKeys()); + this.targetProjection = + (targetColumns == null) ? null : this.tableSchema.project(targetColumns); + + // Initialize reusable converters + this.pojoToRowConverter = PojoToRowConverter.of(pojoClass, tableSchema, tableSchema); + this.pkConverter = PojoToRowConverter.of(pojoClass, tableSchema, pkProjection); + this.targetConverter = + (targetProjection == null) + ? null + : PojoToRowConverter.of(pojoClass, tableSchema, targetProjection); } @Override @@ -73,18 +95,19 @@ public CompletableFuture delete(T record) { } private InternalRow convertPojo(T pojo, boolean forDelete) { - RowType projection; + final RowType projection; + final PojoToRowConverter converter; if (forDelete) { - // for delete we only need primary key columns - projection = tableSchema.project(tableInfo.getPhysicalPrimaryKeys()); - } else if (targetColumns != null) { - projection = tableSchema.project(targetColumns); + projection = pkProjection; + converter = pkConverter; + } else if (targetProjection != null && targetConverter != null) { + projection = targetProjection; + converter = targetConverter; } else { projection = tableSchema; + converter = pojoToRowConverter; } - // TODO: initialize this on the constructor and reuse - PojoToRowConverter converter = PojoToRowConverter.of(pojoClass, tableSchema, projection); GenericRow projected = converter.toRow(pojo); if (projection == tableSchema) { return projected; From 5a891c00172f4aa099ea3c4d20db2e6f49683592 Mon Sep 17 00:00:00 2001 From: ipolyzos Date: Tue, 25 Nov 2025 16:15:37 +0200 Subject: [PATCH 24/33] revert message --- .../main/java/org/apache/fluss/client/converter/PojoType.java | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/fluss-client/src/main/java/org/apache/fluss/client/converter/PojoType.java b/fluss-client/src/main/java/org/apache/fluss/client/converter/PojoType.java index ea098070d4..de848f448f 100644 --- a/fluss-client/src/main/java/org/apache/fluss/client/converter/PojoType.java +++ b/fluss-client/src/main/java/org/apache/fluss/client/converter/PojoType.java @@ -77,7 +77,9 @@ static PojoType of(Class pojoClass) { // Enforce nullable fields: primitives are not allowed in POJO definitions. if (field.getType().isPrimitive()) { throw new IllegalArgumentException( - "Primitive types are not allowed; all fields must be nullable (use wrapper types)."); + String.format( + "POJO class %s has primitive field '%s' of type %s. Primitive types are not allowed; all fields must be nullable (use wrapper types).", + pojoClass.getName(), name, field.getType().getName())); } // use boxed type as effective type Class effectiveType = boxIfPrimitive(field.getType()); From e063083b890353f6f9ab66c4efbf9043495d2b26 Mon Sep 17 00:00:00 2001 From: ipolyzos Date: Tue, 25 Nov 2025 16:31:26 +0200 Subject: [PATCH 25/33] fix test --- .../FlussRowToFlinkRowConverterTest.java | 35 ++++++++++++++++++- 1 file changed, 34 insertions(+), 1 deletion(-) diff --git a/fluss-flink/fluss-flink-common/src/test/java/org/apache/fluss/flink/utils/FlussRowToFlinkRowConverterTest.java b/fluss-flink/fluss-flink-common/src/test/java/org/apache/fluss/flink/utils/FlussRowToFlinkRowConverterTest.java index a8a4191c71..39848864e9 100644 --- a/fluss-flink/fluss-flink-common/src/test/java/org/apache/fluss/flink/utils/FlussRowToFlinkRowConverterTest.java +++ b/fluss-flink/fluss-flink-common/src/test/java/org/apache/fluss/flink/utils/FlussRowToFlinkRowConverterTest.java @@ -18,12 +18,15 @@ package org.apache.fluss.flink.utils; import org.apache.fluss.client.table.scanner.ScanRecord; +import org.apache.fluss.flink.row.FlinkAsFlussArray; import org.apache.fluss.record.ChangeType; import org.apache.fluss.record.LogRecord; +import org.apache.fluss.row.BinaryString; import org.apache.fluss.row.InternalRow; import org.apache.fluss.row.indexed.IndexedRow; import org.apache.fluss.row.indexed.IndexedRowWriter; import org.apache.fluss.types.DataType; +import org.apache.fluss.types.DataTypes; import org.apache.fluss.types.RowType; import org.apache.fluss.utils.DateTimeUtils; @@ -36,6 +39,7 @@ import java.time.LocalDate; import java.time.LocalTime; +import static org.apache.fluss.row.BinaryString.fromString; import static org.apache.fluss.row.TestInternalRowGenerator.createAllRowType; import static org.apache.fluss.row.TestInternalRowGenerator.createAllTypes; import static org.apache.fluss.row.indexed.IndexedRowTest.genRecordForAllTypes; @@ -93,7 +97,36 @@ void testConverter() throws Exception { assertThat(flinkRow.getTimestamp(17, 1).toString()) .isEqualTo("2023-10-25T12:01:13.182"); - assertThat(flinkRow.isNullAt(18)).isTrue(); + assertThat(flinkRow.getTimestamp(18, 5).toString()) + .isEqualTo("2023-10-25T12:01:13.182"); + + // array of int + Integer[] array1 = + new FlinkAsFlussArray(flinkRow.getArray(19)).toObjectArray(DataTypes.INT()); + assertThat(array1).isEqualTo(new Integer[] {1, 2, 3, 4, 5, -11, null, 444, 102234}); + + // array of float + Float[] array2 = + new FlinkAsFlussArray(flinkRow.getArray(20)).toObjectArray(DataTypes.FLOAT()); + assertThat(array2) + .isEqualTo( + new Float[] { + 0.1f, 1.1f, -0.5f, 6.6f, Float.MAX_VALUE, Float.MIN_VALUE + }); + + // array of string + assertThat(flinkRow.getArray(21).size()).isEqualTo(3); + BinaryString[] stringArray1 = + new FlinkAsFlussArray(flinkRow.getArray(21).getArray(0)) + .toObjectArray(DataTypes.STRING()); + assertThat(stringArray1) + .isEqualTo(new BinaryString[] {fromString("a"), null, fromString("c")}); + assertThat(flinkRow.getArray(21).isNullAt(1)).isTrue(); + BinaryString[] stringArray2 = + new FlinkAsFlussArray(flinkRow.getArray(21).getArray(2)) + .toObjectArray(DataTypes.STRING()); + assertThat(stringArray2) + .isEqualTo(new BinaryString[] {fromString("hello"), fromString("world")}); } } From 3ea717c84612dba2bd94b73df06f9484a65eb896 Mon Sep 17 00:00:00 2001 From: ipolyzos Date: Fri, 28 Nov 2025 15:45:35 +0200 Subject: [PATCH 26/33] fix checkstyle --- .../src/main/java/org/apache/fluss/client/lookup/Lookuper.java | 1 + 1 file changed, 1 insertion(+) diff --git a/fluss-client/src/main/java/org/apache/fluss/client/lookup/Lookuper.java b/fluss-client/src/main/java/org/apache/fluss/client/lookup/Lookuper.java index b70adf8a4f..a2b91498ca 100644 --- a/fluss-client/src/main/java/org/apache/fluss/client/lookup/Lookuper.java +++ b/fluss-client/src/main/java/org/apache/fluss/client/lookup/Lookuper.java @@ -20,6 +20,7 @@ import org.apache.fluss.annotation.PublicEvolving; import javax.annotation.concurrent.NotThreadSafe; + import java.util.concurrent.CompletableFuture; /** From 7dc4879ba36564cc7c3b517b8acd8fcdaf039971 Mon Sep 17 00:00:00 2001 From: ipolyzos Date: Mon, 22 Dec 2025 10:11:04 +0200 Subject: [PATCH 27/33] refactor to typed apis --- .../apache/fluss/client/lookup/Lookup.java | 21 ++- .../apache/fluss/client/lookup/Lookuper.java | 22 +-- .../client/lookup/PrefixKeyLookuper.java | 8 +- .../client/lookup/PrimaryKeyLookuper.java | 8 +- .../fluss/client/lookup/TableLookup.java | 12 +- .../fluss/client/lookup/TypedLookuper.java | 65 ++------ .../client/lookup/TypedLookuperImpl.java | 71 +++++++++ .../fluss/client/table/scanner/Scan.java | 8 +- .../fluss/client/table/scanner/TableScan.java | 8 +- .../client/table/scanner/log/LogScanner.java | 5 +- .../table/scanner/log/LogScannerImpl.java | 2 +- .../table/scanner/log/TypedLogScanner.java | 128 ++++++++-------- .../scanner/log/TypedLogScannerImpl.java | 109 ++++++++++++++ .../fluss/client/table/writer/Append.java | 4 +- .../client/table/writer/AppendWriter.java | 7 +- .../client/table/writer/AppendWriterImpl.java | 2 +- .../client/table/writer/TableAppend.java | 6 +- .../client/table/writer/TableUpsert.java | 6 +- .../client/table/writer/TableWriter.java | 7 +- .../table/writer/TypedAppendWriter.java | 48 ++---- .../table/writer/TypedAppendWriterImpl.java | 63 ++++++++ .../table/writer/TypedUpsertWriter.java | 131 +++-------------- .../table/writer/TypedUpsertWriterImpl.java | 139 ++++++++++++++++++ .../fluss/client/table/writer/Upsert.java | 4 +- .../client/table/writer/UpsertWriter.java | 12 +- .../client/table/writer/UpsertWriterImpl.java | 2 +- .../fluss/client/utils/MetadataUtils.java | 21 ++- .../admin/ClientToServerITCaseBase.java | 19 ++- .../metadata/MetadataUpdaterITCase.java | 1 + .../metadata/TestingMetadataUpdater.java | 3 +- .../table/AutoPartitionedTableITCase.java | 8 +- .../table/FlussFailServerTableITCase.java | 2 +- .../client/table/FlussLakeTableITCase.java | 7 +- .../fluss/client/table/FlussTableITCase.java | 26 ++-- .../client/table/FlussTypedClientITCase.java | 70 +++++---- .../client/table/PartitionedTableITCase.java | 2 +- .../log/DefaultCompletedFetchTest.java | 2 +- .../table/scanner/log/LogFetcherITCase.java | 3 +- .../table/scanner/log/LogScannerITCase.java | 12 +- .../scanner/log/RemoteLogScannerITCase.java | 4 +- .../client/write/RecordAccumulatorTest.java | 5 +- .../apache/fluss/client/write/SenderTest.java | 3 +- .../write/StickyStaticBucketAssignerTest.java | 1 + .../org/apache/fluss/cluster/Cluster.java | 29 +++- .../org/apache/fluss/cluster/ClusterTest.java | 27 +++- .../LakeSnapshotAndLogSplitScanner.java | 2 +- .../flink/sink/FlinkTableSinkITCase.java | 2 +- .../fluss/flink/sink/FlussSinkITCase.java | 4 +- 48 files changed, 723 insertions(+), 428 deletions(-) create mode 100644 fluss-client/src/main/java/org/apache/fluss/client/lookup/TypedLookuperImpl.java create mode 100644 fluss-client/src/main/java/org/apache/fluss/client/table/scanner/log/TypedLogScannerImpl.java create mode 100644 fluss-client/src/main/java/org/apache/fluss/client/table/writer/TypedAppendWriterImpl.java create mode 100644 fluss-client/src/main/java/org/apache/fluss/client/table/writer/TypedUpsertWriterImpl.java diff --git a/fluss-client/src/main/java/org/apache/fluss/client/lookup/Lookup.java b/fluss-client/src/main/java/org/apache/fluss/client/lookup/Lookup.java index 5411a0c91d..e64685b9a0 100644 --- a/fluss-client/src/main/java/org/apache/fluss/client/lookup/Lookup.java +++ b/fluss-client/src/main/java/org/apache/fluss/client/lookup/Lookup.java @@ -36,7 +36,7 @@ * column [k STRING]: * *

{@code
- * Lookuper lookuper = table.newLookup().createLookuper();
+ * Lookuper lookuper = table.newLookup().createLookuper();
  * CompletableFuture resultFuture = lookuper.lookup(GenericRow.of("key1"));
  * resultFuture.get().getRowList().forEach(System.out::println);
  * }
@@ -45,7 +45,7 @@ * columns [a INT, b STRING, c BIGINT] and bucket key [a, b]: * *
{@code
- * Lookuper lookuper = table.newLookup().lookupBy("a", "b").createLookuper();
+ * Lookuper lookuper = table.newLookup().lookupBy("a", "b").createLookuper();
  * CompletableFuture resultFuture = lookuper.lookup(GenericRow.of(1, "b1"));
  * resultFuture.get().getRowList().forEach(System.out::println);
  * }
@@ -53,7 +53,7 @@ *

Example 3: Using a POJO key (conversion handled internally): * *

{@code
- * Lookuper lookuper = table.newLookup().createLookuper();
+ * TypedLookuper lookuper = table.newLookup().createTypedLookuper(MyKeyPojo.class);
  * LookupResult result = lookuper.lookup(new MyKeyPojo(...)).get();
  * }
* @@ -99,11 +99,16 @@ default Lookup lookupBy(String... lookupColumnNames) { * lookup columns. By default, the lookup columns are the primary key columns, but can be * changed with ({@link #lookupBy(List)}) method. * - *

The returned lookuper accepts generic keys of type {@code K}. If a key is a POJO, the - * client implementation will convert it to an {@code InternalRow} based on the table schema and - * the active lookup columns. - * * @return the lookuper */ - Lookuper createLookuper(); + Lookuper createLookuper(); + + /** + * Creates a {@link TypedLookuper} instance to lookup rows of a primary key table using POJOs. + * + * @param pojoClass the class of the POJO + * @param the type of the POJO + * @return the typed lookuper + */ + TypedLookuper createTypedLookuper(Class pojoClass); } diff --git a/fluss-client/src/main/java/org/apache/fluss/client/lookup/Lookuper.java b/fluss-client/src/main/java/org/apache/fluss/client/lookup/Lookuper.java index a2b91498ca..9998a2612a 100644 --- a/fluss-client/src/main/java/org/apache/fluss/client/lookup/Lookuper.java +++ b/fluss-client/src/main/java/org/apache/fluss/client/lookup/Lookuper.java @@ -18,6 +18,7 @@ package org.apache.fluss.client.lookup; import org.apache.fluss.annotation.PublicEvolving; +import org.apache.fluss.row.InternalRow; import javax.annotation.concurrent.NotThreadSafe; @@ -27,30 +28,19 @@ * A lookuper performs key-based lookups against a primary key table, using either the full primary * key or a prefix of the primary key (when configured via {@code Lookup#lookupBy}). * - *

This interface is generic on the key type {@code K}: - When used in row mode, implementations - * are typically declared as {@code Lookuper} and accept an {@link - * org.apache.fluss.row.InternalRow} containing the key fields in the configured order. - When used - * with POJO keys, client-provided lookuper instances can also accept a POJO key type (for example - * {@code Lookuper}) and will transparently convert the POJO to an {@code InternalRow} - * using the table schema and active lookup columns. - * *

Usage examples: * *

{@code
  * // Row-based key (InternalRow)
- * Lookuper lookuper = table.newLookup().createLookuper();
+ * Lookuper lookuper = table.newLookup().createLookuper();
  * LookupResult res = lookuper.lookup(keyRow).get();
- *
- * // POJO key (converted internally)
- * Lookuper lookuperPojo = table.newLookup().createLookuper();
- * LookupResult res2 = lookuperPojo.lookup(new MyKeyPojo(...)).get();
  * }
* * @since 0.6 */ @PublicEvolving @NotThreadSafe -public interface Lookuper { +public interface Lookuper { /** * Lookups certain row from the given lookup key. @@ -59,12 +49,8 @@ public interface Lookuper { * {@code table.newLookup().createLookuper()}), or be the prefix key if the lookuper is a Prefix * Key Lookuper (created by {@code table.newLookup().lookupBy(prefixKeys).createLookuper()}). * - *

The key can be either an {@link org.apache.fluss.row.InternalRow} or a POJO representing - * the lookup key. Client-provided implementations returned by the Fluss client handle POJO-to- - * row conversion internally when necessary. - * * @param lookupKey the lookup key * @return the result of lookup. */ - CompletableFuture lookup(K lookupKey); + CompletableFuture lookup(InternalRow lookupKey); } diff --git a/fluss-client/src/main/java/org/apache/fluss/client/lookup/PrefixKeyLookuper.java b/fluss-client/src/main/java/org/apache/fluss/client/lookup/PrefixKeyLookuper.java index c5ba215209..a7b3e6545f 100644 --- a/fluss-client/src/main/java/org/apache/fluss/client/lookup/PrefixKeyLookuper.java +++ b/fluss-client/src/main/java/org/apache/fluss/client/lookup/PrefixKeyLookuper.java @@ -46,13 +46,7 @@ * of the primary key. */ @NotThreadSafe -class PrefixKeyLookuper implements Lookuper { - - private final TableInfo tableInfo; - - private final MetadataUpdater metadataUpdater; - - private final LookupClient lookupClient; +class PrefixKeyLookuper extends AbstractLookuper implements Lookuper { /** Extract bucket key from prefix lookup key row. */ private final KeyEncoder bucketKeyEncoder; diff --git a/fluss-client/src/main/java/org/apache/fluss/client/lookup/PrimaryKeyLookuper.java b/fluss-client/src/main/java/org/apache/fluss/client/lookup/PrimaryKeyLookuper.java index f7c79879b3..d2cd949630 100644 --- a/fluss-client/src/main/java/org/apache/fluss/client/lookup/PrimaryKeyLookuper.java +++ b/fluss-client/src/main/java/org/apache/fluss/client/lookup/PrimaryKeyLookuper.java @@ -40,13 +40,7 @@ /** An implementation of {@link Lookuper} that lookups by primary key. */ @NotThreadSafe -class PrimaryKeyLookuper implements Lookuper { - - private final TableInfo tableInfo; - - private final MetadataUpdater metadataUpdater; - - private final LookupClient lookupClient; +class PrimaryKeyLookuper extends AbstractLookuper implements Lookuper { private final KeyEncoder primaryKeyEncoder; diff --git a/fluss-client/src/main/java/org/apache/fluss/client/lookup/TableLookup.java b/fluss-client/src/main/java/org/apache/fluss/client/lookup/TableLookup.java index a4eb3cb28d..31766b3921 100644 --- a/fluss-client/src/main/java/org/apache/fluss/client/lookup/TableLookup.java +++ b/fluss-client/src/main/java/org/apache/fluss/client/lookup/TableLookup.java @@ -20,7 +20,6 @@ import org.apache.fluss.client.metadata.MetadataUpdater; import org.apache.fluss.metadata.SchemaGetter; import org.apache.fluss.metadata.TableInfo; -import org.apache.fluss.row.InternalRow; import javax.annotation.Nullable; @@ -64,14 +63,17 @@ public Lookup lookupBy(List lookupColumnNames) { } @Override - public Lookuper createLookuper() { - Lookuper lookuper; + public Lookuper createLookuper() { if (lookupColumnNames == null) { return new PrimaryKeyLookuper(tableInfo, schemaGetter, metadataUpdater, lookupClient); } else { return new PrefixKeyLookuper( tableInfo, schemaGetter, metadataUpdater, lookupClient, lookupColumnNames); } - return new TypedLookuper(lookuper, tableInfo, lookupColumnNames); } -} \ No newline at end of file + + @Override + public TypedLookuper createTypedLookuper(Class pojoClass) { + return new TypedLookuperImpl<>(createLookuper(), tableInfo, lookupColumnNames); + } +} diff --git a/fluss-client/src/main/java/org/apache/fluss/client/lookup/TypedLookuper.java b/fluss-client/src/main/java/org/apache/fluss/client/lookup/TypedLookuper.java index bba49366b8..2aa7624c82 100644 --- a/fluss-client/src/main/java/org/apache/fluss/client/lookup/TypedLookuper.java +++ b/fluss-client/src/main/java/org/apache/fluss/client/lookup/TypedLookuper.java @@ -1,5 +1,3 @@ -package org.apache.fluss.client.lookup; - /* * Licensed to the Apache Software Foundation (ASF) under one or more * contributor license agreements. See the NOTICE file distributed with @@ -17,57 +15,26 @@ * limitations under the License. */ -import org.apache.fluss.client.converter.PojoToRowConverter; -import org.apache.fluss.metadata.TableInfo; -import org.apache.fluss.row.InternalRow; -import org.apache.fluss.types.RowType; +package org.apache.fluss.client.lookup; -import javax.annotation.Nullable; +import org.apache.fluss.annotation.PublicEvolving; -import java.util.List; import java.util.concurrent.CompletableFuture; /** - * Decorator for {@link Lookuper} that enables generic key lookup via {@link - * Lookuper#lookup(Object)}. Converts POJO keys to {@link InternalRow} using existing converters - * based on table schema and active lookup columns, and directly delegates when the key is already - * an {@link InternalRow}. + * A typed lookuper performs key-based lookups against a primary key table using POJOs. + * + * @param the type of the lookup key + * @since 0.6 */ -final class TypedLookuper implements Lookuper { - - private final Lookuper delegate; - private final TableInfo tableInfo; - @Nullable private final List lookupColumnNames; - - TypedLookuper( - Lookuper delegate, - TableInfo tableInfo, - @Nullable List lookupColumnNames) { - this.delegate = delegate; - this.tableInfo = tableInfo; - this.lookupColumnNames = lookupColumnNames; - } - - @Override - public CompletableFuture lookup(K key) { - if (key == null) { - throw new IllegalArgumentException("key must not be null"); - } - // Fast-path: already an InternalRow - if (key instanceof InternalRow) { - return delegate.lookup((InternalRow) key); - } - RowType tableSchema = tableInfo.getRowType(); - RowType keyProjection; - if (lookupColumnNames == null) { - keyProjection = tableSchema.project(tableInfo.getPrimaryKeys()); - } else { - keyProjection = tableSchema.project(lookupColumnNames); - } - @SuppressWarnings("unchecked") - Class keyClass = (Class) key.getClass(); - PojoToRowConverter keyConv = PojoToRowConverter.of(keyClass, tableSchema, keyProjection); - InternalRow keyRow = keyConv.toRow(key); - return delegate.lookup(keyRow); - } +@PublicEvolving +public interface TypedLookuper { + + /** + * Lookups certain row from the given lookup key. + * + * @param lookupKey the lookup key + * @return the result of lookup. + */ + CompletableFuture lookup(T lookupKey); } diff --git a/fluss-client/src/main/java/org/apache/fluss/client/lookup/TypedLookuperImpl.java b/fluss-client/src/main/java/org/apache/fluss/client/lookup/TypedLookuperImpl.java new file mode 100644 index 0000000000..9e29f58762 --- /dev/null +++ b/fluss-client/src/main/java/org/apache/fluss/client/lookup/TypedLookuperImpl.java @@ -0,0 +1,71 @@ +package org.apache.fluss.client.lookup; + +/* + * 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. + */ + +import org.apache.fluss.client.converter.PojoToRowConverter; +import org.apache.fluss.metadata.TableInfo; +import org.apache.fluss.row.InternalRow; +import org.apache.fluss.types.RowType; + +import javax.annotation.Nullable; + +import java.util.List; +import java.util.concurrent.CompletableFuture; + +/** + * Decorator for {@link Lookuper} that enables generic key lookup via {@link + * TypedLookuper#lookup(Object)}. Converts POJO keys to {@link InternalRow} using existing + * converters based on table schema and active lookup columns, and directly delegates when the key + * is already an {@link InternalRow}. + */ +final class TypedLookuperImpl implements TypedLookuper { + + private final Lookuper delegate; + private final TableInfo tableInfo; + @Nullable private final List lookupColumnNames; + + TypedLookuperImpl( + Lookuper delegate, TableInfo tableInfo, @Nullable List lookupColumnNames) { + this.delegate = delegate; + this.tableInfo = tableInfo; + this.lookupColumnNames = lookupColumnNames; + } + + @Override + public CompletableFuture lookup(K key) { + if (key == null) { + throw new IllegalArgumentException("key must not be null"); + } + // Fast-path: already an InternalRow + if (key instanceof InternalRow) { + return delegate.lookup((InternalRow) key); + } + RowType tableSchema = tableInfo.getRowType(); + RowType keyProjection; + if (lookupColumnNames == null) { + keyProjection = tableSchema.project(tableInfo.getPrimaryKeys()); + } else { + keyProjection = tableSchema.project(lookupColumnNames); + } + @SuppressWarnings("unchecked") + Class keyClass = (Class) key.getClass(); + PojoToRowConverter keyConv = PojoToRowConverter.of(keyClass, tableSchema, keyProjection); + InternalRow keyRow = keyConv.toRow(key); + return delegate.lookup(keyRow); + } +} diff --git a/fluss-client/src/main/java/org/apache/fluss/client/table/scanner/Scan.java b/fluss-client/src/main/java/org/apache/fluss/client/table/scanner/Scan.java index c8d47a9796..1c05bd3916 100644 --- a/fluss-client/src/main/java/org/apache/fluss/client/table/scanner/Scan.java +++ b/fluss-client/src/main/java/org/apache/fluss/client/table/scanner/Scan.java @@ -20,8 +20,8 @@ import org.apache.fluss.annotation.PublicEvolving; import org.apache.fluss.client.table.scanner.batch.BatchScanner; import org.apache.fluss.client.table.scanner.log.LogScanner; +import org.apache.fluss.client.table.scanner.log.TypedLogScanner; import org.apache.fluss.metadata.TableBucket; -import org.apache.fluss.row.InternalRow; import javax.annotation.Nullable; @@ -64,14 +64,14 @@ public interface Scan { * *

Note: this API doesn't support pre-configured with {@link #limit(int)}. */ - LogScanner createLogScanner(); + LogScanner createLogScanner(); /** - * Creates a typed LogScanner to continuously read log data as POJOs of the given class. + * Creates a {@link TypedLogScanner} to continuously read log data as POJOs of the given class. * *

Note: this API doesn't support pre-configured with {@link #limit(int)}. */ - LogScanner createLogScanner(Class pojoClass); + TypedLogScanner createTypedLogScanner(Class pojoClass); /** * Creates a {@link BatchScanner} to read current data in the given table bucket for this scan. diff --git a/fluss-client/src/main/java/org/apache/fluss/client/table/scanner/TableScan.java b/fluss-client/src/main/java/org/apache/fluss/client/table/scanner/TableScan.java index ac530ab2bb..0d1d28a0e2 100644 --- a/fluss-client/src/main/java/org/apache/fluss/client/table/scanner/TableScan.java +++ b/fluss-client/src/main/java/org/apache/fluss/client/table/scanner/TableScan.java @@ -26,12 +26,12 @@ import org.apache.fluss.client.table.scanner.log.LogScanner; import org.apache.fluss.client.table.scanner.log.LogScannerImpl; import org.apache.fluss.client.table.scanner.log.TypedLogScanner; +import org.apache.fluss.client.table.scanner.log.TypedLogScannerImpl; import org.apache.fluss.config.ConfigOptions; import org.apache.fluss.exception.FlussRuntimeException; import org.apache.fluss.metadata.SchemaGetter; import org.apache.fluss.metadata.TableBucket; import org.apache.fluss.metadata.TableInfo; -import org.apache.fluss.row.InternalRow; import org.apache.fluss.types.RowType; import javax.annotation.Nullable; @@ -116,9 +116,9 @@ public LogScanner createLogScanner() { } @Override - public LogScanner createLogScanner(Class pojoClass) { - LogScanner base = createLogScanner(); - return new TypedLogScanner<>(base, pojoClass, tableInfo, projectedColumns); + public TypedLogScanner createTypedLogScanner(Class pojoClass) { + LogScanner base = createLogScanner(); + return new TypedLogScannerImpl<>(base, pojoClass, tableInfo, projectedColumns); } @Override diff --git a/fluss-client/src/main/java/org/apache/fluss/client/table/scanner/log/LogScanner.java b/fluss-client/src/main/java/org/apache/fluss/client/table/scanner/log/LogScanner.java index a085b9c4ee..f61d864776 100644 --- a/fluss-client/src/main/java/org/apache/fluss/client/table/scanner/log/LogScanner.java +++ b/fluss-client/src/main/java/org/apache/fluss/client/table/scanner/log/LogScanner.java @@ -18,6 +18,7 @@ package org.apache.fluss.client.table.scanner.log; import org.apache.fluss.annotation.PublicEvolving; +import org.apache.fluss.row.InternalRow; import java.time.Duration; @@ -27,7 +28,7 @@ * @since 0.1 */ @PublicEvolving -public interface LogScanner extends AutoCloseable { +public interface LogScanner extends AutoCloseable { /** * The earliest offset to fetch from. Fluss uses "-2" to indicate fetching from log start @@ -48,7 +49,7 @@ public interface LogScanner extends AutoCloseable { * @throws java.lang.IllegalStateException if the scanner is not subscribed to any buckets to * read from. */ - ScanRecords poll(Duration timeout); + ScanRecords poll(Duration timeout); /** * Subscribe to the given table bucket in given offset dynamically. If the table bucket is diff --git a/fluss-client/src/main/java/org/apache/fluss/client/table/scanner/log/LogScannerImpl.java b/fluss-client/src/main/java/org/apache/fluss/client/table/scanner/log/LogScannerImpl.java index 491773a7c8..621a887f56 100644 --- a/fluss-client/src/main/java/org/apache/fluss/client/table/scanner/log/LogScannerImpl.java +++ b/fluss-client/src/main/java/org/apache/fluss/client/table/scanner/log/LogScannerImpl.java @@ -57,7 +57,7 @@ * @since 0.1 */ @PublicEvolving -public class LogScannerImpl implements LogScanner { +public class LogScannerImpl implements LogScanner { private static final Logger LOG = LoggerFactory.getLogger(LogScannerImpl.class); private static final long NO_CURRENT_THREAD = -1L; diff --git a/fluss-client/src/main/java/org/apache/fluss/client/table/scanner/log/TypedLogScanner.java b/fluss-client/src/main/java/org/apache/fluss/client/table/scanner/log/TypedLogScanner.java index f8b7cdb857..16b3a7dae8 100644 --- a/fluss-client/src/main/java/org/apache/fluss/client/table/scanner/log/TypedLogScanner.java +++ b/fluss-client/src/main/java/org/apache/fluss/client/table/scanner/log/TypedLogScanner.java @@ -17,86 +17,74 @@ package org.apache.fluss.client.table.scanner.log; -import org.apache.fluss.client.converter.RowToPojoConverter; -import org.apache.fluss.client.table.scanner.ScanRecord; -import org.apache.fluss.metadata.TableBucket; -import org.apache.fluss.metadata.TableInfo; -import org.apache.fluss.row.InternalRow; -import org.apache.fluss.types.RowType; +import org.apache.fluss.annotation.PublicEvolving; import java.time.Duration; -import java.util.ArrayList; -import java.util.HashMap; -import java.util.List; -import java.util.Map; /** - * Adapter that converts {@link InternalRow} records from a {@link LogScanner} into POJOs of type T. + * A typed scanner is used to scan log data as POJOs of specify table from Fluss. + * + * @param the type of the POJO + * @since 0.6 */ -public class TypedLogScanner implements LogScanner { - - private final LogScanner delegate; - private final RowToPojoConverter converter; +@PublicEvolving +public interface TypedLogScanner extends AutoCloseable { - public TypedLogScanner( - LogScanner delegate, - Class pojoClass, - TableInfo tableInfo, - int[] projectedColumns) { - this.delegate = delegate; - RowType tableSchema = tableInfo.getRowType(); - RowType projection = - projectedColumns == null ? tableSchema : tableSchema.project(projectedColumns); - this.converter = RowToPojoConverter.of(pojoClass, tableSchema, projection); - } + /** + * Poll log data from tablet server. + * + * @param timeout the timeout to poll. + * @return the result of poll. + */ + ScanRecords poll(Duration timeout); - @Override - public ScanRecords poll(Duration timeout) { - ScanRecords records = delegate.poll(timeout); - if (records == null || records.isEmpty()) { - return ScanRecords.empty(); - } - Map>> out = new HashMap<>(); - for (TableBucket bucket : records.buckets()) { - List> list = records.records(bucket); - List> converted = new ArrayList<>(list.size()); - for (ScanRecord r : list) { - InternalRow row = r.getValue(); - T pojo = converter.fromRow(row); - converted.add( - new ScanRecord<>(r.logOffset(), r.timestamp(), r.getChangeType(), pojo)); - } - out.put(bucket, converted); - } - return new ScanRecords<>(out); - } + /** + * Subscribe to the given table bucket from beginning dynamically. If the table bucket is + * already subscribed, the start offset will be updated. + * + * @param bucket the table bucket to subscribe. + */ + void subscribeFromBeginning(int bucket); - @Override - public void subscribe(int bucket, long offset) { - delegate.subscribe(bucket, offset); - } + /** + * Subscribe to the given partitioned table bucket from beginning dynamically. If the table + * bucket is already subscribed, the start offset will be updated. + * + * @param partitionId the partition id of the table partition to subscribe. + * @param bucket the table bucket to subscribe. + */ + void subscribeFromBeginning(long partitionId, int bucket); - @Override - public void subscribe(long partitionId, int bucket, long offset) { - delegate.subscribe(partitionId, bucket, offset); - } + /** + * Subscribe to the given table bucket in given offset dynamically. If the table bucket is + * already subscribed, the offset will be updated. + * + * @param bucket the table bucket to subscribe. + * @param offset the offset to start from. + */ + void subscribe(int bucket, long offset); - @Override - public void unsubscribe(long partitionId, int bucket) { - delegate.unsubscribe(partitionId, bucket); - } + /** + * Subscribe to the given partitioned table bucket in given offset dynamically. If the table + * bucket is already subscribed, the offset will be updated. + * + * @param partitionId the partition id of the table partition to subscribe. + * @param bucket the table bucket to subscribe. + * @param offset the offset to start from. + */ + void subscribe(long partitionId, int bucket, long offset); - @Override - public void wakeup() { - delegate.wakeup(); - } + /** + * Unsubscribe from the given bucket of given partition dynamically. + * + * @param partitionId the partition id of the table partition to unsubscribe. + * @param bucket the table bucket to unsubscribe. + */ + void unsubscribe(long partitionId, int bucket); - @Override - public void close() { - try { - delegate.close(); - } catch (Exception e) { - throw new RuntimeException(e); - } - } + /** + * Wake up the log scanner in case the fetcher thread in log scanner is blocking in {@link + * #poll(Duration timeout)}. + */ + void wakeup(); } diff --git a/fluss-client/src/main/java/org/apache/fluss/client/table/scanner/log/TypedLogScannerImpl.java b/fluss-client/src/main/java/org/apache/fluss/client/table/scanner/log/TypedLogScannerImpl.java new file mode 100644 index 0000000000..09667e0b7c --- /dev/null +++ b/fluss-client/src/main/java/org/apache/fluss/client/table/scanner/log/TypedLogScannerImpl.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.client.table.scanner.log; + +import org.apache.fluss.client.converter.RowToPojoConverter; +import org.apache.fluss.client.table.scanner.ScanRecord; +import org.apache.fluss.metadata.TableBucket; +import org.apache.fluss.metadata.TableInfo; +import org.apache.fluss.row.InternalRow; +import org.apache.fluss.types.RowType; + +import java.time.Duration; +import java.util.ArrayList; +import java.util.HashMap; +import java.util.List; +import java.util.Map; + +/** + * Adapter that converts {@link InternalRow} records from a {@link LogScanner} into POJOs of type T. + */ +public class TypedLogScannerImpl implements TypedLogScanner { + + private final LogScanner delegate; + private final RowToPojoConverter converter; + + public TypedLogScannerImpl( + LogScanner delegate, Class pojoClass, TableInfo tableInfo, int[] projectedColumns) { + this.delegate = delegate; + RowType tableSchema = tableInfo.getRowType(); + RowType projection = + projectedColumns == null ? tableSchema : tableSchema.project(projectedColumns); + this.converter = RowToPojoConverter.of(pojoClass, tableSchema, projection); + } + + @Override + public ScanRecords poll(Duration timeout) { + ScanRecords records = delegate.poll(timeout); + if (records == null || records.isEmpty()) { + return ScanRecords.empty(); + } + Map>> out = new HashMap<>(); + for (TableBucket bucket : records.buckets()) { + List> list = records.records(bucket); + List> converted = new ArrayList<>(list.size()); + for (ScanRecord r : list) { + InternalRow row = r.getValue(); + T pojo = converter.fromRow(row); + converted.add( + new ScanRecord<>(r.logOffset(), r.timestamp(), r.getChangeType(), pojo)); + } + out.put(bucket, converted); + } + return new ScanRecords<>(out); + } + + @Override + public void subscribeFromBeginning(int bucket) { + delegate.subscribeFromBeginning(bucket); + } + + @Override + public void subscribeFromBeginning(long partitionId, int bucket) { + delegate.subscribeFromBeginning(partitionId, bucket); + } + + @Override + public void subscribe(int bucket, long offset) { + delegate.subscribe(bucket, offset); + } + + @Override + public void subscribe(long partitionId, int bucket, long offset) { + delegate.subscribe(partitionId, bucket, offset); + } + + @Override + public void unsubscribe(long partitionId, int bucket) { + delegate.unsubscribe(partitionId, bucket); + } + + @Override + public void wakeup() { + delegate.wakeup(); + } + + @Override + public void close() { + try { + delegate.close(); + } catch (Exception e) { + throw new RuntimeException(e); + } + } +} diff --git a/fluss-client/src/main/java/org/apache/fluss/client/table/writer/Append.java b/fluss-client/src/main/java/org/apache/fluss/client/table/writer/Append.java index 594ef7cbfe..4e705e85de 100644 --- a/fluss-client/src/main/java/org/apache/fluss/client/table/writer/Append.java +++ b/fluss-client/src/main/java/org/apache/fluss/client/table/writer/Append.java @@ -33,8 +33,8 @@ public interface Append { // apply overwrites, etc. /** Create a new {@link AppendWriter} to write data to a Log Table using InternalRow. */ - AppendWriter createWriter(); + AppendWriter createWriter(); /** Create a new typed {@link AppendWriter} to write POJOs directly. */ - AppendWriter createWriter(Class pojoClass); + TypedAppendWriter createWriter(Class pojoClass); } diff --git a/fluss-client/src/main/java/org/apache/fluss/client/table/writer/AppendWriter.java b/fluss-client/src/main/java/org/apache/fluss/client/table/writer/AppendWriter.java index 7223f2f639..1b5cee055f 100644 --- a/fluss-client/src/main/java/org/apache/fluss/client/table/writer/AppendWriter.java +++ b/fluss-client/src/main/java/org/apache/fluss/client/table/writer/AppendWriter.java @@ -28,16 +28,13 @@ * @since 0.2 */ @PublicEvolving -public interface AppendWriter extends TableWriter { +public interface AppendWriter extends TableWriter { /** * Append a record into a Log Table. * - *

If {@code T} is {@link InternalRow}, the row will be written directly. Otherwise, the - * client will convert the POJO into an {@link InternalRow} using the configured converters. - * * @param record the record to append. * @return A {@link CompletableFuture} that always returns append result when complete normally. */ - CompletableFuture append(T record); + CompletableFuture append(InternalRow record); } diff --git a/fluss-client/src/main/java/org/apache/fluss/client/table/writer/AppendWriterImpl.java b/fluss-client/src/main/java/org/apache/fluss/client/table/writer/AppendWriterImpl.java index 823413b2f0..d702e9621e 100644 --- a/fluss-client/src/main/java/org/apache/fluss/client/table/writer/AppendWriterImpl.java +++ b/fluss-client/src/main/java/org/apache/fluss/client/table/writer/AppendWriterImpl.java @@ -37,7 +37,7 @@ import java.util.concurrent.CompletableFuture; /** The writer to write data to the log table. */ -class AppendWriterImpl extends AbstractTableWriter implements AppendWriter { +class AppendWriterImpl extends AbstractTableWriter implements AppendWriter { private static final AppendResult APPEND_SUCCESS = new AppendResult(); private final @Nullable KeyEncoder bucketKeyEncoder; diff --git a/fluss-client/src/main/java/org/apache/fluss/client/table/writer/TableAppend.java b/fluss-client/src/main/java/org/apache/fluss/client/table/writer/TableAppend.java index 975acdd969..6e93c885d6 100644 --- a/fluss-client/src/main/java/org/apache/fluss/client/table/writer/TableAppend.java +++ b/fluss-client/src/main/java/org/apache/fluss/client/table/writer/TableAppend.java @@ -35,13 +35,13 @@ public TableAppend(TablePath tablePath, TableInfo tableInfo, WriterClient writer } @Override - public AppendWriter createWriter() { + public AppendWriter createWriter() { return new AppendWriterImpl(tablePath, tableInfo, writerClient); } @Override - public AppendWriter createWriter(Class pojoClass) { + public TypedAppendWriter createWriter(Class pojoClass) { AppendWriterImpl delegate = new AppendWriterImpl(tablePath, tableInfo, writerClient); - return new TypedAppendWriter<>(delegate, pojoClass, tableInfo); + return new TypedAppendWriterImpl<>(delegate, pojoClass, tableInfo); } } diff --git a/fluss-client/src/main/java/org/apache/fluss/client/table/writer/TableUpsert.java b/fluss-client/src/main/java/org/apache/fluss/client/table/writer/TableUpsert.java index cd8d747848..87d8187f06 100644 --- a/fluss-client/src/main/java/org/apache/fluss/client/table/writer/TableUpsert.java +++ b/fluss-client/src/main/java/org/apache/fluss/client/table/writer/TableUpsert.java @@ -92,14 +92,14 @@ public Upsert partialUpdate(String... targetColumnNames) { } @Override - public UpsertWriter createWriter() { + public UpsertWriter createWriter() { return new UpsertWriterImpl(tablePath, tableInfo, targetColumns, writerClient); } @Override - public UpsertWriter createWriter(Class pojoClass) { + public TypedUpsertWriter createWriter(Class pojoClass) { UpsertWriterImpl delegate = new UpsertWriterImpl(tablePath, tableInfo, targetColumns, writerClient); - return new TypedUpsertWriter<>(delegate, pojoClass, tableInfo, targetColumns); + return new TypedUpsertWriterImpl<>(delegate, pojoClass, tableInfo, targetColumns); } } diff --git a/fluss-client/src/main/java/org/apache/fluss/client/table/writer/TableWriter.java b/fluss-client/src/main/java/org/apache/fluss/client/table/writer/TableWriter.java index 3c4f134803..94cac25668 100644 --- a/fluss-client/src/main/java/org/apache/fluss/client/table/writer/TableWriter.java +++ b/fluss-client/src/main/java/org/apache/fluss/client/table/writer/TableWriter.java @@ -28,7 +28,7 @@ * @since 0.1 */ @PublicEvolving -public interface TableWriter { +public interface TableWriter extends AutoCloseable { /** * Flush data written that have not yet been sent to the server, forcing the client to send the @@ -38,4 +38,9 @@ public interface TableWriter { * results in an error. */ void flush(); + + @Override + default void close() throws Exception { + // by default do nothing + } } diff --git a/fluss-client/src/main/java/org/apache/fluss/client/table/writer/TypedAppendWriter.java b/fluss-client/src/main/java/org/apache/fluss/client/table/writer/TypedAppendWriter.java index d02ec583a2..bff9039139 100644 --- a/fluss-client/src/main/java/org/apache/fluss/client/table/writer/TypedAppendWriter.java +++ b/fluss-client/src/main/java/org/apache/fluss/client/table/writer/TypedAppendWriter.java @@ -17,42 +17,24 @@ package org.apache.fluss.client.table.writer; -import org.apache.fluss.client.converter.PojoToRowConverter; -import org.apache.fluss.metadata.TableInfo; -import org.apache.fluss.row.InternalRow; -import org.apache.fluss.types.RowType; +import org.apache.fluss.annotation.PublicEvolving; import java.util.concurrent.CompletableFuture; /** - * A typed {@link AppendWriter} that converts POJOs to {@link InternalRow} and delegates to the - * existing internal-row based writer implementation. + * The typed writer to write data to the log table using POJOs. + * + * @param the type of the record + * @since 0.6 */ -class TypedAppendWriter implements AppendWriter { - - private final AppendWriterImpl delegate; - private final Class pojoClass; - private final RowType tableSchema; - private final PojoToRowConverter pojoToRowConverter; - - TypedAppendWriter(AppendWriterImpl delegate, Class pojoClass, TableInfo tableInfo) { - this.delegate = delegate; - this.pojoClass = pojoClass; - this.tableSchema = tableInfo.getRowType(); - this.pojoToRowConverter = PojoToRowConverter.of(pojoClass, tableSchema, tableSchema); - } - - @Override - public void flush() { - delegate.flush(); - } - - @Override - public CompletableFuture append(T record) { - if (record instanceof InternalRow) { - return delegate.append((InternalRow) record); - } - InternalRow row = pojoToRowConverter.toRow(record); - return delegate.append(row); - } +@PublicEvolving +public interface TypedAppendWriter extends TableWriter { + + /** + * Append a record into a Log Table. + * + * @param record the record to append. + * @return A {@link CompletableFuture} that always returns append result when complete normally. + */ + CompletableFuture append(T record); } diff --git a/fluss-client/src/main/java/org/apache/fluss/client/table/writer/TypedAppendWriterImpl.java b/fluss-client/src/main/java/org/apache/fluss/client/table/writer/TypedAppendWriterImpl.java new file mode 100644 index 0000000000..c591eea5ea --- /dev/null +++ b/fluss-client/src/main/java/org/apache/fluss/client/table/writer/TypedAppendWriterImpl.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.client.table.writer; + +import org.apache.fluss.client.converter.PojoToRowConverter; +import org.apache.fluss.metadata.TableInfo; +import org.apache.fluss.row.InternalRow; +import org.apache.fluss.types.RowType; + +import java.util.concurrent.CompletableFuture; + +/** + * A typed {@link AppendWriter} that converts POJOs to {@link InternalRow} and delegates to the + * existing internal-row based writer implementation. + */ +class TypedAppendWriterImpl implements TypedAppendWriter { + + private final AppendWriterImpl delegate; + private final Class pojoClass; + private final RowType tableSchema; + private final PojoToRowConverter pojoToRowConverter; + + TypedAppendWriterImpl(AppendWriterImpl delegate, Class pojoClass, TableInfo tableInfo) { + this.delegate = delegate; + this.pojoClass = pojoClass; + this.tableSchema = tableInfo.getRowType(); + this.pojoToRowConverter = PojoToRowConverter.of(pojoClass, tableSchema, tableSchema); + } + + @Override + public void flush() { + delegate.flush(); + } + + @Override + public void close() throws Exception { + delegate.close(); + } + + @Override + public CompletableFuture append(T record) { + if (record instanceof InternalRow) { + return delegate.append((InternalRow) record); + } + InternalRow row = pojoToRowConverter.toRow(record); + return delegate.append(row); + } +} diff --git a/fluss-client/src/main/java/org/apache/fluss/client/table/writer/TypedUpsertWriter.java b/fluss-client/src/main/java/org/apache/fluss/client/table/writer/TypedUpsertWriter.java index f240c6b27c..4d241f5e61 100644 --- a/fluss-client/src/main/java/org/apache/fluss/client/table/writer/TypedUpsertWriter.java +++ b/fluss-client/src/main/java/org/apache/fluss/client/table/writer/TypedUpsertWriter.java @@ -17,117 +17,32 @@ package org.apache.fluss.client.table.writer; -import org.apache.fluss.client.converter.PojoToRowConverter; -import org.apache.fluss.metadata.TableInfo; -import org.apache.fluss.row.GenericRow; -import org.apache.fluss.row.InternalRow; -import org.apache.fluss.types.RowType; - -import javax.annotation.Nullable; +import org.apache.fluss.annotation.PublicEvolving; import java.util.concurrent.CompletableFuture; /** - * A typed {@link UpsertWriter} that converts POJOs to {@link InternalRow} and delegates to the - * existing internal-row based writer implementation. + * The typed writer to write data to the primary key table using POJOs. + * + * @param the type of the record + * @since 0.6 */ -class TypedUpsertWriter implements UpsertWriter { - - @Override - public void flush() { - delegate.flush(); - } - - private final UpsertWriterImpl delegate; - private final Class pojoClass; - private final TableInfo tableInfo; - private final RowType tableSchema; - private final int[] targetColumns; // may be null - - private final RowType pkProjection; - @Nullable private final RowType targetProjection; - - private final PojoToRowConverter pojoToRowConverter; - private final PojoToRowConverter pkConverter; - @Nullable private final PojoToRowConverter targetConverter; - - TypedUpsertWriter( - UpsertWriterImpl delegate, - Class pojoClass, - TableInfo tableInfo, - int[] targetColumns) { - this.delegate = delegate; - this.pojoClass = pojoClass; - this.tableInfo = tableInfo; - this.tableSchema = tableInfo.getRowType(); - this.targetColumns = targetColumns; - - // Precompute projections - this.pkProjection = this.tableSchema.project(tableInfo.getPhysicalPrimaryKeys()); - this.targetProjection = - (targetColumns == null) ? null : this.tableSchema.project(targetColumns); - - // Initialize reusable converters - this.pojoToRowConverter = PojoToRowConverter.of(pojoClass, tableSchema, tableSchema); - this.pkConverter = PojoToRowConverter.of(pojoClass, tableSchema, pkProjection); - this.targetConverter = - (targetProjection == null) - ? null - : PojoToRowConverter.of(pojoClass, tableSchema, targetProjection); - } - - @Override - public CompletableFuture upsert(T record) { - if (record instanceof InternalRow) { - return delegate.upsert((InternalRow) record); - } - InternalRow row = convertPojo(record, /*forDelete=*/ false); - return delegate.upsert(row); - } - - @Override - public CompletableFuture delete(T record) { - if (record instanceof InternalRow) { - return delegate.delete((InternalRow) record); - } - InternalRow pkOnly = convertPojo(record, /*forDelete=*/ true); - return delegate.delete(pkOnly); - } - - private InternalRow convertPojo(T pojo, boolean forDelete) { - final RowType projection; - final PojoToRowConverter converter; - if (forDelete) { - projection = pkProjection; - converter = pkConverter; - } else if (targetProjection != null && targetConverter != null) { - projection = targetProjection; - converter = targetConverter; - } else { - projection = tableSchema; - converter = pojoToRowConverter; - } - - GenericRow projected = converter.toRow(pojo); - if (projection == tableSchema) { - return projected; - } - // expand projected row to full row if needed - GenericRow full = new GenericRow(tableSchema.getFieldCount()); - if (forDelete) { - // set PK fields, others null - for (String pk : tableInfo.getPhysicalPrimaryKeys()) { - int projIndex = projection.getFieldIndex(pk); - int fullIndex = tableSchema.getFieldIndex(pk); - full.setField(fullIndex, projected.getField(projIndex)); - } - } else if (targetColumns != null) { - for (int i = 0; i < projection.getFieldCount(); i++) { - String name = projection.getFieldNames().get(i); - int fullIdx = tableSchema.getFieldIndex(name); - full.setField(fullIdx, projected.getField(i)); - } - } - return full; - } +@PublicEvolving +public interface TypedUpsertWriter extends TableWriter { + + /** + * Inserts a record into Fluss table if it does not already exist, or updates it if it does. + * + * @param record the record to upsert. + * @return A {@link CompletableFuture} that always returns upsert result when complete normally. + */ + CompletableFuture upsert(T record); + + /** + * Delete a certain record from the Fluss table. The input must contain the primary key fields. + * + * @param record the record to delete. + * @return A {@link CompletableFuture} that always delete result when complete normally. + */ + CompletableFuture delete(T record); } diff --git a/fluss-client/src/main/java/org/apache/fluss/client/table/writer/TypedUpsertWriterImpl.java b/fluss-client/src/main/java/org/apache/fluss/client/table/writer/TypedUpsertWriterImpl.java new file mode 100644 index 0000000000..1d679e57e7 --- /dev/null +++ b/fluss-client/src/main/java/org/apache/fluss/client/table/writer/TypedUpsertWriterImpl.java @@ -0,0 +1,139 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.fluss.client.table.writer; + +import org.apache.fluss.client.converter.PojoToRowConverter; +import org.apache.fluss.metadata.TableInfo; +import org.apache.fluss.row.GenericRow; +import org.apache.fluss.row.InternalRow; +import org.apache.fluss.types.RowType; + +import javax.annotation.Nullable; + +import java.util.concurrent.CompletableFuture; + +/** + * A typed {@link UpsertWriter} that converts POJOs to {@link InternalRow} and delegates to the + * existing internal-row based writer implementation. + */ +class TypedUpsertWriterImpl implements TypedUpsertWriter { + + @Override + public void flush() { + delegate.flush(); + } + + private final UpsertWriterImpl delegate; + + @Override + public void close() throws Exception { + delegate.close(); + } + + private final Class pojoClass; + private final TableInfo tableInfo; + private final RowType tableSchema; + private final int[] targetColumns; // may be null + + private final RowType pkProjection; + @Nullable private final RowType targetProjection; + + private final PojoToRowConverter pojoToRowConverter; + private final PojoToRowConverter pkConverter; + @Nullable private final PojoToRowConverter targetConverter; + + TypedUpsertWriterImpl( + UpsertWriterImpl delegate, + Class pojoClass, + TableInfo tableInfo, + int[] targetColumns) { + this.delegate = delegate; + this.pojoClass = pojoClass; + this.tableInfo = tableInfo; + this.tableSchema = tableInfo.getRowType(); + this.targetColumns = targetColumns; + + // Precompute projections + this.pkProjection = this.tableSchema.project(tableInfo.getPhysicalPrimaryKeys()); + this.targetProjection = + (targetColumns == null) ? null : this.tableSchema.project(targetColumns); + + // Initialize reusable converters + this.pojoToRowConverter = PojoToRowConverter.of(pojoClass, tableSchema, tableSchema); + this.pkConverter = PojoToRowConverter.of(pojoClass, tableSchema, pkProjection); + this.targetConverter = + (targetProjection == null) + ? null + : PojoToRowConverter.of(pojoClass, tableSchema, targetProjection); + } + + @Override + public CompletableFuture upsert(T record) { + if (record instanceof InternalRow) { + return delegate.upsert((InternalRow) record); + } + InternalRow row = convertPojo(record, /*forDelete=*/ false); + return delegate.upsert(row); + } + + @Override + public CompletableFuture delete(T record) { + if (record instanceof InternalRow) { + return delegate.delete((InternalRow) record); + } + InternalRow pkOnly = convertPojo(record, /*forDelete=*/ true); + return delegate.delete(pkOnly); + } + + private InternalRow convertPojo(T pojo, boolean forDelete) { + final RowType projection; + final PojoToRowConverter converter; + if (forDelete) { + projection = pkProjection; + converter = pkConverter; + } else if (targetProjection != null && targetConverter != null) { + projection = targetProjection; + converter = targetConverter; + } else { + projection = tableSchema; + converter = pojoToRowConverter; + } + + GenericRow projected = converter.toRow(pojo); + if (projection == tableSchema) { + return projected; + } + // expand projected row to full row if needed + GenericRow full = new GenericRow(tableSchema.getFieldCount()); + if (forDelete) { + // set PK fields, others null + for (String pk : tableInfo.getPhysicalPrimaryKeys()) { + int projIndex = projection.getFieldIndex(pk); + int fullIndex = tableSchema.getFieldIndex(pk); + full.setField(fullIndex, projected.getField(projIndex)); + } + } else if (targetColumns != null) { + for (int i = 0; i < projection.getFieldCount(); i++) { + String name = projection.getFieldNames().get(i); + int fullIdx = tableSchema.getFieldIndex(name); + full.setField(fullIdx, projected.getField(i)); + } + } + return full; + } +} diff --git a/fluss-client/src/main/java/org/apache/fluss/client/table/writer/Upsert.java b/fluss-client/src/main/java/org/apache/fluss/client/table/writer/Upsert.java index c5e3c1c762..c29875faf4 100644 --- a/fluss-client/src/main/java/org/apache/fluss/client/table/writer/Upsert.java +++ b/fluss-client/src/main/java/org/apache/fluss/client/table/writer/Upsert.java @@ -60,8 +60,8 @@ public interface Upsert { * Create a new {@link UpsertWriter} using {@code InternalRow} with the optional {@link * #partialUpdate(String...)} information to upsert and delete data to a Primary Key Table. */ - UpsertWriter createWriter(); + UpsertWriter createWriter(); /** Create a new typed {@link UpsertWriter} to write POJOs directly. */ - UpsertWriter createWriter(Class pojoClass); + TypedUpsertWriter createWriter(Class pojoClass); } 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 c0a4c51f2d..e4d751747d 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 @@ -28,27 +28,21 @@ * @since 0.2 */ @PublicEvolving -public interface UpsertWriter extends TableWriter { +public interface UpsertWriter extends TableWriter { /** * Inserts a record into Fluss table if it does not already exist, or updates it if it does. * - *

If {@code T} is {@link InternalRow}, the row will be written directly. Otherwise, the - * client will convert the POJO into an {@link InternalRow} using the configured converters. - * * @param record the record to upsert. * @return A {@link CompletableFuture} that always returns upsert result when complete normally. */ - CompletableFuture upsert(T record); + CompletableFuture upsert(InternalRow record); /** * Delete a certain record from the Fluss table. The input must contain the primary key fields. * - *

If {@code T} is {@link InternalRow}, the row will be used directly. Otherwise, the client - * will extract primary key fields from the POJO using the configured converters. - * * @param record the record to delete. * @return A {@link CompletableFuture} that always delete result when complete normally. */ - CompletableFuture delete(T record); + CompletableFuture delete(InternalRow record); } 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 626a985f3b..39f65592c1 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 @@ -39,7 +39,7 @@ import java.util.concurrent.CompletableFuture; /** The writer to write data to the primary key table. */ -class UpsertWriterImpl extends AbstractTableWriter implements UpsertWriter { +class UpsertWriterImpl extends AbstractTableWriter implements UpsertWriter { private static final UpsertResult UPSERT_SUCCESS = new UpsertResult(); private static final DeleteResult DELETE_SUCCESS = new DeleteResult(); diff --git a/fluss-client/src/main/java/org/apache/fluss/client/utils/MetadataUtils.java b/fluss-client/src/main/java/org/apache/fluss/client/utils/MetadataUtils.java index 2990054999..cf642beae9 100644 --- a/fluss-client/src/main/java/org/apache/fluss/client/utils/MetadataUtils.java +++ b/fluss-client/src/main/java/org/apache/fluss/client/utils/MetadataUtils.java @@ -24,6 +24,7 @@ import org.apache.fluss.exception.StaleMetadataException; import org.apache.fluss.metadata.PhysicalTablePath; import org.apache.fluss.metadata.TableBucket; +import org.apache.fluss.metadata.TableInfo; import org.apache.fluss.metadata.TablePath; import org.apache.fluss.rpc.GatewayClientProxy; import org.apache.fluss.rpc.RpcClient; @@ -125,6 +126,7 @@ public static Cluster sendMetadataRequestAndRebuildCluster( NewTableMetadata newTableMetadata = getTableMetadataToUpdate(originCluster, response); + Map newTableInfoByPath; if (partialUpdate) { // If partial update, we will clear the to be updated table out ot // the origin cluster. @@ -134,17 +136,20 @@ public static Cluster sendMetadataRequestAndRebuildCluster( new HashMap<>(originCluster.getBucketLocationsByPath()); newPartitionIdByPath = new HashMap<>(originCluster.getPartitionIdByPath()); + newTableInfoByPath = + new HashMap<>(originCluster.getTableInfoByPath()); newTablePathToTableId.putAll(newTableMetadata.tablePathToTableId); newBucketLocations.putAll(newTableMetadata.bucketLocations); newPartitionIdByPath.putAll(newTableMetadata.partitionIdByPath); - + newTableInfoByPath.putAll(newTableMetadata.tableInfoByPath); } else { // If full update, we will clear all tables info out ot the origin // cluster. newTablePathToTableId = newTableMetadata.tablePathToTableId; newBucketLocations = newTableMetadata.bucketLocations; newPartitionIdByPath = newTableMetadata.partitionIdByPath; + newTableInfoByPath = newTableMetadata.tableInfoByPath; } return new Cluster( @@ -152,7 +157,8 @@ public static Cluster sendMetadataRequestAndRebuildCluster( coordinatorServer, newBucketLocations, newTablePathToTableId, - newPartitionIdByPath); + newPartitionIdByPath, + newTableInfoByPath); }) .get(30, TimeUnit.SECONDS); // TODO currently, we don't have timeout logic in // RpcClient, it will let the get() block forever. So we @@ -164,6 +170,7 @@ private static NewTableMetadata getTableMetadataToUpdate( Map newTablePathToTableId = new HashMap<>(); Map> newBucketLocations = new HashMap<>(); Map newPartitionIdByPath = new HashMap<>(); + Map newTableInfoByPath = new HashMap<>(); // iterate all table metadata List pbTableMetadataList = metadataResponse.getTableMetadatasList(); @@ -211,21 +218,27 @@ private static NewTableMetadata getTableMetadataToUpdate( }); return new NewTableMetadata( - newTablePathToTableId, newBucketLocations, newPartitionIdByPath); + newTablePathToTableId, + newBucketLocations, + newPartitionIdByPath, + newTableInfoByPath); } private static final class NewTableMetadata { private final Map tablePathToTableId; private final Map> bucketLocations; private final Map partitionIdByPath; + private final Map tableInfoByPath; public NewTableMetadata( Map tablePathToTableId, Map> bucketLocations, - Map partitionIdByPath) { + Map partitionIdByPath, + Map tableInfoByPath) { this.tablePathToTableId = tablePathToTableId; this.bucketLocations = bucketLocations; this.partitionIdByPath = partitionIdByPath; + this.tableInfoByPath = tableInfoByPath; } } diff --git a/fluss-client/src/test/java/org/apache/fluss/client/admin/ClientToServerITCaseBase.java b/fluss-client/src/test/java/org/apache/fluss/client/admin/ClientToServerITCaseBase.java index c57ef9027c..440aedbd04 100644 --- a/fluss-client/src/test/java/org/apache/fluss/client/admin/ClientToServerITCaseBase.java +++ b/fluss-client/src/test/java/org/apache/fluss/client/admin/ClientToServerITCaseBase.java @@ -26,6 +26,7 @@ import org.apache.fluss.client.table.scanner.ScanRecord; import org.apache.fluss.client.table.scanner.log.LogScanner; import org.apache.fluss.client.table.scanner.log.ScanRecords; +import org.apache.fluss.client.table.scanner.log.TypedLogScanner; import org.apache.fluss.client.table.writer.UpsertWriter; import org.apache.fluss.config.ConfigOptions; import org.apache.fluss.config.Configuration; @@ -129,15 +130,22 @@ private static Configuration initConfig() { return conf; } - protected static LogScanner createLogScanner(Table table) { + protected static LogScanner createLogScanner(Table table) { return table.newScan().createLogScanner(); } - protected static LogScanner createLogScanner(Table table, int[] projectFields) { + protected static LogScanner createLogScanner(Table table, int[] projectFields) { return table.newScan().project(projectFields).createLogScanner(); } - protected static void subscribeFromBeginning(LogScanner logScanner, Table table) { + protected static void subscribeFromBeginning(LogScanner logScanner, Table table) { + int bucketCount = table.getTableInfo().getNumBuckets(); + for (int i = 0; i < bucketCount; i++) { + logScanner.subscribeFromBeginning(i); + } + } + + protected static void subscribeFromBeginning(TypedLogScanner logScanner, Table table) { int bucketCount = table.getTableInfo().getNumBuckets(); for (int i = 0; i < bucketCount; i++) { logScanner.subscribeFromBeginning(i); @@ -272,14 +280,13 @@ protected static void verifyPutAndLookup(Table table, Object[] fields) throws Ex upsertWriter.upsert(row); upsertWriter.flush(); // lookup this key. - Lookuper lookuper = table.newLookup().createLookuper(); + Lookuper lookuper = table.newLookup().createLookuper(); ProjectedRow keyRow = ProjectedRow.from(schema.getPrimaryKeyIndexes()); keyRow.replaceRow(row); assertThatRow(lookupRow(lookuper, keyRow)).withSchema(schema.getRowType()).isEqualTo(row); } - protected static InternalRow lookupRow(Lookuper lookuper, InternalRow keyRow) - throws Exception { + protected static InternalRow lookupRow(Lookuper lookuper, InternalRow keyRow) throws Exception { // lookup this key. return lookuper.lookup(keyRow).get().getSingletonRow(); } diff --git a/fluss-client/src/test/java/org/apache/fluss/client/metadata/MetadataUpdaterITCase.java b/fluss-client/src/test/java/org/apache/fluss/client/metadata/MetadataUpdaterITCase.java index 6d099a24e6..caecbe5452 100644 --- a/fluss-client/src/test/java/org/apache/fluss/client/metadata/MetadataUpdaterITCase.java +++ b/fluss-client/src/test/java/org/apache/fluss/client/metadata/MetadataUpdaterITCase.java @@ -108,6 +108,7 @@ void testUpdateWithEmptyMetadataResponse() throws Exception { null, Collections.emptyMap(), Collections.emptyMap(), + Collections.emptyMap(), Collections.emptyMap()); metadataUpdater = new MetadataUpdater(rpcClient, new Configuration(), newCluster); diff --git a/fluss-client/src/test/java/org/apache/fluss/client/metadata/TestingMetadataUpdater.java b/fluss-client/src/test/java/org/apache/fluss/client/metadata/TestingMetadataUpdater.java index 2063951357..f5e96bb515 100644 --- a/fluss-client/src/test/java/org/apache/fluss/client/metadata/TestingMetadataUpdater.java +++ b/fluss-client/src/test/java/org/apache/fluss/client/metadata/TestingMetadataUpdater.java @@ -218,6 +218,7 @@ private void initializeCluster( coordinatorServer, tablePathToBucketLocations, tableIdByPath, - Collections.emptyMap()); + Collections.emptyMap(), + tableInfos); } } diff --git a/fluss-client/src/test/java/org/apache/fluss/client/table/AutoPartitionedTableITCase.java b/fluss-client/src/test/java/org/apache/fluss/client/table/AutoPartitionedTableITCase.java index d3166b1c09..b6d46337cf 100644 --- a/fluss-client/src/test/java/org/apache/fluss/client/table/AutoPartitionedTableITCase.java +++ b/fluss-client/src/test/java/org/apache/fluss/client/table/AutoPartitionedTableITCase.java @@ -93,7 +93,7 @@ void testPartitionedPrimaryKeyTable() throws Exception { } upsertWriter.flush(); - Lookuper lookuper = table.newLookup().createLookuper(); + Lookuper lookuper = table.newLookup().createLookuper(); // now, let's lookup the written data by look up for (String partition : partitionIdByNames.keySet()) { for (int i = 0; i < recordsPerPartition; i++) { @@ -285,7 +285,7 @@ private Map> writeRows( } private Map> pollRecords( - LogScanner logScanner, int expectRecordsCount) { + LogScanner logScanner, int expectRecordsCount) { int scanRecordCount = 0; Map> actualRows = new HashMap<>(); while (scanRecordCount < expectRecordsCount) { @@ -308,7 +308,7 @@ void testOperateNotExistPartitionShouldThrowException() throws Exception { createPartitionedTable(DATA1_TABLE_PATH_PK, true); Table table = conn.getTable(DATA1_TABLE_PATH_PK); String partitionName = "notExistPartition"; - Lookuper lookuper = table.newLookup().createLookuper(); + Lookuper lookuper = table.newLookup().createLookuper(); // lookup a not exist partition will return null. assertThat(lookuper.lookup(row(1, partitionName)).get().getSingletonRow()).isEqualTo(null); @@ -324,7 +324,7 @@ void testOperateNotExistPartitionShouldThrowException() throws Exception { PhysicalTablePath.of(DATA1_TABLE_PATH_PK, partitionName)); // test scan a not exist partition's log - LogScanner logScanner = table.newScan().createLogScanner(); + LogScanner logScanner = table.newScan().createLogScanner(); assertThatThrownBy(() -> logScanner.subscribe(100L, 0, 0)) .isInstanceOf(PartitionNotExistException.class) .hasMessageContaining("The partition id '100' does not exist"); diff --git a/fluss-client/src/test/java/org/apache/fluss/client/table/FlussFailServerTableITCase.java b/fluss-client/src/test/java/org/apache/fluss/client/table/FlussFailServerTableITCase.java index ccc22af76e..4c52ef5f56 100644 --- a/fluss-client/src/test/java/org/apache/fluss/client/table/FlussFailServerTableITCase.java +++ b/fluss-client/src/test/java/org/apache/fluss/client/table/FlussFailServerTableITCase.java @@ -109,7 +109,7 @@ void testLogScan() throws Exception { // append one row. GenericRow row = row(1, "a"); try (Table table = conn.getTable(DATA1_TABLE_PATH); - LogScanner logScanner = createLogScanner(table)) { + LogScanner logScanner = createLogScanner(table)) { subscribeFromBeginning(logScanner, table); AppendWriter appendWriter = table.newAppend().createWriter(); appendWriter.append(row).get(); diff --git a/fluss-client/src/test/java/org/apache/fluss/client/table/FlussLakeTableITCase.java b/fluss-client/src/test/java/org/apache/fluss/client/table/FlussLakeTableITCase.java index 4e41c80ddd..a1fbdc772e 100644 --- a/fluss-client/src/test/java/org/apache/fluss/client/table/FlussLakeTableITCase.java +++ b/fluss-client/src/test/java/org/apache/fluss/client/table/FlussLakeTableITCase.java @@ -138,7 +138,7 @@ void testDeleteOnPrimaryKeyTable() throws Exception { tableWriter.delete(row(2, null)); tableWriter.flush(); - Lookuper lookuper = table.newLookup().createLookuper(); + Lookuper lookuper = table.newLookup().createLookuper(); List row1 = lookuper.lookup(row(1)).get().getRowList(); assertThatRows(row1) .withSchema(TestData.DATA1_SCHEMA_PK.getRowType()) @@ -208,8 +208,7 @@ void testPrimaryKeyTable(boolean isPartitioned, boolean isDefaultBucketKey) thro } // lookup try (Table table = conn.getTable(tablePath)) { - Lookuper lookuper = - table.newLookup().lookupBy(lookUpColumns).createLookuper(); + Lookuper lookuper = table.newLookup().lookupBy(lookUpColumns).createLookuper(); for (InternalRow row : allRows) { GenericRow lookupKeyRow = new GenericRow(lookUpFieldGetter.size()); for (int i = 0; i < lookUpFieldGetter.size(); i++) { @@ -318,7 +317,7 @@ private Map> writeRowsAndVerifyBucket( int scanCount = 0; Map> actualRows = new HashMap<>(); try (Table table = conn.getTable(tablePath); - LogScanner logScanner = table.newScan().createLogScanner()) { + LogScanner logScanner = table.newScan().createLogScanner()) { for (int bucket = 0; bucket < DEFAULT_BUCKET_COUNT; bucket++) { if (partitionIdByNames != null) { for (long partitionId : partitionIdByNames.values()) { diff --git a/fluss-client/src/test/java/org/apache/fluss/client/table/FlussTableITCase.java b/fluss-client/src/test/java/org/apache/fluss/client/table/FlussTableITCase.java index 12963105c5..e9df7ac6e7 100644 --- a/fluss-client/src/test/java/org/apache/fluss/client/table/FlussTableITCase.java +++ b/fluss-client/src/test/java/org/apache/fluss/client/table/FlussTableITCase.java @@ -144,7 +144,7 @@ void testAppendWithSmallBuffer(boolean indexedFormat) throws Exception { appendWriter.flush(); // assert the written data - LogScanner logScanner = createLogScanner(table); + LogScanner logScanner = createLogScanner(table); subscribeFromBeginning(logScanner, table); int count = 0; while (count < expectedSize) { @@ -178,7 +178,7 @@ void testPollOnce() throws Exception { appendWriter.flush(); // assert the written data - LogScanner logScanner = createLogScanner(table); + LogScanner logScanner = createLogScanner(table); subscribeFromBeginning(logScanner, table); int count = 0; while (count < expectedSize) { @@ -218,7 +218,7 @@ void testUpsertWithSmallBuffer() throws Exception { upsertWriter.flush(); // assert the written data - LogScanner logScanner = createLogScanner(table); + LogScanner logScanner = createLogScanner(table); subscribeFromBeginning(logScanner, table); int count = 0; while (count < expectedSize) { @@ -467,7 +467,7 @@ void testLookupForNotReadyTable() throws Exception { // if you want to test the lookup for not ready table, you can comment the following line. waitAllReplicasReady(tableId, 10); Table table = conn.getTable(tablePath); - Lookuper lookuper = table.newLookup().createLookuper(); + Lookuper lookuper = table.newLookup().createLookuper(); assertThat(lookupRow(lookuper, rowKey)).isNull(); } @@ -597,7 +597,7 @@ void testPartialPutAndDelete() throws Exception { UpsertWriter upsertWriter = table.newUpsert().partialUpdate(new int[] {0, 1}).createWriter(); upsertWriter.upsert(row(1, "aaa", null, null)).get(); - Lookuper lookuper = table.newLookup().createLookuper(); + Lookuper lookuper = table.newLookup().createLookuper(); // check the row GenericRow rowKey = row(1); @@ -696,7 +696,7 @@ void testDelete() throws Exception { try (Table table = conn.getTable(DATA1_TABLE_PATH_PK)) { UpsertWriter upsertWriter = table.newUpsert().createWriter(); upsertWriter.upsert(row).get(); - Lookuper lookuper = table.newLookup().createLookuper(); + Lookuper lookuper = table.newLookup().createLookuper(); // lookup this key. IndexedRow keyRow = keyRow(DATA1_SCHEMA_PK, new Object[] {1, "a"}); @@ -727,7 +727,7 @@ void testAppendWhileTableMaybeNotReady() throws Exception { appendWriter.append(row).get(); // fetch data. - LogScanner logScanner = createLogScanner(table); + LogScanner logScanner = createLogScanner(table); subscribeFromBeginning(logScanner, table); InternalRow result = null; while (result == null) { @@ -808,7 +808,7 @@ void verifyAppendOrPut(boolean append, String logFormat, @Nullable String kvForm // fetch data. try (Table table = conn.getTable(DATA1_TABLE_PATH); - LogScanner logScanner = createLogScanner(table)) { + LogScanner logScanner = createLogScanner(table)) { subscribeFromBeginning(logScanner, table); int count = 0; while (count < expectedSize) { @@ -869,7 +869,7 @@ void testAppendAndProject(String format) throws Exception { } // fetch data. - LogScanner logScanner = createLogScanner(table, new int[] {0, 2}); + LogScanner logScanner = createLogScanner(table, new int[] {0, 2}); subscribeFromBeginning(logScanner, table); int count = 0; while (count < expectedSize) { @@ -1200,7 +1200,7 @@ void testFirstRowMergeEngine(boolean doProjection) throws Exception { upsertWriter.flush(); - Lookuper lookuper = table.newLookup().createLookuper(); + Lookuper lookuper = table.newLookup().createLookuper(); // now, get rows by lookup for (int id = 0; id < rows; id++) { InternalRow gotRow = lookuper.lookup(row(id)).get().getSingletonRow(); @@ -1211,7 +1211,7 @@ void testFirstRowMergeEngine(boolean doProjection) throws Exception { if (doProjection) { scan = scan.project(new int[] {0}); // do projection. } - LogScanner logScanner = scan.createLogScanner(); + LogScanner logScanner = scan.createLogScanner(); logScanner.subscribeFromBeginning(0); List> actualLogRecords = new ArrayList<>(0); @@ -1267,7 +1267,7 @@ void testArrowCompressionAndProject(String compression, String level) throws Exc } // fetch data without project. - LogScanner logScanner = createLogScanner(table); + LogScanner logScanner = createLogScanner(table); subscribeFromBeginning(logScanner, table); int count = 0; while (count < expectedSize) { @@ -1383,7 +1383,7 @@ void testMergeEngineWithVersion(boolean doProjection) throws Exception { if (doProjection) { scan = scan.project(new int[] {0}); // do projection. } - LogScanner logScanner = scan.createLogScanner(); + LogScanner logScanner = scan.createLogScanner(); logScanner.subscribeFromBeginning(0); List> actualLogRecords = new ArrayList<>(rows); while (actualLogRecords.size() < rows) { diff --git a/fluss-client/src/test/java/org/apache/fluss/client/table/FlussTypedClientITCase.java b/fluss-client/src/test/java/org/apache/fluss/client/table/FlussTypedClientITCase.java index ce65ad8ae7..8a5227cf7a 100644 --- a/fluss-client/src/test/java/org/apache/fluss/client/table/FlussTypedClientITCase.java +++ b/fluss-client/src/test/java/org/apache/fluss/client/table/FlussTypedClientITCase.java @@ -21,19 +21,19 @@ import org.apache.fluss.client.converter.RowToPojoConverter; import org.apache.fluss.client.lookup.LookupResult; import org.apache.fluss.client.lookup.Lookuper; +import org.apache.fluss.client.lookup.TypedLookuper; import org.apache.fluss.client.table.scanner.Scan; import org.apache.fluss.client.table.scanner.ScanRecord; -import org.apache.fluss.client.table.scanner.log.LogScanner; import org.apache.fluss.client.table.scanner.log.ScanRecords; -import org.apache.fluss.client.table.writer.AppendWriter; +import org.apache.fluss.client.table.scanner.log.TypedLogScanner; +import org.apache.fluss.client.table.writer.TypedAppendWriter; +import org.apache.fluss.client.table.writer.TypedUpsertWriter; import org.apache.fluss.client.table.writer.Upsert; -import org.apache.fluss.client.table.writer.UpsertWriter; import org.apache.fluss.metadata.Schema; import org.apache.fluss.metadata.TableDescriptor; import org.apache.fluss.metadata.TablePath; import org.apache.fluss.record.ChangeType; import org.apache.fluss.row.GenericRow; -import org.apache.fluss.row.InternalRow; import org.apache.fluss.types.DataTypes; import org.apache.fluss.types.RowType; @@ -71,11 +71,11 @@ public static class AllTypesPojo { public String str; public byte[] bin; public byte[] bytes; - public java.math.BigDecimal dec; - public java.time.LocalDate dt; - public java.time.LocalTime tm; - public java.time.LocalDateTime tsNtz; - public java.time.Instant tsLtz; + public BigDecimal dec; + public LocalDate dt; + public LocalTime tm; + public LocalDateTime tsNtz; + public Instant tsLtz; public AllTypesPojo() {} @@ -92,11 +92,11 @@ public AllTypesPojo( String str, byte[] bin, byte[] bytes, - java.math.BigDecimal dec, - java.time.LocalDate dt, - java.time.LocalTime tm, - java.time.LocalDateTime tsNtz, - java.time.Instant tsLtz) { + BigDecimal dec, + LocalDate dt, + LocalTime tm, + LocalDateTime tsNtz, + Instant tsLtz) { this.a = a; this.bool1 = bool1; this.tiny = tiny; @@ -226,7 +226,8 @@ void testTypedAppendWriteAndScan() throws Exception { try (Table table = conn.getTable(path)) { // write - AppendWriter writer = table.newAppend().createWriter(AllTypesPojo.class); + TypedAppendWriter writer = + table.newAppend().createWriter(AllTypesPojo.class); List expected = new ArrayList<>(); for (int i = 0; i < 5; i++) { AllTypesPojo u = newAllTypesPojo(i); @@ -237,7 +238,7 @@ void testTypedAppendWriteAndScan() throws Exception { // read Scan scan = table.newScan(); - LogScanner scanner = scan.createLogScanner(AllTypesPojo.class); + TypedLogScanner scanner = scan.createTypedLogScanner(AllTypesPojo.class); subscribeFromBeginning(scanner, table); List actual = new ArrayList<>(); @@ -264,7 +265,7 @@ void testTypedUpsertWriteAndScan() throws Exception { try (Table table = conn.getTable(path)) { Upsert upsert = table.newUpsert(); - UpsertWriter writer = upsert.createWriter(AllTypesPojo.class); + TypedUpsertWriter writer = upsert.createWriter(AllTypesPojo.class); AllTypesPojo p1 = newAllTypesPojo(1); AllTypesPojo p2 = newAllTypesPojo(2); @@ -279,7 +280,8 @@ void testTypedUpsertWriteAndScan() throws Exception { writer.flush(); // scan as POJOs and verify change types and values - LogScanner scanner = table.newScan().createLogScanner(AllTypesPojo.class); + TypedLogScanner scanner = + table.newScan().createTypedLogScanner(AllTypesPojo.class); subscribeFromBeginning(scanner, table); List changes = new ArrayList<>(); @@ -307,13 +309,15 @@ void testTypedLookups() throws Exception { createTable(path, td, true); try (Table table = conn.getTable(path)) { - UpsertWriter writer = table.newUpsert().createWriter(AllTypesPojo.class); + TypedUpsertWriter writer = + table.newUpsert().createWriter(AllTypesPojo.class); writer.upsert(newAllTypesPojo(1)).get(); writer.upsert(newAllTypesPojo(2)).get(); - writer.flush(); + writer.close(); // primary key lookup using Lookuper API with POJO key - Lookuper lookuper = table.newLookup().createLookuper(); + TypedLookuper lookuper = + table.newLookup().createTypedLookuper(PLookupKey.class); RowType tableSchema = table.getTableInfo().getRowType(); RowToPojoConverter rowConv = RowToPojoConverter.of(AllTypesPojo.class, tableSchema, tableSchema); @@ -333,13 +337,14 @@ void testInternalRowLookup() throws Exception { try (Table table = conn.getTable(path)) { // write a couple of rows via POJO writer - UpsertWriter writer = table.newUpsert().createWriter(AllTypesPojo.class); + TypedUpsertWriter writer = + table.newUpsert().createWriter(AllTypesPojo.class); writer.upsert(newAllTypesPojo(101)).get(); writer.upsert(newAllTypesPojo(202)).get(); - writer.flush(); + writer.close(); // now perform lookup using the raw InternalRow path to ensure it's still supported - Lookuper lookuper = table.newLookup().createLookuper(); + Lookuper lookuper = table.newLookup().createLookuper(); RowType tableSchema = table.getTableInfo().getRowType(); RowType keyProjection = tableSchema.project(table.getTableInfo().getPrimaryKeys()); @@ -365,16 +370,17 @@ void testTypedProjections() throws Exception { createTable(path, td, true); try (Table table = conn.getTable(path)) { - AppendWriter writer = table.newAppend().createWriter(AllTypesPojo.class); + TypedAppendWriter writer = + table.newAppend().createWriter(AllTypesPojo.class); writer.append(newAllTypesPojo(10)).get(); writer.append(newAllTypesPojo(11)).get(); writer.flush(); // Project only a subset of fields - LogScanner scanner = + TypedLogScanner scanner = table.newScan() .project(Arrays.asList("a", "str")) - .createLogScanner(AllTypesPojo.class); + .createTypedLogScanner(AllTypesPojo.class); subscribeFromBeginning(scanner, table); ScanRecords recs = scanner.poll(Duration.ofSeconds(2)); for (ScanRecord r : recs) { @@ -404,7 +410,7 @@ void testTypedPartialUpdates() throws Exception { try (Table table = conn.getTable(path)) { Upsert upsert = table.newUpsert().partialUpdate("a", "str", "dec"); - UpsertWriter writer = upsert.createWriter(AllTypesPojo.class); + TypedUpsertWriter writer = upsert.createWriter(AllTypesPojo.class); // initial full row writer.upsert(newAllTypesPojo(1)).get(); @@ -415,10 +421,11 @@ void testTypedPartialUpdates() throws Exception { patch.str = "second"; patch.dec = new java.math.BigDecimal("99.99"); writer.upsert(patch).get(); - writer.flush(); + writer.close(); // verify via lookup and scan using Lookuper + POJO key - Lookuper lookuper = table.newLookup().createLookuper(); + TypedLookuper lookuper = + table.newLookup().createTypedLookuper(PLookupKey.class); RowType tableSchema = table.getTableInfo().getRowType(); RowToPojoConverter rowConv = RowToPojoConverter.of(AllTypesPojo.class, tableSchema, tableSchema); @@ -427,7 +434,8 @@ void testTypedPartialUpdates() throws Exception { assertThat(lookedUp.str).isEqualTo("second"); assertThat(lookedUp.dec).isEqualByComparingTo("99.99"); - LogScanner scanner = table.newScan().createLogScanner(AllTypesPojo.class); + TypedLogScanner scanner = + table.newScan().createTypedLogScanner(AllTypesPojo.class); subscribeFromBeginning(scanner, table); boolean sawUpdateAfter = false; while (!sawUpdateAfter) { diff --git a/fluss-client/src/test/java/org/apache/fluss/client/table/PartitionedTableITCase.java b/fluss-client/src/test/java/org/apache/fluss/client/table/PartitionedTableITCase.java index 438a9c0ed6..d204d087a4 100644 --- a/fluss-client/src/test/java/org/apache/fluss/client/table/PartitionedTableITCase.java +++ b/fluss-client/src/test/java/org/apache/fluss/client/table/PartitionedTableITCase.java @@ -90,7 +90,7 @@ void testPartitionedPrimaryKeyTable() throws Exception { upsertWriter.flush(); - Lookuper lookuper = table.newLookup().createLookuper(); + Lookuper lookuper = table.newLookup().createLookuper(); // now, let's lookup the written data by look up. for (PartitionInfo partitionInfo : partitionInfos) { String partitionName = partitionInfo.getPartitionName(); diff --git a/fluss-client/src/test/java/org/apache/fluss/client/table/scanner/log/DefaultCompletedFetchTest.java b/fluss-client/src/test/java/org/apache/fluss/client/table/scanner/log/DefaultCompletedFetchTest.java index 3c8f3a7674..9e2b5349c2 100644 --- a/fluss-client/src/test/java/org/apache/fluss/client/table/scanner/log/DefaultCompletedFetchTest.java +++ b/fluss-client/src/test/java/org/apache/fluss/client/table/scanner/log/DefaultCompletedFetchTest.java @@ -283,7 +283,7 @@ void testComplexTypeFetch() throws Exception { logScannerStatus, true, fetchOffset); - List scanRecords = defaultCompletedFetch.fetchRecords(3); + List> scanRecords = defaultCompletedFetch.fetchRecords(3); // close the read context to release arrow root resource, // this is important to test complex types defaultCompletedFetch.readContext.close(); diff --git a/fluss-client/src/test/java/org/apache/fluss/client/table/scanner/log/LogFetcherITCase.java b/fluss-client/src/test/java/org/apache/fluss/client/table/scanner/log/LogFetcherITCase.java index b6817b327b..7b4f824e25 100644 --- a/fluss-client/src/test/java/org/apache/fluss/client/table/scanner/log/LogFetcherITCase.java +++ b/fluss-client/src/test/java/org/apache/fluss/client/table/scanner/log/LogFetcherITCase.java @@ -258,7 +258,8 @@ void testFetchWhenDestinationIsNullInMetadata() throws Exception { oldCluster.getCoordinatorServer(), oldCluster.getBucketLocationsByPath(), oldCluster.getTableIdByPath(), - oldCluster.getPartitionIdByPath()); + oldCluster.getPartitionIdByPath(), + oldCluster.getTableInfoByPath()); metadataUpdater = new MetadataUpdater(rpcClient, clientConf, newCluster); LogScannerStatus logScannerStatus = new LogScannerStatus(); diff --git a/fluss-client/src/test/java/org/apache/fluss/client/table/scanner/log/LogScannerITCase.java b/fluss-client/src/test/java/org/apache/fluss/client/table/scanner/log/LogScannerITCase.java index 5540760197..9ead41f42b 100644 --- a/fluss-client/src/test/java/org/apache/fluss/client/table/scanner/log/LogScannerITCase.java +++ b/fluss-client/src/test/java/org/apache/fluss/client/table/scanner/log/LogScannerITCase.java @@ -70,7 +70,7 @@ void testPoll() throws Exception { appendWriter.append(row).get(); } - LogScanner logScanner = createLogScanner(table); + LogScanner logScanner = createLogScanner(table); subscribeFromBeginning(logScanner, table); List rowList = new ArrayList<>(); while (rowList.size() < recordSize) { @@ -105,7 +105,7 @@ void testPollWhileCreateTableNotReady() throws Exception { appendWriter.append(row).get(); } - LogScanner logScanner = createLogScanner(table); + LogScanner logScanner = createLogScanner(table); subscribeFromBeginning(logScanner, table); List rowList = new ArrayList<>(); while (rowList.size() < recordSize) { @@ -136,7 +136,7 @@ void testLogScannerMultiThreadAccess() throws Exception { appendWriter.append(row).get(); } - LogScanner logScanner = table.newScan().createLogScanner(); + LogScanner logScanner = table.newScan().createLogScanner(); ExecutorService executor = Executors.newSingleThreadExecutor(); // subscribe in thread1 executor.submit(() -> logScanner.subscribe(0, LogScanner.EARLIEST_OFFSET)).get(); @@ -196,7 +196,7 @@ void testLogHeavyWriteAndScan() throws Exception { } appendWriter.flush(); - LogScanner logScanner = createLogScanner(table); + LogScanner logScanner = createLogScanner(table); subscribeFromBeginning(logScanner, table); long scanned = 0; long total = 0; @@ -252,7 +252,7 @@ void testKvHeavyWriteAndScan() throws Exception { } upsertWriter.flush(); - LogScanner logScanner = createLogScanner(table); + LogScanner logScanner = createLogScanner(table); subscribeFromBeginning(logScanner, table); long scanned = 0; long total = 0; @@ -320,7 +320,7 @@ void testScanFromStartTimestamp(boolean isPartitioned) throws Exception { // as early as possible to avoid potential time backwards long secondStartTimestamp = System.currentTimeMillis(); - LogScanner logScanner = createLogScanner(table); + LogScanner logScanner = createLogScanner(table); // try to fetch from firstStartTimestamp, which smaller than the first batch commit // timestamp. subscribeFromTimestamp( diff --git a/fluss-client/src/test/java/org/apache/fluss/client/table/scanner/log/RemoteLogScannerITCase.java b/fluss-client/src/test/java/org/apache/fluss/client/table/scanner/log/RemoteLogScannerITCase.java index b9a1fce7e6..b44bcb9929 100644 --- a/fluss-client/src/test/java/org/apache/fluss/client/table/scanner/log/RemoteLogScannerITCase.java +++ b/fluss-client/src/test/java/org/apache/fluss/client/table/scanner/log/RemoteLogScannerITCase.java @@ -99,7 +99,7 @@ void testScanFromRemote() throws Exception { FLUSS_CLUSTER_EXTENSION.waitUntilSomeLogSegmentsCopyToRemote(new TableBucket(tableId, 0)); // test fetch. - LogScanner logScanner = table.newScan().createLogScanner(); + LogScanner logScanner = table.newScan().createLogScanner(); logScanner.subscribeFromBeginning(0); List rowList = new ArrayList<>(); while (rowList.size() < recordSize) { @@ -150,7 +150,7 @@ void testScanFromRemoteAndProject(String format) throws Exception { FLUSS_CLUSTER_EXTENSION.waitUntilSomeLogSegmentsCopyToRemote(new TableBucket(tableId, 0)); // test fetch. - LogScanner logScanner = createLogScanner(table, new int[] {0, 2}); + LogScanner logScanner = createLogScanner(table, new int[] {0, 2}); logScanner.subscribeFromBeginning(0); int count = 0; while (count < expectedSize) { diff --git a/fluss-client/src/test/java/org/apache/fluss/client/write/RecordAccumulatorTest.java b/fluss-client/src/test/java/org/apache/fluss/client/write/RecordAccumulatorTest.java index 24f264c926..3844fb43a2 100644 --- a/fluss-client/src/test/java/org/apache/fluss/client/write/RecordAccumulatorTest.java +++ b/fluss-client/src/test/java/org/apache/fluss/client/write/RecordAccumulatorTest.java @@ -562,12 +562,15 @@ private Cluster updateCluster(List bucketLocations) { Map tableIdByPath = new HashMap<>(); tableIdByPath.put(DATA1_TABLE_PATH, DATA1_TABLE_ID); + Map tableInfoByPath = new HashMap<>(); + tableInfoByPath.put(DATA1_TABLE_PATH, DATA1_TABLE_INFO); return new Cluster( aliveTabletServersById, new ServerNode(0, "localhost", 89, ServerType.COORDINATOR), bucketsByPath, tableIdByPath, - Collections.emptyMap()); + Collections.emptyMap(), + tableInfoByPath); } private void delayedInterrupt(final Thread thread, final long delayMs) { diff --git a/fluss-client/src/test/java/org/apache/fluss/client/write/SenderTest.java b/fluss-client/src/test/java/org/apache/fluss/client/write/SenderTest.java index f1d9481a64..2d06057e80 100644 --- a/fluss-client/src/test/java/org/apache/fluss/client/write/SenderTest.java +++ b/fluss-client/src/test/java/org/apache/fluss/client/write/SenderTest.java @@ -671,7 +671,8 @@ void testSendWhenDestinationIsNullInMetadata() throws Exception { oldCluster.getCoordinatorServer(), oldCluster.getBucketLocationsByPath(), oldCluster.getTableIdByPath(), - oldCluster.getPartitionIdByPath()); + oldCluster.getPartitionIdByPath(), + oldCluster.getTableInfoByPath()); metadataUpdater.updateCluster(newCluster); diff --git a/fluss-client/src/test/java/org/apache/fluss/client/write/StickyStaticBucketAssignerTest.java b/fluss-client/src/test/java/org/apache/fluss/client/write/StickyStaticBucketAssignerTest.java index f828906339..6009b96f51 100644 --- a/fluss-client/src/test/java/org/apache/fluss/client/write/StickyStaticBucketAssignerTest.java +++ b/fluss-client/src/test/java/org/apache/fluss/client/write/StickyStaticBucketAssignerTest.java @@ -215,6 +215,7 @@ private Cluster updateCluster(List bucketLocations) { new ServerNode(0, "localhost", 89, ServerType.COORDINATOR), bucketsByPath, tableIdByPath, + Collections.emptyMap(), Collections.emptyMap()); } } diff --git a/fluss-common/src/main/java/org/apache/fluss/cluster/Cluster.java b/fluss-common/src/main/java/org/apache/fluss/cluster/Cluster.java index d5690326a0..a7a46df957 100644 --- a/fluss-common/src/main/java/org/apache/fluss/cluster/Cluster.java +++ b/fluss-common/src/main/java/org/apache/fluss/cluster/Cluster.java @@ -21,6 +21,7 @@ import org.apache.fluss.exception.PartitionNotExistException; import org.apache.fluss.metadata.PhysicalTablePath; import org.apache.fluss.metadata.TableBucket; +import org.apache.fluss.metadata.TableInfo; import org.apache.fluss.metadata.TablePath; import javax.annotation.Nullable; @@ -52,19 +53,22 @@ public final class Cluster { private final Map pathByTableId; private final Map partitionsIdByPath; private final Map partitionNameById; + private final Map tableInfoByPath; public Cluster( Map aliveTabletServersById, @Nullable ServerNode coordinatorServer, Map> bucketLocationsByPath, Map tableIdByPath, - Map partitionsIdByPath) { + Map partitionsIdByPath, + Map tableInfoByPath) { this.coordinatorServer = coordinatorServer; this.aliveTabletServersById = Collections.unmodifiableMap(aliveTabletServersById); this.aliveTabletServers = Collections.unmodifiableList(new ArrayList<>(aliveTabletServersById.values())); this.tableIdByPath = Collections.unmodifiableMap(tableIdByPath); this.partitionsIdByPath = Collections.unmodifiableMap(partitionsIdByPath); + this.tableInfoByPath = Collections.unmodifiableMap(tableInfoByPath); // Index the bucket locations by table path, and index bucket location by bucket. // Note that this code is performance sensitive if there are a large number of buckets, @@ -132,7 +136,8 @@ public Cluster invalidPhysicalTableBucketMeta(Set physicalTab coordinatorServer, newBucketLocationsByPath, new HashMap<>(tableIdByPath), - new HashMap<>(partitionsIdByPath)); + new HashMap<>(partitionsIdByPath), + new HashMap<>(tableInfoByPath)); } @Nullable @@ -166,6 +171,21 @@ public TablePath getTablePathOrElseThrow(long tableId) { + " in cluster")); } + /** Get the table info for this table path. */ + public Optional getTableInfo(TablePath tablePath) { + return Optional.ofNullable(tableInfoByPath.get(tablePath)); + } + + public TableInfo getTableInfoOrElseThrow(TablePath tablePath) { + return getTableInfo(tablePath) + .orElseThrow( + () -> + new IllegalArgumentException( + "table info not found for tablePath " + + tablePath + + " in cluster")); + } + /** Get the bucket location for this table-bucket. */ public Optional getBucketLocation(TableBucket tableBucket) { return Optional.ofNullable(availableLocationByBucket.get(tableBucket)); @@ -258,6 +278,10 @@ public Map getPartitionIdByPath() { return partitionsIdByPath; } + public Map getTableInfoByPath() { + return tableInfoByPath; + } + /** Create an empty cluster instance with no nodes and no table-buckets. */ public static Cluster empty() { return new Cluster( @@ -265,6 +289,7 @@ public static Cluster empty() { null, Collections.emptyMap(), Collections.emptyMap(), + Collections.emptyMap(), Collections.emptyMap()); } diff --git a/fluss-common/src/test/java/org/apache/fluss/cluster/ClusterTest.java b/fluss-common/src/test/java/org/apache/fluss/cluster/ClusterTest.java index ee23c9109a..d1ed342832 100644 --- a/fluss-common/src/test/java/org/apache/fluss/cluster/ClusterTest.java +++ b/fluss-common/src/test/java/org/apache/fluss/cluster/ClusterTest.java @@ -18,6 +18,8 @@ package org.apache.fluss.cluster; import org.apache.fluss.metadata.PhysicalTablePath; +import org.apache.fluss.metadata.SchemaInfo; +import org.apache.fluss.metadata.TableInfo; import org.apache.fluss.metadata.TablePath; import org.junit.jupiter.api.BeforeEach; @@ -32,9 +34,13 @@ import java.util.Set; import static org.apache.fluss.record.TestData.DATA1_PHYSICAL_TABLE_PATH; +import static org.apache.fluss.record.TestData.DATA1_SCHEMA; import static org.apache.fluss.record.TestData.DATA1_TABLE_ID; +import static org.apache.fluss.record.TestData.DATA1_TABLE_INFO; import static org.apache.fluss.record.TestData.DATA1_TABLE_PATH; +import static org.apache.fluss.record.TestData.DATA2_SCHEMA; import static org.apache.fluss.record.TestData.DATA2_TABLE_ID; +import static org.apache.fluss.record.TestData.DATA2_TABLE_INFO; import static org.apache.fluss.record.TestData.DATA2_TABLE_PATH; import static org.assertj.core.api.Assertions.assertThat; import static org.assertj.core.api.Assertions.assertThatThrownBy; @@ -81,6 +87,17 @@ void testReturnModifiableCollections() { .isInstanceOf(UnsupportedOperationException.class); } + @Test + void testGetTable() { + Cluster cluster = createCluster(aliveTabletServersById); + assertThat(cluster.getTableInfo(DATA1_TABLE_PATH).get()).isEqualTo(DATA1_TABLE_INFO); + assertThat(cluster.getTableInfo(DATA2_TABLE_PATH).get()).isEqualTo(DATA2_TABLE_INFO); + assertThat(new SchemaInfo(cluster.getTableInfo(DATA1_TABLE_PATH).get().getSchema(), 1)) + .isEqualTo(new SchemaInfo(DATA1_SCHEMA, 1)); + assertThat(new SchemaInfo(cluster.getTableInfo(DATA2_TABLE_PATH).get().getSchema(), 1)) + .isEqualTo(new SchemaInfo(DATA2_SCHEMA, 1)); + } + @Test void testInvalidMetaAndUpdate() { Cluster cluster = createCluster(aliveTabletServersById); @@ -96,7 +113,8 @@ void testInvalidMetaAndUpdate() { COORDINATOR_SERVER, new HashMap<>(cluster.getBucketLocationsByPath()), new HashMap<>(cluster.getTableIdByPath()), - Collections.emptyMap()); + Collections.emptyMap(), + new HashMap<>(cluster.getTableInfoByPath())); } // verify available buckets @@ -172,11 +190,16 @@ private Cluster createCluster(Map aliveTabletServersById) { tablePathToTableId.put(DATA1_TABLE_PATH, DATA1_TABLE_ID); tablePathToTableId.put(DATA2_TABLE_PATH, DATA2_TABLE_ID); + Map tablePathToTableInfo = new HashMap<>(); + tablePathToTableInfo.put(DATA1_TABLE_PATH, DATA1_TABLE_INFO); + tablePathToTableInfo.put(DATA2_TABLE_PATH, DATA2_TABLE_INFO); + return new Cluster( aliveTabletServersById, COORDINATOR_SERVER, tablePathToBucketLocations, tablePathToTableId, - Collections.emptyMap()); + Collections.emptyMap(), + tablePathToTableInfo); } } diff --git a/fluss-flink/fluss-flink-common/src/main/java/org/apache/fluss/flink/lake/reader/LakeSnapshotAndLogSplitScanner.java b/fluss-flink/fluss-flink-common/src/main/java/org/apache/fluss/flink/lake/reader/LakeSnapshotAndLogSplitScanner.java index d84963dc84..e734eb3083 100644 --- a/fluss-flink/fluss-flink-common/src/main/java/org/apache/fluss/flink/lake/reader/LakeSnapshotAndLogSplitScanner.java +++ b/fluss-flink/fluss-flink-common/src/main/java/org/apache/fluss/flink/lake/reader/LakeSnapshotAndLogSplitScanner.java @@ -65,7 +65,7 @@ public class LakeSnapshotAndLogSplitScanner implements BatchScanner { // the sorted logs in memory, mapping from key -> value private Map logRows; - private final LogScanner logScanner; + private final LogScanner logScanner; private final long stoppingOffset; private boolean logScanFinished; 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 edbc6fc73a..ae816556ed 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 @@ -295,7 +295,7 @@ void testAppendLogWithRoundRobin() throws Exception { Configuration clientConf = FLUSS_CLUSTER_EXTENSION.getClientConfig(); try (Connection conn = ConnectionFactory.createConnection(clientConf); Table table = conn.getTable(TablePath.of(DEFAULT_DB, "sink_test")); - LogScanner logScanner = table.newScan().createLogScanner()) { + LogScanner logScanner = table.newScan().createLogScanner()) { logScanner.subscribeFromBeginning(0); logScanner.subscribeFromBeginning(1); logScanner.subscribeFromBeginning(2); 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 bfebff0ddb..bde62abd0c 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 @@ -149,7 +149,7 @@ public void testRowDataTablePKSink() throws Exception { env.executeAsync("Test RowData Fluss Sink"); Table table = conn.getTable(new TablePath(DEFAULT_DB, pkTableName)); - LogScanner logScanner = table.newScan().createLogScanner(); + LogScanner logScanner = table.newScan().createLogScanner(); int numBuckets = table.getTableInfo().getNumBuckets(); for (int i = 0; i < numBuckets; i++) { @@ -222,7 +222,7 @@ public void testRowDataTableLogSink() throws Exception { env.executeAsync("Test RowData Fluss Sink"); Table table = conn.getTable(new TablePath(DEFAULT_DB, logTableName)); - LogScanner logScanner = table.newScan().createLogScanner(); + LogScanner logScanner = table.newScan().createLogScanner(); int numBuckets = table.getTableInfo().getNumBuckets(); for (int i = 0; i < numBuckets; i++) { From 8b8a33dde94973ac4306bb95260a690c76cd665b Mon Sep 17 00:00:00 2001 From: ipolyzos Date: Mon, 22 Dec 2025 10:53:46 +0200 Subject: [PATCH 28/33] add TypedScanRecords --- .../client/table/scanner/ScanRecord.java | 37 +++--- .../client/table/scanner/TypedScanRecord.java | 88 +++++++++++++ .../table/scanner/log/CompletedFetch.java | 6 +- .../table/scanner/log/LogFetchCollector.java | 18 +-- .../client/table/scanner/log/LogFetcher.java | 3 +- .../client/table/scanner/log/LogScanner.java | 3 +- .../table/scanner/log/LogScannerImpl.java | 15 +-- .../client/table/scanner/log/ScanRecords.java | 36 +++--- .../table/scanner/log/TypedLogScanner.java | 2 +- .../scanner/log/TypedLogScannerImpl.java | 22 ++-- .../table/scanner/log/TypedScanRecords.java | 120 ++++++++++++++++++ .../admin/ClientToServerITCaseBase.java | 6 +- .../table/AutoPartitionedTableITCase.java | 6 +- .../table/FlussFailServerTableITCase.java | 6 +- .../client/table/FlussLakeTableITCase.java | 4 +- .../fluss/client/table/FlussTableITCase.java | 50 ++++---- .../client/table/FlussTypedClientITCase.java | 22 ++-- .../log/DefaultCompletedFetchTest.java | 12 +- .../scanner/log/LogFetchCollectorTest.java | 5 +- .../table/scanner/log/LogScannerITCase.java | 32 ++--- .../scanner/log/RemoteCompletedFetchTest.java | 12 +- .../scanner/log/RemoteLogScannerITCase.java | 12 +- .../lake/reader/SortMergeReaderTest.java | 39 +----- .../flink/sink/FlinkTableSinkITCase.java | 4 +- .../FlussDeserializationSchemaTest.java | 19 ++- 25 files changed, 371 insertions(+), 208 deletions(-) create mode 100644 fluss-client/src/main/java/org/apache/fluss/client/table/scanner/TypedScanRecord.java create mode 100644 fluss-client/src/main/java/org/apache/fluss/client/table/scanner/log/TypedScanRecords.java diff --git a/fluss-client/src/main/java/org/apache/fluss/client/table/scanner/ScanRecord.java b/fluss-client/src/main/java/org/apache/fluss/client/table/scanner/ScanRecord.java index 0f96e06abf..d303b72329 100644 --- a/fluss-client/src/main/java/org/apache/fluss/client/table/scanner/ScanRecord.java +++ b/fluss-client/src/main/java/org/apache/fluss/client/table/scanner/ScanRecord.java @@ -19,51 +19,52 @@ import org.apache.fluss.annotation.Internal; import org.apache.fluss.record.ChangeType; +import org.apache.fluss.record.LogRecord; +import org.apache.fluss.row.InternalRow; import java.util.Objects; /** one scan record. */ +// TODO: replace this with GenericRecord in the future @Internal -public class ScanRecord { +public class ScanRecord implements LogRecord { private static final long INVALID = -1L; private final long offset; private final long timestamp; private final ChangeType changeType; - private final T value; + private final InternalRow row; - public ScanRecord(T value) { - this(INVALID, INVALID, ChangeType.INSERT, value); + public ScanRecord(InternalRow row) { + this(INVALID, INVALID, ChangeType.INSERT, row); } - public ScanRecord(long offset, long timestamp, ChangeType changeType, T value) { + public ScanRecord(long offset, long timestamp, ChangeType changeType, InternalRow row) { this.offset = offset; this.timestamp = timestamp; this.changeType = changeType; - this.value = value; + this.row = row; } /** The position of this record in the corresponding fluss table bucket. */ + @Override public long logOffset() { return offset; } + @Override public long timestamp() { return timestamp; } + @Override public ChangeType getChangeType() { return changeType; } - /** Returns the carried record as InternalRow for backward compatibility. */ - public org.apache.fluss.row.InternalRow getRow() { - return (org.apache.fluss.row.InternalRow) value; - } - - /** Returns the carried record value. */ - public T getValue() { - return value; + @Override + public InternalRow getRow() { + return row; } @Override @@ -74,19 +75,19 @@ public boolean equals(Object o) { if (o == null || getClass() != o.getClass()) { return false; } - ScanRecord that = (ScanRecord) o; + ScanRecord that = (ScanRecord) o; return offset == that.offset && changeType == that.changeType - && Objects.equals(value, that.value); + && Objects.equals(row, that.row); } @Override public int hashCode() { - return Objects.hash(offset, changeType, value); + return Objects.hash(offset, changeType, row); } @Override public String toString() { - return changeType.shortString() + value + "@" + offset; + return changeType.shortString() + row.toString() + "@" + offset; } } diff --git a/fluss-client/src/main/java/org/apache/fluss/client/table/scanner/TypedScanRecord.java b/fluss-client/src/main/java/org/apache/fluss/client/table/scanner/TypedScanRecord.java new file mode 100644 index 0000000000..5a2d3b8a3e --- /dev/null +++ b/fluss-client/src/main/java/org/apache/fluss/client/table/scanner/TypedScanRecord.java @@ -0,0 +1,88 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.fluss.client.table.scanner; + +import org.apache.fluss.annotation.PublicEvolving; +import org.apache.fluss.record.ChangeType; +import org.apache.fluss.row.InternalRow; + +import java.util.Objects; + +/** + * A record produced by a table scanner which contains a typed value. + * + * @param The type of the value. + */ +@PublicEvolving +public class TypedScanRecord { + + private final ScanRecord scanRecord; + private final T value; + + public TypedScanRecord(ScanRecord scanRecord, T value) { + this.scanRecord = scanRecord; + this.value = value; + } + + /** The position of this record in the corresponding fluss table bucket. */ + public long logOffset() { + return scanRecord.logOffset(); + } + + /** The timestamp of this record. */ + public long timestamp() { + return scanRecord.timestamp(); + } + + /** The change type of this record. */ + public ChangeType getChangeType() { + return scanRecord.getChangeType(); + } + + /** Returns the record value. */ + public T getValue() { + return value; + } + + /** Returns the internal row of this record. */ + public InternalRow getRow() { + return scanRecord.getRow(); + } + + @Override + public boolean equals(Object o) { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + TypedScanRecord that = (TypedScanRecord) o; + return Objects.equals(scanRecord, that.scanRecord) && Objects.equals(value, that.value); + } + + @Override + public int hashCode() { + return Objects.hash(scanRecord, value); + } + + @Override + public String toString() { + return scanRecord.getChangeType().shortString() + value + "@" + scanRecord.logOffset(); + } +} diff --git a/fluss-client/src/main/java/org/apache/fluss/client/table/scanner/log/CompletedFetch.java b/fluss-client/src/main/java/org/apache/fluss/client/table/scanner/log/CompletedFetch.java index a4c0ee4e90..8f29f3ef34 100644 --- a/fluss-client/src/main/java/org/apache/fluss/client/table/scanner/log/CompletedFetch.java +++ b/fluss-client/src/main/java/org/apache/fluss/client/table/scanner/log/CompletedFetch.java @@ -95,7 +95,7 @@ public CompletedFetch( // TODO: optimize this to avoid deep copying the record. // refactor #fetchRecords to return an iterator which lazily deserialize // from underlying record stream and arrow buffer. - ScanRecord toScanRecord(LogRecord record) { + ScanRecord toScanRecord(LogRecord record) { GenericRow newRow = new GenericRow(selectedFieldGetters.length); InternalRow internalRow = record.getRow(); for (int i = 0; i < selectedFieldGetters.length; i++) { @@ -148,7 +148,7 @@ void drain() { * maxRecords} * @return {@link ScanRecord scan records} */ - public List> fetchRecords(int maxRecords) { + public List fetchRecords(int maxRecords) { if (corruptLastRecord) { throw new FetchException( "Received exception when fetching the next record from " @@ -161,7 +161,7 @@ public List> fetchRecords(int maxRecords) { return Collections.emptyList(); } - List> scanRecords = new ArrayList<>(); + List scanRecords = new ArrayList<>(); try { for (int i = 0; i < maxRecords; i++) { // Only move to next record if there was no exception in the last fetch. diff --git a/fluss-client/src/main/java/org/apache/fluss/client/table/scanner/log/LogFetchCollector.java b/fluss-client/src/main/java/org/apache/fluss/client/table/scanner/log/LogFetchCollector.java index 87e738fc58..34a3ec86e6 100644 --- a/fluss-client/src/main/java/org/apache/fluss/client/table/scanner/log/LogFetchCollector.java +++ b/fluss-client/src/main/java/org/apache/fluss/client/table/scanner/log/LogFetchCollector.java @@ -29,7 +29,6 @@ import org.apache.fluss.metadata.TablePath; import org.apache.fluss.record.LogRecord; import org.apache.fluss.record.LogRecordBatch; -import org.apache.fluss.row.InternalRow; import org.apache.fluss.rpc.protocol.ApiError; import org.apache.fluss.rpc.protocol.Errors; @@ -84,9 +83,8 @@ public LogFetchCollector( * @throws LogOffsetOutOfRangeException If there is OffsetOutOfRange error in fetchResponse and * the defaultResetPolicy is NONE */ - public Map>> collectFetch( - final LogFetchBuffer logFetchBuffer) { - Map>> fetched = new HashMap<>(); + public Map> collectFetch(final LogFetchBuffer logFetchBuffer) { + Map> fetched = new HashMap<>(); int recordsRemaining = maxPollRecords; try { @@ -117,11 +115,10 @@ public Map>> collectFetch( logFetchBuffer.poll(); } else { - List> records = - fetchRecords(nextInLineFetch, recordsRemaining); + List records = fetchRecords(nextInLineFetch, recordsRemaining); if (!records.isEmpty()) { TableBucket tableBucket = nextInLineFetch.tableBucket; - List> currentRecords = fetched.get(tableBucket); + List currentRecords = fetched.get(tableBucket); if (currentRecords == null) { fetched.put(tableBucket, records); } else { @@ -129,7 +126,7 @@ public Map>> collectFetch( // a time per bucket, but it might conceivably happen in some rare // cases (such as bucket leader changes). we have to copy to a new list // because the old one may be immutable - List> newScanRecords = + List newScanRecords = new ArrayList<>(records.size() + currentRecords.size()); newScanRecords.addAll(currentRecords); newScanRecords.addAll(records); @@ -149,8 +146,7 @@ public Map>> collectFetch( return fetched; } - private List> fetchRecords( - CompletedFetch nextInLineFetch, int maxRecords) { + private List fetchRecords(CompletedFetch nextInLineFetch, int maxRecords) { TableBucket tb = nextInLineFetch.tableBucket; Long offset = logScannerStatus.getBucketOffset(tb); if (offset == null) { @@ -161,7 +157,7 @@ private List> fetchRecords( nextInLineFetch.nextFetchOffset()); } else { if (nextInLineFetch.nextFetchOffset() == offset) { - List> records = nextInLineFetch.fetchRecords(maxRecords); + List records = nextInLineFetch.fetchRecords(maxRecords); LOG.trace( "Returning {} fetched records at offset {} for assigned bucket {}.", records.size(), diff --git a/fluss-client/src/main/java/org/apache/fluss/client/table/scanner/log/LogFetcher.java b/fluss-client/src/main/java/org/apache/fluss/client/table/scanner/log/LogFetcher.java index c74d814468..a97bffac04 100644 --- a/fluss-client/src/main/java/org/apache/fluss/client/table/scanner/log/LogFetcher.java +++ b/fluss-client/src/main/java/org/apache/fluss/client/table/scanner/log/LogFetcher.java @@ -42,7 +42,6 @@ import org.apache.fluss.record.MemoryLogRecords; import org.apache.fluss.remote.RemoteLogFetchInfo; import org.apache.fluss.remote.RemoteLogSegment; -import org.apache.fluss.row.InternalRow; import org.apache.fluss.rpc.entity.FetchLogResultForBucket; import org.apache.fluss.rpc.gateway.TabletServerGateway; import org.apache.fluss.rpc.messages.FetchLogRequest; @@ -162,7 +161,7 @@ public boolean hasAvailableFetches() { return !logFetchBuffer.isEmpty(); } - public Map>> collectFetch() { + public Map> collectFetch() { return logFetchCollector.collectFetch(logFetchBuffer); } diff --git a/fluss-client/src/main/java/org/apache/fluss/client/table/scanner/log/LogScanner.java b/fluss-client/src/main/java/org/apache/fluss/client/table/scanner/log/LogScanner.java index f61d864776..e8d9ba6a0b 100644 --- a/fluss-client/src/main/java/org/apache/fluss/client/table/scanner/log/LogScanner.java +++ b/fluss-client/src/main/java/org/apache/fluss/client/table/scanner/log/LogScanner.java @@ -18,7 +18,6 @@ package org.apache.fluss.client.table.scanner.log; import org.apache.fluss.annotation.PublicEvolving; -import org.apache.fluss.row.InternalRow; import java.time.Duration; @@ -49,7 +48,7 @@ public interface LogScanner extends AutoCloseable { * @throws java.lang.IllegalStateException if the scanner is not subscribed to any buckets to * read from. */ - ScanRecords poll(Duration timeout); + ScanRecords poll(Duration timeout); /** * Subscribe to the given table bucket in given offset dynamically. If the table bucket is diff --git a/fluss-client/src/main/java/org/apache/fluss/client/table/scanner/log/LogScannerImpl.java b/fluss-client/src/main/java/org/apache/fluss/client/table/scanner/log/LogScannerImpl.java index 621a887f56..2a43a4824a 100644 --- a/fluss-client/src/main/java/org/apache/fluss/client/table/scanner/log/LogScannerImpl.java +++ b/fluss-client/src/main/java/org/apache/fluss/client/table/scanner/log/LogScannerImpl.java @@ -28,7 +28,6 @@ import org.apache.fluss.metadata.TableBucket; import org.apache.fluss.metadata.TableInfo; import org.apache.fluss.metadata.TablePath; -import org.apache.fluss.row.InternalRow; import org.apache.fluss.rpc.metrics.ClientMetricGroup; import org.apache.fluss.types.RowType; import org.apache.fluss.utils.Projection; @@ -131,7 +130,7 @@ private Projection sanityProjection(@Nullable int[] projectedFields, TableInfo t } @Override - public ScanRecords poll(Duration timeout) { + public ScanRecords poll(Duration timeout) { acquireAndEnsureOpen(); try { if (!logScannerStatus.prepareToPoll()) { @@ -142,17 +141,17 @@ public ScanRecords poll(Duration timeout) { long timeoutNanos = timeout.toNanos(); long startNanos = System.nanoTime(); do { - Map>> fetchRecords = pollForFetches(); + Map> fetchRecords = pollForFetches(); if (fetchRecords.isEmpty()) { try { if (!logFetcher.awaitNotEmpty(startNanos + timeoutNanos)) { // logFetcher waits for the timeout and no data in buffer, // so we return empty - return new ScanRecords<>(fetchRecords); + return new ScanRecords(fetchRecords); } } catch (WakeupException e) { // wakeup() is called, we need to return empty - return new ScanRecords<>(fetchRecords); + return new ScanRecords(fetchRecords); } } else { // before returning the fetched records, we can send off the next round of @@ -160,7 +159,7 @@ public ScanRecords poll(Duration timeout) { // while the user is handling the fetched records. logFetcher.sendFetches(); - return new ScanRecords<>(fetchRecords); + return new ScanRecords(fetchRecords); } } while (System.nanoTime() - startNanos < timeoutNanos); @@ -231,8 +230,8 @@ public void wakeup() { logFetcher.wakeup(); } - private Map>> pollForFetches() { - Map>> fetchedRecords = logFetcher.collectFetch(); + private Map> pollForFetches() { + Map> fetchedRecords = logFetcher.collectFetch(); if (!fetchedRecords.isEmpty()) { return fetchedRecords; } diff --git a/fluss-client/src/main/java/org/apache/fluss/client/table/scanner/log/ScanRecords.java b/fluss-client/src/main/java/org/apache/fluss/client/table/scanner/log/ScanRecords.java index 1473954c51..f852ba9abd 100644 --- a/fluss-client/src/main/java/org/apache/fluss/client/table/scanner/log/ScanRecords.java +++ b/fluss-client/src/main/java/org/apache/fluss/client/table/scanner/log/ScanRecords.java @@ -36,14 +36,14 @@ * @since 0.1 */ @PublicEvolving -public class ScanRecords implements Iterable> { - public static final ScanRecords empty() { - return new ScanRecords<>(Collections.emptyMap()); +public class ScanRecords implements Iterable { + public static final ScanRecords empty() { + return new ScanRecords(Collections.emptyMap()); } - private final Map>> records; + private final Map> records; - public ScanRecords(Map>> records) { + public ScanRecords(Map> records) { this.records = records; } @@ -52,8 +52,8 @@ public ScanRecords(Map>> records) { * * @param scanBucket The bucket to get records for */ - public List> records(TableBucket scanBucket) { - List> recs = records.get(scanBucket); + public List records(TableBucket scanBucket) { + List recs = records.get(scanBucket); if (recs == null) { return Collections.emptyList(); } @@ -73,7 +73,7 @@ public Set buckets() { /** The number of records for all buckets. */ public int count() { int count = 0; - for (List> recs : records.values()) { + for (List recs : records.values()) { count += recs.size(); } return count; @@ -84,25 +84,25 @@ public boolean isEmpty() { } @Override - public Iterator> iterator() { - return new ConcatenatedIterable<>(records.values()).iterator(); + public Iterator iterator() { + return new ConcatenatedIterable(records.values()).iterator(); } - private static class ConcatenatedIterable implements Iterable> { + private static class ConcatenatedIterable implements Iterable { - private final Iterable>> iterables; + private final Iterable> iterables; - public ConcatenatedIterable(Iterable>> iterables) { + public ConcatenatedIterable(Iterable> iterables) { this.iterables = iterables; } @Override - public Iterator> iterator() { - return new AbstractIterator>() { - final Iterator>> iters = iterables.iterator(); - Iterator> current; + public Iterator iterator() { + return new AbstractIterator() { + final Iterator> iters = iterables.iterator(); + Iterator current; - public ScanRecord makeNext() { + public ScanRecord makeNext() { while (current == null || !current.hasNext()) { if (iters.hasNext()) { current = iters.next().iterator(); diff --git a/fluss-client/src/main/java/org/apache/fluss/client/table/scanner/log/TypedLogScanner.java b/fluss-client/src/main/java/org/apache/fluss/client/table/scanner/log/TypedLogScanner.java index 16b3a7dae8..a700ab4340 100644 --- a/fluss-client/src/main/java/org/apache/fluss/client/table/scanner/log/TypedLogScanner.java +++ b/fluss-client/src/main/java/org/apache/fluss/client/table/scanner/log/TypedLogScanner.java @@ -36,7 +36,7 @@ public interface TypedLogScanner extends AutoCloseable { * @param timeout the timeout to poll. * @return the result of poll. */ - ScanRecords poll(Duration timeout); + TypedScanRecords poll(Duration timeout); /** * Subscribe to the given table bucket from beginning dynamically. If the table bucket is diff --git a/fluss-client/src/main/java/org/apache/fluss/client/table/scanner/log/TypedLogScannerImpl.java b/fluss-client/src/main/java/org/apache/fluss/client/table/scanner/log/TypedLogScannerImpl.java index 09667e0b7c..3b7ea32d4f 100644 --- a/fluss-client/src/main/java/org/apache/fluss/client/table/scanner/log/TypedLogScannerImpl.java +++ b/fluss-client/src/main/java/org/apache/fluss/client/table/scanner/log/TypedLogScannerImpl.java @@ -19,6 +19,7 @@ import org.apache.fluss.client.converter.RowToPojoConverter; import org.apache.fluss.client.table.scanner.ScanRecord; +import org.apache.fluss.client.table.scanner.TypedScanRecord; import org.apache.fluss.metadata.TableBucket; import org.apache.fluss.metadata.TableInfo; import org.apache.fluss.row.InternalRow; @@ -48,24 +49,23 @@ public TypedLogScannerImpl( } @Override - public ScanRecords poll(Duration timeout) { - ScanRecords records = delegate.poll(timeout); + public TypedScanRecords poll(Duration timeout) { + ScanRecords records = delegate.poll(timeout); if (records == null || records.isEmpty()) { - return ScanRecords.empty(); + return TypedScanRecords.empty(); } - Map>> out = new HashMap<>(); + Map>> out = new HashMap<>(); for (TableBucket bucket : records.buckets()) { - List> list = records.records(bucket); - List> converted = new ArrayList<>(list.size()); - for (ScanRecord r : list) { - InternalRow row = r.getValue(); + List list = records.records(bucket); + List> converted = new ArrayList<>(list.size()); + for (ScanRecord r : list) { + InternalRow row = r.getRow(); T pojo = converter.fromRow(row); - converted.add( - new ScanRecord<>(r.logOffset(), r.timestamp(), r.getChangeType(), pojo)); + converted.add(new TypedScanRecord<>(r, pojo)); } out.put(bucket, converted); } - return new ScanRecords<>(out); + return new TypedScanRecords<>(out); } @Override diff --git a/fluss-client/src/main/java/org/apache/fluss/client/table/scanner/log/TypedScanRecords.java b/fluss-client/src/main/java/org/apache/fluss/client/table/scanner/log/TypedScanRecords.java new file mode 100644 index 0000000000..f69347802b --- /dev/null +++ b/fluss-client/src/main/java/org/apache/fluss/client/table/scanner/log/TypedScanRecords.java @@ -0,0 +1,120 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.fluss.client.table.scanner.log; + +import org.apache.fluss.annotation.PublicEvolving; +import org.apache.fluss.client.table.scanner.TypedScanRecord; +import org.apache.fluss.metadata.TableBucket; +import org.apache.fluss.utils.AbstractIterator; + +import java.util.Collections; +import java.util.Iterator; +import java.util.List; +import java.util.Map; +import java.util.Set; + +/** + * A container that holds the list {@link TypedScanRecord} per bucket for a particular table. There + * is one {@link TypedScanRecord} list for every bucket returned by a {@link + * TypedLogScanner#poll(java.time.Duration)} operation. + * + * @param the type of the POJO + * @since 0.6 + */ +@PublicEvolving +public class TypedScanRecords implements Iterable> { + + public static TypedScanRecords empty() { + return new TypedScanRecords<>(Collections.emptyMap()); + } + + private final Map>> records; + + public TypedScanRecords(Map>> records) { + this.records = records; + } + + /** + * Get just the records for the given bucketId. + * + * @param scanBucket The bucket to get records for + */ + public List> records(TableBucket scanBucket) { + List> recs = records.get(scanBucket); + if (recs == null) { + return Collections.emptyList(); + } + return Collections.unmodifiableList(recs); + } + + /** + * Get the bucket ids which have records contained in this record set. + * + * @return the set of partitions with data in this record set (maybe empty if no data was + * returned) + */ + public Set buckets() { + return Collections.unmodifiableSet(records.keySet()); + } + + /** The number of records for all buckets. */ + public int count() { + int count = 0; + for (List> recs : records.values()) { + count += recs.size(); + } + return count; + } + + public boolean isEmpty() { + return records.isEmpty(); + } + + @Override + public Iterator> iterator() { + return new ConcatenatedIterable<>(records.values()).iterator(); + } + + private static class ConcatenatedIterable implements Iterable> { + + private final Iterable>> iterables; + + public ConcatenatedIterable(Iterable>> iterables) { + this.iterables = iterables; + } + + @Override + public Iterator> iterator() { + return new AbstractIterator>() { + final Iterator>> iters = iterables.iterator(); + Iterator> current; + + public TypedScanRecord makeNext() { + while (current == null || !current.hasNext()) { + if (iters.hasNext()) { + current = iters.next().iterator(); + } else { + return allDone(); + } + } + return current.next(); + } + }; + } + } +} diff --git a/fluss-client/src/test/java/org/apache/fluss/client/admin/ClientToServerITCaseBase.java b/fluss-client/src/test/java/org/apache/fluss/client/admin/ClientToServerITCaseBase.java index 440aedbd04..8270ed7882 100644 --- a/fluss-client/src/test/java/org/apache/fluss/client/admin/ClientToServerITCaseBase.java +++ b/fluss-client/src/test/java/org/apache/fluss/client/admin/ClientToServerITCaseBase.java @@ -224,10 +224,10 @@ public static void verifyPartitionLogs( logScanner.subscribeFromBeginning(partitionId, 0); } while (scanRecordCount < totalRecords) { - ScanRecords scanRecords = logScanner.poll(Duration.ofSeconds(1)); + ScanRecords scanRecords = logScanner.poll(Duration.ofSeconds(1)); for (TableBucket scanBucket : scanRecords.buckets()) { - List> records = scanRecords.records(scanBucket); - for (ScanRecord scanRecord : records) { + List records = scanRecords.records(scanBucket); + for (ScanRecord scanRecord : records) { actualRows .computeIfAbsent( scanBucket.getPartitionId(), k -> new ArrayList<>()) diff --git a/fluss-client/src/test/java/org/apache/fluss/client/table/AutoPartitionedTableITCase.java b/fluss-client/src/test/java/org/apache/fluss/client/table/AutoPartitionedTableITCase.java index b6d46337cf..6ae9ed7be7 100644 --- a/fluss-client/src/test/java/org/apache/fluss/client/table/AutoPartitionedTableITCase.java +++ b/fluss-client/src/test/java/org/apache/fluss/client/table/AutoPartitionedTableITCase.java @@ -289,10 +289,10 @@ private Map> pollRecords( int scanRecordCount = 0; Map> actualRows = new HashMap<>(); while (scanRecordCount < expectRecordsCount) { - ScanRecords scanRecords = logScanner.poll(Duration.ofSeconds(1)); + ScanRecords scanRecords = logScanner.poll(Duration.ofSeconds(1)); for (TableBucket scanBucket : scanRecords.buckets()) { - List> records = scanRecords.records(scanBucket); - for (ScanRecord scanRecord : records) { + List records = scanRecords.records(scanBucket); + for (ScanRecord scanRecord : records) { actualRows .computeIfAbsent(scanBucket.getPartitionId(), k -> new ArrayList<>()) .add(scanRecord.getRow()); diff --git a/fluss-client/src/test/java/org/apache/fluss/client/table/FlussFailServerTableITCase.java b/fluss-client/src/test/java/org/apache/fluss/client/table/FlussFailServerTableITCase.java index 4c52ef5f56..341c9a80d8 100644 --- a/fluss-client/src/test/java/org/apache/fluss/client/table/FlussFailServerTableITCase.java +++ b/fluss-client/src/test/java/org/apache/fluss/client/table/FlussFailServerTableITCase.java @@ -115,7 +115,7 @@ void testLogScan() throws Exception { appendWriter.append(row).get(); // poll data util we get one record - ScanRecords scanRecords; + ScanRecords scanRecords; do { scanRecords = logScanner.poll(Duration.ofSeconds(1)); } while (scanRecords.isEmpty()); @@ -147,9 +147,9 @@ void testLogScan() throws Exception { } } - private List toRows(ScanRecords scanRecords) { + private List toRows(ScanRecords scanRecords) { List rows = new ArrayList<>(); - for (ScanRecord scanRecord : scanRecords) { + for (ScanRecord scanRecord : scanRecords) { rows.add(scanRecord.getRow()); } return rows; diff --git a/fluss-client/src/test/java/org/apache/fluss/client/table/FlussLakeTableITCase.java b/fluss-client/src/test/java/org/apache/fluss/client/table/FlussLakeTableITCase.java index a1fbdc772e..5ad6a10ddf 100644 --- a/fluss-client/src/test/java/org/apache/fluss/client/table/FlussLakeTableITCase.java +++ b/fluss-client/src/test/java/org/apache/fluss/client/table/FlussLakeTableITCase.java @@ -328,13 +328,13 @@ private Map> writeRowsAndVerifyBucket( } } while (scanCount < totalRows) { - ScanRecords scanRecords = logScanner.poll(Duration.ofSeconds(1)); + ScanRecords scanRecords = logScanner.poll(Duration.ofSeconds(1)); for (TableBucket tableBucket : scanRecords.buckets()) { actualRows .computeIfAbsent(tableBucket, (k) -> new ArrayList<>()) .addAll( scanRecords.records(tableBucket).stream() - .map(ScanRecord::getRow) + .map(ScanRecord::getRow) .collect(Collectors.toList())); } scanCount += scanRecords.count(); diff --git a/fluss-client/src/test/java/org/apache/fluss/client/table/FlussTableITCase.java b/fluss-client/src/test/java/org/apache/fluss/client/table/FlussTableITCase.java index e9df7ac6e7..8a72d5b90a 100644 --- a/fluss-client/src/test/java/org/apache/fluss/client/table/FlussTableITCase.java +++ b/fluss-client/src/test/java/org/apache/fluss/client/table/FlussTableITCase.java @@ -148,8 +148,8 @@ void testAppendWithSmallBuffer(boolean indexedFormat) throws Exception { subscribeFromBeginning(logScanner, table); int count = 0; while (count < expectedSize) { - ScanRecords scanRecords = logScanner.poll(Duration.ofSeconds(1)); - for (ScanRecord scanRecord : scanRecords) { + ScanRecords scanRecords = logScanner.poll(Duration.ofSeconds(1)); + for (ScanRecord scanRecord : scanRecords) { assertThat(scanRecord.getChangeType()).isEqualTo(ChangeType.APPEND_ONLY); InternalRow row = scanRecord.getRow(); assertThat(row.getInt(0)).isEqualTo(1); @@ -182,9 +182,9 @@ void testPollOnce() throws Exception { subscribeFromBeginning(logScanner, table); int count = 0; while (count < expectedSize) { - ScanRecords scanRecords = logScanner.poll(Duration.ofSeconds(1)); + ScanRecords scanRecords = logScanner.poll(Duration.ofSeconds(1)); assertThat(scanRecords.isEmpty()).isFalse(); - for (ScanRecord scanRecord : scanRecords) { + for (ScanRecord scanRecord : scanRecords) { assertThat(scanRecord.getChangeType()).isEqualTo(ChangeType.APPEND_ONLY); InternalRow row = scanRecord.getRow(); assertThat(row.getInt(0)).isEqualTo(1); @@ -222,8 +222,8 @@ void testUpsertWithSmallBuffer() throws Exception { subscribeFromBeginning(logScanner, table); int count = 0; while (count < expectedSize) { - ScanRecords scanRecords = logScanner.poll(Duration.ofSeconds(1)); - for (ScanRecord scanRecord : scanRecords) { + ScanRecords scanRecords = logScanner.poll(Duration.ofSeconds(1)); + for (ScanRecord scanRecord : scanRecords) { assertThat(scanRecord.getChangeType()).isEqualTo(ChangeType.INSERT); InternalRow row = scanRecord.getRow(); assertThat(row.getInt(0)).isEqualTo(count); @@ -731,8 +731,8 @@ void testAppendWhileTableMaybeNotReady() throws Exception { subscribeFromBeginning(logScanner, table); InternalRow result = null; while (result == null) { - ScanRecords scanRecords = logScanner.poll(Duration.ofSeconds(1)); - for (ScanRecord scanRecord : scanRecords) { + ScanRecords scanRecords = logScanner.poll(Duration.ofSeconds(1)); + for (ScanRecord scanRecord : scanRecords) { assertThat(scanRecord.getChangeType()).isEqualTo(ChangeType.APPEND_ONLY); result = scanRecord.getRow(); } @@ -812,8 +812,8 @@ void verifyAppendOrPut(boolean append, String logFormat, @Nullable String kvForm subscribeFromBeginning(logScanner, table); int count = 0; while (count < expectedSize) { - ScanRecords scanRecords = logScanner.poll(Duration.ofSeconds(1)); - for (ScanRecord scanRecord : scanRecords) { + ScanRecords scanRecords = logScanner.poll(Duration.ofSeconds(1)); + for (ScanRecord scanRecord : scanRecords) { if (append) { assertThat(scanRecord.getChangeType()).isEqualTo(ChangeType.APPEND_ONLY); } else { @@ -873,8 +873,8 @@ void testAppendAndProject(String format) throws Exception { subscribeFromBeginning(logScanner, table); int count = 0; while (count < expectedSize) { - ScanRecords scanRecords = logScanner.poll(Duration.ofSeconds(1)); - for (ScanRecord scanRecord : scanRecords) { + ScanRecords scanRecords = logScanner.poll(Duration.ofSeconds(1)); + for (ScanRecord scanRecord : scanRecords) { assertThat(scanRecord.getChangeType()).isEqualTo(ChangeType.APPEND_ONLY); assertThat(scanRecord.getRow().getFieldCount()).isEqualTo(2); assertThat(scanRecord.getRow().getInt(0)).isEqualTo(count); @@ -896,8 +896,8 @@ void testAppendAndProject(String format) throws Exception { subscribeFromBeginning(logScanner, table); count = 0; while (count < expectedSize) { - ScanRecords scanRecords = logScanner.poll(Duration.ofSeconds(1)); - for (ScanRecord scanRecord : scanRecords) { + ScanRecords scanRecords = logScanner.poll(Duration.ofSeconds(1)); + for (ScanRecord scanRecord : scanRecords) { assertThat(scanRecord.getChangeType()).isEqualTo(ChangeType.APPEND_ONLY); assertThat(scanRecord.getRow().getFieldCount()).isEqualTo(2); assertThat(scanRecord.getRow().getInt(1)).isEqualTo(count); @@ -1214,17 +1214,17 @@ void testFirstRowMergeEngine(boolean doProjection) throws Exception { LogScanner logScanner = scan.createLogScanner(); logScanner.subscribeFromBeginning(0); - List> actualLogRecords = new ArrayList<>(0); + List actualLogRecords = new ArrayList<>(0); while (actualLogRecords.size() < rows) { - ScanRecords scanRecords = logScanner.poll(Duration.ofSeconds(1)); - for (ScanRecord rec : scanRecords) { + ScanRecords scanRecords = logScanner.poll(Duration.ofSeconds(1)); + for (ScanRecord rec : scanRecords) { actualLogRecords.add(rec); } } logScanner.close(); assertThat(actualLogRecords).hasSize(rows); for (int i = 0; i < actualLogRecords.size(); i++) { - ScanRecord scanRecord = actualLogRecords.get(i); + ScanRecord scanRecord = actualLogRecords.get(i); assertThat(scanRecord.getChangeType()).isEqualTo(ChangeType.INSERT); assertThatRow(scanRecord.getRow()) .withSchema(doProjection ? rowType.project(new int[] {0}) : rowType) @@ -1271,8 +1271,8 @@ void testArrowCompressionAndProject(String compression, String level) throws Exc subscribeFromBeginning(logScanner, table); int count = 0; while (count < expectedSize) { - ScanRecords scanRecords = logScanner.poll(Duration.ofSeconds(1)); - for (ScanRecord scanRecord : scanRecords) { + ScanRecords scanRecords = logScanner.poll(Duration.ofSeconds(1)); + for (ScanRecord scanRecord : scanRecords) { assertThat(scanRecord.getChangeType()).isEqualTo(ChangeType.APPEND_ONLY); assertThat(scanRecord.getRow().getInt(0)).isEqualTo(count); @@ -1295,8 +1295,8 @@ void testArrowCompressionAndProject(String compression, String level) throws Exc subscribeFromBeginning(logScanner, table); count = 0; while (count < expectedSize) { - ScanRecords scanRecords = logScanner.poll(Duration.ofSeconds(1)); - for (ScanRecord scanRecord : scanRecords) { + ScanRecords scanRecords = logScanner.poll(Duration.ofSeconds(1)); + for (ScanRecord scanRecord : scanRecords) { assertThat(scanRecord.getChangeType()).isEqualTo(ChangeType.APPEND_ONLY); assertThat(scanRecord.getRow().getFieldCount()).isEqualTo(2); assertThat(scanRecord.getRow().getInt(0)).isEqualTo(count); @@ -1385,10 +1385,10 @@ void testMergeEngineWithVersion(boolean doProjection) throws Exception { } LogScanner logScanner = scan.createLogScanner(); logScanner.subscribeFromBeginning(0); - List> actualLogRecords = new ArrayList<>(rows); + List actualLogRecords = new ArrayList<>(rows); while (actualLogRecords.size() < rows) { - ScanRecords scanRecords = logScanner.poll(Duration.ofSeconds(1)); - for (ScanRecord rec : scanRecords) { + ScanRecords scanRecords = logScanner.poll(Duration.ofSeconds(1)); + for (ScanRecord rec : scanRecords) { actualLogRecords.add(rec); } } diff --git a/fluss-client/src/test/java/org/apache/fluss/client/table/FlussTypedClientITCase.java b/fluss-client/src/test/java/org/apache/fluss/client/table/FlussTypedClientITCase.java index 8a5227cf7a..d58615dfc3 100644 --- a/fluss-client/src/test/java/org/apache/fluss/client/table/FlussTypedClientITCase.java +++ b/fluss-client/src/test/java/org/apache/fluss/client/table/FlussTypedClientITCase.java @@ -23,9 +23,9 @@ import org.apache.fluss.client.lookup.Lookuper; import org.apache.fluss.client.lookup.TypedLookuper; import org.apache.fluss.client.table.scanner.Scan; -import org.apache.fluss.client.table.scanner.ScanRecord; -import org.apache.fluss.client.table.scanner.log.ScanRecords; +import org.apache.fluss.client.table.scanner.TypedScanRecord; import org.apache.fluss.client.table.scanner.log.TypedLogScanner; +import org.apache.fluss.client.table.scanner.log.TypedScanRecords; import org.apache.fluss.client.table.writer.TypedAppendWriter; import org.apache.fluss.client.table.writer.TypedUpsertWriter; import org.apache.fluss.client.table.writer.Upsert; @@ -243,8 +243,8 @@ void testTypedAppendWriteAndScan() throws Exception { List actual = new ArrayList<>(); while (actual.size() < expected.size()) { - ScanRecords recs = scanner.poll(Duration.ofSeconds(2)); - for (ScanRecord r : recs) { + TypedScanRecords recs = scanner.poll(Duration.ofSeconds(2)); + for (TypedScanRecord r : recs) { assertThat(r.getChangeType()).isEqualTo(ChangeType.APPEND_ONLY); actual.add(r.getValue()); } @@ -287,8 +287,8 @@ void testTypedUpsertWriteAndScan() throws Exception { List changes = new ArrayList<>(); List values = new ArrayList<>(); while (values.size() < 4) { // INSERT 1, INSERT 2, UPDATE_BEFORE 1, UPDATE_AFTER 1 - ScanRecords recs = scanner.poll(Duration.ofSeconds(2)); - for (ScanRecord r : recs) { + TypedScanRecords recs = scanner.poll(Duration.ofSeconds(2)); + for (TypedScanRecord r : recs) { changes.add(r.getChangeType()); values.add(r.getValue()); } @@ -382,8 +382,8 @@ void testTypedProjections() throws Exception { .project(Arrays.asList("a", "str")) .createTypedLogScanner(AllTypesPojo.class); subscribeFromBeginning(scanner, table); - ScanRecords recs = scanner.poll(Duration.ofSeconds(2)); - for (ScanRecord r : recs) { + TypedScanRecords recs = scanner.poll(Duration.ofSeconds(2)); + for (TypedScanRecord r : recs) { AllTypesPojo u = r.getValue(); assertThat(u.a).isNotNull(); assertThat(u.str).isNotNull(); @@ -419,7 +419,7 @@ void testTypedPartialUpdates() throws Exception { AllTypesPojo patch = new AllTypesPojo(); patch.a = 1; patch.str = "second"; - patch.dec = new java.math.BigDecimal("99.99"); + patch.dec = new BigDecimal("99.99"); writer.upsert(patch).get(); writer.close(); @@ -439,8 +439,8 @@ void testTypedPartialUpdates() throws Exception { subscribeFromBeginning(scanner, table); boolean sawUpdateAfter = false; while (!sawUpdateAfter) { - ScanRecords recs = scanner.poll(Duration.ofSeconds(2)); - for (ScanRecord r : recs) { + TypedScanRecords recs = scanner.poll(Duration.ofSeconds(2)); + for (TypedScanRecord r : recs) { if (r.getChangeType() == ChangeType.UPDATE_AFTER) { assertThat(r.getValue().str).isEqualTo("second"); sawUpdateAfter = true; diff --git a/fluss-client/src/test/java/org/apache/fluss/client/table/scanner/log/DefaultCompletedFetchTest.java b/fluss-client/src/test/java/org/apache/fluss/client/table/scanner/log/DefaultCompletedFetchTest.java index 9e2b5349c2..72739ef265 100644 --- a/fluss-client/src/test/java/org/apache/fluss/client/table/scanner/log/DefaultCompletedFetchTest.java +++ b/fluss-client/src/test/java/org/apache/fluss/client/table/scanner/log/DefaultCompletedFetchTest.java @@ -103,7 +103,7 @@ void testSimple(byte recordBatchMagic) throws Exception { tb, createMemoryLogRecords(DATA2, LogFormat.ARROW, recordBatchMagic), 10L); DefaultCompletedFetch defaultCompletedFetch = makeCompletedFetch(tb, resultForBucket0, fetchOffset); - List> scanRecords = defaultCompletedFetch.fetchRecords(8); + List scanRecords = defaultCompletedFetch.fetchRecords(8); assertThat(scanRecords.size()).isEqualTo(8); assertThat(scanRecords.get(0).logOffset()).isEqualTo(0L); @@ -126,7 +126,7 @@ void testNegativeFetchCount(byte recordBatchMagic) throws Exception { tb, createMemoryLogRecords(DATA2, LogFormat.ARROW, recordBatchMagic), 10L); DefaultCompletedFetch defaultCompletedFetch = makeCompletedFetch(tb, resultForBucket0, fetchOffset); - List> scanRecords = defaultCompletedFetch.fetchRecords(-10); + List scanRecords = defaultCompletedFetch.fetchRecords(-10); assertThat(scanRecords.size()).isEqualTo(0); } @@ -139,7 +139,7 @@ void testNoRecordsInFetch() { new FetchLogResultForBucket(tb, MemoryLogRecords.EMPTY, 0L); DefaultCompletedFetch defaultCompletedFetch = makeCompletedFetch(tb, resultForBucket0, fetchOffset); - List> scanRecords = defaultCompletedFetch.fetchRecords(10); + List scanRecords = defaultCompletedFetch.fetchRecords(10); assertThat(scanRecords.size()).isEqualTo(0); } @@ -181,7 +181,7 @@ void testProjection(LogFormat logFormat, byte magic) throws Exception { new FetchLogResultForBucket(tb, memoryLogRecords, 10L); DefaultCompletedFetch defaultCompletedFetch = makeCompletedFetch(tb, resultForBucket0, fetchOffset, projection); - List> scanRecords = defaultCompletedFetch.fetchRecords(8); + List scanRecords = defaultCompletedFetch.fetchRecords(8); List expectedObjects = Arrays.asList( new Object[] {1, "hello"}, @@ -195,7 +195,7 @@ void testProjection(LogFormat logFormat, byte magic) throws Exception { assertThat(scanRecords.size()).isEqualTo(8); for (int i = 0; i < scanRecords.size(); i++) { Object[] expectObject = expectedObjects.get(i); - ScanRecord actualRecord = scanRecords.get(i); + ScanRecord actualRecord = scanRecords.get(i); assertThat(actualRecord.logOffset()).isEqualTo(i); assertThat(actualRecord.getChangeType()).isEqualTo(ChangeType.APPEND_ONLY); InternalRow row = actualRecord.getRow(); @@ -283,7 +283,7 @@ void testComplexTypeFetch() throws Exception { logScannerStatus, true, fetchOffset); - List> scanRecords = defaultCompletedFetch.fetchRecords(3); + List scanRecords = defaultCompletedFetch.fetchRecords(3); // close the read context to release arrow root resource, // this is important to test complex types defaultCompletedFetch.readContext.close(); diff --git a/fluss-client/src/test/java/org/apache/fluss/client/table/scanner/log/LogFetchCollectorTest.java b/fluss-client/src/test/java/org/apache/fluss/client/table/scanner/log/LogFetchCollectorTest.java index 4bba0e1682..99a108e2f3 100644 --- a/fluss-client/src/test/java/org/apache/fluss/client/table/scanner/log/LogFetchCollectorTest.java +++ b/fluss-client/src/test/java/org/apache/fluss/client/table/scanner/log/LogFetchCollectorTest.java @@ -23,7 +23,6 @@ import org.apache.fluss.config.Configuration; import org.apache.fluss.metadata.TableBucket; import org.apache.fluss.record.LogRecordReadContext; -import org.apache.fluss.row.InternalRow; import org.apache.fluss.rpc.entity.FetchLogResultForBucket; import org.junit.jupiter.api.AfterEach; @@ -98,7 +97,7 @@ void testNormal() throws Exception { assertThat(completedFetch.isInitialized()).isFalse(); // Fetch the data and validate that we get all the records we want back. - Map>> bucketAndRecords = + Map> bucketAndRecords = logFetchCollector.collectFetch(logFetchBuffer); assertThat(bucketAndRecords.size()).isEqualTo(1); assertThat(bucketAndRecords.get(tb)).size().isEqualTo(10); @@ -148,7 +147,7 @@ void testCollectAfterUnassign() throws Exception { // unassign bucket 2 logScannerStatus.unassignScanBuckets(Collections.singletonList(tb2)); - Map>> bucketAndRecords = + Map> bucketAndRecords = logFetchCollector.collectFetch(logFetchBuffer); // should only contain records for bucket 1 assertThat(bucketAndRecords.keySet()).containsExactly(tb1); diff --git a/fluss-client/src/test/java/org/apache/fluss/client/table/scanner/log/LogScannerITCase.java b/fluss-client/src/test/java/org/apache/fluss/client/table/scanner/log/LogScannerITCase.java index 9ead41f42b..cfe215fb0b 100644 --- a/fluss-client/src/test/java/org/apache/fluss/client/table/scanner/log/LogScannerITCase.java +++ b/fluss-client/src/test/java/org/apache/fluss/client/table/scanner/log/LogScannerITCase.java @@ -74,8 +74,8 @@ void testPoll() throws Exception { subscribeFromBeginning(logScanner, table); List rowList = new ArrayList<>(); while (rowList.size() < recordSize) { - ScanRecords scanRecords = logScanner.poll(Duration.ofSeconds(1)); - for (ScanRecord scanRecord : scanRecords) { + ScanRecords scanRecords = logScanner.poll(Duration.ofSeconds(1)); + for (ScanRecord scanRecord : scanRecords) { assertThat(scanRecord.getChangeType()).isEqualTo(ChangeType.APPEND_ONLY); InternalRow row = scanRecord.getRow(); rowList.add(row(row.getInt(0), row.getString(1))); @@ -109,8 +109,8 @@ void testPollWhileCreateTableNotReady() throws Exception { subscribeFromBeginning(logScanner, table); List rowList = new ArrayList<>(); while (rowList.size() < recordSize) { - ScanRecords scanRecords = logScanner.poll(Duration.ofSeconds(1)); - for (ScanRecord scanRecord : scanRecords) { + ScanRecords scanRecords = logScanner.poll(Duration.ofSeconds(1)); + for (ScanRecord scanRecord : scanRecords) { assertThat(scanRecord.getChangeType()).isEqualTo(ChangeType.APPEND_ONLY); InternalRow row = scanRecord.getRow(); rowList.add(row(row.getInt(0), row.getString(1))); @@ -148,8 +148,8 @@ void testLogScannerMultiThreadAccess() throws Exception { // should be able to poll data from all buckets List rowList = new ArrayList<>(); while (rowList.size() < recordSize) { - ScanRecords scanRecords = logScanner.poll(Duration.ofSeconds(1)); - for (ScanRecord scanRecord : scanRecords) { + ScanRecords scanRecords = logScanner.poll(Duration.ofSeconds(1)); + for (ScanRecord scanRecord : scanRecords) { assertThat(scanRecord.getChangeType()).isEqualTo(ChangeType.APPEND_ONLY); InternalRow row = scanRecord.getRow(); rowList.add(row(row.getInt(0), row.getString(1))); @@ -201,8 +201,8 @@ void testLogHeavyWriteAndScan() throws Exception { long scanned = 0; long total = 0; while (scanned < recordSize) { - ScanRecords scanRecords = logScanner.poll(Duration.ofSeconds(1)); - for (ScanRecord scanRecord : scanRecords) { + ScanRecords scanRecords = logScanner.poll(Duration.ofSeconds(1)); + for (ScanRecord scanRecord : scanRecords) { assertThat(scanRecord.getChangeType()).isEqualTo(ChangeType.APPEND_ONLY); assertThat(scanRecord.getRow().getString(0).getSizeInBytes()).isEqualTo(10); assertThat(scanRecord.getRow().getLong(1)).isEqualTo(scanned); @@ -257,8 +257,8 @@ void testKvHeavyWriteAndScan() throws Exception { long scanned = 0; long total = 0; while (scanned < recordSize) { - ScanRecords scanRecords = logScanner.poll(Duration.ofSeconds(1)); - for (ScanRecord scanRecord : scanRecords) { + ScanRecords scanRecords = logScanner.poll(Duration.ofSeconds(1)); + for (ScanRecord scanRecord : scanRecords) { assertThat(scanRecord.getChangeType()).isEqualTo(ChangeType.INSERT); assertThat(scanRecord.getRow().getString(0).getSizeInBytes()).isEqualTo(10); assertThat(scanRecord.getRow().getLong(1)).isEqualTo(scanned); @@ -333,8 +333,8 @@ void testScanFromStartTimestamp(boolean isPartitioned) throws Exception { firstStartTimestamp); List rowList = new ArrayList<>(); while (rowList.size() < batchRecordSize) { - ScanRecords scanRecords = logScanner.poll(Duration.ofSeconds(1)); - for (ScanRecord scanRecord : scanRecords) { + ScanRecords scanRecords = logScanner.poll(Duration.ofSeconds(1)); + for (ScanRecord scanRecord : scanRecords) { assertThat(scanRecord.getChangeType()).isEqualTo(ChangeType.APPEND_ONLY); InternalRow row = scanRecord.getRow(); rowList.add(row(row.getInt(0), row.getString(1))); @@ -363,8 +363,8 @@ void testScanFromStartTimestamp(boolean isPartitioned) throws Exception { secondStartTimestamp); rowList = new ArrayList<>(); while (rowList.size() < batchRecordSize) { - ScanRecords scanRecords = logScanner.poll(Duration.ofSeconds(1)); - for (ScanRecord scanRecord : scanRecords) { + ScanRecords scanRecords = logScanner.poll(Duration.ofSeconds(1)); + for (ScanRecord scanRecord : scanRecords) { assertThat(scanRecord.getChangeType()).isEqualTo(ChangeType.APPEND_ONLY); InternalRow row = scanRecord.getRow(); rowList.add(row(row.getInt(0), row.getString(1))); @@ -424,8 +424,8 @@ void testScanFromLatestOffsets(boolean isPartitioned) throws Exception { List rowList = new ArrayList<>(); while (rowList.size() < batchRecordSize) { - ScanRecords scanRecords = logScanner.poll(Duration.ofSeconds(1)); - for (ScanRecord scanRecord : scanRecords) { + ScanRecords scanRecords = logScanner.poll(Duration.ofSeconds(1)); + for (ScanRecord scanRecord : scanRecords) { assertThat(scanRecord.getChangeType()).isEqualTo(ChangeType.APPEND_ONLY); InternalRow row = scanRecord.getRow(); rowList.add(row(row.getInt(0), row.getString(1))); diff --git a/fluss-client/src/test/java/org/apache/fluss/client/table/scanner/log/RemoteCompletedFetchTest.java b/fluss-client/src/test/java/org/apache/fluss/client/table/scanner/log/RemoteCompletedFetchTest.java index 64729804e8..edc47a7ecb 100644 --- a/fluss-client/src/test/java/org/apache/fluss/client/table/scanner/log/RemoteCompletedFetchTest.java +++ b/fluss-client/src/test/java/org/apache/fluss/client/table/scanner/log/RemoteCompletedFetchTest.java @@ -113,8 +113,7 @@ void testSimple() throws Exception { null, () -> recycleCalled.set(true)); - List> scanRecords = - completedFetch.fetchRecords(8); + List scanRecords = completedFetch.fetchRecords(8); assertThat(scanRecords.size()).isEqualTo(8); assertThat(scanRecords.get(0).logOffset()).isEqualTo(0L); @@ -147,8 +146,7 @@ void testFetchForPartitionTable() throws Exception { makeCompletedFetch( tb, fileLogRecords, fetchOffset, null, () -> recycleCalled.set(true)); - List> scanRecords = - completedFetch.fetchRecords(8); + List scanRecords = completedFetch.fetchRecords(8); assertThat(scanRecords.size()).isEqualTo(8); assertThat(scanRecords.get(0).logOffset()).isEqualTo(0L); @@ -176,7 +174,7 @@ void testNegativeFetchCount() throws Exception { RemoteCompletedFetch completedFetch = makeCompletedFetch(tableBucket, fileLogRecords, fetchOffset, null); - List> scanRecords = completedFetch.fetchRecords(-10); + List scanRecords = completedFetch.fetchRecords(-10); assertThat(scanRecords.size()).isEqualTo(0); } @@ -193,7 +191,7 @@ void testNoRecordsInFetch() throws Exception { RemoteCompletedFetch completedFetch = makeCompletedFetch(tableBucket, fileLogRecords, fetchOffset, null); - List> scanRecords = completedFetch.fetchRecords(10); + List scanRecords = completedFetch.fetchRecords(10); assertThat(scanRecords.size()).isEqualTo(0); } @@ -233,7 +231,7 @@ void testProjection(String format) throws Exception { fetchOffset, Projection.of(new int[] {0, 2}, schema)); - List> scanRecords = completedFetch.fetchRecords(8); + List scanRecords = completedFetch.fetchRecords(8); List expectedObjects = Arrays.asList( new Object[] {1, "hello"}, diff --git a/fluss-client/src/test/java/org/apache/fluss/client/table/scanner/log/RemoteLogScannerITCase.java b/fluss-client/src/test/java/org/apache/fluss/client/table/scanner/log/RemoteLogScannerITCase.java index b44bcb9929..84b50af2c7 100644 --- a/fluss-client/src/test/java/org/apache/fluss/client/table/scanner/log/RemoteLogScannerITCase.java +++ b/fluss-client/src/test/java/org/apache/fluss/client/table/scanner/log/RemoteLogScannerITCase.java @@ -103,8 +103,8 @@ void testScanFromRemote() throws Exception { logScanner.subscribeFromBeginning(0); List rowList = new ArrayList<>(); while (rowList.size() < recordSize) { - ScanRecords scanRecords = logScanner.poll(Duration.ofSeconds(1)); - for (ScanRecord scanRecord : scanRecords) { + ScanRecords scanRecords = logScanner.poll(Duration.ofSeconds(1)); + for (ScanRecord scanRecord : scanRecords) { assertThat(scanRecord.getChangeType()).isEqualTo(ChangeType.APPEND_ONLY); InternalRow row = scanRecord.getRow(); rowList.add(row(row.getInt(0), row.getString(1))); @@ -154,8 +154,8 @@ void testScanFromRemoteAndProject(String format) throws Exception { logScanner.subscribeFromBeginning(0); int count = 0; while (count < expectedSize) { - ScanRecords scanRecords = logScanner.poll(Duration.ofSeconds(1)); - for (ScanRecord scanRecord : scanRecords) { + ScanRecords scanRecords = logScanner.poll(Duration.ofSeconds(1)); + for (ScanRecord scanRecord : scanRecords) { assertThat(scanRecord.getChangeType()).isEqualTo(ChangeType.APPEND_ONLY); assertThat(scanRecord.getRow().getFieldCount()).isEqualTo(2); assertThat(scanRecord.getRow().getInt(0)).isEqualTo(count); @@ -177,8 +177,8 @@ void testScanFromRemoteAndProject(String format) throws Exception { logScanner.subscribeFromBeginning(0); count = 0; while (count < expectedSize) { - ScanRecords scanRecords = logScanner.poll(Duration.ofSeconds(1)); - for (ScanRecord scanRecord : scanRecords) { + ScanRecords scanRecords = logScanner.poll(Duration.ofSeconds(1)); + for (ScanRecord scanRecord : scanRecords) { assertThat(scanRecord.getChangeType()).isEqualTo(ChangeType.APPEND_ONLY); assertThat(scanRecord.getRow().getFieldCount()).isEqualTo(2); assertThat(scanRecord.getRow().getInt(1)).isEqualTo(count); diff --git a/fluss-flink/fluss-flink-common/src/test/java/org/apache/fluss/flink/lake/reader/SortMergeReaderTest.java b/fluss-flink/fluss-flink-common/src/test/java/org/apache/fluss/flink/lake/reader/SortMergeReaderTest.java index b6828c3773..edeee58291 100644 --- a/fluss-flink/fluss-flink-common/src/test/java/org/apache/fluss/flink/lake/reader/SortMergeReaderTest.java +++ b/fluss-flink/fluss-flink-common/src/test/java/org/apache/fluss/flink/lake/reader/SortMergeReaderTest.java @@ -17,6 +17,7 @@ package org.apache.fluss.flink.lake.reader; +import org.apache.fluss.client.table.scanner.ScanRecord; import org.apache.fluss.record.ChangeType; import org.apache.fluss.record.LogRecord; import org.apache.fluss.row.BinaryString; @@ -152,44 +153,8 @@ private List createRecords(int startId, int count, boolean isLog) { startId + i, BinaryString.fromString(isLog ? "a" + "_updated" : "a"), BinaryString.fromString(isLog ? "A" + "_updated" : "A")); - logRecords.add( - new TestLogRecord(i, System.currentTimeMillis(), ChangeType.INSERT, row)); + logRecords.add(new ScanRecord(i, System.currentTimeMillis(), ChangeType.INSERT, row)); } return logRecords; } - - /** Simple LogRecord for tests. */ - private static final class TestLogRecord implements LogRecord { - private final long offset; - private final long ts; - private final ChangeType ct; - private final InternalRow row; - - private TestLogRecord(long offset, long ts, ChangeType ct, InternalRow row) { - this.offset = offset; - this.ts = ts; - this.ct = ct; - this.row = row; - } - - @Override - public long logOffset() { - return offset; - } - - @Override - public long timestamp() { - return ts; - } - - @Override - public ChangeType getChangeType() { - return ct; - } - - @Override - public InternalRow getRow() { - return row; - } - } } 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 ae816556ed..68fa741d48 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 @@ -301,11 +301,11 @@ void testAppendLogWithRoundRobin() throws Exception { logScanner.subscribeFromBeginning(2); long scanned = 0; while (scanned < 6) { - ScanRecords scanRecords = logScanner.poll(Duration.ofSeconds(1)); + ScanRecords scanRecords = logScanner.poll(Duration.ofSeconds(1)); for (TableBucket bucket : scanRecords.buckets()) { List rowsBucket = rows.computeIfAbsent(bucket.getBucket(), k -> new ArrayList<>()); - for (ScanRecord record : scanRecords.records(bucket)) { + for (ScanRecord record : scanRecords.records(bucket)) { InternalRow row = record.getRow(); rowsBucket.add( Row.of(row.getInt(0), row.getLong(1), row.getString(2).toString()) diff --git a/fluss-flink/fluss-flink-common/src/test/java/org/apache/fluss/flink/source/deserializer/FlussDeserializationSchemaTest.java b/fluss-flink/fluss-flink-common/src/test/java/org/apache/fluss/flink/source/deserializer/FlussDeserializationSchemaTest.java index 001982969e..12f7ba1c8a 100644 --- a/fluss-flink/fluss-flink-common/src/test/java/org/apache/fluss/flink/source/deserializer/FlussDeserializationSchemaTest.java +++ b/fluss-flink/fluss-flink-common/src/test/java/org/apache/fluss/flink/source/deserializer/FlussDeserializationSchemaTest.java @@ -53,11 +53,11 @@ */ public class FlussDeserializationSchemaTest { - /** Lightweight adapter to view a {@code ScanRecord} as a {@link LogRecord}. */ + /** Lightweight adapter to view a {@code ScanRecord} as a {@link LogRecord}. */ private static final class ScanRecordLogRecord implements LogRecord { - private final ScanRecord delegate; + private final ScanRecord delegate; - private ScanRecordLogRecord(ScanRecord delegate) { + private ScanRecordLogRecord(ScanRecord delegate) { this.delegate = delegate; } @@ -91,7 +91,7 @@ public void testDeserialize() throws Exception { row.setField(2, 3); row.setField(3, BinaryString.fromString("123 Main St")); - ScanRecord scanRecord = new ScanRecord<>(row); + ScanRecord scanRecord = new ScanRecord(row); OrderDeserializationSchema deserializer = new OrderDeserializationSchema(); Order result = deserializer.deserialize(new ScanRecordLogRecord(scanRecord)); @@ -110,7 +110,7 @@ public void testDeserializeWithNumericConversion() throws Exception { row.setField(2, 4); row.setField(3, BinaryString.fromString("456 Oak Ave")); - ScanRecord scanRecord = new ScanRecord<>(row); + ScanRecord scanRecord = new ScanRecord(row); OrderDeserializationSchema schema = new OrderDeserializationSchema(); Order result = schema.deserialize(new ScanRecordLogRecord(scanRecord)); @@ -129,7 +129,7 @@ public void testDeserializeWithNullValues() throws Exception { row.setField(2, 5); row.setField(3, null); - ScanRecord scanRecord = new ScanRecord<>(row); + ScanRecord scanRecord = new ScanRecord(row); OrderDeserializationSchema schema = new OrderDeserializationSchema(); Order result = schema.deserialize(new ScanRecordLogRecord(scanRecord)); @@ -208,7 +208,7 @@ public void testJsonStringDeserialize() throws Exception { row.setField(14, TimestampNtz.fromMillis(testTimestampInSeconds * 1000)); row.setField(15, TimestampLtz.fromEpochMillis(testTimestampInSeconds * 1000)); row.setField(16, null); - ScanRecord scanRecord = new ScanRecord<>(row); + ScanRecord scanRecord = new ScanRecord(row); // Create deserializer JsonStringDeserializationSchema deserializer = new JsonStringDeserializationSchema(); @@ -245,8 +245,7 @@ public void testJsonStringDeserialize() throws Exception { + "}"); // Verify with offset and timestamp - ScanRecord scanRecord2 = - new ScanRecord<>(1001, 1743261788400L, ChangeType.DELETE, row); + ScanRecord scanRecord2 = new ScanRecord(1001, 1743261788400L, ChangeType.DELETE, row); String result2 = deserializer.deserialize(new ScanRecordLogRecord(scanRecord2)); assertThat(result2).isNotNull(); assertThat(result2) @@ -260,7 +259,7 @@ public void testJsonStringDeserialize() throws Exception { row.setField(2, true); row.setField(8, 512); row.setField(13, 72000000); - ScanRecord changedRecord = new ScanRecord<>(row); + ScanRecord changedRecord = new ScanRecord(row); String changedResult = deserializer.deserialize(new ScanRecordLogRecord(changedRecord)); String changedRowJson = "{" From 1f2e546e3f6ff3f3b718cc2bdc4117beb613b253 Mon Sep 17 00:00:00 2001 From: ipolyzos Date: Mon, 22 Dec 2025 11:08:21 +0200 Subject: [PATCH 29/33] revert unneeded changes --- .../fluss/client/utils/MetadataUtils.java | 21 ++------ .../admin/ClientToServerITCaseBase.java | 8 ---- .../metadata/MetadataUpdaterITCase.java | 1 - .../metadata/TestingMetadataUpdater.java | 3 +- .../fluss/client/table/FlussTableITCase.java | 8 +--- .../table/scanner/log/LogFetcherITCase.java | 3 +- .../client/write/RecordAccumulatorTest.java | 3 +- .../apache/fluss/client/write/SenderTest.java | 3 +- .../write/StickyStaticBucketAssignerTest.java | 1 - .../org/apache/fluss/cluster/Cluster.java | 29 +---------- .../org/apache/fluss/cluster/ClusterTest.java | 27 +---------- .../LakeSnapshotAndLogSplitScanner.java | 4 +- .../source/emitter/FlinkRecordEmitter.java | 35 +------------- .../lookup/FlinkAsyncLookupFunction.java | 5 +- .../source/lookup/FlinkLookupFunction.java | 2 +- .../source/reader/FlinkSourceSplitReader.java | 7 ++- .../tiering/source/TieringSplitReader.java | 48 +++---------------- .../fluss/flink/sink/FlussSinkITCase.java | 12 ++--- .../FlussDeserializationSchemaTest.java | 46 +++--------------- .../RowDataDeserializationSchemaTest.java | 36 +------------- .../source/ScanRecordLogRecordTest.java | 3 +- .../FlussRowToFlinkRowConverterTest.java | 38 ++------------- 22 files changed, 49 insertions(+), 294 deletions(-) diff --git a/fluss-client/src/main/java/org/apache/fluss/client/utils/MetadataUtils.java b/fluss-client/src/main/java/org/apache/fluss/client/utils/MetadataUtils.java index cf642beae9..2990054999 100644 --- a/fluss-client/src/main/java/org/apache/fluss/client/utils/MetadataUtils.java +++ b/fluss-client/src/main/java/org/apache/fluss/client/utils/MetadataUtils.java @@ -24,7 +24,6 @@ import org.apache.fluss.exception.StaleMetadataException; import org.apache.fluss.metadata.PhysicalTablePath; import org.apache.fluss.metadata.TableBucket; -import org.apache.fluss.metadata.TableInfo; import org.apache.fluss.metadata.TablePath; import org.apache.fluss.rpc.GatewayClientProxy; import org.apache.fluss.rpc.RpcClient; @@ -126,7 +125,6 @@ public static Cluster sendMetadataRequestAndRebuildCluster( NewTableMetadata newTableMetadata = getTableMetadataToUpdate(originCluster, response); - Map newTableInfoByPath; if (partialUpdate) { // If partial update, we will clear the to be updated table out ot // the origin cluster. @@ -136,20 +134,17 @@ public static Cluster sendMetadataRequestAndRebuildCluster( new HashMap<>(originCluster.getBucketLocationsByPath()); newPartitionIdByPath = new HashMap<>(originCluster.getPartitionIdByPath()); - newTableInfoByPath = - new HashMap<>(originCluster.getTableInfoByPath()); newTablePathToTableId.putAll(newTableMetadata.tablePathToTableId); newBucketLocations.putAll(newTableMetadata.bucketLocations); newPartitionIdByPath.putAll(newTableMetadata.partitionIdByPath); - newTableInfoByPath.putAll(newTableMetadata.tableInfoByPath); + } else { // If full update, we will clear all tables info out ot the origin // cluster. newTablePathToTableId = newTableMetadata.tablePathToTableId; newBucketLocations = newTableMetadata.bucketLocations; newPartitionIdByPath = newTableMetadata.partitionIdByPath; - newTableInfoByPath = newTableMetadata.tableInfoByPath; } return new Cluster( @@ -157,8 +152,7 @@ public static Cluster sendMetadataRequestAndRebuildCluster( coordinatorServer, newBucketLocations, newTablePathToTableId, - newPartitionIdByPath, - newTableInfoByPath); + newPartitionIdByPath); }) .get(30, TimeUnit.SECONDS); // TODO currently, we don't have timeout logic in // RpcClient, it will let the get() block forever. So we @@ -170,7 +164,6 @@ private static NewTableMetadata getTableMetadataToUpdate( Map newTablePathToTableId = new HashMap<>(); Map> newBucketLocations = new HashMap<>(); Map newPartitionIdByPath = new HashMap<>(); - Map newTableInfoByPath = new HashMap<>(); // iterate all table metadata List pbTableMetadataList = metadataResponse.getTableMetadatasList(); @@ -218,27 +211,21 @@ private static NewTableMetadata getTableMetadataToUpdate( }); return new NewTableMetadata( - newTablePathToTableId, - newBucketLocations, - newPartitionIdByPath, - newTableInfoByPath); + newTablePathToTableId, newBucketLocations, newPartitionIdByPath); } private static final class NewTableMetadata { private final Map tablePathToTableId; private final Map> bucketLocations; private final Map partitionIdByPath; - private final Map tableInfoByPath; public NewTableMetadata( Map tablePathToTableId, Map> bucketLocations, - Map partitionIdByPath, - Map tableInfoByPath) { + Map partitionIdByPath) { this.tablePathToTableId = tablePathToTableId; this.bucketLocations = bucketLocations; this.partitionIdByPath = partitionIdByPath; - this.tableInfoByPath = tableInfoByPath; } } diff --git a/fluss-client/src/test/java/org/apache/fluss/client/admin/ClientToServerITCaseBase.java b/fluss-client/src/test/java/org/apache/fluss/client/admin/ClientToServerITCaseBase.java index 8270ed7882..a62c1b70d3 100644 --- a/fluss-client/src/test/java/org/apache/fluss/client/admin/ClientToServerITCaseBase.java +++ b/fluss-client/src/test/java/org/apache/fluss/client/admin/ClientToServerITCaseBase.java @@ -26,7 +26,6 @@ import org.apache.fluss.client.table.scanner.ScanRecord; import org.apache.fluss.client.table.scanner.log.LogScanner; import org.apache.fluss.client.table.scanner.log.ScanRecords; -import org.apache.fluss.client.table.scanner.log.TypedLogScanner; import org.apache.fluss.client.table.writer.UpsertWriter; import org.apache.fluss.config.ConfigOptions; import org.apache.fluss.config.Configuration; @@ -145,13 +144,6 @@ protected static void subscribeFromBeginning(LogScanner logScanner, Table table) } } - protected static void subscribeFromBeginning(TypedLogScanner logScanner, Table table) { - int bucketCount = table.getTableInfo().getNumBuckets(); - for (int i = 0; i < bucketCount; i++) { - logScanner.subscribeFromBeginning(i); - } - } - protected static void subscribeFromTimestamp( TablePath tablePath, @Nullable String partitionName, diff --git a/fluss-client/src/test/java/org/apache/fluss/client/metadata/MetadataUpdaterITCase.java b/fluss-client/src/test/java/org/apache/fluss/client/metadata/MetadataUpdaterITCase.java index caecbe5452..6d099a24e6 100644 --- a/fluss-client/src/test/java/org/apache/fluss/client/metadata/MetadataUpdaterITCase.java +++ b/fluss-client/src/test/java/org/apache/fluss/client/metadata/MetadataUpdaterITCase.java @@ -108,7 +108,6 @@ void testUpdateWithEmptyMetadataResponse() throws Exception { null, Collections.emptyMap(), Collections.emptyMap(), - Collections.emptyMap(), Collections.emptyMap()); metadataUpdater = new MetadataUpdater(rpcClient, new Configuration(), newCluster); diff --git a/fluss-client/src/test/java/org/apache/fluss/client/metadata/TestingMetadataUpdater.java b/fluss-client/src/test/java/org/apache/fluss/client/metadata/TestingMetadataUpdater.java index f5e96bb515..2063951357 100644 --- a/fluss-client/src/test/java/org/apache/fluss/client/metadata/TestingMetadataUpdater.java +++ b/fluss-client/src/test/java/org/apache/fluss/client/metadata/TestingMetadataUpdater.java @@ -218,7 +218,6 @@ private void initializeCluster( coordinatorServer, tablePathToBucketLocations, tableIdByPath, - Collections.emptyMap(), - tableInfos); + Collections.emptyMap()); } } diff --git a/fluss-client/src/test/java/org/apache/fluss/client/table/FlussTableITCase.java b/fluss-client/src/test/java/org/apache/fluss/client/table/FlussTableITCase.java index 8a72d5b90a..0cc2f7f10f 100644 --- a/fluss-client/src/test/java/org/apache/fluss/client/table/FlussTableITCase.java +++ b/fluss-client/src/test/java/org/apache/fluss/client/table/FlussTableITCase.java @@ -1217,9 +1217,7 @@ void testFirstRowMergeEngine(boolean doProjection) throws Exception { List actualLogRecords = new ArrayList<>(0); while (actualLogRecords.size() < rows) { ScanRecords scanRecords = logScanner.poll(Duration.ofSeconds(1)); - for (ScanRecord rec : scanRecords) { - actualLogRecords.add(rec); - } + scanRecords.forEach(actualLogRecords::add); } logScanner.close(); assertThat(actualLogRecords).hasSize(rows); @@ -1388,9 +1386,7 @@ void testMergeEngineWithVersion(boolean doProjection) throws Exception { List actualLogRecords = new ArrayList<>(rows); while (actualLogRecords.size() < rows) { ScanRecords scanRecords = logScanner.poll(Duration.ofSeconds(1)); - for (ScanRecord rec : scanRecords) { - actualLogRecords.add(rec); - } + scanRecords.forEach(actualLogRecords::add); } logScanner.close(); diff --git a/fluss-client/src/test/java/org/apache/fluss/client/table/scanner/log/LogFetcherITCase.java b/fluss-client/src/test/java/org/apache/fluss/client/table/scanner/log/LogFetcherITCase.java index 7b4f824e25..b6817b327b 100644 --- a/fluss-client/src/test/java/org/apache/fluss/client/table/scanner/log/LogFetcherITCase.java +++ b/fluss-client/src/test/java/org/apache/fluss/client/table/scanner/log/LogFetcherITCase.java @@ -258,8 +258,7 @@ void testFetchWhenDestinationIsNullInMetadata() throws Exception { oldCluster.getCoordinatorServer(), oldCluster.getBucketLocationsByPath(), oldCluster.getTableIdByPath(), - oldCluster.getPartitionIdByPath(), - oldCluster.getTableInfoByPath()); + oldCluster.getPartitionIdByPath()); metadataUpdater = new MetadataUpdater(rpcClient, clientConf, newCluster); LogScannerStatus logScannerStatus = new LogScannerStatus(); diff --git a/fluss-client/src/test/java/org/apache/fluss/client/write/RecordAccumulatorTest.java b/fluss-client/src/test/java/org/apache/fluss/client/write/RecordAccumulatorTest.java index 3844fb43a2..efebb32782 100644 --- a/fluss-client/src/test/java/org/apache/fluss/client/write/RecordAccumulatorTest.java +++ b/fluss-client/src/test/java/org/apache/fluss/client/write/RecordAccumulatorTest.java @@ -569,8 +569,7 @@ private Cluster updateCluster(List bucketLocations) { new ServerNode(0, "localhost", 89, ServerType.COORDINATOR), bucketsByPath, tableIdByPath, - Collections.emptyMap(), - tableInfoByPath); + Collections.emptyMap()); } private void delayedInterrupt(final Thread thread, final long delayMs) { diff --git a/fluss-client/src/test/java/org/apache/fluss/client/write/SenderTest.java b/fluss-client/src/test/java/org/apache/fluss/client/write/SenderTest.java index 2d06057e80..f1d9481a64 100644 --- a/fluss-client/src/test/java/org/apache/fluss/client/write/SenderTest.java +++ b/fluss-client/src/test/java/org/apache/fluss/client/write/SenderTest.java @@ -671,8 +671,7 @@ void testSendWhenDestinationIsNullInMetadata() throws Exception { oldCluster.getCoordinatorServer(), oldCluster.getBucketLocationsByPath(), oldCluster.getTableIdByPath(), - oldCluster.getPartitionIdByPath(), - oldCluster.getTableInfoByPath()); + oldCluster.getPartitionIdByPath()); metadataUpdater.updateCluster(newCluster); diff --git a/fluss-client/src/test/java/org/apache/fluss/client/write/StickyStaticBucketAssignerTest.java b/fluss-client/src/test/java/org/apache/fluss/client/write/StickyStaticBucketAssignerTest.java index 6009b96f51..f828906339 100644 --- a/fluss-client/src/test/java/org/apache/fluss/client/write/StickyStaticBucketAssignerTest.java +++ b/fluss-client/src/test/java/org/apache/fluss/client/write/StickyStaticBucketAssignerTest.java @@ -215,7 +215,6 @@ private Cluster updateCluster(List bucketLocations) { new ServerNode(0, "localhost", 89, ServerType.COORDINATOR), bucketsByPath, tableIdByPath, - Collections.emptyMap(), Collections.emptyMap()); } } diff --git a/fluss-common/src/main/java/org/apache/fluss/cluster/Cluster.java b/fluss-common/src/main/java/org/apache/fluss/cluster/Cluster.java index a7a46df957..d5690326a0 100644 --- a/fluss-common/src/main/java/org/apache/fluss/cluster/Cluster.java +++ b/fluss-common/src/main/java/org/apache/fluss/cluster/Cluster.java @@ -21,7 +21,6 @@ import org.apache.fluss.exception.PartitionNotExistException; import org.apache.fluss.metadata.PhysicalTablePath; import org.apache.fluss.metadata.TableBucket; -import org.apache.fluss.metadata.TableInfo; import org.apache.fluss.metadata.TablePath; import javax.annotation.Nullable; @@ -53,22 +52,19 @@ public final class Cluster { private final Map pathByTableId; private final Map partitionsIdByPath; private final Map partitionNameById; - private final Map tableInfoByPath; public Cluster( Map aliveTabletServersById, @Nullable ServerNode coordinatorServer, Map> bucketLocationsByPath, Map tableIdByPath, - Map partitionsIdByPath, - Map tableInfoByPath) { + Map partitionsIdByPath) { this.coordinatorServer = coordinatorServer; this.aliveTabletServersById = Collections.unmodifiableMap(aliveTabletServersById); this.aliveTabletServers = Collections.unmodifiableList(new ArrayList<>(aliveTabletServersById.values())); this.tableIdByPath = Collections.unmodifiableMap(tableIdByPath); this.partitionsIdByPath = Collections.unmodifiableMap(partitionsIdByPath); - this.tableInfoByPath = Collections.unmodifiableMap(tableInfoByPath); // Index the bucket locations by table path, and index bucket location by bucket. // Note that this code is performance sensitive if there are a large number of buckets, @@ -136,8 +132,7 @@ public Cluster invalidPhysicalTableBucketMeta(Set physicalTab coordinatorServer, newBucketLocationsByPath, new HashMap<>(tableIdByPath), - new HashMap<>(partitionsIdByPath), - new HashMap<>(tableInfoByPath)); + new HashMap<>(partitionsIdByPath)); } @Nullable @@ -171,21 +166,6 @@ public TablePath getTablePathOrElseThrow(long tableId) { + " in cluster")); } - /** Get the table info for this table path. */ - public Optional getTableInfo(TablePath tablePath) { - return Optional.ofNullable(tableInfoByPath.get(tablePath)); - } - - public TableInfo getTableInfoOrElseThrow(TablePath tablePath) { - return getTableInfo(tablePath) - .orElseThrow( - () -> - new IllegalArgumentException( - "table info not found for tablePath " - + tablePath - + " in cluster")); - } - /** Get the bucket location for this table-bucket. */ public Optional getBucketLocation(TableBucket tableBucket) { return Optional.ofNullable(availableLocationByBucket.get(tableBucket)); @@ -278,10 +258,6 @@ public Map getPartitionIdByPath() { return partitionsIdByPath; } - public Map getTableInfoByPath() { - return tableInfoByPath; - } - /** Create an empty cluster instance with no nodes and no table-buckets. */ public static Cluster empty() { return new Cluster( @@ -289,7 +265,6 @@ public static Cluster empty() { null, Collections.emptyMap(), Collections.emptyMap(), - Collections.emptyMap(), Collections.emptyMap()); } diff --git a/fluss-common/src/test/java/org/apache/fluss/cluster/ClusterTest.java b/fluss-common/src/test/java/org/apache/fluss/cluster/ClusterTest.java index d1ed342832..ee23c9109a 100644 --- a/fluss-common/src/test/java/org/apache/fluss/cluster/ClusterTest.java +++ b/fluss-common/src/test/java/org/apache/fluss/cluster/ClusterTest.java @@ -18,8 +18,6 @@ package org.apache.fluss.cluster; import org.apache.fluss.metadata.PhysicalTablePath; -import org.apache.fluss.metadata.SchemaInfo; -import org.apache.fluss.metadata.TableInfo; import org.apache.fluss.metadata.TablePath; import org.junit.jupiter.api.BeforeEach; @@ -34,13 +32,9 @@ import java.util.Set; import static org.apache.fluss.record.TestData.DATA1_PHYSICAL_TABLE_PATH; -import static org.apache.fluss.record.TestData.DATA1_SCHEMA; import static org.apache.fluss.record.TestData.DATA1_TABLE_ID; -import static org.apache.fluss.record.TestData.DATA1_TABLE_INFO; import static org.apache.fluss.record.TestData.DATA1_TABLE_PATH; -import static org.apache.fluss.record.TestData.DATA2_SCHEMA; import static org.apache.fluss.record.TestData.DATA2_TABLE_ID; -import static org.apache.fluss.record.TestData.DATA2_TABLE_INFO; import static org.apache.fluss.record.TestData.DATA2_TABLE_PATH; import static org.assertj.core.api.Assertions.assertThat; import static org.assertj.core.api.Assertions.assertThatThrownBy; @@ -87,17 +81,6 @@ void testReturnModifiableCollections() { .isInstanceOf(UnsupportedOperationException.class); } - @Test - void testGetTable() { - Cluster cluster = createCluster(aliveTabletServersById); - assertThat(cluster.getTableInfo(DATA1_TABLE_PATH).get()).isEqualTo(DATA1_TABLE_INFO); - assertThat(cluster.getTableInfo(DATA2_TABLE_PATH).get()).isEqualTo(DATA2_TABLE_INFO); - assertThat(new SchemaInfo(cluster.getTableInfo(DATA1_TABLE_PATH).get().getSchema(), 1)) - .isEqualTo(new SchemaInfo(DATA1_SCHEMA, 1)); - assertThat(new SchemaInfo(cluster.getTableInfo(DATA2_TABLE_PATH).get().getSchema(), 1)) - .isEqualTo(new SchemaInfo(DATA2_SCHEMA, 1)); - } - @Test void testInvalidMetaAndUpdate() { Cluster cluster = createCluster(aliveTabletServersById); @@ -113,8 +96,7 @@ void testInvalidMetaAndUpdate() { COORDINATOR_SERVER, new HashMap<>(cluster.getBucketLocationsByPath()), new HashMap<>(cluster.getTableIdByPath()), - Collections.emptyMap(), - new HashMap<>(cluster.getTableInfoByPath())); + Collections.emptyMap()); } // verify available buckets @@ -190,16 +172,11 @@ private Cluster createCluster(Map aliveTabletServersById) { tablePathToTableId.put(DATA1_TABLE_PATH, DATA1_TABLE_ID); tablePathToTableId.put(DATA2_TABLE_PATH, DATA2_TABLE_ID); - Map tablePathToTableInfo = new HashMap<>(); - tablePathToTableInfo.put(DATA1_TABLE_PATH, DATA1_TABLE_INFO); - tablePathToTableInfo.put(DATA2_TABLE_PATH, DATA2_TABLE_INFO); - return new Cluster( aliveTabletServersById, COORDINATOR_SERVER, tablePathToBucketLocations, tablePathToTableId, - Collections.emptyMap(), - tablePathToTableInfo); + Collections.emptyMap()); } } diff --git a/fluss-flink/fluss-flink-common/src/main/java/org/apache/fluss/flink/lake/reader/LakeSnapshotAndLogSplitScanner.java b/fluss-flink/fluss-flink-common/src/main/java/org/apache/fluss/flink/lake/reader/LakeSnapshotAndLogSplitScanner.java index e734eb3083..9d3ce73790 100644 --- a/fluss-flink/fluss-flink-common/src/main/java/org/apache/fluss/flink/lake/reader/LakeSnapshotAndLogSplitScanner.java +++ b/fluss-flink/fluss-flink-common/src/main/java/org/apache/fluss/flink/lake/reader/LakeSnapshotAndLogSplitScanner.java @@ -219,8 +219,8 @@ public CloseableIterator pollBatch(Duration timeout) throws IOExcep } private void pollLogRecords(Duration timeout) { - ScanRecords scanRecords = logScanner.poll(timeout); - for (ScanRecord scanRecord : scanRecords) { + ScanRecords scanRecords = logScanner.poll(timeout); + for (ScanRecord scanRecord : scanRecords) { boolean isDelete = scanRecord.getChangeType() == ChangeType.DELETE || scanRecord.getChangeType() == ChangeType.UPDATE_BEFORE; diff --git a/fluss-flink/fluss-flink-common/src/main/java/org/apache/fluss/flink/source/emitter/FlinkRecordEmitter.java b/fluss-flink/fluss-flink-common/src/main/java/org/apache/fluss/flink/source/emitter/FlinkRecordEmitter.java index a413ca7ffc..82a3ed87e6 100644 --- a/fluss-flink/fluss-flink-common/src/main/java/org/apache/fluss/flink/source/emitter/FlinkRecordEmitter.java +++ b/fluss-flink/fluss-flink-common/src/main/java/org/apache/fluss/flink/source/emitter/FlinkRecordEmitter.java @@ -24,9 +24,6 @@ import org.apache.fluss.flink.source.reader.RecordAndPos; import org.apache.fluss.flink.source.split.HybridSnapshotLogSplitState; import org.apache.fluss.flink.source.split.SourceSplitState; -import org.apache.fluss.record.ChangeType; -import org.apache.fluss.record.LogRecord; -import org.apache.fluss.row.InternalRow; import org.apache.flink.api.connector.source.SourceOutput; import org.apache.flink.connector.base.source.reader.RecordEmitter; @@ -91,7 +88,7 @@ public void emitRecord( private void processAndEmitRecord(ScanRecord scanRecord, SourceOutput sourceOutput) { OUT record; try { - record = deserializationSchema.deserialize(new ScanRecordLogRecord(scanRecord)); + record = deserializationSchema.deserialize(scanRecord); } catch (Exception e) { throw new RuntimeException( "Failed to deserialize record: " + scanRecord + ". Cause: " + e.getMessage(), @@ -107,34 +104,4 @@ record = deserializationSchema.deserialize(new ScanRecordLogRecord(scanRecord)); } } } - - /** Lightweight adapter to view a {@code ScanRecord} as a {@link LogRecord}. */ - private static final class ScanRecordLogRecord implements LogRecord { - private final ScanRecord delegate; - - private ScanRecordLogRecord(ScanRecord delegate) { - // unchecked, but producer in this module always uses InternalRow - this.delegate = (ScanRecord) delegate; - } - - @Override - public long logOffset() { - return delegate.logOffset(); - } - - @Override - public long timestamp() { - return delegate.timestamp(); - } - - @Override - public ChangeType getChangeType() { - return delegate.getChangeType(); - } - - @Override - public InternalRow getRow() { - return delegate.getRow(); - } - } } diff --git a/fluss-flink/fluss-flink-common/src/main/java/org/apache/fluss/flink/source/lookup/FlinkAsyncLookupFunction.java b/fluss-flink/fluss-flink-common/src/main/java/org/apache/fluss/flink/source/lookup/FlinkAsyncLookupFunction.java index 5258e80d30..82fd1bbc36 100644 --- a/fluss-flink/fluss-flink-common/src/main/java/org/apache/fluss/flink/source/lookup/FlinkAsyncLookupFunction.java +++ b/fluss-flink/fluss-flink-common/src/main/java/org/apache/fluss/flink/source/lookup/FlinkAsyncLookupFunction.java @@ -20,7 +20,6 @@ import org.apache.fluss.client.Connection; import org.apache.fluss.client.ConnectionFactory; import org.apache.fluss.client.lookup.Lookup; -import org.apache.fluss.client.lookup.LookupResult; import org.apache.fluss.client.lookup.LookupType; import org.apache.fluss.client.lookup.Lookuper; import org.apache.fluss.client.table.Table; @@ -68,7 +67,7 @@ public class FlinkAsyncLookupFunction extends AsyncLookupFunction { private transient FlussRowToFlinkRowConverter flussRowToFlinkRowConverter; private transient Connection connection; private transient Table table; - private transient Lookuper lookuper; + private transient Lookuper lookuper; private transient FlinkAsFlussRow lookupRow; public FlinkAsyncLookupFunction( @@ -133,7 +132,7 @@ public CompletableFuture> asyncLookup(RowData keyRow) { CompletableFuture> future = new CompletableFuture<>(); lookuper.lookup(flussKeyRow) .whenComplete( - (LookupResult result, Throwable throwable) -> { + (result, throwable) -> { if (throwable != null) { if (ExceptionUtils.findThrowable( throwable, TableNotExistException.class) diff --git a/fluss-flink/fluss-flink-common/src/main/java/org/apache/fluss/flink/source/lookup/FlinkLookupFunction.java b/fluss-flink/fluss-flink-common/src/main/java/org/apache/fluss/flink/source/lookup/FlinkLookupFunction.java index acd3f73112..4776667375 100644 --- a/fluss-flink/fluss-flink-common/src/main/java/org/apache/fluss/flink/source/lookup/FlinkLookupFunction.java +++ b/fluss-flink/fluss-flink-common/src/main/java/org/apache/fluss/flink/source/lookup/FlinkLookupFunction.java @@ -62,7 +62,7 @@ public class FlinkLookupFunction extends LookupFunction { private transient FlussRowToFlinkRowConverter flussRowToFlinkRowConverter; private transient Connection connection; private transient Table table; - private transient Lookuper lookuper; + private transient Lookuper lookuper; private transient FlinkAsFlussRow lookupRow; @Nullable private transient ProjectedRow projectedRow; diff --git a/fluss-flink/fluss-flink-common/src/main/java/org/apache/fluss/flink/source/reader/FlinkSourceSplitReader.java b/fluss-flink/fluss-flink-common/src/main/java/org/apache/fluss/flink/source/reader/FlinkSourceSplitReader.java index 6d6664ca4e..ce50fe4068 100644 --- a/fluss-flink/fluss-flink-common/src/main/java/org/apache/fluss/flink/source/reader/FlinkSourceSplitReader.java +++ b/fluss-flink/fluss-flink-common/src/main/java/org/apache/fluss/flink/source/reader/FlinkSourceSplitReader.java @@ -39,7 +39,6 @@ import org.apache.fluss.lake.source.LakeSplit; import org.apache.fluss.metadata.TableBucket; import org.apache.fluss.metadata.TablePath; -import org.apache.fluss.row.InternalRow; import org.apache.fluss.types.RowType; import org.apache.fluss.utils.CloseableIterator; import org.apache.fluss.utils.ExceptionUtils; @@ -411,7 +410,7 @@ private void checkSnapshotSplitOrStartNext() { } } - private FlinkRecordsWithSplitIds forLogRecords(ScanRecords scanRecords) { + private FlinkRecordsWithSplitIds forLogRecords(ScanRecords scanRecords) { // For calculating the currentFetchEventTimeLag long fetchTimestamp = System.currentTimeMillis(); long maxConsumerRecordTimestampInFetch = -1; @@ -430,7 +429,7 @@ private FlinkRecordsWithSplitIds forLogRecords(ScanRecords scanReco } splitIdByTableBucket.put(scanBucket, splitId); tableScanBuckets.add(scanBucket); - List> bucketScanRecords = scanRecords.records(scanBucket); + List bucketScanRecords = scanRecords.records(scanBucket); if (!bucketScanRecords.isEmpty()) { final ScanRecord lastRecord = bucketScanRecords.get(bucketScanRecords.size() - 1); // We keep the maximum message timestamp in the fetch for calculating lags @@ -482,7 +481,7 @@ public String next() { } private CloseableIterator toRecordAndPos( - Iterator> recordAndPosIterator) { + Iterator recordAndPosIterator) { return new CloseableIterator() { @Override diff --git a/fluss-flink/fluss-flink-common/src/main/java/org/apache/fluss/flink/tiering/source/TieringSplitReader.java b/fluss-flink/fluss-flink-common/src/main/java/org/apache/fluss/flink/tiering/source/TieringSplitReader.java index 649e14c2dc..b9fe79e3d3 100644 --- a/fluss-flink/fluss-flink-common/src/main/java/org/apache/fluss/flink/tiering/source/TieringSplitReader.java +++ b/fluss-flink/fluss-flink-common/src/main/java/org/apache/fluss/flink/tiering/source/TieringSplitReader.java @@ -32,9 +32,6 @@ import org.apache.fluss.metadata.TableBucket; import org.apache.fluss.metadata.TableInfo; import org.apache.fluss.metadata.TablePath; -import org.apache.fluss.record.ChangeType; -import org.apache.fluss.record.LogRecord; -import org.apache.fluss.row.InternalRow; import org.apache.fluss.utils.CloseableIterator; import org.apache.flink.connector.base.source.reader.RecordsWithSplitIds; @@ -252,12 +249,11 @@ private void mayCreateLogScanner() { } private RecordsWithSplitIds> forLogRecords( - ScanRecords scanRecords) throws IOException { + ScanRecords scanRecords) throws IOException { Map> writeResults = new HashMap<>(); Map finishedSplitIds = new HashMap<>(); for (TableBucket bucket : scanRecords.buckets()) { - List> bucketScanRecords = - scanRecords.records(bucket); + List bucketScanRecords = scanRecords.records(bucket); if (bucketScanRecords.isEmpty()) { continue; } @@ -269,14 +265,13 @@ private RecordsWithSplitIds> forLogRecords( LakeWriter lakeWriter = getOrCreateLakeWriter( bucket, currentTableSplitsByBucket.get(bucket).getPartitionName()); - for (ScanRecord record : bucketScanRecords) { + for (ScanRecord record : bucketScanRecords) { // if record is less than stopping offset if (record.logOffset() < stoppingOffset) { - lakeWriter.write(new ScanRecordLogRecord(record)); + lakeWriter.write(record); } } - ScanRecord lastRecord = - bucketScanRecords.get(bucketScanRecords.size() - 1); + ScanRecord lastRecord = bucketScanRecords.get(bucketScanRecords.size() - 1); // has arrived into the end of the split, if (lastRecord.logOffset() >= stoppingOffset - 1) { currentTableStoppingOffsets.remove(bucket); @@ -395,8 +390,8 @@ private TableBucketWriteResultWithSplitIds forSnapshotSplitRecords( getOrCreateLakeWriter( bucket, checkNotNull(currentSnapshotSplit).getPartitionName()); while (recordIterator.hasNext()) { - ScanRecord scanRecord = recordIterator.next().record(); - lakeWriter.write(new ScanRecordLogRecord(scanRecord)); + ScanRecord scanRecord = recordIterator.next().record(); + lakeWriter.write(scanRecord); } recordIterator.close(); return emptyTableBucketWriteResultWithSplitIds(); @@ -465,35 +460,6 @@ public void close() throws Exception { // don't need to close connection, will be closed by TieringSourceReader } - /** Lightweight adapter to view a {@code ScanRecord} as a {@link LogRecord}. */ - private static final class ScanRecordLogRecord implements LogRecord { - private final ScanRecord delegate; - - private ScanRecordLogRecord(ScanRecord delegate) { - this.delegate = delegate; - } - - @Override - public long logOffset() { - return delegate.logOffset(); - } - - @Override - public long timestamp() { - return delegate.timestamp(); - } - - @Override - public ChangeType getChangeType() { - return delegate.getChangeType(); - } - - @Override - public InternalRow getRow() { - return delegate.getRow(); - } - } - private void subscribeLog(TieringLogSplit logSplit) { // assign bucket offset dynamically TableBucket tableBucket = logSplit.getTableBucket(); 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 bde62abd0c..b2d4e47086 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 @@ -159,9 +159,9 @@ public void testRowDataTablePKSink() throws Exception { List rows = new ArrayList<>(); while (rows.size() < inputRows.size()) { - ScanRecords scanRecords = logScanner.poll(Duration.ofSeconds(1)); + ScanRecords scanRecords = logScanner.poll(Duration.ofSeconds(1)); for (TableBucket bucket : scanRecords.buckets()) { - for (ScanRecord record : scanRecords.records(bucket)) { + for (ScanRecord record : scanRecords.records(bucket)) { RowData row = converter.toFlinkRowData(record.getRow()); row.setRowKind(toFlinkRowKind(record.getChangeType())); rows.add(row); @@ -232,9 +232,9 @@ public void testRowDataTableLogSink() throws Exception { List rows = new ArrayList<>(); while (rows.size() < inputRows.size()) { - ScanRecords scanRecords = logScanner.poll(Duration.ofSeconds(1)); + ScanRecords scanRecords = logScanner.poll(Duration.ofSeconds(1)); for (TableBucket bucket : scanRecords.buckets()) { - for (ScanRecord record : scanRecords.records(bucket)) { + for (ScanRecord record : scanRecords.records(bucket)) { RowData row = converter.toFlinkRowData(record.getRow()); row.setRowKind(toFlinkRowKind(record.getChangeType())); rows.add(row); @@ -288,9 +288,9 @@ public void testOrdersTablePKSink() throws Exception { List rows = new ArrayList<>(); while (rows.size() < orders.size()) { - ScanRecords scanRecords = logScanner.poll(Duration.ofSeconds(1)); + ScanRecords scanRecords = logScanner.poll(Duration.ofSeconds(1)); for (TableBucket bucket : scanRecords.buckets()) { - for (ScanRecord record : scanRecords.records(bucket)) { + for (ScanRecord record : scanRecords.records(bucket)) { InternalRow row = record.getRow(); TestOrder order = new TestOrder( diff --git a/fluss-flink/fluss-flink-common/src/test/java/org/apache/fluss/flink/source/deserializer/FlussDeserializationSchemaTest.java b/fluss-flink/fluss-flink-common/src/test/java/org/apache/fluss/flink/source/deserializer/FlussDeserializationSchemaTest.java index 12f7ba1c8a..8a5f3b81d5 100644 --- a/fluss-flink/fluss-flink-common/src/test/java/org/apache/fluss/flink/source/deserializer/FlussDeserializationSchemaTest.java +++ b/fluss-flink/fluss-flink-common/src/test/java/org/apache/fluss/flink/source/deserializer/FlussDeserializationSchemaTest.java @@ -21,11 +21,9 @@ import org.apache.fluss.flink.source.testutils.Order; import org.apache.fluss.flink.source.testutils.OrderDeserializationSchema; import org.apache.fluss.record.ChangeType; -import org.apache.fluss.record.LogRecord; import org.apache.fluss.row.BinaryString; import org.apache.fluss.row.Decimal; import org.apache.fluss.row.GenericRow; -import org.apache.fluss.row.InternalRow; import org.apache.fluss.row.TimestampLtz; import org.apache.fluss.row.TimestampNtz; import org.apache.fluss.types.DataField; @@ -52,36 +50,6 @@ * conversion from Fluss records to various target formats. */ public class FlussDeserializationSchemaTest { - - /** Lightweight adapter to view a {@code ScanRecord} as a {@link LogRecord}. */ - private static final class ScanRecordLogRecord implements LogRecord { - private final ScanRecord delegate; - - private ScanRecordLogRecord(ScanRecord delegate) { - this.delegate = delegate; - } - - @Override - public long logOffset() { - return delegate.logOffset(); - } - - @Override - public long timestamp() { - return delegate.timestamp(); - } - - @Override - public ChangeType getChangeType() { - return delegate.getChangeType(); - } - - @Override - public InternalRow getRow() { - return delegate.getRow(); - } - } - @Test public void testDeserialize() throws Exception { // Create GenericRow with proper types @@ -94,7 +62,7 @@ public void testDeserialize() throws Exception { ScanRecord scanRecord = new ScanRecord(row); OrderDeserializationSchema deserializer = new OrderDeserializationSchema(); - Order result = deserializer.deserialize(new ScanRecordLogRecord(scanRecord)); + Order result = deserializer.deserialize(scanRecord); assertThat(result.getOrderId()).isEqualTo(1001L); assertThat(result.getItemId()).isEqualTo(5001L); @@ -113,7 +81,7 @@ public void testDeserializeWithNumericConversion() throws Exception { ScanRecord scanRecord = new ScanRecord(row); OrderDeserializationSchema schema = new OrderDeserializationSchema(); - Order result = schema.deserialize(new ScanRecordLogRecord(scanRecord)); + Order result = schema.deserialize(scanRecord); assertThat(result.getOrderId()).isEqualTo(1002L); assertThat(result.getItemId()).isEqualTo(5002L); @@ -132,7 +100,7 @@ public void testDeserializeWithNullValues() throws Exception { ScanRecord scanRecord = new ScanRecord(row); OrderDeserializationSchema schema = new OrderDeserializationSchema(); - Order result = schema.deserialize(new ScanRecordLogRecord(scanRecord)); + Order result = schema.deserialize(scanRecord); assertThat(result.getOrderId()).isEqualTo(1003L); assertThat(result.getItemId()).isEqualTo(5003L); @@ -213,8 +181,8 @@ public void testJsonStringDeserialize() throws Exception { // Create deserializer JsonStringDeserializationSchema deserializer = new JsonStringDeserializationSchema(); // Test deserialization - deserializer.open(new DeserializerInitContextImpl(null, null, rowType)); - String result = deserializer.deserialize(new ScanRecordLogRecord(scanRecord)); + deserializer.open(new DeserializerInitContextImpl(null, null, sourceRowType)); + String result = deserializer.deserialize(scanRecord); String rowJson = "{" @@ -246,7 +214,7 @@ public void testJsonStringDeserialize() throws Exception { // Verify with offset and timestamp ScanRecord scanRecord2 = new ScanRecord(1001, 1743261788400L, ChangeType.DELETE, row); - String result2 = deserializer.deserialize(new ScanRecordLogRecord(scanRecord2)); + String result2 = deserializer.deserialize(scanRecord2); assertThat(result2).isNotNull(); assertThat(result2) .isEqualTo( @@ -260,7 +228,7 @@ public void testJsonStringDeserialize() throws Exception { row.setField(8, 512); row.setField(13, 72000000); ScanRecord changedRecord = new ScanRecord(row); - String changedResult = deserializer.deserialize(new ScanRecordLogRecord(changedRecord)); + String changedResult = deserializer.deserialize(changedRecord); String changedRowJson = "{" + "\"char\":\"b\"," diff --git a/fluss-flink/fluss-flink-common/src/test/java/org/apache/fluss/flink/source/deserializer/RowDataDeserializationSchemaTest.java b/fluss-flink/fluss-flink-common/src/test/java/org/apache/fluss/flink/source/deserializer/RowDataDeserializationSchemaTest.java index 146031c2ff..8abbbac8bb 100644 --- a/fluss-flink/fluss-flink-common/src/test/java/org/apache/fluss/flink/source/deserializer/RowDataDeserializationSchemaTest.java +++ b/fluss-flink/fluss-flink-common/src/test/java/org/apache/fluss/flink/source/deserializer/RowDataDeserializationSchemaTest.java @@ -18,11 +18,8 @@ package org.apache.fluss.flink.source.deserializer; import org.apache.fluss.client.table.scanner.ScanRecord; -import org.apache.fluss.record.ChangeType; -import org.apache.fluss.record.LogRecord; import org.apache.fluss.row.BinaryString; import org.apache.fluss.row.GenericRow; -import org.apache.fluss.row.InternalRow; import org.apache.fluss.types.DataField; import org.apache.fluss.types.DataTypes; import org.apache.fluss.types.RowType; @@ -80,10 +77,10 @@ public void testDeserialize() throws Exception { row.setField(2, 45); row.setField(3, BinaryString.fromString("Test addr")); - ScanRecord scanRecord = new ScanRecord<>(row); + ScanRecord scanRecord = new ScanRecord(row); RowDataDeserializationSchema deserializer = getRowDataDeserializationSchema(rowType); - RowData result = deserializer.deserialize(new ScanRecordLogRecord(scanRecord)); + RowData result = deserializer.deserialize(scanRecord); assertThat(result.getArity()).isEqualTo(4); assertThat(result.getLong(0)).isEqualTo(100L); @@ -92,35 +89,6 @@ public void testDeserialize() throws Exception { assertThat(result.getString(3).toString()).isEqualTo("Test addr"); } - /** Lightweight adapter to view a {@code ScanRecord} as a {@link LogRecord}. */ - private static final class ScanRecordLogRecord implements LogRecord { - private final ScanRecord delegate; - - private ScanRecordLogRecord(ScanRecord delegate) { - this.delegate = delegate; - } - - @Override - public long logOffset() { - return delegate.logOffset(); - } - - @Override - public long timestamp() { - return delegate.timestamp(); - } - - @Override - public ChangeType getChangeType() { - return delegate.getChangeType(); - } - - @Override - public InternalRow getRow() { - return delegate.getRow(); - } - } - private @NotNull RowDataDeserializationSchema getRowDataDeserializationSchema(RowType rowType) throws Exception { RowDataDeserializationSchema deserializationSchema = new RowDataDeserializationSchema(); diff --git a/fluss-flink/fluss-flink-common/src/test/java/org/apache/fluss/flink/tiering/source/ScanRecordLogRecordTest.java b/fluss-flink/fluss-flink-common/src/test/java/org/apache/fluss/flink/tiering/source/ScanRecordLogRecordTest.java index d4cc961d11..22002d7123 100644 --- a/fluss-flink/fluss-flink-common/src/test/java/org/apache/fluss/flink/tiering/source/ScanRecordLogRecordTest.java +++ b/fluss-flink/fluss-flink-common/src/test/java/org/apache/fluss/flink/tiering/source/ScanRecordLogRecordTest.java @@ -21,7 +21,6 @@ import org.apache.fluss.record.ChangeType; import org.apache.fluss.record.LogRecord; import org.apache.fluss.row.GenericRow; -import org.apache.fluss.row.InternalRow; import org.junit.jupiter.api.Test; @@ -41,7 +40,7 @@ void testAdapterDelegatesAllMethods() throws Exception { long offset = 1234L; long timestamp = 987654321L; ChangeType changeType = ChangeType.UPDATE_AFTER; - ScanRecord scanRecord = new ScanRecord<>(offset, timestamp, changeType, row); + ScanRecord scanRecord = new ScanRecord(offset, timestamp, changeType, row); // Reflectively construct the private static inner class Class clazz = diff --git a/fluss-flink/fluss-flink-common/src/test/java/org/apache/fluss/flink/utils/FlussRowToFlinkRowConverterTest.java b/fluss-flink/fluss-flink-common/src/test/java/org/apache/fluss/flink/utils/FlussRowToFlinkRowConverterTest.java index 39848864e9..0c7515f3ae 100644 --- a/fluss-flink/fluss-flink-common/src/test/java/org/apache/fluss/flink/utils/FlussRowToFlinkRowConverterTest.java +++ b/fluss-flink/fluss-flink-common/src/test/java/org/apache/fluss/flink/utils/FlussRowToFlinkRowConverterTest.java @@ -20,9 +20,7 @@ import org.apache.fluss.client.table.scanner.ScanRecord; import org.apache.fluss.flink.row.FlinkAsFlussArray; import org.apache.fluss.record.ChangeType; -import org.apache.fluss.record.LogRecord; import org.apache.fluss.row.BinaryString; -import org.apache.fluss.row.InternalRow; import org.apache.fluss.row.indexed.IndexedRow; import org.apache.fluss.row.indexed.IndexedRowWriter; import org.apache.fluss.types.DataType; @@ -58,10 +56,9 @@ void testConverter() throws Exception { try (IndexedRowWriter writer = genRecordForAllTypes(createAllTypes())) { row.pointTo(writer.segment(), 0, writer.position()); - ScanRecord scanRecord = - new ScanRecord<>(0, 1L, ChangeType.UPDATE_BEFORE, row); - RowData flinkRow = - flussRowToFlinkRowConverter.toFlinkRowData(new ScanRecordLogRecord(scanRecord)); + ScanRecord scanRecord = new ScanRecord(0, 1L, ChangeType.UPDATE_BEFORE, row); + + RowData flinkRow = flussRowToFlinkRowConverter.toFlinkRowData(scanRecord); assertThat(flinkRow.getArity()).isEqualTo(rowType.getFieldCount()); assertThat(flinkRow.getRowKind()).isEqualTo(RowKind.UPDATE_BEFORE); @@ -129,33 +126,4 @@ void testConverter() throws Exception { .isEqualTo(new BinaryString[] {fromString("hello"), fromString("world")}); } } - - /** Lightweight adapter to view a {@code ScanRecord} as a {@link LogRecord}. */ - private static final class ScanRecordLogRecord implements LogRecord { - private final ScanRecord delegate; - - private ScanRecordLogRecord(ScanRecord delegate) { - this.delegate = delegate; - } - - @Override - public long logOffset() { - return delegate.logOffset(); - } - - @Override - public long timestamp() { - return delegate.timestamp(); - } - - @Override - public ChangeType getChangeType() { - return delegate.getChangeType(); - } - - @Override - public InternalRow getRow() { - return delegate.getRow(); - } - } } From c1975b3e92a9cca4db69b1b3f7bcf779e2386a62 Mon Sep 17 00:00:00 2001 From: ipolyzos Date: Mon, 22 Dec 2025 11:22:11 +0200 Subject: [PATCH 30/33] address comment for the lookuper --- .../fluss/client/lookup/TableLookup.java | 2 +- .../client/lookup/TypedLookuperImpl.java | 19 +++++++++++++------ .../admin/ClientToServerITCaseBase.java | 8 ++++++++ 3 files changed, 22 insertions(+), 7 deletions(-) diff --git a/fluss-client/src/main/java/org/apache/fluss/client/lookup/TableLookup.java b/fluss-client/src/main/java/org/apache/fluss/client/lookup/TableLookup.java index 31766b3921..e0c92661f1 100644 --- a/fluss-client/src/main/java/org/apache/fluss/client/lookup/TableLookup.java +++ b/fluss-client/src/main/java/org/apache/fluss/client/lookup/TableLookup.java @@ -74,6 +74,6 @@ public Lookuper createLookuper() { @Override public TypedLookuper createTypedLookuper(Class pojoClass) { - return new TypedLookuperImpl<>(createLookuper(), tableInfo, lookupColumnNames); + return new TypedLookuperImpl<>(createLookuper(), tableInfo, lookupColumnNames, pojoClass); } } diff --git a/fluss-client/src/main/java/org/apache/fluss/client/lookup/TypedLookuperImpl.java b/fluss-client/src/main/java/org/apache/fluss/client/lookup/TypedLookuperImpl.java index 9e29f58762..c88eb15d13 100644 --- a/fluss-client/src/main/java/org/apache/fluss/client/lookup/TypedLookuperImpl.java +++ b/fluss-client/src/main/java/org/apache/fluss/client/lookup/TypedLookuperImpl.java @@ -38,12 +38,17 @@ final class TypedLookuperImpl implements TypedLookuper { private final Lookuper delegate; private final TableInfo tableInfo; @Nullable private final List lookupColumnNames; + private final PojoToRowConverter keyConv; TypedLookuperImpl( - Lookuper delegate, TableInfo tableInfo, @Nullable List lookupColumnNames) { + Lookuper delegate, + TableInfo tableInfo, + @Nullable List lookupColumnNames, + Class keyClass) { this.delegate = delegate; this.tableInfo = tableInfo; this.lookupColumnNames = lookupColumnNames; + this.keyConv = createPojoToRowConverter(keyClass); } @Override @@ -55,6 +60,12 @@ public CompletableFuture lookup(K key) { if (key instanceof InternalRow) { return delegate.lookup((InternalRow) key); } + + InternalRow keyRow = keyConv.toRow(key); + return delegate.lookup(keyRow); + } + + private PojoToRowConverter createPojoToRowConverter(Class keyClass) { RowType tableSchema = tableInfo.getRowType(); RowType keyProjection; if (lookupColumnNames == null) { @@ -62,10 +73,6 @@ public CompletableFuture lookup(K key) { } else { keyProjection = tableSchema.project(lookupColumnNames); } - @SuppressWarnings("unchecked") - Class keyClass = (Class) key.getClass(); - PojoToRowConverter keyConv = PojoToRowConverter.of(keyClass, tableSchema, keyProjection); - InternalRow keyRow = keyConv.toRow(key); - return delegate.lookup(keyRow); + return PojoToRowConverter.of(keyClass, tableSchema, keyProjection); } } diff --git a/fluss-client/src/test/java/org/apache/fluss/client/admin/ClientToServerITCaseBase.java b/fluss-client/src/test/java/org/apache/fluss/client/admin/ClientToServerITCaseBase.java index a62c1b70d3..8270ed7882 100644 --- a/fluss-client/src/test/java/org/apache/fluss/client/admin/ClientToServerITCaseBase.java +++ b/fluss-client/src/test/java/org/apache/fluss/client/admin/ClientToServerITCaseBase.java @@ -26,6 +26,7 @@ import org.apache.fluss.client.table.scanner.ScanRecord; import org.apache.fluss.client.table.scanner.log.LogScanner; import org.apache.fluss.client.table.scanner.log.ScanRecords; +import org.apache.fluss.client.table.scanner.log.TypedLogScanner; import org.apache.fluss.client.table.writer.UpsertWriter; import org.apache.fluss.config.ConfigOptions; import org.apache.fluss.config.Configuration; @@ -144,6 +145,13 @@ protected static void subscribeFromBeginning(LogScanner logScanner, Table table) } } + protected static void subscribeFromBeginning(TypedLogScanner logScanner, Table table) { + int bucketCount = table.getTableInfo().getNumBuckets(); + for (int i = 0; i < bucketCount; i++) { + logScanner.subscribeFromBeginning(i); + } + } + protected static void subscribeFromTimestamp( TablePath tablePath, @Nullable String partitionName, From 084cc4ce3874e0e1cf0b67360d6597d11f41474d Mon Sep 17 00:00:00 2001 From: ipolyzos Date: Mon, 22 Dec 2025 11:29:48 +0200 Subject: [PATCH 31/33] delete unneeded test --- .../source/ScanRecordLogRecordTest.java | 64 ------------------- 1 file changed, 64 deletions(-) delete mode 100644 fluss-flink/fluss-flink-common/src/test/java/org/apache/fluss/flink/tiering/source/ScanRecordLogRecordTest.java diff --git a/fluss-flink/fluss-flink-common/src/test/java/org/apache/fluss/flink/tiering/source/ScanRecordLogRecordTest.java b/fluss-flink/fluss-flink-common/src/test/java/org/apache/fluss/flink/tiering/source/ScanRecordLogRecordTest.java deleted file mode 100644 index 22002d7123..0000000000 --- a/fluss-flink/fluss-flink-common/src/test/java/org/apache/fluss/flink/tiering/source/ScanRecordLogRecordTest.java +++ /dev/null @@ -1,64 +0,0 @@ -/* - * 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.tiering.source; - -import org.apache.fluss.client.table.scanner.ScanRecord; -import org.apache.fluss.record.ChangeType; -import org.apache.fluss.record.LogRecord; -import org.apache.fluss.row.GenericRow; - -import org.junit.jupiter.api.Test; - -import java.lang.reflect.Constructor; - -import static org.assertj.core.api.Assertions.assertThat; - -/** Tests for the private adapter TieringSplitReader.ScanRecordLogRecord. */ -class ScanRecordLogRecordTest { - - @Test - void testAdapterDelegatesAllMethods() throws Exception { - // Prepare a simple InternalRow and wrap it in a ScanRecord with known attributes - GenericRow row = new GenericRow(1); - row.setField(0, 42); // content is irrelevant for this test - - long offset = 1234L; - long timestamp = 987654321L; - ChangeType changeType = ChangeType.UPDATE_AFTER; - ScanRecord scanRecord = new ScanRecord(offset, timestamp, changeType, row); - - // Reflectively construct the private static inner class - Class clazz = - Class.forName( - "org.apache.fluss.flink.tiering.source.TieringSplitReader$ScanRecordLogRecord"); - @SuppressWarnings("unchecked") - Constructor ctor = - (Constructor) clazz.getDeclaredConstructor(ScanRecord.class); - ctor.setAccessible(true); - LogRecord adapter = ctor.newInstance(scanRecord); - - // Verify delegation - assertThat(adapter.logOffset()).isEqualTo(offset); - assertThat(adapter.timestamp()).isEqualTo(timestamp); - assertThat(adapter.getChangeType()).isEqualTo(changeType); - assertThat(adapter.getRow()).isSameAs(row); - - assertThat(adapter.logOffset()).isEqualTo(offset); - assertThat(adapter.getRow()).isSameAs(row); - } -} From 604e6f6567dd87f83e94c38809741c10b1687533 Mon Sep 17 00:00:00 2001 From: ipolyzos Date: Mon, 22 Dec 2025 12:03:40 +0200 Subject: [PATCH 32/33] make the api consistent --- .../org/apache/fluss/client/table/writer/Append.java | 2 +- .../fluss/client/table/writer/TableAppend.java | 2 +- .../fluss/client/table/writer/TableUpsert.java | 2 +- .../org/apache/fluss/client/table/writer/Upsert.java | 2 +- .../fluss/client/table/FlussTypedClientITCase.java | 12 ++++++------ 5 files changed, 10 insertions(+), 10 deletions(-) diff --git a/fluss-client/src/main/java/org/apache/fluss/client/table/writer/Append.java b/fluss-client/src/main/java/org/apache/fluss/client/table/writer/Append.java index 4e705e85de..06a0b694e6 100644 --- a/fluss-client/src/main/java/org/apache/fluss/client/table/writer/Append.java +++ b/fluss-client/src/main/java/org/apache/fluss/client/table/writer/Append.java @@ -36,5 +36,5 @@ public interface Append { AppendWriter createWriter(); /** Create a new typed {@link AppendWriter} to write POJOs directly. */ - TypedAppendWriter createWriter(Class pojoClass); + TypedAppendWriter createTypedWriter(Class pojoClass); } diff --git a/fluss-client/src/main/java/org/apache/fluss/client/table/writer/TableAppend.java b/fluss-client/src/main/java/org/apache/fluss/client/table/writer/TableAppend.java index 6e93c885d6..5952762e22 100644 --- a/fluss-client/src/main/java/org/apache/fluss/client/table/writer/TableAppend.java +++ b/fluss-client/src/main/java/org/apache/fluss/client/table/writer/TableAppend.java @@ -40,7 +40,7 @@ public AppendWriter createWriter() { } @Override - public TypedAppendWriter createWriter(Class pojoClass) { + public TypedAppendWriter createTypedWriter(Class pojoClass) { AppendWriterImpl delegate = new AppendWriterImpl(tablePath, tableInfo, writerClient); return new TypedAppendWriterImpl<>(delegate, pojoClass, tableInfo); } diff --git a/fluss-client/src/main/java/org/apache/fluss/client/table/writer/TableUpsert.java b/fluss-client/src/main/java/org/apache/fluss/client/table/writer/TableUpsert.java index 87d8187f06..fea17cdf27 100644 --- a/fluss-client/src/main/java/org/apache/fluss/client/table/writer/TableUpsert.java +++ b/fluss-client/src/main/java/org/apache/fluss/client/table/writer/TableUpsert.java @@ -97,7 +97,7 @@ public UpsertWriter createWriter() { } @Override - public TypedUpsertWriter createWriter(Class pojoClass) { + public TypedUpsertWriter createTypedWriter(Class pojoClass) { UpsertWriterImpl delegate = new UpsertWriterImpl(tablePath, tableInfo, targetColumns, writerClient); return new TypedUpsertWriterImpl<>(delegate, pojoClass, tableInfo, targetColumns); diff --git a/fluss-client/src/main/java/org/apache/fluss/client/table/writer/Upsert.java b/fluss-client/src/main/java/org/apache/fluss/client/table/writer/Upsert.java index c29875faf4..0843437fee 100644 --- a/fluss-client/src/main/java/org/apache/fluss/client/table/writer/Upsert.java +++ b/fluss-client/src/main/java/org/apache/fluss/client/table/writer/Upsert.java @@ -63,5 +63,5 @@ public interface Upsert { UpsertWriter createWriter(); /** Create a new typed {@link UpsertWriter} to write POJOs directly. */ - TypedUpsertWriter createWriter(Class pojoClass); + TypedUpsertWriter createTypedWriter(Class pojoClass); } diff --git a/fluss-client/src/test/java/org/apache/fluss/client/table/FlussTypedClientITCase.java b/fluss-client/src/test/java/org/apache/fluss/client/table/FlussTypedClientITCase.java index d58615dfc3..2d8a859671 100644 --- a/fluss-client/src/test/java/org/apache/fluss/client/table/FlussTypedClientITCase.java +++ b/fluss-client/src/test/java/org/apache/fluss/client/table/FlussTypedClientITCase.java @@ -227,7 +227,7 @@ void testTypedAppendWriteAndScan() throws Exception { try (Table table = conn.getTable(path)) { // write TypedAppendWriter writer = - table.newAppend().createWriter(AllTypesPojo.class); + table.newAppend().createTypedWriter(AllTypesPojo.class); List expected = new ArrayList<>(); for (int i = 0; i < 5; i++) { AllTypesPojo u = newAllTypesPojo(i); @@ -265,7 +265,7 @@ void testTypedUpsertWriteAndScan() throws Exception { try (Table table = conn.getTable(path)) { Upsert upsert = table.newUpsert(); - TypedUpsertWriter writer = upsert.createWriter(AllTypesPojo.class); + TypedUpsertWriter writer = upsert.createTypedWriter(AllTypesPojo.class); AllTypesPojo p1 = newAllTypesPojo(1); AllTypesPojo p2 = newAllTypesPojo(2); @@ -310,7 +310,7 @@ void testTypedLookups() throws Exception { try (Table table = conn.getTable(path)) { TypedUpsertWriter writer = - table.newUpsert().createWriter(AllTypesPojo.class); + table.newUpsert().createTypedWriter(AllTypesPojo.class); writer.upsert(newAllTypesPojo(1)).get(); writer.upsert(newAllTypesPojo(2)).get(); writer.close(); @@ -338,7 +338,7 @@ void testInternalRowLookup() throws Exception { try (Table table = conn.getTable(path)) { // write a couple of rows via POJO writer TypedUpsertWriter writer = - table.newUpsert().createWriter(AllTypesPojo.class); + table.newUpsert().createTypedWriter(AllTypesPojo.class); writer.upsert(newAllTypesPojo(101)).get(); writer.upsert(newAllTypesPojo(202)).get(); writer.close(); @@ -371,7 +371,7 @@ void testTypedProjections() throws Exception { try (Table table = conn.getTable(path)) { TypedAppendWriter writer = - table.newAppend().createWriter(AllTypesPojo.class); + table.newAppend().createTypedWriter(AllTypesPojo.class); writer.append(newAllTypesPojo(10)).get(); writer.append(newAllTypesPojo(11)).get(); writer.flush(); @@ -410,7 +410,7 @@ void testTypedPartialUpdates() throws Exception { try (Table table = conn.getTable(path)) { Upsert upsert = table.newUpsert().partialUpdate("a", "str", "dec"); - TypedUpsertWriter writer = upsert.createWriter(AllTypesPojo.class); + TypedUpsertWriter writer = upsert.createTypedWriter(AllTypesPojo.class); // initial full row writer.upsert(newAllTypesPojo(1)).get(); From 99b0d06842f3d96ea5f0690a9c1f7ca6823c14c3 Mon Sep 17 00:00:00 2001 From: ipolyzos Date: Tue, 23 Dec 2025 16:50:42 +0200 Subject: [PATCH 33/33] address comments --- .../table/scanner/log/LogScannerImpl.java | 2 +- .../client/table/scanner/log/ScanRecords.java | 4 +- .../client/table/writer/TableAppend.java | 3 +- .../client/table/writer/TableUpsert.java | 4 +- .../client/table/writer/TableWriter.java | 7 +- .../table/writer/TypedAppendWriterImpl.java | 11 +- .../table/writer/TypedUpsertWriterImpl.java | 30 ++-- .../client/table/FlussTypedClientITCase.java | 162 +++++++++++++++--- 8 files changed, 152 insertions(+), 71 deletions(-) diff --git a/fluss-client/src/main/java/org/apache/fluss/client/table/scanner/log/LogScannerImpl.java b/fluss-client/src/main/java/org/apache/fluss/client/table/scanner/log/LogScannerImpl.java index 2a43a4824a..b50bc3622a 100644 --- a/fluss-client/src/main/java/org/apache/fluss/client/table/scanner/log/LogScannerImpl.java +++ b/fluss-client/src/main/java/org/apache/fluss/client/table/scanner/log/LogScannerImpl.java @@ -163,7 +163,7 @@ public ScanRecords poll(Duration timeout) { } } while (System.nanoTime() - startNanos < timeoutNanos); - return ScanRecords.empty(); + return ScanRecords.EMPTY; } finally { release(); scannerMetricGroup.recordPollEnd(System.currentTimeMillis()); diff --git a/fluss-client/src/main/java/org/apache/fluss/client/table/scanner/log/ScanRecords.java b/fluss-client/src/main/java/org/apache/fluss/client/table/scanner/log/ScanRecords.java index f852ba9abd..9d58c22b49 100644 --- a/fluss-client/src/main/java/org/apache/fluss/client/table/scanner/log/ScanRecords.java +++ b/fluss-client/src/main/java/org/apache/fluss/client/table/scanner/log/ScanRecords.java @@ -37,9 +37,7 @@ */ @PublicEvolving public class ScanRecords implements Iterable { - public static final ScanRecords empty() { - return new ScanRecords(Collections.emptyMap()); - } + public static final ScanRecords EMPTY = new ScanRecords(Collections.emptyMap()); private final Map> records; diff --git a/fluss-client/src/main/java/org/apache/fluss/client/table/writer/TableAppend.java b/fluss-client/src/main/java/org/apache/fluss/client/table/writer/TableAppend.java index 5952762e22..84ccaf3195 100644 --- a/fluss-client/src/main/java/org/apache/fluss/client/table/writer/TableAppend.java +++ b/fluss-client/src/main/java/org/apache/fluss/client/table/writer/TableAppend.java @@ -41,7 +41,6 @@ public AppendWriter createWriter() { @Override public TypedAppendWriter createTypedWriter(Class pojoClass) { - AppendWriterImpl delegate = new AppendWriterImpl(tablePath, tableInfo, writerClient); - return new TypedAppendWriterImpl<>(delegate, pojoClass, tableInfo); + return new TypedAppendWriterImpl<>(createWriter(), pojoClass, tableInfo); } } diff --git a/fluss-client/src/main/java/org/apache/fluss/client/table/writer/TableUpsert.java b/fluss-client/src/main/java/org/apache/fluss/client/table/writer/TableUpsert.java index fea17cdf27..fe865eac43 100644 --- a/fluss-client/src/main/java/org/apache/fluss/client/table/writer/TableUpsert.java +++ b/fluss-client/src/main/java/org/apache/fluss/client/table/writer/TableUpsert.java @@ -98,8 +98,6 @@ public UpsertWriter createWriter() { @Override public TypedUpsertWriter createTypedWriter(Class pojoClass) { - UpsertWriterImpl delegate = - new UpsertWriterImpl(tablePath, tableInfo, targetColumns, writerClient); - return new TypedUpsertWriterImpl<>(delegate, pojoClass, tableInfo, targetColumns); + return new TypedUpsertWriterImpl<>(createWriter(), pojoClass, tableInfo, targetColumns); } } diff --git a/fluss-client/src/main/java/org/apache/fluss/client/table/writer/TableWriter.java b/fluss-client/src/main/java/org/apache/fluss/client/table/writer/TableWriter.java index 94cac25668..3c4f134803 100644 --- a/fluss-client/src/main/java/org/apache/fluss/client/table/writer/TableWriter.java +++ b/fluss-client/src/main/java/org/apache/fluss/client/table/writer/TableWriter.java @@ -28,7 +28,7 @@ * @since 0.1 */ @PublicEvolving -public interface TableWriter extends AutoCloseable { +public interface TableWriter { /** * Flush data written that have not yet been sent to the server, forcing the client to send the @@ -38,9 +38,4 @@ public interface TableWriter extends AutoCloseable { * results in an error. */ void flush(); - - @Override - default void close() throws Exception { - // by default do nothing - } } diff --git a/fluss-client/src/main/java/org/apache/fluss/client/table/writer/TypedAppendWriterImpl.java b/fluss-client/src/main/java/org/apache/fluss/client/table/writer/TypedAppendWriterImpl.java index c591eea5ea..3b964594bb 100644 --- a/fluss-client/src/main/java/org/apache/fluss/client/table/writer/TypedAppendWriterImpl.java +++ b/fluss-client/src/main/java/org/apache/fluss/client/table/writer/TypedAppendWriterImpl.java @@ -30,14 +30,12 @@ */ class TypedAppendWriterImpl implements TypedAppendWriter { - private final AppendWriterImpl delegate; - private final Class pojoClass; + private final AppendWriter delegate; private final RowType tableSchema; private final PojoToRowConverter pojoToRowConverter; - TypedAppendWriterImpl(AppendWriterImpl delegate, Class pojoClass, TableInfo tableInfo) { + TypedAppendWriterImpl(AppendWriter delegate, Class pojoClass, TableInfo tableInfo) { this.delegate = delegate; - this.pojoClass = pojoClass; this.tableSchema = tableInfo.getRowType(); this.pojoToRowConverter = PojoToRowConverter.of(pojoClass, tableSchema, tableSchema); } @@ -47,11 +45,6 @@ public void flush() { delegate.flush(); } - @Override - public void close() throws Exception { - delegate.close(); - } - @Override public CompletableFuture append(T record) { if (record instanceof InternalRow) { diff --git a/fluss-client/src/main/java/org/apache/fluss/client/table/writer/TypedUpsertWriterImpl.java b/fluss-client/src/main/java/org/apache/fluss/client/table/writer/TypedUpsertWriterImpl.java index 1d679e57e7..45af9f0ef6 100644 --- a/fluss-client/src/main/java/org/apache/fluss/client/table/writer/TypedUpsertWriterImpl.java +++ b/fluss-client/src/main/java/org/apache/fluss/client/table/writer/TypedUpsertWriterImpl.java @@ -33,22 +33,10 @@ */ class TypedUpsertWriterImpl implements TypedUpsertWriter { - @Override - public void flush() { - delegate.flush(); - } - - private final UpsertWriterImpl delegate; - - @Override - public void close() throws Exception { - delegate.close(); - } - - private final Class pojoClass; + private final UpsertWriter delegate; private final TableInfo tableInfo; private final RowType tableSchema; - private final int[] targetColumns; // may be null + @Nullable private final int[] targetColumns; private final RowType pkProjection; @Nullable private final RowType targetProjection; @@ -58,12 +46,8 @@ public void close() throws Exception { @Nullable private final PojoToRowConverter targetConverter; TypedUpsertWriterImpl( - UpsertWriterImpl delegate, - Class pojoClass, - TableInfo tableInfo, - int[] targetColumns) { + UpsertWriter delegate, Class pojoClass, TableInfo tableInfo, int[] targetColumns) { this.delegate = delegate; - this.pojoClass = pojoClass; this.tableInfo = tableInfo; this.tableSchema = tableInfo.getRowType(); this.targetColumns = targetColumns; @@ -82,6 +66,11 @@ public void close() throws Exception { : PojoToRowConverter.of(pojoClass, tableSchema, targetProjection); } + @Override + public void flush() { + delegate.flush(); + } + @Override public CompletableFuture upsert(T record) { if (record instanceof InternalRow) { @@ -124,6 +113,9 @@ private InternalRow convertPojo(T pojo, boolean forDelete) { // set PK fields, others null for (String pk : tableInfo.getPhysicalPrimaryKeys()) { int projIndex = projection.getFieldIndex(pk); + + // TODO: this can be optimized by pre-computing + // the index mapping in the constructor? int fullIndex = tableSchema.getFieldIndex(pk); full.setField(fullIndex, projected.getField(projIndex)); } diff --git a/fluss-client/src/test/java/org/apache/fluss/client/table/FlussTypedClientITCase.java b/fluss-client/src/test/java/org/apache/fluss/client/table/FlussTypedClientITCase.java index 2d8a859671..c7724799be 100644 --- a/fluss-client/src/test/java/org/apache/fluss/client/table/FlussTypedClientITCase.java +++ b/fluss-client/src/test/java/org/apache/fluss/client/table/FlussTypedClientITCase.java @@ -49,6 +49,7 @@ import java.util.ArrayList; import java.util.Arrays; import java.util.List; +import java.util.Objects; import static org.assertj.core.api.Assertions.assertThat; @@ -115,6 +116,86 @@ public AllTypesPojo( this.tsNtz = tsNtz; this.tsLtz = tsLtz; } + + @Override + public boolean equals(Object o) { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + AllTypesPojo that = (AllTypesPojo) o; + return Objects.equals(a, that.a) + && Objects.equals(bool1, that.bool1) + && Objects.equals(tiny, that.tiny) + && Objects.equals(small, that.small) + && Objects.equals(intv, that.intv) + && Objects.equals(big, that.big) + && Objects.equals(flt, that.flt) + && Objects.equals(dbl, that.dbl) + && Objects.equals(ch, that.ch) + && Objects.equals(str, that.str) + && Arrays.equals(bin, that.bin) + && Arrays.equals(bytes, that.bytes) + && Objects.equals(dec, that.dec) + && Objects.equals(dt, that.dt) + && Objects.equals(tm, that.tm) + && Objects.equals(tsNtz, that.tsNtz) + && Objects.equals(tsLtz, that.tsLtz); + } + + @Override + public int hashCode() { + int result = + Objects.hash( + a, bool1, tiny, small, intv, big, flt, dbl, ch, str, dec, dt, tm, tsNtz, + tsLtz); + result = 31 * result + Arrays.hashCode(bin); + result = 31 * result + Arrays.hashCode(bytes); + return result; + } + + @Override + public String toString() { + return "AllTypesPojo{" + + "a=" + + a + + ", bool1=" + + bool1 + + ", tiny=" + + tiny + + ", small=" + + small + + ", intv=" + + intv + + ", big=" + + big + + ", flt=" + + flt + + ", dbl=" + + dbl + + ", ch=" + + ch + + ", str='" + + str + + '\'' + + ", bin=" + + Arrays.toString(bin) + + ", bytes=" + + Arrays.toString(bytes) + + ", dec=" + + dec + + ", dt=" + + dt + + ", tm=" + + tm + + ", tsNtz=" + + tsNtz + + ", tsLtz=" + + tsLtz + + '}'; + } } /** Minimal POJO representing the primary key for {@link AllTypesPojo}. */ @@ -126,6 +207,28 @@ public PLookupKey() {} public PLookupKey(Integer a) { this.a = a; } + + @Override + public boolean equals(Object o) { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + PLookupKey that = (PLookupKey) o; + return Objects.equals(a, that.a); + } + + @Override + public int hashCode() { + return Objects.hash(a); + } + + @Override + public String toString() { + return "PLookupKey{" + "a=" + a + '}'; + } } private static Schema allTypesLogSchema() { @@ -232,7 +335,7 @@ void testTypedAppendWriteAndScan() throws Exception { for (int i = 0; i < 5; i++) { AllTypesPojo u = newAllTypesPojo(i); expected.add(u); - writer.append(u).get(); + writer.append(u); } writer.flush(); @@ -293,11 +396,16 @@ void testTypedUpsertWriteAndScan() throws Exception { values.add(r.getValue()); } } + assertThat(changes) - .contains(ChangeType.INSERT, ChangeType.UPDATE_BEFORE, ChangeType.UPDATE_AFTER); + .containsExactlyInAnyOrder( + ChangeType.INSERT, + ChangeType.INSERT, + ChangeType.UPDATE_BEFORE, + ChangeType.UPDATE_AFTER); // ensure the last update_after reflects new value int lastIdx = changes.lastIndexOf(ChangeType.UPDATE_AFTER); - assertThat(values.get(lastIdx).str).isEqualTo("a1"); + assertThat(values.get(lastIdx)).isEqualTo(p1Updated); } } @@ -313,7 +421,7 @@ void testTypedLookups() throws Exception { table.newUpsert().createTypedWriter(AllTypesPojo.class); writer.upsert(newAllTypesPojo(1)).get(); writer.upsert(newAllTypesPojo(2)).get(); - writer.close(); + writer.flush(); // primary key lookup using Lookuper API with POJO key TypedLookuper lookuper = @@ -324,7 +432,7 @@ void testTypedLookups() throws Exception { LookupResult lr = lookuper.lookup(new PLookupKey(1)).get(); AllTypesPojo one = rowConv.fromRow(lr.getSingletonRow()); - assertThat(one.str).isEqualTo("s1"); + assertThat(one).isEqualTo(newAllTypesPojo(1)); } } @@ -341,7 +449,7 @@ void testInternalRowLookup() throws Exception { table.newUpsert().createTypedWriter(AllTypesPojo.class); writer.upsert(newAllTypesPojo(101)).get(); writer.upsert(newAllTypesPojo(202)).get(); - writer.close(); + writer.flush(); // now perform lookup using the raw InternalRow path to ensure it's still supported Lookuper lookuper = table.newLookup().createLookuper(); @@ -356,9 +464,7 @@ void testInternalRowLookup() throws Exception { RowToPojoConverter rowConv = RowToPojoConverter.of(AllTypesPojo.class, tableSchema, tableSchema); AllTypesPojo pojo = rowConv.fromRow(lr.getSingletonRow()); - assertThat(pojo).isNotNull(); - assertThat(pojo.a).isEqualTo(101); - assertThat(pojo.str).isEqualTo("s101"); + assertThat(pojo).isEqualTo(newAllTypesPojo(101)); } } @@ -383,19 +489,14 @@ void testTypedProjections() throws Exception { .createTypedLogScanner(AllTypesPojo.class); subscribeFromBeginning(scanner, table); TypedScanRecords recs = scanner.poll(Duration.ofSeconds(2)); + int i = 10; for (TypedScanRecord r : recs) { AllTypesPojo u = r.getValue(); - assertThat(u.a).isNotNull(); - assertThat(u.str).isNotNull(); - // non-projected fields should be null - assertThat(u.bool1).isNull(); - assertThat(u.bin).isNull(); - assertThat(u.bytes).isNull(); - assertThat(u.dec).isNull(); - assertThat(u.dt).isNull(); - assertThat(u.tm).isNull(); - assertThat(u.tsNtz).isNull(); - assertThat(u.tsLtz).isNull(); + AllTypesPojo expectedPojo = new AllTypesPojo(); + expectedPojo.a = i; + expectedPojo.str = "s" + i; + assertThat(u).isEqualTo(expectedPojo); + i++; } } } @@ -409,19 +510,22 @@ void testTypedPartialUpdates() throws Exception { createTable(path, td, true); try (Table table = conn.getTable(path)) { + // 1. initial full row + TypedUpsertWriter fullWriter = + table.newUpsert().createTypedWriter(AllTypesPojo.class); + fullWriter.upsert(newAllTypesPojo(1)).get(); + fullWriter.flush(); + + // 2. partial update: only PK + subset fields Upsert upsert = table.newUpsert().partialUpdate("a", "str", "dec"); TypedUpsertWriter writer = upsert.createTypedWriter(AllTypesPojo.class); - // initial full row - writer.upsert(newAllTypesPojo(1)).get(); - - // partial update: only PK + subset fields AllTypesPojo patch = new AllTypesPojo(); patch.a = 1; patch.str = "second"; patch.dec = new BigDecimal("99.99"); writer.upsert(patch).get(); - writer.close(); + writer.flush(); // verify via lookup and scan using Lookuper + POJO key TypedLookuper lookuper = @@ -431,8 +535,10 @@ void testTypedPartialUpdates() throws Exception { RowToPojoConverter.of(AllTypesPojo.class, tableSchema, tableSchema); AllTypesPojo lookedUp = rowConv.fromRow(lookuper.lookup(new PLookupKey(1)).get().getSingletonRow()); - assertThat(lookedUp.str).isEqualTo("second"); - assertThat(lookedUp.dec).isEqualByComparingTo("99.99"); + AllTypesPojo expected = newAllTypesPojo(1); + expected.str = "second"; + expected.dec = new BigDecimal("99.99"); + assertThat(lookedUp).isEqualTo(expected); TypedLogScanner scanner = table.newScan().createTypedLogScanner(AllTypesPojo.class); @@ -442,7 +548,7 @@ void testTypedPartialUpdates() throws Exception { TypedScanRecords recs = scanner.poll(Duration.ofSeconds(2)); for (TypedScanRecord r : recs) { if (r.getChangeType() == ChangeType.UPDATE_AFTER) { - assertThat(r.getValue().str).isEqualTo("second"); + assertThat(r.getValue()).isEqualTo(expected); sawUpdateAfter = true; } }