[feature](insert) Support wal for group commit insert (#23053)
This commit is contained in:
@ -1088,6 +1088,12 @@ DEFINE_Int32(grace_shutdown_wait_seconds, "120");
|
||||
|
||||
DEFINE_Int16(bitmap_serialize_version, "1");
|
||||
|
||||
// group commit insert config
|
||||
DEFINE_String(group_commit_replay_wal_dir, "./wal");
|
||||
DEFINE_Int32(group_commit_replay_wal_retry_num, "10");
|
||||
DEFINE_Int32(group_commit_replay_wal_retry_interval_seconds, "5");
|
||||
DEFINE_Int32(group_commit_sync_wal_batch, "10");
|
||||
|
||||
// the count of thread to group commit insert
|
||||
DEFINE_Int32(group_commit_insert_threads, "10");
|
||||
|
||||
|
||||
@ -1159,6 +1159,12 @@ DECLARE_Int32(grace_shutdown_wait_seconds);
|
||||
// BitmapValue serialize version.
|
||||
DECLARE_Int16(bitmap_serialize_version);
|
||||
|
||||
// group commit insert config
|
||||
DECLARE_String(group_commit_replay_wal_dir);
|
||||
DECLARE_Int32(group_commit_replay_wal_retry_num);
|
||||
DECLARE_Int32(group_commit_replay_wal_retry_interval_seconds);
|
||||
DECLARE_Int32(group_commit_sync_wal_batch);
|
||||
|
||||
// This config can be set to limit thread number in group commit insert thread pool.
|
||||
DECLARE_mInt32(group_commit_insert_threads);
|
||||
|
||||
|
||||
@ -223,12 +223,13 @@ void HttpStreamAction::on_chunk_data(HttpRequest* req) {
|
||||
if (ctx == nullptr || !ctx->status.ok()) {
|
||||
return;
|
||||
}
|
||||
|
||||
if (!req->header(HTTP_WAL_ID_KY).empty()) {
|
||||
ctx->wal_id = std::stoll(req->header(HTTP_WAL_ID_KY));
|
||||
}
|
||||
struct evhttp_request* ev_req = req->get_evhttp_request();
|
||||
auto evbuf = evhttp_request_get_input_buffer(ev_req);
|
||||
|
||||
int64_t start_read_data_time = MonotonicNanos();
|
||||
|
||||
while (evbuffer_get_length(evbuf) > 0) {
|
||||
auto bb = ByteBuffer::allocate(128 * 1024);
|
||||
auto remove_bytes = evbuffer_remove(evbuf, bb->ptr, bb->capacity);
|
||||
@ -306,6 +307,7 @@ Status HttpStreamAction::_process_put(HttpRequest* http_req,
|
||||
ctx->db = ctx->put_result.params.db_name;
|
||||
ctx->table = ctx->put_result.params.table_name;
|
||||
ctx->txn_id = ctx->put_result.params.txn_conf.txn_id;
|
||||
ctx->put_result.params.__set_wal_id(ctx->wal_id);
|
||||
return _exec_env->stream_load_executor()->execute_plan_fragment(ctx);
|
||||
}
|
||||
|
||||
|
||||
@ -63,5 +63,7 @@ static const std::string HTTP_TWO_PHASE_COMMIT = "two_phase_commit";
|
||||
static const std::string HTTP_TXN_ID_KEY = "txn_id";
|
||||
static const std::string HTTP_TXN_OPERATION_KEY = "txn_operation";
|
||||
static const std::string HTTP_MEMTABLE_ON_SINKNODE = "memtable_on_sink_node";
|
||||
static const std::string HTTP_WAL_ID_KY = "wal_id";
|
||||
static const std::string HTTP_AUTH_CODE = "auth_code";
|
||||
|
||||
} // namespace doris
|
||||
|
||||
@ -30,6 +30,7 @@
|
||||
#include "common/status.h"
|
||||
#include "common/utils.h"
|
||||
#include "http/http_channel.h"
|
||||
#include "http/http_common.h"
|
||||
#include "http/http_headers.h"
|
||||
#include "http/http_method.h"
|
||||
#include "http/http_request.h"
|
||||
@ -72,7 +73,12 @@ bool parse_basic_auth(const HttpRequest& req, std::string* user, std::string* pa
|
||||
|
||||
bool parse_basic_auth(const HttpRequest& req, AuthInfo* auth) {
|
||||
auto& token = req.header("token");
|
||||
if (token.empty()) {
|
||||
auto& auth_code = req.header(HTTP_AUTH_CODE);
|
||||
if (!token.empty()) {
|
||||
auth->token = token;
|
||||
} else if (!auth_code.empty()) {
|
||||
auth->auth_code = std::stoll(auth_code);
|
||||
} else {
|
||||
std::string full_user;
|
||||
if (!parse_basic_auth(req, &full_user, &auth->passwd)) {
|
||||
return false;
|
||||
@ -84,8 +90,6 @@ bool parse_basic_auth(const HttpRequest& req, AuthInfo* auth) {
|
||||
} else {
|
||||
auth->user = full_user;
|
||||
}
|
||||
} else {
|
||||
auth->token = token;
|
||||
}
|
||||
|
||||
// set user ip
|
||||
|
||||
252
be/src/olap/wal_manager.cpp
Normal file
252
be/src/olap/wal_manager.cpp
Normal file
@ -0,0 +1,252 @@
|
||||
// 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.
|
||||
|
||||
#include "olap/wal_manager.h"
|
||||
|
||||
#include <thrift/protocol/TDebugProtocol.h>
|
||||
|
||||
#include <chrono>
|
||||
#include <filesystem>
|
||||
|
||||
#include "io/fs/local_file_system.h"
|
||||
#include "runtime/client_cache.h"
|
||||
#include "runtime/fragment_mgr.h"
|
||||
#include "runtime/plan_fragment_executor.h"
|
||||
#include "runtime/stream_load/stream_load_context.h"
|
||||
#include "util/path_util.h"
|
||||
#include "util/thrift_rpc_helper.h"
|
||||
#include "vec/exec/format/wal/wal_reader.h"
|
||||
|
||||
namespace doris {
|
||||
WalManager::WalManager(ExecEnv* exec_env, const std::string& wal_dir_list)
|
||||
: _exec_env(exec_env), _stop_background_threads_latch(1) {
|
||||
doris::vectorized::WalReader::string_split(wal_dir_list, ",", _wal_dirs);
|
||||
}
|
||||
|
||||
WalManager::~WalManager() {
|
||||
_stop_background_threads_latch.count_down();
|
||||
if (_replay_thread) {
|
||||
_replay_thread->join();
|
||||
}
|
||||
LOG(INFO) << "WalManager is destoried";
|
||||
}
|
||||
void WalManager::stop() {
|
||||
_stop = true;
|
||||
LOG(INFO) << "WalManager is stopped";
|
||||
}
|
||||
|
||||
Status WalManager::init() {
|
||||
bool exists = false;
|
||||
for (auto wal_dir : _wal_dirs) {
|
||||
std::string tmp_dir = wal_dir + "/tmp";
|
||||
LOG(INFO) << "wal_dir:" << wal_dir << ",tmp_dir:" << tmp_dir;
|
||||
RETURN_IF_ERROR(io::global_local_filesystem()->exists(wal_dir, &exists));
|
||||
if (!exists) {
|
||||
RETURN_IF_ERROR(io::global_local_filesystem()->create_directory(wal_dir));
|
||||
}
|
||||
RETURN_IF_ERROR(io::global_local_filesystem()->exists(tmp_dir, &exists));
|
||||
if (!exists) {
|
||||
RETURN_IF_ERROR(io::global_local_filesystem()->create_directory(tmp_dir));
|
||||
}
|
||||
RETURN_IF_ERROR(scan_wals(wal_dir));
|
||||
}
|
||||
return Thread::create(
|
||||
"WalMgr", "replay_wal", [this]() { this->replay(); }, &_replay_thread);
|
||||
}
|
||||
|
||||
Status WalManager::add_wal_path(int64_t db_id, int64_t table_id, int64_t wal_id,
|
||||
const std::string& label) {
|
||||
std::string base_path =
|
||||
_wal_dirs.size() == 1 ? _wal_dirs[0] : _wal_dirs[rand() % _wal_dirs.size()];
|
||||
std::stringstream ss;
|
||||
ss << base_path << "/" << std::to_string(db_id) << "/" << std::to_string(table_id) << "/"
|
||||
<< std::to_string(wal_id) << "_" << label;
|
||||
{
|
||||
std::lock_guard<std::shared_mutex> wrlock(_wal_lock);
|
||||
_wal_path_map.emplace(wal_id, ss.str());
|
||||
}
|
||||
return Status::OK();
|
||||
}
|
||||
|
||||
Status WalManager::get_wal_path(int64_t wal_id, std::string& wal_path) {
|
||||
std::shared_lock rdlock(_wal_lock);
|
||||
auto it = _wal_path_map.find(wal_id);
|
||||
if (it != _wal_path_map.end()) {
|
||||
wal_path = _wal_path_map[wal_id];
|
||||
} else {
|
||||
return Status::InternalError("can not find wal_id {} in wal_path_map", wal_id);
|
||||
}
|
||||
return Status::OK();
|
||||
}
|
||||
|
||||
Status WalManager::create_wal_reader(const std::string& wal_path,
|
||||
std::shared_ptr<WalReader>& wal_reader) {
|
||||
wal_reader = std::make_shared<WalReader>(wal_path);
|
||||
RETURN_IF_ERROR(wal_reader->init());
|
||||
return Status::OK();
|
||||
}
|
||||
|
||||
Status WalManager::create_wal_writer(int64_t wal_id, std::shared_ptr<WalWriter>& wal_writer) {
|
||||
std::string wal_path;
|
||||
RETURN_IF_ERROR(get_wal_path(wal_id, wal_path));
|
||||
std::vector<std::string> path_element;
|
||||
doris::vectorized::WalReader::string_split(wal_path, "/", path_element);
|
||||
std::stringstream ss;
|
||||
for (int i = 0; i < path_element.size() - 1; i++) {
|
||||
ss << path_element[i] << "/";
|
||||
}
|
||||
std::string base_path = ss.str();
|
||||
bool exists = false;
|
||||
RETURN_IF_ERROR(io::global_local_filesystem()->exists(base_path, &exists));
|
||||
if (!exists) {
|
||||
RETURN_IF_ERROR(io::global_local_filesystem()->create_directory(base_path));
|
||||
}
|
||||
LOG(INFO) << "create wal " << wal_path;
|
||||
wal_writer = std::make_shared<WalWriter>(wal_path);
|
||||
RETURN_IF_ERROR(wal_writer->init());
|
||||
return Status::OK();
|
||||
}
|
||||
|
||||
Status WalManager::scan_wals(const std::string& wal_path) {
|
||||
size_t count = 0;
|
||||
bool exists = true;
|
||||
std::vector<io::FileInfo> dbs;
|
||||
Status st = io::global_local_filesystem()->list(wal_path, false, &dbs, &exists);
|
||||
if (!st.ok()) {
|
||||
LOG(WARNING) << "Failed list files for dir=" << wal_path << ", st=" << st.to_string();
|
||||
return st;
|
||||
}
|
||||
for (const auto& db_id : dbs) {
|
||||
if (db_id.is_file) {
|
||||
continue;
|
||||
}
|
||||
std::vector<io::FileInfo> tables;
|
||||
auto db_path = wal_path + "/" + db_id.file_name;
|
||||
st = io::global_local_filesystem()->list(db_path, false, &tables, &exists);
|
||||
if (!st.ok()) {
|
||||
LOG(WARNING) << "Failed list files for dir=" << db_path << ", st=" << st.to_string();
|
||||
return st;
|
||||
}
|
||||
for (const auto& table_id : tables) {
|
||||
if (table_id.is_file) {
|
||||
continue;
|
||||
}
|
||||
std::vector<io::FileInfo> wals;
|
||||
auto table_path = db_path + "/" + table_id.file_name;
|
||||
st = io::global_local_filesystem()->list(table_path, false, &wals, &exists);
|
||||
if (!st.ok()) {
|
||||
LOG(WARNING) << "Failed list files for dir=" << table_path
|
||||
<< ", st=" << st.to_string();
|
||||
return st;
|
||||
}
|
||||
if (wals.size() == 0) {
|
||||
continue;
|
||||
}
|
||||
std::vector<std::string> res;
|
||||
for (const auto& wal : wals) {
|
||||
auto wal_file = table_path + "/" + wal.file_name;
|
||||
res.emplace_back(wal_file);
|
||||
{
|
||||
std::lock_guard<std::shared_mutex> wrlock(_wal_lock);
|
||||
int64_t wal_id = std::strtoll(wal.file_name.c_str(), NULL, 10);
|
||||
_wal_path_map.emplace(wal_id, wal_file);
|
||||
}
|
||||
}
|
||||
st = add_recover_wal(db_id.file_name, table_id.file_name, res);
|
||||
count += res.size();
|
||||
if (!st.ok()) {
|
||||
LOG(WARNING) << "Failed add replay wal, db=" << db_id.file_name
|
||||
<< ", table=" << table_id.file_name << ", st=" << st.to_string();
|
||||
return st;
|
||||
}
|
||||
}
|
||||
}
|
||||
LOG(INFO) << "Finish list all wals, size:" << count;
|
||||
return Status::OK();
|
||||
}
|
||||
|
||||
Status WalManager::replay() {
|
||||
do {
|
||||
if (_stop || _exec_env->master_info() == nullptr) {
|
||||
break;
|
||||
}
|
||||
// port == 0 means not received heartbeat yet
|
||||
while (_exec_env->master_info()->network_address.port == 0) {
|
||||
sleep(1);
|
||||
continue;
|
||||
}
|
||||
std::vector<std::string> replay_tables;
|
||||
{
|
||||
std::lock_guard<std::shared_mutex> wrlock(_lock);
|
||||
auto it = _table_map.begin();
|
||||
while (it != _table_map.end()) {
|
||||
if (it->second->size() == 0) {
|
||||
it = _table_map.erase(it);
|
||||
} else {
|
||||
replay_tables.push_back(it->first);
|
||||
it++;
|
||||
}
|
||||
}
|
||||
}
|
||||
for (const auto& table_id : replay_tables) {
|
||||
auto st = _table_map[table_id]->replay_wals();
|
||||
if (!st.ok()) {
|
||||
LOG(WARNING) << "Failed add replay wal on table " << table_id;
|
||||
}
|
||||
}
|
||||
} while (!_stop_background_threads_latch.wait_for(
|
||||
std::chrono::seconds(config::group_commit_replay_wal_retry_interval_seconds)));
|
||||
return Status::OK();
|
||||
}
|
||||
|
||||
Status WalManager::add_recover_wal(const std::string& db_id, const std::string& table_id,
|
||||
std::vector<std::string> wals) {
|
||||
std::lock_guard<std::shared_mutex> wrlock(_lock);
|
||||
std::shared_ptr<WalTable> table_ptr;
|
||||
auto it = _table_map.find(table_id);
|
||||
if (it == _table_map.end()) {
|
||||
table_ptr = std::make_shared<WalTable>(_exec_env, std::stoll(db_id), std::stoll(table_id));
|
||||
_table_map.emplace(table_id, table_ptr);
|
||||
} else {
|
||||
table_ptr = it->second;
|
||||
}
|
||||
table_ptr->add_wals(wals);
|
||||
return Status::OK();
|
||||
}
|
||||
|
||||
size_t WalManager::get_wal_table_size(const std::string& table_id) {
|
||||
std::shared_lock rdlock(_lock);
|
||||
auto it = _table_map.find(table_id);
|
||||
if (it != _table_map.end()) {
|
||||
return it->second->size();
|
||||
} else {
|
||||
return 0;
|
||||
}
|
||||
}
|
||||
|
||||
Status WalManager::delete_wal(int64_t wal_id) {
|
||||
{
|
||||
std::lock_guard<std::shared_mutex> wrlock(_wal_lock);
|
||||
std::string wal_path = _wal_path_map[wal_id];
|
||||
RETURN_IF_ERROR(io::global_local_filesystem()->delete_file(wal_path));
|
||||
LOG(INFO) << "delete file=" << wal_path;
|
||||
_wal_path_map.erase(wal_id);
|
||||
}
|
||||
return Status::OK();
|
||||
}
|
||||
|
||||
} // namespace doris
|
||||
61
be/src/olap/wal_manager.h
Normal file
61
be/src/olap/wal_manager.h
Normal file
@ -0,0 +1,61 @@
|
||||
// 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.
|
||||
|
||||
#include "common/config.h"
|
||||
#include "gen_cpp/FrontendService.h"
|
||||
#include "gen_cpp/FrontendService_types.h"
|
||||
#include "gen_cpp/HeartbeatService_types.h"
|
||||
#include "olap/wal_reader.h"
|
||||
#include "olap/wal_table.h"
|
||||
#include "olap/wal_writer.h"
|
||||
#include "runtime/exec_env.h"
|
||||
#include "runtime/stream_load/stream_load_context.h"
|
||||
#include "util/thread.h"
|
||||
|
||||
namespace doris {
|
||||
class WalManager {
|
||||
ENABLE_FACTORY_CREATOR(WalManager);
|
||||
|
||||
public:
|
||||
WalManager(ExecEnv* exec_env, const std::string& wal_dir);
|
||||
~WalManager();
|
||||
Status delete_wal(int64_t wal_id);
|
||||
Status init();
|
||||
Status scan_wals(const std::string& wal_path);
|
||||
Status replay();
|
||||
Status create_wal_reader(const std::string& wal_path, std::shared_ptr<WalReader>& wal_reader);
|
||||
Status create_wal_writer(int64_t wal_id, std::shared_ptr<WalWriter>& wal_writer);
|
||||
Status scan();
|
||||
size_t get_wal_table_size(const std::string& table_id);
|
||||
Status add_recover_wal(const std::string& db_id, const std::string& table_id,
|
||||
std::vector<std::string> wals);
|
||||
Status add_wal_path(int64_t db_id, int64_t table_id, int64_t wal_id, const std::string& label);
|
||||
Status get_wal_path(int64_t wal_id, std::string& wal_path);
|
||||
void stop();
|
||||
|
||||
private:
|
||||
ExecEnv* _exec_env;
|
||||
std::shared_mutex _lock;
|
||||
scoped_refptr<Thread> _replay_thread;
|
||||
CountDownLatch _stop_background_threads_latch;
|
||||
std::map<std::string, std::shared_ptr<WalTable>> _table_map;
|
||||
std::vector<std::string> _wal_dirs;
|
||||
std::shared_mutex _wal_lock;
|
||||
std::unordered_map<int64_t, std::string> _wal_path_map;
|
||||
bool _stop = false;
|
||||
};
|
||||
} // namespace doris
|
||||
91
be/src/olap/wal_reader.cpp
Normal file
91
be/src/olap/wal_reader.cpp
Normal file
@ -0,0 +1,91 @@
|
||||
// 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.
|
||||
|
||||
#include "olap/wal_reader.h"
|
||||
|
||||
#include "common/status.h"
|
||||
#include "io/fs/file_reader.h"
|
||||
#include "io/fs/local_file_system.h"
|
||||
#include "io/fs/path.h"
|
||||
#include "util/crc32c.h"
|
||||
#include "wal_writer.h"
|
||||
|
||||
namespace doris {
|
||||
|
||||
WalReader::WalReader(const std::string& file_name) : _file_name(file_name), _offset(0) {}
|
||||
|
||||
WalReader::~WalReader() {}
|
||||
|
||||
Status WalReader::init() {
|
||||
RETURN_IF_ERROR(io::global_local_filesystem()->open_file(_file_name, &file_reader));
|
||||
return Status::OK();
|
||||
}
|
||||
|
||||
Status WalReader::finalize() {
|
||||
auto st = file_reader->close();
|
||||
if (!st.ok()) {
|
||||
LOG(WARNING) << "fail to close file " << _file_name;
|
||||
}
|
||||
return Status::OK();
|
||||
}
|
||||
|
||||
Status WalReader::read_block(PBlock& block) {
|
||||
if (_offset >= file_reader->size()) {
|
||||
return Status::EndOfFile("end of wal file");
|
||||
}
|
||||
size_t bytes_read = 0;
|
||||
uint8_t row_len_buf[WalWriter::LENGTH_SIZE];
|
||||
RETURN_IF_ERROR(
|
||||
file_reader->read_at(_offset, {row_len_buf, WalWriter::LENGTH_SIZE}, &bytes_read));
|
||||
_offset += WalWriter::LENGTH_SIZE;
|
||||
size_t block_len;
|
||||
memcpy(&block_len, row_len_buf, WalWriter::LENGTH_SIZE);
|
||||
// read block
|
||||
std::string block_buf;
|
||||
block_buf.resize(block_len);
|
||||
RETURN_IF_ERROR(file_reader->read_at(_offset, {block_buf.c_str(), block_len}, &bytes_read));
|
||||
_offset += block_len;
|
||||
RETURN_IF_ERROR(_deserialize(block, block_buf));
|
||||
// checksum
|
||||
uint8_t checksum_len_buf[WalWriter::CHECKSUM_SIZE];
|
||||
RETURN_IF_ERROR(file_reader->read_at(_offset, {checksum_len_buf, WalWriter::CHECKSUM_SIZE},
|
||||
&bytes_read));
|
||||
_offset += WalWriter::CHECKSUM_SIZE;
|
||||
uint32_t checksum;
|
||||
memcpy(&checksum, checksum_len_buf, WalWriter::CHECKSUM_SIZE);
|
||||
RETURN_IF_ERROR(_check_checksum(block_buf.data(), block_len, checksum));
|
||||
return Status::OK();
|
||||
}
|
||||
|
||||
Status WalReader::_deserialize(PBlock& block, std::string& buf) {
|
||||
if (UNLIKELY(!block.ParseFromString(buf))) {
|
||||
return Status::InternalError("failed to deserialize row");
|
||||
}
|
||||
return Status::OK();
|
||||
}
|
||||
|
||||
Status WalReader::_check_checksum(const char* binary, size_t size, uint32_t checksum) {
|
||||
uint32_t computed_checksum = crc32c::Value(binary, size);
|
||||
if (LIKELY(computed_checksum == checksum)) {
|
||||
return Status::OK();
|
||||
}
|
||||
return Status::InternalError("checksum failed for wal=" + _file_name +
|
||||
", computed checksum=" + std::to_string(computed_checksum) +
|
||||
", expected=" + std::to_string(checksum));
|
||||
}
|
||||
|
||||
} // namespace doris
|
||||
45
be/src/olap/wal_reader.h
Normal file
45
be/src/olap/wal_reader.h
Normal file
@ -0,0 +1,45 @@
|
||||
// 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 "common/status.h"
|
||||
#include "gen_cpp/internal_service.pb.h"
|
||||
#include "io/fs/file_reader_writer_fwd.h"
|
||||
|
||||
namespace doris {
|
||||
|
||||
class WalReader {
|
||||
public:
|
||||
explicit WalReader(const std::string& file_name);
|
||||
~WalReader();
|
||||
|
||||
Status init();
|
||||
Status finalize();
|
||||
|
||||
Status read_block(PBlock& block);
|
||||
|
||||
private:
|
||||
Status _deserialize(PBlock& block, std::string& buf);
|
||||
Status _check_checksum(const char* binary, size_t size, uint32_t checksum);
|
||||
|
||||
std::string _file_name;
|
||||
size_t _offset;
|
||||
io::FileReaderSPtr file_reader;
|
||||
};
|
||||
|
||||
} // namespace doris
|
||||
265
be/src/olap/wal_table.cpp
Normal file
265
be/src/olap/wal_table.cpp
Normal file
@ -0,0 +1,265 @@
|
||||
// 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.
|
||||
|
||||
#include "olap/wal_table.h"
|
||||
|
||||
#include <event2/bufferevent.h>
|
||||
#include <event2/event.h>
|
||||
#include <event2/event_struct.h>
|
||||
#include <event2/http.h>
|
||||
#include <thrift/protocol/TDebugProtocol.h>
|
||||
|
||||
#include "evhttp.h"
|
||||
#include "http/action/stream_load.h"
|
||||
#include "http/ev_http_server.h"
|
||||
#include "http/http_common.h"
|
||||
#include "http/http_headers.h"
|
||||
#include "http/utils.h"
|
||||
#include "io/fs/local_file_system.h"
|
||||
#include "olap/wal_manager.h"
|
||||
#include "runtime/client_cache.h"
|
||||
#include "runtime/fragment_mgr.h"
|
||||
#include "runtime/plan_fragment_executor.h"
|
||||
#include "util/path_util.h"
|
||||
#include "util/thrift_rpc_helper.h"
|
||||
#include "vec/exec/format/wal/wal_reader.h"
|
||||
|
||||
namespace doris {
|
||||
|
||||
WalTable::WalTable(ExecEnv* exec_env, int64_t db_id, int64_t table_id)
|
||||
: _exec_env(exec_env), _db_id(db_id), _table_id(table_id) {}
|
||||
WalTable::~WalTable() {}
|
||||
|
||||
#ifdef BE_TEST
|
||||
std::string k_request_line;
|
||||
#endif
|
||||
|
||||
void WalTable::add_wals(std::vector<std::string> wals) {
|
||||
std::lock_guard<std::mutex> lock(_replay_wal_lock);
|
||||
for (const auto& wal : wals) {
|
||||
LOG(INFO) << "add replay wal " << wal;
|
||||
_replay_wal_map.emplace(wal, replay_wal_info {0, UnixMillis(), false});
|
||||
}
|
||||
}
|
||||
Status WalTable::replay_wals() {
|
||||
std::vector<std::string> need_replay_wals;
|
||||
{
|
||||
std::lock_guard<std::mutex> lock(_replay_wal_lock);
|
||||
if (_replay_wal_map.empty()) {
|
||||
return Status::OK();
|
||||
}
|
||||
VLOG_DEBUG << "Start replay wals for db=" << _db_id << ", table=" << _table_id
|
||||
<< ", wal size=" << _replay_wal_map.size();
|
||||
for (auto& [wal, info] : _replay_wal_map) {
|
||||
auto& [retry_num, start_ts, replaying] = info;
|
||||
if (replaying) {
|
||||
continue;
|
||||
}
|
||||
if (retry_num >= config::group_commit_replay_wal_retry_num) {
|
||||
LOG(WARNING) << "All replay wal failed, db=" << _db_id << ", table=" << _table_id
|
||||
<< ", wal=" << wal
|
||||
<< ", retry_num=" << config::group_commit_replay_wal_retry_num;
|
||||
std::string rename_path = get_tmp_path(wal);
|
||||
LOG(INFO) << "rename wal from " << wal << " to " << rename_path;
|
||||
std::rename(wal.c_str(), rename_path.c_str());
|
||||
_replay_wal_map.erase(wal);
|
||||
continue;
|
||||
}
|
||||
if (need_replay(info)) {
|
||||
replaying = true;
|
||||
need_replay_wals.push_back(wal);
|
||||
}
|
||||
}
|
||||
std::sort(need_replay_wals.begin(), need_replay_wals.end());
|
||||
}
|
||||
for (const auto& wal : need_replay_wals) {
|
||||
auto st = replay_wal_internal(wal);
|
||||
if (!st.ok()) {
|
||||
std::lock_guard<std::mutex> lock(_replay_wal_lock);
|
||||
auto it = _replay_wal_map.find(wal);
|
||||
if (it != _replay_wal_map.end()) {
|
||||
auto& [retry_num, start_time, replaying] = it->second;
|
||||
replaying = false;
|
||||
}
|
||||
LOG(WARNING) << "failed replay wal, drop this round, db=" << _db_id
|
||||
<< ", table=" << _table_id << ", wal=" << wal << ", st=" << st.to_string();
|
||||
break;
|
||||
}
|
||||
VLOG_NOTICE << "replay wal, db=" << _db_id << ", table=" << _table_id << ", label=" << wal
|
||||
<< ", st=" << st.to_string();
|
||||
}
|
||||
return Status::OK();
|
||||
}
|
||||
|
||||
std::string WalTable::get_tmp_path(const std::string wal) {
|
||||
std::vector<std::string> path_element;
|
||||
doris::vectorized::WalReader::string_split(wal, "/", path_element);
|
||||
std::stringstream ss;
|
||||
int index = 0;
|
||||
while (index < path_element.size() - 3) {
|
||||
ss << path_element[index] << "/";
|
||||
index++;
|
||||
}
|
||||
ss << "tmp/";
|
||||
while (index < path_element.size()) {
|
||||
if (index != path_element.size() - 1) {
|
||||
ss << path_element[index] << "_";
|
||||
} else {
|
||||
ss << path_element[index];
|
||||
}
|
||||
index++;
|
||||
}
|
||||
return ss.str();
|
||||
}
|
||||
|
||||
bool WalTable::need_replay(const doris::WalTable::replay_wal_info& info) {
|
||||
#ifndef BE_TEST
|
||||
auto& [retry_num, start_ts, replaying] = info;
|
||||
auto replay_interval =
|
||||
pow(2, retry_num) * config::group_commit_replay_wal_retry_interval_seconds * 1000;
|
||||
return UnixMillis() - start_ts >= replay_interval;
|
||||
#else
|
||||
return true;
|
||||
#endif
|
||||
}
|
||||
|
||||
Status WalTable::replay_wal_internal(const std::string& wal) {
|
||||
LOG(INFO) << "Start replay wal for db=" << _db_id << ", table=" << _table_id << ", wal=" << wal;
|
||||
// start a new stream load
|
||||
{
|
||||
std::lock_guard<std::mutex> lock(_replay_wal_lock);
|
||||
auto it = _replay_wal_map.find(wal);
|
||||
if (it != _replay_wal_map.end()) {
|
||||
auto& [retry_num, start_time, replaying] = it->second;
|
||||
++retry_num;
|
||||
replaying = true;
|
||||
} else {
|
||||
LOG(WARNING) << "can not find wal in stream load replay map. db=" << _db_id
|
||||
<< ", table=" << _table_id << ", wal=" << wal;
|
||||
return Status::OK();
|
||||
}
|
||||
}
|
||||
auto pair = get_wal_info(wal);
|
||||
auto wal_id = pair.first;
|
||||
auto label = pair.second;
|
||||
RETURN_IF_ERROR(send_request(wal_id, wal, label));
|
||||
return Status::OK();
|
||||
}
|
||||
|
||||
std::pair<int64_t, std::string> WalTable::get_wal_info(const std::string& wal) {
|
||||
std::vector<std::string> path_element;
|
||||
doris::vectorized::WalReader::string_split(wal, "/", path_element);
|
||||
auto pos = path_element[path_element.size() - 1].find("_");
|
||||
int64_t wal_id =
|
||||
std::strtoll(path_element[path_element.size() - 1].substr(0, pos).c_str(), NULL, 10);
|
||||
auto label = path_element[path_element.size() - 1].substr(pos + 1);
|
||||
return std::make_pair(wal_id, label);
|
||||
}
|
||||
|
||||
void http_request_done(struct evhttp_request* req, void* arg) {
|
||||
event_base_loopbreak((struct event_base*)arg);
|
||||
}
|
||||
|
||||
Status WalTable::send_request(int64_t wal_id, const std::string& wal, const std::string& label) {
|
||||
#ifndef BE_TEST
|
||||
struct event_base* base = nullptr;
|
||||
struct evhttp_connection* conn = nullptr;
|
||||
struct evhttp_request* req = nullptr;
|
||||
event_init();
|
||||
base = event_base_new();
|
||||
conn = evhttp_connection_new("127.0.0.1", doris::config::webserver_port);
|
||||
evhttp_connection_set_base(conn, base);
|
||||
req = evhttp_request_new(http_request_done, base);
|
||||
evhttp_add_header(req->output_headers, HTTP_LABEL_KEY.c_str(), label.c_str());
|
||||
evhttp_add_header(req->output_headers, HTTP_AUTH_CODE.c_str(), std::to_string(wal_id).c_str());
|
||||
evhttp_add_header(req->output_headers, HTTP_WAL_ID_KY.c_str(), std::to_string(wal_id).c_str());
|
||||
std::stringstream ss;
|
||||
ss << "insert into " << std::to_string(_table_id) << " WITH LABEL " << label
|
||||
<< " select * from "
|
||||
"http_stream(\"format\" = \"wal\", \"table_id\" = \""
|
||||
<< std::to_string(_table_id) << "\")";
|
||||
evhttp_add_header(req->output_headers, HTTP_SQL.c_str(), ss.str().c_str());
|
||||
evbuffer* output = evhttp_request_get_output_buffer(req);
|
||||
evbuffer_add_printf(output, "replay wal %s", std::to_string(wal_id).c_str());
|
||||
|
||||
evhttp_make_request(conn, req, EVHTTP_REQ_PUT, "/api/_http_stream");
|
||||
evhttp_connection_set_timeout(req->evcon, 300);
|
||||
|
||||
event_base_dispatch(base);
|
||||
evhttp_connection_free(conn);
|
||||
event_base_free(base);
|
||||
|
||||
#endif
|
||||
bool retry = false;
|
||||
std::string status;
|
||||
std::string msg;
|
||||
std::stringstream out;
|
||||
rapidjson::Document doc;
|
||||
#ifndef BE_TEST
|
||||
size_t len = 0;
|
||||
auto input = evhttp_request_get_input_buffer(req);
|
||||
char* request_line = evbuffer_readln(input, &len, EVBUFFER_EOL_CRLF);
|
||||
while (request_line != nullptr) {
|
||||
std::string s(request_line);
|
||||
out << request_line;
|
||||
request_line = evbuffer_readln(input, &len, EVBUFFER_EOL_CRLF);
|
||||
}
|
||||
#else
|
||||
out << k_request_line;
|
||||
#endif
|
||||
auto out_str = out.str();
|
||||
if (!out_str.empty()) {
|
||||
doc.Parse(out.str().c_str());
|
||||
status = std::string(doc["Status"].GetString());
|
||||
msg = std::string(doc["Message"].GetString());
|
||||
LOG(INFO) << "replay wal " << wal_id << " status:" << status << ",msg:" << msg;
|
||||
if (status.find("Fail") != status.npos) {
|
||||
if (msg.find("Label") != msg.npos && msg.find("has already been used") != msg.npos) {
|
||||
retry = false;
|
||||
} else {
|
||||
retry = true;
|
||||
}
|
||||
} else {
|
||||
retry = false;
|
||||
}
|
||||
} else {
|
||||
retry = true;
|
||||
}
|
||||
if (retry) {
|
||||
LOG(INFO) << "fail to replay wal =" << wal << ",status:" << status << ",msg:" << msg;
|
||||
std::lock_guard<std::mutex> lock(_replay_wal_lock);
|
||||
auto it = _replay_wal_map.find(wal);
|
||||
if (it != _replay_wal_map.end()) {
|
||||
auto& [retry_num, start_time, replaying] = it->second;
|
||||
replaying = false;
|
||||
} else {
|
||||
_replay_wal_map.emplace(wal, replay_wal_info {0, UnixMillis(), false});
|
||||
}
|
||||
} else {
|
||||
LOG(INFO) << "success to replay wal =" << wal << ",status:" << status << ",msg:" << msg;
|
||||
RETURN_IF_ERROR(_exec_env->wal_mgr()->delete_wal(wal_id));
|
||||
std::lock_guard<std::mutex> lock(_replay_wal_lock);
|
||||
_replay_wal_map.erase(wal);
|
||||
}
|
||||
return Status::OK();
|
||||
}
|
||||
|
||||
size_t WalTable::size() {
|
||||
std::lock_guard<std::mutex> lock(_replay_wal_lock);
|
||||
return _replay_wal_map.size();
|
||||
}
|
||||
} // namespace doris
|
||||
58
be/src/olap/wal_table.h
Normal file
58
be/src/olap/wal_table.h
Normal file
@ -0,0 +1,58 @@
|
||||
// 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 <mutex>
|
||||
#include <string>
|
||||
#include <unordered_map>
|
||||
#include <vector>
|
||||
|
||||
#include "common/status.h"
|
||||
#include "gen_cpp/FrontendService.h"
|
||||
#include "gen_cpp/FrontendService_types.h"
|
||||
#include "gen_cpp/HeartbeatService_types.h"
|
||||
#include "runtime/exec_env.h"
|
||||
#include "runtime/stream_load/stream_load_context.h"
|
||||
namespace doris {
|
||||
class WalTable {
|
||||
public:
|
||||
WalTable(ExecEnv* exec_env, int64_t db_id, int64_t table_id);
|
||||
~WalTable();
|
||||
// <retry_num, start_time_ms, is_doing_replay>
|
||||
using replay_wal_info = std::tuple<int64_t, int64_t, bool>;
|
||||
// used when be start and there are wals need to do recovery
|
||||
void add_wals(std::vector<std::string> wals);
|
||||
Status replay_wals();
|
||||
size_t size();
|
||||
|
||||
private:
|
||||
std::pair<int64_t, std::string> get_wal_info(const std::string& wal);
|
||||
std::string get_tmp_path(const std::string wal);
|
||||
Status send_request(int64_t wal_id, const std::string& wal, const std::string& label);
|
||||
|
||||
private:
|
||||
ExecEnv* _exec_env;
|
||||
int64_t _db_id;
|
||||
int64_t _table_id;
|
||||
std::string _relay = "relay";
|
||||
std::string _split = "_";
|
||||
mutable std::mutex _replay_wal_lock;
|
||||
// key is wal_id
|
||||
std::map<std::string, replay_wal_info> _replay_wal_map;
|
||||
bool need_replay(const replay_wal_info& info);
|
||||
Status replay_wal_internal(const std::string& wal);
|
||||
};
|
||||
} // namespace doris
|
||||
75
be/src/olap/wal_writer.cpp
Normal file
75
be/src/olap/wal_writer.cpp
Normal file
@ -0,0 +1,75 @@
|
||||
// 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.
|
||||
|
||||
#include "olap/wal_writer.h"
|
||||
|
||||
#include "io/fs/file_writer.h"
|
||||
#include "io/fs/local_file_system.h"
|
||||
#include "io/fs/path.h"
|
||||
#include "olap/storage_engine.h"
|
||||
#include "util/crc32c.h"
|
||||
|
||||
namespace doris {
|
||||
|
||||
WalWriter::WalWriter(const std::string& file_name) : _file_name(file_name) {}
|
||||
|
||||
WalWriter::~WalWriter() {}
|
||||
|
||||
Status WalWriter::init() {
|
||||
_batch = config::group_commit_sync_wal_batch;
|
||||
RETURN_IF_ERROR(io::global_local_filesystem()->create_file(_file_name, &_file_writer));
|
||||
return Status::OK();
|
||||
}
|
||||
|
||||
Status WalWriter::finalize() {
|
||||
auto st = _file_writer->close();
|
||||
if (!st.ok()) {
|
||||
LOG(WARNING) << "fail to close file " << _file_name;
|
||||
}
|
||||
return Status::OK();
|
||||
}
|
||||
|
||||
Status WalWriter::append_blocks(const PBlockArray& blocks) {
|
||||
size_t total_size = 0;
|
||||
for (const auto& block : blocks) {
|
||||
total_size += LENGTH_SIZE + block->ByteSizeLong() + CHECKSUM_SIZE;
|
||||
}
|
||||
std::string binary(total_size, '\0');
|
||||
char* row_binary = binary.data();
|
||||
size_t offset = 0;
|
||||
for (const auto& block : blocks) {
|
||||
unsigned long row_length = block->GetCachedSize();
|
||||
memcpy(row_binary + offset, &row_length, LENGTH_SIZE);
|
||||
offset += LENGTH_SIZE;
|
||||
memcpy(row_binary + offset, block->SerializeAsString().data(), row_length);
|
||||
offset += row_length;
|
||||
uint32_t checksum = crc32c::Value(block->SerializeAsString().data(), row_length);
|
||||
memcpy(row_binary + offset, &checksum, CHECKSUM_SIZE);
|
||||
offset += CHECKSUM_SIZE;
|
||||
}
|
||||
DCHECK(offset == total_size);
|
||||
// write rows
|
||||
RETURN_IF_ERROR(_file_writer->append({row_binary, offset}));
|
||||
_count++;
|
||||
if (_count % _batch == 0) {
|
||||
//todo sync data
|
||||
//LOG(INFO) << "count=" << count << ",do sync";
|
||||
}
|
||||
return Status::OK();
|
||||
}
|
||||
|
||||
} // namespace doris
|
||||
49
be/src/olap/wal_writer.h
Normal file
49
be/src/olap/wal_writer.h
Normal file
@ -0,0 +1,49 @@
|
||||
// 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 "common/status.h"
|
||||
#include "gen_cpp/internal_service.pb.h"
|
||||
#include "io/fs/file_reader_writer_fwd.h"
|
||||
|
||||
namespace doris {
|
||||
|
||||
using PBlockArray = std::vector<PBlock*>;
|
||||
|
||||
class WalWriter {
|
||||
public:
|
||||
explicit WalWriter(const std::string& file_name);
|
||||
~WalWriter();
|
||||
|
||||
Status init();
|
||||
Status finalize();
|
||||
|
||||
Status append_blocks(const PBlockArray& blocks);
|
||||
|
||||
std::string file_name() { return _file_name; };
|
||||
static const int64_t LENGTH_SIZE = 8;
|
||||
static const int64_t CHECKSUM_SIZE = 4;
|
||||
|
||||
private:
|
||||
std::string _file_name;
|
||||
io::FileWriterPtr _file_writer;
|
||||
int64_t _count;
|
||||
int64_t _batch;
|
||||
};
|
||||
|
||||
} // namespace doris
|
||||
@ -99,6 +99,7 @@ class SegmentLoader;
|
||||
class LookupConnectionCache;
|
||||
class RowCache;
|
||||
class CacheManager;
|
||||
class WalManager;
|
||||
|
||||
inline bool k_doris_exit = false;
|
||||
|
||||
@ -209,6 +210,7 @@ public:
|
||||
doris::vectorized::ScannerScheduler* scanner_scheduler() { return _scanner_scheduler; }
|
||||
FileMetaCache* file_meta_cache() { return _file_meta_cache; }
|
||||
MemTableMemoryLimiter* memtable_memory_limiter() { return _memtable_memory_limiter.get(); }
|
||||
WalManager* wal_mgr() { return _wal_manager.get(); }
|
||||
#ifdef BE_TEST
|
||||
void set_ready() { this->_s_ready = true; }
|
||||
void set_not_ready() { this->_s_ready = false; }
|
||||
@ -344,6 +346,7 @@ private:
|
||||
std::unique_ptr<MemTableMemoryLimiter> _memtable_memory_limiter;
|
||||
std::unique_ptr<stream_load::LoadStreamStubPool> _load_stream_stub_pool;
|
||||
std::unique_ptr<vectorized::DeltaWriterV2Pool> _delta_writer_v2_pool;
|
||||
std::shared_ptr<WalManager> _wal_manager;
|
||||
|
||||
std::mutex _frontends_lock;
|
||||
std::map<TNetworkAddress, FrontendInfo> _frontends;
|
||||
|
||||
@ -48,6 +48,7 @@
|
||||
#include "olap/schema_cache.h"
|
||||
#include "olap/segment_loader.h"
|
||||
#include "olap/storage_engine.h"
|
||||
#include "olap/wal_manager.h"
|
||||
#include "pipeline/task_queue.h"
|
||||
#include "pipeline/task_scheduler.h"
|
||||
#include "runtime/block_spill_manager.h"
|
||||
@ -210,6 +211,7 @@ Status ExecEnv::_init(const std::vector<StorePath>& store_paths,
|
||||
_memtable_memory_limiter = std::make_unique<MemTableMemoryLimiter>();
|
||||
_load_stream_stub_pool = std::make_unique<stream_load::LoadStreamStubPool>();
|
||||
_delta_writer_v2_pool = std::make_unique<vectorized::DeltaWriterV2Pool>();
|
||||
_wal_manager = WalManager::create_shared(this, config::group_commit_replay_wal_dir);
|
||||
|
||||
_backend_client_cache->init_metrics("backend");
|
||||
_frontend_client_cache->init_metrics("frontend");
|
||||
@ -232,6 +234,7 @@ Status ExecEnv::_init(const std::vector<StorePath>& store_paths,
|
||||
|
||||
RETURN_IF_ERROR(_memtable_memory_limiter->init(MemInfo::mem_limit()));
|
||||
RETURN_IF_ERROR(_load_channel_mgr->init(MemInfo::mem_limit()));
|
||||
RETURN_IF_ERROR(_wal_manager->init());
|
||||
_heartbeat_flags = new HeartbeatFlags();
|
||||
_register_metrics();
|
||||
|
||||
@ -526,6 +529,8 @@ void ExecEnv::destroy() {
|
||||
// Memory barrier to prevent other threads from accessing destructed resources
|
||||
_s_ready = false;
|
||||
|
||||
SAFE_STOP(_wal_manager);
|
||||
_wal_manager.reset();
|
||||
SAFE_STOP(_tablet_schema_cache);
|
||||
SAFE_STOP(_load_channel_mgr);
|
||||
SAFE_STOP(_scanner_scheduler);
|
||||
|
||||
@ -27,6 +27,7 @@
|
||||
#include "common/object_pool.h"
|
||||
#include "exec/data_sink.h"
|
||||
#include "io/fs/stream_load_pipe.h"
|
||||
#include "olap/wal_manager.h"
|
||||
#include "runtime/exec_env.h"
|
||||
#include "runtime/fragment_mgr.h"
|
||||
#include "runtime/runtime_state.h"
|
||||
@ -185,16 +186,17 @@ Status GroupCommitTable::get_first_block_load_queue(
|
||||
Status GroupCommitTable::_create_group_commit_load(
|
||||
int64_t table_id, std::shared_ptr<LoadBlockQueue>& load_block_queue) {
|
||||
TStreamLoadPutRequest request;
|
||||
std::stringstream ss;
|
||||
ss << "insert into " << table_id << " select * from group_commit(\"table_id\"=\"" << table_id
|
||||
<< "\")";
|
||||
request.__set_load_sql(ss.str());
|
||||
UniqueId load_id = UniqueId::gen_uid();
|
||||
TUniqueId tload_id;
|
||||
tload_id.__set_hi(load_id.hi);
|
||||
tload_id.__set_lo(load_id.lo);
|
||||
std::regex reg("-");
|
||||
std::string label = "group_commit_" + std::regex_replace(load_id.to_string(), reg, "_");
|
||||
std::stringstream ss;
|
||||
ss << "insert into " << table_id << " WITH LABEL " << label
|
||||
<< " select * from group_commit(\"table_id\"=\"" << table_id << "\")";
|
||||
request.__set_load_sql(ss.str());
|
||||
request.__set_loadId(tload_id);
|
||||
std::string label = "group_commit_" + load_id.to_string();
|
||||
request.__set_label(label);
|
||||
request.__set_token("group_commit"); // this is a fake, fe not check it now
|
||||
request.__set_max_filter_ratio(1.0);
|
||||
@ -243,6 +245,7 @@ Status GroupCommitTable::_create_group_commit_load(
|
||||
std::unique_lock l(_lock);
|
||||
_load_block_queues.emplace(instance_id, load_block_queue);
|
||||
}
|
||||
params.__set_import_label(label);
|
||||
st = _exec_plan_fragment(_db_id, table_id, label, txn_id, is_pipeline, params, pipeline_params);
|
||||
if (!st.ok()) {
|
||||
_finish_group_commit_load(_db_id, table_id, label, txn_id, instance_id, st, true, nullptr);
|
||||
@ -308,9 +311,27 @@ Status GroupCommitTable::_finish_group_commit_load(int64_t db_id, int64_t table_
|
||||
<< ", instance_id=" << print_id(instance_id)
|
||||
<< ", executor status=" << status.to_string()
|
||||
<< ", request commit status=" << st.to_string();
|
||||
if (!prepare_failed) {
|
||||
RETURN_IF_ERROR(_exec_env->wal_mgr()->add_wal_path(_db_id, table_id, txn_id, label));
|
||||
std::string wal_path;
|
||||
RETURN_IF_ERROR(_exec_env->wal_mgr()->get_wal_path(txn_id, wal_path));
|
||||
RETURN_IF_ERROR(_exec_env->wal_mgr()->add_recover_wal(
|
||||
std::to_string(db_id), std::to_string(table_id),
|
||||
std::vector<std::string> {wal_path}));
|
||||
}
|
||||
return st;
|
||||
}
|
||||
// TODO handle execute and commit error
|
||||
if (!prepare_failed && !result_status.ok()) {
|
||||
RETURN_IF_ERROR(_exec_env->wal_mgr()->add_wal_path(_db_id, table_id, txn_id, label));
|
||||
std::string wal_path;
|
||||
RETURN_IF_ERROR(_exec_env->wal_mgr()->get_wal_path(txn_id, wal_path));
|
||||
RETURN_IF_ERROR(_exec_env->wal_mgr()->add_recover_wal(std::to_string(db_id),
|
||||
std::to_string(table_id),
|
||||
std::vector<std::string> {wal_path}));
|
||||
} else {
|
||||
RETURN_IF_ERROR(_exec_env->wal_mgr()->delete_wal(txn_id));
|
||||
}
|
||||
std::stringstream ss;
|
||||
ss << "finish group commit, db_id=" << db_id << ", table_id=" << table_id << ", label=" << label
|
||||
<< ", txn_id=" << txn_id << ", instance_id=" << print_id(instance_id);
|
||||
|
||||
@ -156,6 +156,9 @@ Status PlanFragmentExecutor::prepare(const TExecPlanFragmentParams& request) {
|
||||
if (request.__isset.load_job_id) {
|
||||
_runtime_state->set_load_job_id(request.load_job_id);
|
||||
}
|
||||
if (request.__isset.wal_id) {
|
||||
_runtime_state->set_wal_id(request.wal_id);
|
||||
}
|
||||
|
||||
if (request.query_options.__isset.is_report_success) {
|
||||
_is_report_success = request.query_options.is_report_success;
|
||||
|
||||
@ -229,6 +229,12 @@ public:
|
||||
|
||||
const std::string& db_name() { return _db_name; }
|
||||
|
||||
void set_wal_id(int64_t wal_id) { _wal_id = wal_id; }
|
||||
|
||||
int64_t wal_id() { return _wal_id; }
|
||||
|
||||
const std::string& import_label() { return _import_label; }
|
||||
|
||||
const std::string& load_dir() const { return _load_dir; }
|
||||
|
||||
void set_load_job_id(int64_t job_id) { _load_job_id = job_id; }
|
||||
@ -547,6 +553,7 @@ private:
|
||||
std::string _db_name;
|
||||
std::string _load_dir;
|
||||
int64_t _load_job_id;
|
||||
int64_t _wal_id = -1;
|
||||
|
||||
// mini load
|
||||
int64_t _normal_row_number;
|
||||
|
||||
@ -134,6 +134,7 @@ public:
|
||||
|
||||
std::string db;
|
||||
int64_t db_id = -1;
|
||||
int64_t wal_id = -1;
|
||||
std::string table;
|
||||
std::string label;
|
||||
// optional
|
||||
|
||||
@ -67,6 +67,8 @@ void LoadUtil::parse_format(const std::string& format_str, const std::string& co
|
||||
*format_type = TFileFormatType::FORMAT_PARQUET;
|
||||
} else if (iequal(format_str, "ORC")) {
|
||||
*format_type = TFileFormatType::FORMAT_ORC;
|
||||
} else if (iequal(format_str, "WAL")) {
|
||||
*format_type = TFileFormatType::FORMAT_WAL;
|
||||
}
|
||||
return;
|
||||
}
|
||||
@ -82,6 +84,7 @@ bool LoadUtil::is_format_support_streaming(TFileFormatType::type format) {
|
||||
case TFileFormatType::FORMAT_CSV_LZO:
|
||||
case TFileFormatType::FORMAT_CSV_LZOP:
|
||||
case TFileFormatType::FORMAT_JSON:
|
||||
case TFileFormatType::FORMAT_WAL:
|
||||
return true;
|
||||
default:
|
||||
return false;
|
||||
|
||||
77
be/src/vec/exec/format/wal/wal_reader.cpp
Normal file
77
be/src/vec/exec/format/wal/wal_reader.cpp
Normal file
@ -0,0 +1,77 @@
|
||||
// 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.
|
||||
|
||||
#include "wal_reader.h"
|
||||
|
||||
#include "common/logging.h"
|
||||
#include "olap/wal_manager.h"
|
||||
#include "runtime/runtime_state.h"
|
||||
namespace doris::vectorized {
|
||||
WalReader::WalReader(RuntimeState* state) : _state(state) {
|
||||
_wal_id = state->wal_id();
|
||||
}
|
||||
WalReader::~WalReader() {
|
||||
if (_wal_reader.get() != nullptr) {
|
||||
_wal_reader->finalize();
|
||||
}
|
||||
}
|
||||
Status WalReader::init_reader() {
|
||||
RETURN_IF_ERROR(_state->exec_env()->wal_mgr()->get_wal_path(_wal_id, _wal_path));
|
||||
RETURN_IF_ERROR(_state->exec_env()->wal_mgr()->create_wal_reader(_wal_path, _wal_reader));
|
||||
return Status::OK();
|
||||
}
|
||||
Status WalReader::get_next_block(Block* block, size_t* read_rows, bool* eof) {
|
||||
PBlock pblock;
|
||||
auto st = _wal_reader->read_block(pblock);
|
||||
if (st.is<ErrorCode::END_OF_FILE>()) {
|
||||
LOG(INFO) << "read eof on wal:" << _wal_path;
|
||||
*read_rows = 0;
|
||||
*eof = true;
|
||||
return Status::OK();
|
||||
}
|
||||
if (!st.ok()) {
|
||||
LOG(WARNING) << "Failed to read wal on path = " << _wal_path;
|
||||
return st;
|
||||
}
|
||||
vectorized::Block tmp_block;
|
||||
tmp_block.deserialize(pblock);
|
||||
block->swap(tmp_block);
|
||||
*read_rows = block->rows();
|
||||
VLOG_DEBUG << "read block rows:" << *read_rows;
|
||||
return Status::OK();
|
||||
}
|
||||
|
||||
void WalReader::string_split(const std::string& str, const std::string& splits,
|
||||
std::vector<std::string>& res) {
|
||||
if (str == "") return;
|
||||
std::string strs = str + splits;
|
||||
size_t pos = strs.find(splits);
|
||||
int step = splits.size();
|
||||
while (pos != strs.npos) {
|
||||
std::string temp = strs.substr(0, pos);
|
||||
res.push_back(temp);
|
||||
strs = strs.substr(pos + step, strs.size());
|
||||
pos = strs.find(splits);
|
||||
}
|
||||
}
|
||||
|
||||
Status WalReader::get_columns(std::unordered_map<std::string, TypeDescriptor>* name_to_type,
|
||||
std::unordered_set<std::string>* missing_cols) {
|
||||
return Status::OK();
|
||||
}
|
||||
|
||||
} // namespace doris::vectorized
|
||||
43
be/src/vec/exec/format/wal/wal_reader.h
Normal file
43
be/src/vec/exec/format/wal/wal_reader.h
Normal file
@ -0,0 +1,43 @@
|
||||
// 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 "olap/wal_reader.h"
|
||||
#include "vec/exec/format/generic_reader.h"
|
||||
namespace doris {
|
||||
namespace vectorized {
|
||||
struct ScannerCounter;
|
||||
class WalReader : public GenericReader {
|
||||
public:
|
||||
WalReader(RuntimeState* state);
|
||||
~WalReader() override;
|
||||
Status init_reader();
|
||||
Status get_next_block(Block* block, size_t* read_rows, bool* eof) override;
|
||||
Status get_columns(std::unordered_map<std::string, TypeDescriptor>* name_to_type,
|
||||
std::unordered_set<std::string>* missing_cols) override;
|
||||
static void string_split(const std::string& str, const std::string& splits,
|
||||
std::vector<std::string>& res);
|
||||
|
||||
private:
|
||||
RuntimeState* _state;
|
||||
std::string _wal_path;
|
||||
std::string _path_split = "/";
|
||||
int64_t _wal_id;
|
||||
std::shared_ptr<doris::WalReader> _wal_reader = nullptr;
|
||||
};
|
||||
} // namespace vectorized
|
||||
} // namespace doris
|
||||
@ -32,6 +32,7 @@
|
||||
#include <utility>
|
||||
|
||||
#include "vec/data_types/data_type_factory.hpp"
|
||||
#include "vec/exec/format/wal/wal_reader.h"
|
||||
|
||||
// IWYU pragma: no_include <opentelemetry/common/threadlocal.h>
|
||||
#include "common/compiler_util.h" // IWYU pragma: keep
|
||||
@ -258,6 +259,10 @@ Status VFileScanner::_get_block_impl(RuntimeState* state, Block* block, bool* eo
|
||||
RETURN_IF_ERROR(
|
||||
_cur_reader->get_next_block(_src_block_ptr, &read_rows, &_cur_reader_eof));
|
||||
}
|
||||
if (_params->format_type == TFileFormatType::FORMAT_WAL) {
|
||||
block->swap(*_src_block_ptr);
|
||||
break;
|
||||
}
|
||||
// use read_rows instead of _src_block_ptr->rows(), because the first column of _src_block_ptr
|
||||
// may not be filled after calling `get_next_block()`, so _src_block_ptr->rows() may return wrong result.
|
||||
if (read_rows > 0) {
|
||||
@ -784,6 +789,11 @@ Status VFileScanner::_get_next_reader() {
|
||||
->init_fetch_table_reader(_colname_to_value_range);
|
||||
break;
|
||||
}
|
||||
case TFileFormatType::FORMAT_WAL: {
|
||||
_cur_reader.reset(new WalReader(_state));
|
||||
init_status = ((WalReader*)(_cur_reader.get()))->init_reader();
|
||||
break;
|
||||
}
|
||||
default:
|
||||
return Status::InternalError("Not supported file format: {}", _params->format_type);
|
||||
}
|
||||
|
||||
@ -66,6 +66,7 @@ Status OlapTabletFinder::find_tablet(RuntimeState* state, Block* block, int row_
|
||||
},
|
||||
&stop_processing));
|
||||
_num_filtered_rows++;
|
||||
_filter_bitmap.Set(row_index, true);
|
||||
if (stop_processing) {
|
||||
return Status::EndOfFile("Encountered unqualified data, stop processing");
|
||||
}
|
||||
|
||||
@ -21,6 +21,7 @@
|
||||
|
||||
#include "common/status.h"
|
||||
#include "exec/tablet_info.h"
|
||||
#include "util/bitmap.h"
|
||||
#include "vec/core/block.h"
|
||||
|
||||
namespace doris::vectorized {
|
||||
@ -36,7 +37,7 @@ public:
|
||||
enum FindTabletMode { FIND_TABLET_EVERY_ROW, FIND_TABLET_EVERY_BATCH, FIND_TABLET_EVERY_SINK };
|
||||
|
||||
OlapTabletFinder(VOlapTablePartitionParam* vpartition, FindTabletMode mode)
|
||||
: _vpartition(vpartition), _find_tablet_mode(mode) {};
|
||||
: _vpartition(vpartition), _find_tablet_mode(mode), _filter_bitmap(1024) {};
|
||||
|
||||
Status find_tablet(RuntimeState* state, vectorized::Block* block, int row_index,
|
||||
const VOlapTablePartition** partition, uint32_t& tablet_index,
|
||||
@ -62,6 +63,8 @@ public:
|
||||
return _num_immutable_partition_filtered_rows;
|
||||
}
|
||||
|
||||
Bitmap& filter_bitmap() { return _filter_bitmap; }
|
||||
|
||||
private:
|
||||
VOlapTablePartitionParam* _vpartition;
|
||||
FindTabletMode _find_tablet_mode;
|
||||
@ -70,6 +73,7 @@ private:
|
||||
|
||||
int64_t _num_filtered_rows = 0;
|
||||
int64_t _num_immutable_partition_filtered_rows = 0;
|
||||
Bitmap _filter_bitmap;
|
||||
};
|
||||
|
||||
} // namespace doris::vectorized
|
||||
@ -134,7 +134,7 @@ VOlapTableSink::VOlapTableSink(ObjectPool* pool, const RowDescriptor& row_desc,
|
||||
|
||||
Status VOlapTableSink::init(const TDataSink& t_sink) {
|
||||
RETURN_IF_ERROR(AsyncWriterSink::init(t_sink));
|
||||
_writer->init_properties(_pool, _group_commit);
|
||||
RETURN_IF_ERROR(_writer->init_properties(_pool, _group_commit));
|
||||
return Status::OK();
|
||||
}
|
||||
|
||||
|
||||
@ -57,6 +57,7 @@
|
||||
#include "exec/data_sink.h"
|
||||
#include "exec/tablet_info.h"
|
||||
#include "gutil/ref_counted.h"
|
||||
#include "olap/wal_writer.h"
|
||||
#include "runtime/decimalv2_value.h"
|
||||
#include "runtime/exec_env.h"
|
||||
#include "runtime/memory/mem_tracker.h"
|
||||
|
||||
@ -43,6 +43,7 @@
|
||||
#include <unordered_map>
|
||||
#include <utility>
|
||||
|
||||
#include "olap/wal_manager.h"
|
||||
#include "util/runtime_profile.h"
|
||||
#include "vec/data_types/data_type.h"
|
||||
#include "vec/exprs/vexpr_fwd.h"
|
||||
@ -969,11 +970,17 @@ void VNodeChannel::mark_close() {
|
||||
VTabletWriter::VTabletWriter(const TDataSink& t_sink, const VExprContextSPtrs& output_exprs)
|
||||
: AsyncResultWriter(output_exprs), _t_sink(t_sink) {
|
||||
_transfer_large_data_by_brpc = config::transfer_large_data_by_brpc;
|
||||
DCHECK(t_sink.__isset.olap_table_sink);
|
||||
auto& table_sink = t_sink.olap_table_sink;
|
||||
_db_id = table_sink.db_id;
|
||||
_tb_id = table_sink.table_id;
|
||||
_wal_id = table_sink.txn_id;
|
||||
}
|
||||
|
||||
void VTabletWriter::init_properties(doris::ObjectPool* pool, bool group_commit) {
|
||||
Status VTabletWriter::init_properties(doris::ObjectPool* pool, bool group_commit) {
|
||||
_pool = pool;
|
||||
_group_commit = group_commit;
|
||||
return Status::OK();
|
||||
}
|
||||
|
||||
void VTabletWriter::_send_batch_process() {
|
||||
@ -1198,6 +1205,11 @@ Status VTabletWriter::_init(RuntimeState* state, RuntimeProfile* profile) {
|
||||
auto [part_ctx, part_func] = _get_partition_function();
|
||||
RETURN_IF_ERROR(part_func->prepare(_state, *_output_row_desc, part_ctx.get()));
|
||||
}
|
||||
if (_group_commit) {
|
||||
RETURN_IF_ERROR(_state->exec_env()->wal_mgr()->add_wal_path(_db_id, _tb_id, _wal_id,
|
||||
_state->import_label()));
|
||||
RETURN_IF_ERROR(_state->exec_env()->wal_mgr()->create_wal_writer(_wal_id, _wal_writer));
|
||||
}
|
||||
|
||||
_prepare = true;
|
||||
return Status::OK();
|
||||
@ -1619,6 +1631,9 @@ Status VTabletWriter::close(Status exec_status) {
|
||||
[](const std::shared_ptr<VNodeChannel>& ch) { ch->clear_all_blocks(); });
|
||||
}
|
||||
|
||||
if (_wal_writer.get() != nullptr) {
|
||||
_wal_writer->finalize();
|
||||
}
|
||||
return _close_status;
|
||||
}
|
||||
|
||||
@ -1659,6 +1674,7 @@ Status VTabletWriter::append_block(doris::vectorized::Block& input_block) {
|
||||
_tablet_finder->clear_for_new_batch();
|
||||
_row_distribution_watch.start();
|
||||
auto num_rows = block->rows();
|
||||
_tablet_finder->filter_bitmap().Reset(num_rows);
|
||||
size_t partition_num = _vpartition->get_partitions().size();
|
||||
if (!_vpartition->is_auto_partition() && partition_num == 1 &&
|
||||
_tablet_finder->is_find_tablet_every_sink()) {
|
||||
@ -1778,9 +1794,10 @@ Status VTabletWriter::append_block(doris::vectorized::Block& input_block) {
|
||||
}
|
||||
|
||||
if (_group_commit) {
|
||||
_group_commit_block(&input_block, rows,
|
||||
_group_commit_block(&input_block, num_rows,
|
||||
_block_convertor->num_filtered_rows() +
|
||||
_tablet_finder->num_filtered_rows() - filtered_rows);
|
||||
_tablet_finder->num_filtered_rows() - filtered_rows,
|
||||
_state, block.get(), _block_convertor.get(), _tablet_finder.get());
|
||||
}
|
||||
// TODO: Before load, we need to projection unuseful column
|
||||
// auto slots = _schema->tuple_desc()->slots();
|
||||
@ -1808,11 +1825,48 @@ Status VTabletWriter::append_block(doris::vectorized::Block& input_block) {
|
||||
return Status::OK();
|
||||
}
|
||||
|
||||
void VTabletWriter::_group_commit_block(Block* input_block, int64_t rows, int64_t filter_rows) {
|
||||
Status VTabletWriter::write_wal(OlapTableBlockConvertor* block_convertor,
|
||||
OlapTabletFinder* tablet_finder, vectorized::Block* block,
|
||||
RuntimeState* state, int64_t num_rows, int64_t filtered_rows) {
|
||||
PBlock pblock;
|
||||
size_t uncompressed_bytes = 0, compressed_bytes = 0;
|
||||
if (filtered_rows == 0) {
|
||||
RETURN_IF_ERROR(block->serialize(state->be_exec_version(), &pblock, &uncompressed_bytes,
|
||||
&compressed_bytes, segment_v2::CompressionTypePB::SNAPPY));
|
||||
RETURN_IF_ERROR(_wal_writer->append_blocks(std::vector<PBlock*> {&pblock}));
|
||||
} else {
|
||||
auto cloneBlock = block->clone_without_columns();
|
||||
auto res_block = vectorized::MutableBlock::build_mutable_block(&cloneBlock);
|
||||
for (int i = 0; i < num_rows; ++i) {
|
||||
if (block_convertor->num_filtered_rows() > 0 &&
|
||||
block_convertor->filter_bitmap().Get(i)) {
|
||||
continue;
|
||||
}
|
||||
if (tablet_finder->num_filtered_rows() > 0 && tablet_finder->filter_bitmap().Get(i)) {
|
||||
continue;
|
||||
}
|
||||
res_block.add_row(block, i);
|
||||
}
|
||||
RETURN_IF_ERROR(res_block.to_block().serialize(state->be_exec_version(), &pblock,
|
||||
&uncompressed_bytes, &compressed_bytes,
|
||||
segment_v2::CompressionTypePB::SNAPPY));
|
||||
RETURN_IF_ERROR(_wal_writer->append_blocks(std::vector<PBlock*> {&pblock}));
|
||||
}
|
||||
return Status::OK();
|
||||
}
|
||||
|
||||
void VTabletWriter::_group_commit_block(vectorized::Block* input_block, int64_t num_rows,
|
||||
int64_t filter_rows, RuntimeState* state,
|
||||
vectorized::Block* block,
|
||||
OlapTableBlockConvertor* block_convertor,
|
||||
OlapTabletFinder* tablet_finder) {
|
||||
write_wal(block_convertor, tablet_finder, block, state, num_rows, filter_rows);
|
||||
#ifndef BE_TEST
|
||||
auto* future_block = assert_cast<FutureBlock*>(input_block);
|
||||
std::unique_lock<doris::Mutex> l(*(future_block->lock));
|
||||
future_block->set_result(Status::OK(), rows, rows - filter_rows);
|
||||
future_block->set_result(Status::OK(), num_rows, num_rows - filter_rows);
|
||||
future_block->cv->notify_all();
|
||||
#endif
|
||||
}
|
||||
|
||||
} // namespace vectorized
|
||||
|
||||
@ -33,6 +33,8 @@
|
||||
#include <stdint.h>
|
||||
|
||||
#include <atomic>
|
||||
|
||||
#include "olap/wal_writer.h"
|
||||
// IWYU pragma: no_include <bits/chrono.h>
|
||||
#include <chrono> // IWYU pragma: keep
|
||||
#include <cstdint>
|
||||
@ -527,7 +529,7 @@ class VTabletWriter final : public AsyncResultWriter {
|
||||
public:
|
||||
VTabletWriter(const TDataSink& t_sink, const VExprContextSPtrs& output_exprs);
|
||||
|
||||
void init_properties(ObjectPool* pool, bool group_commit);
|
||||
Status init_properties(ObjectPool* pool, bool group_commit);
|
||||
|
||||
Status append_block(Block& block) override;
|
||||
|
||||
@ -575,7 +577,14 @@ private:
|
||||
|
||||
Status _incremental_open_node_channel(const std::vector<TOlapTablePartition>& partitions);
|
||||
|
||||
void _group_commit_block(vectorized::Block* input_block, int64_t rows, int64_t filter_rows);
|
||||
Status write_wal(OlapTableBlockConvertor* block_convertor, OlapTabletFinder* tablet_finder,
|
||||
vectorized::Block* block, RuntimeState* state, int64_t num_rows,
|
||||
int64_t filtered_rows);
|
||||
|
||||
void _group_commit_block(vectorized::Block* input_block, int64_t num_rows, int64_t filter_rows,
|
||||
RuntimeState* state, vectorized::Block* block,
|
||||
OlapTableBlockConvertor* block_convertor,
|
||||
OlapTabletFinder* tablet_finder);
|
||||
|
||||
TDataSink _t_sink;
|
||||
|
||||
@ -673,5 +682,9 @@ private:
|
||||
RuntimeState* _state = nullptr; // not owned, set when open
|
||||
RuntimeProfile* _profile = nullptr; // not owned, set when open
|
||||
bool _group_commit = false;
|
||||
std::shared_ptr<WalWriter> _wal_writer = nullptr;
|
||||
int64_t _tb_id;
|
||||
int64_t _db_id;
|
||||
int64_t _wal_id;
|
||||
};
|
||||
} // namespace doris::vectorized
|
||||
|
||||
BIN
be/test/exec/test_data/wal_scanner/wal
Normal file
BIN
be/test/exec/test_data/wal_scanner/wal
Normal file
Binary file not shown.
165
be/test/olap/wal_manager_test.cpp
Normal file
165
be/test/olap/wal_manager_test.cpp
Normal file
@ -0,0 +1,165 @@
|
||||
// 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.
|
||||
#include "olap/wal_manager.h"
|
||||
|
||||
#include <gtest/gtest.h>
|
||||
|
||||
#include <filesystem>
|
||||
#include <map>
|
||||
#include <string>
|
||||
#include <vector>
|
||||
|
||||
#include "common/config.h"
|
||||
#include "gen_cpp/HeartbeatService_types.h"
|
||||
#include "gen_cpp/internal_service.pb.h"
|
||||
#include "io/fs/local_file_system.h"
|
||||
#include "runtime/decimalv2_value.h"
|
||||
#include "runtime/exec_env.h"
|
||||
#include "runtime/result_queue_mgr.h"
|
||||
#include "runtime/runtime_state.h"
|
||||
#include "runtime/stream_load/new_load_stream_mgr.h"
|
||||
#include "runtime/types.h"
|
||||
#include "service/brpc.h"
|
||||
#include "util/brpc_client_cache.h"
|
||||
#include "util/cpu_info.h"
|
||||
#include "util/debug/leakcheck_disabler.h"
|
||||
#include "util/proto_util.h"
|
||||
|
||||
namespace doris {
|
||||
|
||||
extern TLoadTxnBeginResult k_stream_load_begin_result;
|
||||
extern Status k_stream_load_plan_status;
|
||||
extern std::string k_request_line;
|
||||
|
||||
ExecEnv* _env = nullptr;
|
||||
std::string wal_dir = "./wal_test";
|
||||
std::string tmp_dir = "./wal_test/tmp";
|
||||
|
||||
class WalManagerTest : public testing::Test {
|
||||
public:
|
||||
WalManagerTest() {}
|
||||
virtual ~WalManagerTest() {}
|
||||
void SetUp() override {
|
||||
prepare();
|
||||
_env = ExecEnv::GetInstance();
|
||||
_env->_master_info = new TMasterInfo();
|
||||
_env->_master_info->network_address.hostname = "host name";
|
||||
_env->_master_info->network_address.port = 1234;
|
||||
_env->new_load_stream_mgr() = NewLoadStreamMgr::create_shared();
|
||||
_env->_internal_client_cache = new BrpcClientCache<PBackendService_Stub>();
|
||||
_env->_function_client_cache = new BrpcClientCache<PFunctionService_Stub>();
|
||||
_env->_stream_load_executor = StreamLoadExecutor::create_shared(_env);
|
||||
_env->_wal_manager = WalManager::create_shared(_env, wal_dir);
|
||||
k_stream_load_begin_result = TLoadTxnBeginResult();
|
||||
k_stream_load_plan_status = Status::OK();
|
||||
}
|
||||
void TearDown() override {
|
||||
io::global_local_filesystem()->delete_directory(wal_dir);
|
||||
SAFE_DELETE(_env->_function_client_cache);
|
||||
SAFE_DELETE(_env->_internal_client_cache);
|
||||
SAFE_DELETE(_env->_master_info);
|
||||
}
|
||||
|
||||
void prepare() { io::global_local_filesystem()->create_directory(wal_dir); }
|
||||
|
||||
void createWal(const std::string& wal_path) {
|
||||
auto wal_writer = WalWriter(wal_path);
|
||||
wal_writer.init();
|
||||
wal_writer.finalize();
|
||||
}
|
||||
};
|
||||
|
||||
TEST_F(WalManagerTest, recovery_normal) {
|
||||
k_request_line = "{\"Status\": \"Success\", \"Message\": \"Test\"}";
|
||||
|
||||
std::string db_id = "1";
|
||||
std::string tb_1_id = "1";
|
||||
std::string wal_100_id = "100";
|
||||
std::string wal_101_id = "101";
|
||||
std::string tb_2_id = "2";
|
||||
std::string wal_200_id = "200";
|
||||
std::string wal_201_id = "201";
|
||||
|
||||
std::filesystem::create_directory(wal_dir + "/" + db_id);
|
||||
std::filesystem::create_directory(wal_dir + "/" + db_id + "/" + tb_1_id);
|
||||
std::string wal_100 = wal_dir + "/" + db_id + "/" + tb_1_id + "/" + wal_100_id;
|
||||
std::string wal_101 = wal_dir + "/" + db_id + "/" + tb_1_id + "/" + wal_101_id;
|
||||
createWal(wal_100);
|
||||
createWal(wal_101);
|
||||
|
||||
std::filesystem::create_directory(wal_dir + "/" + db_id);
|
||||
std::filesystem::create_directory(wal_dir + "/" + db_id + "/" + tb_2_id);
|
||||
std::string wal_200 = wal_dir + "/" + db_id + "/" + tb_2_id + "/" + wal_200_id;
|
||||
std::string wal_201 = wal_dir + "/" + db_id + "/" + tb_2_id + "/" + wal_201_id;
|
||||
createWal(wal_200);
|
||||
createWal(wal_201);
|
||||
_env->wal_mgr()->init();
|
||||
|
||||
while (_env->wal_mgr()->get_wal_table_size(tb_1_id) > 0 ||
|
||||
_env->wal_mgr()->get_wal_table_size(tb_2_id) > 0) {
|
||||
sleep(1);
|
||||
continue;
|
||||
}
|
||||
ASSERT_TRUE(!std::filesystem::exists(wal_100));
|
||||
ASSERT_TRUE(!std::filesystem::exists(wal_101));
|
||||
ASSERT_TRUE(!std::filesystem::exists(wal_200));
|
||||
ASSERT_TRUE(!std::filesystem::exists(wal_201));
|
||||
}
|
||||
|
||||
TEST_F(WalManagerTest, not_need_recovery) {
|
||||
std::string db_id = "1";
|
||||
std::string tb_id = "1";
|
||||
std::string wal_id = "100";
|
||||
std::filesystem::create_directory(wal_dir + "/" + db_id);
|
||||
std::filesystem::create_directory(wal_dir + "/" + db_id + "/" + tb_id);
|
||||
std::string wal_100 = wal_dir + "/" + db_id + "/" + tb_id + "/" + wal_id;
|
||||
createWal(wal_100);
|
||||
|
||||
_env->wal_mgr()->init();
|
||||
|
||||
while (_env->wal_mgr()->get_wal_table_size("1") > 0) {
|
||||
sleep(1);
|
||||
continue;
|
||||
}
|
||||
ASSERT_TRUE(!std::filesystem::exists(wal_100));
|
||||
}
|
||||
|
||||
TEST_F(WalManagerTest, recover_fail) {
|
||||
k_request_line = "{\"Status\": \"Fail\", \"Message\": \"Test\"}";
|
||||
config::group_commit_replay_wal_retry_num = 3;
|
||||
config::group_commit_replay_wal_retry_interval_seconds = 1;
|
||||
|
||||
std::string db_id = "1";
|
||||
std::string tb_id = "1";
|
||||
std::string wal_id = "100";
|
||||
std::filesystem::create_directory(wal_dir + "/" + db_id);
|
||||
std::filesystem::create_directory(wal_dir + "/" + db_id + "/" + tb_id);
|
||||
std::string wal_100 = wal_dir + "/" + db_id + "/" + tb_id + "/" + wal_id;
|
||||
createWal(wal_100);
|
||||
|
||||
_env->wal_mgr()->init();
|
||||
|
||||
while (_env->wal_mgr()->get_wal_table_size("1") > 0) {
|
||||
sleep(1);
|
||||
continue;
|
||||
}
|
||||
std::string tmp_file = tmp_dir + "/" + db_id + "_" + tb_id + "_" + wal_id;
|
||||
ASSERT_TRUE(std::filesystem::exists(tmp_file));
|
||||
ASSERT_TRUE(!std::filesystem::exists(wal_100));
|
||||
}
|
||||
|
||||
} // namespace doris
|
||||
137
be/test/olap/wal_reader_writer_test.cpp
Normal file
137
be/test/olap/wal_reader_writer_test.cpp
Normal file
@ -0,0 +1,137 @@
|
||||
// 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.
|
||||
#include <gtest/gtest.h>
|
||||
|
||||
#include <filesystem>
|
||||
|
||||
#include "agent/be_exec_version_manager.h"
|
||||
#include "common/object_pool.h"
|
||||
#include "gen_cpp/internal_service.pb.h"
|
||||
#include "gmock/gmock.h"
|
||||
#include "io/fs/local_file_system.h"
|
||||
#include "olap/wal_reader.h"
|
||||
#include "olap/wal_writer.h"
|
||||
#include "runtime/exec_env.h"
|
||||
#include "service/brpc.h"
|
||||
#include "testutil/test_util.h"
|
||||
#include "util/proto_util.h"
|
||||
#include "vec/columns/columns_number.h"
|
||||
#include "vec/data_types/data_type_number.h"
|
||||
#include "vec/runtime/vdata_stream_mgr.h"
|
||||
#include "vec/runtime/vdata_stream_recvr.h"
|
||||
|
||||
using ::testing::_;
|
||||
using ::testing::Return;
|
||||
using ::testing::SetArgPointee;
|
||||
using std::string;
|
||||
|
||||
namespace doris {
|
||||
|
||||
class WalReaderWriterTest : public testing::Test {
|
||||
public:
|
||||
// create a mock cgroup folder
|
||||
virtual void SetUp() { io::global_local_filesystem()->create_directory(_s_test_data_path); }
|
||||
|
||||
// delete the mock cgroup folder
|
||||
virtual void TearDown() { io::global_local_filesystem()->delete_directory(_s_test_data_path); }
|
||||
|
||||
static std::string _s_test_data_path;
|
||||
};
|
||||
|
||||
std::string WalReaderWriterTest::_s_test_data_path = "./log/wal_reader_writer_test";
|
||||
size_t block_rows = 1024;
|
||||
|
||||
void covert_block_to_pb(
|
||||
const vectorized::Block& block, PBlock* pblock,
|
||||
segment_v2::CompressionTypePB compression_type = segment_v2::CompressionTypePB::SNAPPY) {
|
||||
size_t uncompressed_bytes = 0;
|
||||
size_t compressed_bytes = 0;
|
||||
Status st = block.serialize(BeExecVersionManager::get_newest_version(), pblock,
|
||||
&uncompressed_bytes, &compressed_bytes, compression_type);
|
||||
EXPECT_TRUE(st.ok());
|
||||
EXPECT_TRUE(uncompressed_bytes >= compressed_bytes);
|
||||
EXPECT_EQ(compressed_bytes, pblock->column_values().size());
|
||||
|
||||
const vectorized::ColumnWithTypeAndName& type_and_name =
|
||||
block.get_columns_with_type_and_name()[0];
|
||||
EXPECT_EQ(type_and_name.name, pblock->column_metas()[0].name());
|
||||
}
|
||||
|
||||
void generate_block(PBlock& pblock, int row_index) {
|
||||
auto vec = vectorized::ColumnVector<int32_t>::create();
|
||||
auto& data = vec->get_data();
|
||||
for (int i = 0; i < block_rows; ++i) {
|
||||
data.push_back(i + row_index);
|
||||
}
|
||||
vectorized::DataTypePtr data_type(std::make_shared<vectorized::DataTypeInt32>());
|
||||
vectorized::ColumnWithTypeAndName type_and_name(vec->get_ptr(), data_type, "test_int");
|
||||
vectorized::Block block({type_and_name});
|
||||
covert_block_to_pb(block, &pblock, segment_v2::CompressionTypePB::SNAPPY);
|
||||
}
|
||||
|
||||
TEST_F(WalReaderWriterTest, TestWriteAndRead1) {
|
||||
std::string file_name = _s_test_data_path + "/abcd123.txt";
|
||||
auto wal_writer = WalWriter(file_name);
|
||||
wal_writer.init();
|
||||
size_t file_len = 0;
|
||||
int64_t file_size = -1;
|
||||
// add 1 block
|
||||
{
|
||||
PBlock pblock;
|
||||
generate_block(pblock, 0);
|
||||
|
||||
EXPECT_EQ(Status::OK(), wal_writer.append_blocks(std::vector<PBlock*> {&pblock}));
|
||||
file_len += pblock.ByteSizeLong() + WalWriter::LENGTH_SIZE + WalWriter::CHECKSUM_SIZE;
|
||||
io::global_local_filesystem()->file_size(file_name, &file_size);
|
||||
EXPECT_EQ(file_len, file_size);
|
||||
}
|
||||
// add 2 block
|
||||
{
|
||||
PBlock pblock;
|
||||
generate_block(pblock, 1024);
|
||||
file_len += pblock.ByteSizeLong() + WalWriter::LENGTH_SIZE + WalWriter::CHECKSUM_SIZE;
|
||||
|
||||
PBlock pblock1;
|
||||
generate_block(pblock1, 2048);
|
||||
file_len += pblock1.ByteSizeLong() + WalWriter::LENGTH_SIZE + WalWriter::CHECKSUM_SIZE;
|
||||
|
||||
EXPECT_EQ(Status::OK(), wal_writer.append_blocks(std::vector<PBlock*> {&pblock, &pblock1}));
|
||||
io::global_local_filesystem()->file_size(file_name, &file_size);
|
||||
EXPECT_EQ(file_len, file_size);
|
||||
}
|
||||
wal_writer.finalize();
|
||||
// read block
|
||||
auto wal_reader = WalReader(file_name);
|
||||
wal_reader.init();
|
||||
auto block_count = 0;
|
||||
while (true) {
|
||||
doris::PBlock pblock;
|
||||
Status st = wal_reader.read_block(pblock);
|
||||
EXPECT_TRUE(st.ok() || st.is<ErrorCode::END_OF_FILE>());
|
||||
if (st.ok()) {
|
||||
++block_count;
|
||||
} else if (st.is<ErrorCode::END_OF_FILE>()) {
|
||||
break;
|
||||
}
|
||||
vectorized::Block block;
|
||||
block.deserialize(pblock);
|
||||
EXPECT_EQ(block_rows, block.rows());
|
||||
}
|
||||
wal_reader.finalize();
|
||||
EXPECT_EQ(3, block_count);
|
||||
}
|
||||
} // namespace doris
|
||||
@ -24,6 +24,7 @@
|
||||
#include <gtest/gtest-message.h>
|
||||
#include <gtest/gtest-test-part.h>
|
||||
|
||||
#include <filesystem>
|
||||
#include <string>
|
||||
#include <vector>
|
||||
|
||||
@ -32,7 +33,9 @@
|
||||
#include "gen_cpp/HeartbeatService_types.h"
|
||||
#include "gen_cpp/internal_service.pb.h"
|
||||
#include "gtest/gtest_pred_impl.h"
|
||||
#include "io/fs/local_file_system.h"
|
||||
#include "olap/olap_define.h"
|
||||
#include "olap/wal_manager.h"
|
||||
#include "runtime/decimalv2_value.h"
|
||||
#include "runtime/define_primitive_type.h"
|
||||
#include "runtime/descriptor_helper.h"
|
||||
@ -358,8 +361,12 @@ public:
|
||||
k_add_batch_status = Status::OK();
|
||||
_env = ExecEnv::GetInstance();
|
||||
_env->_master_info = new TMasterInfo();
|
||||
_env->_master_info->network_address.hostname = "host name";
|
||||
_env->_master_info->network_address.port = 1234;
|
||||
_env->_internal_client_cache = new BrpcClientCache<PBackendService_Stub>();
|
||||
_env->_function_client_cache = new BrpcClientCache<PFunctionService_Stub>();
|
||||
_env->_wal_manager = WalManager::create_shared(_env, wal_dir);
|
||||
_env->wal_mgr()->init();
|
||||
ThreadPoolBuilder("SendBatchThreadPool")
|
||||
.set_min_threads(1)
|
||||
.set_max_threads(5)
|
||||
@ -370,6 +377,7 @@ public:
|
||||
}
|
||||
|
||||
void TearDown() override {
|
||||
io::global_local_filesystem()->delete_directory(wal_dir);
|
||||
SAFE_DELETE(_env->_internal_client_cache);
|
||||
SAFE_DELETE(_env->_function_client_cache);
|
||||
SAFE_DELETE(_env->_master_info);
|
||||
@ -488,6 +496,7 @@ public:
|
||||
private:
|
||||
ExecEnv* _env = nullptr;
|
||||
brpc::Server* _server = nullptr;
|
||||
std::string wal_dir = "./wal_test";
|
||||
};
|
||||
|
||||
TEST_F(VOlapTableSinkTest, normal) {
|
||||
@ -835,5 +844,259 @@ TEST_F(VOlapTableSinkTest, decimal) {
|
||||
ASSERT_TRUE(output_set.count("(12, 12.300000000)") > 0);
|
||||
ASSERT_TRUE(output_set.count("(13, 123.120000000)") > 0);
|
||||
}
|
||||
|
||||
TEST_F(VOlapTableSinkTest, group_commit) {
|
||||
// start brpc service first
|
||||
_server = new brpc::Server();
|
||||
auto service = new VTestInternalService();
|
||||
ASSERT_EQ(_server->AddService(service, brpc::SERVER_OWNS_SERVICE), 0);
|
||||
brpc::ServerOptions options;
|
||||
{
|
||||
debug::ScopedLeakCheckDisabler disable_lsan;
|
||||
_server->Start(4356, &options);
|
||||
}
|
||||
|
||||
TUniqueId fragment_id;
|
||||
TQueryOptions query_options;
|
||||
query_options.batch_size = 1;
|
||||
query_options.be_exec_version = 0;
|
||||
RuntimeState state(fragment_id, query_options, TQueryGlobals(), _env);
|
||||
state.init_mem_trackers(TUniqueId());
|
||||
|
||||
ObjectPool obj_pool;
|
||||
TDescriptorTable tdesc_tbl;
|
||||
auto t_data_sink = get_data_sink(&tdesc_tbl);
|
||||
|
||||
// crate desc_tabl
|
||||
DescriptorTbl* desc_tbl = nullptr;
|
||||
auto st = DescriptorTbl::create(&obj_pool, tdesc_tbl, &desc_tbl);
|
||||
ASSERT_TRUE(st.ok());
|
||||
state._desc_tbl = desc_tbl;
|
||||
state._wal_id = 789;
|
||||
state._import_label = "test";
|
||||
|
||||
TupleDescriptor* tuple_desc = desc_tbl->get_tuple_descriptor(0);
|
||||
LOG(INFO) << "tuple_desc=" << tuple_desc->debug_string();
|
||||
|
||||
RowDescriptor row_desc(*desc_tbl, {0}, {false});
|
||||
service->_row_desc = &row_desc;
|
||||
std::set<std::string> output_set;
|
||||
service->_output_set = &output_set;
|
||||
|
||||
std::vector<TExpr> exprs;
|
||||
VOlapTableSink sink(&obj_pool, row_desc, exprs, true);
|
||||
|
||||
// init
|
||||
st = sink.init(t_data_sink);
|
||||
ASSERT_TRUE(st.ok());
|
||||
// prepare
|
||||
st = sink.prepare(&state);
|
||||
ASSERT_TRUE(st.ok());
|
||||
// open
|
||||
st = sink.open(&state);
|
||||
ASSERT_TRUE(st.ok());
|
||||
|
||||
int slot_count = tuple_desc->slots().size();
|
||||
std::vector<vectorized::MutableColumnPtr> columns(slot_count);
|
||||
for (int i = 0; i < slot_count; i++) {
|
||||
columns[i] = tuple_desc->slots()[i]->get_empty_mutable_column();
|
||||
}
|
||||
|
||||
int col_idx = 0;
|
||||
auto* column_ptr = columns[col_idx++].get();
|
||||
auto column_vector_int = column_ptr;
|
||||
int int_val = 12;
|
||||
column_vector_int->insert_data((const char*)&int_val, 0);
|
||||
int_val = 13;
|
||||
column_vector_int->insert_data((const char*)&int_val, 0);
|
||||
int_val = 14;
|
||||
column_vector_int->insert_data((const char*)&int_val, 0);
|
||||
|
||||
column_ptr = columns[col_idx++].get();
|
||||
auto column_vector_bigint = column_ptr;
|
||||
int64_t int64_val = 9;
|
||||
column_vector_bigint->insert_data((const char*)&int64_val, 0);
|
||||
int64_val = 25;
|
||||
column_vector_bigint->insert_data((const char*)&int64_val, 0);
|
||||
int64_val = 50;
|
||||
column_vector_bigint->insert_data((const char*)&int64_val, 0);
|
||||
|
||||
column_ptr = columns[col_idx++].get();
|
||||
auto column_vector_str = column_ptr;
|
||||
column_vector_str->insert_data("abc", 3);
|
||||
column_vector_str->insert_data("abcd", 4);
|
||||
column_vector_str->insert_data("1234567890", 10);
|
||||
|
||||
vectorized::Block block;
|
||||
col_idx = 0;
|
||||
for (const auto slot_desc : tuple_desc->slots()) {
|
||||
block.insert(vectorized::ColumnWithTypeAndName(std::move(columns[col_idx++]),
|
||||
slot_desc->get_data_type_ptr(),
|
||||
slot_desc->col_name()));
|
||||
}
|
||||
vectorized::Block org_block(block);
|
||||
|
||||
// send
|
||||
st = sink.send(&state, &block);
|
||||
ASSERT_TRUE(st.ok());
|
||||
// close
|
||||
st = sink.close(&state, Status::OK());
|
||||
ASSERT_TRUE(st.ok() || st.to_string() == "Internal error: wait close failed. ")
|
||||
<< st.to_string();
|
||||
|
||||
// each node has a eof
|
||||
ASSERT_EQ(2, service->_eof_counters);
|
||||
ASSERT_EQ(2 * 3, service->_row_counters);
|
||||
|
||||
// 2node * 2
|
||||
ASSERT_EQ(0, state.num_rows_load_filtered());
|
||||
|
||||
std::string wal_path = wal_dir + "/" + std::to_string(t_data_sink.olap_table_sink.db_id) + "/" +
|
||||
std::to_string(t_data_sink.olap_table_sink.table_id) + "/" +
|
||||
std::to_string(t_data_sink.olap_table_sink.txn_id) + "_" +
|
||||
state.import_label();
|
||||
doris::PBlock pblock;
|
||||
auto wal_reader = WalReader(wal_path);
|
||||
st = wal_reader.init();
|
||||
ASSERT_TRUE(st.ok());
|
||||
st = wal_reader.read_block(pblock);
|
||||
ASSERT_TRUE(st.ok());
|
||||
vectorized::Block wal_block;
|
||||
wal_block.deserialize(pblock);
|
||||
ASSERT_TRUE(st.ok() || st.is<ErrorCode::END_OF_FILE>());
|
||||
ASSERT_EQ(org_block.rows(), wal_block.rows());
|
||||
for (int i = 0; i < org_block.rows(); i++) {
|
||||
std::string srcRow = org_block.dump_one_line(i, org_block.columns());
|
||||
std::string walRow = wal_block.dump_one_line(i, org_block.columns());
|
||||
ASSERT_TRUE(std::strcmp(srcRow.c_str(), walRow.c_str()) == 0);
|
||||
}
|
||||
}
|
||||
|
||||
TEST_F(VOlapTableSinkTest, group_commit_with_filter_row) {
|
||||
// start brpc service first
|
||||
_server = new brpc::Server();
|
||||
auto service = new VTestInternalService();
|
||||
ASSERT_EQ(_server->AddService(service, brpc::SERVER_OWNS_SERVICE), 0);
|
||||
brpc::ServerOptions options;
|
||||
{
|
||||
debug::ScopedLeakCheckDisabler disable_lsan;
|
||||
_server->Start(4356, &options);
|
||||
}
|
||||
|
||||
TUniqueId fragment_id;
|
||||
TQueryOptions query_options;
|
||||
query_options.batch_size = 1;
|
||||
query_options.be_exec_version = 0;
|
||||
RuntimeState state(fragment_id, query_options, TQueryGlobals(), _env);
|
||||
state.init_mem_trackers(TUniqueId());
|
||||
|
||||
ObjectPool obj_pool;
|
||||
TDescriptorTable tdesc_tbl;
|
||||
auto t_data_sink = get_data_sink(&tdesc_tbl);
|
||||
|
||||
// crate desc_tabl
|
||||
DescriptorTbl* desc_tbl = nullptr;
|
||||
auto st = DescriptorTbl::create(&obj_pool, tdesc_tbl, &desc_tbl);
|
||||
ASSERT_TRUE(st.ok());
|
||||
state._desc_tbl = desc_tbl;
|
||||
state._wal_id = 789;
|
||||
state._import_label = "test";
|
||||
|
||||
TupleDescriptor* tuple_desc = desc_tbl->get_tuple_descriptor(0);
|
||||
LOG(INFO) << "tuple_desc=" << tuple_desc->debug_string();
|
||||
|
||||
RowDescriptor row_desc(*desc_tbl, {0}, {false});
|
||||
service->_row_desc = &row_desc;
|
||||
std::set<std::string> output_set;
|
||||
service->_output_set = &output_set;
|
||||
|
||||
std::vector<TExpr> exprs;
|
||||
VOlapTableSink sink(&obj_pool, row_desc, exprs, true);
|
||||
|
||||
// init
|
||||
st = sink.init(t_data_sink);
|
||||
ASSERT_TRUE(st.ok());
|
||||
// prepare
|
||||
st = sink.prepare(&state);
|
||||
ASSERT_TRUE(st.ok());
|
||||
// open
|
||||
st = sink.open(&state);
|
||||
ASSERT_TRUE(st.ok());
|
||||
|
||||
int slot_count = tuple_desc->slots().size();
|
||||
std::vector<vectorized::MutableColumnPtr> columns(slot_count);
|
||||
for (int i = 0; i < slot_count; i++) {
|
||||
columns[i] = tuple_desc->slots()[i]->get_empty_mutable_column();
|
||||
}
|
||||
|
||||
int col_idx = 0;
|
||||
auto* column_ptr = columns[col_idx++].get();
|
||||
auto column_vector_int = column_ptr;
|
||||
int int_val = 12;
|
||||
column_vector_int->insert_data((const char*)&int_val, 0);
|
||||
int_val = 13;
|
||||
column_vector_int->insert_data((const char*)&int_val, 0);
|
||||
int_val = 14;
|
||||
column_vector_int->insert_data((const char*)&int_val, 0);
|
||||
|
||||
column_ptr = columns[col_idx++].get();
|
||||
auto column_vector_bigint = column_ptr;
|
||||
int64_t int64_val = 9;
|
||||
column_vector_bigint->insert_data((const char*)&int64_val, 0);
|
||||
int64_val = 25;
|
||||
column_vector_bigint->insert_data((const char*)&int64_val, 0);
|
||||
int64_val = 50;
|
||||
column_vector_bigint->insert_data((const char*)&int64_val, 0);
|
||||
|
||||
column_ptr = columns[col_idx++].get();
|
||||
auto column_vector_str = column_ptr;
|
||||
column_vector_str->insert_data("abc", 3);
|
||||
column_vector_str->insert_data("abcd", 4);
|
||||
column_vector_str->insert_data("abcde1234567890", 15);
|
||||
|
||||
vectorized::Block block;
|
||||
col_idx = 0;
|
||||
for (const auto slot_desc : tuple_desc->slots()) {
|
||||
block.insert(vectorized::ColumnWithTypeAndName(std::move(columns[col_idx++]),
|
||||
slot_desc->get_data_type_ptr(),
|
||||
slot_desc->col_name()));
|
||||
}
|
||||
vectorized::Block org_block(block);
|
||||
|
||||
// send
|
||||
st = sink.send(&state, &block);
|
||||
ASSERT_TRUE(st.ok());
|
||||
// close
|
||||
st = sink.close(&state, Status::OK());
|
||||
ASSERT_TRUE(st.ok() || st.to_string() == "Internal error: wait close failed. ")
|
||||
<< st.to_string();
|
||||
|
||||
// each node has a eof
|
||||
ASSERT_EQ(2, service->_eof_counters);
|
||||
ASSERT_EQ(2 * 2, service->_row_counters);
|
||||
|
||||
// 2node * 2
|
||||
ASSERT_EQ(1, state.num_rows_load_filtered());
|
||||
|
||||
std::string wal_path = wal_dir + "/" + std::to_string(t_data_sink.olap_table_sink.db_id) + "/" +
|
||||
std::to_string(t_data_sink.olap_table_sink.table_id) + "/" +
|
||||
std::to_string(t_data_sink.olap_table_sink.txn_id) + "_" +
|
||||
state.import_label();
|
||||
doris::PBlock pblock;
|
||||
auto wal_reader = WalReader(wal_path);
|
||||
st = wal_reader.init();
|
||||
ASSERT_TRUE(st.ok());
|
||||
st = wal_reader.read_block(pblock);
|
||||
ASSERT_TRUE(st.ok());
|
||||
vectorized::Block wal_block;
|
||||
wal_block.deserialize(pblock);
|
||||
ASSERT_TRUE(st.ok() || st.is<ErrorCode::END_OF_FILE>());
|
||||
ASSERT_EQ(org_block.rows() - 1, wal_block.rows());
|
||||
for (int i = 0; i < wal_block.rows(); i++) {
|
||||
std::string srcRow = org_block.dump_one_line(i, org_block.columns());
|
||||
std::string walRow = wal_block.dump_one_line(i, org_block.columns());
|
||||
ASSERT_TRUE(std::strcmp(srcRow.c_str(), walRow.c_str()) == 0);
|
||||
}
|
||||
}
|
||||
} // namespace vectorized
|
||||
} // namespace doris
|
||||
|
||||
@ -354,6 +354,9 @@ public class DataDescription implements InsertStmt.DataDesc {
|
||||
case FORMAT_JSON:
|
||||
this.fileFormat = "json";
|
||||
break;
|
||||
case FORMAT_WAL:
|
||||
this.fileFormat = "wal";
|
||||
break;
|
||||
default:
|
||||
this.fileFormat = "unknown";
|
||||
break;
|
||||
@ -1114,6 +1117,7 @@ public class DataDescription implements InsertStmt.DataDesc {
|
||||
&& !fileFormat.equalsIgnoreCase(FeConstants.csv)
|
||||
&& !fileFormat.equalsIgnoreCase("orc")
|
||||
&& !fileFormat.equalsIgnoreCase("json")
|
||||
&& !fileFormat.equalsIgnoreCase("wal")
|
||||
&& !fileFormat.equalsIgnoreCase(FeConstants.csv_with_names)
|
||||
&& !fileFormat.equalsIgnoreCase(FeConstants.csv_with_names_and_types)
|
||||
&& !fileFormat.equalsIgnoreCase("hive_text")) {
|
||||
|
||||
@ -563,6 +563,8 @@ public class Util {
|
||||
// TODO: Add TEXTFILE to TFileFormatType to Support hive text file format.
|
||||
|| lowerFileFormat.equals(FeConstants.text)) {
|
||||
return TFileFormatType.FORMAT_CSV_PLAIN;
|
||||
} else if (lowerFileFormat.equals("wal")) {
|
||||
return TFileFormatType.FORMAT_WAL;
|
||||
} else {
|
||||
return TFileFormatType.FORMAT_UNKNOWN;
|
||||
}
|
||||
|
||||
@ -274,7 +274,8 @@ public abstract class FileQueryScanNode extends FileScanNode {
|
||||
TFileFormatType fileFormatType = getFileFormatType();
|
||||
params.setFormatType(fileFormatType);
|
||||
boolean isCsvOrJson = Util.isCsvFormat(fileFormatType) || fileFormatType == TFileFormatType.FORMAT_JSON;
|
||||
if (isCsvOrJson) {
|
||||
boolean isWal = fileFormatType == TFileFormatType.FORMAT_WAL;
|
||||
if (isCsvOrJson || isWal) {
|
||||
params.setFileAttributes(getFileAttributes());
|
||||
if (getLocationType() == TFileType.FILE_STREAM) {
|
||||
params.setFileType(TFileType.FILE_STREAM);
|
||||
|
||||
@ -1174,8 +1174,9 @@ public class FrontendServiceImpl implements FrontendService.Iface {
|
||||
if (Strings.isNullOrEmpty(cluster)) {
|
||||
cluster = SystemInfoService.DEFAULT_CLUSTER;
|
||||
}
|
||||
|
||||
if (Strings.isNullOrEmpty(request.getToken())) {
|
||||
if (request.isSetAuthCode()) {
|
||||
// TODO(cmy): find a way to check
|
||||
} else if (Strings.isNullOrEmpty(request.getToken())) {
|
||||
checkPasswordAndPrivs(cluster, request.getUser(), request.getPasswd(), request.getDb(), request.getTbl(),
|
||||
request.getUserIp(), PrivPredicate.LOAD);
|
||||
}
|
||||
@ -2040,7 +2041,9 @@ public class FrontendServiceImpl implements FrontendService.Iface {
|
||||
cluster = SystemInfoService.DEFAULT_CLUSTER;
|
||||
}
|
||||
ConnectContext ctx = new ConnectContext();
|
||||
if (Strings.isNullOrEmpty(request.getToken())) {
|
||||
if (request.isSetAuthCode()) {
|
||||
// TODO(cmy): find a way to check
|
||||
} else if (Strings.isNullOrEmpty(request.getToken())) {
|
||||
checkPasswordAndPrivs(cluster, request.getUser(), request.getPasswd(), request.getDb(), request.getTbl(),
|
||||
request.getUserIp(), PrivPredicate.LOAD);
|
||||
}
|
||||
|
||||
@ -25,10 +25,12 @@ import org.apache.doris.catalog.Column;
|
||||
import org.apache.doris.catalog.Env;
|
||||
import org.apache.doris.catalog.HdfsResource;
|
||||
import org.apache.doris.catalog.MapType;
|
||||
import org.apache.doris.catalog.OlapTable;
|
||||
import org.apache.doris.catalog.PrimitiveType;
|
||||
import org.apache.doris.catalog.ScalarType;
|
||||
import org.apache.doris.catalog.StructField;
|
||||
import org.apache.doris.catalog.StructType;
|
||||
import org.apache.doris.catalog.Table;
|
||||
import org.apache.doris.catalog.Type;
|
||||
import org.apache.doris.common.AnalysisException;
|
||||
import org.apache.doris.common.FeConstants;
|
||||
@ -94,6 +96,7 @@ public abstract class ExternalFileTableValuedFunction extends TableValuedFunctio
|
||||
protected static String DEFAULT_COLUMN_SEPARATOR = ",";
|
||||
protected static final String DEFAULT_LINE_DELIMITER = "\n";
|
||||
public static final String FORMAT = "format";
|
||||
public static final String TABLE_ID = "table_id";
|
||||
public static final String COLUMN_SEPARATOR = "column_separator";
|
||||
public static final String LINE_DELIMITER = "line_delimiter";
|
||||
protected static final String JSON_ROOT = "json_root";
|
||||
@ -114,6 +117,7 @@ public abstract class ExternalFileTableValuedFunction extends TableValuedFunctio
|
||||
|
||||
protected static final ImmutableSet<String> FILE_FORMAT_PROPERTIES = new ImmutableSet.Builder<String>()
|
||||
.add(FORMAT)
|
||||
.add(TABLE_ID)
|
||||
.add(JSON_ROOT)
|
||||
.add(JSON_PATHS)
|
||||
.add(STRIP_OUTER_ARRAY)
|
||||
@ -157,6 +161,7 @@ public abstract class ExternalFileTableValuedFunction extends TableValuedFunctio
|
||||
private boolean fuzzyParse;
|
||||
private boolean trimDoubleQuotes;
|
||||
private int skipLines;
|
||||
private long tableId;
|
||||
|
||||
public abstract TFileType getTFileType();
|
||||
|
||||
@ -236,10 +241,14 @@ public abstract class ExternalFileTableValuedFunction extends TableValuedFunctio
|
||||
case "avro":
|
||||
this.fileFormatType = TFileFormatType.FORMAT_AVRO;
|
||||
break;
|
||||
case "wal":
|
||||
this.fileFormatType = TFileFormatType.FORMAT_WAL;
|
||||
break;
|
||||
default:
|
||||
throw new AnalysisException("format:" + formatString + " is not supported.");
|
||||
}
|
||||
|
||||
tableId = Long.valueOf(validParams.getOrDefault(TABLE_ID, "-1")).longValue();
|
||||
columnSeparator = validParams.getOrDefault(COLUMN_SEPARATOR, DEFAULT_COLUMN_SEPARATOR);
|
||||
if (Strings.isNullOrEmpty(columnSeparator)) {
|
||||
throw new AnalysisException("column_separator can not be empty.");
|
||||
@ -401,6 +410,20 @@ public abstract class ExternalFileTableValuedFunction extends TableValuedFunctio
|
||||
throw new AnalysisException("No Alive backends");
|
||||
}
|
||||
|
||||
if (this.fileFormatType == TFileFormatType.FORMAT_WAL) {
|
||||
List<Column> fileColumns = new ArrayList<>();
|
||||
Table table = Env.getCurrentInternalCatalog().getTableByTableId(tableId);
|
||||
List<Column> tableColumns = table.getBaseSchema(false);
|
||||
for (int i = 1; i <= tableColumns.size(); i++) {
|
||||
fileColumns.add(new Column("c" + i, tableColumns.get(i - 1).getDataType(), true));
|
||||
}
|
||||
Column deleteSignColumn = ((OlapTable) table).getDeleteSignColumn();
|
||||
if (deleteSignColumn != null) {
|
||||
fileColumns.add(new Column("c" + (tableColumns.size() + 1), deleteSignColumn.getDataType(), true));
|
||||
}
|
||||
return fileColumns;
|
||||
}
|
||||
|
||||
TNetworkAddress address = new TNetworkAddress(be.getHost(), be.getBrpcPort());
|
||||
try {
|
||||
PFetchTableSchemaRequest request = getFetchTableStructureRequest();
|
||||
|
||||
@ -666,6 +666,16 @@ struct TStreamLoadWithLoadStatusResult {
|
||||
6: optional i64 unselected_rows
|
||||
}
|
||||
|
||||
struct TCheckWalRequest {
|
||||
1: optional i64 wal_id
|
||||
2: optional i64 db_id
|
||||
}
|
||||
|
||||
struct TCheckWalResult {
|
||||
1: optional Status.TStatus status
|
||||
2: optional bool need_recovery
|
||||
}
|
||||
|
||||
struct TKafkaRLTaskProgress {
|
||||
1: required map<i32,i64> partitionCmtOffset
|
||||
}
|
||||
|
||||
@ -445,6 +445,8 @@ struct TExecPlanFragmentParams {
|
||||
|
||||
// scan node id -> scan range params, only for external file scan
|
||||
24: optional map<Types.TPlanNodeId, PlanNodes.TFileScanRangeParams> file_scan_params
|
||||
|
||||
25: optional i64 wal_id
|
||||
}
|
||||
|
||||
struct TExecPlanFragmentParamsList {
|
||||
|
||||
@ -119,6 +119,7 @@ enum TFileFormatType {
|
||||
FORMAT_AVRO,
|
||||
FORMAT_CSV_LZ4BLOCK,
|
||||
FORMAT_CSV_SNAPPYBLOCK,
|
||||
FORMAT_WAL,
|
||||
}
|
||||
|
||||
// In previous versions, the data compression format and file format were stored together, as TFileFormatType,
|
||||
|
||||
Reference in New Issue
Block a user