github-actions[bot] commented on code in PR #30746:
URL: https://github.com/apache/doris/pull/30746#discussion_r1480819025


##########
be/src/vec/exec/scan/scanner_scheduler.cpp:
##########
@@ -139,155 +117,97 @@ Status ScannerScheduler::init(ExecEnv* env) {
     return Status::OK();
 }
 
-Status ScannerScheduler::submit(std::shared_ptr<ScannerContext> ctx) {
+void ScannerScheduler::submit(std::shared_ptr<ScannerContext> ctx,
+                              std::shared_ptr<ScanTask> scan_task) {
+    scan_task->last_submit_time = GetCurrentTimeNanos();
     if (ctx->done()) {
-        return Status::EndOfFile("ScannerContext is done");
-    }
-    ctx->queue_idx = (_queue_idx++ % QUEUE_NUM);
-    if (!_pending_queues[ctx->queue_idx]->blocking_put(ctx)) {
-        return Status::InternalError("failed to submit scanner context to 
scheduler");
-    }
-    return Status::OK();
-}
-
-std::unique_ptr<ThreadPoolToken> ScannerScheduler::new_limited_scan_pool_token(
-        ThreadPool::ExecutionMode mode, int max_concurrency) {
-    return _limited_scan_thread_pool->new_token(mode, max_concurrency);
-}
-
-void ScannerScheduler::_schedule_thread(int queue_id) {
-    BlockingQueue<std::shared_ptr<ScannerContext>>* queue = 
_pending_queues[queue_id];
-    while (!_is_closed) {
-        std::shared_ptr<ScannerContext> ctx;
-        bool ok = queue->blocking_get(&ctx);
-        if (!ok) {
-            // maybe closed
-            continue;
-        }
-
-        _schedule_scanners(ctx);
-        // If ctx is done, no need to schedule it again.
-        // But should notice that there may still scanners running in scanner 
pool.
+        return;
     }
-}
-
-void ScannerScheduler::_schedule_scanners(std::shared_ptr<ScannerContext> ctx) 
{
     auto task_lock = ctx->task_exec_ctx();
     if (task_lock == nullptr) {
         LOG(INFO) << "could not lock task execution context, query " << 
ctx->debug_string()
                   << " maybe finished";
         return;
     }
-    MonotonicStopWatch watch;
-    watch.reset();
-    watch.start();
-    ctx->incr_num_ctx_scheduling(1);
-
-    if (ctx->done()) {
-        return;
-    }
-
-    std::list<std::weak_ptr<ScannerDelegate>> this_run;
-    ctx->get_next_batch_of_scanners(&this_run);
-    if (this_run.empty()) {
-        // There will be 2 cases when this_run is empty:
-        // 1. The blocks queue reaches limit.
-        //      The consumer will continue scheduling the ctx.
-        // 2. All scanners are running.
-        //      There running scanner will schedule the ctx after they are 
finished.
-        // So here we just return to stop scheduling ctx.
-        return;
-    }
-
-    ctx->inc_num_running_scanners(this_run.size());
 
     // Submit scanners to thread pool
     // TODO(cmy): How to handle this "nice"?
     int nice = 1;
-    auto iter = this_run.begin();
     if (ctx->thread_token != nullptr) {
-        // TODO llj tg how to treat this?
-        while (iter != this_run.end()) {
-            std::shared_ptr<ScannerDelegate> scanner_delegate = (*iter).lock();
-            if (scanner_delegate == nullptr) {
-                // Has to ++, or there is a dead loop
-                iter++;
-                continue;
-            }
-            scanner_delegate->_scanner->start_wait_worker_timer();
-            auto s = ctx->thread_token->submit_func([this, scanner_ref = 
*iter, ctx]() {
-                this->_scanner_scan(this, ctx, scanner_ref);
-            });
-            if (s.ok()) {
-                iter++;
-            } else {
-                ctx->set_status_on_error(s);
-                break;
-            }
+        std::shared_ptr<ScannerDelegate> scanner_delegate = 
scan_task->scanner.lock();
+        if (scanner_delegate == nullptr) {
+            return;
+        }
+
+        scanner_delegate->_scanner->start_wait_worker_timer();
+        auto s = ctx->thread_token->submit_func(
+                [this, scanner_ref = scan_task, ctx]() { 
this->_scanner_scan(ctx, scanner_ref); });
+        if (!s.ok()) {
+            scan_task->set_status(s);
+            ctx->append_block_to_queue(scan_task);
+            return;
         }
     } else {
-        while (iter != this_run.end()) {
-            std::shared_ptr<ScannerDelegate> scanner_delegate = (*iter).lock();
-            if (scanner_delegate == nullptr) {
-                // Has to ++, or there is a dead loop
-                iter++;
-                continue;
-            }
-            scanner_delegate->_scanner->start_wait_worker_timer();
-            TabletStorageType type = 
scanner_delegate->_scanner->get_storage_type();
-            bool ret = false;
-            if (type == TabletStorageType::STORAGE_TYPE_LOCAL) {
-                if (auto* scan_sche = ctx->get_simple_scan_scheduler()) {
-                    auto work_func = [this, scanner_ref = *iter, ctx]() {
-                        this->_scanner_scan(this, ctx, scanner_ref);
-                    };
-                    SimplifiedScanTask simple_scan_task = {work_func, ctx};
-                    ret = 
scan_sche->get_scan_queue()->try_put(simple_scan_task);
-                } else {
-                    PriorityThreadPool::Task task;
-                    task.work_function = [this, scanner_ref = *iter, ctx]() {
-                        this->_scanner_scan(this, ctx, scanner_ref);
-                    };
-                    task.priority = nice;
-                    ret = _local_scan_thread_pool->offer(task);
-                }
+        std::shared_ptr<ScannerDelegate> scanner_delegate = 
scan_task->scanner.lock();
+        if (scanner_delegate == nullptr) {
+            return;
+        }
+
+        scanner_delegate->_scanner->start_wait_worker_timer();
+        TabletStorageType type = 
scanner_delegate->_scanner->get_storage_type();
+        bool ret = false;
+        if (type == TabletStorageType::STORAGE_TYPE_LOCAL) {
+            if (auto* scan_sche = ctx->get_simple_scan_scheduler()) {
+                auto work_func = [this, scanner_ref = scan_task, ctx]() {
+                    this->_scanner_scan(ctx, scanner_ref);
+                };
+                SimplifiedScanTask simple_scan_task = {work_func, ctx};
+                ret = scan_sche->get_scan_queue()->try_put(simple_scan_task);
             } else {
                 PriorityThreadPool::Task task;
-                task.work_function = [this, scanner_ref = *iter, ctx]() {
-                    this->_scanner_scan(this, ctx, scanner_ref);
+                task.work_function = [this, scanner_ref = scan_task, ctx]() {
+                    this->_scanner_scan(ctx, scanner_ref);
                 };
                 task.priority = nice;
-                ret = _remote_scan_thread_pool->offer(task);
-            }
-            if (ret) {
-                iter++;
-            } else {
-                ctx->set_status_on_error(
-                        Status::InternalError("failed to submit scanner to 
scanner pool"));
-                break;
+                ret = _local_scan_thread_pool->offer(task);
             }
+        } else {
+            PriorityThreadPool::Task task;
+            task.work_function = [this, scanner_ref = scan_task, ctx]() {
+                this->_scanner_scan(ctx, scanner_ref);
+            };
+            task.priority = nice;
+            ret = _remote_scan_thread_pool->offer(task);
+        }
+        if (!ret) {
+            scan_task->set_status(
+                    Status::InternalError("Failed to submit scanner to scanner 
pool"));
+            ctx->append_block_to_queue(scan_task);
+            return;
         }
     }
-    ctx->incr_ctx_scheduling_time(watch.elapsed_time());
 }
 
-void ScannerScheduler::_scanner_scan(ScannerScheduler* scheduler,
-                                     std::shared_ptr<ScannerContext> ctx,
-                                     std::weak_ptr<ScannerDelegate> 
scanner_ref) {
+std::unique_ptr<ThreadPoolToken> ScannerScheduler::new_limited_scan_pool_token(
+        ThreadPool::ExecutionMode mode, int max_concurrency) {
+    return _limited_scan_thread_pool->new_token(mode, max_concurrency);
+}
+
+void ScannerScheduler::_scanner_scan(std::shared_ptr<ScannerContext> ctx,

Review Comment:
   warning: function '_scanner_scan' exceeds recommended size/complexity 
thresholds [readability-function-size]
   ```cpp
   void ScannerScheduler::_scanner_scan(std::shared_ptr<ScannerContext> ctx,
                          ^
   ```
   <details>
   <summary>Additional context</summary>
   
   **be/src/vec/exec/scan/scanner_scheduler.cpp:195:** 104 lines including 
whitespace and comments (threshold 80)
   ```cpp
   void ScannerScheduler::_scanner_scan(std::shared_ptr<ScannerContext> ctx,
                          ^
   ```
   
   </details>
   



##########
be/src/vec/exec/scan/scanner_context.cpp:
##########
@@ -220,138 +162,221 @@ std::string ScannerContext::parent_name() {
 vectorized::BlockUPtr ScannerContext::get_free_block() {
     vectorized::BlockUPtr block;
     if (_free_blocks.try_dequeue(block)) {
+        std::lock_guard<std::mutex> fl(_free_blocks_lock);
         DCHECK(block->mem_reuse());
-        _free_blocks_memory_usage->add(-block->allocated_bytes());
-        _serving_blocks_num++;
+        _free_blocks_memory_usage -= block->allocated_bytes();
+        _free_blocks_memory_usage_mark->set(_free_blocks_memory_usage);
         return block;
     }
 
-    block = vectorized::Block::create_unique(_output_tuple_desc->slots(), 
_batch_size,
-                                             true /*ignore invalid slots*/);
-
-    COUNTER_UPDATE(_newly_create_free_blocks_num, 1);
-
-    _serving_blocks_num++;
-    return block;
+    _newly_create_free_blocks_num->update(1);
+    return vectorized::Block::create_unique(_output_tuple_desc->slots(), 
_batch_size,
+                                            true /*ignore invalid slots*/);
 }
 
-void ScannerContext::return_free_block(std::unique_ptr<vectorized::Block> 
block) {
-    _serving_blocks_num--;
-    if (block->mem_reuse()) {
-        // Only put blocks with schema to free blocks, because colocate blocks
-        // need schema.
-        _estimated_block_bytes = std::max(block->allocated_bytes(), 
(size_t)16);
+void ScannerContext::return_free_block(vectorized::BlockUPtr block) {
+    std::lock_guard<std::mutex> fl(_free_blocks_lock);
+    if (block->mem_reuse() && _free_blocks_memory_usage < _max_bytes_in_queue) 
{
         block->clear_column_data();
-        _free_blocks_memory_usage->add(block->allocated_bytes());
+        _free_blocks_memory_usage += block->allocated_bytes();
+        _free_blocks_memory_usage_mark->set(_free_blocks_memory_usage);
         _free_blocks.enqueue(std::move(block));
     }
 }
 
-void 
ScannerContext::append_blocks_to_queue(std::vector<vectorized::BlockUPtr>& 
blocks) {
-    std::lock_guard l(_transfer_lock);
-    auto old_bytes_in_queue = _cur_bytes_in_queue;
-    for (auto& b : blocks) {
-        auto st = validate_block_schema(b.get());
+bool ScannerContext::empty_in_queue(int id) {
+    std::lock_guard<std::mutex> l(_transfer_lock);
+    return _blocks_queue.empty();
+}
+
+void ScannerContext::submit_scan_task(std::shared_ptr<ScanTask> scan_task) {
+    _scanner_sched_counter->update(1);
+    _num_scheduled_scanners++;
+    _scanner_scheduler->submit(shared_from_this(), scan_task);
+}
+
+void ScannerContext::append_block_to_queue(std::shared_ptr<ScanTask> 
scan_task) {
+    if (scan_task->status_ok() && scan_task->current_block->rows() > 0) {
+        Status st = validate_block_schema(scan_task->current_block.get());
         if (!st.ok()) {
-            set_status_on_error(st, false);
+            scan_task->set_status(st);
         }
-        _cur_bytes_in_queue += b->allocated_bytes();
-        _blocks_queue.push_back(std::move(b));
     }
-    blocks.clear();
-    if (_dependency) {
-        _dependency->set_ready();
+    std::lock_guard<std::mutex> l(_transfer_lock);
+    if (!scan_task->status_ok()) {
+        _process_status = scan_task->get_status();
+    }
+    if (_last_scale_up_time == 0) {
+        _last_scale_up_time = UnixMillis();
+    }
+    if (_blocks_queue.empty() && _last_fetch_time != 0) {
+        // there's no block in queue before current block, so the consumer is 
waiting
+        _total_wait_block_time += UnixMillis() - _last_fetch_time;
     }
+    _num_scheduled_scanners--;
+    _blocks_queue.emplace_back(scan_task);
     _blocks_queue_added_cv.notify_one();
-    _queued_blocks_memory_usage->add(_cur_bytes_in_queue - old_bytes_in_queue);
-    g_bytes_in_scanner_queue.set_value(_cur_bytes_in_queue);
 }
 
-bool ScannerContext::empty_in_queue(int id) {
+Status ScannerContext::get_block_from_queue(RuntimeState* state, 
vectorized::Block* block,

Review Comment:
   warning: function 'get_block_from_queue' exceeds recommended size/complexity 
thresholds [readability-function-size]
   ```cpp
   Status ScannerContext::get_block_from_queue(RuntimeState* state, 
vectorized::Block* block,
                          ^
   ```
   <details>
   <summary>Additional context</summary>
   
   **be/src/vec/exec/scan/scanner_context.cpp:220:** 87 lines including 
whitespace and comments (threshold 80)
   ```cpp
   Status ScannerContext::get_block_from_queue(RuntimeState* state, 
vectorized::Block* block,
                          ^
   ```
   
   </details>
   



-- 
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: commits-unsubscr...@doris.apache.org

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscr...@doris.apache.org
For additional commands, e-mail: commits-h...@doris.apache.org

Reply via email to