github-actions[bot] commented on code in PR #62661:
URL: https://github.com/apache/doris/pull/62661#discussion_r3373011366
##########
be/src/cloud/cloud_delta_writer.cpp:
##########
@@ -64,10 +66,18 @@ Status
CloudDeltaWriter::batch_init(std::vector<CloudDeltaWriter*> writers) {
return cloud::bthread_fork_join(tasks, 10);
}
-Status CloudDeltaWriter::write(const Block* block, const
DorisVector<uint32_t>& row_idxs) {
+Status CloudDeltaWriter::write(const Block* block, const
DorisVector<uint32_t>& row_idxs,
+ bool* memtable_flushed) {
+ if (memtable_flushed != nullptr) {
+ *memtable_flushed = false;
+ }
if (row_idxs.empty()) [[unlikely]] {
return Status::OK();
}
+ if (_req.enable_table_memtable_backpressure) {
Review Comment:
This new table-level backpressure wait cannot observe load cancellation
because it is called with a null `cancel_check`. In the adaptive cloud path
this runs before taking the writer mutex, so if the table already has
`table_memtable_flush_pending_count_limit` pending memtables, a cancelled load
can remain stuck in `handle_table_memtable_backpressure()` until enough
unrelated flushes complete; if flush is stalled, the add-batch RPC never
returns and close/cancel cannot make progress. Please pass a cancellation
predicate wired to the writer/channel state, or otherwise make this wait
interruptible, matching the existing cancellation-aware memory-limit waits.
--
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]
---------------------------------------------------------------------
To unsubscribe, e-mail: [email protected]
For additional commands, e-mail: [email protected]