Files
doris/be/src/runtime/query_fragments_ctx.h
Mingyu Chen c996334ad1 [improvement] Optimize send fragment logic to reduce send fragment timeout error (#9720)
This CL mainly changes:
1. Reducing the rpc timeout problem caused by rpc waiting for the worker thread of brpc.
    1. Merge multiple fragment instances on the same BE to send requests to reduce the number of send fragment rpcs
    2. If fragments size >= 3, use 2 phase RPC: one is to send all fragments, two is to start these fragments. So that there
         will be at most 2 RPC for each query on one BE.

3. Set the timeout of send fragment rpc to the query timeout to ensure the consistency of users' expectation of query timeout period.

4. Do not close the connection anymore when rpc timeout occurs.
5. Change some log level from info to debug to simplify the fe.log content.

NOTICE:
1. Change the definition of execPlanFragment rpc, must first upgrade BE.
3. Remove FE config `remote_fragment_exec_timeout_ms`
2022-06-03 15:47:40 +08:00

119 lines
4.3 KiB
C++

// 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.
#pragma once
#include <atomic>
#include <string>
#include "common/object_pool.h"
#include "gen_cpp/PaloInternalService_types.h" // for TQueryOptions
#include "gen_cpp/Types_types.h" // for TUniqueId
#include "runtime/datetime_value.h"
#include "runtime/exec_env.h"
#include "util/threadpool.h"
namespace doris {
// Save the common components of fragments in a query.
// Some components like DescriptorTbl may be very large
// that will slow down each execution of fragments when DeSer them every time.
class DescriptorTbl;
class QueryFragmentsCtx {
public:
QueryFragmentsCtx(int total_fragment_num, ExecEnv* exec_env)
: fragment_num(total_fragment_num), timeout_second(-1), _exec_env(exec_env) {
_start_time = DateTimeValue::local_time();
}
bool countdown() { return fragment_num.fetch_sub(1) == 1; }
bool is_timeout(const DateTimeValue& now) const {
if (timeout_second <= 0) {
return false;
}
if (now.second_diff(_start_time) > timeout_second) {
return true;
}
return false;
}
void set_thread_token(int cpu_limit) {
if (cpu_limit > 0) {
// For now, cpu_limit will be the max concurrency of the scan thread pool token.
_thread_token = _exec_env->limited_scan_thread_pool()->new_token(
ThreadPool::ExecutionMode::CONCURRENT, cpu_limit);
}
}
ThreadPoolToken* get_token() { return _thread_token.get(); }
void set_ready_to_execute() {
{
std::lock_guard<std::mutex> l(_start_lock);
_ready_to_execute = true;
}
_start_cond.notify_all();
}
void wait_for_start() {
std::unique_lock<std::mutex> l(_start_lock);
while (!_ready_to_execute.load()) {
_start_cond.wait(l);
}
}
public:
TUniqueId query_id;
DescriptorTbl* desc_tbl;
bool set_rsc_info = false;
std::string user;
std::string group;
TNetworkAddress coord_addr;
TQueryGlobals query_globals;
/// In the current implementation, for multiple fragments executed by a query on the same BE node,
/// we store some common components in QueryFragmentsCtx, and save QueryFragmentsCtx in FragmentMgr.
/// When all Fragments are executed, QueryFragmentsCtx needs to be deleted from FragmentMgr.
/// Here we use a counter to store the number of Fragments that have not yet been completed,
/// and after each Fragment is completed, this value will be reduced by one.
/// When the last Fragment is completed, the counter is cleared, and the worker thread of the last Fragment
/// will clean up QueryFragmentsCtx.
std::atomic<int> fragment_num;
int timeout_second;
ObjectPool obj_pool;
private:
ExecEnv* _exec_env;
DateTimeValue _start_time;
// A token used to submit olap scanner to the "_limited_scan_thread_pool",
// This thread pool token is created from "_limited_scan_thread_pool" from exec env.
// And will be shared by all instances of this query.
// So that we can control the max thread that a query can be used to execute.
// If this token is not set, the scanner will be executed in "_scan_thread_pool" in exec env.
std::unique_ptr<ThreadPoolToken> _thread_token;
std::mutex _start_lock;
std::condition_variable _start_cond;
// Only valid when _need_wait_execution_trigger is set to true in FragmentExecState.
// And all fragments of this query will start execution when this is set to true.
std::atomic<bool> _ready_to_execute {false};
};
} // namespace doris