wuchong commented on code in PR #2485:
URL: https://github.com/apache/fluss/pull/2485#discussion_r2751064982
##########
fluss-server/src/main/java/org/apache/fluss/server/replica/ReplicaManager.java:
##########
@@ -633,8 +693,84 @@ public void lookups(
tb, new LookupResultForBucket(tb,
ApiError.fromThrowable(e)));
}
}
+
+ if (insertIfNotExists) {
+ checkArgument(
+ timeoutMs != null && requiredAcks != null,
+ "timeoutMs and requiredAcks must be set");
+ Map<TableBucket, MissingKeysContext> entriesPerBucketToInsert =
new HashMap<>();
+ Map<TableBucket, KvRecordBatch> produceEntryData = new HashMap<>();
+ collectMissingKeysForInsert(
+ entriesPerBucket,
+ lookupResultForBucketMap,
+ entriesPerBucketToInsert,
+ produceEntryData);
+ if (!produceEntryData.isEmpty()) {
+ // TODO: Performance optimization: during
lookup-with-insert-if-not-exists flow,
+ // the original key bytes are wrapped in KeyRecordBatch, then
during putRecordsToKv
+ // they are decoded to rows and immediately re-encoded back to
key bytes, causing
+ // redundant encode/decode overhead.
+ putRecordsToKv(
+ timeoutMs,
+ requiredAcks,
+ produceEntryData,
+ null,
Review Comment:
This is incorrect. We should use **partial updates** to modify only the
primary key fields. Otherwise, non-primary-key columns—including auto-increment
fields—will be overwritten with `null`. This issue manifests when multiple
threads concurrently call `lookupAndInsert` on the same keys.
To reproduce this, I’ve added a test:
`testConcurrentLookupWithInsertIfNotExistsAutoIncrement`.
Additionally, we should enforce a validation in `PutRequest`: when
performing a put operation, **auto-increment fields must be excluded from the
target columns**. This ensures that auto-incremented values are never
accidentally overwritten during updates. That means `INSERT INTO t (id,
auto_inc) VALUES ...` or `INSERT INTO t VALUES ...` should fail with explicit
error message that the auto increment fields can't be updated.
--
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
To unsubscribe, e-mail: [email protected]
For queries about this service, please contact Infrastructure at:
[email protected]