github-actions[bot] commented on code in PR #62661:
URL: https://github.com/apache/doris/pull/62661#discussion_r3378000569
##########
be/src/load/delta_writer/delta_writer.cpp:
##########
@@ -155,10 +161,18 @@ Status BaseDeltaWriter::init() {
return Status::OK();
}
-Status DeltaWriter::write(const Block* block, const DorisVector<uint32_t>&
row_idxs) {
+Status DeltaWriter::write(const Block* block, const DorisVector<uint32_t>&
row_idxs,
+ bool* memtable_flushed) {
+ if (memtable_flushed != nullptr) {
+ *memtable_flushed = false;
+ }
if (UNLIKELY(row_idxs.empty())) {
return Status::OK();
}
+ if (_req.enable_table_memtable_backpressure) {
+
ExecEnv::GetInstance()->memtable_memory_limiter()->handle_table_memtable_backpressure(
+ nullptr, table_id());
+ }
Review Comment:
This new table-level backpressure wait is still uninterruptible on the
non-cloud `DeltaWriter` path. `enable_table_memtable_backpressure` is set for
receiver-side random-bucket opens in
`TabletsChannel::_open_all_writers()`/`incremental_open()`, so a shared-nothing
load can enter this branch before taking `_lock`; if the load is cancelled
while the table already has `table_memtable_flush_pending_count_limit` pending
memtables, `cancel_with_status()` cannot make this wait return because the
predicate is `nullptr`. If flushes are stalled, the add-batch RPC remains stuck
here and close/cancel cannot make progress. Please pass a cancellation
predicate that observes `_is_cancelled` (matching the fixed `CloudDeltaWriter`
path) or otherwise make this wait interruptible.
--
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]